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.client;
21
22 import org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.classification.InterfaceAudience;
25 import org.apache.hadoop.classification.InterfaceStability;
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.hbase.HRegionInfo;
28 import org.apache.hadoop.hbase.HRegionLocation;
29 import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
30 import org.apache.hadoop.hbase.util.Bytes;
31 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
32
33 import java.io.IOException;
34 import java.util.AbstractMap.SimpleEntry;
35 import java.util.ArrayList;
36 import java.util.HashMap;
37 import java.util.HashSet;
38 import java.util.List;
39 import java.util.Map;
40 import java.util.Set;
41 import java.util.concurrent.ConcurrentHashMap;
42 import java.util.concurrent.ConcurrentSkipListMap;
43 import java.util.concurrent.LinkedBlockingQueue;
44 import java.util.concurrent.atomic.AtomicInteger;
45 import java.util.concurrent.atomic.AtomicLong;
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61 @InterfaceAudience.Public
62 @InterfaceStability.Evolving
63 public class HTableMultiplexer {
64 private static final Log LOG = LogFactory.getLog(HTableMultiplexer.class.getName());
65 private static int poolID = 0;
66
67 static final String TABLE_MULTIPLEXER_FLUSH_FREQ_MS = "hbase.tablemultiplexer.flush.frequency.ms";
68
69 private Map<byte[], HTable> tableNameToHTableMap;
70
71
72 private Map<HRegionLocation, LinkedBlockingQueue<PutStatus>>
73 serverToBufferQueueMap;
74
75
76 private Map<HRegionLocation, HTableFlushWorker> serverToFlushWorkerMap;
77
78 private Configuration conf;
79 private int retryNum;
80 private int perRegionServerBufferQueueSize;
81
82
83
84
85
86
87
88 public HTableMultiplexer(Configuration conf,
89 int perRegionServerBufferQueueSize) throws ZooKeeperConnectionException {
90 this.conf = conf;
91 this.serverToBufferQueueMap = new ConcurrentHashMap<HRegionLocation,
92 LinkedBlockingQueue<PutStatus>>();
93 this.serverToFlushWorkerMap = new ConcurrentHashMap<HRegionLocation, HTableFlushWorker>();
94 this.tableNameToHTableMap = new ConcurrentSkipListMap<byte[], HTable>(Bytes.BYTES_COMPARATOR);
95 this.retryNum = conf.getInt("hbase.client.retries.number", 10);
96 this.perRegionServerBufferQueueSize = perRegionServerBufferQueueSize;
97 }
98
99
100
101
102
103
104
105
106
107 public boolean put(final byte[] table, final Put put) throws IOException {
108 return put(table, put, this.retryNum);
109 }
110
111
112
113
114
115
116
117
118
119 public List<Put> put(final byte[] table, final List<Put> puts)
120 throws IOException {
121 if (puts == null)
122 return null;
123
124 List <Put> failedPuts = null;
125 boolean result;
126 for (Put put : puts) {
127 result = put(table, put, this.retryNum);
128 if (result == false) {
129
130
131 if (failedPuts == null) {
132 failedPuts = new ArrayList<Put>();
133 }
134
135 failedPuts.add(put);
136 }
137 }
138 return failedPuts;
139 }
140
141
142
143
144
145
146
147
148
149
150
151 public boolean put(final byte[] table, final Put put, int retry)
152 throws IOException {
153 if (retry <= 0) {
154 return false;
155 }
156
157 LinkedBlockingQueue<PutStatus> queue;
158 HTable htable = getHTable(table);
159 try {
160 htable.validatePut(put);
161 HRegionLocation loc = htable.getRegionLocation(put.getRow(), false);
162 if (loc != null) {
163
164 queue = addNewRegionServer(loc, htable);
165
166 PutStatus s = new PutStatus(loc.getRegionInfo(), put, retry);
167
168 return queue.offer(s);
169 }
170 } catch (Exception e) {
171 LOG.debug("Cannot process the put " + put + " because of " + e);
172 }
173 return false;
174 }
175
176
177
178
179 public HTableMultiplexerStatus getHTableMultiplexerStatus() {
180 return new HTableMultiplexerStatus(serverToFlushWorkerMap);
181 }
182
183
184 private HTable getHTable(final byte[] table) throws IOException {
185 HTable htable = this.tableNameToHTableMap.get(table);
186 if (htable == null) {
187 synchronized (this.tableNameToHTableMap) {
188 htable = this.tableNameToHTableMap.get(table);
189 if (htable == null) {
190 htable = new HTable(conf, table);
191 this.tableNameToHTableMap.put(table, htable);
192 }
193 }
194 }
195 return htable;
196 }
197
198 private synchronized LinkedBlockingQueue<PutStatus> addNewRegionServer(
199 HRegionLocation addr, HTable htable) {
200 LinkedBlockingQueue<PutStatus> queue =
201 serverToBufferQueueMap.get(addr);
202 if (queue == null) {
203
204 queue = new LinkedBlockingQueue<PutStatus>(perRegionServerBufferQueueSize);
205 serverToBufferQueueMap.put(addr, queue);
206
207
208 HTableFlushWorker worker = new HTableFlushWorker(conf, addr,
209 this, queue, htable);
210 this.serverToFlushWorkerMap.put(addr, worker);
211
212
213
214 String name = "HTableFlushWorker-" + addr.getHostnamePort() + "-"
215 + (poolID++);
216 Thread t = new Thread(worker, name);
217 t.setDaemon(true);
218 t.start();
219 }
220 return queue;
221 }
222
223
224
225
226
227
228 static class HTableMultiplexerStatus {
229 private long totalFailedPutCounter;
230 private long totalBufferedPutCounter;
231 private long maxLatency;
232 private long overallAverageLatency;
233 private Map<String, Long> serverToFailedCounterMap;
234 private Map<String, Long> serverToBufferedCounterMap;
235 private Map<String, Long> serverToAverageLatencyMap;
236 private Map<String, Long> serverToMaxLatencyMap;
237
238 public HTableMultiplexerStatus(
239 Map<HRegionLocation, HTableFlushWorker> serverToFlushWorkerMap) {
240 this.totalBufferedPutCounter = 0;
241 this.totalFailedPutCounter = 0;
242 this.maxLatency = 0;
243 this.overallAverageLatency = 0;
244 this.serverToBufferedCounterMap = new HashMap<String, Long>();
245 this.serverToFailedCounterMap = new HashMap<String, Long>();
246 this.serverToAverageLatencyMap = new HashMap<String, Long>();
247 this.serverToMaxLatencyMap = new HashMap<String, Long>();
248 this.initialize(serverToFlushWorkerMap);
249 }
250
251 private void initialize(
252 Map<HRegionLocation, HTableFlushWorker> serverToFlushWorkerMap) {
253 if (serverToFlushWorkerMap == null) {
254 return;
255 }
256
257 long averageCalcSum = 0;
258 int averageCalcCount = 0;
259 for (Map.Entry<HRegionLocation, HTableFlushWorker> entry : serverToFlushWorkerMap
260 .entrySet()) {
261 HRegionLocation addr = entry.getKey();
262 HTableFlushWorker worker = entry.getValue();
263
264 long bufferedCounter = worker.getTotalBufferedCount();
265 long failedCounter = worker.getTotalFailedCount();
266 long serverMaxLatency = worker.getMaxLatency();
267 AtomicAverageCounter averageCounter = worker.getAverageLatencyCounter();
268
269 SimpleEntry<Long, Integer> averageComponents = averageCounter
270 .getComponents();
271 long serverAvgLatency = averageCounter.getAndReset();
272
273 this.totalBufferedPutCounter += bufferedCounter;
274 this.totalFailedPutCounter += failedCounter;
275 if (serverMaxLatency > this.maxLatency) {
276 this.maxLatency = serverMaxLatency;
277 }
278 averageCalcSum += averageComponents.getKey();
279 averageCalcCount += averageComponents.getValue();
280
281 this.serverToBufferedCounterMap.put(addr.getHostnamePort(),
282 bufferedCounter);
283 this.serverToFailedCounterMap
284 .put(addr.getHostnamePort(),
285 failedCounter);
286 this.serverToAverageLatencyMap.put(addr.getHostnamePort(),
287 serverAvgLatency);
288 this.serverToMaxLatencyMap
289 .put(addr.getHostnamePort(),
290 serverMaxLatency);
291 }
292 this.overallAverageLatency = averageCalcCount != 0 ? averageCalcSum
293 / averageCalcCount : 0;
294 }
295
296 public long getTotalBufferedCounter() {
297 return this.totalBufferedPutCounter;
298 }
299
300 public long getTotalFailedCounter() {
301 return this.totalFailedPutCounter;
302 }
303
304 public long getMaxLatency() {
305 return this.maxLatency;
306 }
307
308 public long getOverallAverageLatency() {
309 return this.overallAverageLatency;
310 }
311
312 public Map<String, Long> getBufferedCounterForEachRegionServer() {
313 return this.serverToBufferedCounterMap;
314 }
315
316 public Map<String, Long> getFailedCounterForEachRegionServer() {
317 return this.serverToFailedCounterMap;
318 }
319
320 public Map<String, Long> getMaxLatencyForEachRegionServer() {
321 return this.serverToMaxLatencyMap;
322 }
323
324 public Map<String, Long> getAverageLatencyForEachRegionServer() {
325 return this.serverToAverageLatencyMap;
326 }
327 }
328
329 private static class PutStatus {
330 private final HRegionInfo regionInfo;
331 private final Put put;
332 private final int retryCount;
333 public PutStatus(final HRegionInfo regionInfo, final Put put,
334 final int retryCount) {
335 this.regionInfo = regionInfo;
336 this.put = put;
337 this.retryCount = retryCount;
338 }
339
340 public HRegionInfo getRegionInfo() {
341 return regionInfo;
342 }
343 public Put getPut() {
344 return put;
345 }
346 public int getRetryCount() {
347 return retryCount;
348 }
349 }
350
351
352
353
354 private static class AtomicAverageCounter {
355 private long sum;
356 private int count;
357
358 public AtomicAverageCounter() {
359 this.sum = 0L;
360 this.count = 0;
361 }
362
363 public synchronized long getAndReset() {
364 long result = this.get();
365 this.reset();
366 return result;
367 }
368
369 public synchronized long get() {
370 if (this.count == 0) {
371 return 0;
372 }
373 return this.sum / this.count;
374 }
375
376 public synchronized SimpleEntry<Long, Integer> getComponents() {
377 return new SimpleEntry<Long, Integer>(sum, count);
378 }
379
380 public synchronized void reset() {
381 this.sum = 0l;
382 this.count = 0;
383 }
384
385 public synchronized void add(long value) {
386 this.sum += value;
387 this.count++;
388 }
389 }
390
391 private static class HTableFlushWorker implements Runnable {
392 private HRegionLocation addr;
393 private Configuration conf;
394 private LinkedBlockingQueue<PutStatus> queue;
395 private HTableMultiplexer htableMultiplexer;
396 private AtomicLong totalFailedPutCount;
397 private AtomicInteger currentProcessingPutCount;
398 private AtomicAverageCounter averageLatency;
399 private AtomicLong maxLatency;
400 private HTable htable;
401
402 public HTableFlushWorker(Configuration conf, HRegionLocation addr,
403 HTableMultiplexer htableMultiplexer,
404 LinkedBlockingQueue<PutStatus> queue, HTable htable) {
405 this.addr = addr;
406 this.conf = conf;
407 this.htableMultiplexer = htableMultiplexer;
408 this.queue = queue;
409 this.totalFailedPutCount = new AtomicLong(0);
410 this.currentProcessingPutCount = new AtomicInteger(0);
411 this.averageLatency = new AtomicAverageCounter();
412 this.maxLatency = new AtomicLong(0);
413 this.htable = htable;
414 }
415
416 public long getTotalFailedCount() {
417 return totalFailedPutCount.get();
418 }
419
420 public long getTotalBufferedCount() {
421 return queue.size() + currentProcessingPutCount.get();
422 }
423
424 public AtomicAverageCounter getAverageLatencyCounter() {
425 return this.averageLatency;
426 }
427
428 public long getMaxLatency() {
429 return this.maxLatency.getAndSet(0);
430 }
431
432 private boolean resubmitFailedPut(PutStatus failedPutStatus,
433 HRegionLocation oldLoc) throws IOException {
434 Put failedPut = failedPutStatus.getPut();
435
436 byte[] tableName = failedPutStatus.getRegionInfo().getTableName();
437
438 int retryCount = failedPutStatus.getRetryCount() - 1;
439
440 if (retryCount <= 0) {
441
442 return false;
443 } else {
444
445 return this.htableMultiplexer.put(tableName, failedPut, retryCount);
446 }
447 }
448
449 @Override
450 public void run() {
451 List<PutStatus> processingList = new ArrayList<PutStatus>();
452
453
454
455
456 long frequency = conf.getLong(TABLE_MULTIPLEXER_FLUSH_FREQ_MS, 100);
457
458
459 try {
460 Thread.sleep(frequency);
461 } catch (InterruptedException e) {
462 }
463
464 long start, elapsed;
465 int failedCount = 0;
466 while (true) {
467 try {
468 start = elapsed = EnvironmentEdgeManager.currentTimeMillis();
469
470
471 processingList.clear();
472 failedCount = 0;
473
474
475 queue.drainTo(processingList);
476 currentProcessingPutCount.set(processingList.size());
477
478 if (processingList.size() > 0) {
479 ArrayList<Put> list = new ArrayList<Put>(processingList.size());
480 for (PutStatus putStatus: processingList) {
481 list.add(putStatus.getPut());
482 }
483
484
485 List<Put> failed = null;
486 Object[] results = new Object[list.size()];
487 try {
488 htable.batch(list, results);
489 } catch (IOException e) {
490 LOG.debug("Caught some exceptions " + e
491 + " when flushing puts to region server " + addr.getHostnamePort());
492 } finally {
493
494
495
496
497
498 for (int i = results.length - 1; i >= 0; i--) {
499 if (results[i] instanceof Result) {
500
501 list.remove(i);
502 }
503 }
504 failed = list;
505 }
506
507 if (failed != null) {
508 if (failed.size() == processingList.size()) {
509
510 for (PutStatus putStatus: processingList) {
511 if (!resubmitFailedPut(putStatus, this.addr)) {
512 failedCount++;
513 }
514 }
515 } else {
516 Set<Put> failedPutSet = new HashSet<Put>(failed);
517 for (PutStatus putStatus: processingList) {
518 if (failedPutSet.contains(putStatus.getPut())
519 && !resubmitFailedPut(putStatus, this.addr)) {
520 failedCount++;
521 }
522 }
523 }
524 }
525
526 this.totalFailedPutCount.addAndGet(failedCount);
527
528 elapsed = EnvironmentEdgeManager.currentTimeMillis() - start;
529
530 averageLatency.add(elapsed);
531 if (elapsed > maxLatency.get()) {
532 maxLatency.set(elapsed);
533 }
534
535
536 if (LOG.isDebugEnabled()) {
537 LOG.debug("Processed " + currentProcessingPutCount
538 + " put requests for " + addr.getHostnamePort() + " and "
539 + failedCount + " failed" + ", latency for this send: "
540 + elapsed);
541 }
542
543
544 currentProcessingPutCount.set(0);
545 }
546
547
548 if (elapsed == start) {
549 elapsed = EnvironmentEdgeManager.currentTimeMillis() - start;
550 }
551 if (elapsed < frequency) {
552 Thread.sleep(frequency - elapsed);
553 }
554 } catch (Exception e) {
555
556 LOG.debug("Caught some exceptions " + e
557 + " when flushing puts to region server "
558 + addr.getHostnamePort());
559 }
560 }
561 }
562 }
563 }