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 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.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.KeyValueUtil;
45 import org.apache.hadoop.hbase.Stoppable;
46 import org.apache.hadoop.hbase.client.Delete;
47 import org.apache.hadoop.hbase.client.HConnection;
48 import org.apache.hadoop.hbase.client.HConnectionManager;
49 import org.apache.hadoop.hbase.client.HTableInterface;
50 import org.apache.hadoop.hbase.client.Mutation;
51 import org.apache.hadoop.hbase.client.Put;
52 import org.apache.hadoop.hbase.client.Row;
53 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
54 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70 @InterfaceAudience.Private
71 public class ReplicationSink {
72
73 private static final Log LOG = LogFactory.getLog(ReplicationSink.class);
74 private final Configuration conf;
75 private final HConnection sharedHtableCon;
76 private final MetricsSink metrics;
77 private final AtomicLong totalReplicatedEdits = new AtomicLong();
78
79
80
81
82
83
84
85
86 public ReplicationSink(Configuration conf, Stoppable stopper)
87 throws IOException {
88 this.conf = HBaseConfiguration.create(conf);
89 decorateConf();
90 this.metrics = new MetricsSink();
91 this.sharedHtableCon = HConnectionManager.createConnection(this.conf);
92 }
93
94
95
96
97
98 private void decorateConf() {
99 this.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
100 this.conf.getInt("replication.sink.client.retries.number", 4));
101 this.conf.setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
102 this.conf.getInt("replication.sink.client.ops.timeout", 10000));
103 String replicationCodec = this.conf.get(HConstants.REPLICATION_CODEC_CONF_KEY);
104 if (StringUtils.isNotEmpty(replicationCodec)) {
105 this.conf.set(HConstants.RPC_CODEC_CONF_KEY, replicationCodec);
106 }
107 }
108
109
110
111
112
113
114
115
116 public void replicateEntries(List<WALEntry> entries, final CellScanner cells) throws IOException {
117 if (entries.isEmpty()) return;
118 if (cells == null) throw new NullPointerException("TODO: Add handling of null CellScanner");
119
120
121 try {
122 long totalReplicated = 0;
123
124
125 Map<TableName, Map<List<UUID>, List<Row>>> rowMap =
126 new TreeMap<TableName, Map<List<UUID>, List<Row>>>();
127 for (WALEntry entry : entries) {
128 TableName table =
129 TableName.valueOf(entry.getKey().getTableName().toByteArray());
130 Cell previousCell = null;
131 Mutation m = null;
132 int count = entry.getAssociatedCellCount();
133 for (int i = 0; i < count; i++) {
134
135 if (!cells.advance()) {
136 throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
137 }
138 Cell cell = cells.current();
139 if (isNewRowOrType(previousCell, cell)) {
140
141 m = CellUtil.isDelete(cell)?
142 new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()):
143 new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
144 List<UUID> clusterIds = new ArrayList<UUID>();
145 for(HBaseProtos.UUID clusterId : entry.getKey().getClusterIdsList()){
146 clusterIds.add(toUUID(clusterId));
147 }
148 m.setClusterIds(clusterIds);
149 addToHashMultiMap(rowMap, table, clusterIds, m);
150 }
151 if (CellUtil.isDelete(cell)) {
152 ((Delete)m).addDeleteMarker(KeyValueUtil.ensureKeyValue(cell));
153 } else {
154 ((Put)m).add(KeyValueUtil.ensureKeyValue(cell));
155 }
156 previousCell = cell;
157 }
158 totalReplicated++;
159 }
160 for (Entry<TableName, Map<List<UUID>,List<Row>>> entry : rowMap.entrySet()) {
161 batch(entry.getKey(), entry.getValue().values());
162 }
163 int size = entries.size();
164 this.metrics.setAgeOfLastAppliedOp(entries.get(size - 1).getKey().getWriteTime());
165 this.metrics.applyBatch(size);
166 this.totalReplicatedEdits.addAndGet(totalReplicated);
167 } catch (IOException ex) {
168 LOG.error("Unable to accept edit because:", ex);
169 throw ex;
170 }
171 }
172
173
174
175
176
177
178 private boolean isNewRowOrType(final Cell previousCell, final Cell cell) {
179 return previousCell == null || previousCell.getTypeByte() != cell.getTypeByte() ||
180 !CellUtil.matchingRow(previousCell, cell);
181 }
182
183 private java.util.UUID toUUID(final HBaseProtos.UUID uuid) {
184 return new java.util.UUID(uuid.getMostSigBits(), uuid.getLeastSigBits());
185 }
186
187
188
189
190
191
192
193
194
195
196 private <K1, K2, V> List<V> addToHashMultiMap(Map<K1, Map<K2,List<V>>> map, K1 key1, K2 key2, V value) {
197 Map<K2,List<V>> innerMap = map.get(key1);
198 if (innerMap == null) {
199 innerMap = new HashMap<K2, List<V>>();
200 map.put(key1, innerMap);
201 }
202 List<V> values = innerMap.get(key2);
203 if (values == null) {
204 values = new ArrayList<V>();
205 innerMap.put(key2, values);
206 }
207 values.add(value);
208 return values;
209 }
210
211
212
213
214 public void stopReplicationSinkServices() {
215 try {
216 this.sharedHtableCon.close();
217 } catch (IOException e) {
218 LOG.warn("IOException while closing the connection", e);
219 }
220 }
221
222
223
224
225
226
227
228
229 protected void batch(TableName tableName, Collection<List<Row>> allRows) throws IOException {
230 if (allRows.isEmpty()) {
231 return;
232 }
233 HTableInterface table = null;
234 try {
235 table = this.sharedHtableCon.getTable(tableName);
236 for (List<Row> rows : allRows) {
237 table.batch(rows);
238 }
239 } catch (InterruptedException ix) {
240 throw (InterruptedIOException)new InterruptedIOException().initCause(ix);
241 } finally {
242 if (table != null) {
243 table.close();
244 }
245 }
246 }
247
248
249
250
251
252
253 public String getStats() {
254 return this.totalReplicatedEdits.get() == 0 ? "" : "Sink: " +
255 "age in ms of last applied edit: " + this.metrics.refreshAgeOfLastAppliedOp() +
256 ", total replicated edits: " + this.totalReplicatedEdits;
257 }
258 }