mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
LimitReadBuffer
This commit is contained in:
parent
e0f083ac0b
commit
cea797e254
@ -94,8 +94,7 @@ void LocalConnection::sendProfileEvents()
|
||||
Block profile_block;
|
||||
state->after_send_profile_events.restart();
|
||||
next_packet_type = Protocol::Server::ProfileEvents;
|
||||
ProfileEvents::getProfileEvents(server_display_name, state->profile_queue, profile_block, last_sent_snapshots);
|
||||
state->block.emplace(std::move(profile_block));
|
||||
state->block.emplace(ProfileEvents::getProfileEvents(server_display_name, state->profile_queue, last_sent_snapshots));
|
||||
}
|
||||
|
||||
void LocalConnection::sendQuery(
|
||||
|
@ -51,7 +51,7 @@ StatusFile::StatusFile(std::string path_, FillFunction fill_)
|
||||
std::string contents;
|
||||
{
|
||||
ReadBufferFromFile in(path, 1024);
|
||||
LimitReadBuffer limit_in(in, 1024, /* throw_exception */ false, /* exact_limit */ {});
|
||||
LimitReadBuffer limit_in(in, {.read_no_more = 1024});
|
||||
readStringUntilEOF(contents, limit_in);
|
||||
}
|
||||
|
||||
|
@ -189,11 +189,11 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header,
|
||||
if (settings[Setting::http_max_multipart_form_data_size])
|
||||
read_buffer = std::make_unique<LimitReadBuffer>(
|
||||
stream,
|
||||
settings[Setting::http_max_multipart_form_data_size],
|
||||
/* trow_exception */ true,
|
||||
/* exact_limit */ std::optional<size_t>(),
|
||||
"the maximum size of multipart/form-data. "
|
||||
"This limit can be tuned by 'http_max_multipart_form_data_size' setting");
|
||||
LimitReadBuffer::Settings{
|
||||
.read_no_more = settings[Setting::http_max_multipart_form_data_size],
|
||||
.expect_eof = true,
|
||||
.excetion_hint = "the maximum size of multipart/form-data. This limit can be tuned by 'http_max_multipart_form_data_size' setting",
|
||||
});
|
||||
else
|
||||
read_buffer = wrapReadBufferReference(stream);
|
||||
|
||||
|
@ -33,13 +33,15 @@ void IMySQLReadPacket::readPayloadWithUnpacked(ReadBuffer & in)
|
||||
|
||||
void LimitedReadPacket::readPayload(ReadBuffer &in, uint8_t &sequence_id)
|
||||
{
|
||||
LimitReadBuffer limited(in, 10000, /* trow_exception */ true, /* exact_limit */ {}, "too long MySQL packet.");
|
||||
LimitReadBuffer limited(in, {.read_no_more = 10000, .expect_eof = true, .excetion_hint = "too long MySQL packet."});
|
||||
//10000, /* trow_exception */ true, /* exact_limit */ {}, "too long MySQL packet.");
|
||||
IMySQLReadPacket::readPayload(limited, sequence_id);
|
||||
}
|
||||
|
||||
void LimitedReadPacket::readPayloadWithUnpacked(ReadBuffer & in)
|
||||
{
|
||||
LimitReadBuffer limited(in, 10000, /* trow_exception */ true, /* exact_limit */ {}, "too long MySQL packet.");
|
||||
LimitReadBuffer limited(in,{.read_no_more = 10000, .expect_eof = true, .excetion_hint = "too long MySQL packet."});
|
||||
// 10000, /* trow_exception */ true, /* exact_limit */ {}, "too long MySQL packet.");
|
||||
IMySQLReadPacket::readPayloadWithUnpacked(limited);
|
||||
}
|
||||
|
||||
|
@ -429,7 +429,8 @@ 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, 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);
|
||||
to = event;
|
||||
|
@ -1,5 +1,8 @@
|
||||
#include <limits>
|
||||
#include <IO/LimitReadBuffer.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -19,63 +22,50 @@ bool LimitReadBuffer::nextImpl()
|
||||
/// Let underlying buffer calculate read bytes in `next()` call.
|
||||
in->position() = position();
|
||||
|
||||
if (bytes >= limit)
|
||||
if (bytes >= settings.read_no_less)
|
||||
{
|
||||
if (exact_limit && bytes == *exact_limit)
|
||||
if (settings.expect_eof && bytes > settings.read_no_more)
|
||||
throw Exception(ErrorCodes::LIMIT_EXCEEDED, "Limit for LimitReadBuffer exceeded: {}", settings.excetion_hint);
|
||||
|
||||
if (bytes >= settings.read_no_more)
|
||||
return false;
|
||||
|
||||
if (exact_limit && bytes != *exact_limit)
|
||||
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected data, got {} bytes, expected {}", bytes, *exact_limit);
|
||||
|
||||
if (throw_exception)
|
||||
throw Exception(ErrorCodes::LIMIT_EXCEEDED, "Limit for LimitReadBuffer exceeded: {}", exception_message);
|
||||
|
||||
return false;
|
||||
//throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected data, got {} bytes, expected {}", bytes, settings.read_atmost);
|
||||
}
|
||||
|
||||
if (!in->next())
|
||||
{
|
||||
if (exact_limit && bytes != *exact_limit)
|
||||
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected EOF, got {} of {} bytes", bytes, *exact_limit);
|
||||
if (bytes < settings.read_no_less)
|
||||
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected EOF, got {} of {} bytes", bytes, settings.read_no_less);
|
||||
|
||||
/// Clearing the buffer with existing data.
|
||||
BufferBase::set(in->position(), 0, 0);
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
BufferBase::set(in->position(), std::min(in->available(), limit - bytes), 0);
|
||||
BufferBase::set(in->position(), std::min(in->available(), getEffectiveBufferSize() - bytes), 0);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, size_t limit_, bool throw_exception_,
|
||||
std::optional<size_t> exact_limit_, std::string exception_message_)
|
||||
: ReadBuffer(in_ ? in_->position() : nullptr, 0)
|
||||
, in(in_)
|
||||
, owns_in(owns)
|
||||
, limit(limit_)
|
||||
, throw_exception(throw_exception_)
|
||||
, exact_limit(exact_limit_)
|
||||
, exception_message(std::move(exception_message_))
|
||||
LimitReadBuffer::LimitReadBuffer(ReadBuffer & in_, Settings settings_)
|
||||
: ReadBuffer(in_.position(), 0)
|
||||
, in(&in_)
|
||||
, settings(std::move(settings_))
|
||||
{
|
||||
chassert(in);
|
||||
chassert(settings.read_no_less <= settings.read_no_more);
|
||||
|
||||
BufferBase::set(in->position(), std::min(in->available(), limit), 0);
|
||||
BufferBase::set(in->position(), std::min(in->available(), getEffectiveBufferSize()), 0);
|
||||
}
|
||||
|
||||
|
||||
LimitReadBuffer::LimitReadBuffer(ReadBuffer & in_, size_t limit_, bool throw_exception_,
|
||||
std::optional<size_t> exact_limit_, std::string exception_message_)
|
||||
: LimitReadBuffer(&in_, false, limit_, throw_exception_, exact_limit_, exception_message_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
LimitReadBuffer::LimitReadBuffer(std::unique_ptr<ReadBuffer> in_, size_t limit_, bool throw_exception_,
|
||||
std::optional<size_t> exact_limit_, std::string exception_message_)
|
||||
: LimitReadBuffer(in_.release(), true, limit_, throw_exception_, exact_limit_, exception_message_)
|
||||
LimitReadBuffer::LimitReadBuffer(std::unique_ptr<ReadBuffer> in_, Settings settings_)
|
||||
: LimitReadBuffer(*in_, std::move(settings_))
|
||||
{
|
||||
holder = std::move(in_);
|
||||
}
|
||||
|
||||
|
||||
@ -84,9 +74,12 @@ LimitReadBuffer::~LimitReadBuffer()
|
||||
/// Update underlying buffer's position in case when limit wasn't reached.
|
||||
if (!working_buffer.empty())
|
||||
in->position() = position();
|
||||
|
||||
if (owns_in)
|
||||
delete in;
|
||||
}
|
||||
|
||||
size_t LimitReadBuffer::getEffectiveBufferSize() const
|
||||
{
|
||||
if (settings.read_no_less)
|
||||
return settings.read_no_less;
|
||||
return settings.read_no_more;
|
||||
}
|
||||
}
|
||||
|
@ -1,7 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#include <limits>
|
||||
#include <memory>
|
||||
#include <base/types.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include "Core/Settings.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -13,26 +16,29 @@ namespace DB
|
||||
class LimitReadBuffer : public ReadBuffer
|
||||
{
|
||||
public:
|
||||
LimitReadBuffer(ReadBuffer & in_, size_t limit_, bool throw_exception_,
|
||||
std::optional<size_t> exact_limit_, std::string exception_message_ = {});
|
||||
LimitReadBuffer(std::unique_ptr<ReadBuffer> in_, size_t limit_, bool throw_exception_, std::optional<size_t> exact_limit_,
|
||||
std::string exception_message_ = {});
|
||||
struct Settings
|
||||
{
|
||||
size_t read_no_less = 0;
|
||||
size_t read_no_more = std::numeric_limits<size_t>::max();
|
||||
bool expect_eof = false;
|
||||
std::string excetion_hint = {};
|
||||
};
|
||||
|
||||
LimitReadBuffer(ReadBuffer & in_, Settings settings);
|
||||
LimitReadBuffer(std::unique_ptr<ReadBuffer> in_, Settings settings);
|
||||
|
||||
~LimitReadBuffer() override;
|
||||
|
||||
private:
|
||||
ReadBuffer * in;
|
||||
const bool owns_in;
|
||||
std::unique_ptr<ReadBuffer> holder;
|
||||
|
||||
const size_t limit;
|
||||
const bool throw_exception;
|
||||
const std::optional<size_t> exact_limit;
|
||||
const std::string exception_message;
|
||||
|
||||
LoggerPtr log;
|
||||
const Settings settings;
|
||||
|
||||
LimitReadBuffer(ReadBuffer * in_, bool owns, size_t limit_, bool throw_exception_, std::optional<size_t> exact_limit_, std::string exception_message_);
|
||||
|
||||
bool nextImpl() override;
|
||||
size_t getEffectiveBufferSize() const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -172,6 +172,39 @@ public:
|
||||
next();
|
||||
}
|
||||
|
||||
size_t rejectDataSafe()
|
||||
{
|
||||
size_t before = count();
|
||||
if (count() == offset())
|
||||
position() -= offset();
|
||||
return before - count();
|
||||
}
|
||||
|
||||
size_t rejectDataSafe(size_t partial_data_to_reject)
|
||||
{
|
||||
size_t before = count();
|
||||
if (count() == offset() && offset() >= partial_data_to_reject)
|
||||
position() -= partial_data_to_reject;
|
||||
return before - count();
|
||||
}
|
||||
|
||||
bool rejectDataHard()
|
||||
{
|
||||
size_t before = count();
|
||||
position() -= offset();
|
||||
return before - count();
|
||||
}
|
||||
|
||||
size_t rejectDataHard(size_t partial_data_to_reject)
|
||||
{
|
||||
size_t before = count();
|
||||
if (offset() >= partial_data_to_reject)
|
||||
{
|
||||
position() -= partial_data_to_reject;
|
||||
}
|
||||
return before - count();
|
||||
}
|
||||
|
||||
protected:
|
||||
WriteBuffer(Position ptr, size_t size) : BufferBase(ptr, size, 0) {}
|
||||
|
||||
|
@ -175,7 +175,7 @@ void WriteBufferFromPocoSocketChunked::nextImpl()
|
||||
LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}",
|
||||
ourAddress().toString(), peerAddress().toString(),
|
||||
static_cast<unsigned int>(*(reinterpret_cast<char *>(chunk_size_ptr) + sizeof(*chunk_size_ptr))),
|
||||
*chunk_size_ptr);
|
||||
*chunk_size_ptr);
|
||||
}
|
||||
else
|
||||
LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr);
|
||||
|
@ -381,9 +381,10 @@ AsynchronousInsertQueue::pushQueryWithInlinedData(ASTPtr query, ContextPtr query
|
||||
|
||||
LimitReadBuffer limit_buf(
|
||||
*read_buf,
|
||||
query_context->getSettingsRef()[Setting::async_insert_max_data_size],
|
||||
/*throw_exception=*/false,
|
||||
/*exact_limit=*/{});
|
||||
{.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);
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
#include "Core/Block.h"
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
@ -98,11 +99,7 @@ static void dumpMemoryTracker(ProfileEventsSnapshot const & snapshot, DB::Mutabl
|
||||
columns[i]->insert(snapshot.peak_memory_usage);
|
||||
}
|
||||
|
||||
void getProfileEvents(
|
||||
const String & host_name,
|
||||
DB::InternalProfileEventsQueuePtr profile_queue,
|
||||
DB::Block & block,
|
||||
ThreadIdToCountersSnapshot & last_sent_snapshots)
|
||||
DB::Block getSampleBlock()
|
||||
{
|
||||
using namespace DB;
|
||||
static const NamesAndTypesList column_names_and_types = {
|
||||
@ -118,8 +115,16 @@ void getProfileEvents(
|
||||
for (auto const & name_and_type : column_names_and_types)
|
||||
temp_columns.emplace_back(name_and_type.type, name_and_type.name);
|
||||
|
||||
block = std::move(temp_columns);
|
||||
MutableColumns columns = block.mutateColumns();
|
||||
return Block(std::move(temp_columns));
|
||||
}
|
||||
|
||||
DB::Block getProfileEvents(
|
||||
const String & host_name,
|
||||
DB::InternalProfileEventsQueuePtr profile_queue,
|
||||
ThreadIdToCountersSnapshot & last_sent_snapshots)
|
||||
{
|
||||
using namespace DB;
|
||||
|
||||
auto thread_group = CurrentThread::getGroup();
|
||||
ThreadIdToCountersSnapshot new_snapshots;
|
||||
|
||||
@ -139,6 +144,9 @@ void getProfileEvents(
|
||||
}
|
||||
last_sent_snapshots = std::move(new_snapshots);
|
||||
|
||||
auto block = getSampleBlock();
|
||||
MutableColumns columns = block.mutateColumns();
|
||||
|
||||
dumpProfileEvents(group_snapshot, columns, host_name);
|
||||
dumpMemoryTracker(group_snapshot, columns, host_name);
|
||||
|
||||
@ -154,6 +162,8 @@ void getProfileEvents(
|
||||
bool empty = columns[0]->empty();
|
||||
if (!empty)
|
||||
block.setColumns(std::move(columns));
|
||||
|
||||
return block;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -25,10 +25,11 @@ using ThreadIdToCountersSnapshot = std::unordered_map<UInt64, Counters::Snapshot
|
||||
/// Dumps profile events to columns Map(String, UInt64)
|
||||
void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, bool nonzero_only = true);
|
||||
|
||||
void getProfileEvents(
|
||||
DB::Block getSampleBlock();
|
||||
|
||||
DB::Block getProfileEvents(
|
||||
const String & host_name,
|
||||
DB::InternalProfileEventsQueuePtr profile_queue,
|
||||
DB::Block & block,
|
||||
ThreadIdToCountersSnapshot & last_sent_snapshots);
|
||||
|
||||
/// This is for ProfileEvents packets.
|
||||
|
@ -1626,7 +1626,7 @@ void executeQuery(
|
||||
|
||||
/// If not - copy enough data into 'parse_buf'.
|
||||
WriteBufferFromVector<PODArray<char>> out(parse_buf);
|
||||
LimitReadBuffer limit(istr, max_query_size + 1, /* trow_exception */ false, /* exact_limit */ {});
|
||||
LimitReadBuffer limit(istr, {.read_no_more = max_query_size + 1});
|
||||
copyData(limit, out);
|
||||
out.finalize();
|
||||
|
||||
|
@ -71,23 +71,11 @@ void BlockIO::onException()
|
||||
void BlockIO::onCancelOrConnectionLoss()
|
||||
{
|
||||
/// Query was not finished gracefully, so we should call exception_callback
|
||||
/// But we don't have a real exception
|
||||
try
|
||||
{
|
||||
throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled or a client has unexpectedly dropped the connection");
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
if (exception_callback)
|
||||
{
|
||||
exception_callback(/* log_error */ false);
|
||||
}
|
||||
|
||||
/// destroy pipeline and write buffers with an exception context
|
||||
pipeline.cancel();
|
||||
pipeline.reset();
|
||||
}
|
||||
if (exception_callback)
|
||||
exception_callback(/* log_error */ false);
|
||||
|
||||
pipeline.cancel();
|
||||
pipeline.reset();
|
||||
}
|
||||
|
||||
void BlockIO::setAllDataSent() const
|
||||
|
@ -60,8 +60,7 @@ HTTPServerRequest::HTTPServerRequest(HTTPContextPtr context, HTTPServerResponse
|
||||
else if (hasContentLength())
|
||||
{
|
||||
size_t content_length = getContentLength();
|
||||
stream = std::make_unique<LimitReadBuffer>(std::move(in), content_length,
|
||||
/* trow_exception */ true, /* exact_limit */ content_length);
|
||||
stream = std::make_unique<LimitReadBuffer>(std::move(in), LimitReadBuffer::Settings{.read_no_less = content_length, .read_no_more = content_length, .expect_eof = true});
|
||||
}
|
||||
else if (getMethod() != HTTPRequest::HTTP_GET && getMethod() != HTTPRequest::HTTP_HEAD && getMethod() != HTTPRequest::HTTP_DELETE)
|
||||
{
|
||||
|
@ -257,17 +257,11 @@ bool WriteBufferFromHTTPServerResponse::cancelWithException(HTTPServerRequest &
|
||||
{
|
||||
data_sent |= (compression_buffer->count() != compression_buffer->offset());
|
||||
if (!data_sent)
|
||||
{
|
||||
compression_discarded_data = std::distance(compression_buffer->buffer().begin(), compression_buffer->position());
|
||||
compression_buffer->position() = compression_buffer->buffer().begin();
|
||||
}
|
||||
compression_discarded_data = compression_buffer->rejectDataSafe();
|
||||
}
|
||||
data_sent |= (count() != offset());
|
||||
if (!data_sent)
|
||||
{
|
||||
discarded_data = std::distance(buffer().begin(), position());
|
||||
position() = buffer().begin();
|
||||
}
|
||||
discarded_data = rejectDataSafe();
|
||||
|
||||
bool is_response_sent = response.sent();
|
||||
// proper senging bad http code
|
||||
|
@ -275,7 +275,7 @@ void MySQLHandler::run()
|
||||
payload.readStrict(command);
|
||||
|
||||
// For commands which are executed without MemoryTracker.
|
||||
LimitReadBuffer limited_payload(payload, 10000, /* trow_exception */ true, /* exact_limit */ {}, "too long MySQL packet.");
|
||||
LimitReadBuffer limited_payload(payload, {.read_no_more = 1000, .expect_eof = true, .excetion_hint = "too long MySQL packet."});
|
||||
|
||||
LOG_DEBUG(log, "Received command: {}. Connection id: {}.",
|
||||
static_cast<int>(static_cast<unsigned char>(command)), connection_id);
|
||||
|
@ -1361,8 +1361,7 @@ void TCPHandler::sendExtremes(const Block & extremes)
|
||||
void TCPHandler::sendProfileEvents()
|
||||
{
|
||||
Stopwatch stopwatch;
|
||||
Block block;
|
||||
ProfileEvents::getProfileEvents(host_name, state.profile_queue, block, last_sent_snapshots);
|
||||
Block block = ProfileEvents::getProfileEvents(host_name, state.profile_queue, last_sent_snapshots);
|
||||
if (block.rows() != 0)
|
||||
{
|
||||
initProfileEventsBlockOutput(block);
|
||||
@ -1429,7 +1428,7 @@ bool TCPHandler::receiveProxyHeader()
|
||||
/// Only PROXYv1 is supported.
|
||||
/// Validation of protocol is not fully performed.
|
||||
|
||||
LimitReadBuffer limit_in(*in, 107, /* trow_exception */ true, /* exact_limit */ {}); /// Maximum length from the specs.
|
||||
LimitReadBuffer limit_in(*in, {.read_no_more=107, .expect_eof=true}); /// Maximum length from the specs.
|
||||
|
||||
assertString("PROXY ", limit_in);
|
||||
|
||||
|
@ -134,7 +134,7 @@ private:
|
||||
|
||||
if (limited_by_file_size)
|
||||
{
|
||||
limited.emplace(*plain, file_size - offset, /* trow_exception */ false, /* exact_limit */ std::optional<size_t>());
|
||||
limited.emplace(*plain, LimitReadBuffer::Settings{.read_no_more = file_size - offset});
|
||||
compressed.emplace(*limited);
|
||||
}
|
||||
else
|
||||
|
Loading…
Reference in New Issue
Block a user