You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@phoenix.apache.org by "Josh Elser (JIRA)" <ji...@apache.org> on 2018/07/11 19:12:00 UTC
[jira] [Commented] (PHOENIX-4809) connectionQueue never cleared in
ConnectionQueryServicesImpl
[ https://issues.apache.org/jira/browse/PHOENIX-4809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16540555#comment-16540555 ]
Josh Elser commented on PHOENIX-4809:
-------------------------------------
I ran a quick test locally, the odd thing is that I see fewer {{WeakReference<PhoenixConnection>}} objects than I'd expect..
{code:java}
public class PhoenixQuery {
private static final String URL = "jdbc:phoenix:localhost:2181:/hbase-1.4";
public static void main(String[] args) throws Exception {
ExecutorService svc = Executors.newFixedThreadPool(8);
try (Connection conn = DriverManager.getConnection(URL, "", "")) {
PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
ConnectionQueryServicesImpl queryServices = (ConnectionQueryServicesImpl) phxConn.getQueryServices();
System.out.println(numConnectionsCached(queryServices));
}
CountDownLatch latch = new CountDownLatch(8);
for (int i = 0; i < 8; i++) {
svc.submit(new Runnable() {
@Override public void run() {
for (int j = 0; j < 1_000_000; j++) {
try (Connection conn = DriverManager.getConnection(URL, "", "")) {
conn.close();
} catch (Exception e) {
System.out.println("Caught error");
}
}
latch.countDown();
}
});
}
latch.await();
try (Connection conn = DriverManager.getConnection(URL, "", "")) {
PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
ConnectionQueryServicesImpl queryServices = (ConnectionQueryServicesImpl) phxConn.getQueryServices();
System.out.println(numConnectionsCached(queryServices));
}
}
private static long numConnectionsCached(ConnectionQueryServicesImpl cqs) {
try {
Field f = ConnectionQueryServicesImpl.class.getDeclaredField("connectionQueues");
f.setAccessible(true);
List<LinkedBlockingQueue<WeakReference<PhoenixConnection>>> list = (List<LinkedBlockingQueue<WeakReference<PhoenixConnection>>>) f.get(cqs);
return list.stream().collect(Collectors.summingLong(LinkedBlockingQueue::size));
} catch (Exception e) {
e.printStackTrace(System.err);
throw new RuntimeException(e);
}
}
}{code}
I'd expect to see 8M objects cached, but I'm actually seeing 615,696. Something must clean this up somehow.. Maybe related to lease renewal not being on?
> connectionQueue never cleared in ConnectionQueryServicesImpl
> ------------------------------------------------------------
>
> Key: PHOENIX-4809
> URL: https://issues.apache.org/jira/browse/PHOENIX-4809
> Project: Phoenix
> Issue Type: Bug
> Reporter: Josh Elser
> Assignee: Josh Elser
> Priority: Major
> Fix For: 4.15.0, 5.1.0
>
>
> When we create a new {{PhoenixConnection}}, we update {{connectionQueues}} in CQSI:
> {code:java}
> @Override
> public void addConnection(PhoenixConnection connection) throws SQLException {
> connectionQueues.get(getQueueIndex(connection)).add(new WeakReference<PhoenixConnection>(connection));
> if (returnSequenceValues) {
> synchronized (connectionCountLock) {
> connectionCount++;
> }
> }
> }{code}
> We use connectionQueues to determine what needs lease renewal done.
> However, when the user closes a connection, this datastructure is never cleaned up.
> {code:java}
> @Override
> public void removeConnection(PhoenixConnection connection) throws SQLException {
> if (returnSequenceValues) {
> ConcurrentMap<SequenceKey,Sequence> formerSequenceMap = null;
> synchronized (connectionCountLock) {
> if (--connectionCount <= 0) {
> if (!this.sequenceMap.isEmpty()) {
> formerSequenceMap = this.sequenceMap;
> this.sequenceMap = Maps.newConcurrentMap();
> }
> }
> if (connectionCount < 0) {
> connectionCount = 0;
> }
> }
> // Since we're using the former sequenceMap, we can do this outside
> // the lock.
> if (formerSequenceMap != null) {
> // When there are no more connections, attempt to return any sequences
> returnAllSequences(formerSequenceMap);
> }
> } else if (shouldThrottleNumConnections){ //still need to decrement connection count
> synchronized (connectionCountLock) {
> if (connectionCount > 0) {
> --connectionCount;
> }
> }
> }
> }{code}
> Running a test now, but seems to be the case on master.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)