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 com.google.common.util.concurrent.ThreadFactoryBuilder;
23
24 import java.io.IOException;
25 import java.util.AbstractMap.SimpleEntry;
26 import java.util.ArrayList;
27 import java.util.Collections;
28 import java.util.HashMap;
29 import java.util.List;
30 import java.util.Map;
31 import java.util.concurrent.ConcurrentHashMap;
32 import java.util.concurrent.ExecutorService;
33 import java.util.concurrent.Executors;
34 import java.util.concurrent.LinkedBlockingQueue;
35 import java.util.concurrent.ScheduledExecutorService;
36 import java.util.concurrent.TimeUnit;
37 import java.util.concurrent.atomic.AtomicInteger;
38 import java.util.concurrent.atomic.AtomicLong;
39
40 import org.apache.commons.logging.Log;
41 import org.apache.commons.logging.LogFactory;
42 import org.apache.hadoop.conf.Configuration;
43 import org.apache.hadoop.hbase.HBaseConfiguration;
44 import org.apache.hadoop.hbase.HConstants;
45 import org.apache.hadoop.hbase.HRegionInfo;
46 import org.apache.hadoop.hbase.HRegionLocation;
47 import org.apache.hadoop.hbase.ServerName;
48 import org.apache.hadoop.hbase.TableName;
49 import org.apache.hadoop.hbase.classification.InterfaceAudience;
50 import org.apache.hadoop.hbase.classification.InterfaceStability;
51 import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
52 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
53 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69 @InterfaceAudience.Public
70 @InterfaceStability.Evolving
71 public class HTableMultiplexer {
72 private static final Log LOG = LogFactory.getLog(HTableMultiplexer.class.getName());
73
74 public static final String TABLE_MULTIPLEXER_FLUSH_PERIOD_MS =
75 "hbase.tablemultiplexer.flush.period.ms";
76 public static final String TABLE_MULTIPLEXER_INIT_THREADS = "hbase.tablemultiplexer.init.threads";
77 public static final String TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE =
78 "hbase.client.max.retries.in.queue";
79
80
81 private final Map<HRegionLocation, FlushWorker> serverToFlushWorkerMap =
82 new ConcurrentHashMap<>();
83
84 private final Configuration workerConf;
85 private final ClusterConnection conn;
86 private final ExecutorService pool;
87 private final int retryNum;
88 private final int perRegionServerBufferQueueSize;
89 private final int maxKeyValueSize;
90 private final ScheduledExecutorService executor;
91 private final long flushPeriod;
92
93
94
95
96
97
98 public HTableMultiplexer(Configuration conf, int perRegionServerBufferQueueSize)
99 throws IOException {
100 this(ConnectionFactory.createConnection(conf), conf, perRegionServerBufferQueueSize);
101 }
102
103
104
105
106
107
108
109 public HTableMultiplexer(Connection conn, Configuration conf,
110 int perRegionServerBufferQueueSize) {
111 this.conn = (ClusterConnection) conn;
112 this.pool = HTable.getDefaultExecutor(conf);
113 this.retryNum = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
114 HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
115 this.perRegionServerBufferQueueSize = perRegionServerBufferQueueSize;
116 this.maxKeyValueSize = HTable.getMaxKeyValueSize(conf);
117 this.flushPeriod = conf.getLong(TABLE_MULTIPLEXER_FLUSH_PERIOD_MS, 100);
118 int initThreads = conf.getInt(TABLE_MULTIPLEXER_INIT_THREADS, 10);
119 this.executor =
120 Executors.newScheduledThreadPool(initThreads,
121 new ThreadFactoryBuilder().setDaemon(true).setNameFormat("HTableFlushWorker-%d").build());
122
123 this.workerConf = HBaseConfiguration.create(conf);
124
125
126 this.workerConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0);
127 }
128
129
130
131
132
133
134 @SuppressWarnings("deprecation")
135 public synchronized void close() throws IOException {
136 if (!getConnection().isClosed()) {
137 getConnection().close();
138 }
139 }
140
141
142
143
144
145
146
147
148 public boolean put(TableName tableName, final Put put) {
149 return put(tableName, put, this.retryNum);
150 }
151
152
153
154
155
156
157
158
159 public List<Put> put(TableName tableName, final List<Put> puts) {
160 if (puts == null)
161 return null;
162
163 List <Put> failedPuts = null;
164 boolean result;
165 for (Put put : puts) {
166 result = put(tableName, put, this.retryNum);
167 if (result == false) {
168
169
170 if (failedPuts == null) {
171 failedPuts = new ArrayList<Put>();
172 }
173
174 failedPuts.add(put);
175 }
176 }
177 return failedPuts;
178 }
179
180
181
182
183 @Deprecated
184 public List<Put> put(byte[] tableName, final List<Put> puts) {
185 return put(TableName.valueOf(tableName), puts);
186 }
187
188
189
190
191
192
193
194 public boolean put(final TableName tableName, final Put put, int maxAttempts) {
195 if (maxAttempts <= 0) {
196 return false;
197 }
198
199 try {
200 HTable.validatePut(put, maxKeyValueSize);
201
202 ClusterConnection conn = (ClusterConnection) getConnection();
203
204
205 HRegionLocation loc = conn.getRegionLocation(tableName, put.getRow(), false);
206 if (loc != null) {
207
208 LinkedBlockingQueue<PutStatus> queue = getQueue(loc);
209
210
211 PutStatus s = new PutStatus(loc.getRegionInfo(), put, maxAttempts);
212
213 return queue.offer(s);
214 }
215 } catch (IOException e) {
216 LOG.debug("Cannot process the put " + put, e);
217 }
218 return false;
219 }
220
221
222
223
224 @Deprecated
225 public boolean put(final byte[] tableName, final Put put, int retry) {
226 return put(TableName.valueOf(tableName), put, retry);
227 }
228
229
230
231
232 @Deprecated
233 public boolean put(final byte[] tableName, Put put) {
234 return put(TableName.valueOf(tableName), put);
235 }
236
237
238
239
240 public HTableMultiplexerStatus getHTableMultiplexerStatus() {
241 return new HTableMultiplexerStatus(serverToFlushWorkerMap);
242 }
243
244 @InterfaceAudience.Private
245 LinkedBlockingQueue<PutStatus> getQueue(HRegionLocation addr) {
246 FlushWorker worker = serverToFlushWorkerMap.get(addr);
247 if (worker == null) {
248 synchronized (this.serverToFlushWorkerMap) {
249 worker = serverToFlushWorkerMap.get(addr);
250 if (worker == null) {
251
252 worker = new FlushWorker(workerConf, this.conn, addr, this, perRegionServerBufferQueueSize,
253 pool, executor);
254 this.serverToFlushWorkerMap.put(addr, worker);
255 executor.scheduleAtFixedRate(worker, flushPeriod, flushPeriod, TimeUnit.MILLISECONDS);
256 }
257 }
258 }
259 return worker.getQueue();
260 }
261
262 @InterfaceAudience.Private
263 ClusterConnection getConnection() {
264 return this.conn;
265 }
266
267
268
269
270
271
272 @InterfaceAudience.Public
273 @InterfaceStability.Evolving
274 public static class HTableMultiplexerStatus {
275 private long totalFailedPutCounter;
276 private long totalBufferedPutCounter;
277 private long maxLatency;
278 private long overallAverageLatency;
279 private Map<String, Long> serverToFailedCounterMap;
280 private Map<String, Long> serverToBufferedCounterMap;
281 private Map<String, Long> serverToAverageLatencyMap;
282 private Map<String, Long> serverToMaxLatencyMap;
283
284 public HTableMultiplexerStatus(
285 Map<HRegionLocation, FlushWorker> serverToFlushWorkerMap) {
286 this.totalBufferedPutCounter = 0;
287 this.totalFailedPutCounter = 0;
288 this.maxLatency = 0;
289 this.overallAverageLatency = 0;
290 this.serverToBufferedCounterMap = new HashMap<String, Long>();
291 this.serverToFailedCounterMap = new HashMap<String, Long>();
292 this.serverToAverageLatencyMap = new HashMap<String, Long>();
293 this.serverToMaxLatencyMap = new HashMap<String, Long>();
294 this.initialize(serverToFlushWorkerMap);
295 }
296
297 private void initialize(
298 Map<HRegionLocation, FlushWorker> serverToFlushWorkerMap) {
299 if (serverToFlushWorkerMap == null) {
300 return;
301 }
302
303 long averageCalcSum = 0;
304 int averageCalcCount = 0;
305 for (Map.Entry<HRegionLocation, FlushWorker> entry : serverToFlushWorkerMap
306 .entrySet()) {
307 HRegionLocation addr = entry.getKey();
308 FlushWorker worker = entry.getValue();
309
310 long bufferedCounter = worker.getTotalBufferedCount();
311 long failedCounter = worker.getTotalFailedCount();
312 long serverMaxLatency = worker.getMaxLatency();
313 AtomicAverageCounter averageCounter = worker.getAverageLatencyCounter();
314
315 SimpleEntry<Long, Integer> averageComponents = averageCounter
316 .getComponents();
317 long serverAvgLatency = averageCounter.getAndReset();
318
319 this.totalBufferedPutCounter += bufferedCounter;
320 this.totalFailedPutCounter += failedCounter;
321 if (serverMaxLatency > this.maxLatency) {
322 this.maxLatency = serverMaxLatency;
323 }
324 averageCalcSum += averageComponents.getKey();
325 averageCalcCount += averageComponents.getValue();
326
327 this.serverToBufferedCounterMap.put(addr.getHostnamePort(),
328 bufferedCounter);
329 this.serverToFailedCounterMap
330 .put(addr.getHostnamePort(),
331 failedCounter);
332 this.serverToAverageLatencyMap.put(addr.getHostnamePort(),
333 serverAvgLatency);
334 this.serverToMaxLatencyMap
335 .put(addr.getHostnamePort(),
336 serverMaxLatency);
337 }
338 this.overallAverageLatency = averageCalcCount != 0 ? averageCalcSum
339 / averageCalcCount : 0;
340 }
341
342 public long getTotalBufferedCounter() {
343 return this.totalBufferedPutCounter;
344 }
345
346 public long getTotalFailedCounter() {
347 return this.totalFailedPutCounter;
348 }
349
350 public long getMaxLatency() {
351 return this.maxLatency;
352 }
353
354 public long getOverallAverageLatency() {
355 return this.overallAverageLatency;
356 }
357
358 public Map<String, Long> getBufferedCounterForEachRegionServer() {
359 return this.serverToBufferedCounterMap;
360 }
361
362 public Map<String, Long> getFailedCounterForEachRegionServer() {
363 return this.serverToFailedCounterMap;
364 }
365
366 public Map<String, Long> getMaxLatencyForEachRegionServer() {
367 return this.serverToMaxLatencyMap;
368 }
369
370 public Map<String, Long> getAverageLatencyForEachRegionServer() {
371 return this.serverToAverageLatencyMap;
372 }
373 }
374
375 @InterfaceAudience.Private
376 static class PutStatus {
377 public final HRegionInfo regionInfo;
378 public final Put put;
379 public final int retryCount;
380
381 public PutStatus(HRegionInfo regionInfo, Put put, int retryCount) {
382 this.regionInfo = regionInfo;
383 this.put = put;
384 this.retryCount = retryCount;
385 }
386 }
387
388
389
390
391 private static class AtomicAverageCounter {
392 private long sum;
393 private int count;
394
395 public AtomicAverageCounter() {
396 this.sum = 0L;
397 this.count = 0;
398 }
399
400 public synchronized long getAndReset() {
401 long result = this.get();
402 this.reset();
403 return result;
404 }
405
406 public synchronized long get() {
407 if (this.count == 0) {
408 return 0;
409 }
410 return this.sum / this.count;
411 }
412
413 public synchronized SimpleEntry<Long, Integer> getComponents() {
414 return new SimpleEntry<Long, Integer>(sum, count);
415 }
416
417 public synchronized void reset() {
418 this.sum = 0l;
419 this.count = 0;
420 }
421
422 public synchronized void add(long value) {
423 this.sum += value;
424 this.count++;
425 }
426 }
427
428 @InterfaceAudience.Private
429 static class FlushWorker implements Runnable {
430 private final HRegionLocation addr;
431 private final LinkedBlockingQueue<PutStatus> queue;
432 private final HTableMultiplexer multiplexer;
433 private final AtomicLong totalFailedPutCount = new AtomicLong(0);
434 private final AtomicInteger currentProcessingCount = new AtomicInteger(0);
435 private final AtomicAverageCounter averageLatency = new AtomicAverageCounter();
436 private final AtomicLong maxLatency = new AtomicLong(0);
437
438 private final AsyncProcess ap;
439 private final List<PutStatus> processingList = new ArrayList<>();
440 private final ScheduledExecutorService executor;
441 private final int maxRetryInQueue;
442 private final AtomicInteger retryInQueue = new AtomicInteger(0);
443 private final int writeRpcTimeout;
444
445 public FlushWorker(Configuration conf, ClusterConnection conn, HRegionLocation addr,
446 HTableMultiplexer htableMultiplexer, int perRegionServerBufferQueueSize,
447 ExecutorService pool, ScheduledExecutorService executor) {
448 this.addr = addr;
449 this.multiplexer = htableMultiplexer;
450 this.queue = new LinkedBlockingQueue<>(perRegionServerBufferQueueSize);
451 RpcRetryingCallerFactory rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf);
452 RpcControllerFactory rpcControllerFactory = RpcControllerFactory.instantiate(conf);
453 this.writeRpcTimeout = conf.getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY,
454 conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
455 HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
456 this.ap = new AsyncProcess(conn, conf, pool, rpcCallerFactory, false, rpcControllerFactory, writeRpcTimeout);
457 this.executor = executor;
458 this.maxRetryInQueue = conf.getInt(TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE, 10000);
459 }
460
461 protected LinkedBlockingQueue<PutStatus> getQueue() {
462 return this.queue;
463 }
464
465 public long getTotalFailedCount() {
466 return totalFailedPutCount.get();
467 }
468
469 public long getTotalBufferedCount() {
470 return (long) queue.size() + currentProcessingCount.get();
471 }
472
473 public AtomicAverageCounter getAverageLatencyCounter() {
474 return this.averageLatency;
475 }
476
477 public long getMaxLatency() {
478 return this.maxLatency.getAndSet(0);
479 }
480
481 boolean resubmitFailedPut(PutStatus ps, HRegionLocation oldLoc) throws IOException {
482
483 final int retryCount = ps.retryCount - 1;
484
485 if (retryCount <= 0) {
486
487 return false;
488 }
489
490 int cnt = getRetryInQueue().incrementAndGet();
491 if (cnt > getMaxRetryInQueue()) {
492
493 getRetryInQueue().decrementAndGet();
494 return false;
495 }
496
497 final Put failedPut = ps.put;
498
499 final TableName tableName = ps.regionInfo.getTable();
500
501 long delayMs = getNextDelay(retryCount);
502 if (LOG.isDebugEnabled()) {
503 LOG.debug("resubmitting after " + delayMs + "ms: " + retryCount);
504 }
505
506
507
508
509
510 getExecutor().schedule(new Runnable() {
511 @Override
512 public void run() {
513 boolean succ = false;
514 try {
515 succ = FlushWorker.this.getMultiplexer().put(tableName, failedPut, retryCount);
516 } finally {
517 FlushWorker.this.getRetryInQueue().decrementAndGet();
518 if (!succ) {
519 FlushWorker.this.getTotalFailedPutCount().incrementAndGet();
520 }
521 }
522 }
523 }, delayMs, TimeUnit.MILLISECONDS);
524 return true;
525 }
526
527 @InterfaceAudience.Private
528 long getNextDelay(int retryCount) {
529 return ConnectionUtils.getPauseTime(multiplexer.flushPeriod,
530 multiplexer.retryNum - retryCount - 1);
531 }
532
533 @InterfaceAudience.Private
534 AtomicInteger getRetryInQueue() {
535 return this.retryInQueue;
536 }
537
538 @InterfaceAudience.Private
539 int getMaxRetryInQueue() {
540 return this.maxRetryInQueue;
541 }
542
543 @InterfaceAudience.Private
544 AtomicLong getTotalFailedPutCount() {
545 return this.totalFailedPutCount;
546 }
547
548 @InterfaceAudience.Private
549 HTableMultiplexer getMultiplexer() {
550 return this.multiplexer;
551 }
552
553 @InterfaceAudience.Private
554 ScheduledExecutorService getExecutor() {
555 return this.executor;
556 }
557
558 @Override
559 public void run() {
560 int failedCount = 0;
561 try {
562 long start = EnvironmentEdgeManager.currentTime();
563
564
565 processingList.clear();
566 queue.drainTo(processingList);
567 if (processingList.size() == 0) {
568
569 return;
570 }
571
572 currentProcessingCount.set(processingList.size());
573
574 failedCount = processingList.size();
575
576 List<Action<Row>> retainedActions = new ArrayList<>(processingList.size());
577 MultiAction<Row> actions = new MultiAction<>();
578 for (int i = 0; i < processingList.size(); i++) {
579 PutStatus putStatus = processingList.get(i);
580 Action<Row> action = new Action<Row>(putStatus.put, i);
581 actions.add(putStatus.regionInfo.getRegionName(), action);
582 retainedActions.add(action);
583 }
584
585
586 List<PutStatus> failed = null;
587 Object[] results = new Object[actions.size()];
588 ServerName server = addr.getServerName();
589 Map<ServerName, MultiAction<Row>> actionsByServer =
590 Collections.singletonMap(server, actions);
591 try {
592 AsyncRequestFuture arf =
593 ap.submitMultiActions(null, retainedActions, 0L, null, results, true, null,
594 null, actionsByServer, null);
595 arf.waitUntilDone();
596 if (arf.hasError()) {
597
598 LOG.debug("Caught some exceptions when flushing puts to region server "
599 + addr.getHostnamePort(), arf.getErrors());
600 }
601 } finally {
602 for (int i = 0; i < results.length; i++) {
603 if (results[i] instanceof Result) {
604 failedCount--;
605 } else {
606 if (failed == null) {
607 failed = new ArrayList<PutStatus>();
608 }
609 failed.add(processingList.get(i));
610 }
611 }
612 }
613
614 if (failed != null) {
615
616 for (PutStatus putStatus : failed) {
617 if (resubmitFailedPut(putStatus, this.addr)) {
618 failedCount--;
619 }
620 }
621 }
622
623 long elapsed = EnvironmentEdgeManager.currentTime() - start;
624
625 averageLatency.add(elapsed);
626 if (elapsed > maxLatency.get()) {
627 maxLatency.set(elapsed);
628 }
629
630
631 if (LOG.isDebugEnabled()) {
632 LOG.debug("Processed " + currentProcessingCount + " put requests for "
633 + addr.getHostnamePort() + " and " + failedCount + " failed"
634 + ", latency for this send: " + elapsed);
635 }
636
637
638 currentProcessingCount.set(0);
639 } catch (RuntimeException e) {
640
641
642 LOG.debug(
643 "Caught some exceptions " + e + " when flushing puts to region server "
644 + addr.getHostnamePort(), e);
645 } catch (Exception e) {
646 if (e instanceof InterruptedException) {
647 Thread.currentThread().interrupt();
648 }
649
650 LOG.debug(
651 "Caught some exceptions " + e + " when flushing puts to region server "
652 + addr.getHostnamePort(), e);
653 } finally {
654
655 this.totalFailedPutCount.addAndGet(failedCount);
656 }
657 }
658 }
659 }