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.Collections; 022import java.util.List; 023import java.util.concurrent.ExecutorService; 024import java.util.concurrent.LinkedBlockingQueue; 025import java.util.concurrent.ThreadLocalRandom; 026import java.util.concurrent.ThreadPoolExecutor; 027import java.util.concurrent.TimeUnit; 028import org.apache.hadoop.conf.Configuration; 029import org.apache.hadoop.hbase.HConstants; 030import org.apache.hadoop.hbase.TableName; 031import org.apache.hadoop.hbase.client.ClusterConnection; 032import org.apache.hadoop.hbase.client.Connection; 033import org.apache.hadoop.hbase.client.ConnectionFactory; 034import org.apache.hadoop.hbase.client.HTable; 035import org.apache.hadoop.hbase.client.Row; 036import org.apache.hadoop.hbase.client.coprocessor.Batch; 037import org.apache.yetus.audience.InterfaceAudience; 038import org.slf4j.Logger; 039import org.slf4j.LoggerFactory; 040 041import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; 042 043/** 044 * Provides ability to create multiple Connection instances and allows to process a batch of actions 045 * using CHTable.doBatchWithCallback() 046 */ 047@InterfaceAudience.Private 048public class MultiHConnection { 049 private static final Logger LOG = LoggerFactory.getLogger(MultiHConnection.class); 050 private Connection[] connections; 051 private final Object connectionsLock = new Object(); 052 private final int noOfConnections; 053 private ExecutorService batchPool; 054 055 /** 056 * Create multiple Connection instances and initialize a thread pool executor 057 * @param conf configuration 058 * @param noOfConnections total no of Connections to create 059 * @throws IOException if IO failure occurs 060 */ 061 public MultiHConnection(Configuration conf, int noOfConnections) throws IOException { 062 this.noOfConnections = noOfConnections; 063 synchronized (this.connectionsLock) { 064 connections = new Connection[noOfConnections]; 065 for (int i = 0; i < noOfConnections; i++) { 066 Connection conn = ConnectionFactory.createConnection(conf); 067 connections[i] = conn; 068 } 069 } 070 createBatchPool(conf); 071 } 072 073 /** 074 * Close the open connections and shutdown the batchpool 075 */ 076 public void close() { 077 synchronized (connectionsLock) { 078 if (connections != null) { 079 for (Connection conn : connections) { 080 if (conn != null) { 081 try { 082 conn.close(); 083 } catch (IOException e) { 084 LOG.info("Got exception in closing connection", e); 085 } finally { 086 conn = null; 087 } 088 } 089 } 090 connections = null; 091 } 092 } 093 if (this.batchPool != null && !this.batchPool.isShutdown()) { 094 this.batchPool.shutdown(); 095 try { 096 if (!this.batchPool.awaitTermination(10, TimeUnit.SECONDS)) { 097 this.batchPool.shutdownNow(); 098 } 099 } catch (InterruptedException e) { 100 this.batchPool.shutdownNow(); 101 } 102 } 103 104 } 105 106 /** 107 * Randomly pick a connection and process the batch of actions for a given table 108 * @param actions the actions 109 * @param tableName table name 110 * @param results the results array 111 * @param callback to run when results are in 112 * @throws IOException If IO failure occurs 113 */ 114 @SuppressWarnings("deprecation") 115 public <R> void processBatchCallback(List<? extends Row> actions, TableName tableName, 116 Object[] results, Batch.Callback<R> callback) throws IOException { 117 // Currently used by RegionStateStore 118 ClusterConnection conn = 119 (ClusterConnection) connections[ThreadLocalRandom.current().nextInt(noOfConnections)]; 120 121 HTable.doBatchWithCallback(actions, results, callback, conn, batchPool, tableName, 122 Collections.emptyMap()); 123 } 124 125 // Copied from ConnectionImplementation.getBatchPool() 126 // We should get rid of this when Connection.processBatchCallback is un-deprecated and provides 127 // an API to manage a batch pool 128 private void createBatchPool(Configuration conf) { 129 // Use the same config for keep alive as in ConnectionImplementation.getBatchPool(); 130 int maxThreads = conf.getInt("hbase.multihconnection.threads.max", 256); 131 if (maxThreads == 0) { 132 maxThreads = Runtime.getRuntime().availableProcessors() * 8; 133 } 134 long keepAliveTime = conf.getLong("hbase.multihconnection.threads.keepalivetime", 60); 135 LinkedBlockingQueue<Runnable> workQueue = 136 new LinkedBlockingQueue<>(maxThreads * conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS, 137 HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS)); 138 ThreadPoolExecutor tpe = 139 new ThreadPoolExecutor(maxThreads, maxThreads, keepAliveTime, TimeUnit.SECONDS, workQueue, 140 new ThreadFactoryBuilder().setNameFormat("MultiHConnection" + "-shared-pool-%d") 141 .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build()); 142 tpe.allowCoreThreadTimeOut(true); 143 this.batchPool = tpe; 144 } 145 146}