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.concurrent.ExecutorService;
26  import java.util.concurrent.LinkedBlockingQueue;
27  import java.util.concurrent.ThreadLocalRandom;
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.Connection;
38  import org.apache.hadoop.hbase.client.HConnection;
39  import org.apache.hadoop.hbase.client.HConnectionManager;
40  import org.apache.hadoop.hbase.client.Row;
41  import org.apache.hadoop.hbase.client.coprocessor.Batch;
42  
43  /**
44   * Provides ability to create multiple HConnection instances and allows to process a batch of
45   * actions using HConnection.processBatchCallback()
46   */
47  @InterfaceAudience.Private
48  public class MultiHConnection {
49    private static final Log LOG = LogFactory.getLog(MultiHConnection.class);
50    private HConnection[] hConnections;
51    private final Object hConnectionsLock =  new Object();
52    private int noOfConnections;
53    private ExecutorService batchPool;
54  
55    /**
56     * Create multiple HConnection instances and initialize a thread pool executor
57     * @param conf configuration
58     * @param noOfConnections total no of HConnections to create
59     * @throws IOException
60     */
61    public MultiHConnection(Configuration conf, int noOfConnections)
62        throws IOException {
63      this.noOfConnections = noOfConnections;
64      synchronized (this.hConnectionsLock) {
65        hConnections = new HConnection[noOfConnections];
66        for (int i = 0; i < noOfConnections; i++) {
67          HConnection conn = HConnectionManager.createConnection(conf);
68          hConnections[i] = conn;
69        }
70      }
71      createBatchPool(conf);
72    }
73  
74    /**
75     * Close the open connections and shutdown the batchpool
76     */
77    public void close() {
78      synchronized (hConnectionsLock) {
79        if (hConnections != null) {
80          for (Connection conn : hConnections) {
81            if (conn != null) {
82              try {
83                conn.close();
84              } catch (IOException e) {
85                LOG.info("Got exception in closing connection", e);
86              } finally {
87                conn = null;
88              }
89            }
90          }
91          hConnections = null;
92        }
93      }
94      if (this.batchPool != null && !this.batchPool.isShutdown()) {
95        this.batchPool.shutdown();
96        try {
97          if (!this.batchPool.awaitTermination(10, TimeUnit.SECONDS)) {
98            this.batchPool.shutdownNow();
99          }
100       } catch (InterruptedException e) {
101         this.batchPool.shutdownNow();
102       }
103     }
104 
105   }
106 
107   /**
108    * Randomly pick a connection and process the batch of actions for a given table
109    * @param actions the actions
110    * @param tableName table name
111    * @param results the results array
112    * @param callback 
113    * @throws IOException
114    */
115   @SuppressWarnings("deprecation")
116   public <R> void processBatchCallback(List<? extends Row> actions, TableName tableName,
117       Object[] results, Batch.Callback<R> callback) throws IOException {
118     // Currently used by RegionStateStore
119     // A deprecated method is used as multiple threads accessing RegionStateStore do a single put
120     // and htable is not thread safe. Alternative would be to create an Htable instance for each 
121     // put but that is not very efficient.
122     // See HBASE-11610 for more details.
123     try {
124       hConnections[ThreadLocalRandom.current().nextInt(noOfConnections)].processBatchCallback(
125         actions, tableName, this.batchPool, results, callback);
126     } catch (InterruptedException e) {
127       throw new InterruptedIOException(e.getMessage());
128     }
129   }
130 
131   
132   // Copied from HConnectionImplementation.getBatchPool()
133   // We should get rid of this when HConnection.processBatchCallback is un-deprecated and provides
134   // an API to manage a batch pool
135   private void createBatchPool(Configuration conf) {
136     // Use the same config for keep alive as in HConnectionImplementation.getBatchPool();
137     int maxThreads = conf.getInt("hbase.multihconnection.threads.max", 256);
138     int coreThreads = conf.getInt("hbase.multihconnection.threads.core", 256);
139     if (maxThreads == 0) {
140       maxThreads = Runtime.getRuntime().availableProcessors() * 8;
141     }
142     if (coreThreads == 0) {
143       coreThreads = Runtime.getRuntime().availableProcessors() * 8;
144     }
145     long keepAliveTime = conf.getLong("hbase.multihconnection.threads.keepalivetime", 60);
146     LinkedBlockingQueue<Runnable> workQueue =
147         new LinkedBlockingQueue<Runnable>(maxThreads
148             * conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
149               HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS));
150     ThreadPoolExecutor tpe =
151         new ThreadPoolExecutor(coreThreads, maxThreads, keepAliveTime, TimeUnit.SECONDS, workQueue,
152             Threads.newDaemonThreadFactory("MultiHConnection" + "-shared-"));
153     tpe.allowCoreThreadTimeOut(true);
154     this.batchPool = tpe;
155   }
156   
157 }