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.List;
24 import java.util.Map;
25 import java.util.Map.Entry;
26 import java.util.TreeMap;
27 import java.util.concurrent.ExecutorService;
28 import java.util.concurrent.SynchronousQueue;
29 import java.util.concurrent.ThreadPoolExecutor;
30 import java.util.concurrent.TimeUnit;
31 import java.util.concurrent.atomic.AtomicLong;
32
33 import org.apache.commons.logging.Log;
34 import org.apache.commons.logging.LogFactory;
35 import org.apache.hadoop.classification.InterfaceAudience;
36 import org.apache.hadoop.conf.Configuration;
37 import org.apache.hadoop.hbase.HBaseConfiguration;
38 import org.apache.hadoop.hbase.HConstants;
39 import org.apache.hadoop.hbase.KeyValue;
40 import org.apache.hadoop.hbase.Stoppable;
41 import org.apache.hadoop.hbase.client.Delete;
42 import org.apache.hadoop.hbase.client.HConnection;
43 import org.apache.hadoop.hbase.client.HConnectionManager;
44 import org.apache.hadoop.hbase.client.HTable;
45 import org.apache.hadoop.hbase.client.HTableInterface;
46 import org.apache.hadoop.hbase.client.Put;
47 import org.apache.hadoop.hbase.client.Row;
48 import org.apache.hadoop.hbase.regionserver.wal.HLog;
49 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
50 import org.apache.hadoop.hbase.util.Bytes;
51 import org.apache.hadoop.hbase.util.Threads;
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67 @InterfaceAudience.Private
68 public class ReplicationSink {
69
70 private static final Log LOG = LogFactory.getLog(ReplicationSink.class);
71
72 public static final String REPLICATION_LOG_DIR = ".replogs";
73 private final Configuration conf;
74 private final ExecutorService sharedThreadPool;
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 this.sharedThreadPool = new ThreadPoolExecutor(1,
93 conf.getInt("hbase.htable.threads.max", Integer.MAX_VALUE),
94 conf.getLong("hbase.htable.threads.keepalivetime", 60), TimeUnit.SECONDS,
95 new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("hbase-repl"));
96 ((ThreadPoolExecutor) this.sharedThreadPool).allowCoreThreadTimeOut(true);
97 }
98
99
100
101
102
103 private void decorateConf() {
104 this.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
105 this.conf.getInt("replication.sink.client.retries.number", 4));
106 this.conf.setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
107 this.conf.getInt("replication.sink.client.ops.timeout", 10000));
108 }
109
110
111
112
113
114
115
116
117 public void replicateEntries(HLog.Entry[] entries)
118 throws IOException {
119 if (entries.length == 0) {
120 return;
121 }
122
123
124 try {
125 long totalReplicated = 0;
126
127
128 Map<byte[], List<Row>> rows = new TreeMap<byte[], List<Row>>(Bytes.BYTES_COMPARATOR);
129 for (HLog.Entry entry : entries) {
130 WALEdit edit = entry.getEdit();
131 byte[] table = entry.getKey().getTablename();
132 Put put = null;
133 Delete del = null;
134 KeyValue lastKV = null;
135 List<KeyValue> kvs = edit.getKeyValues();
136 for (KeyValue kv : kvs) {
137 if (lastKV == null || lastKV.getType() != kv.getType() || !lastKV.matchingRow(kv)) {
138 if (kv.isDelete()) {
139 del = new Delete(kv.getRow());
140 del.setClusterId(entry.getKey().getClusterId());
141 addToMultiMap(rows, table, del);
142 } else {
143 put = new Put(kv.getRow());
144 put.setClusterId(entry.getKey().getClusterId());
145 addToMultiMap(rows, table, put);
146 }
147 }
148 if (kv.isDelete()) {
149 del.addDeleteMarker(kv);
150 } else {
151 put.add(kv);
152 }
153 lastKV = kv;
154 }
155 totalReplicated++;
156 }
157 for (Entry<byte[], List<Row>> entry : rows.entrySet()) {
158 batch(entry.getKey(), entry.getValue());
159 }
160 this.metrics.setAgeOfLastAppliedOp(
161 entries[entries.length-1].getKey().getWriteTime());
162 this.metrics.applyBatch(entries.length);
163 this.totalReplicatedEdits.addAndGet(totalReplicated);
164 } catch (IOException ex) {
165 LOG.error("Unable to accept edit because:", ex);
166 throw ex;
167 }
168 }
169
170
171
172
173
174
175
176
177
178 private <K, V> List<V> addToMultiMap(Map<K, List<V>> map, K key, V value) {
179 List<V> values = map.get(key);
180 if (values == null) {
181 values = new ArrayList<V>();
182 map.put(key, values);
183 }
184 values.add(value);
185 return values;
186 }
187
188
189
190
191 public void stopReplicationSinkServices() {
192 try {
193 this.sharedThreadPool.shutdown();
194 if (!this.sharedThreadPool.awaitTermination(60000, TimeUnit.MILLISECONDS)) {
195 this.sharedThreadPool.shutdownNow();
196 }
197 } catch (InterruptedException e) {
198 LOG.warn("Interrupted while closing the table pool", e);
199 Thread.currentThread().interrupt();
200 }
201 try {
202 this.sharedHtableCon.close();
203 } catch (IOException e) {
204 LOG.warn("IOException while closing the connection", e);
205 }
206 }
207
208
209
210
211
212
213
214
215 private void batch(byte[] tableName, List<Row> rows) throws IOException {
216 if (rows.isEmpty()) {
217 return;
218 }
219 HTableInterface table = null;
220 try {
221 table = new HTable(tableName, this.sharedHtableCon, this.sharedThreadPool);
222 table.batch(rows);
223 } catch (InterruptedException ix) {
224 throw new IOException(ix);
225 } finally {
226 if (table != null) {
227 table.close();
228 }
229 }
230 }
231
232
233
234
235
236
237 public String getStats() {
238 return this.totalReplicatedEdits.get() == 0 ? "" : "Sink: " +
239 "age in ms of last applied edit: " + this.metrics.refreshAgeOfLastAppliedOp() +
240 ", total replicated edits: " + this.totalReplicatedEdits;
241 }
242 }