1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.replication.regionserver;
20
21 import java.io.IOException;
22 import java.util.NavigableMap;
23 import java.util.TreeMap;
24 import java.util.concurrent.Executors;
25 import java.util.concurrent.ScheduledExecutorService;
26 import java.util.concurrent.TimeUnit;
27 import java.util.concurrent.atomic.AtomicBoolean;
28
29 import com.google.common.util.concurrent.ThreadFactoryBuilder;
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.classification.InterfaceAudience;
33 import org.apache.hadoop.conf.Configuration;
34 import org.apache.hadoop.fs.FileSystem;
35 import org.apache.hadoop.fs.Path;
36 import org.apache.hadoop.hbase.HRegionInfo;
37 import org.apache.hadoop.hbase.HTableDescriptor;
38 import org.apache.hadoop.hbase.KeyValue;
39 import org.apache.hadoop.hbase.Server;
40 import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
41 import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
42 import org.apache.hadoop.hbase.regionserver.wal.HLog;
43 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
44 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
45 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
46 import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
47 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
48 import org.apache.hadoop.hbase.util.Bytes;
49 import org.apache.zookeeper.KeeperException;
50
51 import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
52 import static org.apache.hadoop.hbase.HConstants.REPLICATION_ENABLE_KEY;
53 import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
54
55
56
57
58 @InterfaceAudience.Private
59 public class Replication implements WALActionsListener,
60 ReplicationSourceService, ReplicationSinkService {
61 private static final Log LOG =
62 LogFactory.getLog(Replication.class);
63 private boolean replication;
64 private ReplicationSourceManager replicationManager;
65 private final AtomicBoolean replicating = new AtomicBoolean(true);
66 private ReplicationZookeeper zkHelper;
67 private Configuration conf;
68 private ReplicationSink replicationSink;
69
70 private Server server;
71
72 private ScheduledExecutorService scheduleThreadPool;
73 private int statsThreadPeriod;
74
75
76
77
78
79
80
81
82
83 public Replication(final Server server, final FileSystem fs,
84 final Path logDir, final Path oldLogDir) throws IOException{
85 initialize(server, fs, logDir, oldLogDir);
86 }
87
88
89
90
91 public Replication() {
92 }
93
94 public void initialize(final Server server, final FileSystem fs,
95 final Path logDir, final Path oldLogDir) throws IOException {
96 this.server = server;
97 this.conf = this.server.getConfiguration();
98 this.replication = isReplication(this.conf);
99 this.scheduleThreadPool = Executors.newScheduledThreadPool(1,
100 new ThreadFactoryBuilder()
101 .setNameFormat(server.getServerName() + "Replication Statistics #%d")
102 .setDaemon(true)
103 .build());
104 if (replication) {
105 try {
106 this.zkHelper = new ReplicationZookeeper(server, this.replicating);
107 } catch (KeeperException ke) {
108 throw new IOException("Failed replication handler create " +
109 "(replicating=" + this.replicating, ke);
110 }
111 this.replicationManager = new ReplicationSourceManager(zkHelper, conf, this.server, fs,
112 this.replicating, logDir, oldLogDir);
113 this.statsThreadPeriod =
114 this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
115 LOG.debug("ReplicationStatisticsThread " + this.statsThreadPeriod);
116 } else {
117 this.replicationManager = null;
118 this.zkHelper = null;
119 }
120 }
121
122
123
124
125
126 public static boolean isReplication(final Configuration c) {
127 return c.getBoolean(REPLICATION_ENABLE_KEY, false);
128 }
129
130
131
132
133 public WALActionsListener getWALActionsListener() {
134 return this;
135 }
136
137
138
139 public void stopReplicationService() {
140 join();
141 }
142
143
144
145
146 public void join() {
147 if (this.replication) {
148 this.replicationManager.join();
149 this.replicationSink.stopReplicationSinkServices();
150 }
151 }
152
153
154
155
156
157
158 public void replicateLogEntries(HLog.Entry[] entries) throws IOException {
159 if (this.replication) {
160 this.replicationSink.replicateEntries(entries);
161 }
162 }
163
164
165
166
167
168
169 public void startReplicationService() throws IOException {
170 if (this.replication) {
171 this.replicationManager.init();
172 this.replicationSink = new ReplicationSink(this.conf, this.server);
173 this.scheduleThreadPool.scheduleAtFixedRate(
174 new ReplicationStatisticsThread(this.replicationSink, this.replicationManager),
175 statsThreadPeriod, statsThreadPeriod, TimeUnit.SECONDS);
176 }
177 }
178
179
180
181
182
183 public ReplicationSourceManager getReplicationManager() {
184 return this.replicationManager;
185 }
186
187 @Override
188 public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey,
189 WALEdit logEdit) {
190
191 }
192
193 @Override
194 public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey,
195 WALEdit logEdit) {
196 NavigableMap<byte[], Integer> scopes =
197 new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
198 byte[] family;
199 for (KeyValue kv : logEdit.getKeyValues()) {
200 family = kv.getFamily();
201 int scope = htd.getFamily(family).getScope();
202 if (scope != REPLICATION_SCOPE_LOCAL &&
203 !scopes.containsKey(family)) {
204 scopes.put(family, scope);
205 }
206 }
207 if (!scopes.isEmpty()) {
208 logEdit.setScopes(scopes);
209 }
210 }
211
212 @Override
213 public void preLogRoll(Path oldPath, Path newPath) throws IOException {
214 getReplicationManager().preLogRoll(newPath);
215 }
216
217 @Override
218 public void postLogRoll(Path oldPath, Path newPath) throws IOException {
219 getReplicationManager().postLogRoll(newPath);
220 }
221
222 @Override
223 public void preLogArchive(Path oldPath, Path newPath) throws IOException {
224
225 }
226
227 @Override
228 public void postLogArchive(Path oldPath, Path newPath) throws IOException {
229
230 }
231
232
233
234
235
236
237 public static void decorateMasterConfiguration(Configuration conf) {
238 if (!isReplication(conf)) {
239 return;
240 }
241 String plugins = conf.get(HBASE_MASTER_LOGCLEANER_PLUGINS);
242 String cleanerClass = ReplicationLogCleaner.class.getCanonicalName();
243 if (!plugins.contains(cleanerClass)) {
244 conf.set(HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
245 }
246 }
247
248 @Override
249 public void logRollRequested() {
250
251 }
252
253 @Override
254 public void logCloseRequested() {
255
256 }
257
258
259
260
261 static class ReplicationStatisticsThread extends Thread {
262
263 private final ReplicationSink replicationSink;
264 private final ReplicationSourceManager replicationManager;
265
266 public ReplicationStatisticsThread(final ReplicationSink replicationSink,
267 final ReplicationSourceManager replicationManager) {
268 super("ReplicationStatisticsThread");
269 this.replicationManager = replicationManager;
270 this.replicationSink = replicationSink;
271 }
272
273 @Override
274 public void run() {
275 printStats(this.replicationManager.getStats());
276 printStats(this.replicationSink.getStats());
277 }
278
279 private void printStats(String stats) {
280 if (!stats.isEmpty()) {
281 LOG.info(stats);
282 }
283 }
284 }
285 }