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