Skip to content

Fix crash when receiving from disconnected Connection#92807

Merged
rienath merged 6 commits intomasterfrom
fix-disconnect-crash
Dec 24, 2025
Merged

Fix crash when receiving from disconnected Connection#92807
rienath merged 6 commits intomasterfrom
fix-disconnect-crash

Conversation

@rienath
Copy link
Member

@rienath rienath commented Dec 22, 2025

Idealogical successor of #89740. Avoid doing finish() twice.

Changelog category (leave one):

  • Bug Fix (user-visible misbehavior in an official stable release)

Changelog entry (a user-readable short description of the changes that goes into CHANGELOG.md):

Avoid crash due to reading from a disconnected Connection

@clickhouse-gh
Copy link
Contributor

clickhouse-gh bot commented Dec 22, 2025

Workflow [PR], commit [73a3269]

Summary:

job_name test_name status info comment
BuzzHouse (amd_debug) failure
Logical error: 'Inconsistent AST formatting: the query: (STID: 1941-1bfa) FAIL issue ISSUE EXISTS
Finish Workflow failure
python3 ./ci/jobs/scripts/workflow_hooks/new_tests_check.py failure IGNORED
  • Finish Workflow failure is expected as this change does not have a test

@clickhouse-gh clickhouse-gh bot added the pr-bugfix Pull request with bugfix, not backported by default label Dec 22, 2025
@azat azat self-assigned this Dec 22, 2025
return *last_input_packet_type;

/// If we already disconnected
if (!in)
Copy link
Member

Choose a reason for hiding this comment

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

In which case it is possible got have reseted in here and below?

Copy link
Member Author

Choose a reason for hiding this comment

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

I know it happens because I have seen crash with the following stack trace

2.0. inlined from ./src/IO/BufferBase.h:40: DB::BufferBase::Buffer::end() const
2.1. inlined from ./src/IO/VarInt.h:98: DB::readVarUInt(unsigned long&, DB::ReadBuffer&)
2. ./ci/tmp/build/./src/Client/Connection.cpp:1338: DB::Connection::receivePacket() @ 0x0000000019429ae5
3. ./ci/tmp/build/./src/Client/MultiplexedConnections.cpp:398: DB::MultiplexedConnections::receivePacketUnlocked(std::function<void (int, Poco::Timespan, DB::AsyncEventTimeoutType, String const&, unsigned int)>) @ 0x0000000019472964
4. ./ci/tmp/build/./src/Client/MultiplexedConnections.cpp:253: DB::MultiplexedConnections::receivePacket() @ 0x0000000019472704
5. ./ci/tmp/build/./src/QueryPipeline/RemoteQueryExecutor.cpp:808: DB::RemoteQueryExecutor::finish() @ 0x00000000168a4154
6. ./ci/tmp/build/./src/Processors/Sources/RemoteSource.cpp:216: DB::RemoteSource::tryGenerate() @ 0x0000000019affd68
7. ./ci/tmp/build/./src/Processors/ISource.cpp:110: DB::ISource::work() @ 0x0000000019727d20
...

I suspect, in RemoteQueryExecutor::finish() we call tryCancel and then enter the while loop. In the loop, something happens (i.e. network error or timeout) that triggers disconnect(). In the next iteration, connections->receivePacket() would crash

Copy link
Member

Choose a reason for hiding this comment

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

I know it happens because I have seen crash with the following stack trace

Yes, but we need to make sure that we are fixing correct place

I suspect, in RemoteQueryExecutor::finish() we call tryCancel and then enter the while loop. In the loop, something happens (i.e. network error or timeout) that triggers disconnect(). In the next iteration, connections->receivePacket() would crash

If it will call sendCancel then it should reset only out, and we have proper check for it now, so it is still unclear how in can be not set.

@azat azat force-pushed the fix-disconnect-crash branch from 8025e42 to 537cfdf Compare December 22, 2025 15:47
rienath and others added 3 commits December 22, 2025 17:16
It is possible that RemoteSource will finish() query multiple times,
while the exception will be suppressed by the PipelineExecutor

