001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.util;
019
020import java.io.IOException;
021import java.util.List;
022import java.util.concurrent.ExecutorService;
023import java.util.concurrent.LinkedBlockingQueue;
024import java.util.concurrent.ThreadLocalRandom;
025import java.util.concurrent.ThreadPoolExecutor;
026import java.util.concurrent.TimeUnit;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.hbase.HConstants;
029import org.apache.hadoop.hbase.TableName;
030import org.apache.hadoop.hbase.client.ClusterConnection;
031import org.apache.hadoop.hbase.client.Connection;
032import org.apache.hadoop.hbase.client.ConnectionFactory;
033import org.apache.hadoop.hbase.client.HTable;
034import org.apache.hadoop.hbase.client.Row;
035import org.apache.hadoop.hbase.client.coprocessor.Batch;
036import org.apache.yetus.audience.InterfaceAudience;
037import org.slf4j.Logger;
038import org.slf4j.LoggerFactory;
039
040import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
041
042/**
043 * Provides ability to create multiple Connection instances and allows to process a batch of actions
044 * using CHTable.doBatchWithCallback()
045 */
046@InterfaceAudience.Private
047public class MultiHConnection {
048  private static final Logger LOG = LoggerFactory.getLogger(MultiHConnection.class);
049  private Connection[] connections;
050  private final Object connectionsLock = new Object();
051  private final int noOfConnections;
052  private ExecutorService batchPool;
053
054  /**
055   * Create multiple Connection instances and initialize a thread pool executor
056   * @param conf            configuration
057   * @param noOfConnections total no of Connections to create
058   * @throws IOException if IO failure occurs
059   */
060  public MultiHConnection(Configuration conf, int noOfConnections) throws IOException {
061    this.noOfConnections = noOfConnections;
062    synchronized (this.connectionsLock) {
063      connections = new Connection[noOfConnections];
064      for (int i = 0; i < noOfConnections; i++) {
065        Connection conn = ConnectionFactory.createConnection(conf);
066        connections[i] = conn;
067      }
068    }
069    createBatchPool(conf);
070  }
071
072  /**
073   * Close the open connections and shutdown the batchpool
074   */
075  public void close() {
076    synchronized (connectionsLock) {
077      if (connections != null) {
078        for (Connection conn : connections) {
079          if (conn != null) {
080            try {
081              conn.close();
082            } catch (IOException e) {
083              LOG.info("Got exception in closing connection", e);
084            } finally {
085              conn = null;
086            }
087          }
088        }
089        connections = null;
090      }
091    }
092    if (this.batchPool != null && !this.batchPool.isShutdown()) {
093      this.batchPool.shutdown();
094      try {
095        if (!this.batchPool.awaitTermination(10, TimeUnit.SECONDS)) {
096          this.batchPool.shutdownNow();
097        }
098      } catch (InterruptedException e) {
099        this.batchPool.shutdownNow();
100      }
101    }
102
103  }
104
105  /**
106   * Randomly pick a connection and process the batch of actions for a given table
107   * @param actions   the actions
108   * @param tableName table name
109   * @param results   the results array
110   * @param callback  to run when results are in
111   * @throws IOException If IO failure occurs
112   */
113  @SuppressWarnings("deprecation")
114  public <R> void processBatchCallback(List<? extends Row> actions, TableName tableName,
115    Object[] results, Batch.Callback<R> callback) throws IOException {
116    // Currently used by RegionStateStore
117    ClusterConnection conn =
118      (ClusterConnection) connections[ThreadLocalRandom.current().nextInt(noOfConnections)];
119
120    HTable.doBatchWithCallback(actions, results, callback, conn, batchPool, tableName);
121  }
122
123  // Copied from ConnectionImplementation.getBatchPool()
124  // We should get rid of this when Connection.processBatchCallback is un-deprecated and provides
125  // an API to manage a batch pool
126  private void createBatchPool(Configuration conf) {
127    // Use the same config for keep alive as in ConnectionImplementation.getBatchPool();
128    int maxThreads = conf.getInt("hbase.multihconnection.threads.max", 256);
129    if (maxThreads == 0) {
130      maxThreads = Runtime.getRuntime().availableProcessors() * 8;
131    }
132    long keepAliveTime = conf.getLong("hbase.multihconnection.threads.keepalivetime", 60);
133    LinkedBlockingQueue<Runnable> workQueue =
134      new LinkedBlockingQueue<>(maxThreads * conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
135        HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS));
136    ThreadPoolExecutor tpe =
137      new ThreadPoolExecutor(maxThreads, maxThreads, keepAliveTime, TimeUnit.SECONDS, workQueue,
138        new ThreadFactoryBuilder().setNameFormat("MultiHConnection" + "-shared-pool-%d")
139          .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build());
140    tpe.allowCoreThreadTimeOut(true);
141    this.batchPool = tpe;
142  }
143
144}