forked from apache/hbase
-
Notifications
You must be signed in to change notification settings - Fork 0
/
ConnectionCache.java
256 lines (236 loc) · 8.26 KB
/
ConnectionCache.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util;
import java.io.IOException;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.locks.Lock;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.security.UserGroupInformation;
/**
* A utility to store user specific HConnections in memory.
* There is a chore to clean up connections idle for too long.
* This class is used by REST server and Thrift server to
* support authentication and impersonation.
*/
@InterfaceAudience.Private
public class ConnectionCache {
private static final Logger LOG = LoggerFactory.getLogger(ConnectionCache.class);
private final Map<String, ConnectionInfo> connections = new ConcurrentHashMap<>();
private final KeyLocker<String> locker = new KeyLocker<>();
private final String realUserName;
private final UserGroupInformation realUser;
private final UserProvider userProvider;
private final Configuration conf;
private final ChoreService choreService;
private final ThreadLocal<String> effectiveUserNames =
new ThreadLocal<String>() {
@Override
protected String initialValue() {
return realUserName;
}
};
public ConnectionCache(final Configuration conf,
final UserProvider userProvider,
final int cleanInterval, final int maxIdleTime) throws IOException {
Stoppable stoppable = new Stoppable() {
private volatile boolean isStopped = false;
@Override public void stop(String why) { isStopped = true;}
@Override public boolean isStopped() {return isStopped;}
};
this.choreService = new ChoreService("ConnectionCache");
ScheduledChore cleaner = new ScheduledChore("ConnectionCleaner", stoppable, cleanInterval) {
@Override
protected void chore() {
for (Map.Entry<String, ConnectionInfo> entry: connections.entrySet()) {
ConnectionInfo connInfo = entry.getValue();
if (connInfo.timedOut(maxIdleTime)) {
if (connInfo.admin != null) {
try {
connInfo.admin.close();
} catch (Throwable t) {
LOG.info("Got exception in closing idle admin", t);
}
}
try {
connInfo.connection.close();
} catch (Throwable t) {
LOG.info("Got exception in closing idle connection", t);
}
}
}
}
};
// Start the daemon cleaner chore
choreService.scheduleChore(cleaner);
this.realUser = userProvider.getCurrent().getUGI();
this.realUserName = realUser.getShortUserName();
this.userProvider = userProvider;
this.conf = conf;
}
/**
* Set the current thread local effective user
*/
public void setEffectiveUser(String user) {
effectiveUserNames.set(user);
}
/**
* Get the current thread local effective user
*/
public String getEffectiveUser() {
return effectiveUserNames.get();
}
/**
* Called when cache is no longer needed so that it can perform cleanup operations
*/
public void shutdown() {
if (choreService != null) choreService.shutdown();
}
/**
* Caller doesn't close the admin afterwards.
* We need to manage it and close it properly.
*/
public Admin getAdmin() throws IOException {
ConnectionInfo connInfo = getCurrentConnection();
if (connInfo.admin == null) {
Lock lock = locker.acquireLock(getEffectiveUser());
try {
if (connInfo.admin == null) {
connInfo.admin = connInfo.connection.getAdmin();
}
} finally {
lock.unlock();
}
}
return connInfo.admin;
}
/**
* Caller closes the table afterwards.
*/
public Table getTable(String tableName) throws IOException {
ConnectionInfo connInfo = getCurrentConnection();
return connInfo.connection.getTable(TableName.valueOf(tableName));
}
/**
* Retrieve a regionLocator for the table. The user should close the RegionLocator.
*/
public RegionLocator getRegionLocator(byte[] tableName) throws IOException {
return getCurrentConnection().connection.getRegionLocator(TableName.valueOf(tableName));
}
/**
* Get the cached connection for the current user.
* If none or timed out, create a new one.
*/
ConnectionInfo getCurrentConnection() throws IOException {
String userName = getEffectiveUser();
ConnectionInfo connInfo = connections.get(userName);
if (connInfo == null || !connInfo.updateAccessTime()) {
Lock lock = locker.acquireLock(userName);
try {
connInfo = connections.get(userName);
if (connInfo == null) {
UserGroupInformation ugi = realUser;
if (!userName.equals(realUserName)) {
ugi = UserGroupInformation.createProxyUser(userName, realUser);
}
User user = userProvider.create(ugi);
Connection conn = ConnectionFactory.createConnection(conf, user);
connInfo = new ConnectionInfo(conn, userName);
connections.put(userName, connInfo);
}
} finally {
lock.unlock();
}
}
return connInfo;
}
/**
* Updates the access time for the current connection. Used to keep Connections alive for
* long-lived scanners.
* @return whether we successfully updated the last access time
*/
public boolean updateConnectionAccessTime() {
String userName = getEffectiveUser();
ConnectionInfo connInfo = connections.get(userName);
if (connInfo != null) {
return connInfo.updateAccessTime();
}
return false;
}
/**
* @return Cluster ID for the HBase cluster or null if there is an err making the connection.
*/
public String getClusterId() {
try {
ConnectionInfo connInfo = getCurrentConnection();
return connInfo.connection.getClusterId();
} catch (IOException e) {
LOG.error("Error getting connection: ", e);
}
return null;
}
class ConnectionInfo {
final Connection connection;
final String userName;
volatile Admin admin;
private long lastAccessTime;
private boolean closed;
ConnectionInfo(Connection conn, String user) {
lastAccessTime = EnvironmentEdgeManager.currentTime();
connection = conn;
closed = false;
userName = user;
}
synchronized boolean updateAccessTime() {
if (closed) {
return false;
}
if (connection.isAborted() || connection.isClosed()) {
LOG.info("Unexpected: cached Connection is aborted/closed, removed from cache");
connections.remove(userName);
return false;
}
lastAccessTime = EnvironmentEdgeManager.currentTime();
return true;
}
synchronized boolean timedOut(int maxIdleTime) {
long timeoutTime = lastAccessTime + maxIdleTime;
if (EnvironmentEdgeManager.currentTime() > timeoutTime) {
connections.remove(userName);
closed = true;
return true;
}
return false;
}
}
}