View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
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   * Gateway to Replication.  Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
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    // Hosting server
87    private Server server;
88    /** Statistics thread schedule pool */
89    private ScheduledExecutorService scheduleThreadPool;
90    private int statsThreadPeriod;
91    // ReplicationLoad to access replication metrics
92    private ReplicationLoad replicationLoad;
93    /**
94     * Instantiate the replication management (if rep is enabled).
95     * @param server Hosting server
96     * @param fs handle to the filesystem
97     * @param logDir
98     * @param oldLogDir directory where logs are archived
99     * @throws IOException
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    * Empty constructor
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     * @param c Configuration to look at
169     * @return True if replication is enabled.
170     */
171   public static boolean isReplication(final Configuration c) {
172     return c.getBoolean(REPLICATION_ENABLE_KEY, HConstants.REPLICATION_ENABLE_DEFAULT);
173   }
174 
175   /**
176    * @param c Configuration to look at
177    * @return True if replication for bulk load data is enabled.
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     * Returns an object to listen to new wal changes
186     **/
187   public WALActionsListener getWALActionsListener() {
188     return this;
189   }
190   /**
191    * Stops replication service.
192    */
193   public void stopReplicationService() {
194     join();
195   }
196 
197   /**
198    * Join with the replication threads
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    * Carry on the list of log entries down to the sink
212    * @param entries list of entries to replicate
213    * @param cells The data -- the cells -- that <code>entries</code> describes (the entries do not
214    *          contain the Cells we are replicating; they are passed here on the side in this
215    *          CellScanner).
216    * @param replicationClusterId Id which will uniquely identify source cluster FS client
217    *          configurations in the replication configuration directory
218    * @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace
219    *          directory required for replicating hfiles
220    * @param sourceHFileArchiveDirPath Path that point to the source cluster hfile archive directory
221    * @throws IOException
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    * If replication is enabled and this cluster is a master,
234    * it starts
235    * @throws IOException
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    * Get the replication sources manager
253    * @return the manager if replication is enabled, else returns false
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         // Only check for bulk load events
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             // We have already scoped the entries as part
285             // WALActionsListener#visitLogEntryBeforeWrite notification
286             if (scopes.containsKey(fam)) {
287               addHFileRefsToQueue(this.getReplicationManager(), tableName, fam, s);
288             }
289           }
290         }
291       }
292     }
293   }
294 
295   /**
296    * Utility method used to set the correct scopes on each log key. Doesn't set a scope on keys from
297    * compaction WAL edits and if the scope is local.
298    * @param htd Descriptor used to find the scope to use
299    * @param logKey Key that may get scoped according to its edits
300    * @param logEdit Edits used to lookup the scopes
301    * @param replicationManager Manager used to add bulk load events hfile references
302    * @throws IOException If failed to parse the WALEdit
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           // Skip the flush/compaction/region events
315           continue;
316         }
317       } else {
318         family = CellUtil.cloneFamily(cell);
319         // Unexpected, has a tendency to happen in unit tests
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    * This method modifies the master's configuration in order to inject replication-related features
378    * @param conf
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    * Statistics thread. Periodically prints the cache statistics to the log.
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     // always build for latest data
432     buildReplicationLoad();
433     return this.replicationLoad;
434   }
435 
436   private void buildReplicationLoad() {
437     // get source
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     // get sink
447     MetricsSink sinkMetrics = this.replicationSink.getSinkMetrics();
448     this.replicationLoad.buildReplicationLoad(sourceMetricsList, sinkMetrics);
449   }
450 }