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