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.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
45
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
57
58
59
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
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
109
110
111
112
113
114
115
116 @SuppressWarnings("deprecation")
117 public <R> void processBatchCallback(List<? extends Row> actions, TableName tableName,
118 Object[] results, Batch.Callback<R> callback) throws IOException {
119
120
121
122
123
124 try {
125 hConnections[ThreadLocalRandom.current().nextInt(noOfConnections)].processBatchCallback(
126 actions, tableName, this.batchPool, results, callback);
127 } catch (InterruptedException e) {
128 throw new InterruptedIOException(e.getMessage());
129 }
130 }
131
132
133
134
135
136 private void createBatchPool(Configuration conf) {
137
138 int maxThreads = conf.getInt("hbase.multihconnection.threads.max", 256);
139 int coreThreads = conf.getInt("hbase.multihconnection.threads.core", 256);
140 if (maxThreads == 0) {
141 maxThreads = Runtime.getRuntime().availableProcessors() * 8;
142 }
143 if (coreThreads == 0) {
144 coreThreads = Runtime.getRuntime().availableProcessors() * 8;
145 }
146 long keepAliveTime = conf.getLong("hbase.multihconnection.threads.keepalivetime", 60);
147 LinkedBlockingQueue<Runnable> workQueue =
148 new LinkedBlockingQueue<Runnable>(maxThreads
149 * conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
150 HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS));
151 ThreadPoolExecutor tpe =
152 new ThreadPoolExecutor(coreThreads, maxThreads, keepAliveTime, TimeUnit.SECONDS, workQueue,
153 Threads.newDaemonThreadFactory("MultiHConnection" + "-shared-"));
154 tpe.allowCoreThreadTimeOut(true);
155 this.batchPool = tpe;
156 }
157
158 }