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