First finish():

    2025-12-21 15:32:06.673840      Debug   2236    ReadBuffer      ReadBuffer is canceled by the exception: Code: 209. DB::NetException: Timeout exceeded while reading from socket (peer: 10.49.85.26:9000, local: 10.49.18.205:49572, 300000 ms). (SOCKET_TIMEOUT), Stack trace (when copying this message, always include the lines below):

    0. ./ci/tmp/build/./src/Common/Exception.cpp:131: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x0000000010ff8d70
    1. DB::Exception::Exception(String&&, int, String, bool) @ 0x000000000baa45d8
    2. ./src/Common/NetException.h:26: DB::NetException::NetException<String, String, long>(int, FormatStringHelperImpl<std::type_identity<String>::type, std::type_identity<String>::type, std::type_identity<long>::type>, String&&, String&&, long&&) @ 0x00000000111e84ac
    3. ./ci/tmp/build/./src/IO/ReadBufferFromPocoSocket.cpp:87: DB::ReadBufferFromPocoSocketBase::socketReceiveBytesImpl(char*, unsigned long) @ 0x00000000111e8094
    4. ./ci/tmp/build/./src/IO/ReadBufferFromPocoSocket.cpp:107: DB::ReadBufferFromPocoSocketBase::nextImpl() @ 0x00000000111e876c
    5. ./ci/tmp/build/./src/IO/ReadBuffer.cpp:96: DB::ReadBuffer::next() @ 0x00000000110c4774
    6. ./src/IO/ReadBuffer.h:81: DB::Connection::receivePacket() @ 0x0000000016e00200
    7. ./ci/tmp/build/./src/Client/MultiplexedConnections.cpp:398: DB::MultiplexedConnections::receivePacketUnlocked(std::function<void (int, Poco::Timespan, DB::AsyncEventTimeoutType, String const&, unsigned int)>) @ 0x0000000016e4090c
    8. ./ci/tmp/build/./src/Client/MultiplexedConnections.cpp:253: DB::MultiplexedConnections::receivePacket() @ 0x0000000016e406c4
    9. ./ci/tmp/build/./src/QueryPipeline/RemoteQueryExecutor.cpp:808: DB::RemoteQueryExecutor::finish() @ 0x0000000014571abc
    10. ./ci/tmp/build/./src/Processors/Executors/ExecutingGraph.cpp:253: DB::ExecutingGraph::updateNode(unsigned long, std::queue<DB::ExecutingGraph::Node*, std::deque<DB::ExecutingGraph::Node*, AllocatorWithMemoryTracking<DB::ExecutingGraph::Node*>>>&, std::queue<DB::ExecutingGraph::Node*, std::deque<DB::ExecutingGraph::Node*, AllocatorWithMemoryTracking<DB::ExecutingGraph::Node*>>>&) @ 0x00000000170a97ec

Second finish():

    2.1. inlined from ./src/IO/VarInt.h:98: DB::readVarUInt(unsigned long&, DB::ReadBuffer&)
    2. ./ci/tmp/build/./src/Client/Connection.cpp:1310: DB::Connection::receivePacket() @ 0x0000000016e000b9
    3. ./ci/tmp/build/./src/Client/MultiplexedConnections.cpp:398: DB::MultiplexedConnections::receivePacketUnlocked(std::function<void (int, Poco::Timespan, DB::AsyncEventTimeoutType, String const&, unsigned int)>) @ 0x0000000016e4090c
    4. ./ci/tmp/build/./src/Client/MultiplexedConnections.cpp:253: DB::MultiplexedConnections::receivePacket() @ 0x0000000016e406c4
    5. ./ci/tmp/build/./src/QueryPipeline/RemoteQueryExecutor.cpp:808: DB::RemoteQueryExecutor::finish() @ 0x0000000014571abc
    6. ./ci/tmp/build/./src/Processors/Sources/RemoteSource.cpp:216: DB::RemoteSource::tryGenerate() @ 0x0000000017417ea8
    7. ./ci/tmp/build/./src/Processors/ISource.cpp:110: DB::ISource::work() @ 0x0000000017096e80
    8.0. inlined from ./ci/tmp/build/./src/Processors/Executors/ExecutionThreadContext.cpp:53: DB::executeJob(DB::ExecutingGraph::Node*, DB::ReadProgressCallback*)
@azat azat force-pushed the fix-disconnect-crash branch from 8279083 to b63ef1a Compare December 22, 2025 16:16

