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 @InterfaceAudience.Public
242 @InterfaceStability.Evolving
243 public static class HTableMultiplexerStatus {
244 private long totalFailedPutCounter;
245 private long totalBufferedPutCounter;
246 private long maxLatency;
247 private long overallAverageLatency;
248 private Map<String, Long> serverToFailedCounterMap;
249 private Map<String, Long> serverToBufferedCounterMap;
250 private Map<String, Long> serverToAverageLatencyMap;
251 private Map<String, Long> serverToMaxLatencyMap;
252
253 public HTableMultiplexerStatus(
254 Map<HRegionLocation, FlushWorker> serverToFlushWorkerMap) {
255 this.totalBufferedPutCounter = 0;
256 this.totalFailedPutCounter = 0;
257 this.maxLatency = 0;
258 this.overallAverageLatency = 0;
259 this.serverToBufferedCounterMap = new HashMap<String, Long>();
260 this.serverToFailedCounterMap = new HashMap<String, Long>();
261 this.serverToAverageLatencyMap = new HashMap<String, Long>();
262 this.serverToMaxLatencyMap = new HashMap<String, Long>();
263 this.initialize(serverToFlushWorkerMap);
264 }
265
266 private void initialize(
267 Map<HRegionLocation, FlushWorker> serverToFlushWorkerMap) {
268 if (serverToFlushWorkerMap == null) {
269 return;
270 }
271
272 long averageCalcSum = 0;
273 int averageCalcCount = 0;
274 for (Map.Entry<HRegionLocation, FlushWorker> entry : serverToFlushWorkerMap
275 .entrySet()) {
276 HRegionLocation addr = entry.getKey();
277 FlushWorker worker = entry.getValue();
278
279 long bufferedCounter = worker.getTotalBufferedCount();
280 long failedCounter = worker.getTotalFailedCount();
281 long serverMaxLatency = worker.getMaxLatency();
282 AtomicAverageCounter averageCounter = worker.getAverageLatencyCounter();
283
284 SimpleEntry<Long, Integer> averageComponents = averageCounter
285 .getComponents();
286 long serverAvgLatency = averageCounter.getAndReset();
287
288 this.totalBufferedPutCounter += bufferedCounter;
289 this.totalFailedPutCounter += failedCounter;
290 if (serverMaxLatency > this.maxLatency) {
291 this.maxLatency = serverMaxLatency;
292 }
293 averageCalcSum += averageComponents.getKey();
294 averageCalcCount += averageComponents.getValue();
295
296 this.serverToBufferedCounterMap.put(addr.getHostnamePort(),
297 bufferedCounter);
298 this.serverToFailedCounterMap
299 .put(addr.getHostnamePort(),
300 failedCounter);
301 this.serverToAverageLatencyMap.put(addr.getHostnamePort(),
302 serverAvgLatency);
303 this.serverToMaxLatencyMap
304 .put(addr.getHostnamePort(),
305 serverMaxLatency);
306 }
307 this.overallAverageLatency = averageCalcCount != 0 ? averageCalcSum
308 / averageCalcCount : 0;
309 }
310
311 public long getTotalBufferedCounter() {
312 return this.totalBufferedPutCounter;
313 }
314
315 public long getTotalFailedCounter() {
316 return this.totalFailedPutCounter;
317 }
318
319 public long getMaxLatency() {
320 return this.maxLatency;
321 }
322
323 public long getOverallAverageLatency() {
324 return this.overallAverageLatency;
325 }
326
327 public Map<String, Long> getBufferedCounterForEachRegionServer() {
328 return this.serverToBufferedCounterMap;
329 }
330
331 public Map<String, Long> getFailedCounterForEachRegionServer() {
332 return this.serverToFailedCounterMap;
333 }
334
335 public Map<String, Long> getMaxLatencyForEachRegionServer() {
336 return this.serverToMaxLatencyMap;
337 }
338
339 public Map<String, Long> getAverageLatencyForEachRegionServer() {
340 return this.serverToAverageLatencyMap;
341 }
342 }
343
344 private static class PutStatus {
345 public final HRegionInfo regionInfo;
346 public final Put put;
347 public final int retryCount;
348
349 public PutStatus(HRegionInfo regionInfo, Put put, int retryCount) {
350 this.regionInfo = regionInfo;
351 this.put = put;
352 this.retryCount = retryCount;
353 }
354 }
355
356
357
358
359 private static class AtomicAverageCounter {
360 private long sum;
361 private int count;
362
363 public AtomicAverageCounter() {
364 this.sum = 0L;
365 this.count = 0;
366 }
367
368 public synchronized long getAndReset() {
369 long result = this.get();
370 this.reset();
371 return result;
372 }
373
374 public synchronized long get() {
375 if (this.count == 0) {
376 return 0;
377 }
378 return this.sum / this.count;
379 }
380
381 public synchronized SimpleEntry<Long, Integer> getComponents() {
382 return new SimpleEntry<Long, Integer>(sum, count);
383 }
384
385 public synchronized void reset() {
386 this.sum = 0l;
387 this.count = 0;
388 }
389
390 public synchronized void add(long value) {
391 this.sum += value;
392 this.count++;
393 }
394 }
395
396 private static class FlushWorker implements Runnable, AsyncProcessCallback<Object> {
397 private final HRegionLocation addr;
398 private final LinkedBlockingQueue<PutStatus> queue;
399 private final HTableMultiplexer multiplexer;
400 private final AtomicLong totalFailedPutCount = new AtomicLong(0);
401 private final AtomicInteger currentProcessingCount = new AtomicInteger(0);
402 private final AtomicAverageCounter averageLatency = new AtomicAverageCounter();
403 private final AtomicLong maxLatency = new AtomicLong(0);
404
405 private final AsyncProcess<Object> ap;
406 private final List<Object> results = new ArrayList<Object>();
407 private final List<PutStatus> processingList = new ArrayList<PutStatus>();
408 private final ScheduledExecutorService executor;
409 private final int maxRetryInQueue;
410 private final AtomicInteger retryInQueue = new AtomicInteger(0);
411
412 public FlushWorker(Configuration conf, HConnection conn, HRegionLocation addr,
413 HTableMultiplexer multiplexer, int perRegionServerBufferQueueSize,
414 ExecutorService pool, ScheduledExecutorService executor) {
415 this.addr = addr;
416 this.multiplexer = multiplexer;
417 this.queue = new LinkedBlockingQueue<PutStatus>(perRegionServerBufferQueueSize);
418 RpcRetryingCallerFactory rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf,
419 conn.getStatisticsTracker());
420 RpcControllerFactory rpcControllerFactory = RpcControllerFactory.instantiate(conf);
421 this.ap = new AsyncProcess<Object>(conn, null, pool, this, conf, rpcCallerFactory,
422 rpcControllerFactory);
423 this.executor = executor;
424 this.maxRetryInQueue = conf.getInt(TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE, 10000);
425 }
426
427 protected LinkedBlockingQueue<PutStatus> getQueue() {
428 return this.queue;
429 }
430
431 public long getTotalFailedCount() {
432 return totalFailedPutCount.get();
433 }
434
435 public long getTotalBufferedCount() {
436 return queue.size() + currentProcessingCount.get();
437 }
438
439 public AtomicAverageCounter getAverageLatencyCounter() {
440 return this.averageLatency;
441 }
442
443 public long getMaxLatency() {
444 return this.maxLatency.getAndSet(0);
445 }
446
447 private boolean resubmitFailedPut(PutStatus ps, HRegionLocation oldLoc) throws IOException {
448
449 final int retryCount = ps.retryCount - 1;
450
451 if (retryCount <= 0) {
452
453 return false;
454 }
455
456 int cnt = retryInQueue.incrementAndGet();
457 if (cnt > maxRetryInQueue) {
458
459 retryInQueue.decrementAndGet();
460 return false;
461 }
462
463 final Put failedPut = ps.put;
464
465 final TableName tableName = ps.regionInfo.getTable();
466
467 ap.hConnection.clearCaches(oldLoc.getServerName());
468
469 long delayMs = ConnectionUtils.getPauseTime(multiplexer.flushPeriod,
470 multiplexer.retryNum - retryCount - 1);
471 if (LOG.isDebugEnabled()) {
472 LOG.debug("resubmitting after " + delayMs + "ms: " + retryCount);
473 }
474
475 executor.schedule(new Runnable() {
476 @Override
477 public void run() {
478 boolean succ = false;
479 try {
480 succ = FlushWorker.this.multiplexer.put(tableName, failedPut, retryCount);
481 } finally {
482 FlushWorker.this.retryInQueue.decrementAndGet();
483 if (!succ) {
484 FlushWorker.this.totalFailedPutCount.incrementAndGet();
485 }
486 }
487 }
488 }, delayMs, TimeUnit.MILLISECONDS);
489 return true;
490 }
491
492 @Override
493 public void run() {
494 int failedCount = 0;
495 try {
496 long start = EnvironmentEdgeManager.currentTimeMillis();
497
498
499 processingList.clear();
500 failedCount = 0;
501
502
503 queue.drainTo(processingList);
504 currentProcessingCount.set(processingList.size());
505
506 if (processingList.size() > 0) {
507 this.results.clear();
508 List<Action<Row>> retainedActions = new ArrayList<Action<Row>>(processingList.size());
509 MultiAction<Row> actions = new MultiAction<Row>();
510 for (int i = 0; i < processingList.size(); i++) {
511 PutStatus putStatus = processingList.get(i);
512 Action<Row> action = new Action<Row>(putStatus.put, i);
513 actions.add(putStatus.regionInfo.getRegionName(), action);
514 retainedActions.add(action);
515 this.results.add(null);
516 }
517
518
519 List<PutStatus> failed = null;
520 Map<HRegionLocation, MultiAction<Row>> actionsByServer =
521 Collections.singletonMap(addr, actions);
522 try {
523 HConnectionManager.ServerErrorTracker errorsByServer =
524 new HConnectionManager.ServerErrorTracker(1, 10);
525 ap.sendMultiAction(retainedActions, actionsByServer, 10, errorsByServer, null);
526 ap.waitUntilDone();
527
528 if (ap.hasError()) {
529
530 LOG.debug("Caught some exceptions when flushing puts to region server "
531 + addr.getHostnamePort(), ap.getErrors());
532 }
533 } finally {
534
535
536
537
538
539 for (int i = 0; i < results.size(); i++) {
540 if (results.get(i) instanceof Result) {
541 failedCount--;
542 } else {
543 if (failed == null) {
544 failed = new ArrayList<PutStatus>();
545 }
546 failed.add(processingList.get(i));
547 }
548 }
549 }
550
551 if (failed != null) {
552
553 for (PutStatus putStatus : failed) {
554 if (resubmitFailedPut(putStatus, this.addr)) {
555 failedCount--;
556 }
557 }
558
559 this.totalFailedPutCount.addAndGet(failedCount);
560 }
561
562 long elapsed = EnvironmentEdgeManager.currentTimeMillis() - start;
563
564 averageLatency.add(elapsed);
565 if (elapsed > maxLatency.get()) {
566 maxLatency.set(elapsed);
567 }
568
569
570 if (LOG.isDebugEnabled()) {
571 LOG.debug("Processed " + currentProcessingCount + " put requests for "
572 + addr.getHostnamePort() + " and " + failedCount + " failed"
573 + ", latency for this send: " + elapsed);
574 }
575
576
577 currentProcessingCount.set(0);
578 }
579 } catch (RuntimeException e) {
580
581
582 LOG.debug("Caught some exceptions " + e + " when flushing puts to region server "
583 + addr.getHostnamePort(), e);
584 } catch (Exception e) {
585 if (e instanceof InterruptedException) {
586 Thread.currentThread().interrupt();
587 }
588
589 LOG.debug("Caught some exceptions " + e + " when flushing puts to region server "
590 + addr.getHostnamePort(), e);
591 } finally {
592
593 this.totalFailedPutCount.addAndGet(failedCount);
594 }
595 }
596
597 @Override
598 public void success(int originalIndex, byte[] region, Row row, Object result) {
599 if (results == null || originalIndex >= results.size()) {
600 return;
601 }
602 results.set(originalIndex, result);
603 }
604
605 @Override
606 public boolean failure(int originalIndex, byte[] region, Row row, Throwable t) {
607 return false;
608 }
609
610 @Override
611 public boolean retriableFailure(int originalIndex, Row row, byte[] region, Throwable exception) {
612 return false;
613 }
614 }
615 }