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 com.google.common.collect.Lists;
22  
23  import java.io.IOException;
24  import java.io.InterruptedIOException;
25  import java.util.ArrayList;
26  import java.util.Collection;
27  import java.util.Collections;
28  import java.util.HashMap;
29  import java.util.List;
30  import java.util.Map;
31  import java.util.Map.Entry;
32  import java.util.TreeMap;
33  import java.util.UUID;
34  import java.util.concurrent.atomic.AtomicLong;
35  
36  import org.apache.commons.lang.StringUtils;
37  import org.apache.commons.logging.Log;
38  import org.apache.commons.logging.LogFactory;
39  import org.apache.hadoop.conf.Configuration;
40  import org.apache.hadoop.fs.Path;
41  import org.apache.hadoop.hbase.Cell;
42  import org.apache.hadoop.hbase.CellScanner;
43  import org.apache.hadoop.hbase.CellUtil;
44  import org.apache.hadoop.hbase.HBaseConfiguration;
45  import org.apache.hadoop.hbase.HConstants;
46  import org.apache.hadoop.hbase.TableName;
47  import org.apache.hadoop.hbase.TableNotFoundException;
48  import org.apache.hadoop.hbase.classification.InterfaceAudience;
49  import org.apache.hadoop.hbase.client.Connection;
50  import org.apache.hadoop.hbase.client.ConnectionFactory;
51  import org.apache.hadoop.hbase.client.Delete;
52  import org.apache.hadoop.hbase.client.Mutation;
53  import org.apache.hadoop.hbase.client.Put;
54  import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
55  import org.apache.hadoop.hbase.client.Row;
56  import org.apache.hadoop.hbase.client.Table;
57  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
58  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
59  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
60  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
61  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
62  import org.apache.hadoop.hbase.util.Bytes;
63  import org.apache.hadoop.hbase.util.Pair;
64  
65  /**
66   * <p>
67   * This class is responsible for replicating the edits coming
68   * from another cluster.
69   * </p><p>
70   * This replication process is currently waiting for the edits to be applied
71   * before the method can return. This means that the replication of edits
72   * is synchronized (after reading from WALs in ReplicationSource) and that a
73   * single region server cannot receive edits from two sources at the same time
74   * </p><p>
75   * This class uses the native HBase client in order to replicate entries.
76   * </p>
77   *
78   * TODO make this class more like ReplicationSource wrt log handling
79   */
80  @InterfaceAudience.Private
81  public class ReplicationSink {
82  
83    private static final Log LOG = LogFactory.getLog(ReplicationSink.class);
84    private final Configuration conf;
85    // Volatile because of note in here -- look for double-checked locking:
86    // http://www.oracle.com/technetwork/articles/javase/bloch-effective-08-qa-140880.html
87    private volatile Connection sharedHtableCon;
88    private final MetricsSink metrics;
89    private final AtomicLong totalReplicatedEdits = new AtomicLong();
90    private final Object sharedHtableConLock = new Object();
91    // Number of hfiles that we successfully replicated
92    private long hfilesReplicated = 0;
93    private SourceFSConfigurationProvider provider;
94  
95    /**
96     * Row size threshold for multi requests above which a warning is logged
97     */
98    private final int rowSizeWarnThreshold;
99  
100   /**
101    * Create a sink for replication
102    * @param conf conf object
103    * @throws IOException thrown when HDFS goes bad or bad file name
104    */
105   public ReplicationSink(Configuration conf)
106       throws IOException {
107     this.conf = HBaseConfiguration.create(conf);
108     rowSizeWarnThreshold = conf.getInt(
109       HConstants.BATCH_ROWS_THRESHOLD_NAME, HConstants.BATCH_ROWS_THRESHOLD_DEFAULT);
110     decorateConf();
111     this.metrics = new MetricsSink();
112 
113     String className =
114         conf.get("hbase.replication.source.fs.conf.provider",
115           DefaultSourceFSConfigurationProvider.class.getCanonicalName());
116     try {
117       @SuppressWarnings("rawtypes")
118       Class c = Class.forName(className);
119       this.provider = (SourceFSConfigurationProvider) c.newInstance();
120     } catch (Exception e) {
121       throw new IllegalArgumentException("Configured source fs configuration provider class "
122           + className + " throws error.", e);
123     }
124   }
125 
126   /**
127    * decorate the Configuration object to make replication more receptive to delays:
128    * lessen the timeout and numTries.
129    */
130   private void decorateConf() {
131     this.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
132         this.conf.getInt("replication.sink.client.retries.number", 4));
133     this.conf.setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
134         this.conf.getInt("replication.sink.client.ops.timeout", 10000));
135     String replicationCodec = this.conf.get(HConstants.REPLICATION_CODEC_CONF_KEY);
136     if (StringUtils.isNotEmpty(replicationCodec)) {
137       this.conf.set(HConstants.RPC_CODEC_CONF_KEY, replicationCodec);
138     }
139    }
140 
141   /**
142    * Replicate this array of entries directly into the local cluster using the native client. Only
143    * operates against raw protobuf type saving on a conversion from pb to pojo.
144    * @param entries
145    * @param cells
146    * @param replicationClusterId Id which will uniquely identify source cluster FS client
147    *          configurations in the replication configuration directory
148    * @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace
149    *          directory
150    * @param sourceHFileArchiveDirPath Path that point to the source cluster hfile archive directory
151    * @throws IOException If failed to replicate the data
152    */
153   public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
154       String replicationClusterId, String sourceBaseNamespaceDirPath,
155       String sourceHFileArchiveDirPath) throws IOException {
156     if (entries.isEmpty()) return;
157     if (cells == null) throw new NullPointerException("TODO: Add handling of null CellScanner");
158     // Very simple optimization where we batch sequences of rows going
159     // to the same table.
160     try {
161       long totalReplicated = 0;
162       // Map of table => list of Rows, grouped by cluster id, we only want to flushCommits once per
163       // invocation of this method per table and cluster id.
164       Map<TableName, Map<List<UUID>, List<Row>>> rowMap =
165           new TreeMap<TableName, Map<List<UUID>, List<Row>>>();
166 
167       Map<List<String>, Map<String, List<Pair<byte[], List<String>>>>> bulkLoadsPerClusters = null;
168       for (WALEntry entry : entries) {
169         TableName table =
170             TableName.valueOf(entry.getKey().getTableName().toByteArray());
171         Cell previousCell = null;
172         Mutation m = null;
173         int count = entry.getAssociatedCellCount();
174         for (int i = 0; i < count; i++) {
175           // Throw index out of bounds if our cell count is off
176           if (!cells.advance()) {
177             throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
178           }
179           Cell cell = cells.current();
180           // Handle bulk load hfiles replication
181           if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {
182             BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);
183             if(bulkLoadsPerClusters == null) {
184               bulkLoadsPerClusters = new HashMap<>();
185             }
186             // Map of table name Vs list of pair of family and list of
187             // hfile paths from its namespace
188             Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap =
189               bulkLoadsPerClusters.get(bld.getClusterIdsList());
190             if (bulkLoadHFileMap == null) {
191               bulkLoadHFileMap = new HashMap<>();
192               bulkLoadsPerClusters.put(bld.getClusterIdsList(), bulkLoadHFileMap);
193             }
194             buildBulkLoadHFileMap(bulkLoadHFileMap, table, bld);
195           } else {
196             // Handle wal replication
197             if (isNewRowOrType(previousCell, cell)) {
198               // Create new mutation
199               m =
200                   CellUtil.isDelete(cell) ? new Delete(cell.getRowArray(), cell.getRowOffset(),
201                       cell.getRowLength()) : new Put(cell.getRowArray(), cell.getRowOffset(),
202                       cell.getRowLength());
203               List<UUID> clusterIds = new ArrayList<UUID>();
204               for (HBaseProtos.UUID clusterId : entry.getKey().getClusterIdsList()) {
205                 clusterIds.add(toUUID(clusterId));
206               }
207               m.setClusterIds(clusterIds);
208               addToHashMultiMap(rowMap, table, clusterIds, m);
209             }
210             if (CellUtil.isDelete(cell)) {
211               ((Delete) m).addDeleteMarker(cell);
212             } else {
213               ((Put) m).add(cell);
214             }
215             previousCell = cell;
216           }
217         }
218         totalReplicated++;
219       }
220 
221       // TODO Replicating mutations and bulk loaded data can be made parallel
222       if (!rowMap.isEmpty()) {
223         LOG.debug("Started replicating mutations.");
224         for (Entry<TableName, Map<List<UUID>, List<Row>>> entry : rowMap.entrySet()) {
225           batch(entry.getKey(), entry.getValue().values(), rowSizeWarnThreshold);
226         }
227         LOG.debug("Finished replicating mutations.");
228       }
229 
230       if(bulkLoadsPerClusters != null) {
231         for (Entry<List<String>, Map<String, List<Pair<byte[],
232           List<String>>>>> entry : bulkLoadsPerClusters.entrySet()) {
233           Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap = entry.getValue();
234           if (bulkLoadHFileMap != null && !bulkLoadHFileMap.isEmpty()) {
235             if(LOG.isDebugEnabled()) {
236               LOG.debug("Started replicating bulk loaded data from cluster ids: " +
237                 entry.getKey().toString());
238             }
239             HFileReplicator hFileReplicator =
240               new HFileReplicator(this.provider.getConf(this.conf, replicationClusterId),
241                 sourceBaseNamespaceDirPath, sourceHFileArchiveDirPath, bulkLoadHFileMap, conf,
242                 getConnection(), entry.getKey());
243             hFileReplicator.replicate();
244             if(LOG.isDebugEnabled()) {
245               LOG.debug("Finished replicating bulk loaded data from cluster id: " +
246                 entry.getKey().toString());
247             }
248           }
249         }
250       }
251 
252       int size = entries.size();
253       this.metrics.setAgeOfLastAppliedOp(entries.get(size - 1).getKey().getWriteTime());
254       this.metrics.applyBatch(size + hfilesReplicated, hfilesReplicated);
255       this.totalReplicatedEdits.addAndGet(totalReplicated);
256     } catch (IOException ex) {
257       LOG.error("Unable to accept edit because:", ex);
258       throw ex;
259     }
260   }
261 
262   private void buildBulkLoadHFileMap(
263       final Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap, TableName table,
264       BulkLoadDescriptor bld) throws IOException {
265     List<StoreDescriptor> storesList = bld.getStoresList();
266     int storesSize = storesList.size();
267     for (int j = 0; j < storesSize; j++) {
268       StoreDescriptor storeDescriptor = storesList.get(j);
269       List<String> storeFileList = storeDescriptor.getStoreFileList();
270       int storeFilesSize = storeFileList.size();
271       hfilesReplicated += storeFilesSize;
272       for (int k = 0; k < storeFilesSize; k++) {
273         byte[] family = storeDescriptor.getFamilyName().toByteArray();
274 
275         // Build hfile relative path from its namespace
276         String pathToHfileFromNS = getHFilePath(table, bld, storeFileList.get(k), family);
277         String tableName = table.getNameWithNamespaceInclAsString();
278         List<Pair<byte[], List<String>>> familyHFilePathsList = bulkLoadHFileMap.get(tableName);
279         if (familyHFilePathsList != null) {
280           boolean foundFamily = false;
281           for (Pair<byte[], List<String>> familyHFilePathsPair :  familyHFilePathsList) {
282             if (Bytes.equals(familyHFilePathsPair.getFirst(), family)) {
283               // Found family already present, just add the path to the existing list
284               familyHFilePathsPair.getSecond().add(pathToHfileFromNS);
285               foundFamily = true;
286               break;
287             }
288           }
289           if (!foundFamily) {
290             // Family not found, add this family and its hfile paths pair to the list
291             addFamilyAndItsHFilePathToTableInMap(family, pathToHfileFromNS, familyHFilePathsList);
292           }
293         } else {
294           // Add this table entry into the map
295           addNewTableEntryInMap(bulkLoadHFileMap, family, pathToHfileFromNS, tableName);
296         }
297       }
298     }
299   }
300 
301   private void addFamilyAndItsHFilePathToTableInMap(byte[] family, String pathToHfileFromNS,
302       List<Pair<byte[], List<String>>> familyHFilePathsList) {
303     List<String> hfilePaths = new ArrayList<String>();
304     hfilePaths.add(pathToHfileFromNS);
305     familyHFilePathsList.add(new Pair<byte[], List<String>>(family, hfilePaths));
306   }
307 
308   private void addNewTableEntryInMap(
309       final Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap, byte[] family,
310       String pathToHfileFromNS, String tableName) {
311     List<String> hfilePaths = new ArrayList<String>();
312     hfilePaths.add(pathToHfileFromNS);
313     Pair<byte[], List<String>> newFamilyHFilePathsPair =
314         new Pair<byte[], List<String>>(family, hfilePaths);
315     List<Pair<byte[], List<String>>> newFamilyHFilePathsList =
316         new ArrayList<Pair<byte[], List<String>>>();
317     newFamilyHFilePathsList.add(newFamilyHFilePathsPair);
318     bulkLoadHFileMap.put(tableName, newFamilyHFilePathsList);
319   }
320 
321   private String getHFilePath(TableName table, BulkLoadDescriptor bld, String storeFile,
322       byte[] family) {
323     return new StringBuilder(100).append(table.getNamespaceAsString()).append(Path.SEPARATOR)
324         .append(table.getQualifierAsString()).append(Path.SEPARATOR)
325         .append(Bytes.toString(bld.getEncodedRegionName().toByteArray())).append(Path.SEPARATOR)
326         .append(Bytes.toString(family)).append(Path.SEPARATOR).append(storeFile).toString();
327   }
328 
329   /**
330    * @param previousCell
331    * @param cell
332    * @return True if we have crossed over onto a new row or type
333    */
334   private boolean isNewRowOrType(final Cell previousCell, final Cell cell) {
335     return previousCell == null || previousCell.getTypeByte() != cell.getTypeByte() ||
336         !CellUtil.matchingRow(previousCell, cell);
337   }
338 
339   private java.util.UUID toUUID(final HBaseProtos.UUID uuid) {
340     return new java.util.UUID(uuid.getMostSigBits(), uuid.getLeastSigBits());
341   }
342 
343   /**
344    * Simple helper to a map from key to (a list of) values
345    * TODO: Make a general utility method
346    * @param map
347    * @param key1
348    * @param key2
349    * @param value
350    * @return the list of values corresponding to key1 and key2
351    */
352   private <K1, K2, V> List<V> addToHashMultiMap(Map<K1, Map<K2,List<V>>> map, K1 key1, K2 key2, V value) {
353     Map<K2,List<V>> innerMap = map.get(key1);
354     if (innerMap == null) {
355       innerMap = new HashMap<K2, List<V>>();
356       map.put(key1, innerMap);
357     }
358     List<V> values = innerMap.get(key2);
359     if (values == null) {
360       values = new ArrayList<V>();
361       innerMap.put(key2, values);
362     }
363     values.add(value);
364     return values;
365   }
366 
367   /**
368    * stop the thread pool executor. It is called when the regionserver is stopped.
369    */
370   public void stopReplicationSinkServices() {
371     try {
372       if (this.sharedHtableCon != null) {
373         synchronized (sharedHtableConLock) {
374           if (this.sharedHtableCon != null) {
375             this.sharedHtableCon.close();
376             this.sharedHtableCon = null;
377           }
378         }
379       }
380     } catch (IOException e) {
381       LOG.warn("IOException while closing the connection", e); // ignoring as we are closing.
382     }
383   }
384 
385 
386   /**
387    * Do the changes and handle the pool
388    * @param tableName table to insert into
389    * @param allRows list of actions
390    * @param batchRowSizeThreshold rowSize threshold for batch mutation
391    */
392   private void batch(TableName tableName, Collection<List<Row>> allRows, int batchRowSizeThreshold)
393       throws IOException {
394     if (allRows.isEmpty()) {
395       return;
396     }
397     Table table = null;
398     try {
399       Connection connection = getConnection();
400       table = connection.getTable(tableName);
401       for (List<Row> rows : allRows) {
402         List<List<Row>> batchRows;
403         if (rows.size() > batchRowSizeThreshold) {
404           batchRows = Lists.partition(rows, batchRowSizeThreshold);
405         } else {
406           batchRows = Collections.singletonList(rows);
407         }
408         for (List<Row> rowList : batchRows) {
409           table.batch(rowList);
410         }
411       }
412     } catch (RetriesExhaustedWithDetailsException rewde) {
413       for (Throwable ex : rewde.getCauses()) {
414         if (ex instanceof TableNotFoundException) {
415           throw new TableNotFoundException("'" + tableName + "'");
416         }
417       }
418       throw rewde;
419     } catch (InterruptedException ix) {
420       throw (InterruptedIOException) new InterruptedIOException().initCause(ix);
421     } finally {
422       if (table != null) {
423         table.close();
424       }
425     }
426   }
427 
428   private Connection getConnection() throws IOException {
429     // See https://en.wikipedia.org/wiki/Double-checked_locking
430     Connection connection = sharedHtableCon;
431     if (connection == null) {
432       synchronized (sharedHtableConLock) {
433         connection = sharedHtableCon;
434         if (connection == null) {
435           connection = sharedHtableCon = ConnectionFactory.createConnection(conf);
436         }
437       }
438     }
439     return connection;
440   }
441 
442   /**
443    * Get a string representation of this sink's metrics
444    * @return string with the total replicated edits count and the date
445    * of the last edit that was applied
446    */
447   public String getStats() {
448     return this.totalReplicatedEdits.get() == 0 ? "" : "Sink: " +
449       "age in ms of last applied edit: " + this.metrics.refreshAgeOfLastAppliedOp() +
450       ", total replicated edits: " + this.totalReplicatedEdits;
451   }
452 
453   /**
454    * Get replication Sink Metrics
455    * @return MetricsSink
456    */
457   public MetricsSink getSinkMetrics() {
458     return this.metrics;
459   }
460 }