Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
27 changes: 12 additions & 15 deletions client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoResultSet.java
Original file line number Diff line number Diff line change
Expand Up @@ -183,7 +183,9 @@ public AsyncIterator(Iterator<T> dataIterator, StatementClient client)
}
}
catch (InterruptedException e) {
interrupt(e);
client.close();
rowQueue.clear();
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.

why does cleaning rowQueue here matter?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

here it is probably not important. it's more like "why not?"

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

discussed offline and agreed to keep

throw new RuntimeException(new SQLException("ResultSet thread was interrupted", e));
}
finally {
semaphore.release();
Expand All @@ -196,18 +198,6 @@ public void cancel()
{
cancelled = true;
future.cancel(true);
cleanup();
}

public void interrupt(InterruptedException e)
{
cleanup();
Thread.currentThread().interrupt();
throw new RuntimeException(new SQLException("ResultSet thread was interrupted", e));
}

private void cleanup()
{
// When thread interruption is mis-handled by underlying implementation of `client`, the thread which
// is working for `future` may be blocked by `rowQueue.put` (`rowQueue` is full) and will never finish
// its work. It is necessary to close `client` and drain `rowQueue` to avoid such leaks.
Expand All @@ -234,7 +224,7 @@ protected T computeNext()
semaphore.acquire();
}
catch (InterruptedException e) {
interrupt(e);
handleInterrupt(e);
}
if (rowQueue.isEmpty()) {
// If we got here and the queue is empty the thread fetching from the underlying iterator is done.
Expand All @@ -243,7 +233,7 @@ protected T computeNext()
future.get();
}
catch (InterruptedException e) {
interrupt(e);
handleInterrupt(e);
}
catch (ExecutionException e) {
throwIfUnchecked(e.getCause());
Expand All @@ -253,6 +243,13 @@ protected T computeNext()
}
return rowQueue.poll();
}

private void handleInterrupt(InterruptedException e)
{
cancel();
Thread.currentThread().interrupt();
throw new RuntimeException(new SQLException("Interrupted", e));
}
}

private static class ResultsPageIterator
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -906,7 +906,7 @@ public void testQueryCancelByInterrupt()
assertTrue(queryFinished.await(10, SECONDS));
assertThat(queryFailure.get())
.isInstanceOf(SQLException.class)
.hasMessage("ResultSet thread was interrupted");
.hasMessage("Interrupted");
assertEquals(getQueryState(queryId.get()), FAILED);
}

Expand Down