1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
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
44
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
55
56
57
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
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
114
115
116
117
118
119
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
125
126
127
128
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
139
140
141 private void createBatchPool(Configuration conf) {
142
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 }