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    */
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    */
136   public List<Put> put(TableName tableName, final List<Put> puts) {
137     if (puts == null)
138       return null;
139 
140     List <Put> failedPuts = null;
141     boolean result;
142     for (Put put : puts) {
143       result = put(tableName, put, this.retryNum);
144       if (result == false) {
145 
146         // Create the failed puts list if necessary
147         if (failedPuts == null) {
148           failedPuts = new ArrayList<Put>();
149         }
150         // Add the put to the failed puts list
151         failedPuts.add(put);
152       }
153     }
154     return failedPuts;
155   }
156 
157   /**
158    * @deprecated Use {@link #put(TableName, List) } instead.
159    */
160   @Deprecated
161   public List<Put> put(byte[] tableName, final List<Put> puts) {
162     return put(TableName.valueOf(tableName), puts);
163   }
164 
165   /**
166    * The put request will be buffered by its corresponding buffer queue. And the put request will be
167    * retried before dropping the request.
168    * Return false if the queue is already full.
169    * @return true if the request can be accepted by its corresponding buffer queue.
170    */
171   public boolean put(final TableName tableName, final Put put, int retry) {
172     if (retry <= 0) {
173       return false;
174     }
175 
176     try {
177       HTable.validatePut(put, maxKeyValueSize);
178       HRegionLocation loc = conn.getRegionLocation(tableName, put.getRow(), false);
179       if (loc != null) {
180         // Add the put pair into its corresponding queue.
181         LinkedBlockingQueue<PutStatus> queue = getQueue(loc);
182 
183         // Generate a MultiPutStatus object and offer it into the queue
184         PutStatus s = new PutStatus(loc.getRegionInfo(), put, retry);
185 
186         return queue.offer(s);
187       }
188     } catch (IOException e) {
189       LOG.debug("Cannot process the put " + put, e);
190     }
191     return false;
192   }
193 
194   /**
195    * @deprecated Use {@link #put(TableName, Put) } instead.
196    */
197   @Deprecated
198   public boolean put(final byte[] tableName, final Put put, int retry) {
199     return put(TableName.valueOf(tableName), put, retry);
200   }
201 
202   /**
203    * @deprecated Use {@link #put(TableName, Put)} instead.
204    */
205   @Deprecated
206   public boolean put(final byte[] tableName, Put put) {
207     return put(TableName.valueOf(tableName), put);
208   }
209 
210   /**
211    * @return the current HTableMultiplexerStatus
212    */
213   public HTableMultiplexerStatus getHTableMultiplexerStatus() {
214     return new HTableMultiplexerStatus(serverToFlushWorkerMap);
215   }
216 
217   private LinkedBlockingQueue<PutStatus> getQueue(HRegionLocation addr) {
218     FlushWorker worker = serverToFlushWorkerMap.get(addr);
219     if (worker == null) {
220       synchronized (this.serverToFlushWorkerMap) {
221         worker = serverToFlushWorkerMap.get(addr);
222         if (worker == null) {
223           // Create the flush worker
224           worker = new FlushWorker(workerConf, this.conn, addr, this,
225               perRegionServerBufferQueueSize, pool, executor);
226           this.serverToFlushWorkerMap.put(addr, worker);
227           executor.scheduleAtFixedRate(worker, flushPeriod, flushPeriod, TimeUnit.MILLISECONDS);
228         }
229       }
230     }
231     return worker.getQueue();
232   }
233 
234   /**
235    * HTableMultiplexerStatus keeps track of the current status of the HTableMultiplexer.
236    * report the number of buffered requests and the number of the failed (dropped) requests
237    * in total or on per region server basis.
238    */
239   @InterfaceAudience.Public
240   @InterfaceStability.Evolving
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     private final HRegionInfo regionInfo;
344     private final Put put;
345     private 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 {
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 ap;
404     private final List<PutStatus> processingList = new ArrayList<>();
405     private final ScheduledExecutorService executor;
406     private final int maxRetryInQueue;
407     private final AtomicInteger retryInQueue = new AtomicInteger(0);
408 
409     public FlushWorker(Configuration conf, ClusterConnection conn, HRegionLocation addr,
410         HTableMultiplexer htableMultiplexer, int perRegionServerBufferQueueSize,
411         ExecutorService pool, ScheduledExecutorService executor) {
412       this.addr = addr;
413       this.multiplexer = htableMultiplexer;
414       this.queue = new LinkedBlockingQueue<>(perRegionServerBufferQueueSize);
415       RpcRetryingCallerFactory rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf);
416       RpcControllerFactory rpcControllerFactory = RpcControllerFactory.instantiate(conf);
417       this.ap = new AsyncProcess(conn, conf, pool, rpcCallerFactory, false, rpcControllerFactory);
418       this.executor = executor;
419       this.maxRetryInQueue = conf.getInt(TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE, 10000);
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       // Decrease the retry count
444       final int retryCount = ps.retryCount - 1;
445 
446       if (retryCount <= 0) {
447         // Update the failed counter and no retry any more.
448         return false;
449       }
450 
451       int cnt = retryInQueue.incrementAndGet();
452       if (cnt > maxRetryInQueue) {
453         // Too many Puts in queue for resubmit, give up this
454         retryInQueue.decrementAndGet();
455         return false;
456       }
457 
458       final Put failedPut = ps.put;
459       // The currentPut is failed. So get the table name for the currentPut.
460       final TableName tableName = ps.regionInfo.getTable();
461 
462       long delayMs = ConnectionUtils.getPauseTime(multiplexer.flushPeriod,
463         multiplexer.retryNum - retryCount - 1);
464       if (LOG.isDebugEnabled()) {
465         LOG.debug("resubmitting after " + delayMs + "ms: " + retryCount);
466       }
467 
468       executor.schedule(new Runnable() {
469         @Override
470         public void run() {
471           boolean succ = false;
472           try {
473             succ = FlushWorker.this.multiplexer.put(tableName, failedPut, retryCount);
474           } finally {
475             FlushWorker.this.retryInQueue.decrementAndGet();
476             if (!succ) {
477               FlushWorker.this.totalFailedPutCount.incrementAndGet();
478             }
479           }
480         }
481       }, delayMs, TimeUnit.MILLISECONDS);
482       return true;
483     }
484 
485     @Override
486     public void run() {
487       int failedCount = 0;
488       try {
489         long start = EnvironmentEdgeManager.currentTime();
490 
491         // drain all the queued puts into the tmp list
492         processingList.clear();
493         queue.drainTo(processingList);
494         if (processingList.size() == 0) {
495           // Nothing to flush
496           return;
497         }
498 
499         currentProcessingCount.set(processingList.size());
500         // failedCount is decreased whenever a Put is success or resubmit.
501         failedCount = processingList.size();
502 
503         List<Action<Row>> retainedActions = new ArrayList<>(processingList.size());
504         MultiAction<Row> actions = new MultiAction<>();
505         for (int i = 0; i < processingList.size(); i++) {
506           PutStatus putStatus = processingList.get(i);
507           Action<Row> action = new Action<Row>(putStatus.put, i);
508           actions.add(putStatus.regionInfo.getRegionName(), action);
509           retainedActions.add(action);
510         }
511 
512         // Process this multi-put request
513         List<PutStatus> failed = null;
514         Object[] results = new Object[actions.size()];
515         ServerName server = addr.getServerName();
516         Map<ServerName, MultiAction<Row>> actionsByServer =
517             Collections.singletonMap(server, actions);
518         try {
519           AsyncRequestFuture arf =
520               ap.submitMultiActions(null, retainedActions, 0L, null, results, true, null,
521                 null, actionsByServer, null);
522           arf.waitUntilDone();
523           if (arf.hasError()) {
524             // We just log and ignore the exception here since failed Puts will be resubmit again.
525             LOG.debug("Caught some exceptions when flushing puts to region server "
526                 + addr.getHostnamePort(), arf.getErrors());
527           }
528         } finally {
529           for (int i = 0; i < results.length; i++) {
530             if (results[i] instanceof Result) {
531               failedCount--;
532             } else {
533               if (failed == null) {
534                 failed = new ArrayList<PutStatus>();
535               }
536               failed.add(processingList.get(i));
537             }
538           }
539         }
540 
541         if (failed != null) {
542           // Resubmit failed puts
543           for (PutStatus putStatus : failed) {
544             if (resubmitFailedPut(putStatus, this.addr)) {
545               failedCount--;
546             }
547           }
548         }
549 
550         long elapsed = EnvironmentEdgeManager.currentTime() - start;
551         // Update latency counters
552         averageLatency.add(elapsed);
553         if (elapsed > maxLatency.get()) {
554           maxLatency.set(elapsed);
555         }
556 
557         // Log some basic info
558         if (LOG.isDebugEnabled()) {
559           LOG.debug("Processed " + currentProcessingCount + " put requests for "
560               + addr.getHostnamePort() + " and " + failedCount + " failed"
561               + ", latency for this send: " + elapsed);
562         }
563 
564         // Reset the current processing put count
565         currentProcessingCount.set(0);
566       } catch (RuntimeException e) {
567         // To make findbugs happy
568         // Log all the exceptions and move on
569         LOG.debug(
570           "Caught some exceptions " + e + " when flushing puts to region server "
571               + addr.getHostnamePort(), e);
572       } catch (Exception e) {
573         if (e instanceof InterruptedException) {
574           Thread.currentThread().interrupt();
575         }
576         // Log all the exceptions and move on
577         LOG.debug(
578           "Caught some exceptions " + e + " when flushing puts to region server "
579               + addr.getHostnamePort(), e);
580       } finally {
581         // Update the totalFailedCount
582         this.totalFailedPutCount.addAndGet(failedCount);
583       }
584     }
585   }
586 }