void RemoteSource::finishExecutor()
{
if (executor_finished)
Copy link
Member

Choose a reason for hiding this comment

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

As long as there is a data race on that flag, maybe we can move a logic to RemoteQueryExecutor::finish?

There is already a mutex taken

LockAndBlocker guard(was_cancelled_mutex);

so looks like we can additionally check whether RemoteQueryExecutor::finish was called before and exit.

@rienath rienath force-pushed the fix-disconnect-crash branch from 762e69b to a388199 Compare December 23, 2025 15:48
/// We only allow one finish call to avoid double-cancellation
if (isFinishCalled())
return;
finish_called = true;
Copy link
Member

Choose a reason for hiding this comment

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

This class already has too many flags... can we reuse finished flag?

I.e. smth like this:

$ git di
diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp
index fa4794555ee..448821d278d 100644
--- a/src/QueryPipeline/RemoteQueryExecutor.cpp
+++ b/src/QueryPipeline/RemoteQueryExecutor.cpp
@@ -778,6 +778,8 @@ void RemoteQueryExecutor::finish()
 {
     LockAndBlocker guard(was_cancelled_mutex);

+    SCOPE_EXIT(finished = true);
+
     /** If one of:
       * - nothing started to do;
       * - received all packets before EndOfStream;

Copy link
Member Author

Choose a reason for hiding this comment

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

Great idea! We would go over

    if (!isQueryPending() || hasThrownException())
        return;

    /** If you have not read all the data yet, but they are no longer needed.
      * This may be due to the fact that the data is sufficient (for example, when using LIMIT).
      */

    /// Send the request to abort the execution of the request, if not already sent.
    tryCancel("Cancelling query because enough data has been read");

upon entering finish the second time, but it's harmless

Copy link
Member

Choose a reason for hiding this comment

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

Once finished = true, isQueryPending will return false and it will be effectively the same as your version with extra flag

Copy link
Member

Choose a reason for hiding this comment

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

Actually that was a bad idea, this leaves the connection in a broken state after, since destructor relies in this flag, and calls disconnect if the connection was in a broken state, but now, this does not work, I will prepare a fix (need to debug another problem first)

Copy link
Member

Choose a reason for hiding this comment

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

And also what will happen if onUpdatePorts will throw? It will break the whole query AFAICS, while we do not need this

Copy link
Member

@azat azat Dec 29, 2025

Choose a reason for hiding this comment

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

Actually that was a bad idea, this leaves the connection in a broken state after, since destructor relies in this flag, and calls disconnect if the connection was in a broken state, but now, this does not work, I will prepare a fix (need to debug another problem first)

@rienath rienath force-pushed the fix-disconnect-crash branch from 8a03f1b to 73a3269 Compare December 23, 2025 18:29
@rienath rienath added this pull request to the merge queue Dec 24, 2025
Merged via the queue into master with commit fd31195 Dec 24, 2025
251 of 258 checks passed
@rienath rienath deleted the fix-disconnect-crash branch December 24, 2025 11:10
robot-ch-test-poll2 added a commit that referenced this pull request Dec 24, 2025
Cherry pick #92807 to 25.8: Fix crash when receiving from disconnected `Connection`
robot-clickhouse added a commit that referenced this pull request Dec 24, 2025
robot-ch-test-poll2 added a commit that referenced this pull request Dec 24, 2025
Cherry pick #92807 to 25.10: Fix crash when receiving from disconnected `Connection`
robot-clickhouse added a commit that referenced this pull request Dec 24, 2025
robot-ch-test-poll2 added a commit that referenced this pull request Dec 24, 2025
Cherry pick #92807 to 25.11: Fix crash when receiving from disconnected `Connection`
robot-clickhouse added a commit that referenced this pull request Dec 24, 2025
robot-ch-test-poll2 added a commit that referenced this pull request Dec 24, 2025
Cherry pick #92807 to 25.12: Fix crash when receiving from disconnected `Connection`
robot-clickhouse added a commit that referenced this pull request Dec 24, 2025
@robot-clickhouse-ci-1 robot-clickhouse-ci-1 added the pr-synced-to-cloud The PR is synced to the cloud repo label Dec 24, 2025
@robot-ch-test-poll2 robot-ch-test-poll2 added the pr-must-backport-synced The `*-must-backport` labels are synced into the cloud Sync PR label Dec 24, 2025
@robot-ch-test-poll robot-ch-test-poll added the pr-backports-created Backport PRs are successfully created, it won't be processed by CI script anymore label Dec 24, 2025
clickhouse-gh bot added a commit that referenced this pull request Dec 24, 2025
Backport #92807 to 25.12: Fix crash when receiving from disconnected `Connection`
clickhouse-gh bot added a commit that referenced this pull request Dec 24, 2025
Backport #92807 to 25.8: Fix crash when receiving from disconnected `Connection`
clickhouse-gh bot added a commit that referenced this pull request Dec 24, 2025
Backport #92807 to 25.10: Fix crash when receiving from disconnected `Connection`
clickhouse-gh bot added a commit that referenced this pull request Dec 24, 2025
Backport #92807 to 25.11: Fix crash when receiving from disconnected `Connection`
azat added a commit to azat/ClickHouse that referenced this pull request Dec 24, 2025
…on distributed queries

The problem is that some places in the code (i.e.
RemoteSource::onUpdatePorts()), can call finish() even before
sendQueryAsync() finishes sending the query, and so if after it will try
to call finish() again (after sendQueryAsync() finishes) it will be
no-op.

The problem pops up after ClickHouse#92807, since it has these (anti-)pattern.
azat added a commit to azat/ClickHouse that referenced this pull request Dec 24, 2025
…on distributed queries

The problem is that some places in the code (i.e.
RemoteSource::onUpdatePorts()), can call finish() even before
sendQueryAsync() finishes sending the query, and so if after it will try
to call finish() again (after sendQueryAsync() finishes) it will be
no-op.

The problem pops up after ClickHouse#92807, since RemoteSource has these (anti-)pattern.
@@ -248,9 +250,7 @@ void RemoteSource::onCancel() noexcept
void RemoteSource::onUpdatePorts()
Copy link
Member

@azat azat Dec 24, 2025

Choose a reason for hiding this comment

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

The problem is here, this function can run in parallel and can cancel connections that has been never sent, previously it was no-op and generate will still wait for the query to be sent and subsequent finish will handle it correctly, but now it leads to a situation when after this finish is no-op, and this is a problem

In long term I think this method should be removed (RemoteSource is the only caller), or made more safe @KochetovNicolai WDYT?

For now here is a fix - #93029

azat added a commit to azat/ClickHouse that referenced this pull request Dec 29, 2025
Previous fix (ClickHouse#92807) introduce another issue, due to which the connection for
distributed queries can be preserved in the pool even after some
exception (usually it is socket timeout), which leads to a reconnect in
the next query, or even query failure.

The problem was that `RemoteQueryExecutor` uses `finished` flag to
decide should it call `finish()` again in `RemoteSource`, but this
`finished` flag is also used in dtor of `RemoteQueryExecutor`, and if
the query is not finished it is aborted, which does not work with this
new logic.

Let's use `got_exception_from_replica` instead, which does not affect
dtor. And it is OK to call `finish()` again in `RemoteSource`, since
subsequent will be no-op.
azat added a commit to azat/ClickHouse that referenced this pull request Dec 29, 2025
Previous fix (ClickHouse#92807) introduce another issue, due to which the connection for
distributed queries can be preserved in the pool even after some
exception (usually it is socket timeout), which leads to a reconnect in
the next query, or even query failure.

The problem was that `RemoteQueryExecutor` uses `finished` flag to
decide should it call `finish()` again in `RemoteSource`, but this
`finished` flag is also used in dtor of `RemoteQueryExecutor`, and if
the query is not finished it is aborted, which does not work with this
new logic.

Let's use `got_exception_from_replica` instead, which does not affect
dtor. And it is OK to call `finish()` again in `RemoteSource`, since
subsequent will be no-op.
AVMusorin pushed a commit to AVMusorin/ClickHouse that referenced this pull request Jan 28, 2026
The race condition causes a crash when reading from a disconnected
connection in distributed queries.

The `in` buffer:

```
(gdb) frame 1
(gdb) print this->in
$19 = {__ptr_ = 0x0, __cntrl_ = 0x0}
(gdb) print this->connected
$18 = false
```

But the same time:

```
(gdb) frame 2
    async_callback=...) at ./ci/tmp/build/./src/Client/MultiplexedConnections.cpp:398
(gdb) print this->sent_query
$15 = true
(gdb) print this->cancelled
$16 = false
(gdb) print this->active_connection_count
$17 = 1
```

Current solution will not fix the issue, it just apply a check like in
`sendCancel()`.

Related PRs:
- ClickHouse#89740
- ClickHouse#92807
- ClickHouse#93029
batkovic75 pushed a commit to batkovic75/ClickHouse that referenced this pull request Jan 30, 2026
The race condition causes a crash when reading from a disconnected
connection in distributed queries.

The `in` buffer:

```
(gdb) frame 1
(gdb) print this->in
$19 = {__ptr_ = 0x0, __cntrl_ = 0x0}
(gdb) print this->connected
$18 = false
```

But the same time:

```
(gdb) frame 2
    async_callback=...) at ./ci/tmp/build/./src/Client/MultiplexedConnections.cpp:398
(gdb) print this->sent_query
$15 = true
(gdb) print this->cancelled
$16 = false
(gdb) print this->active_connection_count
$17 = 1
```

Current solution will not fix the issue, it just apply a check like in
`sendCancel()`.

Related PRs:
- ClickHouse#89740
- ClickHouse#92807
- ClickHouse#93029
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

pr-backports-created Backport PRs are successfully created, it won't be processed by CI script anymore pr-bugfix Pull request with bugfix, not backported by default pr-must-backport-synced The `*-must-backport` labels are synced into the cloud Sync PR pr-synced-to-cloud The PR is synced to the cloud repo v25.8-must-backport v25.10-must-backport v25.11-must-backport v25.12-must-backport

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants