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 java.io.IOException;
22  import java.util.ArrayList;
23  import java.util.Collection;
24  import java.util.HashMap;
25  import java.util.List;
26  import java.util.Map;
27  import java.util.Map.Entry;
28  import java.util.TreeMap;
29  import java.util.UUID;
30  import java.util.concurrent.atomic.AtomicLong;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.classification.InterfaceAudience;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.hbase.Cell;
37  import org.apache.hadoop.hbase.CellScanner;
38  import org.apache.hadoop.hbase.CellUtil;
39  import org.apache.hadoop.hbase.TableName;
40  import org.apache.hadoop.hbase.HBaseConfiguration;
41  import org.apache.hadoop.hbase.HConstants;
42  import org.apache.hadoop.hbase.KeyValueUtil;
43  import org.apache.hadoop.hbase.Stoppable;
44  import org.apache.hadoop.hbase.client.Delete;
45  import org.apache.hadoop.hbase.client.HConnection;
46  import org.apache.hadoop.hbase.client.HConnectionManager;
47  import org.apache.hadoop.hbase.client.HTableInterface;
48  import org.apache.hadoop.hbase.client.Mutation;
49  import org.apache.hadoop.hbase.client.Put;
50  import org.apache.hadoop.hbase.client.Row;
51  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
52  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
53  
54  /**
55   * This class is responsible for replicating the edits coming
56   * from another cluster.
57   * <p/>
58   * This replication process is currently waiting for the edits to be applied
59   * before the method can return. This means that the replication of edits
60   * is synchronized (after reading from HLogs in ReplicationSource) and that a
61   * single region server cannot receive edits from two sources at the same time
62   * <p/>
63   * This class uses the native HBase client in order to replicate entries.
64   * <p/>
65   *
66   * TODO make this class more like ReplicationSource wrt log handling
67   */
68  @InterfaceAudience.Private
69  public class ReplicationSink {
70  
71    private static final Log LOG = LogFactory.getLog(ReplicationSink.class);
72    private final Configuration conf;
73    private final HConnection sharedHtableCon;
74    private final MetricsSink metrics;
75    private final AtomicLong totalReplicatedEdits = new AtomicLong();
76  
77    /**
78     * Create a sink for replication
79     *
80     * @param conf                conf object
81     * @param stopper             boolean to tell this thread to stop
82     * @throws IOException thrown when HDFS goes bad or bad file name
83     */
84    public ReplicationSink(Configuration conf, Stoppable stopper)
85        throws IOException {
86      this.conf = HBaseConfiguration.create(conf);
87      decorateConf();
88      this.metrics = new MetricsSink();
89      this.sharedHtableCon = HConnectionManager.createConnection(this.conf);
90    }
91  
92    /**
93     * decorate the Configuration object to make replication more receptive to delays:
94     * lessen the timeout and numTries.
95     */
96    private void decorateConf() {
97      this.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
98          this.conf.getInt("replication.sink.client.retries.number", 4));
99      this.conf.setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
100         this.conf.getInt("replication.sink.client.ops.timeout", 10000));
101    }
102 
103   /**
104    * Replicate this array of entries directly into the local cluster using the native client. Only
105    * operates against raw protobuf type saving on a conversion from pb to pojo.
106    * @param entries
107    * @param cells
108    * @throws IOException
109    */
110   public void replicateEntries(List<WALEntry> entries, final CellScanner cells) throws IOException {
111     if (entries.isEmpty()) return;
112     if (cells == null) throw new NullPointerException("TODO: Add handling of null CellScanner");
113     // Very simple optimization where we batch sequences of rows going
114     // to the same table.
115     try {
116       long totalReplicated = 0;
117       // Map of table => list of Rows, grouped by cluster id, we only want to flushCommits once per
118       // invocation of this method per table and cluster id.
119       Map<TableName, Map<UUID,List<Row>>> rowMap = new TreeMap<TableName, Map<UUID,List<Row>>>();
120       for (WALEntry entry : entries) {
121         TableName table =
122             TableName.valueOf(entry.getKey().getTableName().toByteArray());
123         Cell previousCell = null;
124         Mutation m = null;
125         java.util.UUID uuid = toUUID(entry.getKey().getClusterId());
126         int count = entry.getAssociatedCellCount();
127         for (int i = 0; i < count; i++) {
128           // Throw index out of bounds if our cell count is off
129           if (!cells.advance()) {
130             throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
131           }
132           Cell cell = cells.current();
133           if (isNewRowOrType(previousCell, cell)) {
134             // Create new mutation
135             m = CellUtil.isDelete(cell)?
136               new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()):
137               new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
138             m.setClusterId(uuid);
139             addToHashMultiMap(rowMap, table, uuid, m);
140           }
141           if (CellUtil.isDelete(cell)) {
142             ((Delete)m).addDeleteMarker(KeyValueUtil.ensureKeyValue(cell));
143           } else {
144             ((Put)m).add(KeyValueUtil.ensureKeyValue(cell));
145           }
146           previousCell = cell;
147         }
148         totalReplicated++;
149       }
150       for (Entry<TableName, Map<UUID,List<Row>>> entry : rowMap.entrySet()) {
151         batch(entry.getKey(), entry.getValue().values());
152       }
153       int size = entries.size();
154       this.metrics.setAgeOfLastAppliedOp(entries.get(size - 1).getKey().getWriteTime());
155       this.metrics.applyBatch(size);
156       this.totalReplicatedEdits.addAndGet(totalReplicated);
157     } catch (IOException ex) {
158       LOG.error("Unable to accept edit because:", ex);
159       throw ex;
160     }
161   }
162 
163   /**
164    * @param previousCell
165    * @param cell
166    * @return True if we have crossed over onto a new row or type
167    */
168   private boolean isNewRowOrType(final Cell previousCell, final Cell cell) {
169     return previousCell == null || previousCell.getTypeByte() != cell.getTypeByte() ||
170         !CellUtil.matchingRow(previousCell, cell);
171   }
172 
173   private java.util.UUID toUUID(final HBaseProtos.UUID uuid) {
174     return new java.util.UUID(uuid.getMostSigBits(), uuid.getLeastSigBits());
175   }
176 
177   /**
178    * Simple helper to a map from key to (a list of) values
179    * TODO: Make a general utility method
180    * @param map
181    * @param key1
182    * @param key2
183    * @param value
184    * @return
185    */
186   private <K1, K2, V> List<V> addToHashMultiMap(Map<K1, Map<K2,List<V>>> map, K1 key1, K2 key2, V value) {
187     Map<K2,List<V>> innerMap = map.get(key1);
188     if (innerMap == null) {
189       innerMap = new HashMap<K2, List<V>>();
190       map.put(key1, innerMap);
191     }
192     List<V> values = innerMap.get(key2);
193     if (values == null) {
194       values = new ArrayList<V>();
195       innerMap.put(key2, values);
196     }
197     values.add(value);
198     return values;
199   }
200 
201   /**
202    * stop the thread pool executor. It is called when the regionserver is stopped.
203    */
204   public void stopReplicationSinkServices() {
205     try {
206       this.sharedHtableCon.close();
207     } catch (IOException e) {
208       LOG.warn("IOException while closing the connection", e); // ignoring as we are closing.
209     }
210   }
211 
212 
213   /**
214    * Do the changes and handle the pool
215    * @param tableName table to insert into
216    * @param rows list of actions
217    * @throws IOException
218    */
219   private void batch(TableName tableName, Collection<List<Row>> allRows) throws IOException {
220     if (allRows.isEmpty()) {
221       return;
222     }
223     HTableInterface table = null;
224     try {
225       table = this.sharedHtableCon.getTable(tableName);
226       for (List<Row> rows : allRows) {
227         table.batch(rows);
228       }
229     } catch (InterruptedException ix) {
230       throw new IOException(ix);
231     } finally {
232       if (table != null) {
233         table.close();
234       }
235     }
236   }
237 
238   /**
239    * Get a string representation of this sink's metrics
240    * @return string with the total replicated edits count and the date
241    * of the last edit that was applied
242    */
243   public String getStats() {
244     return this.totalReplicatedEdits.get() == 0 ? "" : "Sink: " +
245       "age in ms of last applied edit: " + this.metrics.refreshAgeOfLastAppliedOp() +
246       ", total replicated edits: " + this.totalReplicatedEdits;
247   }
248 }