JDBC Client: fix a bug where TrinoResultSet cannot be properly closed#13775
Conversation
0689910 to
7d01a1a
Compare
7d01a1a to
d38f6c4
Compare
There was a problem hiding this comment.
Looks good.
Now I wonder why we have used a CompletableFuture in the first place.
client/trino-jdbc/src/test/java/io/trino/jdbc/TestTrinoResultSet.java
Outdated
Show resolved
Hide resolved
31d876d to
bbb777b
Compare
client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoResultSet.java
Outdated
Show resolved
Hide resolved
bbb777b to
7c383f6
Compare
client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoResultSet.java
Outdated
Show resolved
Hide resolved
e26dadf to
c859f44
Compare
client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoResultSet.java
Outdated
Show resolved
Hide resolved
client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoResultSet.java
Outdated
Show resolved
Hide resolved
client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoResultSet.java
Outdated
Show resolved
Hide resolved
c859f44 to
32c03c5
Compare
client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoResultSet.java
Outdated
Show resolved
Hide resolved
32c03c5 to
688e75b
Compare
client/trino-jdbc/src/test/java/io/trino/jdbc/TestTrinoResultSet.java
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
We already have TestJdbcResultSet to test the ResultSet implementation
There was a problem hiding this comment.
Yes, I notice that. But TestJdbcResultSet is a integration test with test trino server set up. The test cases here are all unit tests. Is it better to keep them separate?
There was a problem hiding this comment.
Add a javadoc explaining what it is and also @see tag -- in both these tests.
like
/**
* A unit test for {@link TrinoResultSet}.
*
* @see TestJdbcResultSet an integration test.
*/
client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoResultSet.java
Outdated
Show resolved
Hide resolved
client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoResultSet.java
Outdated
Show resolved
Hide resolved
|
@electrum please take a look |
688e75b to
6733e10
Compare
client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoResultSet.java
Outdated
Show resolved
Hide resolved
client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoResultSet.java
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Does the order of these actions (client.close, draining) matter?
There was a problem hiding this comment.
It doesn't matter for most cases. I am just worried that client may fill rowQueue after it has been drained.
There was a problem hiding this comment.
So it may matter. Let's document the reasoning as a code comment
There was a problem hiding this comment.
Add a javadoc explaining what it is and also @see tag -- in both these tests.
like
/**
* A unit test for {@link TrinoResultSet}.
*
* @see TestJdbcResultSet an integration test.
*/
client/trino-jdbc/src/test/java/io/trino/jdbc/TestTrinoResultSet.java
Outdated
Show resolved
Hide resolved
client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoResultSet.java
Outdated
Show resolved
Hide resolved
6733e10 to
5ab6597
Compare
ResultSet cannot be properly closed because the inner Thread cannot be interrupted and stop data row iteration. That will lead to thread and memory leaks on the client side. This patch uses FutureTask, which is created by ThreadPoolExecutor, instead of CompletableFuture to make sure `Thread.interrupt()` can be invoked as expected. And for the case that interruption is not properly handled by the underlying StatementClient, a status check is added to the loop condition so that loop can terminate and thread can be released.
5ab6597 to
2b63a62
Compare
trino-jdbc 406 a lot of connection in timed_wating, there is the thread stack locked <0x0000000722d630e0> (a io.trino.jdbc.$internal.okhttp3.internal.connection.RealConnectionPool) <0x000000072287d118> (a java.util.concurrent.ThreadPoolExecutor$Worker) <0x000000072261f628> (a java.util.concurrent.ThreadPoolExecutor$Worker) |
ResultSet cannot be properly closed since the inner Thread cannot be interrupted and stop data row iteration. That will lead to thread and memory leaks on the client side. This patch uses FutureTask, which is created by ThreadPoolExecutor, instead of CompletableFuture to make sure InterruptedException can be raised as expected. And for the case that interruption is not properly handled by the underlying StatementClient, a status check is added to the loop condition so that loop can terminate and thread can be released.
Description
a fix
JDBC client
fix a bug where
TrinoResultSetcannot be properly closed which will lead to thread and memory leaks on the client side.Related issues, pull requests, and links
Documentation
(x) No documentation is needed.
( ) Sufficient documentation is included in this PR.
( ) Documentation PR is available with #prnumber.
( ) Documentation issue #issuenumber is filed, and can be handled later.
Release notes
( ) No release notes entries required.
(x) Release notes entries required with the following suggested text: