View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.util;
21  
22  import java.io.IOException;
23  import java.io.InterruptedIOException;
24  import java.util.List;
25  import java.util.Random;
26  import java.util.concurrent.ExecutorService;
27  import java.util.concurrent.LinkedBlockingQueue;
28  import java.util.concurrent.ThreadPoolExecutor;
29  import java.util.concurrent.TimeUnit;
30  
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.hbase.classification.InterfaceAudience;
34  import org.apache.hadoop.conf.Configuration;
35  import org.apache.hadoop.hbase.HConstants;
36  import org.apache.hadoop.hbase.TableName;
37  import org.apache.hadoop.hbase.client.HConnection;
38  import org.apache.hadoop.hbase.client.HConnectionManager;
39  import org.apache.hadoop.hbase.client.Row;
40  import org.apache.hadoop.hbase.client.coprocessor.Batch;
41  
42  /**
43   * Provides ability to create multiple HConnection instances and allows to process a batch of
44   * actions using HConnection.processBatchCallback()
45   */
46  @InterfaceAudience.Private
47  public class MultiHConnection {
48    private static final Log LOG = LogFactory.getLog(MultiHConnection.class);
49    private HConnection[] hConnections;
50    private int noOfConnections;
51    private ExecutorService batchPool;
52  
53    /**
54     * Create multiple HConnection instances and initialize a thread pool executor
55     * @param conf configuration
56     * @param noOfConnections total no of HConnections to create
57     * @throws IOException
58     */
59    public MultiHConnection(Configuration conf, int noOfConnections)
60        throws IOException {
61      this.noOfConnections = noOfConnections;
62      hConnections = new HConnection[noOfConnections];
63      for (int i = 0; i < noOfConnections; i++) {
64        HConnection conn = HConnectionManager.createConnection(conf);
65        hConnections[i] = conn;
66      }
67      createBatchPool(conf);
68    }
69  
70    /**
71     * Close the open connections and shutdown the batchpool
72     */
73    public void close() {
74      if (hConnections != null) {
75        synchronized (hConnections) {
76          if (hConnections != null) {
77            for (HConnection conn : hConnections) {
78              if (conn != null) {
79                try {
80                  conn.close();
81                } catch (IOException e) {
82                  LOG.info("Got exception in closing connection", e);
83                } finally {
84                  conn = null;
85                }
86              }
87            }
88            hConnections = null;
89          }
90        }
91      }
92      if (this.batchPool != null && !this.batchPool.isShutdown()) {
93        this.batchPool.shutdown();
94        try {
95          if (!this.batchPool.awaitTermination(10, TimeUnit.SECONDS)) {
96            this.batchPool.shutdownNow();
97          }
98        } catch (InterruptedException e) {
99          this.batchPool.shutdownNow();
100       }
101     }
102 
103   }
104 
105   private static ThreadLocal<Random> threadLocalRandom = new ThreadLocal<Random>() {
106     @Override
107     protected Random initialValue() {
108       return new Random();
109     }
110   };
111 
112   /**
113    * Randomly pick a connection and process the batch of actions for a given table
114    * @param actions the actions
115    * @param tableName table name
116    * @param results the results array
117    * @param callback 
118    * @throws IOException
119    * @throws InterruptedException
120    */
121   @SuppressWarnings("deprecation")
122   public <R> void processBatchCallback(List<? extends Row> actions, TableName tableName,
123       Object[] results, Batch.Callback<R> callback) throws IOException {
124     // Currently used by RegionStateStore
125     // A deprecated method is used as multiple threads accessing RegionStateStore do a single put
126     // and htable is not thread safe. Alternative would be to create an Htable instance for each 
127     // put but that is not very efficient.
128     // See HBASE-11610 for more details.
129     try {
130       hConnections[threadLocalRandom.get().nextInt(noOfConnections)].processBatchCallback(
131         actions, tableName, this.batchPool, results, callback);
132     } catch (InterruptedException e) {
133       throw new InterruptedIOException(e.getMessage());
134     }
135   }
136 
137   
138   // Copied from HConnectionImplementation.getBatchPool()
139   // We should get rid of this when HConnection.processBatchCallback is un-deprecated and provides
140   // an API to manage a batch pool
141   private void createBatchPool(Configuration conf) {
142     // Use the same config for keep alive as in HConnectionImplementation.getBatchPool();
143     int maxThreads = conf.getInt("hbase.multihconnection.threads.max", 256);
144     int coreThreads = conf.getInt("hbase.multihconnection.threads.core", 256);
145     if (maxThreads == 0) {
146       maxThreads = Runtime.getRuntime().availableProcessors() * 8;
147     }
148     if (coreThreads == 0) {
149       coreThreads = Runtime.getRuntime().availableProcessors() * 8;
150     }
151     long keepAliveTime = conf.getLong("hbase.multihconnection.threads.keepalivetime", 60);
152     LinkedBlockingQueue<Runnable> workQueue =
153         new LinkedBlockingQueue<Runnable>(maxThreads
154             * conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
155               HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS));
156     ThreadPoolExecutor tpe =
157         new ThreadPoolExecutor(coreThreads, maxThreads, keepAliveTime, TimeUnit.SECONDS, workQueue,
158             Threads.newDaemonThreadFactory("MultiHConnection" + "-shared-"));
159     tpe.allowCoreThreadTimeOut(true);
160     this.batchPool = tpe;
161   }
162   
163 }