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 static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
22 import static org.apache.hadoop.hbase.HConstants.REPLICATION_ENABLE_KEY;
23 import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
24
25 import java.io.IOException;
26 import java.util.ArrayList;
27 import java.util.List;
28 import java.util.NavigableMap;
29 import java.util.TreeMap;
30 import java.util.UUID;
31 import java.util.concurrent.Executors;
32 import java.util.concurrent.ScheduledExecutorService;
33 import java.util.concurrent.TimeUnit;
34
35 import org.apache.commons.logging.Log;
36 import org.apache.commons.logging.LogFactory;
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.fs.FileSystem;
39 import org.apache.hadoop.fs.Path;
40 import org.apache.hadoop.hbase.Cell;
41 import org.apache.hadoop.hbase.CellScanner;
42 import org.apache.hadoop.hbase.CellUtil;
43 import org.apache.hadoop.hbase.HConstants;
44 import org.apache.hadoop.hbase.HTableDescriptor;
45 import org.apache.hadoop.hbase.Server;
46 import org.apache.hadoop.hbase.TableName;
47 import org.apache.hadoop.hbase.classification.InterfaceAudience;
48 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
49 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
50 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
51 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
52 import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
53 import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
54 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
55 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
56 import org.apache.hadoop.hbase.replication.ReplicationException;
57 import org.apache.hadoop.hbase.replication.ReplicationFactory;
58 import org.apache.hadoop.hbase.replication.ReplicationPeers;
59 import org.apache.hadoop.hbase.replication.ReplicationQueues;
60 import org.apache.hadoop.hbase.replication.ReplicationTracker;
61 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
62 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
63 import org.apache.hadoop.hbase.util.Bytes;
64 import org.apache.hadoop.hbase.wal.WALKey;
65 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
66 import org.apache.zookeeper.KeeperException;
67
68 import com.google.common.util.concurrent.ThreadFactoryBuilder;
69
70
71
72
73 @InterfaceAudience.Private
74 public class Replication extends WALActionsListener.Base implements
75 ReplicationSourceService, ReplicationSinkService {
76 private static final Log LOG =
77 LogFactory.getLog(Replication.class);
78 private boolean replication;
79 private boolean replicationForBulkLoadData;
80 private ReplicationSourceManager replicationManager;
81 private ReplicationQueues replicationQueues;
82 private ReplicationPeers replicationPeers;
83 private ReplicationTracker replicationTracker;
84 private Configuration conf;
85 private ReplicationSink replicationSink;
86
87 private Server server;
88
89 private ScheduledExecutorService scheduleThreadPool;
90 private int statsThreadPeriod;
91
92 private ReplicationLoad replicationLoad;
93
94
95
96
97
98
99
100
101 public Replication(final Server server, final FileSystem fs,
102 final Path logDir, final Path oldLogDir) throws IOException{
103 initialize(server, fs, logDir, oldLogDir);
104 }
105
106
107
108
109 public Replication() {
110 }
111
112 public void initialize(final Server server, final FileSystem fs,
113 final Path logDir, final Path oldLogDir) throws IOException {
114 this.server = server;
115 this.conf = this.server.getConfiguration();
116 this.replication = isReplication(this.conf);
117 this.replicationForBulkLoadData = isReplicationForBulkLoadDataEnabled(this.conf);
118 this.scheduleThreadPool = Executors.newScheduledThreadPool(1,
119 new ThreadFactoryBuilder()
120 .setNameFormat(server.getServerName().toShortString() + "Replication Statistics #%d")
121 .setDaemon(true)
122 .build());
123 if (this.replicationForBulkLoadData) {
124 if (conf.get(HConstants.REPLICATION_CLUSTER_ID) == null
125 || conf.get(HConstants.REPLICATION_CLUSTER_ID).isEmpty()) {
126 throw new IllegalArgumentException(HConstants.REPLICATION_CLUSTER_ID
127 + " cannot be null/empty when " + HConstants.REPLICATION_BULKLOAD_ENABLE_KEY
128 + " is set to true.");
129 }
130 }
131 if (replication) {
132 try {
133 this.replicationQueues =
134 ReplicationFactory.getReplicationQueues(server.getZooKeeper(), this.conf, this.server);
135 this.replicationQueues.init(this.server.getServerName().toString());
136 this.replicationPeers =
137 ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf, this.server);
138 this.replicationPeers.init();
139 this.replicationTracker =
140 ReplicationFactory.getReplicationTracker(server.getZooKeeper(), this.replicationPeers,
141 this.conf, this.server, this.server);
142 } catch (ReplicationException e) {
143 throw new IOException("Failed replication handler create", e);
144 }
145 UUID clusterId = null;
146 try {
147 clusterId = ZKClusterId.getUUIDForCluster(this.server.getZooKeeper());
148 } catch (KeeperException ke) {
149 throw new IOException("Could not read cluster id", ke);
150 }
151 this.replicationManager =
152 new ReplicationSourceManager(replicationQueues, replicationPeers, replicationTracker,
153 conf, this.server, fs, logDir, oldLogDir, clusterId);
154 this.statsThreadPeriod =
155 this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
156 LOG.debug("ReplicationStatisticsThread " + this.statsThreadPeriod);
157 this.replicationLoad = new ReplicationLoad();
158 } else {
159 this.replicationManager = null;
160 this.replicationQueues = null;
161 this.replicationPeers = null;
162 this.replicationTracker = null;
163 this.replicationLoad = null;
164 }
165 }
166
167
168
169
170
171 public static boolean isReplication(final Configuration c) {
172 return c.getBoolean(REPLICATION_ENABLE_KEY, HConstants.REPLICATION_ENABLE_DEFAULT);
173 }
174
175
176
177
178
179 public static boolean isReplicationForBulkLoadDataEnabled(final Configuration c) {
180 return c.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
181 HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
182 }
183
184
185
186
187 public WALActionsListener getWALActionsListener() {
188 return this;
189 }
190
191
192
193 public void stopReplicationService() {
194 join();
195 }
196
197
198
199
200 public void join() {
201 if (this.replication) {
202 this.replicationManager.join();
203 if (this.replicationSink != null) {
204 this.replicationSink.stopReplicationSinkServices();
205 }
206 }
207 scheduleThreadPool.shutdown();
208 }
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223 public void replicateLogEntries(List<WALEntry> entries, CellScanner cells,
224 String replicationClusterId, String sourceBaseNamespaceDirPath,
225 String sourceHFileArchiveDirPath) throws IOException {
226 if (this.replication) {
227 this.replicationSink.replicateEntries(entries, cells, replicationClusterId,
228 sourceBaseNamespaceDirPath, sourceHFileArchiveDirPath);
229 }
230 }
231
232
233
234
235
236
237 public void startReplicationService() throws IOException {
238 if (this.replication) {
239 try {
240 this.replicationManager.init();
241 } catch (ReplicationException e) {
242 throw new IOException(e);
243 }
244 this.replicationSink = new ReplicationSink(this.conf, this.server);
245 this.scheduleThreadPool.scheduleAtFixedRate(
246 new ReplicationStatisticsThread(this.replicationSink, this.replicationManager),
247 statsThreadPeriod, statsThreadPeriod, TimeUnit.SECONDS);
248 }
249 }
250
251
252
253
254
255 public ReplicationSourceManager getReplicationManager() {
256 return this.replicationManager;
257 }
258
259 @Override
260 public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit)
261 throws IOException {
262 scopeWALEdits(htd, logKey, logEdit, this.conf, this.getReplicationManager());
263 }
264
265 @Override
266 public void postAppend(long entryLen, long elapsedTimeMillis, final WALKey logKey,
267 final WALEdit edit) throws IOException {
268 NavigableMap<byte[], Integer> scopes = logKey.getScopes();
269 if (this.replicationForBulkLoadData && scopes != null && !scopes.isEmpty()) {
270 TableName tableName = logKey.getTablename();
271 for (Cell c : edit.getCells()) {
272
273 if (CellUtil.matchingQualifier(c, WALEdit.BULK_LOAD)) {
274 BulkLoadDescriptor bld = null;
275 try {
276 bld = WALEdit.getBulkLoadDescriptor(c);
277 } catch (IOException e) {
278 LOG.error("Failed to get bulk load events information from the wal file.", e);
279 throw e;
280 }
281
282 for (StoreDescriptor s : bld.getStoresList()) {
283 byte[] fam = s.getFamilyName().toByteArray();
284
285
286 if (scopes.containsKey(fam)) {
287 addHFileRefsToQueue(this.getReplicationManager(), tableName, fam, s);
288 }
289 }
290 }
291 }
292 }
293 }
294
295
296
297
298
299
300
301
302
303
304 public static void scopeWALEdits(HTableDescriptor htd, WALKey logKey, WALEdit logEdit,
305 Configuration conf, ReplicationSourceManager replicationManager) throws IOException {
306 NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
307 byte[] family;
308 boolean replicationForBulkLoadEnabled = isReplicationForBulkLoadDataEnabled(conf);
309 for (Cell cell : logEdit.getCells()) {
310 if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
311 if (replicationForBulkLoadEnabled && CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {
312 scopeBulkLoadEdits(htd, replicationManager, scopes, logKey.getTablename(), cell);
313 } else {
314
315 continue;
316 }
317 } else {
318 family = CellUtil.cloneFamily(cell);
319
320 assert htd.getFamily(family) != null;
321
322 if (!scopes.containsKey(family)) {
323 int scope = htd.getFamily(family).getScope();
324 if (scope != REPLICATION_SCOPE_LOCAL) {
325 scopes.put(family, scope);
326 }
327 }
328 }
329 }
330 if (!scopes.isEmpty()) {
331 logKey.setScopes(scopes);
332 }
333 }
334
335 private static void scopeBulkLoadEdits(HTableDescriptor htd,
336 ReplicationSourceManager replicationManager, NavigableMap<byte[], Integer> scopes,
337 TableName tableName, Cell cell) throws IOException {
338 byte[] family;
339 try {
340 BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);
341 for (StoreDescriptor s : bld.getStoresList()) {
342 family = s.getFamilyName().toByteArray();
343 if (!scopes.containsKey(family)) {
344 int scope = htd.getFamily(family).getScope();
345 if (scope != REPLICATION_SCOPE_LOCAL) {
346 scopes.put(family, scope);
347 }
348 }
349 }
350 } catch (IOException e) {
351 LOG.error("Failed to get bulk load events information from the wal file.", e);
352 throw e;
353 }
354 }
355
356 private static void addHFileRefsToQueue(ReplicationSourceManager replicationManager,
357 TableName tableName, byte[] family, StoreDescriptor s) throws IOException {
358 try {
359 replicationManager.addHFileRefs(tableName, family, s.getStoreFileList());
360 } catch (ReplicationException e) {
361 LOG.error("Failed to add hfile references in the replication queue.", e);
362 throw new IOException(e);
363 }
364 }
365
366 @Override
367 public void preLogRoll(Path oldPath, Path newPath) throws IOException {
368 getReplicationManager().preLogRoll(newPath);
369 }
370
371 @Override
372 public void postLogRoll(Path oldPath, Path newPath) throws IOException {
373 getReplicationManager().postLogRoll(newPath);
374 }
375
376
377
378
379
380 public static void decorateMasterConfiguration(Configuration conf) {
381 if (!isReplication(conf)) {
382 return;
383 }
384 String plugins = conf.get(HBASE_MASTER_LOGCLEANER_PLUGINS);
385 String cleanerClass = ReplicationLogCleaner.class.getCanonicalName();
386 if (!plugins.contains(cleanerClass)) {
387 conf.set(HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
388 }
389 if (isReplicationForBulkLoadDataEnabled(conf)) {
390 plugins = conf.get(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS);
391 cleanerClass = ReplicationHFileCleaner.class.getCanonicalName();
392 if (!plugins.contains(cleanerClass)) {
393 conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, plugins + "," + cleanerClass);
394 }
395 }
396 }
397
398
399
400
401 static class ReplicationStatisticsThread extends Thread {
402
403 private final ReplicationSink replicationSink;
404 private final ReplicationSourceManager replicationManager;
405
406 public ReplicationStatisticsThread(final ReplicationSink replicationSink,
407 final ReplicationSourceManager replicationManager) {
408 super("ReplicationStatisticsThread");
409 this.replicationManager = replicationManager;
410 this.replicationSink = replicationSink;
411 }
412
413 @Override
414 public void run() {
415 printStats(this.replicationManager.getStats());
416 printStats(this.replicationSink.getStats());
417 }
418
419 private void printStats(String stats) {
420 if (!stats.isEmpty()) {
421 LOG.info(stats);
422 }
423 }
424 }
425
426 @Override
427 public ReplicationLoad refreshAndGetReplicationLoad() {
428 if (this.replicationLoad == null) {
429 return null;
430 }
431
432 buildReplicationLoad();
433 return this.replicationLoad;
434 }
435
436 private void buildReplicationLoad() {
437
438 List<ReplicationSourceInterface> sources = this.replicationManager.getSources();
439 List<MetricsSource> sourceMetricsList = new ArrayList<MetricsSource>();
440
441 for (ReplicationSourceInterface source : sources) {
442 if (source instanceof ReplicationSource) {
443 sourceMetricsList.add(((ReplicationSource) source).getSourceMetrics());
444 }
445 }
446
447 MetricsSink sinkMetrics = this.replicationSink.getSinkMetrics();
448 this.replicationLoad.buildReplicationLoad(sourceMetricsList, sinkMetrics);
449 }
450 }