fix style and review notes

This commit is contained in:
Sema Checherinda 2024-10-04 11:13:03 +02:00
parent 1e49313ed6
commit bb0b573193
12 changed files with 13 additions and 21 deletions

View File

@ -401,7 +401,6 @@ protected:
std::atomic_bool cancelled_printed = false;
/// Unpacked descriptors and streams for the ease of use.
int err_fd = STDERR_FILENO;
std::istream & input_stream;
std::ostream & output_stream;
std::ostream & error_stream;

View File

@ -429,7 +429,6 @@ bool BinlogFromFile::tryReadEvent(BinlogEventPtr & to, UInt64 /*ms*/)
EventHeader event_header;
event_header.parse(*in);
//LimitReadBuffer limit_read_buffer(*in, event_header.event_size - EVENT_HEADER_LENGTH, /* throw_exception */ false, /* exact_limit */ {});
LimitReadBuffer limit_read_buffer(*in, {.read_no_more = event_header.event_size - EVENT_HEADER_LENGTH});
MySQLBinlogEventReadBuffer event_payload(limit_read_buffer, checksum_signature_length);
parseEvent(event_header, event_payload);

View File

@ -30,7 +30,7 @@ int main(int argc, char ** argv)
writeCString("\n--- second ---\n", out);
{
LimitReadBuffer limit_in( in,{.read_no_more=limit});
LimitReadBuffer limit_in(in,{.read_no_more=limit});
copyData(limit_in, out);
}

View File

@ -382,9 +382,6 @@ AsynchronousInsertQueue::pushQueryWithInlinedData(ASTPtr query, ContextPtr query
LimitReadBuffer limit_buf(
*read_buf,
{.read_no_more = query_context->getSettingsRef()[Setting::async_insert_max_data_size]});
// query_context->getSettingsRef()[Setting::async_insert_max_data_size],
// /*throw_exception=*/false,
// /*exact_limit=*/{});
{
WriteBufferFromString write_buf(bytes);

View File

@ -131,7 +131,7 @@ public:
void setPlain(size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE)
{
if (count() > 0)
if (count() > 0 && count() != offset())
throw Exception(ErrorCodes::LOGICAL_ERROR, "can't do setPlain on HTTPWriteBuffer, buffer is already in use");
chunked = false;

View File

@ -267,7 +267,7 @@ bool WriteBufferFromHTTPServerResponse::cancelWithException(HTTPServerRequest &
// proper senging bad http code
if (!is_response_sent)
{
drainRequstIfNeded(request, response);
drainRequestIfNeeded(request, response);
// We try to send the exception message when the transmission has not been started yet
// Set HTTP code and HTTP message. Add "X-ClickHouse-Exception-Code" header.
// If it is not HEAD request send the message in the body.

View File

@ -7,7 +7,7 @@
namespace DB
{
void drainRequstIfNeded(HTTPServerRequest & request, HTTPServerResponse & response) noexcept
void drainRequestIfNeeded(HTTPServerRequest & request, HTTPServerResponse & response) noexcept
{
/// If HTTP method is POST and Keep-Alive is turned on, we should try to read the whole request body
/// to avoid reading part of the current request body in the next request.

View File

@ -5,6 +5,6 @@ namespace DB
class HTTPServerRequest;
class HTTPServerResponse;
void drainRequstIfNeded(HTTPServerRequest & request, HTTPServerResponse & response) noexcept;
void drainRequestIfNeeded(HTTPServerRequest & request, HTTPServerResponse & response) noexcept;
}

View File

@ -544,7 +544,7 @@ void HTTPHandler::processQuery(
auto header = current_output_format.getPort(IOutputFormat::PortKind::Main).getHeader();
used_output.exception_writer = [&, format_name, header, context_, format_settings](WriteBuffer & buf, int code, const String & message)
{
drainRequstIfNeded(request, response);
drainRequestIfNeeded(request, response);
used_output.out_holder->setExceptionCode(code);
auto output_format = FormatFactory::instance().getOutputFormat(format_name, buf, header, context_, format_settings);
output_format->setException(message);
@ -557,7 +557,7 @@ void HTTPHandler::processQuery(
bool with_stacktrace = (params.getParsed<bool>("stacktrace", false) && server.config().getBool("enable_http_stacktrace", true));
ExecutionStatus status = ExecutionStatus::fromCurrentException("", with_stacktrace);
drainRequstIfNeded(request, response);
drainRequestIfNeeded(request, response);
used_output.out_holder->setExceptionCode(status.code);
current_output_format.setException(status.message);
current_output_format.finalize();
@ -595,7 +595,6 @@ try
{
if (!used_output.out_holder && !used_output.exception_is_written)
{
LOG_DEBUG(getLogger("trySendExceptionToClient"), "1");
/// If nothing was sent yet and we don't even know if we must compress the response.
auto wb = WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD);
return wb.cancelWithException(request, exception_code, message, nullptr);

View File

@ -78,7 +78,8 @@ void MergeTreeDataPartWriterOnDisk::Stream<only_plain_file>::cancel() noexcept
}
plain_file->cancel();
marks_file->cancel();
if constexpr (!only_plain_file)
marks_file->cancel();
}
template<bool only_plain_file>
@ -584,11 +585,6 @@ void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(bool sync)
stream->sync();
}
// std::unique_ptr<WriteBufferFromFileBase> index_file_stream;
// std::unique_ptr<HashingWriteBuffer> index_file_hashing_stream;
// std::unique_ptr<CompressedWriteBuffer> index_compressor_stream;
// std::unique_ptr<HashingWriteBuffer> index_source_hashing_stream;
for (auto & store: gin_index_stores)
store.second->finalize();

View File

@ -1,9 +1,10 @@
-- { echoOn }
select table, errorCodeToName(error), * from system.part_log where
select table, errorCodeToName(error), count() from system.part_log where
database = currentDatabase()
and error > 0
and errorCodeToName(error) not in ('FAULT_INJECTED', 'NO_REPLICA_HAS_PART', 'ATTEMPT_TO_READ_AFTER_EOF')
and (errorCodeToName(error) != 'POCO_EXCEPTION' or exception not like '%Malformed message: Unexpected EOF%')
group by 1, 2
order by 1, 2;
select count() from data_r1;
100000

View File

@ -15,11 +15,12 @@ SET max_rows_to_read = 0; -- system.text_log can be really big
select event_time_microseconds, logger_name, message from system.text_log where level = 'Error' and message like '%Malformed chunked encoding%' order by 1 format LineAsString;
-- { echoOn }
select table, errorCodeToName(error), * from system.part_log where
select table, errorCodeToName(error), count() from system.part_log where
database = currentDatabase()
and error > 0
and errorCodeToName(error) not in ('FAULT_INJECTED', 'NO_REPLICA_HAS_PART', 'ATTEMPT_TO_READ_AFTER_EOF')
and (errorCodeToName(error) != 'POCO_EXCEPTION' or exception not like '%Malformed message: Unexpected EOF%')
group by 1, 2
order by 1, 2;
select count() from data_r1;