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