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.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
56
57
58
59
60
61
62
63
64
65
66
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
79
80
81
82
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
94
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
105
106
107
108
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
114
115 try {
116 long totalReplicated = 0;
117
118
119 Map<TableName, Map<List<UUID>, List<Row>>> rowMap =
120 new TreeMap<TableName, Map<List<UUID>, List<Row>>>();
121 for (WALEntry entry : entries) {
122 TableName table =
123 TableName.valueOf(entry.getKey().getTableName().toByteArray());
124 Cell previousCell = null;
125 Mutation m = null;
126 int count = entry.getAssociatedCellCount();
127 for (int i = 0; i < count; i++) {
128
129 if (!cells.advance()) {
130 throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
131 }
132 Cell cell = cells.current();
133 if (isNewRowOrType(previousCell, cell)) {
134
135 m = CellUtil.isDelete(cell)?
136 new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()):
137 new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
138 List<UUID> clusterIds = new ArrayList<UUID>();
139 for(HBaseProtos.UUID clusterId : entry.getKey().getClusterIdsList()){
140 clusterIds.add(toUUID(clusterId));
141 }
142 m.setClusterIds(clusterIds);
143 addToHashMultiMap(rowMap, table, clusterIds, m);
144 }
145 if (CellUtil.isDelete(cell)) {
146 ((Delete)m).addDeleteMarker(KeyValueUtil.ensureKeyValue(cell));
147 } else {
148 ((Put)m).add(KeyValueUtil.ensureKeyValue(cell));
149 }
150 previousCell = cell;
151 }
152 totalReplicated++;
153 }
154 for (Entry<TableName, Map<List<UUID>,List<Row>>> entry : rowMap.entrySet()) {
155 batch(entry.getKey(), entry.getValue().values());
156 }
157 int size = entries.size();
158 this.metrics.setAgeOfLastAppliedOp(entries.get(size - 1).getKey().getWriteTime());
159 this.metrics.applyBatch(size);
160 this.totalReplicatedEdits.addAndGet(totalReplicated);
161 } catch (IOException ex) {
162 LOG.error("Unable to accept edit because:", ex);
163 throw ex;
164 }
165 }
166
167
168
169
170
171
172 private boolean isNewRowOrType(final Cell previousCell, final Cell cell) {
173 return previousCell == null || previousCell.getTypeByte() != cell.getTypeByte() ||
174 !CellUtil.matchingRow(previousCell, cell);
175 }
176
177 private java.util.UUID toUUID(final HBaseProtos.UUID uuid) {
178 return new java.util.UUID(uuid.getMostSigBits(), uuid.getLeastSigBits());
179 }
180
181
182
183
184
185
186
187
188
189
190 private <K1, K2, V> List<V> addToHashMultiMap(Map<K1, Map<K2,List<V>>> map, K1 key1, K2 key2, V value) {
191 Map<K2,List<V>> innerMap = map.get(key1);
192 if (innerMap == null) {
193 innerMap = new HashMap<K2, List<V>>();
194 map.put(key1, innerMap);
195 }
196 List<V> values = innerMap.get(key2);
197 if (values == null) {
198 values = new ArrayList<V>();
199 innerMap.put(key2, values);
200 }
201 values.add(value);
202 return values;
203 }
204
205
206
207
208 public void stopReplicationSinkServices() {
209 try {
210 this.sharedHtableCon.close();
211 } catch (IOException e) {
212 LOG.warn("IOException while closing the connection", e);
213 }
214 }
215
216
217
218
219
220
221
222
223 protected void batch(TableName tableName, Collection<List<Row>> allRows) throws IOException {
224 if (allRows.isEmpty()) {
225 return;
226 }
227 HTableInterface table = null;
228 try {
229 table = this.sharedHtableCon.getTable(tableName);
230 for (List<Row> rows : allRows) {
231 table.batch(rows);
232 }
233 } catch (InterruptedException ix) {
234 throw new IOException(ix);
235 } finally {
236 if (table != null) {
237 table.close();
238 }
239 }
240 }
241
242
243
244
245
246
247 public String getStats() {
248 return this.totalReplicatedEdits.get() == 0 ? "" : "Sink: " +
249 "age in ms of last applied edit: " + this.metrics.refreshAgeOfLastAppliedOp() +
250 ", total replicated edits: " + this.totalReplicatedEdits;
251 }
252 }