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.ServerName;
46  import org.apache.hadoop.hbase.TableName;
47  import org.apache.hadoop.hbase.classification.InterfaceAudience;
48  import org.apache.hadoop.hbase.classification.InterfaceStability;
49  import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
50  import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
51  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
52  
53  import com.google.common.util.concurrent.ThreadFactoryBuilder;
54  
55  /**
56   * HTableMultiplexer provides a thread-safe non blocking PUT API across all the tables.
57   * Each put will be sharded into different buffer queues based on its destination region server.
58   * So each region server buffer queue will only have the puts which share the same destination.
59   * And each queue will have a flush worker thread to flush the puts request to the region server.
60   * If any queue is full, the HTableMultiplexer starts to drop the Put requests for that 
61   * particular queue.
62   * 
63   * Also all the puts will be retried as a configuration number before dropping.
64   * And the HTableMultiplexer can report the number of buffered requests and the number of the
65   * failed (dropped) requests in total or on per region server basis.
66   * 
67   * This class is thread safe.
68   */
69  @InterfaceAudience.Public
70  @InterfaceStability.Evolving
71  public class HTableMultiplexer {
72    private static final Log LOG = LogFactory.getLog(HTableMultiplexer.class.getName());
73    
74    public static final String TABLE_MULTIPLEXER_FLUSH_PERIOD_MS =
75        "hbase.tablemultiplexer.flush.period.ms";
76    public static final String TABLE_MULTIPLEXER_INIT_THREADS = "hbase.tablemultiplexer.init.threads";
77    public static final String TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE =
78        "hbase.client.max.retries.in.queue";
79  
80    /** The map between each region server to its flush worker */
81    private final Map<HRegionLocation, FlushWorker> serverToFlushWorkerMap =
82        new ConcurrentHashMap<>();
83  
84    private final Configuration workerConf;
85    private final ClusterConnection conn;
86    private final ExecutorService pool;
87    private final int retryNum;
88    private final int perRegionServerBufferQueueSize;
89    private final int maxKeyValueSize;
90    private final ScheduledExecutorService executor;
91    private final long flushPeriod;
92    
93    /**
94     * @param conf The HBaseConfiguration
95     * @param perRegionServerBufferQueueSize determines the max number of the buffered Put ops for
96     *          each region server before dropping the request.
97     */
98    public HTableMultiplexer(Configuration conf, int perRegionServerBufferQueueSize)
99        throws IOException {
100     this.conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
101     this.pool = HTable.getDefaultExecutor(conf);
102     this.retryNum = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
103         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
104     this.perRegionServerBufferQueueSize = perRegionServerBufferQueueSize;
105     this.maxKeyValueSize = HTable.getMaxKeyValueSize(conf);
106     this.flushPeriod = conf.getLong(TABLE_MULTIPLEXER_FLUSH_PERIOD_MS, 100);
107     int initThreads = conf.getInt(TABLE_MULTIPLEXER_INIT_THREADS, 10);
108     this.executor =
109         Executors.newScheduledThreadPool(initThreads,
110           new ThreadFactoryBuilder().setDaemon(true).setNameFormat("HTableFlushWorker-%d").build());
111 
112     this.workerConf = HBaseConfiguration.create(conf);
113     // We do not do the retry because we need to reassign puts to different queues if regions are
114     // moved.
115     this.workerConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0);
116   }
117 
118   /**
119    * The put request will be buffered by its corresponding buffer queue. Return false if the queue
120    * is already full.
121    * @param tableName
122    * @param put
123    * @return true if the request can be accepted by its corresponding buffer queue.
124    * @throws IOException
125    */
126   public boolean put(TableName tableName, final Put put) {
127     return put(tableName, put, this.retryNum);
128   }
129 
130   /**
131    * The puts request will be buffered by their corresponding buffer queue. 
132    * Return the list of puts which could not be queued.
133    * @param tableName
134    * @param puts
135    * @return the list of puts which could not be queued
136    * @throws IOException
137    */
138   public List<Put> put(TableName tableName, final List<Put> puts) {
139     if (puts == null)
140       return null;
141     
142     List <Put> failedPuts = null;
143     boolean result;
144     for (Put put : puts) {
145       result = put(tableName, put, this.retryNum);
146       if (result == false) {
147         
148         // Create the failed puts list if necessary
149         if (failedPuts == null) {
150           failedPuts = new ArrayList<Put>();
151         }
152         // Add the put to the failed puts list
153         failedPuts.add(put);
154       }
155     }
156     return failedPuts;
157   }
158 
159   /**
160    * Deprecated. Use {@link #put(TableName, List) } instead.
161    */
162   @Deprecated
163   public List<Put> put(byte[] tableName, final List<Put> puts) {
164     return put(TableName.valueOf(tableName), puts);
165   }
166   
167   /**
168    * The put request will be buffered by its corresponding buffer queue. And the put request will be
169    * retried before dropping the request.
170    * Return false if the queue is already full.
171    * @return true if the request can be accepted by its corresponding buffer queue.
172    * @throws IOException
173    */
174   public boolean put(final TableName tableName, final Put put, int retry) {
175     if (retry <= 0) {
176       return false;
177     }
178 
179     try {
180       HTable.validatePut(put, maxKeyValueSize);
181       HRegionLocation loc = conn.getRegionLocation(tableName, put.getRow(), false);
182       if (loc != null) {
183         // Add the put pair into its corresponding queue.
184         LinkedBlockingQueue<PutStatus> queue = getQueue(loc);
185 
186         // Generate a MultiPutStatus object and offer it into the queue
187         PutStatus s = new PutStatus(loc.getRegionInfo(), put, retry);
188         
189         return queue.offer(s);
190       }
191     } catch (IOException e) {
192       LOG.debug("Cannot process the put " + put, e);
193     }
194     return false;
195   }
196 
197   /**
198    * Deprecated. Use {@link #put(TableName, Put) } instead.
199    */
200   @Deprecated
201   public boolean put(final byte[] tableName, final Put put, int retry) {
202     return put(TableName.valueOf(tableName), put, retry);
203   }
204 
205   /**
206    * Deprecated. Use {@link #put(TableName, Put)} instead.
207    */
208   @Deprecated
209   public boolean put(final byte[] tableName, Put put) {
210     return put(TableName.valueOf(tableName), put);
211   }
212   
213   /**
214    * @return the current HTableMultiplexerStatus
215    */
216   public HTableMultiplexerStatus getHTableMultiplexerStatus() {
217     return new HTableMultiplexerStatus(serverToFlushWorkerMap);
218   }
219 
220   private LinkedBlockingQueue<PutStatus> getQueue(HRegionLocation addr) {
221     FlushWorker worker = serverToFlushWorkerMap.get(addr);
222     if (worker == null) {
223       synchronized (this.serverToFlushWorkerMap) {
224         worker = serverToFlushWorkerMap.get(addr);
225         if (worker == null) {
226           // Create the flush worker
227           worker = new FlushWorker(workerConf, this.conn, addr, this, perRegionServerBufferQueueSize,
228                   pool, executor);
229           this.serverToFlushWorkerMap.put(addr, worker);
230           executor.scheduleAtFixedRate(worker, flushPeriod, flushPeriod, TimeUnit.MILLISECONDS);
231         }
232       }
233     }
234     return worker.getQueue();
235   }
236 
237   /**
238    * HTableMultiplexerStatus keeps track of the current status of the HTableMultiplexer.
239    * report the number of buffered requests and the number of the failed (dropped) requests
240    * in total or on per region server basis.
241    */
242   public static class HTableMultiplexerStatus {
243     private long totalFailedPutCounter;
244     private long totalBufferedPutCounter;
245     private long maxLatency;
246     private long overallAverageLatency;
247     private Map<String, Long> serverToFailedCounterMap;
248     private Map<String, Long> serverToBufferedCounterMap;
249     private Map<String, Long> serverToAverageLatencyMap;
250     private Map<String, Long> serverToMaxLatencyMap;
251 
252     public HTableMultiplexerStatus(
253         Map<HRegionLocation, FlushWorker> serverToFlushWorkerMap) {
254       this.totalBufferedPutCounter = 0;
255       this.totalFailedPutCounter = 0;
256       this.maxLatency = 0;
257       this.overallAverageLatency = 0;
258       this.serverToBufferedCounterMap = new HashMap<String, Long>();
259       this.serverToFailedCounterMap = new HashMap<String, Long>();
260       this.serverToAverageLatencyMap = new HashMap<String, Long>();
261       this.serverToMaxLatencyMap = new HashMap<String, Long>();
262       this.initialize(serverToFlushWorkerMap);
263     }
264 
265     private void initialize(
266         Map<HRegionLocation, FlushWorker> serverToFlushWorkerMap) {
267       if (serverToFlushWorkerMap == null) {
268         return;
269       }
270 
271       long averageCalcSum = 0;
272       int averageCalcCount = 0;
273       for (Map.Entry<HRegionLocation, FlushWorker> entry : serverToFlushWorkerMap
274           .entrySet()) {
275         HRegionLocation addr = entry.getKey();
276         FlushWorker worker = entry.getValue();
277 
278         long bufferedCounter = worker.getTotalBufferedCount();
279         long failedCounter = worker.getTotalFailedCount();
280         long serverMaxLatency = worker.getMaxLatency();
281         AtomicAverageCounter averageCounter = worker.getAverageLatencyCounter();
282         // Get sum and count pieces separately to compute overall average
283         SimpleEntry<Long, Integer> averageComponents = averageCounter
284             .getComponents();
285         long serverAvgLatency = averageCounter.getAndReset();
286 
287         this.totalBufferedPutCounter += bufferedCounter;
288         this.totalFailedPutCounter += failedCounter;
289         if (serverMaxLatency > this.maxLatency) {
290           this.maxLatency = serverMaxLatency;
291         }
292         averageCalcSum += averageComponents.getKey();
293         averageCalcCount += averageComponents.getValue();
294 
295         this.serverToBufferedCounterMap.put(addr.getHostnamePort(),
296             bufferedCounter);
297         this.serverToFailedCounterMap
298             .put(addr.getHostnamePort(),
299             failedCounter);
300         this.serverToAverageLatencyMap.put(addr.getHostnamePort(),
301             serverAvgLatency);
302         this.serverToMaxLatencyMap
303             .put(addr.getHostnamePort(),
304             serverMaxLatency);
305       }
306       this.overallAverageLatency = averageCalcCount != 0 ? averageCalcSum
307           / averageCalcCount : 0;
308     }
309 
310     public long getTotalBufferedCounter() {
311       return this.totalBufferedPutCounter;
312     }
313 
314     public long getTotalFailedCounter() {
315       return this.totalFailedPutCounter;
316     }
317 
318     public long getMaxLatency() {
319       return this.maxLatency;
320     }
321 
322     public long getOverallAverageLatency() {
323       return this.overallAverageLatency;
324     }
325 
326     public Map<String, Long> getBufferedCounterForEachRegionServer() {
327       return this.serverToBufferedCounterMap;
328     }
329 
330     public Map<String, Long> getFailedCounterForEachRegionServer() {
331       return this.serverToFailedCounterMap;
332     }
333 
334     public Map<String, Long> getMaxLatencyForEachRegionServer() {
335       return this.serverToMaxLatencyMap;
336     }
337 
338     public Map<String, Long> getAverageLatencyForEachRegionServer() {
339       return this.serverToAverageLatencyMap;
340     }
341   }
342   
343   private static class PutStatus {
344     public final HRegionInfo regionInfo;
345     public final Put put;
346     public final int retryCount;
347 
348     public PutStatus(HRegionInfo regionInfo, Put put, int retryCount) {
349       this.regionInfo = regionInfo;
350       this.put = put;
351       this.retryCount = retryCount;
352     }
353   }
354 
355   /**
356    * Helper to count the average over an interval until reset.
357    */
358   private static class AtomicAverageCounter {
359     private long sum;
360     private int count;
361 
362     public AtomicAverageCounter() {
363       this.sum = 0L;
364       this.count = 0;
365     }
366 
367     public synchronized long getAndReset() {
368       long result = this.get();
369       this.reset();
370       return result;
371     }
372 
373     public synchronized long get() {
374       if (this.count == 0) {
375         return 0;
376       }
377       return this.sum / this.count;
378     }
379 
380     public synchronized SimpleEntry<Long, Integer> getComponents() {
381       return new SimpleEntry<Long, Integer>(sum, count);
382     }
383 
384     public synchronized void reset() {
385       this.sum = 0l;
386       this.count = 0;
387     }
388 
389     public synchronized void add(long value) {
390       this.sum += value;
391       this.count++;
392     }
393   }
394 
395   private static class FlushWorker implements Runnable {
396     private final HRegionLocation addr;
397     private final LinkedBlockingQueue<PutStatus> queue;
398     private final HTableMultiplexer multiplexer;
399     private final AtomicLong totalFailedPutCount = new AtomicLong(0);
400     private final AtomicInteger currentProcessingCount = new AtomicInteger(0);
401     private final AtomicAverageCounter averageLatency = new AtomicAverageCounter();
402     private final AtomicLong maxLatency = new AtomicLong(0);
403 
404     private final AsyncProcess ap;
405     private final List<PutStatus> processingList = new ArrayList<>();
406     private final ScheduledExecutorService executor;
407     private final int maxRetryInQueue;
408     private final AtomicInteger retryInQueue = new AtomicInteger(0);
409     
410     public FlushWorker(Configuration conf, ClusterConnection conn, HRegionLocation addr,
411         HTableMultiplexer htableMultiplexer, int perRegionServerBufferQueueSize,
412         ExecutorService pool, ScheduledExecutorService executor) {
413       this.addr = addr;
414       this.multiplexer = htableMultiplexer;
415       this.queue = new LinkedBlockingQueue<>(perRegionServerBufferQueueSize);
416       RpcRetryingCallerFactory rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf);
417       RpcControllerFactory rpcControllerFactory = RpcControllerFactory.instantiate(conf);
418       this.ap = new AsyncProcess(conn, conf, pool, rpcCallerFactory, false, rpcControllerFactory);
419       this.executor = executor;
420       this.maxRetryInQueue = conf.getInt(TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE, 10000);
421     }
422 
423     protected LinkedBlockingQueue<PutStatus> getQueue() {
424       return this.queue;
425     }
426 
427     public long getTotalFailedCount() {
428       return totalFailedPutCount.get();
429     }
430 
431     public long getTotalBufferedCount() {
432       return queue.size() + currentProcessingCount.get();
433     }
434 
435     public AtomicAverageCounter getAverageLatencyCounter() {
436       return this.averageLatency;
437     }
438 
439     public long getMaxLatency() {
440       return this.maxLatency.getAndSet(0);
441     }
442 
443     private boolean resubmitFailedPut(PutStatus ps, HRegionLocation oldLoc) throws IOException {
444       // Decrease the retry count
445       final int retryCount = ps.retryCount - 1;
446       
447       if (retryCount <= 0) {
448         // Update the failed counter and no retry any more.
449         return false;
450       }
451 
452       int cnt = retryInQueue.incrementAndGet();
453       if (cnt > maxRetryInQueue) {
454         // Too many Puts in queue for resubmit, give up this
455         retryInQueue.decrementAndGet();
456         return false;
457       }
458 
459       final Put failedPut = ps.put;
460       // The currentPut is failed. So get the table name for the currentPut.
461       final TableName tableName = ps.regionInfo.getTable();
462 
463       long delayMs = ConnectionUtils.getPauseTime(multiplexer.flushPeriod,
464         multiplexer.retryNum - retryCount - 1);
465       if (LOG.isDebugEnabled()) {
466         LOG.debug("resubmitting after " + delayMs + "ms: " + retryCount);
467       }
468 
469       executor.schedule(new Runnable() {
470         @Override
471         public void run() {
472           boolean succ = false;
473           try {
474             succ = FlushWorker.this.multiplexer.put(tableName, failedPut, retryCount);
475           } finally {
476             FlushWorker.this.retryInQueue.decrementAndGet();
477             if (!succ) {
478               FlushWorker.this.totalFailedPutCount.incrementAndGet();
479             }
480           }
481         }
482       }, delayMs, TimeUnit.MILLISECONDS);
483       return true;
484     }
485 
486     @Override
487     public void run() {
488       int failedCount = 0;
489       try {
490         long start = EnvironmentEdgeManager.currentTime();
491 
492         // drain all the queued puts into the tmp list
493         processingList.clear();
494         queue.drainTo(processingList);
495         if (processingList.size() == 0) {
496           // Nothing to flush
497           return;
498         }
499 
500         currentProcessingCount.set(processingList.size());
501         // failedCount is decreased whenever a Put is success or resubmit.
502         failedCount = processingList.size();
503 
504         List<Action<Row>> retainedActions = new ArrayList<>(processingList.size());
505         MultiAction<Row> actions = new MultiAction<>();
506         for (int i = 0; i < processingList.size(); i++) {
507           PutStatus putStatus = processingList.get(i);
508           Action<Row> action = new Action<Row>(putStatus.put, i);
509           actions.add(putStatus.regionInfo.getRegionName(), action);
510           retainedActions.add(action);
511         }
512 
513         // Process this multi-put request
514         List<PutStatus> failed = null;
515         Object[] results = new Object[actions.size()];
516         ServerName server = addr.getServerName();
517         Map<ServerName, MultiAction<Row>> actionsByServer =
518             Collections.singletonMap(server, actions);
519         try {
520           AsyncRequestFuture arf =
521               ap.submitMultiActions(null, retainedActions, 0L, null, results, true, null,
522                 null, actionsByServer, null);
523           arf.waitUntilDone();
524           if (arf.hasError()) {
525             // We just log and ignore the exception here since failed Puts will be resubmit again.
526             LOG.debug("Caught some exceptions when flushing puts to region server "
527                 + addr.getHostnamePort(), arf.getErrors());
528           }
529         } finally {
530           for (int i = 0; i < results.length; i++) {
531             if (results[i] instanceof Result) {
532               failedCount--;
533             } else {
534               if (failed == null) {
535                 failed = new ArrayList<PutStatus>();
536               }
537               failed.add(processingList.get(i));
538             }
539           }
540         }
541 
542         if (failed != null) {
543           // Resubmit failed puts
544           for (PutStatus putStatus : failed) {
545             if (resubmitFailedPut(putStatus, this.addr)) {
546               failedCount--;
547             }
548           }
549         }
550 
551         long elapsed = EnvironmentEdgeManager.currentTime() - start;
552         // Update latency counters
553         averageLatency.add(elapsed);
554         if (elapsed > maxLatency.get()) {
555           maxLatency.set(elapsed);
556         }
557 
558         // Log some basic info
559         if (LOG.isDebugEnabled()) {
560           LOG.debug("Processed " + currentProcessingCount + " put requests for "
561               + addr.getHostnamePort() + " and " + failedCount + " failed"
562               + ", latency for this send: " + elapsed);
563         }
564 
565         // Reset the current processing put count
566         currentProcessingCount.set(0);
567       } catch (RuntimeException e) {
568         // To make findbugs happy
569         // Log all the exceptions and move on
570         LOG.debug(
571           "Caught some exceptions " + e + " when flushing puts to region server "
572               + addr.getHostnamePort(), e);
573       } catch (Exception e) {
574         if (e instanceof InterruptedException) {
575           Thread.currentThread().interrupt();
576         }
577         // Log all the exceptions and move on
578         LOG.debug(
579           "Caught some exceptions " + e + " when flushing puts to region server "
580               + addr.getHostnamePort(), e);
581       } finally {
582         // Update the totalFailedCount
583         this.totalFailedPutCount.addAndGet(failedCount);
584       }
585     }
586   }
587 }