1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.replication.regionserver;
21
22 import org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.conf.Configuration;
25 import org.apache.hadoop.hbase.HBaseConfiguration;
26 import org.apache.hadoop.hbase.HConstants;
27 import org.apache.hadoop.hbase.KeyValue;
28 import org.apache.hadoop.hbase.client.Delete;
29 import org.apache.hadoop.hbase.client.HConnection;
30 import org.apache.hadoop.hbase.client.HConnectionManager;
31 import org.apache.hadoop.hbase.client.HTableInterface;
32 import org.apache.hadoop.hbase.client.Put;
33 import org.apache.hadoop.hbase.client.Row;
34 import org.apache.hadoop.hbase.regionserver.wal.HLog;
35 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
36 import org.apache.hadoop.hbase.util.Bytes;
37 import org.apache.hadoop.hbase.Stoppable;
38
39 import java.io.IOException;
40 import java.util.ArrayList;
41 import java.util.Collection;
42 import java.util.HashMap;
43 import java.util.List;
44 import java.util.Map;
45 import java.util.TreeMap;
46 import java.util.UUID;
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62 public class ReplicationSink {
63
64 private static final Log LOG = LogFactory.getLog(ReplicationSink.class);
65
66 public static final String REPLICATION_LOG_DIR = ".replogs";
67 private final Configuration conf;
68 private final HConnection sharedHtableCon;
69 private final ReplicationSinkMetrics metrics;
70
71
72
73
74
75
76
77
78 public ReplicationSink(Configuration conf, Stoppable stopper)
79 throws IOException {
80 this.conf = HBaseConfiguration.create(conf);
81 decorateConf();
82 this.sharedHtableCon = HConnectionManager.createConnection(this.conf);
83 this.metrics = new ReplicationSinkMetrics();
84 }
85
86
87
88
89
90 private void decorateConf() {
91 this.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
92 this.conf.getInt("replication.sink.client.retries.number", 4));
93 this.conf.setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
94 this.conf.getInt("replication.sink.client.ops.timeout", 10000));
95 }
96
97
98
99
100
101
102
103
104 public void replicateEntries(HLog.Entry[] entries)
105 throws IOException {
106 if (entries.length == 0) {
107 return;
108 }
109
110
111 try {
112 long totalReplicated = 0;
113
114
115 Map<byte[], Map<UUID,List<Row>>> rowMap = new TreeMap<byte[], Map<UUID,List<Row>>>(Bytes.BYTES_COMPARATOR);
116 for (HLog.Entry entry : entries) {
117 WALEdit edit = entry.getEdit();
118 byte[] table = entry.getKey().getTablename();
119 Put put = null;
120 Delete del = null;
121 KeyValue lastKV = null;
122 List<KeyValue> kvs = edit.getKeyValues();
123 for (KeyValue kv : kvs) {
124 if (lastKV == null || lastKV.getType() != kv.getType() || !lastKV.matchingRow(kv)) {
125 UUID clusterId = entry.getKey().getClusterId();
126 if (kv.isDelete()) {
127 del = new Delete(kv.getRow());
128 del.setClusterId(clusterId);
129 addToHashMultiMap(rowMap, table, clusterId, del);
130 } else {
131 put = new Put(kv.getRow());
132 put.setClusterId(clusterId);
133 addToHashMultiMap(rowMap, table, clusterId, put);
134 }
135 }
136 if (kv.isDelete()) {
137 del.addDeleteMarker(kv);
138 } else {
139 put.add(kv);
140 }
141 lastKV = kv;
142 }
143 totalReplicated++;
144 }
145 for(Map.Entry<byte[], Map<UUID, List<Row>>> entry : rowMap.entrySet()) {
146 batch(entry.getKey(), entry.getValue().values());
147 }
148 this.metrics.setAgeOfLastAppliedOp(
149 entries[entries.length-1].getKey().getWriteTime());
150 this.metrics.appliedBatchesRate.inc(1);
151 LOG.info("Total replicated: " + totalReplicated);
152 } catch (IOException ex) {
153 LOG.error("Unable to accept edit because:", ex);
154 throw ex;
155 }
156 }
157
158
159
160
161
162
163
164
165
166
167 private <K1, K2, V> List<V> addToHashMultiMap(Map<K1, Map<K2,List<V>>> map, K1 key1, K2 key2, V value) {
168 Map<K2,List<V>> innerMap = map.get(key1);
169 if (innerMap == null) {
170 innerMap = new HashMap<K2, List<V>>();
171 map.put(key1, innerMap);
172 }
173 List<V> values = innerMap.get(key2);
174 if (values == null) {
175 values = new ArrayList<V>();
176 innerMap.put(key2, values);
177 }
178 values.add(value);
179 return values;
180 }
181
182
183
184
185 public void stopReplicationSinkServices() {
186 try {
187 this.sharedHtableCon.close();
188 } catch (IOException e) {
189 LOG.warn("IOException while closing the connection", e);
190 }
191 }
192
193
194
195
196
197
198
199 private void batch(byte[] tableName, Collection<List<Row>> allRows) throws IOException {
200 if (allRows.isEmpty()) {
201 return;
202 }
203 HTableInterface table = null;
204 try {
205 table = this.sharedHtableCon.getTable(tableName);
206 for (List<Row> rows : allRows) {
207 table.batch(rows);
208 this.metrics.appliedOpsRate.inc(rows.size());
209 }
210 } catch (InterruptedException ix) {
211 throw new IOException(ix);
212 } finally {
213 if (table != null) {
214 table.close();
215 }
216 }
217 }
218 }