mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-09-29 05:00:47 +00:00
better
This commit is contained in:
parent
8b17ce492f
commit
c7084ff5dc
@ -11,6 +11,7 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction()
|
||||
while (!is_cancelled && !is_exception_occured)
|
||||
{
|
||||
++segmentator_ticket_number;
|
||||
// std::cout << "SEGMENATATING " << segmentator_ticket_number << std::endl;
|
||||
const auto current_unit_number = segmentator_ticket_number % max_threads_to_use;
|
||||
|
||||
{
|
||||
@ -21,9 +22,20 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction()
|
||||
if (is_exception_occured)
|
||||
break;
|
||||
|
||||
if (original_buffer.eof())
|
||||
{
|
||||
// std::cout << "ORIGINAL BUFFER EOF" << std::endl;
|
||||
is_last[current_unit_number] = true;
|
||||
status[current_unit_number] = READY_TO_PARSE;
|
||||
scheduleParserThreadForUnitWithNumber(current_unit_number);
|
||||
break;
|
||||
}
|
||||
|
||||
// Segmentating the original input.
|
||||
segments[current_unit_number].used_size = 0;
|
||||
bool has_data = file_segmentation_engine(original_buffer, segments[current_unit_number].memory, segments[current_unit_number].used_size, min_chunk_size);
|
||||
|
||||
//It returns bool, but it is useless
|
||||
file_segmentation_engine(original_buffer, segments[current_unit_number].memory, segments[current_unit_number].used_size, min_chunk_size);
|
||||
|
||||
// Creating buffer from the segment of data.
|
||||
auto new_buffer = BufferBase::Buffer(segments[current_unit_number].memory.data(),
|
||||
@ -32,13 +44,14 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction()
|
||||
buffers[current_unit_number]->buffer().swap(new_buffer);
|
||||
buffers[current_unit_number]->position() = buffers[current_unit_number]->buffer().begin();
|
||||
|
||||
if (!has_data)
|
||||
{
|
||||
is_last[current_unit_number] = true;
|
||||
status[current_unit_number] = READY_TO_PARSE;
|
||||
scheduleParserThreadForUnitWithNumber(current_unit_number);
|
||||
break;
|
||||
}
|
||||
// std::cout << "current_unit_number " << current_unit_number << std::endl;
|
||||
|
||||
readers[current_unit_number] = std::make_unique<InputStreamFromInputFormat>(input_processor_creator(
|
||||
*buffers[current_unit_number],
|
||||
header,
|
||||
context,
|
||||
row_input_format_params,
|
||||
format_settings));
|
||||
|
||||
status[current_unit_number] = READY_TO_PARSE;
|
||||
scheduleParserThreadForUnitWithNumber(current_unit_number);
|
||||
@ -62,12 +75,13 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
|
||||
blocks[current_unit_number].block.resize(0);
|
||||
blocks[current_unit_number].block_missing_values.resize(0);
|
||||
blocks[current_unit_number].block.clear();
|
||||
blocks[current_unit_number].block_missing_values.clear();
|
||||
|
||||
if (is_last[current_unit_number] || buffers[current_unit_number]->position() == nullptr)
|
||||
{
|
||||
blocks[current_unit_number].block.push_back(Block());
|
||||
blocks[current_unit_number].block.emplace_back(Block());
|
||||
blocks[current_unit_number].block_missing_values.emplace_back(BlockMissingValues());
|
||||
status[current_unit_number] = READY_TO_READ;
|
||||
reader_condvar.notify_all();
|
||||
return;
|
||||
@ -78,15 +92,20 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n
|
||||
|
||||
while (true)
|
||||
{
|
||||
// std::cout << "CURRENT_UNIT_NUMBER " << current_unit_number << " " << blocks[current_unit_number].block.size() << std::endl;
|
||||
auto block = readers[current_unit_number]->read();
|
||||
|
||||
blocks[current_unit_number].block.push_back(block);
|
||||
blocks[current_unit_number].block_missing_values.push_back(readers[current_unit_number]->getMissingValues());
|
||||
|
||||
if (block == Block())
|
||||
{
|
||||
// std::cout << "EMPTY BLOCK" << std::endl;
|
||||
break;
|
||||
}
|
||||
|
||||
blocks[current_unit_number].block.emplace_back(block);
|
||||
blocks[current_unit_number].block_missing_values.emplace_back(readers[current_unit_number]->getMissingValues());
|
||||
// std::cout << "rows " << blocks[current_unit_number].block[0].rows() << std::endl;
|
||||
}
|
||||
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
status[current_unit_number] = READY_TO_READ;
|
||||
@ -123,7 +142,11 @@ Block ParallelParsingBlockInputStream::readImpl()
|
||||
rethrowFirstException(exceptions);
|
||||
}
|
||||
|
||||
res = std::move(blocks[current_number].block[internal_block_iter]);
|
||||
// std::cout << "blocks size: " << blocks[current_number].block.size() << std::endl;
|
||||
// std::cout << "number and iter " << current_number << " " << internal_block_iter << std::endl;
|
||||
res = std::move(blocks[current_number].block.at(internal_block_iter));
|
||||
|
||||
// std::cout << "missing values size: " << blocks[current_number].block_missing_values.size() << std::endl;
|
||||
last_block_missing_values = std::move(blocks[current_number].block_missing_values[internal_block_iter]);
|
||||
|
||||
if (++internal_block_iter == blocks[current_number].block.size())
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <Processors/Formats/IRowInputFormat.h>
|
||||
#include <Processors/Formats/InputStreamFromInputFormat.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -44,8 +45,13 @@ public:
|
||||
size_t min_chunk_size;
|
||||
};
|
||||
|
||||
ParallelParsingBlockInputStream(Builder builder)
|
||||
: max_threads_to_use(builder.max_threads_to_use),
|
||||
explicit ParallelParsingBlockInputStream(const Builder & builder)
|
||||
: header(builder.input_creator_params.sample),
|
||||
context(builder.input_creator_params.context),
|
||||
row_input_format_params(builder.input_creator_params.row_input_format_params),
|
||||
format_settings(builder.input_creator_params.settings),
|
||||
input_processor_creator(builder.input_processor_creator),
|
||||
max_threads_to_use(builder.max_threads_to_use),
|
||||
min_chunk_size(builder.min_chunk_size),
|
||||
original_buffer(builder.read_buffer),
|
||||
pool(builder.max_threads_to_use),
|
||||
@ -55,7 +61,7 @@ public:
|
||||
blocks.resize(max_threads_to_use);
|
||||
exceptions.resize(max_threads_to_use);
|
||||
buffers.reserve(max_threads_to_use);
|
||||
readers.reserve(max_threads_to_use);
|
||||
readers.resize(max_threads_to_use);
|
||||
is_last.assign(max_threads_to_use, false);
|
||||
|
||||
for (size_t i = 0; i < max_threads_to_use; ++i)
|
||||
@ -95,7 +101,7 @@ public:
|
||||
|
||||
Block getHeader() const override
|
||||
{
|
||||
return readers.at(0)->getHeader();
|
||||
return header;
|
||||
}
|
||||
|
||||
protected:
|
||||
@ -110,6 +116,11 @@ protected:
|
||||
}
|
||||
|
||||
private:
|
||||
const Block header;
|
||||
const Context context;
|
||||
const RowInputFormatParams row_input_format_params;
|
||||
const FormatSettings format_settings;
|
||||
const InputProcessorCreator input_processor_creator;
|
||||
|
||||
const std::atomic<size_t> max_threads_to_use;
|
||||
const size_t min_chunk_size;
|
||||
|
@ -130,7 +130,7 @@ BlockInputStreamPtr FormatFactory::getInput(
|
||||
max_threads_to_use = settings.max_threads;
|
||||
|
||||
auto params = ParallelParsingBlockInputStream::InputCreatorParams{sample, context, row_input_format_params, format_settings};
|
||||
auto builder = ParallelParsingBlockInputStream::Builder{buf, input_getter, params, file_segmentation_engine, max_threads_to_use, settings.min_chunk_size_for_parallel_reading};
|
||||
ParallelParsingBlockInputStream::Builder builder{buf, input_getter, params, file_segmentation_engine, max_threads_to_use, settings.min_chunk_size_for_parallel_reading};
|
||||
return std::make_shared<ParallelParsingBlockInputStream>(builder);
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user