Skip to content

JDBC Client: fix a bug where TrinoResultSet cannot be properly closed#13775

Merged
findepi merged 1 commit intotrinodb:masterfrom
xiacongling:fix-not-interrupt-when-cancel
Sep 19, 2022
Merged

JDBC Client: fix a bug where TrinoResultSet cannot be properly closed#13775
findepi merged 1 commit intotrinodb:masterfrom
xiacongling:fix-not-interrupt-when-cancel

Conversation

@xiacongling
Copy link
Copy Markdown
Contributor

@xiacongling xiacongling commented Aug 22, 2022

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

Is this change a fix, improvement, new feature, refactoring, or other?

a fix

Is this a change to the core query engine, a connector, client library, or the SPI interfaces? (be specific)

JDBC client

How would you describe this change to a non-technical end user or system administrator?

fix a bug where TrinoResultSet cannot 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:

# JDBC Client
* Fix a bug where TrinoResultSet cannot be properly closed. ({issue}`7374`)

@cla-bot cla-bot bot added the cla-signed label Aug 22, 2022
@xiacongling xiacongling force-pushed the fix-not-interrupt-when-cancel branch from 0689910 to 7d01a1a Compare August 22, 2022 14:00
@github-actions github-actions bot added the jdbc Relates to Trino JDBC driver label Aug 22, 2022
@xiacongling xiacongling force-pushed the fix-not-interrupt-when-cancel branch from 7d01a1a to d38f6c4 Compare August 22, 2022 14:53
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good.
Now I wonder why we have used a CompletableFuture in the first place.

@xiacongling xiacongling force-pushed the fix-not-interrupt-when-cancel branch 2 times, most recently from 31d876d to bbb777b Compare August 23, 2022 01:19
@xiacongling xiacongling force-pushed the fix-not-interrupt-when-cancel branch from bbb777b to 7c383f6 Compare August 24, 2022 17:27
@xiacongling xiacongling force-pushed the fix-not-interrupt-when-cancel branch 3 times, most recently from e26dadf to c859f44 Compare August 25, 2022 03:47
@xiacongling xiacongling force-pushed the fix-not-interrupt-when-cancel branch from c859f44 to 32c03c5 Compare August 25, 2022 08:48
@xiacongling xiacongling force-pushed the fix-not-interrupt-when-cancel branch from 32c03c5 to 688e75b Compare August 25, 2022 12:53
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We already have TestJdbcResultSet to test the ResultSet implementation

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.
 */

@findepi
Copy link
Copy Markdown
Member

findepi commented Aug 26, 2022

@electrum please take a look

@xiacongling xiacongling force-pushed the fix-not-interrupt-when-cancel branch from 688e75b to 6733e10 Compare August 29, 2022 08:56
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does the order of these actions (client.close, draining) matter?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It doesn't matter for most cases. I am just worried that client may fill rowQueue after it has been drained.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So it may matter. Let's document the reasoning as a code comment

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.
 */

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.
@findepi findepi force-pushed the fix-not-interrupt-when-cancel branch from 5ab6597 to 2b63a62 Compare September 19, 2022 10:20
@findepi findepi merged commit 601a3a7 into trinodb:master Sep 19, 2022
@findepi findepi mentioned this pull request Sep 19, 2022
@github-actions github-actions bot added this to the 397 milestone Sep 19, 2022
@Twilightuse
Copy link
Copy Markdown

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

Is this change a fix, improvement, new feature, refactoring, or other?

a fix

Is this a change to the core query engine, a connector, client library, or the SPI interfaces? (be specific)

JDBC client

How would you describe this change to a non-technical end user or system administrator?

fix a bug where TrinoResultSet cannot 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:

# JDBC Client
* Fix a bug where TrinoResultSet cannot be properly closed. ({issue}`7374`)

trino-jdbc 406 a lot of connection in timed_wating, there is the thread stack
"OkHttp ConnectionPool" #3363 daemon prio=5 os_prio=0 tid=0x00007f403810d800 nid=0x1195 in Object.wait() [0x00007f362a373000]
java.lang.Thread.State: TIMED_WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
at java.lang.Object.wait(Object.java:460)
at io.trino.jdbc.$internal.okhttp3.internal.connection.RealConnectionPool.lambda$new$0(RealConnectionPool.java:62)

locked <0x0000000722d630e0> (a io.trino.jdbc.$internal.okhttp3.internal.connection.RealConnectionPool)
at io.trino.jdbc.$internal.okhttp3.internal.connection.RealConnectionPool$$Lambda$2010/1587346617.run(Unknown Source)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:750)
Locked ownable synchronizers:

<0x000000072287d118> (a java.util.concurrent.ThreadPoolExecutor$Worker)
"OkHttp ConnectionPool" #3358 daemon prio=5 os_prio=0 tid=0x00007f3eb0095000 nid=0x118d in Object.wait() [0x00007f362ab7b000]
java.lang.Thread.State: TIMED_WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
at java.lang.Object.wait(Object.java:460)
at io.trino.jdbc.$internal.okhttp3.internal.connection.RealConnectionPool.lambda$new$0(RealConnectionPool.java:62)
locked <0x0000000722ae7c90> (a io.trino.jdbc.$internal.okhttp3.internal.connection.RealConnectionPool)
at io.trino.jdbc.$internal.okhttp3.internal.connection.RealConnectionPool$$Lambda$2010/1587346617.run(Unknown Source)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:750)
Locked ownable synchronizers:

<0x000000072261f628> (a java.util.concurrent.ThreadPoolExecutor$Worker)
a

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

cla-signed jdbc Relates to Trino JDBC driver

Development

Successfully merging this pull request may close these issues.

An increasing number of WAITING state threads in JDBC driver In Trino-JDBC calling TrinoResultSet.cancel can't really close completablaFuture thread

5 participants