This commit is contained in:
Nikita Mikhaylov 2019-10-24 17:00:51 +03:00
parent 8b17ce492f
commit c7084ff5dc
3 changed files with 54 additions and 20 deletions

View File

@ -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,13 +92,18 @@ 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;
}
{
@ -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())

View File

@ -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;

View File

@ -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);
}