Skip to content

fix server HTTP keep-alive#81595

Merged
CheSema merged 6 commits intomasterfrom
chesema-fix-server-keepalive
Jun 11, 2025
Merged

fix server HTTP keep-alive#81595
CheSema merged 6 commits intomasterfrom
chesema-fix-server-keepalive

Conversation

@CheSema
Copy link
Member

@CheSema CheSema commented Jun 10, 2025

Previously, the error Invalid HTTP version string: /?query=I might have occurred while inserting some data over HTTP. This happened because ClickHouse didn't fully consume the input bytes (\r\n0\r\n\r\n) and reused the connection for the subsequent requests. This behavior is now fixed.

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 to CHANGELOG.md):

Server should not preserve a HTTP connection if the request has not been fully read from the socket.

Documentation entry for user-facing changes

  • Documentation is written (mandatory for new features)

@clickhouse-gh
Copy link
Contributor

clickhouse-gh bot commented Jun 10, 2025

Workflow [PR], commit [8969007]

@clickhouse-gh clickhouse-gh bot added the pr-bugfix Pull request with bugfix, not backported by default label Jun 10, 2025
@CheSema
Copy link
Member Author

CheSema commented Jun 10, 2025

also I borrowed tests from #81561
Thnx @al13n321

and from #81461
Thnx @nikitamikhaylov

@CheSema CheSema force-pushed the chesema-fix-server-keepalive branch from 149f268 to 4966edb Compare June 10, 2025 15:28
@CheSema CheSema assigned CheSema, al13n321, vdimir and Algunenano and unassigned CheSema Jun 10, 2025
@CheSema
Copy link
Member Author

CheSema commented Jun 10, 2025

I forced the assignment list.
Could we all please check my changes just for sure, please?

@CheSema CheSema marked this pull request as draft June 10, 2025 15:48
@CheSema
Copy link
Member Author

CheSema commented Jun 10, 2025

Some fast tests are failed. PR is in draft until I fix them

memory.resize(chunk_size);
in->readStrict(memory.data(), chunk_size);
working_buffer = Buffer(memory.data(), memory.data() + chunk_size);
in.reset();
Copy link
Member Author

Choose a reason for hiding this comment

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

If the stream is broken, we should not read from it anymore.

{
/// We have to distinguish empty buffer and nullptr.
stream = std::make_unique<EmptyReadBuffer>();
stream_is_bounded = true;
Copy link
Member Author

@CheSema CheSema Jun 10, 2025

Choose a reason for hiding this comment

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

In the first iteration I missed this line. All GET requests have been closed after response.

return stream;
}

std::pair<std::shared_ptr<WriteBuffer>, std::shared_ptr<WriteBuffer>> HTTPServerResponse::beginSend()
Copy link
Member Author

Choose a reason for hiding this comment

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

That was an unused piece of code.

@CheSema CheSema marked this pull request as ready for review June 10, 2025 18:16
setKeepAlive(false);
// Send header
Poco::Net::HTTPHeaderOutputStream hs(session);
write(hs);
Copy link
Member Author

Choose a reason for hiding this comment

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

it is HTTPResponse::write method. It is not virtual method.
It calls HTTPResponse::beginWrite method.

But HTTPServerResponse has its own beginWrite method. That method should be used here.
It is ugly. May be it is better to make a refactoring here without backporting it to the releases branches.

