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