View Javadoc

1   /**
2    * Copyright The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
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   * HTableMultiplexer provides a thread-safe non blocking PUT API across all the tables.
56   * Each put will be sharded into different buffer queues based on its destination region server.
57   * So each region server buffer queue will only have the puts which share the same destination.
58   * And each queue will have a flush worker thread to flush the puts request to the region server.
59   * If any queue is full, the HTableMultiplexer starts to drop the Put requests for that 
60   * particular queue.
61   * 
62   * Also all the puts will be retried as a configuration number before dropping.
63   * And the HTableMultiplexer can report the number of buffered requests and the number of the
64   * failed (dropped) requests in total or on per region server basis.
65   * 
66   * This class is thread safe.
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    /** The map between each region server to its flush worker */
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     * @param conf The HBaseConfiguration
94     * @param perRegionServerBufferQueueSize determines the max number of the buffered Put ops for
95     *          each region server before dropping the request.
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     // We do not do the retry because we need to reassign puts to different queues if regions are
113     // moved.
114     this.workerConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0);
115   }
116 
117   /**
118    * The put request will be buffered by its corresponding buffer queue. Return false if the queue
119    * is already full.
120    * @param tableName
121    * @param put
122    * @return true if the request can be accepted by its corresponding buffer queue.
123    * @throws IOException
124    */
125   public boolean put(TableName tableName, final Put put) {
126     return put(tableName, put, this.retryNum);
127   }
128 
129   /**
130    * The puts request will be buffered by their corresponding buffer queue. 
131    * Return the list of puts which could not be queued.
132    * @param tableName
133    * @param puts
134    * @return the list of puts which could not be queued
135    * @throws IOException
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         // Create the failed puts list if necessary
148         if (failedPuts == null) {
149           failedPuts = new ArrayList<Put>();
150         }
151         // Add the put to the failed puts list
152         failedPuts.add(put);
153       }
154     }
155     return failedPuts;
156   }
157 
158   /**
159    * Deprecated. Use {@link #put(TableName, List) } instead.
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    * The put request will be buffered by its corresponding buffer queue. And the put request will be
168    * retried before dropping the request.
169    * Return false if the queue is already full.
170    * @return true if the request can be accepted by its corresponding buffer queue.
171    * @throws IOException
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         // Add the put pair into its corresponding queue.
183         LinkedBlockingQueue<PutStatus> queue = getQueue(loc);
184 
185         // Generate a MultiPutStatus object and offer it into the queue
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    * Deprecated. Use {@link #put(TableName, Put) } instead.
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    * Deprecated. Use {@link #put(TableName, Put)} instead.
206    */
207   @Deprecated
208   public boolean put(final byte[] tableName, Put put) {
209     return put(TableName.valueOf(tableName), put);
210   }
211   
212   /**
213    * @return the current HTableMultiplexerStatus
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           // Create the flush worker
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    * HTableMultiplexerStatus keeps track of the current status of the HTableMultiplexer.
238    * report the number of buffered requests and the number of the failed (dropped) requests
239    * in total or on per region server basis.
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         // Get sum and count pieces separately to compute overall average
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    * Helper to count the average over an interval until reset.
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         conn.getStatisticsTracker());
418       RpcControllerFactory rpcControllerFactory = RpcControllerFactory.instantiate(conf);
419       this.ap = new AsyncProcess<Object>(conn, null, pool, this, conf, rpcCallerFactory,
420               rpcControllerFactory);
421       this.executor = executor;
422       this.maxRetryInQueue = conf.getInt(TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE, 10000);
423     }
424 
425     protected LinkedBlockingQueue<PutStatus> getQueue() {
426       return this.queue;
427     }
428 
429     public long getTotalFailedCount() {
430       return totalFailedPutCount.get();
431     }
432 
433     public long getTotalBufferedCount() {
434       return queue.size() + currentProcessingCount.get();
435     }
436 
437     public AtomicAverageCounter getAverageLatencyCounter() {
438       return this.averageLatency;
439     }
440 
441     public long getMaxLatency() {
442       return this.maxLatency.getAndSet(0);
443     }
444 
445     private boolean resubmitFailedPut(PutStatus ps, HRegionLocation oldLoc) throws IOException {
446       // Decrease the retry count
447       final int retryCount = ps.retryCount - 1;
448       
449       if (retryCount <= 0) {
450         // Update the failed counter and no retry any more.
451         return false;
452       }
453 
454       int cnt = retryInQueue.incrementAndGet();
455       if (cnt > maxRetryInQueue) {
456         // Too many Puts in queue for resubmit, give up this
457         retryInQueue.decrementAndGet();
458         return false;
459       }
460 
461       final Put failedPut = ps.put;
462       // The currentPut is failed. So get the table name for the currentPut.
463       final TableName tableName = ps.regionInfo.getTable();
464       // Flush the cache for the failed server
465       ap.hConnection.clearCaches(oldLoc.getServerName());
466 
467       long delayMs = ConnectionUtils.getPauseTime(multiplexer.flushPeriod,
468         multiplexer.retryNum - retryCount - 1);
469       if (LOG.isDebugEnabled()) {
470         LOG.debug("resubmitting after " + delayMs + "ms: " + retryCount);
471       }
472 
473       executor.schedule(new Runnable() {
474         @Override
475         public void run() {
476           boolean succ = false;
477           try {
478             succ = FlushWorker.this.multiplexer.put(tableName, failedPut, retryCount);
479           } finally {
480             FlushWorker.this.retryInQueue.decrementAndGet();
481             if (!succ) {
482               FlushWorker.this.totalFailedPutCount.incrementAndGet();
483             }
484           }
485         }
486       }, delayMs, TimeUnit.MILLISECONDS);
487       return true;
488     }
489 
490     @Override
491     public void run() {
492       int failedCount = 0;
493       try {
494         long start = EnvironmentEdgeManager.currentTimeMillis();
495 
496         // Clear the processingList, putToStatusMap and failedCount
497         processingList.clear();
498         failedCount = 0;
499         
500         // drain all the queued puts into the tmp list
501         queue.drainTo(processingList);
502         currentProcessingCount.set(processingList.size());
503 
504         if (processingList.size() > 0) {
505           this.results.clear();
506           List<Action<Row>> retainedActions = new ArrayList<Action<Row>>(processingList.size());
507           MultiAction<Row> actions = new MultiAction<Row>();
508           for (int i = 0; i < processingList.size(); i++) {
509             PutStatus putStatus = processingList.get(i);
510             Action<Row> action = new Action<Row>(putStatus.put, i);
511             actions.add(putStatus.regionInfo.getRegionName(), action);
512             retainedActions.add(action);
513             this.results.add(null);
514           }
515           
516           // Process this multi-put request
517           List<PutStatus> failed = null;
518           Map<HRegionLocation, MultiAction<Row>> actionsByServer =
519               Collections.singletonMap(addr, actions);
520           try {
521             HConnectionManager.ServerErrorTracker errorsByServer =
522                 new HConnectionManager.ServerErrorTracker(1, 10);
523             ap.sendMultiAction(retainedActions, actionsByServer, 10, errorsByServer, null);
524             ap.waitUntilDone();
525 
526             if (ap.hasError()) {
527               // We just log and ignore the exception here since failed Puts will be resubmit again.
528               LOG.debug("Caught some exceptions when flushing puts to region server "
529                   + addr.getHostnamePort(), ap.getErrors());
530             }
531           } finally {
532             // mutate list so that it is empty for complete success, or
533             // contains only failed records
534             // results are returned in the same order as the requests in list
535             // walk the list backwards, so we can remove from list without
536             // impacting the indexes of earlier members
537             for (int i = 0; i < results.size(); i++) {
538               if (results.get(i) instanceof Result) {
539                 failedCount--;
540               } else {
541                 if (failed == null) {
542                   failed = new ArrayList<PutStatus>();
543                 }
544                 failed.add(processingList.get(i));
545               }
546             }
547           }
548 
549           if (failed != null) {
550             // Resubmit failed puts
551             for (PutStatus putStatus : failed) {
552               if (resubmitFailedPut(putStatus, this.addr)) {
553                 failedCount--;
554               }
555             }
556             // Update the totalFailedCount
557             this.totalFailedPutCount.addAndGet(failedCount);
558           }
559           
560           long elapsed = EnvironmentEdgeManager.currentTimeMillis() - start;
561           // Update latency counters
562           averageLatency.add(elapsed);
563           if (elapsed > maxLatency.get()) {
564             maxLatency.set(elapsed);
565           }
566 
567           // Log some basic info
568           if (LOG.isDebugEnabled()) {
569             LOG.debug("Processed " + currentProcessingCount + " put requests for "
570                 + addr.getHostnamePort() + " and " + failedCount + " failed"
571                 + ", latency for this send: " + elapsed);
572           }
573   
574           // Reset the current processing put count
575           currentProcessingCount.set(0);
576         }
577       } catch (RuntimeException e) {
578         // To make findbugs happy
579         // Log all the exceptions and move on
580         LOG.debug("Caught some exceptions " + e + " when flushing puts to region server "
581               + addr.getHostnamePort(), e);
582       } catch (Exception e) {
583         if (e instanceof InterruptedException) {
584           Thread.currentThread().interrupt();
585         }
586         // Log all the exceptions and move on
587         LOG.debug("Caught some exceptions " + e + " when flushing puts to region server "
588               + addr.getHostnamePort(), e);
589       } finally {
590         // Update the totalFailedCount
591         this.totalFailedPutCount.addAndGet(failedCount);
592       }
593     }
594 
595     @Override
596     public void success(int originalIndex, byte[] region, Row row, Object result) {
597       if (results == null || originalIndex >= results.size()) {
598         return;
599       }
600       results.set(originalIndex, result);
601     }
602 
603     @Override
604     public boolean failure(int originalIndex, byte[] region, Row row, Throwable t) {
605       return false;
606     }
607 
608     @Override
609     public boolean retriableFailure(int originalIndex, Row row, byte[] region, Throwable exception) {
610       return false;
611     }
612   }
613 }