bool canKeepAlive() const
{
if (stream && stream_is_bounded)
return stream->eof();
Copy link
Member

Choose a reason for hiding this comment

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

This is not enough: it takes care of chunked encoding end marker, but not compression end marker. (stream doesn't know about compression, HTTPHandler adds decompressing buffer in_post_maybe_compressed over it.)

The way to test it in test_http_connection_drain_before_reuse is to add lz4 compression similar to the kotlin test, making sure the final \0\0\0\0 are in their own chunked-encoding chunk, and sleep for a second before sending that chunk. (Make sure the compressed data is bigger than max_query_size = 256 KiB.) (I guess make make_arrow_stream_data use incompressible random data instead of repeated b"xxxxxxxx", then lz4-compress it in python, split off the last 4 bytes, and yield them in a separate chunk from yield_then_sleep-like generator?) (After that, I think we wouldn't need to commit the kotlin test, test_http_connection_drain_before_reuse would cover all known problems that that test found.)

To fix it, I'm not necessarily suggesting draining in_post_maybe_compressed like in #81561 . We can instead say that IInputFormat should drain the input till eof(); most formats do that anyway, ArrowStream might be the only one that needs a change.

Copy link
Member Author

@CheSema CheSema Jun 11, 2025

Choose a reason for hiding this comment

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

You are right here and I'm right also.

My changes are made on the lowest level before format readers and below compression. It does not drain any data that is left, it only check that data is fully read and consumes final empty chunk of transfer chunk encoding format if necessary. So it would always work correctly no matter how format reader misbehaves. It either close connection or preserve it if request was fully read.
The case when there is compressions final block is left in the stream is considerable rare and then the connection would be closed, which is a correct behavior.

You are talking about making data reader read all data that helps in preserving the connections. Before my changes that leaded to the error, with my changes that leads to closing connection, you are proposing how to preserve it.

Comment on lines +57 to +58
if (resp.getheader('Connection').lower() == 'close'):
return
Copy link
Member

Choose a reason for hiding this comment

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

Don't we want to check that keepalive is working here? Seems important since we don't even print a warning when connection is closed because of leftover data. How else would we notice if the canKeepAlive() mechanism sometimes unnecessarily closes connections?

Copy link
Member Author

@CheSema CheSema Jun 11, 2025

Choose a reason for hiding this comment

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

You are right. I think I change it while I was debuggin tests on the first commit when I missed stream_is_bounded = true; for get requests.
After I found the bug I forgot to remove this lines.

Comment on lines +57 to +58
if (resp.getheader('Connection').lower() == 'close'):
return
Copy link
Member

@al13n321 al13n321 Jun 10, 2025

Choose a reason for hiding this comment

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

Hold on, I ran this test a few times, and this seems to just always get a 'close'. In the HTTPServerRequest::canKeepAlive() call, stream has > 200 KB available() in the buffer. That's probably because reading is done through a wrapper around stream (without compression it's a ReadBufferWrapper), which doesn't necessarily propagate position() updates back to stream. I.e. in_post_maybe_compressed's position() was moved to the end the buffer, but this change will only be propagated to the inner buffer (stream) when in_post_maybe_compressed->nextImpl() is called.

Copy link
Member

Choose a reason for hiding this comment

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

So I guess what we'll do is just rely on IInputFormat draining to eof() and not try to do anything else to separately drain the chunked encoding tail in particular.

I'll add eof() in ArrowBlockInputFormat and remove half of the long comment in allowKeepAliveIFFRequestIsFullyRead().

Copy link
Member

@al13n321 al13n321 left a comment

Choose a reason for hiding this comment

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

Looks good to me. Before merging, plz review my commit and consider removing the kotlin test.

@al13n321
Copy link
Member

Failed test 02769_parallel_replicas_unavailable_shards is flaky: #81628

@CheSema
Copy link
Member Author

CheSema commented Jun 11, 2025

Looks good to me. Before merging, plz review my commit and consider removing the kotlin test.

Why should we remove it?
I do not mind to leave it to work.

Your changes are OK for me.

@CheSema CheSema added this pull request to the merge queue Jun 11, 2025
@CheSema CheSema removed this pull request from the merge queue due to a manual request Jun 11, 2025
@CheSema CheSema added this pull request to the merge queue Jun 11, 2025
Merged via the queue into master with commit e714aeb Jun 11, 2025
119 of 121 checks passed
@CheSema CheSema deleted the chesema-fix-server-keepalive branch June 11, 2025 11:26
@CheSema CheSema added the pr-must-backport Pull request should be backported intentionally. Use this label with great care! label Jun 11, 2025
@robot-ch-test-poll1 robot-ch-test-poll1 added the pr-synced-to-cloud The PR is synced to the cloud repo label Jun 11, 2025
@robot-clickhouse-ci-2 robot-clickhouse-ci-2 added the pr-backports-created Backport PRs are successfully created, it won't be processed by CI script anymore label Jun 11, 2025
@robot-ch-test-poll3 robot-ch-test-poll3 added the pr-backports-created-cloud deprecated label, NOOP label Jun 11, 2025
@m0r0zk01
Copy link

m0r0zk01 commented Jun 12, 2025

@CheSema @al13n321 Hi!

Could you please explain why not just use something like that at the end of HTTPHandler::processQuery:

    size_t ignored = in->tryIgnore(std::numeric_limits<size_t>::max());
    if (ignored > 0)
    {
#if defined(DEBUG_OR_SANITIZER_BUILD)
        throw Exception(ErrorCodes::LOGICAL_ERROR,
            "HTTP request body was not fully read. Bytes ignored: {}", ignored);
#else
        LOG_WARNING(log, "HTTP request body was not fully read. Bytes ignored: {}", ignored);
#endif
    }

Or we could add drainRequestIfNeeded(request, response); right after handler->handleRequest(request, response); in the HTTPServerConnection::run (to cover all handlers, not just HTTPHandler)

Or even use both

Then all other changes become unnecessary

It passes all of the new tests in test_http_connection_drain_before_reuse

The reason I am asking is because I am currently merging these changes into my PR: #80141 and some of them look redundant

@CheSema
Copy link
Member Author

CheSema commented Jun 13, 2025

Sema Checherinda Michael Kolupaev Hi!

Could you please explain why not just use something like that at the end of HTTPHandler::processQuery:

    size_t ignored = in->tryIgnore(std::numeric_limits<size_t>::max());
    if (ignored > 0)
    {
#if defined(DEBUG_OR_SANITIZER_BUILD)
        throw Exception(ErrorCodes::LOGICAL_ERROR,
            "HTTP request body was not fully read. Bytes ignored: {}", ignored);
#else
        LOG_WARNING(log, "HTTP request body was not fully read. Bytes ignored: {}", ignored);
#endif
    }

Or we could add drainRequestIfNeeded(request, response); right after handler->handleRequest(request, response); in the HTTPServerConnection::run (to cover all handlers, not just HTTPHandler)

Or even use both

Then all other changes become unnecessary

It passes all of the new tests in test_http_connection_drain_before_reuse

The reason I am asking is because I am currently merging these changes into my PR: #80141 and some of them look redundant

Read this #81595 (comment)

HTTPHandler::processQuery is not a right place to check HTTP protocol invariants. It should be done on the level HTTPServerRequest, HTTPServerResponse. No mater how our HTTP handler misbehave or how even deeper processQuery function misbehave, HTTP protocol has to be correct.
Right now we found one format reader which leaves the data in request, but how many other format reader or future format readers would have such flaw?

drainRequestIfNeeded -- should not be called on green path. On green path we understand what the data we are reading and how much. This function exists only for being able to send exception when not all request data are read. Because the client would not even read the answer till the request is sent.

@m0r0zk01
Copy link

m0r0zk01 commented Jun 13, 2025

Sema Checherinda Michael Kolupaev Hi!
Could you please explain why not just use something like that at the end of HTTPHandler::processQuery:

    size_t ignored = in->tryIgnore(std::numeric_limits<size_t>::max());
    if (ignored > 0)
    {
#if defined(DEBUG_OR_SANITIZER_BUILD)
        throw Exception(ErrorCodes::LOGICAL_ERROR,
            "HTTP request body was not fully read. Bytes ignored: {}", ignored);
#else
        LOG_WARNING(log, "HTTP request body was not fully read. Bytes ignored: {}", ignored);
#endif
    }

Or we could add drainRequestIfNeeded(request, response); right after handler->handleRequest(request, response); in the HTTPServerConnection::run (to cover all handlers, not just HTTPHandler)
Or even use both
Then all other changes become unnecessary
It passes all of the new tests in test_http_connection_drain_before_reuse
The reason I am asking is because I am currently merging these changes into my PR: #80141 and some of them look redundant

Read this #81595 (comment)

HTTPHandler::processQuery is not a right place to check HTTP protocol invariants. It should be done on the level HTTPServerRequest, HTTPServerResponse. No mater how our HTTP handler misbehave or how even deeper processQuery function misbehave, HTTP protocol has to be correct. Right now we found one format reader which leaves the data in request, but how many other format reader or future format readers would have such flaw?

drainRequestIfNeeded -- should not be called on green path. On green path we understand what the data we are reading and how much. This function exists only for being able to send exception when not all request data are read. Because the client would not even read the answer till the request is sent.

I mostly agree about HTTPHandler::processQuery. Though ignoring the rest of the in there looks pretty much equivalent to draining the read buffer in IInputFormat?

But then there is another problem I think I've found. The eof() check is executed in HTTPServerResponse::beginWrite. But the server is allowed to start sending response before it finishes reading the whole request body. It causes some false-positive keepalive closings. As an example:

$ cat 10-million-rows-insert.txt | curl -v 'http://localhost:8123/' -H "Transfer-Encoding: chunked" --data-binary @- 2>&1 | grep 'Connection:'
< Connection: Keep-Alive

$ cat 10-million-rows-insert.txt | curl -v 'http://localhost:8123/?send_progress_in_http_headers=1' -H "Transfer-Encoding: chunked" --data-binary @- 2>&1 | grep 'Connection:'
< Connection: Close

So I still propose moving the eof() check to HTTPServerConnection::run(). On "green path" it will do nothing. But it will also detect the leftover bytes (if any) so we can handle them somehow

P.S. I am still not sure if not consuming the whole request body (!)by an HTTPRequestHandler(!) is a misbehaviour that deserves a connection close

@CheSema
Copy link
Member Author

CheSema commented Jun 13, 2025

But then there is another problem I think I've found. The eof() check is executed in HTTPServerResponse::beginWrite. But the server is allowed to start sending response before it finishes reading the whole request body. It causes some false-positive keepalive closings. As an example:

Good catch.

P.S. I am still not sure if not consuming the whole request body (!)by an HTTPRequestHandler(!) is a misbehaviour that deserves a connection close

Actually this violates HTTP protocol. send_progress_in_http_headers should send the progress while the work in the process. It should not break the HTTP protocol.
Client does not read response till the request is fully send.

We need to fix that.

@robot-clickhouse robot-clickhouse added the pr-must-backport-synced The `*-must-backport` labels are synced into the cloud Sync PR label Jul 2, 2025
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-backports-created-cloud deprecated label, NOOP pr-bugfix Pull request with bugfix, not backported by default pr-must-backport Pull request should be backported intentionally. Use this label with great care! 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

Projects

None yet

Development

Successfully merging this pull request may close these issues.

10 participants