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