2022-10-28 16:41:10 +00:00
|
|
|
#include <Storages/MessageQueueSink.h>
|
|
|
|
#include <Formats/FormatFactory.h>
|
|
|
|
#include <Processors/Formats/IRowOutputFormat.h>
|
2023-02-10 17:14:49 +00:00
|
|
|
#include <Common/logger_useful.h>
|
2022-10-28 16:41:10 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
MessageQueueSink::MessageQueueSink(
|
|
|
|
const Block & header,
|
|
|
|
const String & format_name_,
|
|
|
|
size_t max_rows_per_message_,
|
|
|
|
std::unique_ptr<IMessageProducer> producer_,
|
|
|
|
const String & storage_name_,
|
|
|
|
const ContextPtr & context_)
|
2022-12-21 10:43:48 +00:00
|
|
|
: SinkToStorage(header), format_name(format_name_), max_rows_per_message(max_rows_per_message_), producer(std::move(producer_)), storage_name(storage_name_), context(context_)
|
2022-10-28 16:41:10 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
void MessageQueueSink::onStart()
|
|
|
|
{
|
2023-02-10 17:14:49 +00:00
|
|
|
LOG_TEST(
|
2024-01-23 17:04:50 +00:00
|
|
|
getLogger("MessageQueueSink"),
|
2023-02-10 17:14:49 +00:00
|
|
|
"Executing startup for MessageQueueSink");
|
|
|
|
|
2022-10-28 16:41:10 +00:00
|
|
|
initialize();
|
2022-12-21 10:43:48 +00:00
|
|
|
producer->start(context);
|
2022-10-28 16:41:10 +00:00
|
|
|
|
|
|
|
buffer = std::make_unique<WriteBufferFromOwnString>();
|
|
|
|
|
2022-12-21 10:43:48 +00:00
|
|
|
auto format_settings = getFormatSettings(context);
|
2022-10-28 16:41:10 +00:00
|
|
|
format_settings.protobuf.allow_multiple_rows_without_delimiter = true;
|
|
|
|
|
2022-12-21 10:43:48 +00:00
|
|
|
format = FormatFactory::instance().getOutputFormat(format_name, *buffer, getHeader(), context, format_settings);
|
2022-10-28 16:41:10 +00:00
|
|
|
row_format = dynamic_cast<IRowOutputFormat *>(format.get());
|
|
|
|
}
|
|
|
|
|
|
|
|
void MessageQueueSink::onFinish()
|
|
|
|
{
|
|
|
|
producer->finish();
|
|
|
|
}
|
|
|
|
|
|
|
|
void MessageQueueSink::consume(Chunk chunk)
|
|
|
|
{
|
|
|
|
const auto & columns = chunk.getColumns();
|
|
|
|
if (columns.empty())
|
|
|
|
return;
|
|
|
|
|
|
|
|
if (row_format)
|
|
|
|
{
|
|
|
|
size_t row = 0;
|
|
|
|
while (row < chunk.getNumRows())
|
|
|
|
{
|
2022-12-15 19:47:10 +00:00
|
|
|
row_format->writePrefixIfNeeded();
|
2022-10-28 16:41:10 +00:00
|
|
|
size_t i = 0;
|
|
|
|
for (; i < max_rows_per_message && row < chunk.getNumRows(); ++i, ++row)
|
|
|
|
{
|
|
|
|
if (i != 0)
|
|
|
|
row_format->writeRowBetweenDelimiter();
|
|
|
|
row_format->writeRow(columns, row);
|
|
|
|
}
|
|
|
|
row_format->finalize();
|
|
|
|
row_format->resetFormatter();
|
|
|
|
producer->produce(buffer->str(), i, columns, row - 1);
|
|
|
|
/// Reallocate buffer if it's capacity is large then DBMS_DEFAULT_BUFFER_SIZE,
|
|
|
|
/// because most likely in this case we serialized abnormally large row
|
|
|
|
/// and won't need this large allocated buffer anymore.
|
|
|
|
buffer->restart(DBMS_DEFAULT_BUFFER_SIZE);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
format->write(getHeader().cloneWithColumns(chunk.detachColumns()));
|
|
|
|
format->finalize();
|
|
|
|
producer->produce(buffer->str(), chunk.getNumRows(), columns, chunk.getNumRows() - 1);
|
|
|
|
format->resetFormatter();
|
|
|
|
buffer->restart();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
}
|