From d47d4cd6c15a13e6ed4e888053ee878337a4e87b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 1 Oct 2019 13:48:46 +0300 Subject: [PATCH 01/90] parallel parsing --- dbms/programs/client/Client.cpp | 4 ++ dbms/src/Core/Settings.h | 4 ++ dbms/src/DataStreams/UnionBlockInputStream.h | 2 + dbms/src/Formats/FormatFactory.cpp | 55 +++++++++++++++- dbms/src/Formats/FormatFactory.h | 15 ++++- dbms/src/IO/BufferBase.h | 1 + dbms/src/IO/ReadHelpers.cpp | 19 ++++++ dbms/src/IO/ReadHelpers.h | 7 ++ .../Formats/Impl/CSVRowInputFormat.cpp | 63 ++++++++++++++++++ .../Impl/JSONEachRowRowInputFormat.cpp | 66 +++++++++++++++++++ .../Formats/Impl/TSKVRowInputFormat.cpp | 35 ++++++++++ .../Impl/TabSeparatedRowInputFormat.cpp | 42 ++++++++++++ .../Formats/Impl/ValuesRowInputFormat.cpp | 53 +++++++++++++++ .../Storages/System/StorageSystemFormats.cpp | 2 +- .../queries/0_stateless/00395_nullable.sql | 10 +-- .../0_stateless/00937_ipv4_cidr_range.sql | 2 + .../0_stateless/00938_ipv6_cidr_range.sql | 2 + 17 files changed, 373 insertions(+), 9 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index ce04561453f..3986ad6cdb0 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -1058,7 +1058,11 @@ private: /// Check if server send Exception packet auto packet_type = connection->checkPacket(); if (packet_type && *packet_type == Protocol::Server::Exception) + { + async_block_input->cancel(false); return; + } + connection->sendData(block); processed_rows += block.rows(); diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 4b0e1e7fb9e..1a5187dab4a 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -108,6 +108,10 @@ struct Settings : public SettingsCollection M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.") \ M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.") \ \ + M(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for several data formats (JSON, TSV, TKSV, Values, CSV).") \ + M(SettingUInt64, max_threads_for_parallel_reading, 10, "The maximum number of threads to parallel reading. By default, it is set to max_threads.") \ + M(SettingUInt64, min_chunk_size_for_parallel_reading, (1024 * 1024), "The minimum chunk size in bytes, which each thread tries to parse under mutex in parallel reading.") \ + \ M(SettingUInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.") \ M(SettingUInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.") \ M(SettingUInt64, merge_tree_min_rows_for_seek, 0, "You can skip reading more than that number of rows at the price of one seek per file.") \ diff --git a/dbms/src/DataStreams/UnionBlockInputStream.h b/dbms/src/DataStreams/UnionBlockInputStream.h index c4e84e85845..684892a3003 100644 --- a/dbms/src/DataStreams/UnionBlockInputStream.h +++ b/dbms/src/DataStreams/UnionBlockInputStream.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -41,6 +42,7 @@ private: }; public: + std::vector> buffers; using ExceptionCallback = std::function; UnionBlockInputStream( diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index cadf2ca2dc9..37c331c343a 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -3,8 +3,11 @@ #include #include #include +#include #include #include +#include +#include #include #include #include @@ -90,7 +93,7 @@ BlockInputStreamPtr FormatFactory::getInput( if (!getCreators(name).input_processor_creator) { - const auto & input_getter = getCreators(name).inout_creator; + const auto & input_getter = getCreators(name).input_creator; if (!input_getter) throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); @@ -100,6 +103,34 @@ BlockInputStreamPtr FormatFactory::getInput( return input_getter(buf, sample, context, max_block_size, callback ? callback : ReadCallback(), format_settings); } + const Settings & settings = context.getSettingsRef(); + const auto & file_segmentation_engine = getCreators(name).file_segmentation_engine; + + if (name != "Values" && settings.input_format_parallel_parsing && file_segmentation_engine) + { + const auto & input_getter = getCreators(name).input_processor_creator; + if (!input_getter) + throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); + + FormatSettings format_settings = getInputFormatSetting(settings); + + RowInputFormatParams row_input_format_params; + row_input_format_params.max_block_size = max_block_size; + row_input_format_params.allow_errors_num = format_settings.input_allow_errors_num; + row_input_format_params.allow_errors_ratio = format_settings.input_allow_errors_ratio; + row_input_format_params.callback = std::move(callback); + row_input_format_params.max_execution_time = settings.max_execution_time; + row_input_format_params.timeout_overflow_mode = settings.timeout_overflow_mode; + + size_t max_threads_to_use = settings.max_threads_for_parallel_reading; + if (!max_threads_to_use) + 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}; + return std::make_shared(builder); + } + auto format = getInputFormat(name, buf, sample, context, max_block_size, std::move(callback)); return std::make_shared(std::move(format)); } @@ -188,7 +219,7 @@ OutputFormatPtr FormatFactory::getOutputFormat( void FormatFactory::registerInputFormat(const String & name, InputCreator input_creator) { - auto & target = dict[name].inout_creator; + auto & target = dict[name].input_creator; if (target) throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); target = std::move(input_creator); @@ -218,6 +249,13 @@ void FormatFactory::registerOutputFormatProcessor(const String & name, OutputPro target = std::move(output_creator); } +void FormatFactory::registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine) +{ + auto & target = dict[name].file_segmentation_engine; + if (target) + throw Exception("FormatFactory: File segmentation engine " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); + target = file_segmentation_engine; +} /// Formats for both input/output. @@ -246,6 +284,14 @@ void registerOutputFormatProcessorProtobuf(FormatFactory & factory); void registerInputFormatProcessorTemplate(FormatFactory & factory); void registerOutputFormatProcessorTemplate(FormatFactory &factory); +/// File Segmentation Engines for parallel reading + +void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); +void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); +void registerFileSegmentationEngineValues(FormatFactory & factory); +void registerFileSegmentationEngineCSV(FormatFactory & factory); +void registerFileSegmentationEngineTSKV(FormatFactory & factory); + /// Output only (presentational) formats. void registerOutputFormatNull(FormatFactory & factory); @@ -298,6 +344,11 @@ FormatFactory::FormatFactory() registerInputFormatProcessorTemplate(*this); registerOutputFormatProcessorTemplate(*this); + registerFileSegmentationEngineJSONEachRow(*this); + registerFileSegmentationEngineTabSeparated(*this); + registerFileSegmentationEngineValues(*this); + registerFileSegmentationEngineCSV(*this); + registerFileSegmentationEngineTSKV(*this); registerOutputFormatNull(*this); diff --git a/dbms/src/Formats/FormatFactory.h b/dbms/src/Formats/FormatFactory.h index 1c6fbc1b97e..4e044652eb7 100644 --- a/dbms/src/Formats/FormatFactory.h +++ b/dbms/src/Formats/FormatFactory.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -41,6 +42,16 @@ public: /// It's initial purpose was to extract payload for virtual columns from Kafka Consumer ReadBuffer. using ReadCallback = std::function; + /** Fast reading data from buffer and save result to memory. + * Reads at least min_chunk_size bytes and some more until the end of the chunk, depends on the format. + * Used in SharedReadBuffer. + */ + using FileSegmentationEngine = std::function & memory, + size_t & used_size, + size_t min_chunk_size)>; + /// This callback allows to perform some additional actions after writing a single row. /// It's initial purpose was to flush Kafka message for each row. using WriteCallback = std::function; @@ -77,10 +88,11 @@ private: struct Creators { - InputCreator inout_creator; + InputCreator input_creator; OutputCreator output_creator; InputProcessorCreator input_processor_creator; OutputProcessorCreator output_processor_creator; + FileSegmentationEngine file_segmentation_engine; }; using FormatsDictionary = std::unordered_map; @@ -114,6 +126,7 @@ public: /// Register format by its name. void registerInputFormat(const String & name, InputCreator input_creator); void registerOutputFormat(const String & name, OutputCreator output_creator); + void registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine); void registerInputFormatProcessor(const String & name, InputProcessorCreator input_creator); void registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator); diff --git a/dbms/src/IO/BufferBase.h b/dbms/src/IO/BufferBase.h index c22dcbecf7b..f2191e41a0b 100644 --- a/dbms/src/IO/BufferBase.h +++ b/dbms/src/IO/BufferBase.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB diff --git a/dbms/src/IO/ReadHelpers.cpp b/dbms/src/IO/ReadHelpers.cpp index 7c0c2301c28..d7ae52cacd4 100644 --- a/dbms/src/IO/ReadHelpers.cpp +++ b/dbms/src/IO/ReadHelpers.cpp @@ -1053,4 +1053,23 @@ void skipToUnescapedNextLineOrEOF(ReadBuffer & buf) } } +bool eofWithSavingBufferState(ReadBuffer & buf, DB::Memory<> & memory, size_t & used_size, char * & begin_pos, bool save_buffer_state) +{ + if (save_buffer_state || !buf.hasPendingData()) + { + const size_t capacity = memory.size(); + const size_t block_size = static_cast(buf.position() - begin_pos); + if (capacity <= block_size + used_size) + { + memory.resize(used_size + block_size); + } + memcpy(memory.data() + used_size, begin_pos, buf.position() - begin_pos); + used_size += block_size; + bool res = buf.eof(); + begin_pos = buf.position(); + return res; + } + return false; +} + } diff --git a/dbms/src/IO/ReadHelpers.h b/dbms/src/IO/ReadHelpers.h index a5572c4df99..71a5647c469 100644 --- a/dbms/src/IO/ReadHelpers.h +++ b/dbms/src/IO/ReadHelpers.h @@ -24,6 +24,7 @@ #include #include +#include #include #include @@ -911,4 +912,10 @@ void skipToNextLineOrEOF(ReadBuffer & buf); /// Skip to next character after next unescaped \n. If no \n in stream, skip to end. Does not throw on invalid escape sequences. void skipToUnescapedNextLineOrEOF(ReadBuffer & buf); +/** Return buffer eof() result. + * If there is no pending data in buffer or it was explicitly asked + * save current state to memory. + */ +bool eofWithSavingBufferState(ReadBuffer & buf, DB::Memory<> & memory, size_t & used_size, char * & begin_pos, bool save_buffer_state = false); + } diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index b5ee30fb7f8..3393b90272f 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -433,4 +433,67 @@ void registerInputFormatProcessorCSV(FormatFactory & factory) } } +void registerFileSegmentationEngineCSV(FormatFactory & factory) +{ + factory.registerFileSegmentationEngine("CSV", []( + ReadBuffer & in, + DB::Memory<> & memory, + size_t & used_size, + size_t min_chunk_size) + { + skipWhitespacesAndTabs(in); + char * begin_pos = in.position(); + bool quotes = false; + bool need_more_data = true; + memory.resize(min_chunk_size); + while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && need_more_data) + { + if (quotes) + { + in.position() = find_first_symbols<'"'>(in.position(), in.buffer().end()); + if (in.position() == in.buffer().end()) + continue; + if (*in.position() == '"') + { + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && *in.position() == '"') + ++in.position(); + else + quotes = false; + } + } + else + { + in.position() = find_first_symbols<'"','\r', '\n'>(in.position(), in.buffer().end()); + if (in.position() == in.buffer().end()) + continue; + if (*in.position() == '"') + { + quotes = true; + ++in.position(); + } + else if (*in.position() == '\n') + { + if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) + need_more_data = false; + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && *in.position() == '\r') + ++in.position(); + } + else if (*in.position() == '\r') + { + if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) + need_more_data = false; + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && *in.position() == '\n') + ++in.position(); + } + } + } + eofWithSavingBufferState(in, memory, used_size, begin_pos, true); + return true; + }); +} + + } diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 20830d2eccf..5d14f324a2d 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -267,4 +267,70 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory) }); } +void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory) +{ + factory.registerFileSegmentationEngine("JSONEachRow", []( + ReadBuffer & in, + DB::Memory<> & memory, + size_t & used_size, + size_t min_chunk_size) + { + skipWhitespaceIfAny(in); + char * begin_pos = in.position(); + size_t balance = 0; + bool quotes = false; + memory.resize(min_chunk_size); + while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) + && (balance || used_size + static_cast(in.position() - begin_pos) < min_chunk_size)) + { + if (quotes) + { + in.position() = find_first_symbols<'\\', '"'>(in.position(), in.buffer().end()); + if (in.position() == in.buffer().end()) + continue; + if (*in.position() == '\\') + { + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos)) + ++in.position(); + } + else if (*in.position() == '"') + { + ++in.position(); + quotes = false; + } + } + else + { + in.position() = find_first_symbols<'{', '}', '\\', '"'>(in.position(), in.buffer().end()); + if (in.position() == in.buffer().end()) + continue; + if (*in.position() == '{') + { + ++balance; + ++in.position(); + } + else if (*in.position() == '}') + { + --balance; + ++in.position(); + } + else if (*in.position() == '\\') + { + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos)) + ++in.position(); + } + else if (*in.position() == '"') + { + quotes = true; + ++in.position(); + } + } + } + eofWithSavingBufferState(in, memory, used_size, begin_pos, true); + return true; + }); +} + } diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index 8cf3702d3bf..c55e64af285 100644 --- a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -205,4 +205,39 @@ void registerInputFormatProcessorTSKV(FormatFactory & factory) }); } +void registerFileSegmentationEngineTSKV(FormatFactory & factory) +{ + factory.registerFileSegmentationEngine("TSKV", []( + ReadBuffer & in, + DB::Memory<> & memory, + size_t & used_size, + size_t min_chunk_size) + { + char * begin_pos = in.position(); + bool need_more_data = true; + memory.resize(min_chunk_size); + while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && need_more_data) + { + in.position() = find_first_symbols<'\\','\r', '\n'>(in.position(), in.buffer().end()); + if (in.position() == in.buffer().end()) + continue; + if (*in.position() == '\\') + { + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos)) + ++in.position(); + } + else if (*in.position() == '\n' || *in.position() == '\r') + { + if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) + need_more_data = false; + ++in.position(); + } + } + eofWithSavingBufferState(in, memory, used_size, begin_pos, true); + return true; + }); +} + + } diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index cb9ff5b53be..5b460e91703 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -360,4 +360,46 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) } } +void registerFileSegmentationEngineTabSeparated(FormatFactory & factory) +{ + for (auto name : {"TabSeparated", "TSV"}) + { + factory.registerFileSegmentationEngine(name, []( + ReadBuffer & in, + DB::Memory<> & memory, + size_t & used_size, + size_t min_chunk_size) + { + if (in.eof()) + return false; + + char * begin_pos = in.position(); + bool need_more_data = true; + memory.resize(min_chunk_size); + while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && need_more_data) + { + in.position() = find_first_symbols<'\\', '\r', '\n'>(in.position(), in.buffer().end()); + if (in.position() == in.buffer().end()) + { + continue; + } + + if (*in.position() == '\\') + { + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos, true)) + ++in.position(); + } else if (*in.position() == '\n' || *in.position() == '\r') + { + if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) + need_more_data = false; + ++in.position(); + } + } + eofWithSavingBufferState(in, memory, used_size, begin_pos, true); + return true; + }); + } +} + } diff --git a/dbms/src/Processors/Formats/Impl/ValuesRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ValuesRowInputFormat.cpp index a0340fe4e25..7accde926b3 100644 --- a/dbms/src/Processors/Formats/Impl/ValuesRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ValuesRowInputFormat.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -162,4 +163,56 @@ void registerInputFormatProcessorValues(FormatFactory & factory) }); } +void registerFileSegmentationEngineValues(FormatFactory & factory) +{ + factory.registerFileSegmentationEngine("Values", []( + ReadBuffer & in, + DB::Memory<> & memory, + size_t & used_size, + size_t min_chunk_size) + { + skipWhitespaceIfAny(in); + if (in.eof() || *in.position() == ';') + return false; + char * begin_pos = in.position(); + int balance = 0; + bool quoted = false; + memory.resize(min_chunk_size); + while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) + && (balance || used_size + static_cast(in.position() - begin_pos) < min_chunk_size)) + { + in.position() = find_first_symbols<'\\', '\'', ')', '('>(in.position(), in.buffer().end()); + if (in.position() == in.buffer().end()) + continue; + if (*in.position() == '\\') + { + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos)) + ++in.position(); + } + else if (*in.position() == '\'') + { + quoted ^= true; + ++in.position(); + } + else if (*in.position() == ')') + { + ++in.position(); + if (!quoted) + --balance; + } + else if (*in.position() == '(') + { + ++in.position(); + if (!quoted) + ++balance; + } + } + eofWithSavingBufferState(in, memory, used_size, begin_pos, true); + if (!in.eof() && *in.position() == ',') + ++in.position(); + return true; + }); +} + } diff --git a/dbms/src/Storages/System/StorageSystemFormats.cpp b/dbms/src/Storages/System/StorageSystemFormats.cpp index 158d0a662f2..7048ab98a0d 100644 --- a/dbms/src/Storages/System/StorageSystemFormats.cpp +++ b/dbms/src/Storages/System/StorageSystemFormats.cpp @@ -21,7 +21,7 @@ void StorageSystemFormats::fillData(MutableColumns & res_columns, const Context for (const auto & pair : formats) { const auto & [format_name, creators] = pair; - UInt64 has_input_format(creators.inout_creator != nullptr || creators.input_processor_creator != nullptr); + UInt64 has_input_format(creators.input_creator != nullptr || creators.input_processor_creator != nullptr); UInt64 has_output_format(creators.output_creator != nullptr || creators.output_processor_creator != nullptr); res_columns[0]->insert(format_name); res_columns[1]->insert(has_input_format); diff --git a/dbms/tests/queries/0_stateless/00395_nullable.sql b/dbms/tests/queries/0_stateless/00395_nullable.sql index 71dc045ad09..f41e8131372 100644 --- a/dbms/tests/queries/0_stateless/00395_nullable.sql +++ b/dbms/tests/queries/0_stateless/00395_nullable.sql @@ -102,11 +102,11 @@ SELECT col1 FROM test1_00395 ORDER BY col1 ASC; SELECT '----- Insert. Source and target columns have same types up to nullability. -----'; DROP TABLE IF EXISTS test1_00395; CREATE TABLE test1_00395(col1 Nullable(UInt64), col2 UInt64) Engine=Memory; -DROP TABLE IF EXISTS test2; -CREATE TABLE test2(col1 UInt64, col2 Nullable(UInt64)) Engine=Memory; +DROP TABLE IF EXISTS test2_00395; +CREATE TABLE test2_00395(col1 UInt64, col2 Nullable(UInt64)) Engine=Memory; INSERT INTO test1_00395(col1,col2) VALUES (2,7)(6,9)(5,1)(4,3)(8,2); -INSERT INTO test2(col1,col2) SELECT col1,col2 FROM test1_00395; -SELECT col1,col2 FROM test2 ORDER BY col1,col2 ASC; +INSERT INTO test2_00395(col1,col2) SELECT col1,col2 FROM test1_00395; +SELECT col1,col2 FROM test2_00395 ORDER BY col1,col2 ASC; SELECT '----- Apply functions and aggregate functions on columns that may contain null values -----'; @@ -497,4 +497,4 @@ INSERT INTO test1_00395(col1,col2) VALUES([NULL], 'ACDEFBGH'); SELECT col1, count() FROM test1_00395 GROUP BY col1 ORDER BY col1; DROP TABLE IF EXISTS test1_00395; -DROP TABLE test2; +DROP TABLE test2_00395; diff --git a/dbms/tests/queries/0_stateless/00937_ipv4_cidr_range.sql b/dbms/tests/queries/0_stateless/00937_ipv4_cidr_range.sql index d0a04dd1cce..b79c6e3709d 100644 --- a/dbms/tests/queries/0_stateless/00937_ipv4_cidr_range.sql +++ b/dbms/tests/queries/0_stateless/00937_ipv4_cidr_range.sql @@ -1,3 +1,5 @@ +USE test; + DROP TABLE IF EXISTS ipv4_range; CREATE TABLE ipv4_range(ip IPv4, cidr UInt8) ENGINE = Memory; diff --git a/dbms/tests/queries/0_stateless/00938_ipv6_cidr_range.sql b/dbms/tests/queries/0_stateless/00938_ipv6_cidr_range.sql index 5f69710b220..653f789f4e2 100644 --- a/dbms/tests/queries/0_stateless/00938_ipv6_cidr_range.sql +++ b/dbms/tests/queries/0_stateless/00938_ipv6_cidr_range.sql @@ -1,3 +1,5 @@ +USE test; + DROP TABLE IF EXISTS ipv6_range; CREATE TABLE ipv6_range(ip IPv6, cidr UInt8) ENGINE = Memory; From d9c12f7f96f6c1e8a253c12d71304b780191bae5 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 1 Oct 2019 13:51:17 +0300 Subject: [PATCH 02/90] lost files --- .../ParallelParsingBlockInputStream.cpp | 145 +++++++++++ .../ParallelParsingBlockInputStream.h | 229 ++++++++++++++++++ dbms/src/IO/SharedReadBuffer.h | 70 ++++++ 3 files changed, 444 insertions(+) create mode 100644 dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp create mode 100644 dbms/src/DataStreams/ParallelParsingBlockInputStream.h create mode 100644 dbms/src/IO/SharedReadBuffer.h diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp new file mode 100644 index 00000000000..e7d09f0401f --- /dev/null +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -0,0 +1,145 @@ +#include + +namespace DB +{ + +void ParallelParsingBlockInputStream::segmentatorThreadFunction() +{ + setThreadName("Segmentator"); + try + { + while (!is_cancelled && !is_exception_occured) + { + ++segmentator_ticket_number; + const auto current_unit_number = segmentator_ticket_number % max_threads_to_use; + auto & current_unit = working_field[current_unit_number]; + + { + std::unique_lock lock(mutex); + segmentator_condvar.wait(lock, [&](){ return current_unit.status == READY_TO_INSERT || is_exception_occured || is_cancelled; }); + } + + if (is_exception_occured) + break; + + if (original_buffer.eof()) + { + std::unique_lock lock(mutex); + current_unit.is_last_unit = true; + current_unit.status = READY_TO_PARSE; + scheduleParserThreadForUnitWithNumber(current_unit_number); + break; + } + + // Segmentating the original input. + current_unit.chunk.used_size = 0; + bool res = file_segmentation_engine(original_buffer, current_unit.chunk.memory, current_unit.chunk.used_size, min_chunk_size); + + // Creating buffer from the segment of data. + auto new_buffer = BufferBase::Buffer(current_unit.chunk.memory.data(), current_unit.chunk.memory.data() + current_unit.chunk.used_size); + current_unit.readbuffer.buffer().swap(new_buffer); + current_unit.readbuffer.position() = current_unit.readbuffer.buffer().begin(); + + if (!res) + { + std::unique_lock lock(mutex); + current_unit.is_last_unit = true; + current_unit.status = READY_TO_PARSE; + scheduleParserThreadForUnitWithNumber(current_unit_number); + break; + } + + std::unique_lock lock(mutex); + current_unit.status = READY_TO_PARSE; + scheduleParserThreadForUnitWithNumber(current_unit_number); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +void ParallelParsingBlockInputStream::parserThreadFunction(size_t bucket_num) +{ + setThreadName("ChunkParser"); + + if (is_exception_occured && is_cancelled) + return; + + auto & current_unit = working_field[bucket_num]; + + try + { + { + std::unique_lock lock(mutex); + if (current_unit.is_last_unit || current_unit.readbuffer.position() == nullptr) + { + current_unit.block = Block(); + current_unit.status = READY_TO_READ; + reader_condvar.notify_all(); + return; + } + + } + + current_unit.block = current_unit.reader->read(); + + { + std::lock_guard missing_values_lock(missing_values_mutex); + block_missing_values = current_unit.reader->getMissingValues(); + } + + { + std::unique_lock lock(mutex); + current_unit.status = READY_TO_READ; + reader_condvar.notify_all(); + } + + } + catch (...) + { + std::unique_lock lock(mutex); + exceptions[bucket_num] = std::current_exception(); + is_exception_occured = true; + reader_condvar.notify_all(); + } +} + + +Block ParallelParsingBlockInputStream::readImpl() +{ + Block res; + if (isCancelledOrThrowIfKilled()) + return res; + + std::unique_lock lock(mutex); + + ++reader_ticket_number; + const auto unit_number = reader_ticket_number % max_threads_to_use; + auto & current_processed_unit = working_field[unit_number]; + + reader_condvar.wait(lock, [&](){ return current_processed_unit.status == READY_TO_READ || is_exception_occured || is_cancelled; }); + + /// Check for an exception and rethrow it + if (is_exception_occured) + { + segmentator_condvar.notify_all(); + lock.unlock(); + cancel(false); + rethrowFirstException(exceptions); + } + + res = std::move(current_processed_unit.block); + + if (current_processed_unit.is_last_unit) + is_cancelled = true; + else + { + current_processed_unit.status = READY_TO_INSERT; + segmentator_condvar.notify_all(); + } + return res; +} + +} diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h new file mode 100644 index 00000000000..42de8bdb023 --- /dev/null +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -0,0 +1,229 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + + +class ParallelParsingBlockInputStream : public IBlockInputStream +{ +private: + using ReadCallback = std::function; + + using InputProcessorCreator = std::function; +public: + struct InputCreatorParams + { + const Block &sample; + const Context &context; + const RowInputFormatParams& row_input_format_params; + const FormatSettings &settings; + }; + + struct Builder + { + ReadBuffer &read_buffer; + const InputProcessorCreator &input_processor_creator; + const InputCreatorParams &input_creator_params; + FormatFactory::FileSegmentationEngine file_segmentation_engine; + size_t max_threads_to_use; + size_t min_chunk_size; + }; + + ParallelParsingBlockInputStream(Builder builder) + : 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), + file_segmentation_engine(builder.file_segmentation_engine) + { + segments.resize(max_threads_to_use); + blocks.resize(max_threads_to_use); + exceptions.resize(max_threads_to_use); + buffers.reserve(max_threads_to_use); + working_field.reserve(max_threads_to_use); + + for (size_t i = 0; i < max_threads_to_use; ++i) + { + buffers.emplace_back(std::make_unique(segments[i].memory.data(), segments[i].used_size, 0)); + working_field.emplace_back(builder.input_processor_creator, builder.input_creator_params, segments[i], *buffers[i], blocks[i], exceptions[i]); + } + + segmentator_thread = ThreadFromGlobalPool([this] { segmentatorThreadFunction(); }); + } + + String getName() const override { return "ParallelParsing"; } + + ~ParallelParsingBlockInputStream() override + { + { + std::unique_lock lock(mutex); + segmentator_condvar.notify_all(); + reader_condvar.notify_all(); + } + + if (segmentator_thread.joinable()) + segmentator_thread.join(); + + try + { + pool.wait(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + void cancel(bool kill) override + { + if (kill) + is_killed = true; + bool old_val = false; + if (!is_cancelled.compare_exchange_strong(old_val, true)) + return; + + for (auto& unit: working_field) + unit.reader->cancel(kill); + + { + std::unique_lock lock(mutex); + segmentator_condvar.notify_all(); + reader_condvar.notify_all(); + } + + segmentator_thread.join(); + + try + { + pool.wait(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + } + + Block getHeader() const override + { + return working_field.at(0).reader->getHeader(); + } + +protected: + void readPrefix() override {} + + //Reader routine + Block readImpl() override; + + const BlockMissingValues & getMissingValues() const override + { + std::lock_guard missing_values_lock(missing_values_mutex); + return block_missing_values; + } + +private: + + const std::atomic max_threads_to_use; + const size_t min_chunk_size; + + std::atomic is_exception_occured{false}; + + BlockMissingValues block_missing_values; + mutable std::mutex missing_values_mutex; + + // Original ReadBuffer to read from. + ReadBuffer & original_buffer; + + //Non-atomic because it is used in one thread. + size_t reader_ticket_number{0}; + size_t segmentator_ticket_number{0}; + + std::mutex mutex; + std::condition_variable reader_condvar; + std::condition_variable segmentator_condvar; + + // There are multiple "parsers", that's why we use thread pool. + ThreadPool pool; + // Reading and segmentating the file + ThreadFromGlobalPool segmentator_thread; + + // Function to segment the file. Then "parsers" will parse that segments. + FormatFactory::FileSegmentationEngine file_segmentation_engine; + + enum ProcessingUnitStatus + { + READY_TO_INSERT, + READY_TO_PARSE, + READY_TO_READ + }; + + struct MemoryExt + { + Memory<> memory; + size_t used_size{0}; + }; + + + struct ProcessingUnit + { + ProcessingUnit(const InputProcessorCreator & input_getter, + const InputCreatorParams & params, + MemoryExt & chunk_, + ReadBuffer & readbuffer_, + Block & block_, + std::exception_ptr & exception_) : chunk(chunk_), readbuffer(readbuffer_), block(block_), exception(exception_) + { + reader = std::make_shared(input_getter(readbuffer, params.sample, params.context, params.row_input_format_params, params.settings)); + } + + MemoryExt & chunk; + ReadBuffer & readbuffer; + Block & block; + BlockInputStreamPtr reader; + + std::exception_ptr & exception; + ProcessingUnitStatus status{READY_TO_INSERT}; + bool is_last_unit{false}; + }; + + + using Blocks = std::vector; + using ReadBuffers = std::vector>; + using Segments = std::vector; + using ProcessingUnits = std::vector; + + Segments segments; + ReadBuffers buffers; + Blocks blocks; + Exceptions exceptions; + + ProcessingUnits working_field; + + void scheduleParserThreadForUnitWithNumber(size_t unit_number) + { + pool.schedule(std::bind(&ParallelParsingBlockInputStream::parserThreadFunction, this, unit_number)); + } + + void segmentatorThreadFunction(); + void parserThreadFunction(size_t bucket_num); +}; + +}; + + + diff --git a/dbms/src/IO/SharedReadBuffer.h b/dbms/src/IO/SharedReadBuffer.h new file mode 100644 index 00000000000..13c8c8522ec --- /dev/null +++ b/dbms/src/IO/SharedReadBuffer.h @@ -0,0 +1,70 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +/** Allows many threads to read from single ReadBuffer + * Each SharedReadBuffer has his own memory, + * which he filles under shared mutex using FileSegmentationEngine. + */ +class SharedReadBuffer : public BufferWithOwnMemory +{ +public: + SharedReadBuffer( + ReadBuffer & in_, + std::shared_ptr & mutex_, + FormatFactory::FileSegmentationEngine file_segmentation_engine_, + size_t min_chunk_size_) + : BufferWithOwnMemory(), + mutex(mutex_), + in(in_), + file_segmentation_engine(file_segmentation_engine_), + min_chunk_size(min_chunk_size_) + { + } + +private: + bool nextImpl() override + { + if (eof || !mutex) + return false; + + std::lock_guard lock(*mutex); + if (in.eof()) + { + eof = true; + return false; + } + + size_t used_size = 0; + bool res = file_segmentation_engine(in, memory, used_size, min_chunk_size); + if (!res) + return false; + + working_buffer = Buffer(memory.data(), memory.data() + used_size); + return true; + } + +private: + // Pointer to common mutex. + std::shared_ptr mutex; + + // Original ReadBuffer to read from. + ReadBuffer & in; + + // Function to fill working_buffer. + FormatFactory::FileSegmentationEngine file_segmentation_engine; + + // FileSegmentationEngine parameter. + size_t min_chunk_size; + + // Indicator of the eof. Save extra lock acquiring. + bool eof{false}; +}; +} From 5a34e39286207376dd3b4eed2a528da973646350 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 2 Oct 2019 17:26:15 +0300 Subject: [PATCH 03/90] small changes after review --- .../ParallelParsingBlockInputStream.cpp | 13 +---- .../ParallelParsingBlockInputStream.h | 57 ++++++++----------- 2 files changed, 25 insertions(+), 45 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index e7d09f0401f..c4d8d3f4497 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -22,25 +22,16 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() if (is_exception_occured) break; - if (original_buffer.eof()) - { - std::unique_lock lock(mutex); - current_unit.is_last_unit = true; - current_unit.status = READY_TO_PARSE; - scheduleParserThreadForUnitWithNumber(current_unit_number); - break; - } - // Segmentating the original input. current_unit.chunk.used_size = 0; - bool res = file_segmentation_engine(original_buffer, current_unit.chunk.memory, current_unit.chunk.used_size, min_chunk_size); + bool has_data = file_segmentation_engine(original_buffer, current_unit.chunk.memory, current_unit.chunk.used_size, min_chunk_size); // Creating buffer from the segment of data. auto new_buffer = BufferBase::Buffer(current_unit.chunk.memory.data(), current_unit.chunk.memory.data() + current_unit.chunk.used_size); current_unit.readbuffer.buffer().swap(new_buffer); current_unit.readbuffer.position() = current_unit.readbuffer.buffer().begin(); - if (!res) + if (!has_data) { std::unique_lock lock(mutex); current_unit.is_last_unit = true; diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 42de8bdb023..9fe08b5074f 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -70,23 +70,7 @@ public: ~ParallelParsingBlockInputStream() override { - { - std::unique_lock lock(mutex); - segmentator_condvar.notify_all(); - reader_condvar.notify_all(); - } - - if (segmentator_thread.joinable()) - segmentator_thread.join(); - - try - { - pool.wait(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + waitForAllThreads(); } void cancel(bool kill) override @@ -100,23 +84,7 @@ public: for (auto& unit: working_field) unit.reader->cancel(kill); - { - std::unique_lock lock(mutex); - segmentator_condvar.notify_all(); - reader_condvar.notify_all(); - } - - segmentator_thread.join(); - - try - { - pool.wait(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - + waitForAllThreads(); } Block getHeader() const override @@ -219,6 +187,27 @@ private: pool.schedule(std::bind(&ParallelParsingBlockInputStream::parserThreadFunction, this, unit_number)); } + void waitForAllThreads() + { + { + std::unique_lock lock(mutex); + segmentator_condvar.notify_all(); + reader_condvar.notify_all(); + } + + if (segmentator_thread.joinable()) + segmentator_thread.join(); + + try + { + pool.wait(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + void segmentatorThreadFunction(); void parserThreadFunction(size_t bucket_num); }; From 905a65b3d0135aca037a3e38bc95819beacb3e5b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 22 Oct 2019 21:01:44 +0300 Subject: [PATCH 04/90] some changes after review --- .../ParallelParsingBlockInputStream.cpp | 51 ++++++++--------- .../ParallelParsingBlockInputStream.h | 57 ++++++++----------- dbms/src/Formats/FormatFactory.cpp | 4 +- 3 files changed, 49 insertions(+), 63 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index c4d8d3f4497..65ed478d221 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -12,36 +12,33 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() { ++segmentator_ticket_number; const auto current_unit_number = segmentator_ticket_number % max_threads_to_use; - auto & current_unit = working_field[current_unit_number]; { std::unique_lock lock(mutex); - segmentator_condvar.wait(lock, [&](){ return current_unit.status == READY_TO_INSERT || is_exception_occured || is_cancelled; }); + segmentator_condvar.wait(lock, [&]{ return status[current_unit_number] == READY_TO_INSERT || is_exception_occured || is_cancelled; }); } if (is_exception_occured) break; // Segmentating the original input. - current_unit.chunk.used_size = 0; - bool has_data = file_segmentation_engine(original_buffer, current_unit.chunk.memory, current_unit.chunk.used_size, min_chunk_size); + 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); // Creating buffer from the segment of data. - auto new_buffer = BufferBase::Buffer(current_unit.chunk.memory.data(), current_unit.chunk.memory.data() + current_unit.chunk.used_size); - current_unit.readbuffer.buffer().swap(new_buffer); - current_unit.readbuffer.position() = current_unit.readbuffer.buffer().begin(); + auto new_buffer = BufferBase::Buffer(segments[current_unit_number].memory.data(), segments[current_unit_number].memory.data() + segments[current_unit_number].used_size); + buffers[current_unit_number]->buffer().swap(new_buffer); + buffers[current_unit_number]->position() = buffers[current_unit_number]->buffer().begin(); if (!has_data) { - std::unique_lock lock(mutex); - current_unit.is_last_unit = true; - current_unit.status = READY_TO_PARSE; + is_last[current_unit_number] = true; + status[current_unit_number] = READY_TO_PARSE; scheduleParserThreadForUnitWithNumber(current_unit_number); break; } - std::unique_lock lock(mutex); - current_unit.status = READY_TO_PARSE; + status[current_unit_number] = READY_TO_PARSE; scheduleParserThreadForUnitWithNumber(current_unit_number); } } @@ -51,39 +48,38 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() } } -void ParallelParsingBlockInputStream::parserThreadFunction(size_t bucket_num) +void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_number) { setThreadName("ChunkParser"); if (is_exception_occured && is_cancelled) return; - auto & current_unit = working_field[bucket_num]; - try { { std::unique_lock lock(mutex); - if (current_unit.is_last_unit || current_unit.readbuffer.position() == nullptr) + + if (is_last[current_unit_number] || buffers[current_unit_number]->position() == nullptr) { - current_unit.block = Block(); - current_unit.status = READY_TO_READ; + blocks[current_unit_number].block = Block(); + status[current_unit_number] = READY_TO_READ; reader_condvar.notify_all(); return; } } - current_unit.block = current_unit.reader->read(); + blocks[current_unit_number].block = readers[current_unit_number]->read(); { std::lock_guard missing_values_lock(missing_values_mutex); - block_missing_values = current_unit.reader->getMissingValues(); + blocks[current_unit_number].block_missing_values = readers[current_unit_number]->getMissingValues(); } { std::unique_lock lock(mutex); - current_unit.status = READY_TO_READ; + status[current_unit_number] = READY_TO_READ; reader_condvar.notify_all(); } @@ -91,7 +87,7 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t bucket_num) catch (...) { std::unique_lock lock(mutex); - exceptions[bucket_num] = std::current_exception(); + exceptions[current_unit_number] = std::current_exception(); is_exception_occured = true; reader_condvar.notify_all(); } @@ -108,9 +104,8 @@ Block ParallelParsingBlockInputStream::readImpl() ++reader_ticket_number; const auto unit_number = reader_ticket_number % max_threads_to_use; - auto & current_processed_unit = working_field[unit_number]; - reader_condvar.wait(lock, [&](){ return current_processed_unit.status == READY_TO_READ || is_exception_occured || is_cancelled; }); + reader_condvar.wait(lock, [&](){ return status[unit_number] == READY_TO_READ || is_exception_occured || is_cancelled; }); /// Check for an exception and rethrow it if (is_exception_occured) @@ -121,16 +116,16 @@ Block ParallelParsingBlockInputStream::readImpl() rethrowFirstException(exceptions); } - res = std::move(current_processed_unit.block); + res = std::move(blocks[unit_number].block); + last_block_missing_values = std::move(blocks[unit_number].block_missing_values); - if (current_processed_unit.is_last_unit) + if (is_last[unit_number]) is_cancelled = true; else { - current_processed_unit.status = READY_TO_INSERT; + status[unit_number] = READY_TO_INSERT; segmentator_condvar.notify_all(); } return res; } - } diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 9fe08b5074f..4f2d428896f 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -55,12 +55,18 @@ public: blocks.resize(max_threads_to_use); exceptions.resize(max_threads_to_use); buffers.reserve(max_threads_to_use); - working_field.reserve(max_threads_to_use); + readers.reserve(max_threads_to_use); + is_last.assign(max_threads_to_use, false); for (size_t i = 0; i < max_threads_to_use; ++i) { + status.emplace_back(ProcessingUnitStatus::READY_TO_INSERT); buffers.emplace_back(std::make_unique(segments[i].memory.data(), segments[i].used_size, 0)); - working_field.emplace_back(builder.input_processor_creator, builder.input_creator_params, segments[i], *buffers[i], blocks[i], exceptions[i]); + readers.emplace_back(std::make_unique(builder.input_processor_creator(*buffers[i], + builder.input_creator_params.sample, + builder.input_creator_params.context, + builder.input_creator_params.row_input_format_params, + builder.input_creator_params.settings))); } segmentator_thread = ThreadFromGlobalPool([this] { segmentatorThreadFunction(); }); @@ -81,15 +87,15 @@ public: if (!is_cancelled.compare_exchange_strong(old_val, true)) return; - for (auto& unit: working_field) - unit.reader->cancel(kill); + for (auto& reader: readers) + reader->cancel(kill); waitForAllThreads(); } Block getHeader() const override { - return working_field.at(0).reader->getHeader(); + return readers.at(0)->getHeader(); } protected: @@ -101,7 +107,7 @@ protected: const BlockMissingValues & getMissingValues() const override { std::lock_guard missing_values_lock(missing_values_mutex); - return block_missing_values; + return last_block_missing_values; } private: @@ -111,7 +117,7 @@ private: std::atomic is_exception_occured{false}; - BlockMissingValues block_missing_values; + BlockMissingValues last_block_missing_values; mutable std::mutex missing_values_mutex; // Original ReadBuffer to read from. @@ -146,45 +152,30 @@ private: size_t used_size{0}; }; - - struct ProcessingUnit + struct BlockExt { - ProcessingUnit(const InputProcessorCreator & input_getter, - const InputCreatorParams & params, - MemoryExt & chunk_, - ReadBuffer & readbuffer_, - Block & block_, - std::exception_ptr & exception_) : chunk(chunk_), readbuffer(readbuffer_), block(block_), exception(exception_) - { - reader = std::make_shared(input_getter(readbuffer, params.sample, params.context, params.row_input_format_params, params.settings)); - } - - MemoryExt & chunk; - ReadBuffer & readbuffer; - Block & block; - BlockInputStreamPtr reader; - - std::exception_ptr & exception; - ProcessingUnitStatus status{READY_TO_INSERT}; - bool is_last_unit{false}; + Block block; + BlockMissingValues block_missing_values; }; - - using Blocks = std::vector; + using Blocks = std::vector; using ReadBuffers = std::vector>; using Segments = std::vector; - using ProcessingUnits = std::vector; + using Status = std::deque>; + using InputStreamFromInputFormats = std::vector>; + using IsLastFlags = std::vector; Segments segments; ReadBuffers buffers; Blocks blocks; Exceptions exceptions; - - ProcessingUnits working_field; + Status status; + InputStreamFromInputFormats readers; + IsLastFlags is_last; void scheduleParserThreadForUnitWithNumber(size_t unit_number) { - pool.schedule(std::bind(&ParallelParsingBlockInputStream::parserThreadFunction, this, unit_number)); + pool.scheduleOrThrowOnError(std::bind(&ParallelParsingBlockInputStream::parserThreadFunction, this, unit_number)); } void waitForAllThreads() diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 82b37cf3ffb..787eaadf76f 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -291,7 +291,7 @@ void registerOutputFormatProcessorTemplate(FormatFactory &factory); void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); -void registerFileSegmentationEngineValues(FormatFactory & factory); +//void registerFileSegmentationEngineValues(FormatFactory & factory); void registerFileSegmentationEngineCSV(FormatFactory & factory); void registerFileSegmentationEngineTSKV(FormatFactory & factory); @@ -349,7 +349,7 @@ FormatFactory::FormatFactory() registerFileSegmentationEngineJSONEachRow(*this); registerFileSegmentationEngineTabSeparated(*this); - registerFileSegmentationEngineValues(*this); + //registerFileSegmentationEngineValues(*this); registerFileSegmentationEngineCSV(*this); registerFileSegmentationEngineTSKV(*this); From 31b69285faf96bbe3686aa6006abb008a6537a21 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 23 Oct 2019 13:39:33 +0300 Subject: [PATCH 05/90] fix data race on vector --- dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp | 4 +++- dbms/src/DataStreams/ParallelParsingBlockInputStream.h | 5 ++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index 65ed478d221..f617d9901f7 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -26,7 +26,9 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() bool has_data = 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(), segments[current_unit_number].memory.data() + segments[current_unit_number].used_size); + auto new_buffer = BufferBase::Buffer(segments[current_unit_number].memory.data(), + segments[current_unit_number].memory.data() + segments[current_unit_number].used_size); + buffers[current_unit_number]->buffer().swap(new_buffer); buffers[current_unit_number]->position() = buffers[current_unit_number]->buffer().begin(); diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 4f2d428896f..36313a21b98 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -163,7 +163,10 @@ private: using Segments = std::vector; using Status = std::deque>; using InputStreamFromInputFormats = std::vector>; - using IsLastFlags = std::vector; + + //We cannot use std::vector because it is equal to bitset (which stores 8 bool in one byte). + //That's why dataraces occured. + using IsLastFlags = std::vector; Segments segments; ReadBuffers buffers; From c867d7a306d0c0c773f994a0bec68d8382b03a9c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 23 Oct 2019 16:15:03 +0300 Subject: [PATCH 06/90] fix max_insert_block_size --- .../ParallelParsingBlockInputStream.cpp | 41 ++++++++++++------- .../ParallelParsingBlockInputStream.h | 7 ++-- 2 files changed, 30 insertions(+), 18 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index f617d9901f7..b8b961b3717 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -62,9 +62,12 @@ 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); + if (is_last[current_unit_number] || buffers[current_unit_number]->position() == nullptr) { - blocks[current_unit_number].block = Block(); + blocks[current_unit_number].block.push_back(Block()); status[current_unit_number] = READY_TO_READ; reader_condvar.notify_all(); return; @@ -72,11 +75,19 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n } - blocks[current_unit_number].block = readers[current_unit_number]->read(); + while (true) { - std::lock_guard missing_values_lock(missing_values_mutex); - blocks[current_unit_number].block_missing_values = readers[current_unit_number]->getMissingValues(); + auto block = readers[current_unit_number]->read(); + + blocks[current_unit_number].block.push_back(block); + { + std::lock_guard missing_values_lock(missing_values_mutex); + blocks[current_unit_number].block_missing_values.push_back(readers[current_unit_number]->getMissingValues()); + } + + if (block == Block()) + break; } { @@ -103,11 +114,8 @@ Block ParallelParsingBlockInputStream::readImpl() return res; std::unique_lock lock(mutex); - - ++reader_ticket_number; - const auto unit_number = reader_ticket_number % max_threads_to_use; - - reader_condvar.wait(lock, [&](){ return status[unit_number] == READY_TO_READ || is_exception_occured || is_cancelled; }); + const auto current_number = reader_ticket_number % max_threads_to_use; + reader_condvar.wait(lock, [&](){ return status[current_number] == READY_TO_READ || is_exception_occured || is_cancelled; }); /// Check for an exception and rethrow it if (is_exception_occured) @@ -118,16 +126,19 @@ Block ParallelParsingBlockInputStream::readImpl() rethrowFirstException(exceptions); } - res = std::move(blocks[unit_number].block); - last_block_missing_values = std::move(blocks[unit_number].block_missing_values); + res = std::move(blocks[current_number].block[internal_block_iter]); + last_block_missing_values = std::move(blocks[current_number].block_missing_values[internal_block_iter]); - if (is_last[unit_number]) - is_cancelled = true; - else + if (++internal_block_iter == blocks[current_number].block.size()) { - status[unit_number] = READY_TO_INSERT; + if (is_last[current_number]) + is_cancelled = true; + internal_block_iter = 0; + ++reader_ticket_number; + status[current_number] = READY_TO_INSERT; segmentator_condvar.notify_all(); } + return res; } } diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 36313a21b98..3f2801f63e5 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -124,7 +124,8 @@ private: ReadBuffer & original_buffer; //Non-atomic because it is used in one thread. - size_t reader_ticket_number{0}; + size_t reader_ticket_number{1}; + size_t internal_block_iter{0}; size_t segmentator_ticket_number{0}; std::mutex mutex; @@ -154,8 +155,8 @@ private: struct BlockExt { - Block block; - BlockMissingValues block_missing_values; + std::vector block; + std::vector block_missing_values; }; using Blocks = std::vector; From 8b17ce492f166c695da927fbf8f97cf397406f6f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 23 Oct 2019 16:23:42 +0300 Subject: [PATCH 07/90] remove unused mutex --- dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp | 5 +---- dbms/src/DataStreams/ParallelParsingBlockInputStream.h | 2 -- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index b8b961b3717..cb5b730b540 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -81,10 +81,7 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n auto block = readers[current_unit_number]->read(); blocks[current_unit_number].block.push_back(block); - { - std::lock_guard missing_values_lock(missing_values_mutex); - blocks[current_unit_number].block_missing_values.push_back(readers[current_unit_number]->getMissingValues()); - } + blocks[current_unit_number].block_missing_values.push_back(readers[current_unit_number]->getMissingValues()); if (block == Block()) break; diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 3f2801f63e5..37fa5f23136 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -106,7 +106,6 @@ protected: const BlockMissingValues & getMissingValues() const override { - std::lock_guard missing_values_lock(missing_values_mutex); return last_block_missing_values; } @@ -118,7 +117,6 @@ private: std::atomic is_exception_occured{false}; BlockMissingValues last_block_missing_values; - mutable std::mutex missing_values_mutex; // Original ReadBuffer to read from. ReadBuffer & original_buffer; From c7084ff5dcb0e91cb78f62c86d8ab4d421acb364 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 24 Oct 2019 17:00:51 +0300 Subject: [PATCH 08/90] better --- .../ParallelParsingBlockInputStream.cpp | 53 +++++++++++++------ .../ParallelParsingBlockInputStream.h | 19 +++++-- dbms/src/Formats/FormatFactory.cpp | 2 +- 3 files changed, 54 insertions(+), 20 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index cb5b730b540..2594029097d 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -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(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()) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 37fa5f23136..a37ec75f118 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -9,6 +9,7 @@ #include #include #include +#include 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 max_threads_to_use; const size_t min_chunk_size; diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 787eaadf76f..4c9e74ef977 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -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(builder); } From 342f259a1cddc04991672379e738d13cef19fc41 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 24 Oct 2019 19:52:55 +0300 Subject: [PATCH 09/90] comments --- .../ParallelParsingBlockInputStream.cpp | 26 ++++--------------- .../ParallelParsingBlockInputStream.h | 11 ++++++-- 2 files changed, 14 insertions(+), 23 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index 2594029097d..a56733f77ac 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -11,7 +11,6 @@ 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; { @@ -24,7 +23,6 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() 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); @@ -44,14 +42,9 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() buffers[current_unit_number]->buffer().swap(new_buffer); buffers[current_unit_number]->position() = buffers[current_unit_number]->buffer().begin(); -// std::cout << "current_unit_number " << current_unit_number << std::endl; - - readers[current_unit_number] = std::make_unique(input_processor_creator( - *buffers[current_unit_number], - header, - context, - row_input_format_params, - format_settings)); + readers[current_unit_number] = std::make_unique ( + 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); @@ -89,21 +82,16 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n } - + //We don't know how many blocks will be. So we have to read them all until an empty block occured. 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(); 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; } { @@ -116,13 +104,13 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n catch (...) { std::unique_lock lock(mutex); + tryLogCurrentException(__PRETTY_FUNCTION__); exceptions[current_unit_number] = std::current_exception(); is_exception_occured = true; reader_condvar.notify_all(); } } - Block ParallelParsingBlockInputStream::readImpl() { Block res; @@ -142,11 +130,7 @@ Block ParallelParsingBlockInputStream::readImpl() rethrowFirstException(exceptions); } -// 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()) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index a37ec75f118..6f6cd8111e6 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -14,6 +14,12 @@ namespace DB { +/** + * ORDER-PRESERVING parallel parsing of data formats. + * It splits original data into chunks. Then each chunk is parsed by different thread. + * The number of chunks equals to max_threads_for_parallel_reading setting. + * The size of chunk is equal to min_chunk_size_for_parallel_reading setting. + */ class ParallelParsingBlockInputStream : public IBlockInputStream { @@ -61,7 +67,7 @@ public: blocks.resize(max_threads_to_use); exceptions.resize(max_threads_to_use); buffers.reserve(max_threads_to_use); - readers.resize(max_threads_to_use); + readers.reserve(max_threads_to_use); is_last.assign(max_threads_to_use, false); for (size_t i = 0; i < max_threads_to_use; ++i) @@ -94,7 +100,8 @@ public: return; for (auto& reader: readers) - reader->cancel(kill); + if (!reader->isCancelled()) + reader->cancel(kill); waitForAllThreads(); } From 76e64d1c1ba78ca58a60e6d93c45ba9d9d2852cf Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 24 Oct 2019 21:06:45 +0300 Subject: [PATCH 10/90] empty commit --- dbms/src/DataStreams/ParallelParsingBlockInputStream.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 6f6cd8111e6..4cd3f4e7609 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -224,6 +224,3 @@ private: }; }; - - - From 78b6322797cf0ede8bcc9c4c7e9a158135290004 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 25 Oct 2019 00:17:00 +0300 Subject: [PATCH 11/90] empty commit 2 --- dbms/src/DataStreams/ParallelParsingBlockInputStream.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 4cd3f4e7609..c09254d1cf6 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -20,7 +20,6 @@ namespace DB * The number of chunks equals to max_threads_for_parallel_reading setting. * The size of chunk is equal to min_chunk_size_for_parallel_reading setting. */ - class ParallelParsingBlockInputStream : public IBlockInputStream { private: From 682b9df17f11dcb42c31102fc320f1cbbe232180 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 25 Oct 2019 15:28:24 +0300 Subject: [PATCH 12/90] remove SharedReadBuffer + remove lambdas --- .../ParallelParsingBlockInputStream.h | 1 - dbms/src/Formats/FormatFactory.cpp | 2 - dbms/src/Formats/FormatFactory.h | 2 +- dbms/src/IO/SharedReadBuffer.h | 70 ------------ .../Formats/Impl/CSVRowInputFormat.cpp | 104 +++++++++-------- .../Impl/JSONEachRowRowInputFormat.cpp | 108 +++++++++--------- .../Formats/Impl/TSKVRowInputFormat.cpp | 58 +++++----- .../Impl/TabSeparatedRowInputFormat.cpp | 68 ++++++----- 8 files changed, 166 insertions(+), 247 deletions(-) delete mode 100644 dbms/src/IO/SharedReadBuffer.h diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index c09254d1cf6..ad921167193 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 4c9e74ef977..bb1e93f6c02 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -349,7 +348,6 @@ FormatFactory::FormatFactory() registerFileSegmentationEngineJSONEachRow(*this); registerFileSegmentationEngineTabSeparated(*this); - //registerFileSegmentationEngineValues(*this); registerFileSegmentationEngineCSV(*this); registerFileSegmentationEngineTSKV(*this); diff --git a/dbms/src/Formats/FormatFactory.h b/dbms/src/Formats/FormatFactory.h index 4e044652eb7..b3359a36003 100644 --- a/dbms/src/Formats/FormatFactory.h +++ b/dbms/src/Formats/FormatFactory.h @@ -44,7 +44,7 @@ public: /** Fast reading data from buffer and save result to memory. * Reads at least min_chunk_size bytes and some more until the end of the chunk, depends on the format. - * Used in SharedReadBuffer. + * Used in ParallelParsingBlockInputStream. */ using FileSegmentationEngine = std::function -#include -#include -#include - - -namespace DB -{ - -/** Allows many threads to read from single ReadBuffer - * Each SharedReadBuffer has his own memory, - * which he filles under shared mutex using FileSegmentationEngine. - */ -class SharedReadBuffer : public BufferWithOwnMemory -{ -public: - SharedReadBuffer( - ReadBuffer & in_, - std::shared_ptr & mutex_, - FormatFactory::FileSegmentationEngine file_segmentation_engine_, - size_t min_chunk_size_) - : BufferWithOwnMemory(), - mutex(mutex_), - in(in_), - file_segmentation_engine(file_segmentation_engine_), - min_chunk_size(min_chunk_size_) - { - } - -private: - bool nextImpl() override - { - if (eof || !mutex) - return false; - - std::lock_guard lock(*mutex); - if (in.eof()) - { - eof = true; - return false; - } - - size_t used_size = 0; - bool res = file_segmentation_engine(in, memory, used_size, min_chunk_size); - if (!res) - return false; - - working_buffer = Buffer(memory.data(), memory.data() + used_size); - return true; - } - -private: - // Pointer to common mutex. - std::shared_ptr mutex; - - // Original ReadBuffer to read from. - ReadBuffer & in; - - // Function to fill working_buffer. - FormatFactory::FileSegmentationEngine file_segmentation_engine; - - // FileSegmentationEngine parameter. - size_t min_chunk_size; - - // Indicator of the eof. Save extra lock acquiring. - bool eof{false}; -}; -} diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index f33b6706c0a..2dbb2f4308e 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -422,66 +422,64 @@ void registerInputFormatProcessorCSV(FormatFactory & factory) } } -void registerFileSegmentationEngineCSV(FormatFactory & factory) +bool fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t & used_size, size_t min_chunk_size) { - factory.registerFileSegmentationEngine("CSV", []( - ReadBuffer & in, - DB::Memory<> & memory, - size_t & used_size, - size_t min_chunk_size) + skipWhitespacesAndTabs(in); + char * begin_pos = in.position(); + bool quotes = false; + bool need_more_data = true; + memory.resize(min_chunk_size); + while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && need_more_data) { - skipWhitespacesAndTabs(in); - char * begin_pos = in.position(); - bool quotes = false; - bool need_more_data = true; - memory.resize(min_chunk_size); - while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && need_more_data) + if (quotes) { - if (quotes) + in.position() = find_first_symbols<'"'>(in.position(), in.buffer().end()); + if (in.position() == in.buffer().end()) + continue; + if (*in.position() == '"') { - in.position() = find_first_symbols<'"'>(in.position(), in.buffer().end()); - if (in.position() == in.buffer().end()) - continue; - if (*in.position() == '"') - { + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && *in.position() == '"') ++in.position(); - if (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && *in.position() == '"') - ++in.position(); - else - quotes = false; - } - } - else - { - in.position() = find_first_symbols<'"','\r', '\n'>(in.position(), in.buffer().end()); - if (in.position() == in.buffer().end()) - continue; - if (*in.position() == '"') - { - quotes = true; - ++in.position(); - } - else if (*in.position() == '\n') - { - if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) - need_more_data = false; - ++in.position(); - if (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && *in.position() == '\r') - ++in.position(); - } - else if (*in.position() == '\r') - { - if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) - need_more_data = false; - ++in.position(); - if (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && *in.position() == '\n') - ++in.position(); - } + else + quotes = false; } } - eofWithSavingBufferState(in, memory, used_size, begin_pos, true); - return true; - }); + else + { + in.position() = find_first_symbols<'"','\r', '\n'>(in.position(), in.buffer().end()); + if (in.position() == in.buffer().end()) + continue; + if (*in.position() == '"') + { + quotes = true; + ++in.position(); + } + else if (*in.position() == '\n') + { + if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) + need_more_data = false; + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && *in.position() == '\r') + ++in.position(); + } + else if (*in.position() == '\r') + { + if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) + need_more_data = false; + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && *in.position() == '\n') + ++in.position(); + } + } + } + eofWithSavingBufferState(in, memory, used_size, begin_pos, true); + return true; +} + +void registerFileSegmentationEngineCSV(FormatFactory & factory) +{ + factory.registerFileSegmentationEngine("CSV", &fileSegmentationEngineCSVImpl); } diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index d5a4b50de74..e1bf94e4c23 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -270,70 +270,68 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory) }); } -void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory) +bool fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t & used_size, size_t min_chunk_size) { - factory.registerFileSegmentationEngine("JSONEachRow", []( - ReadBuffer & in, - DB::Memory<> & memory, - size_t & used_size, - size_t min_chunk_size) + skipWhitespaceIfAny(in); + char * begin_pos = in.position(); + size_t balance = 0; + bool quotes = false; + memory.resize(min_chunk_size); + while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) + && (balance || used_size + static_cast(in.position() - begin_pos) < min_chunk_size)) { - skipWhitespaceIfAny(in); - char * begin_pos = in.position(); - size_t balance = 0; - bool quotes = false; - memory.resize(min_chunk_size); - while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) - && (balance || used_size + static_cast(in.position() - begin_pos) < min_chunk_size)) + if (quotes) { - if (quotes) + in.position() = find_first_symbols<'\\', '"'>(in.position(), in.buffer().end()); + if (in.position() == in.buffer().end()) + continue; + if (*in.position() == '\\') { - in.position() = find_first_symbols<'\\', '"'>(in.position(), in.buffer().end()); - if (in.position() == in.buffer().end()) - continue; - if (*in.position() == '\\') - { + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos)) ++in.position(); - if (!eofWithSavingBufferState(in, memory, used_size, begin_pos)) - ++in.position(); - } - else if (*in.position() == '"') - { - ++in.position(); - quotes = false; - } } - else + else if (*in.position() == '"') { - in.position() = find_first_symbols<'{', '}', '\\', '"'>(in.position(), in.buffer().end()); - if (in.position() == in.buffer().end()) - continue; - if (*in.position() == '{') - { - ++balance; - ++in.position(); - } - else if (*in.position() == '}') - { - --balance; - ++in.position(); - } - else if (*in.position() == '\\') - { - ++in.position(); - if (!eofWithSavingBufferState(in, memory, used_size, begin_pos)) - ++in.position(); - } - else if (*in.position() == '"') - { - quotes = true; - ++in.position(); - } + ++in.position(); + quotes = false; } } - eofWithSavingBufferState(in, memory, used_size, begin_pos, true); - return true; - }); + else + { + in.position() = find_first_symbols<'{', '}', '\\', '"'>(in.position(), in.buffer().end()); + if (in.position() == in.buffer().end()) + continue; + if (*in.position() == '{') + { + ++balance; + ++in.position(); + } + else if (*in.position() == '}') + { + --balance; + ++in.position(); + } + else if (*in.position() == '\\') + { + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos)) + ++in.position(); + } + else if (*in.position() == '"') + { + quotes = true; + ++in.position(); + } + } + } + eofWithSavingBufferState(in, memory, used_size, begin_pos, true); + return true; +} + +void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory) +{ + factory.registerFileSegmentationEngine("JSONEachRow", &fileSegmentationEngineJSONEachRowImpl); } } diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index 5d990cc7fef..e14a5dce16d 100644 --- a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -210,38 +210,36 @@ void registerInputFormatProcessorTSKV(FormatFactory & factory) }); } +bool fileSegmentationEngineTSKVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t & used_size, size_t min_chunk_size) +{ + char * begin_pos = in.position(); + bool need_more_data = true; + memory.resize(min_chunk_size); + while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && need_more_data) + { + in.position() = find_first_symbols<'\\','\r', '\n'>(in.position(), in.buffer().end()); + if (in.position() == in.buffer().end()) + continue; + if (*in.position() == '\\') + { + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos)) + ++in.position(); + } + else if (*in.position() == '\n' || *in.position() == '\r') + { + if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) + need_more_data = false; + ++in.position(); + } + } + eofWithSavingBufferState(in, memory, used_size, begin_pos, true); + return true; +} + void registerFileSegmentationEngineTSKV(FormatFactory & factory) { - factory.registerFileSegmentationEngine("TSKV", []( - ReadBuffer & in, - DB::Memory<> & memory, - size_t & used_size, - size_t min_chunk_size) - { - char * begin_pos = in.position(); - bool need_more_data = true; - memory.resize(min_chunk_size); - while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && need_more_data) - { - in.position() = find_first_symbols<'\\','\r', '\n'>(in.position(), in.buffer().end()); - if (in.position() == in.buffer().end()) - continue; - if (*in.position() == '\\') - { - ++in.position(); - if (!eofWithSavingBufferState(in, memory, used_size, begin_pos)) - ++in.position(); - } - else if (*in.position() == '\n' || *in.position() == '\r') - { - if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) - need_more_data = false; - ++in.position(); - } - } - eofWithSavingBufferState(in, memory, used_size, begin_pos, true); - return true; - }); + factory.registerFileSegmentationEngine("TSKV", &fileSegmentationEngineTSKVImpl); } diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 521d8ad27c5..f13b732b371 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -384,45 +384,43 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) } } +bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t & used_size, size_t min_chunk_size) +{ + if (in.eof()) + return false; + + char * begin_pos = in.position(); + bool need_more_data = true; + memory.resize(min_chunk_size); + while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && need_more_data) + { + in.position() = find_first_symbols<'\\', '\r', '\n'>(in.position(), in.buffer().end()); + if (in.position() == in.buffer().end()) + { + continue; + } + + if (*in.position() == '\\') + { + ++in.position(); + if (!eofWithSavingBufferState(in, memory, used_size, begin_pos, true)) + ++in.position(); + } else if (*in.position() == '\n' || *in.position() == '\r') + { + if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) + need_more_data = false; + ++in.position(); + } + } + eofWithSavingBufferState(in, memory, used_size, begin_pos, true); + return true; +} + void registerFileSegmentationEngineTabSeparated(FormatFactory & factory) { for (auto name : {"TabSeparated", "TSV"}) { - factory.registerFileSegmentationEngine(name, []( - ReadBuffer & in, - DB::Memory<> & memory, - size_t & used_size, - size_t min_chunk_size) - { - if (in.eof()) - return false; - - char * begin_pos = in.position(); - bool need_more_data = true; - memory.resize(min_chunk_size); - while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && need_more_data) - { - in.position() = find_first_symbols<'\\', '\r', '\n'>(in.position(), in.buffer().end()); - if (in.position() == in.buffer().end()) - { - continue; - } - - if (*in.position() == '\\') - { - ++in.position(); - if (!eofWithSavingBufferState(in, memory, used_size, begin_pos, true)) - ++in.position(); - } else if (*in.position() == '\n' || *in.position() == '\r') - { - if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) - need_more_data = false; - ++in.position(); - } - } - eofWithSavingBufferState(in, memory, used_size, begin_pos, true); - return true; - }); + factory.registerFileSegmentationEngine(name, &fileSegmentationEngineTabSeparatedImpl); } } From 60ee52cb8d7e2dc0b17e36815c84336f7efe4645 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 25 Oct 2019 15:28:52 +0300 Subject: [PATCH 13/90] remove unrelated changes in tests --- dbms/tests/queries/0_stateless/00395_nullable.sql | 10 +++++----- .../queries/0_stateless/00937_ipv4_cidr_range.sql | 2 -- .../queries/0_stateless/00938_ipv6_cidr_range.sql | 2 -- 3 files changed, 5 insertions(+), 9 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00395_nullable.sql b/dbms/tests/queries/0_stateless/00395_nullable.sql index f41e8131372..71dc045ad09 100644 --- a/dbms/tests/queries/0_stateless/00395_nullable.sql +++ b/dbms/tests/queries/0_stateless/00395_nullable.sql @@ -102,11 +102,11 @@ SELECT col1 FROM test1_00395 ORDER BY col1 ASC; SELECT '----- Insert. Source and target columns have same types up to nullability. -----'; DROP TABLE IF EXISTS test1_00395; CREATE TABLE test1_00395(col1 Nullable(UInt64), col2 UInt64) Engine=Memory; -DROP TABLE IF EXISTS test2_00395; -CREATE TABLE test2_00395(col1 UInt64, col2 Nullable(UInt64)) Engine=Memory; +DROP TABLE IF EXISTS test2; +CREATE TABLE test2(col1 UInt64, col2 Nullable(UInt64)) Engine=Memory; INSERT INTO test1_00395(col1,col2) VALUES (2,7)(6,9)(5,1)(4,3)(8,2); -INSERT INTO test2_00395(col1,col2) SELECT col1,col2 FROM test1_00395; -SELECT col1,col2 FROM test2_00395 ORDER BY col1,col2 ASC; +INSERT INTO test2(col1,col2) SELECT col1,col2 FROM test1_00395; +SELECT col1,col2 FROM test2 ORDER BY col1,col2 ASC; SELECT '----- Apply functions and aggregate functions on columns that may contain null values -----'; @@ -497,4 +497,4 @@ INSERT INTO test1_00395(col1,col2) VALUES([NULL], 'ACDEFBGH'); SELECT col1, count() FROM test1_00395 GROUP BY col1 ORDER BY col1; DROP TABLE IF EXISTS test1_00395; -DROP TABLE test2_00395; +DROP TABLE test2; diff --git a/dbms/tests/queries/0_stateless/00937_ipv4_cidr_range.sql b/dbms/tests/queries/0_stateless/00937_ipv4_cidr_range.sql index b79c6e3709d..d0a04dd1cce 100644 --- a/dbms/tests/queries/0_stateless/00937_ipv4_cidr_range.sql +++ b/dbms/tests/queries/0_stateless/00937_ipv4_cidr_range.sql @@ -1,5 +1,3 @@ -USE test; - DROP TABLE IF EXISTS ipv4_range; CREATE TABLE ipv4_range(ip IPv4, cidr UInt8) ENGINE = Memory; diff --git a/dbms/tests/queries/0_stateless/00938_ipv6_cidr_range.sql b/dbms/tests/queries/0_stateless/00938_ipv6_cidr_range.sql index 653f789f4e2..5f69710b220 100644 --- a/dbms/tests/queries/0_stateless/00938_ipv6_cidr_range.sql +++ b/dbms/tests/queries/0_stateless/00938_ipv6_cidr_range.sql @@ -1,5 +1,3 @@ -USE test; - DROP TABLE IF EXISTS ipv6_range; CREATE TABLE ipv6_range(ip IPv6, cidr UInt8) ENGINE = Memory; From 980528ed75235a1f251fada748f8752e9bf4a55a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 25 Oct 2019 17:32:57 +0300 Subject: [PATCH 14/90] docs --- dbms/src/Core/Settings.h | 6 +++--- .../ParallelParsingBlockInputStream.h | 4 ++-- dbms/src/Formats/FormatFactory.cpp | 4 ++-- docs/en/operations/settings/settings.md | 21 +++++++++++++++++++ 4 files changed, 28 insertions(+), 7 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index d909874d7d8..98d4e7f9f7f 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -108,9 +108,9 @@ struct Settings : public SettingsCollection M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.") \ M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.") \ \ - M(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for several data formats (JSON, TSV, TKSV, Values, CSV).") \ - M(SettingUInt64, max_threads_for_parallel_reading, 10, "The maximum number of threads to parallel reading. By default, it is set to max_threads.") \ - M(SettingUInt64, min_chunk_size_for_parallel_reading, (1024 * 1024), "The minimum chunk size in bytes, which each thread tries to parse under mutex in parallel reading.") \ + M(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for several data formats (JSONEachRow, TSV, TKSV, CSV).") \ + M(SettingUInt64, max_threads_for_parallel_parsing, 10, "The maximum number of threads to parallel parsing.") \ + M(SettingUInt64, min_chunk_size_for_parallel_parsing, (1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.") \ \ M(SettingUInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.") \ M(SettingUInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.") \ diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index ad921167193..3111f8ed5a0 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -16,8 +16,8 @@ namespace DB /** * ORDER-PRESERVING parallel parsing of data formats. * It splits original data into chunks. Then each chunk is parsed by different thread. - * The number of chunks equals to max_threads_for_parallel_reading setting. - * The size of chunk is equal to min_chunk_size_for_parallel_reading setting. + * The number of chunks equals to max_threads_for_parallel_parsing setting. + * The size of chunk is equal to min_chunk_size_for_parallel_parsing setting. */ class ParallelParsingBlockInputStream : public IBlockInputStream { diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index bb1e93f6c02..ca1506b177d 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -124,12 +124,12 @@ BlockInputStreamPtr FormatFactory::getInput( row_input_format_params.max_execution_time = settings.max_execution_time; row_input_format_params.timeout_overflow_mode = settings.timeout_overflow_mode; - size_t max_threads_to_use = settings.max_threads_for_parallel_reading; + size_t max_threads_to_use = settings.max_threads_for_parallel_parsing; if (!max_threads_to_use) max_threads_to_use = settings.max_threads; auto params = ParallelParsingBlockInputStream::InputCreatorParams{sample, context, row_input_format_params, format_settings}; - ParallelParsingBlockInputStream::Builder 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_parsing}; return std::make_shared(builder); } diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ef1b664272e..126dc1a2ab4 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -980,4 +980,25 @@ Lower values mean higher priority. Threads with low `nice` priority values are e Default value: 0. +## input_format_parallel_parsing + +- Type: bool +- Default value: True + +Enable order-preserving parallel parsing of data formats such as JSONEachRow, TSV, TKSV and CSV. Reading will be single threaded and parsing will be multithreaded. + +## max_threads_for_parallel_parsing + +- Type: unsigned int +- Default value: 10 + +The maximum number of threads for order-preserving parallel parsing of data formats. + +## min_chunk_size_for_parallel_parsing + +- Type: unsigned int +- Default value: 1024 * 1024 + +The minimum chunk size in bytes, which each thread will parse in parallel. By default it equals to one megabyte. + [Original article](https://clickhouse.yandex/docs/en/operations/settings/settings/) From 4fe0dda4539aab47836b531adc7ea96f7bb46fb6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 25 Oct 2019 21:03:38 +0300 Subject: [PATCH 15/90] add cancel test --- dbms/src/Formats/FormatFactory.cpp | 4 +- dbms/src/IO/ReadHelpers.h | 2 +- .../01019_parallel_parsing_cancel.reference | 1 + .../01019_parallel_parsing_cancel.sh | 48 +++++++++++++++++++ 4 files changed, 51 insertions(+), 4 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01019_parallel_parsing_cancel.reference create mode 100755 dbms/tests/queries/0_stateless/01019_parallel_parsing_cancel.sh diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index ca1506b177d..e3a3217010a 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -108,7 +108,7 @@ BlockInputStreamPtr FormatFactory::getInput( const Settings & settings = context.getSettingsRef(); const auto & file_segmentation_engine = getCreators(name).file_segmentation_engine; - if (name != "Values" && settings.input_format_parallel_parsing && file_segmentation_engine) + if (settings.input_format_parallel_parsing && file_segmentation_engine) { const auto & input_getter = getCreators(name).input_processor_creator; if (!input_getter) @@ -125,8 +125,6 @@ BlockInputStreamPtr FormatFactory::getInput( row_input_format_params.timeout_overflow_mode = settings.timeout_overflow_mode; size_t max_threads_to_use = settings.max_threads_for_parallel_parsing; - if (!max_threads_to_use) - max_threads_to_use = settings.max_threads; auto params = ParallelParsingBlockInputStream::InputCreatorParams{sample, context, row_input_format_params, format_settings}; ParallelParsingBlockInputStream::Builder builder{buf, input_getter, params, file_segmentation_engine, max_threads_to_use, settings.min_chunk_size_for_parallel_parsing}; diff --git a/dbms/src/IO/ReadHelpers.h b/dbms/src/IO/ReadHelpers.h index 71a5647c469..341b2438912 100644 --- a/dbms/src/IO/ReadHelpers.h +++ b/dbms/src/IO/ReadHelpers.h @@ -916,6 +916,6 @@ void skipToUnescapedNextLineOrEOF(ReadBuffer & buf); * If there is no pending data in buffer or it was explicitly asked * save current state to memory. */ -bool eofWithSavingBufferState(ReadBuffer & buf, DB::Memory<> & memory, size_t & used_size, char * & begin_pos, bool save_buffer_state = false); +bool eofWithSavingBufferState(ReadBuffer & buf, DB::Memory<> & memory, size_t & used_size, char * & begin_pos, bool force_saving_buffer_state = false); } diff --git a/dbms/tests/queries/0_stateless/01019_parallel_parsing_cancel.reference b/dbms/tests/queries/0_stateless/01019_parallel_parsing_cancel.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01019_parallel_parsing_cancel.reference @@ -0,0 +1 @@ +OK diff --git a/dbms/tests/queries/0_stateless/01019_parallel_parsing_cancel.sh b/dbms/tests/queries/0_stateless/01019_parallel_parsing_cancel.sh new file mode 100755 index 00000000000..646823e2821 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01019_parallel_parsing_cancel.sh @@ -0,0 +1,48 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS a;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS b;" + +$CLICKHOUSE_CLIENT --query="CREATE TABLE a (x UInt64) ENGINE = Memory;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE b (x UInt64) ENGINE = Memory;" + +function thread1() +{ + while true; do + seq 1 11000000 | $CLICKHOUSE_CLIENT --query_id=11 --query="INSERT INTO a(x) FORMAT TSV" + sleep 1 + $CLICKHOUSE_CLIENT --query="kill query where query_id='22'" SYNC + + done +} + +function thread2() +{ + while true; do + seq 1 11000000 | $CLICKHOUSE_CLIENT --query_id=22 --query="INSERT INTO b(x) FORMAT TSV" + sleep 1 + $CLICKHOUSE_CLIENT --query="kill query where query_id='11'" SYNC + done +} + + +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; + +TIMEOUT=20 + +timeout $TIMEOUT bash -c thread1 2>&1 > /dev/null & +timeout $TIMEOUT bash -c thread2 2>&1 > /dev/null & + +wait + +echo OK + +$CLICKHOUSE_CLIENT --query "DROP TABLE a" +$CLICKHOUSE_CLIENT --query "DROP TABLE b" + + From 206800b8fcb6e4b184ccbf5ab5dbd34bb805222b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 28 Oct 2019 13:51:43 +0300 Subject: [PATCH 16/90] add stderr configs to kafka test --- .../test_storage_kafka/configs/log_conf.xml | 11 +++++++++++ dbms/tests/integration/test_storage_kafka/test.py | 2 +- 2 files changed, 12 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/integration/test_storage_kafka/configs/log_conf.xml diff --git a/dbms/tests/integration/test_storage_kafka/configs/log_conf.xml b/dbms/tests/integration/test_storage_kafka/configs/log_conf.xml new file mode 100644 index 00000000000..95466269afe --- /dev/null +++ b/dbms/tests/integration/test_storage_kafka/configs/log_conf.xml @@ -0,0 +1,11 @@ + + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + \ No newline at end of file diff --git a/dbms/tests/integration/test_storage_kafka/test.py b/dbms/tests/integration/test_storage_kafka/test.py index a2e1511537a..efea7786f46 100644 --- a/dbms/tests/integration/test_storage_kafka/test.py +++ b/dbms/tests/integration/test_storage_kafka/test.py @@ -31,7 +31,7 @@ import kafka_pb2 cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', config_dir='configs', - main_configs=['configs/kafka.xml'], + main_configs=['configs/kafka.xml', 'configs/log_conf.xml'], with_kafka=True, clickhouse_path_dir='clickhouse_path') kafka_id = '' From 2bd79325d7e738c912b7b5fe231ce5ac92b52c4d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 28 Oct 2019 17:41:24 +0300 Subject: [PATCH 17/90] test kafka limit --- dbms/src/DataStreams/UnionBlockInputStream.h | 2 -- dbms/src/IO/BufferBase.h | 1 - dbms/tests/integration/test_storage_kafka/test.py | 4 ++-- 3 files changed, 2 insertions(+), 5 deletions(-) diff --git a/dbms/src/DataStreams/UnionBlockInputStream.h b/dbms/src/DataStreams/UnionBlockInputStream.h index 684892a3003..c4e84e85845 100644 --- a/dbms/src/DataStreams/UnionBlockInputStream.h +++ b/dbms/src/DataStreams/UnionBlockInputStream.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB @@ -42,7 +41,6 @@ private: }; public: - std::vector> buffers; using ExceptionCallback = std::function; UnionBlockInputStream( diff --git a/dbms/src/IO/BufferBase.h b/dbms/src/IO/BufferBase.h index f2191e41a0b..c22dcbecf7b 100644 --- a/dbms/src/IO/BufferBase.h +++ b/dbms/src/IO/BufferBase.h @@ -2,7 +2,6 @@ #include #include -#include namespace DB diff --git a/dbms/tests/integration/test_storage_kafka/test.py b/dbms/tests/integration/test_storage_kafka/test.py index efea7786f46..51325000f93 100644 --- a/dbms/tests/integration/test_storage_kafka/test.py +++ b/dbms/tests/integration/test_storage_kafka/test.py @@ -31,7 +31,7 @@ import kafka_pb2 cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', config_dir='configs', - main_configs=['configs/kafka.xml', 'configs/log_conf.xml'], + main_configs=['configs/kafka.xml', 'configs/log_conf.xml' ], with_kafka=True, clickhouse_path_dir='clickhouse_path') kafka_id = '' @@ -557,7 +557,7 @@ def test_kafka_insert(kafka_cluster): kafka_check_result(result, True) -@pytest.mark.timeout(180) +@pytest.mark.timeout(240) def test_kafka_produce_consume(kafka_cluster): instance.query(''' DROP TABLE IF EXISTS test.view; From c5085b86aacc3d383229095f2e1ce79fd62bde82 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 29 Oct 2019 02:43:22 +0300 Subject: [PATCH 18/90] max_threads limit --- dbms/src/Formats/FormatFactory.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index e3a3217010a..a5e0a56040d 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -124,7 +125,10 @@ BlockInputStreamPtr FormatFactory::getInput( row_input_format_params.max_execution_time = settings.max_execution_time; row_input_format_params.timeout_overflow_mode = settings.timeout_overflow_mode; - size_t max_threads_to_use = settings.max_threads_for_parallel_parsing; + //The number of threads for parallel parsing must be less or equal settings.max_threads. + const size_t global_max_threads = settings.max_threads; + const size_t max_threads_for_parallel_parsing = settings.max_threads_for_parallel_parsing; + const size_t max_threads_to_use = max_threads_for_parallel_parsing == 0 ? global_max_threads : std::min(max_threads_for_parallel_parsing, global_max_threads); auto params = ParallelParsingBlockInputStream::InputCreatorParams{sample, context, row_input_format_params, format_settings}; ParallelParsingBlockInputStream::Builder builder{buf, input_getter, params, file_segmentation_engine, max_threads_to_use, settings.min_chunk_size_for_parallel_parsing}; From 5bcfee117aa26bc1b9ef89cc2a98c21dffd61f31 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 29 Oct 2019 20:05:10 +0300 Subject: [PATCH 19/90] empty + master --- dbms/src/Formats/FormatFactory.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index a5e0a56040d..b883e42e8ad 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include From b691cc744cd26f51e23d6f8519062cf0366a9621 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 30 Oct 2019 18:49:10 +0300 Subject: [PATCH 20/90] comments and checks --- .../ParallelParsingBlockInputStream.cpp | 14 +++++++------- dbms/src/IO/ReadHelpers.cpp | 11 +++++++---- dbms/src/IO/ReadHelpers.h | 5 ++--- .../Processors/Formats/Impl/CSVRowInputFormat.cpp | 3 +++ .../Formats/Impl/JSONEachRowRowInputFormat.cpp | 3 +++ .../Processors/Formats/Impl/TSKVRowInputFormat.cpp | 3 +++ 6 files changed, 25 insertions(+), 14 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index a56733f77ac..96d45ba2c79 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -21,7 +21,13 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() if (is_exception_occured) break; - if (original_buffer.eof()) + // Segmentating the original input. + segments[current_unit_number].used_size = 0; + + //It returns bool, but it is useless + const auto res = file_segmentation_engine(original_buffer, segments[current_unit_number].memory, segments[current_unit_number].used_size, min_chunk_size); + + if (!res) { is_last[current_unit_number] = true; status[current_unit_number] = READY_TO_PARSE; @@ -29,12 +35,6 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() break; } - // Segmentating the original input. - segments[current_unit_number].used_size = 0; - - //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(), segments[current_unit_number].memory.data() + segments[current_unit_number].used_size); diff --git a/dbms/src/IO/ReadHelpers.cpp b/dbms/src/IO/ReadHelpers.cpp index d7ae52cacd4..e85c40ee84d 100644 --- a/dbms/src/IO/ReadHelpers.cpp +++ b/dbms/src/IO/ReadHelpers.cpp @@ -1053,12 +1053,15 @@ void skipToUnescapedNextLineOrEOF(ReadBuffer & buf) } } -bool eofWithSavingBufferState(ReadBuffer & buf, DB::Memory<> & memory, size_t & used_size, char * & begin_pos, bool save_buffer_state) +bool eofWithSavingBufferState(ReadBuffer & buf, DB::Memory<> & memory, size_t & used_size, char * & begin_pos, bool force_saving_buffer_state) { - if (save_buffer_state || !buf.hasPendingData()) + /// If there is some pending data - no need to copy data from buffer to memory. + if (force_saving_buffer_state || !buf.hasPendingData()) { - const size_t capacity = memory.size(); - const size_t block_size = static_cast(buf.position() - begin_pos); + const auto capacity = memory.size(); + const auto block_size = static_cast(buf.position() - begin_pos); + + /// To avoid calling a function when not needed. if (capacity <= block_size + used_size) { memory.resize(used_size + block_size); diff --git a/dbms/src/IO/ReadHelpers.h b/dbms/src/IO/ReadHelpers.h index 341b2438912..12158871bcd 100644 --- a/dbms/src/IO/ReadHelpers.h +++ b/dbms/src/IO/ReadHelpers.h @@ -912,9 +912,8 @@ void skipToNextLineOrEOF(ReadBuffer & buf); /// Skip to next character after next unescaped \n. If no \n in stream, skip to end. Does not throw on invalid escape sequences. void skipToUnescapedNextLineOrEOF(ReadBuffer & buf); -/** Return buffer eof() result. - * If there is no pending data in buffer or it was explicitly asked - * save current state to memory. +/** Returns buffer eof() result. + * And saves data if there is no pending data in buffer or it was explicitly asked. */ bool eofWithSavingBufferState(ReadBuffer & buf, DB::Memory<> & memory, size_t & used_size, char * & begin_pos, bool force_saving_buffer_state = false); diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 2dbb2f4308e..ef50e585d71 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -424,6 +424,9 @@ void registerInputFormatProcessorCSV(FormatFactory & factory) bool fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t & used_size, size_t min_chunk_size) { + if (in.eof()) + return false; + skipWhitespacesAndTabs(in); char * begin_pos = in.position(); bool quotes = false; diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index e1bf94e4c23..a477a8bbd5c 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -272,6 +272,9 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory) bool fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t & used_size, size_t min_chunk_size) { + if (in.eof()) + return false; + skipWhitespaceIfAny(in); char * begin_pos = in.position(); size_t balance = 0; diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index e14a5dce16d..2dd101fd6d9 100644 --- a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -212,6 +212,9 @@ void registerInputFormatProcessorTSKV(FormatFactory & factory) bool fileSegmentationEngineTSKVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t & used_size, size_t min_chunk_size) { + if (in.eof()) + return false; + char * begin_pos = in.position(); bool need_more_data = true; memory.resize(min_chunk_size); From 874f88e5a5f4fd522d8e2b8417bbe54b0e01d538 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 30 Oct 2019 18:54:02 +0300 Subject: [PATCH 21/90] disable feature to test --- dbms/src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index cc02253b4d9..ff876c30427 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -108,7 +108,7 @@ struct Settings : public SettingsCollection M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.") \ M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.") \ \ - M(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for several data formats (JSONEachRow, TSV, TKSV, CSV).") \ + M(SettingBool, input_format_parallel_parsing, false, "Enable parallel parsing for several data formats (JSONEachRow, TSV, TKSV, CSV).") \ M(SettingUInt64, max_threads_for_parallel_parsing, 10, "The maximum number of threads to parallel parsing.") \ M(SettingUInt64, min_chunk_size_for_parallel_parsing, (1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.") \ \ From cb48ee42a3aa39680985d07dff52426e521f9afd Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 30 Oct 2019 23:51:23 +0300 Subject: [PATCH 22/90] enable feature --- dbms/src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index ff876c30427..cc02253b4d9 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -108,7 +108,7 @@ struct Settings : public SettingsCollection M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.") \ M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.") \ \ - M(SettingBool, input_format_parallel_parsing, false, "Enable parallel parsing for several data formats (JSONEachRow, TSV, TKSV, CSV).") \ + M(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for several data formats (JSONEachRow, TSV, TKSV, CSV).") \ M(SettingUInt64, max_threads_for_parallel_parsing, 10, "The maximum number of threads to parallel parsing.") \ M(SettingUInt64, min_chunk_size_for_parallel_parsing, (1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.") \ \ From 9d8bbeb473bcc6319563f9aa511ff6a42bf4b615 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 31 Oct 2019 17:31:19 +0300 Subject: [PATCH 23/90] bad fix for PrettyCompact --- dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 3adbedff2a7..55e65cf65db 100644 --- a/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -201,6 +201,8 @@ void PrettyBlockOutputFormat::writeValueWithPadding( { auto writePadding = [&]() { + if (pad_to_width < value_width) + return; for (size_t k = 0; k < pad_to_width - value_width; ++k) writeChar(' ', out); }; From 0e04d14b7db9b3245ea691130d8bf09ef416f5ea Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 1 Nov 2019 15:31:48 +0300 Subject: [PATCH 24/90] cancel --- .../DataStreams/ParallelParsingBlockInputStream.cpp | 10 ++++++++-- dbms/src/DataStreams/ParallelParsingBlockInputStream.h | 4 ++++ 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index 96d45ba2c79..e16c72c7c67 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -124,7 +124,7 @@ Block ParallelParsingBlockInputStream::readImpl() /// Check for an exception and rethrow it if (is_exception_occured) { - segmentator_condvar.notify_all(); + LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::readImpl()"), "Exception occured. Will cancel the query."); lock.unlock(); cancel(false); rethrowFirstException(exceptions); @@ -136,7 +136,13 @@ Block ParallelParsingBlockInputStream::readImpl() if (++internal_block_iter == blocks[current_number].block.size()) { if (is_last[current_number]) - is_cancelled = true; + { + LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::readImpl()"), "Last unit. Will cancel the query."); + lock.unlock(); + cancel(false); + return res; + } + internal_block_iter = 0; ++reader_ticket_number; status[current_number] = READY_TO_INSERT; diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 3111f8ed5a0..c3d9ee41be9 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -61,6 +61,7 @@ public: pool(builder.max_threads_to_use), file_segmentation_engine(builder.file_segmentation_engine) { + LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream()"), "Constructor"); segments.resize(max_threads_to_use); blocks.resize(max_threads_to_use); exceptions.resize(max_threads_to_use); @@ -87,10 +88,12 @@ public: ~ParallelParsingBlockInputStream() override { waitForAllThreads(); + LOG_TRACE(&Poco::Logger::get("~ParallelParsingBLockInputStream()"), "All threads are killed."); } void cancel(bool kill) override { + LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::cancel()"), "Try to cancel."); if (kill) is_killed = true; bool old_val = false; @@ -102,6 +105,7 @@ public: reader->cancel(kill); waitForAllThreads(); + LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::cancel()"), "Cancelled succsessfully."); } Block getHeader() const override From 3c57b8e9c275fb59c0cf513ef24000af0af49b4f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 1 Nov 2019 21:59:54 +0300 Subject: [PATCH 25/90] test --- dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp | 7 +++---- dbms/src/DataStreams/ParallelParsingBlockInputStream.h | 9 +++++++-- dbms/src/Formats/FormatFactory.cpp | 2 ++ .../Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 8 +++++++- 4 files changed, 19 insertions(+), 7 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index e16c72c7c67..02b0c9d47fa 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -114,7 +114,7 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n Block ParallelParsingBlockInputStream::readImpl() { Block res; - if (isCancelledOrThrowIfKilled()) + if (isCancelledOrThrowIfKilled() || executed) return res; std::unique_lock lock(mutex); @@ -137,9 +137,8 @@ Block ParallelParsingBlockInputStream::readImpl() { if (is_last[current_number]) { - LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::readImpl()"), "Last unit. Will cancel the query."); - lock.unlock(); - cancel(false); + //In case that all data was read we don't need to cancel. + executed= true; return res; } diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index c3d9ee41be9..1021f44fcc9 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -88,7 +88,6 @@ public: ~ParallelParsingBlockInputStream() override { waitForAllThreads(); - LOG_TRACE(&Poco::Logger::get("~ParallelParsingBLockInputStream()"), "All threads are killed."); } void cancel(bool kill) override @@ -114,7 +113,12 @@ public: } protected: - void readPrefix() override {} + //void readPrefix() override {} + + void readSuffix() override { + readers[segmentator_ticket_number % max_threads_to_use]->readPrefix(); + LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::readSuffix()"), "ReadSuffix"); + } //Reader routine Block readImpl() override; @@ -135,6 +139,7 @@ private: const size_t min_chunk_size; std::atomic is_exception_occured{false}; + std::atomic executed{false}; BlockMissingValues last_block_missing_values; diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index b883e42e8ad..9fa22ab8248 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -129,6 +129,8 @@ BlockInputStreamPtr FormatFactory::getInput( const size_t max_threads_for_parallel_parsing = settings.max_threads_for_parallel_parsing; const size_t max_threads_to_use = max_threads_for_parallel_parsing == 0 ? global_max_threads : std::min(max_threads_for_parallel_parsing, global_max_threads); + LOG_TRACE(&Poco::Logger::get("FormatFactory::getInput()"), "Will use " << max_threads_to_use << " threads for parallel parsing."); + auto params = ParallelParsingBlockInputStream::InputCreatorParams{sample, context, row_input_format_params, format_settings}; ParallelParsingBlockInputStream::Builder builder{buf, input_getter, params, file_segmentation_engine, max_threads_to_use, settings.min_chunk_size_for_parallel_parsing}; return std::make_shared(builder); diff --git a/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 55e65cf65db..35b39fc8a69 100644 --- a/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -8,7 +8,6 @@ #include #include - namespace DB { @@ -201,8 +200,15 @@ void PrettyBlockOutputFormat::writeValueWithPadding( { auto writePadding = [&]() { +// if (pad_to_width < value_width) +// return; if (pad_to_width < value_width) + { + std::cout << "pad_to_width and value_width " << pad_to_width << " " << value_width << std::endl; + std::cout << StackTrace().toString() << std::endl; return; + } + for (size_t k = 0; k < pad_to_width - value_width; ++k) writeChar(' ', out); }; From 0d3a05ce3b64ec7673e6b3a1c918e25d183e6ce2 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 5 Nov 2019 15:58:32 +0300 Subject: [PATCH 26/90] remove logging + fix for PrettySpace + overrided readSuffix and readPrefix --- .../DataStreams/ParallelParsingBlockInputStream.h | 12 +++--------- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 9 --------- 2 files changed, 3 insertions(+), 18 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 1021f44fcc9..96d459aa2fe 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -61,7 +61,7 @@ public: pool(builder.max_threads_to_use), file_segmentation_engine(builder.file_segmentation_engine) { - LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream()"), "Constructor"); + //LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream()"), "Constructor"); segments.resize(max_threads_to_use); blocks.resize(max_threads_to_use); exceptions.resize(max_threads_to_use); @@ -92,7 +92,7 @@ public: void cancel(bool kill) override { - LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::cancel()"), "Try to cancel."); + //LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::cancel()"), "Try to cancel."); if (kill) is_killed = true; bool old_val = false; @@ -104,7 +104,7 @@ public: reader->cancel(kill); waitForAllThreads(); - LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::cancel()"), "Cancelled succsessfully."); + //LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::cancel()"), "Cancelled succsessfully."); } Block getHeader() const override @@ -113,12 +113,6 @@ public: } protected: - //void readPrefix() override {} - - void readSuffix() override { - readers[segmentator_ticket_number % max_threads_to_use]->readPrefix(); - LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::readSuffix()"), "ReadSuffix"); - } //Reader routine Block readImpl() override; diff --git a/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 35b39fc8a69..5a0a43f487a 100644 --- a/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -200,15 +200,6 @@ void PrettyBlockOutputFormat::writeValueWithPadding( { auto writePadding = [&]() { -// if (pad_to_width < value_width) -// return; - if (pad_to_width < value_width) - { - std::cout << "pad_to_width and value_width " << pad_to_width << " " << value_width << std::endl; - std::cout << StackTrace().toString() << std::endl; - return; - } - for (size_t k = 0; k < pad_to_width - value_width; ++k) writeChar(' ', out); }; From 5416914862dcaa3febaea81be8d9583bd71c76bd Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 5 Nov 2019 16:26:11 +0300 Subject: [PATCH 27/90] remove logging --- dbms/src/Formats/FormatFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index ec218ead382..d54cff43551 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -129,7 +129,7 @@ BlockInputStreamPtr FormatFactory::getInput( const size_t max_threads_for_parallel_parsing = settings.max_threads_for_parallel_parsing; const size_t max_threads_to_use = max_threads_for_parallel_parsing == 0 ? global_max_threads : std::min(max_threads_for_parallel_parsing, global_max_threads); - LOG_TRACE(&Poco::Logger::get("FormatFactory::getInput()"), "Will use " << max_threads_to_use << " threads for parallel parsing."); + //LOG_TRACE(&Poco::Logger::get("FormatFactory::getInput()"), "Will use " << max_threads_to_use << " threads for parallel parsing."); auto params = ParallelParsingBlockInputStream::InputCreatorParams{sample, context, row_input_format_params, format_settings}; ParallelParsingBlockInputStream::Builder builder{buf, input_getter, params, file_segmentation_engine, max_threads_to_use, settings.min_chunk_size_for_parallel_parsing}; From 220ccca2820658bddf4a57b7a918ef9478211be1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 6 Nov 2019 21:54:34 +0300 Subject: [PATCH 28/90] executed in destructor --- dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp | 8 ++++---- dbms/src/DataStreams/ParallelParsingBlockInputStream.h | 1 + 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index 02b0c9d47fa..2f164e5186d 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -8,14 +8,14 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() setThreadName("Segmentator"); try { - while (!is_cancelled && !is_exception_occured) + while (!is_cancelled && !is_exception_occured && !executed) { ++segmentator_ticket_number; const auto current_unit_number = segmentator_ticket_number % max_threads_to_use; { std::unique_lock lock(mutex); - segmentator_condvar.wait(lock, [&]{ return status[current_unit_number] == READY_TO_INSERT || is_exception_occured || is_cancelled; }); + segmentator_condvar.wait(lock, [&]{ return status[current_unit_number] == READY_TO_INSERT || is_exception_occured || is_cancelled || executed; }); } if (is_exception_occured) @@ -119,12 +119,12 @@ Block ParallelParsingBlockInputStream::readImpl() std::unique_lock lock(mutex); const auto current_number = reader_ticket_number % max_threads_to_use; - reader_condvar.wait(lock, [&](){ return status[current_number] == READY_TO_READ || is_exception_occured || is_cancelled; }); + reader_condvar.wait(lock, [&](){ return status[current_number] == READY_TO_READ || is_exception_occured || is_cancelled || executed; }); /// Check for an exception and rethrow it if (is_exception_occured) { - LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::readImpl()"), "Exception occured. Will cancel the query."); + //LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::readImpl()"), "Exception occured. Will cancel the query."); lock.unlock(); cancel(false); rethrowFirstException(exceptions); diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 96d459aa2fe..961c36b7bb3 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -87,6 +87,7 @@ public: ~ParallelParsingBlockInputStream() override { + executed = true; waitForAllThreads(); } From 8358b648721c0d1fc1ecdf3be58e34436fdc0540 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 11 Nov 2019 13:32:25 +0300 Subject: [PATCH 29/90] comment --- dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp | 4 ++-- dbms/src/DataStreams/ParallelParsingBlockInputStream.h | 2 ++ dbms/src/IO/ReadHelpers.h | 8 ++++++++ 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index 2f164e5186d..d2b8d65050c 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -15,7 +15,7 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() { std::unique_lock lock(mutex); - segmentator_condvar.wait(lock, [&]{ return status[current_unit_number] == READY_TO_INSERT || is_exception_occured || is_cancelled || executed; }); + segmentator_condvar.wait(lock, [&]{ return status[current_unit_number] == READY_TO_INSERT || is_exception_occured || executed; }); } if (is_exception_occured) @@ -119,7 +119,7 @@ Block ParallelParsingBlockInputStream::readImpl() std::unique_lock lock(mutex); const auto current_number = reader_ticket_number % max_threads_to_use; - reader_condvar.wait(lock, [&](){ return status[current_number] == READY_TO_READ || is_exception_occured || is_cancelled || executed; }); + reader_condvar.wait(lock, [&](){ return status[current_number] == READY_TO_READ || is_exception_occured || executed; }); /// Check for an exception and rethrow it if (is_exception_occured) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 961c36b7bb3..cf56c12ed09 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -100,6 +100,8 @@ public: if (!is_cancelled.compare_exchange_strong(old_val, true)) return; + executed = true; + for (auto& reader: readers) if (!reader->isCancelled()) reader->cancel(kill); diff --git a/dbms/src/IO/ReadHelpers.h b/dbms/src/IO/ReadHelpers.h index 44941d741d7..e6b3e7e91e3 100644 --- a/dbms/src/IO/ReadHelpers.h +++ b/dbms/src/IO/ReadHelpers.h @@ -914,6 +914,14 @@ void skipToUnescapedNextLineOrEOF(ReadBuffer & buf); /** Returns buffer eof() result. * And saves data if there is no pending data in buffer or it was explicitly asked. + * Why we have to use this strange function? Consider we have begin_pos in the middle of our buffer + * and the cursor in the end of the buffer. When we call eof() it calls next(). + * And this function can fill the buffer with new data, so we will lose the data from previous buffer state. + * @param buf - original buffer to read from. + * memory - where to put data from buf + * used_size - special parameter not to do useless reallocations + * begin_pos - defines from which position we will copy the data. + * forse_saving_buffer_state - allows to explicitly copy all the data from begin_pos to current_position. */ bool eofWithSavingBufferState(ReadBuffer & buf, DB::Memory<> & memory, size_t & used_size, char * & begin_pos, bool force_saving_buffer_state = false); From 57663c395a1ce0642907887ad0eb5a37d56d2e8e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 11 Nov 2019 14:20:11 +0300 Subject: [PATCH 30/90] processing unit added --- .../ParallelParsingBlockInputStream.cpp | 64 ++++++++++--------- .../ParallelParsingBlockInputStream.h | 58 +++++++---------- 2 files changed, 59 insertions(+), 63 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index d2b8d65050c..1613671e240 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -1,4 +1,5 @@ #include +#include "ParallelParsingBlockInputStream.h" namespace DB { @@ -12,41 +13,42 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() { ++segmentator_ticket_number; const auto current_unit_number = segmentator_ticket_number % max_threads_to_use; + auto & unit = processing_units[current_unit_number]; { std::unique_lock lock(mutex); - segmentator_condvar.wait(lock, [&]{ return status[current_unit_number] == READY_TO_INSERT || is_exception_occured || executed; }); + segmentator_condvar.wait(lock, [&]{ return unit.status == READY_TO_INSERT || is_exception_occured || executed; }); } if (is_exception_occured) break; // Segmentating the original input. - segments[current_unit_number].used_size = 0; + unit.segment.used_size = 0; //It returns bool, but it is useless - const auto res = file_segmentation_engine(original_buffer, segments[current_unit_number].memory, segments[current_unit_number].used_size, min_chunk_size); + const auto res = file_segmentation_engine(original_buffer, unit.segment.memory, unit.segment.used_size, min_chunk_size); if (!res) { - is_last[current_unit_number] = true; - status[current_unit_number] = READY_TO_PARSE; + unit.is_last = true; + unit.status = READY_TO_PARSE; scheduleParserThreadForUnitWithNumber(current_unit_number); break; } // Creating buffer from the segment of data. - auto new_buffer = BufferBase::Buffer(segments[current_unit_number].memory.data(), - segments[current_unit_number].memory.data() + segments[current_unit_number].used_size); + auto new_buffer = BufferBase::Buffer(unit.segment.memory.data(), + unit.segment.memory.data() + unit.segment.used_size); - buffers[current_unit_number]->buffer().swap(new_buffer); - buffers[current_unit_number]->position() = buffers[current_unit_number]->buffer().begin(); + unit.readbuffer->buffer().swap(new_buffer); + unit.readbuffer->position() = unit.readbuffer->buffer().begin(); - readers[current_unit_number] = std::make_unique ( - input_processor_creator(*buffers[current_unit_number], header, context, row_input_format_params, format_settings) + unit.parser = std::make_unique( + input_processor_creator(*unit.readbuffer, header, context, row_input_format_params, format_settings) ); - status[current_unit_number] = READY_TO_PARSE; + unit.status = READY_TO_PARSE; scheduleParserThreadForUnitWithNumber(current_unit_number); } } @@ -63,19 +65,21 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n if (is_exception_occured && is_cancelled) return; + auto & unit = processing_units[current_unit_number]; + try { { std::unique_lock lock(mutex); - blocks[current_unit_number].block.clear(); - blocks[current_unit_number].block_missing_values.clear(); + unit.block_ext.block.clear(); + unit.block_ext.block_missing_values.clear(); - if (is_last[current_unit_number] || buffers[current_unit_number]->position() == nullptr) + if (unit.is_last || unit.readbuffer->position() == nullptr) { - 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; + unit.block_ext.block.emplace_back(Block()); + unit.block_ext.block_missing_values.emplace_back(BlockMissingValues()); + unit.status = READY_TO_READ; reader_condvar.notify_all(); return; } @@ -85,18 +89,18 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n //We don't know how many blocks will be. So we have to read them all until an empty block occured. while (true) { - auto block = readers[current_unit_number]->read(); + auto block = unit.parser->read(); if (block == Block()) break; - blocks[current_unit_number].block.emplace_back(block); - blocks[current_unit_number].block_missing_values.emplace_back(readers[current_unit_number]->getMissingValues()); + unit.block_ext.block.emplace_back(block); + unit.block_ext.block_missing_values.emplace_back(unit.parser->getMissingValues()); } { std::unique_lock lock(mutex); - status[current_unit_number] = READY_TO_READ; + unit.status = READY_TO_READ; reader_condvar.notify_all(); } @@ -118,8 +122,10 @@ Block ParallelParsingBlockInputStream::readImpl() return res; std::unique_lock lock(mutex); - const auto current_number = reader_ticket_number % max_threads_to_use; - reader_condvar.wait(lock, [&](){ return status[current_number] == READY_TO_READ || is_exception_occured || executed; }); + const auto current_unit_number = reader_ticket_number % max_threads_to_use; + auto & unit = processing_units[current_unit_number]; + + reader_condvar.wait(lock, [&](){ return unit.status == READY_TO_READ || is_exception_occured || executed; }); /// Check for an exception and rethrow it if (is_exception_occured) @@ -130,12 +136,12 @@ Block ParallelParsingBlockInputStream::readImpl() rethrowFirstException(exceptions); } - res = std::move(blocks[current_number].block.at(internal_block_iter)); - last_block_missing_values = std::move(blocks[current_number].block_missing_values[internal_block_iter]); + res = std::move(unit.block_ext.block.at(internal_block_iter)); + last_block_missing_values = std::move(unit.block_ext.block_missing_values[internal_block_iter]); - if (++internal_block_iter == blocks[current_number].block.size()) + if (++internal_block_iter == unit.block_ext.block.size()) { - if (is_last[current_number]) + if (unit.is_last) { //In case that all data was read we don't need to cancel. executed= true; @@ -144,7 +150,7 @@ Block ParallelParsingBlockInputStream::readImpl() internal_block_iter = 0; ++reader_ticket_number; - status[current_number] = READY_TO_INSERT; + unit.status = READY_TO_INSERT; segmentator_condvar.notify_all(); } diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index cf56c12ed09..78f3c24d4d1 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -41,7 +41,7 @@ public: struct Builder { - ReadBuffer &read_buffer; + ReadBuffer & read_buffer; const InputProcessorCreator &input_processor_creator; const InputCreatorParams &input_creator_params; FormatFactory::FileSegmentationEngine file_segmentation_engine; @@ -62,23 +62,9 @@ public: file_segmentation_engine(builder.file_segmentation_engine) { //LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream()"), "Constructor"); - segments.resize(max_threads_to_use); - blocks.resize(max_threads_to_use); - exceptions.resize(max_threads_to_use); - buffers.reserve(max_threads_to_use); - readers.reserve(max_threads_to_use); - is_last.assign(max_threads_to_use, false); for (size_t i = 0; i < max_threads_to_use; ++i) - { - status.emplace_back(ProcessingUnitStatus::READY_TO_INSERT); - buffers.emplace_back(std::make_unique(segments[i].memory.data(), segments[i].used_size, 0)); - readers.emplace_back(std::make_unique(builder.input_processor_creator(*buffers[i], - builder.input_creator_params.sample, - builder.input_creator_params.context, - builder.input_creator_params.row_input_format_params, - builder.input_creator_params.settings))); - } + processing_units.emplace_back(builder); segmentator_thread = ThreadFromGlobalPool([this] { segmentatorThreadFunction(); }); } @@ -102,9 +88,9 @@ public: executed = true; - for (auto& reader: readers) - if (!reader->isCancelled()) - reader->cancel(kill); + for (auto& unit: processing_units) + if (!unit.parser->isCancelled()) + unit.parser->cancel(kill); waitForAllThreads(); //LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::cancel()"), "Cancelled succsessfully."); @@ -116,7 +102,6 @@ public: } protected: - //Reader routine Block readImpl() override; @@ -179,23 +164,28 @@ private: std::vector block_missing_values; }; - using Blocks = std::vector; - using ReadBuffers = std::vector>; - using Segments = std::vector; - using Status = std::deque>; - using InputStreamFromInputFormats = std::vector>; + struct ProcessingUnit + { + explicit ProcessingUnit(const Builder & builder) : status(ProcessingUnitStatus::READY_TO_INSERT) + { + readbuffer = std::make_unique(segment.memory.data(), segment.used_size, 0); + parser = std::make_unique(builder.input_processor_creator(*readbuffer, + builder.input_creator_params.sample, + builder.input_creator_params.context, + builder.input_creator_params.row_input_format_params, + builder.input_creator_params.settings)); + } - //We cannot use std::vector because it is equal to bitset (which stores 8 bool in one byte). - //That's why dataraces occured. - using IsLastFlags = std::vector; + BlockExt block_ext; + std::unique_ptr readbuffer; + MemoryExt segment; + std::unique_ptr parser; + std::atomic status; + char is_last{false}; + }; - Segments segments; - ReadBuffers buffers; - Blocks blocks; Exceptions exceptions; - Status status; - InputStreamFromInputFormats readers; - IsLastFlags is_last; + std::deque processing_units; void scheduleParserThreadForUnitWithNumber(size_t unit_number) { From 7dda8a902d1d85b66688a479211e24f55066a7ef Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 11 Nov 2019 16:42:07 +0300 Subject: [PATCH 31/90] fix stupid mistake --- dbms/src/DataStreams/ParallelParsingBlockInputStream.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 78f3c24d4d1..ad9464f13c3 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -62,7 +62,7 @@ public: file_segmentation_engine(builder.file_segmentation_engine) { //LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream()"), "Constructor"); - + exceptions.resize(max_threads_to_use); for (size_t i = 0; i < max_threads_to_use; ++i) processing_units.emplace_back(builder); @@ -184,6 +184,7 @@ private: char is_last{false}; }; + /// We use separate exceptions because there is convenient rethrowFirstException function. Exceptions exceptions; std::deque processing_units; From c335c4f2dc72c90a1c3ca365fd2bfd48dc41b743 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 12 Nov 2019 01:42:21 +0300 Subject: [PATCH 32/90] empty --- dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index 1613671e240..12f977c6feb 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -103,7 +103,6 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n unit.status = READY_TO_READ; reader_condvar.notify_all(); } - } catch (...) { From 94c2c7bbd28c926834fa931cffc532692705bacf Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 13 Nov 2019 21:35:35 +0300 Subject: [PATCH 33/90] more tests for table functions --- .../integration/test_globs_in_filepath/test.py | 8 +++++++- .../test_insert_into_distributed/test.py | 12 ++++++++++++ .../test_mysql_database_engine/test.py | 15 +++++++++++++++ 3 files changed, 34 insertions(+), 1 deletion(-) diff --git a/dbms/tests/integration/test_globs_in_filepath/test.py b/dbms/tests/integration/test_globs_in_filepath/test.py index db794c35d47..cce575d6e84 100644 --- a/dbms/tests/integration/test_globs_in_filepath/test.py +++ b/dbms/tests/integration/test_globs_in_filepath/test.py @@ -112,4 +112,10 @@ def test_deep_structure(start_cluster): '''.format(pattern)) == '{}\n'.format(value) assert node.query(''' select count(*) from file('{}{}', 'TSV', 'text String, number Float64') - '''.format(path_to_userfiles_from_defaut_config, pattern)) == '{}\n'.format(value) \ No newline at end of file + '''.format(path_to_userfiles_from_defaut_config, pattern)) == '{}\n'.format(value) + +def test_table_function(start_cluster): + node.exec_in_container(['bash', '-c', 'mkdir -p {}some/path/to/'.format(path_to_userfiles_from_defaut_config)]) + node.exec_in_container(['bash', '-c', 'touch {}some/path/to/data.CSV'.format(path_to_userfiles_from_defaut_config)]) + node.query("insert into table function file('some/path/to/data.CSV', CSV, 'n UInt8, s String') select number, concat('str_', toString(number)) from numbers(100000)") + assert node.query("select count() from file('some/path/to/data.CSV', CSV, 'n UInt8, s String')").rstrip() == '100000' diff --git a/dbms/tests/integration/test_insert_into_distributed/test.py b/dbms/tests/integration/test_insert_into_distributed/test.py index 622fb01eff6..8d656cef3ea 100644 --- a/dbms/tests/integration/test_insert_into_distributed/test.py +++ b/dbms/tests/integration/test_insert_into_distributed/test.py @@ -68,6 +68,14 @@ CREATE TABLE low_cardinality (d Date, x UInt32, s LowCardinality(String)) ENGINE shard1.query(''' CREATE TABLE low_cardinality_all (d Date, x UInt32, s LowCardinality(String)) ENGINE = Distributed('shard_with_low_cardinality', 'default', 'low_cardinality', sipHash64(s))''') + node1.query(''' +CREATE TABLE table_function (n UInt8, s String) ENGINE = MergeTree() ORDER BY n''') + + node2.query(''' +CREATE TABLE table_function (n UInt8, s String) ENGINE = MergeTree() ORDER BY n''') + + + yield cluster finally: @@ -189,3 +197,7 @@ def test_inserts_low_cardinality(started_cluster): instance.query("INSERT INTO low_cardinality_all (d,x,s) VALUES ('2018-11-12',1,'123')") time.sleep(0.5) assert instance.query("SELECT count(*) FROM low_cardinality_all").strip() == '1' + +def test_table_function(started_cluster): + node1.query("insert into table function cluster('shard_with_local_replica', 'default', 'table_function') select number, concat('str_', toString(number)) from numbers(100000)") + assert node1.query("select count() from cluster('shard_with_local_replica', 'default', 'table_function')").rstrip() == '100000' diff --git a/dbms/tests/integration/test_mysql_database_engine/test.py b/dbms/tests/integration/test_mysql_database_engine/test.py index 430083ccc82..de996c791dc 100644 --- a/dbms/tests/integration/test_mysql_database_engine/test.py +++ b/dbms/tests/integration/test_mysql_database_engine/test.py @@ -108,10 +108,25 @@ def test_insert_select_with_mysql_style_table(started_cluster): assert node1.query("SELECT sum(`float`) FROM `clickhouse_mysql`.`{}`".format('test_mysql\`_style_table')).rstrip() == '30000' mysql_connection.close() +def test_table_function(started_cluster): + mysql_connection = get_mysql_conn() + create_normal_mysql_table(mysql_connection, 'table_function') + table_function = get_mysql_table_function_expr('table_function') + node1.query("INSERT INTO {} (id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000)".format('TABLE FUNCTION ' + table_function)) + node1.query("SELECT * FROM {}".format(table_function)) # should fail with asan + assert node1.query("SELECT count() FROM {}".format(table_function)).rstrip() == '10000' + assert node1.query("SELECT sum(`float`) FROM {}".format(table_function)).rstrip() == '30000' + mysql_connection.close() + + def get_mysql_conn(): conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=3308) return conn +def get_mysql_table_function_expr(table_name): + # FIXME We need extra round brackets after "mysql" to distinguish table function with list of columns from an aggregate function with list of parameters (see ParserInsertQuery and ParserFunction) + return "mysql()('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format(table_name) + def create_mysql_db(conn, name): with conn.cursor() as cursor: cursor.execute( From 5e789e4250906066c172f0dfe12d2475394db021 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 14 Nov 2019 16:57:10 +0300 Subject: [PATCH 34/90] Some renames & remove weird logic from cancel() --- .../ParallelParsingBlockInputStream.cpp | 11 ++++--- .../ParallelParsingBlockInputStream.h | 30 ++++++++++--------- 2 files changed, 21 insertions(+), 20 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index 12f977c6feb..001cb4a2ea9 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -9,7 +9,7 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() setThreadName("Segmentator"); try { - while (!is_cancelled && !is_exception_occured && !executed) + while (!is_cancelled && !is_exception_occured && !finished) { ++segmentator_ticket_number; const auto current_unit_number = segmentator_ticket_number % max_threads_to_use; @@ -17,7 +17,7 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() { std::unique_lock lock(mutex); - segmentator_condvar.wait(lock, [&]{ return unit.status == READY_TO_INSERT || is_exception_occured || executed; }); + segmentator_condvar.wait(lock, [&]{ return unit.status == READY_TO_INSERT || is_exception_occured || finished; }); } if (is_exception_occured) @@ -117,19 +117,18 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n Block ParallelParsingBlockInputStream::readImpl() { Block res; - if (isCancelledOrThrowIfKilled() || executed) + if (isCancelledOrThrowIfKilled() || finished) return res; std::unique_lock lock(mutex); const auto current_unit_number = reader_ticket_number % max_threads_to_use; auto & unit = processing_units[current_unit_number]; - reader_condvar.wait(lock, [&](){ return unit.status == READY_TO_READ || is_exception_occured || executed; }); + reader_condvar.wait(lock, [&](){ return unit.status == READY_TO_READ || is_exception_occured || finished; }); /// Check for an exception and rethrow it if (is_exception_occured) { - //LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::readImpl()"), "Exception occured. Will cancel the query."); lock.unlock(); cancel(false); rethrowFirstException(exceptions); @@ -143,7 +142,7 @@ Block ParallelParsingBlockInputStream::readImpl() if (unit.is_last) { //In case that all data was read we don't need to cancel. - executed= true; + finished = true; return res; } diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index ad9464f13c3..435fa390ab5 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -73,27 +73,23 @@ public: ~ParallelParsingBlockInputStream() override { - executed = true; - waitForAllThreads(); + finishAndWait(); } void cancel(bool kill) override { - //LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::cancel()"), "Try to cancel."); + /** + * Can be called multiple times, from different threads. Saturate the + * the kill flag with OR. + */ if (kill) is_killed = true; - bool old_val = false; - if (!is_cancelled.compare_exchange_strong(old_val, true)) - return; - - executed = true; + is_cancelled = true; for (auto& unit: processing_units) - if (!unit.parser->isCancelled()) - unit.parser->cancel(kill); + unit.parser->cancel(kill); - waitForAllThreads(); - //LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream::cancel()"), "Cancelled succsessfully."); + finishAndWait(); } Block getHeader() const override @@ -121,7 +117,11 @@ private: const size_t min_chunk_size; std::atomic is_exception_occured{false}; - std::atomic executed{false}; + /* + * This is declared as atomic to avoid UB, because parser threads access it + * without synchronization. + */ + std::atomic finished{false}; BlockMissingValues last_block_missing_values; @@ -193,8 +193,10 @@ private: pool.scheduleOrThrowOnError(std::bind(&ParallelParsingBlockInputStream::parserThreadFunction, this, unit_number)); } - void waitForAllThreads() + void finishAndWait() { + finished.store(true, std::memory_order_release); + { std::unique_lock lock(mutex); segmentator_condvar.notify_all(); From db81aae8f0667bd2e9e685849b090d8909ae246b Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 14 Nov 2019 18:53:20 +0300 Subject: [PATCH 35/90] Cleaup --- .../ParallelParsingBlockInputStream.cpp | 191 +++++++++++------- .../ParallelParsingBlockInputStream.h | 68 +++++-- 2 files changed, 170 insertions(+), 89 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index 001cb4a2ea9..45c49a6dba8 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -9,31 +9,44 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() setThreadName("Segmentator"); try { - while (!is_cancelled && !is_exception_occured && !finished) + while (!finished) { - ++segmentator_ticket_number; - const auto current_unit_number = segmentator_ticket_number % max_threads_to_use; + const auto current_unit_number = segmentator_ticket_number % processing_units.size(); auto & unit = processing_units[current_unit_number]; { std::unique_lock lock(mutex); - segmentator_condvar.wait(lock, [&]{ return unit.status == READY_TO_INSERT || is_exception_occured || finished; }); + segmentator_condvar.wait(lock, + [&]{ return unit.status == READY_TO_INSERT || finished; }); } - if (is_exception_occured) + if (finished) + { break; + } + + assert(unit.status == READY_TO_INSERT); // Segmentating the original input. unit.segment.used_size = 0; - //It returns bool, but it is useless - const auto res = file_segmentation_engine(original_buffer, unit.segment.memory, unit.segment.used_size, min_chunk_size); + const bool have_more_data = file_segmentation_engine(original_buffer, + unit.segment.memory, unit.segment.used_size, min_chunk_size); - if (!res) + if (!have_more_data) { - unit.is_last = true; - unit.status = READY_TO_PARSE; - scheduleParserThreadForUnitWithNumber(current_unit_number); + /** + * On EOF, the buffer must be empty. We don't have to start + * the parser since we have no data, and can wake up the reader + * directly. + */ + assert(unit.segment.used_size == 0); + unit.is_past_the_end = true; + unit.status = READY_TO_READ; + + std::unique_lock lock(mutex); + reader_condvar.notify_all(); + break; } @@ -50,54 +63,36 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() unit.status = READY_TO_PARSE; scheduleParserThreadForUnitWithNumber(current_unit_number); + ++segmentator_ticket_number; } } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + onBackgroundException(); } } void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_number) { - setThreadName("ChunkParser"); - - if (is_exception_occured && is_cancelled) - return; - - auto & unit = processing_units[current_unit_number]; - try { + setThreadName("ChunkParser"); + + auto & unit = processing_units[current_unit_number]; + + unit.block_ext.block.clear(); + unit.block_ext.block_missing_values.clear(); + + // We don't know how many blocks will be. So we have to read them all + // until an empty block occured. + Block block; + while (!finished && (block = unit.parser->read()) != Block()) { - std::unique_lock lock(mutex); - - unit.block_ext.block.clear(); - unit.block_ext.block_missing_values.clear(); - - if (unit.is_last || unit.readbuffer->position() == nullptr) - { - unit.block_ext.block.emplace_back(Block()); - unit.block_ext.block_missing_values.emplace_back(BlockMissingValues()); - unit.status = READY_TO_READ; - reader_condvar.notify_all(); - return; - } - - } - - //We don't know how many blocks will be. So we have to read them all until an empty block occured. - while (true) - { - auto block = unit.parser->read(); - - if (block == Block()) - break; - unit.block_ext.block.emplace_back(block); unit.block_ext.block_missing_values.emplace_back(unit.parser->getMissingValues()); } + if (!finished) { std::unique_lock lock(mutex); unit.status = READY_TO_READ; @@ -106,52 +101,100 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n } catch (...) { - std::unique_lock lock(mutex); - tryLogCurrentException(__PRETTY_FUNCTION__); - exceptions[current_unit_number] = std::current_exception(); - is_exception_occured = true; - reader_condvar.notify_all(); + onBackgroundException(); } } +void ParallelParsingBlockInputStream::onBackgroundException() +{ + tryLogCurrentException(__PRETTY_FUNCTION__); + + std::unique_lock lock(mutex); + if (!background_exception) + { + background_exception = std::current_exception(); + } + finished = true; + reader_condvar.notify_all(); + segmentator_condvar.notify_all(); +} + Block ParallelParsingBlockInputStream::readImpl() { - Block res; if (isCancelledOrThrowIfKilled() || finished) - return res; - - std::unique_lock lock(mutex); - const auto current_unit_number = reader_ticket_number % max_threads_to_use; - auto & unit = processing_units[current_unit_number]; - - reader_condvar.wait(lock, [&](){ return unit.status == READY_TO_READ || is_exception_occured || finished; }); - - /// Check for an exception and rethrow it - if (is_exception_occured) { - lock.unlock(); - cancel(false); - rethrowFirstException(exceptions); - } - - res = std::move(unit.block_ext.block.at(internal_block_iter)); - last_block_missing_values = std::move(unit.block_ext.block_missing_values[internal_block_iter]); - - if (++internal_block_iter == unit.block_ext.block.size()) - { - if (unit.is_last) + /** + * Check for background exception and rethrow it before we return. + */ + std::unique_lock lock(mutex); + if (background_exception) { - //In case that all data was read we don't need to cancel. - finished = true; - return res; + lock.unlock(); + cancel(false); + std::rethrow_exception(background_exception); } - internal_block_iter = 0; + return Block{}; + } + + const auto current_unit_number = reader_ticket_number % processing_units.size(); + auto & unit = processing_units[current_unit_number]; + + if (!next_block_in_current_unit.has_value()) + { + // We have read out all the Blocks from the current Processing Unit, + // time to move to the next one. Wait for it to become ready. + std::unique_lock lock(mutex); + reader_condvar.wait(lock, [&](){ return unit.status == READY_TO_READ || finished; }); + + if (finished) + { + /** + * Check for background exception and rethrow it before we return. + */ + if (background_exception) + { + lock.unlock(); + cancel(false); + std::rethrow_exception(background_exception); + } + + return Block{}; + } + + assert(unit.status == READY_TO_READ); + + if (unit.is_past_the_end) + { + // No more data. + return Block{}; + } + next_block_in_current_unit = 0; + } + + assert(next_block_in_current_unit); + assert(!unit.is_past_the_end); + + Block res = std::move(unit.block_ext.block.at(*next_block_in_current_unit)); + last_block_missing_values = std::move(unit.block_ext.block_missing_values[*next_block_in_current_unit]); + + ++*next_block_in_current_unit; + + if (*next_block_in_current_unit == unit.block_ext.block.size()) + { + /** + * Finished reading this Processing Unit, pass it back to the segmentator. + */ + next_block_in_current_unit.reset(); ++reader_ticket_number; + + std::unique_lock lock(mutex); unit.status = READY_TO_INSERT; segmentator_condvar.notify_all(); } return res; } + + } diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 435fa390ab5..242ae49f656 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -18,6 +18,35 @@ namespace DB * It splits original data into chunks. Then each chunk is parsed by different thread. * The number of chunks equals to max_threads_for_parallel_parsing setting. * The size of chunk is equal to min_chunk_size_for_parallel_parsing setting. + * + * This stream has three kinds of threads: one segmentator, multiple parsers + * (max_threads_for_parallel_parsing) and one reader thread -- that is, the one + * from which readImpl() is called. + * They operate one after another on parts of data called "processing units". + * One unit consists of buffer with raw data from file, filled by segmentator + * thread. This raw data is then parsed by a parser thread to form a number of + * Blocks. These Blocks are returned to the parent stream from readImpl(). + * After being read out, a processing unit is reused, to save on allocating + * memory for the raw buffer. The processing units are organized into a circular + * array to facilitate reuse and to apply backpressure on the segmentator thread + * -- after it runs out of processing units, it has to wait for the reader to + * read out the previous blocks. + * The outline of what the threads do is as follows: + * segmentator thread: + * 1) wait for the next processing unit to become empty + * 2) fill it with a part of input file + * 3) start a parser thread + * 4) repeat until eof + * parser thread: + * 1) parse the given raw buffer without any synchronization + * 2) signal that the given unit is ready to read + * 3) finish + * readImpl(): + * 1) wait for the next processing unit to become ready to read + * 2) take the blocks from the processing unit to return them to the caller + * 3) signal that the processing unit is empty + * 4) repeat until it encounters unit that is marked as "past_the_end" + * All threads must also check for cancel/eof/exception flags. */ class ParallelParsingBlockInputStream : public IBlockInputStream { @@ -61,9 +90,9 @@ public: pool(builder.max_threads_to_use), file_segmentation_engine(builder.file_segmentation_engine) { - //LOG_TRACE(&Poco::Logger::get("ParallelParsingBLockInputStream()"), "Constructor"); - exceptions.resize(max_threads_to_use); - for (size_t i = 0; i < max_threads_to_use; ++i) + // Allocate more units than threads to decrease segmentator + // waiting on reader on wraparound. The number is random. + for (size_t i = 0; i < builder.max_threads_to_use + 4; ++i) processing_units.emplace_back(builder); segmentator_thread = ThreadFromGlobalPool([this] { segmentatorThreadFunction(); }); @@ -116,7 +145,6 @@ private: const std::atomic max_threads_to_use; const size_t min_chunk_size; - std::atomic is_exception_occured{false}; /* * This is declared as atomic to avoid UB, because parser threads access it * without synchronization. @@ -129,9 +157,9 @@ private: ReadBuffer & original_buffer; //Non-atomic because it is used in one thread. - size_t reader_ticket_number{1}; - size_t internal_block_iter{0}; + std::optional next_block_in_current_unit; size_t segmentator_ticket_number{0}; + size_t reader_ticket_number{0}; std::mutex mutex; std::condition_variable reader_condvar; @@ -169,11 +197,12 @@ private: explicit ProcessingUnit(const Builder & builder) : status(ProcessingUnitStatus::READY_TO_INSERT) { readbuffer = std::make_unique(segment.memory.data(), segment.used_size, 0); - parser = std::make_unique(builder.input_processor_creator(*readbuffer, - builder.input_creator_params.sample, - builder.input_creator_params.context, - builder.input_creator_params.row_input_format_params, - builder.input_creator_params.settings)); + parser = std::make_unique( + builder.input_processor_creator(*readbuffer, + builder.input_creator_params.sample, + builder.input_creator_params.context, + builder.input_creator_params.row_input_format_params, + builder.input_creator_params.settings)); } BlockExt block_ext; @@ -181,13 +210,16 @@ private: MemoryExt segment; std::unique_ptr parser; std::atomic status; - char is_last{false}; + bool is_past_the_end{false}; }; - /// We use separate exceptions because there is convenient rethrowFirstException function. - Exceptions exceptions; + std::exception_ptr background_exception = nullptr; + + // We use deque instead of vector, because it does not require a move + // constructor, which is absent for atomics that are inside ProcessingUnit. std::deque processing_units; + void scheduleParserThreadForUnitWithNumber(size_t unit_number) { pool.scheduleOrThrowOnError(std::bind(&ParallelParsingBlockInputStream::parserThreadFunction, this, unit_number)); @@ -195,7 +227,7 @@ private: void finishAndWait() { - finished.store(true, std::memory_order_release); + finished = true; { std::unique_lock lock(mutex); @@ -218,6 +250,12 @@ private: void segmentatorThreadFunction(); void parserThreadFunction(size_t bucket_num); + + // Save/log a background exception, set termination flag, wake up all + // threads. This function is used by segmentator and parsed threads. + // readImpl() is called from the main thread, so the exception handling + // is different. + void onBackgroundException(); }; }; From 2beed0d4cf2491c0500587596ed4b2173378fc26 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 14 Nov 2019 19:00:02 +0300 Subject: [PATCH 36/90] check for sanitizer assert in integration tests --- dbms/tests/integration/helpers/cluster.py | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/dbms/tests/integration/helpers/cluster.py b/dbms/tests/integration/helpers/cluster.py index 72bd070f8ec..2510312b1b7 100644 --- a/dbms/tests/integration/helpers/cluster.py +++ b/dbms/tests/integration/helpers/cluster.py @@ -29,6 +29,9 @@ from .hdfs_api import HDFSApi HELPERS_DIR = p.dirname(__file__) DEFAULT_ENV_NAME = 'env_file' +SANITIZER_SIGN = "==================" + + def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): full_path = os.path.join(path, fname) with open(full_path, 'w') as f: @@ -81,6 +84,7 @@ class ClickHouseCluster: # docker-compose removes everything non-alphanumeric from project names so we do it too. self.project_name = re.sub(r'[^a-z0-9]', '', self.project_name.lower()) self.instances_dir = p.join(self.base_dir, '_instances' + ('' if not self.name else '_' + self.name)) + self.docker_logs_path = p.join(self.instances_dir, 'docker.log') custom_dockerd_host = custom_dockerd_host or os.environ.get('CLICKHOUSE_TESTS_DOCKERD_HOST') self.docker_api_version = os.environ.get("DOCKER_API_VERSION") @@ -395,6 +399,15 @@ class ClickHouseCluster: def shutdown(self, kill=True): + sanitizer_assert_instance = None + with open(self.docker_logs_path, "w+") as f: + subprocess.check_call(self.base_cmd + ['logs'], stdout=f) + f.seek(0) + for line in f: + if SANITIZER_SIGN in line: + sanitizer_assert_instance = line.split('|')[0].strip() + break + if kill: subprocess_check_call(self.base_cmd + ['kill']) subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans']) @@ -407,6 +420,9 @@ class ClickHouseCluster: instance.ip_address = None instance.client = None + if sanitizer_assert_instance is not None: + raise Exception("Sanitizer assert found in {} for instance {}".format(self.docker_logs_path, sanitizer_assert_instance)) + def open_bash_shell(self, instance_name): os.system(' '.join(self.base_cmd + ['exec', instance_name, '/bin/bash'])) From ce92615addd554a62f79e8ed2ff4c85f5bf468a4 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 14 Nov 2019 21:53:57 +0300 Subject: [PATCH 37/90] remove unused variable --- dbms/src/DataStreams/ParallelParsingBlockInputStream.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 242ae49f656..4269960c1a2 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -84,7 +84,6 @@ public: 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), @@ -142,7 +141,6 @@ private: const FormatSettings format_settings; const InputProcessorCreator input_processor_creator; - const std::atomic max_threads_to_use; const size_t min_chunk_size; /* From 8f3bd8f54613e3fceb050f315e4c6804f64a1664 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 15 Nov 2019 20:23:57 +0300 Subject: [PATCH 38/90] Fix typo in Memory: reuse buffer if capacity allows. --- dbms/src/IO/BufferWithOwnMemory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/IO/BufferWithOwnMemory.h b/dbms/src/IO/BufferWithOwnMemory.h index 4849a52344b..713c9b12cd1 100644 --- a/dbms/src/IO/BufferWithOwnMemory.h +++ b/dbms/src/IO/BufferWithOwnMemory.h @@ -77,7 +77,7 @@ struct Memory : boost::noncopyable, Allocator m_capacity = new_size; alloc(); } - else if (new_size <= m_size) + else if (new_size <= m_capacity - pad_right) { m_size = new_size; return; From 5d5882d92b83a7d7e86905e2a02faccbec192783 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 15 Nov 2019 21:08:17 +0300 Subject: [PATCH 39/90] wip: a saner segmentation function for TSV --- .../ParallelParsingBlockInputStream.cpp | 71 ++++++++-------- .../ParallelParsingBlockInputStream.h | 2 +- .../Impl/TabSeparatedRowInputFormat.cpp | 84 ++++++++++++++----- 3 files changed, 99 insertions(+), 58 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index 45c49a6dba8..5e3bebf1de9 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -29,27 +29,11 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() // Segmentating the original input. unit.segment.used_size = 0; + unit.segment.memory.resize(0); const bool have_more_data = file_segmentation_engine(original_buffer, unit.segment.memory, unit.segment.used_size, min_chunk_size); - if (!have_more_data) - { - /** - * On EOF, the buffer must be empty. We don't have to start - * the parser since we have no data, and can wake up the reader - * directly. - */ - assert(unit.segment.used_size == 0); - unit.is_past_the_end = true; - unit.status = READY_TO_READ; - - std::unique_lock lock(mutex); - reader_condvar.notify_all(); - - break; - } - // Creating buffer from the segment of data. auto new_buffer = BufferBase::Buffer(unit.segment.memory.data(), unit.segment.memory.data() + unit.segment.used_size); @@ -61,9 +45,19 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() input_processor_creator(*unit.readbuffer, header, context, row_input_format_params, format_settings) ); + if (!have_more_data) + { + unit.is_last = true; + } + unit.status = READY_TO_PARSE; scheduleParserThreadForUnitWithNumber(current_unit_number); ++segmentator_ticket_number; + + if (!have_more_data) + { + break; + } } } catch (...) @@ -142,8 +136,8 @@ Block ParallelParsingBlockInputStream::readImpl() if (!next_block_in_current_unit.has_value()) { - // We have read out all the Blocks from the current Processing Unit, - // time to move to the next one. Wait for it to become ready. + // We have read out all the Blocks from the previous Processing Unit, + // wait for the current one to become ready. std::unique_lock lock(mutex); reader_condvar.wait(lock, [&](){ return unit.status == READY_TO_READ || finished; }); @@ -163,34 +157,41 @@ Block ParallelParsingBlockInputStream::readImpl() } assert(unit.status == READY_TO_READ); - - if (unit.is_past_the_end) - { - // No more data. - return Block{}; - } next_block_in_current_unit = 0; } - assert(next_block_in_current_unit); - assert(!unit.is_past_the_end); + if (unit.block_ext.block.size() == 0) + { + assert(unit.is_last); + finished = true; + return Block{}; + } + + assert(next_block_in_current_unit.value() < unit.block_ext.block.size()); Block res = std::move(unit.block_ext.block.at(*next_block_in_current_unit)); last_block_missing_values = std::move(unit.block_ext.block_missing_values[*next_block_in_current_unit]); - ++*next_block_in_current_unit; + next_block_in_current_unit.value() += 1; - if (*next_block_in_current_unit == unit.block_ext.block.size()) + if (next_block_in_current_unit.value() == unit.block_ext.block.size()) { - /** - * Finished reading this Processing Unit, pass it back to the segmentator. - */ + // Finished reading this Processing Unit, move to the next one. next_block_in_current_unit.reset(); ++reader_ticket_number; - std::unique_lock lock(mutex); - unit.status = READY_TO_INSERT; - segmentator_condvar.notify_all(); + if (unit.is_last) + { + // It it was the last unit, we're finished. + finished = true; + } + else + { + // Pass the unit back to the segmentator. + std::unique_lock lock(mutex); + unit.status = READY_TO_INSERT; + segmentator_condvar.notify_all(); + } } return res; diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 4269960c1a2..f08e7103011 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -208,7 +208,7 @@ private: MemoryExt segment; std::unique_ptr parser; std::atomic status; - bool is_past_the_end{false}; + bool is_last{false}; }; std::exception_ptr background_exception = nullptr; diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index f13b732b371..db2aa052d72 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -384,36 +384,76 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) } } -bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t & used_size, size_t min_chunk_size) +bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t & used_size, size_t min_chunk_bytes) { - if (in.eof()) - return false; - - char * begin_pos = in.position(); - bool need_more_data = true; - memory.resize(min_chunk_size); - while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && need_more_data) + for (;;) { - in.position() = find_first_symbols<'\\', '\r', '\n'>(in.position(), in.buffer().end()); - if (in.position() == in.buffer().end()) + if (in.eof()) { - continue; + used_size = memory.size(); // remove me + return false; } - if (*in.position() == '\\') + const auto old_total_bytes = memory.size(); + + // Calculate the minimal amount of bytes we must read for this chunk. + // The chunk size may be already bigger than the required minimum, if + // we have a giant row and still haven't read up to the separator. + const auto min_bytes_needed = (min_chunk_bytes >= old_total_bytes) + ? min_chunk_bytes - old_total_bytes : 0; + + // The start position might be over the in.buffer().end(), it's OK -- + // find_first_symbols will process this correctly and return + // in.buffer().end(). + //char * next_separator = in.position() + min_bytes_needed; + bool found_separator = false; + char * chunk_end = in.position() + min_bytes_needed; + // Loop to skip the escaped line separators. + for (;;) { - ++in.position(); - if (!eofWithSavingBufferState(in, memory, used_size, begin_pos, true)) - ++in.position(); - } else if (*in.position() == '\n' || *in.position() == '\r') - { - if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) - need_more_data = false; - ++in.position(); + const auto next_separator = find_first_symbols<'\r', '\n'>(chunk_end, + in.buffer().end()); + assert(next_separator <= in.buffer().end()); + + if (next_separator == in.buffer().end()) + { + // Got to end of buffer, return it. + chunk_end = in.buffer().end(); + break; + } + + chunk_end = next_separator + 1; + + // We found a line separator character, check whether it is escaped by + // checking if there is a '\' to the left. The previous character may + // have been read on the previous loop, in this case we read it from + // 'memory' buffer. + if ((next_separator > in.position() && *(next_separator - 1) != '\\') + || (next_separator == in.position() && memory[memory.size() - 1] != '\\')) + { + found_separator = true; + break; + } + // This is an escaped separator, loop further. } + + const auto bytes_read_now = chunk_end - in.position(); + const auto new_total_bytes = old_total_bytes + bytes_read_now; + memory.resize(new_total_bytes); + memcpy(memory.data() + old_total_bytes, in.position(), bytes_read_now); + + in.position() = chunk_end; + + if (found_separator && new_total_bytes >= min_chunk_bytes) + { + // Found the separator and the chunk big enough so that we can + // return it. + used_size = memory.size(); //FIXME + return true; + } + // Didn't find the separator, or the chunk is not big enough. Read more + // from the file. } - eofWithSavingBufferState(in, memory, used_size, begin_pos, true); - return true; } void registerFileSegmentationEngineTabSeparated(FormatFactory & factory) From 55f60a6badbec21ca3a07b92151a711db2975f84 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 15 Nov 2019 21:41:18 +0300 Subject: [PATCH 40/90] hold shared pointers to table function Storage and Context from InterpreterSelectQuery --- .../ClickHouseDictionarySource.cpp | 6 +- .../Interpreters/InterpreterExplainQuery.h | 2 +- .../Interpreters/InterpreterInsertQuery.cpp | 4 +- .../Interpreters/InterpreterSelectQuery.cpp | 208 +++++++++--------- .../src/Interpreters/InterpreterSelectQuery.h | 9 +- .../InterpreterSelectWithUnionQuery.cpp | 8 +- .../InterpreterSelectWithUnionQuery.h | 4 +- dbms/src/Processors/QueryPipeline.cpp | 4 + dbms/src/Processors/QueryPipeline.h | 9 + dbms/src/Storages/StorageView.cpp | 4 +- .../test_mysql_database_engine/test.py | 4 +- 11 files changed, 146 insertions(+), 116 deletions(-) diff --git a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp index 6586f979687..2d2afd4c3fe 100644 --- a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -125,7 +125,11 @@ BlockInputStreamPtr ClickHouseDictionarySource::loadAll() * the necessity of holding process_list_element shared pointer. */ if (is_local) - return executeQuery(load_all_query, context, true).in; + { + BlockIO res = executeQuery(load_all_query, context, true); + /// FIXME res.in may implicitly use some objects owned be res, but them will be destructed after return + return res.in; + } return std::make_shared(pool, load_all_query, sample_block, context); } diff --git a/dbms/src/Interpreters/InterpreterExplainQuery.h b/dbms/src/Interpreters/InterpreterExplainQuery.h index 0d3b183857b..058c51737b0 100644 --- a/dbms/src/Interpreters/InterpreterExplainQuery.h +++ b/dbms/src/Interpreters/InterpreterExplainQuery.h @@ -22,7 +22,7 @@ public: private: ASTPtr query; - Context context; + const Context & context; BlockInputStreamPtr executeImpl(); }; diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index b052bb82c1e..8c95362a6f7 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -98,7 +98,10 @@ BlockIO InterpreterInsertQuery::execute() { const auto & query = query_ptr->as(); checkAccess(query); + + BlockIO res; StoragePtr table = getTable(query); + res.pipeline.addStorageHolder(table); auto table_lock = table->lockStructureForShare(true, context.getInitialQueryId()); @@ -134,7 +137,6 @@ BlockIO InterpreterInsertQuery::execute() out_wrapper->setProcessListElement(context.getProcessListElement()); out = std::move(out_wrapper); - BlockIO res; res.out = std::move(out); /// What type of query: INSERT or INSERT SELECT? diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 09afaad90fa..aa64ddbe93f 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -230,7 +230,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( : options(options_) /// NOTE: the query almost always should be cloned because it will be modified during analysis. , query_ptr(options.modify_inplace ? query_ptr_ : query_ptr_->clone()) - , context(context_) + , context(std::make_shared(context_)) , storage(storage_) , input(input_) , log(&Logger::get("InterpreterSelectQuery")) @@ -238,7 +238,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( checkStackSize(); initSettings(); - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); if (settings.max_subquery_depth && options.subquery_depth > settings.max_subquery_depth) throw Exception("Too deep subqueries. Maximum: " + settings.max_subquery_depth.toString(), @@ -252,7 +252,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (settings.allow_experimental_multiple_joins_emulation) { - JoinToSubqueryTransformVisitor::Data join_to_subs_data{context}; + JoinToSubqueryTransformVisitor::Data join_to_subs_data{*context}; JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query_ptr); } @@ -278,7 +278,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( { /// Read from subquery. interpreter_subquery = std::make_unique( - table_expression, getSubqueryContext(context), options.subquery(), required_columns); + table_expression, getSubqueryContext(*context), options.subquery(), required_columns); source_header = interpreter_subquery->getSampleBlock(); } @@ -288,48 +288,48 @@ InterpreterSelectQuery::InterpreterSelectQuery( { /// Read from table function. propagate all settings from initSettings(), /// alternative is to call on current `context`, but that can potentially pollute it. - storage = getSubqueryContext(context).executeTableFunction(table_expression); + storage = getSubqueryContext(*context).executeTableFunction(table_expression); } else { String database_name; String table_name; - getDatabaseAndTableNames(query, database_name, table_name, context); + getDatabaseAndTableNames(query, database_name, table_name, *context); - if (auto view_source = context.getViewSource()) + if (auto view_source = context->getViewSource()) { auto & storage_values = static_cast(*view_source); if (storage_values.getDatabaseName() == database_name && storage_values.getTableName() == table_name) { /// Read from view source. - storage = context.getViewSource(); + storage = context->getViewSource(); } } if (!storage) { /// Read from table. Even without table expression (implicit SELECT ... FROM system.one). - storage = context.getTable(database_name, table_name); + storage = context->getTable(database_name, table_name); } } } if (storage) - table_lock = storage->lockStructureForShare(false, context.getInitialQueryId()); + table_lock = storage->lockStructureForShare(false, context->getInitialQueryId()); auto analyze = [&] () { - syntax_analyzer_result = SyntaxAnalyzer(context, options).analyze( + syntax_analyzer_result = SyntaxAnalyzer(*context, options).analyze( query_ptr, source_header.getNamesAndTypesList(), required_result_column_names, storage, NamesAndTypesList()); /// Save scalar sub queries's results in the query context - if (context.hasQueryContext()) + if (context->hasQueryContext()) for (const auto & it : syntax_analyzer_result->getScalars()) - context.getQueryContext().addScalar(it.first, it.second); + context->getQueryContext().addScalar(it.first, it.second); query_analyzer = std::make_unique( - query_ptr, syntax_analyzer_result, context, + query_ptr, syntax_analyzer_result, *context, NameSet(required_result_column_names.begin(), required_result_column_names.end()), options.subquery_depth, !options.only_analyze); @@ -346,8 +346,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Save the new temporary tables in the query context for (const auto & it : query_analyzer->getExternalTables()) - if (!context.tryGetExternalTable(it.first)) - context.addExternalTable(it.first, it.second); + if (!context->tryGetExternalTable(it.first)) + context->addExternalTable(it.first, it.second); } if (!options.only_analyze || options.modify_inplace) @@ -358,7 +358,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (is_subquery) interpreter_subquery = std::make_unique( table_expression, - getSubqueryContext(context), + getSubqueryContext(*context), options.subquery(), required_columns); } @@ -378,10 +378,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( source_header = storage->getSampleBlockForColumns(required_columns); /// Fix source_header for filter actions. - if (context.hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) + if (context->hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) { filter_info = std::make_shared(); - filter_info->column_name = generateFilterActions(filter_info->actions, storage, context, required_columns); + filter_info->column_name = generateFilterActions(filter_info->actions, storage, *context, required_columns); source_header = storage->getSampleBlockForColumns(filter_info->actions->getRequiredColumns()); } } @@ -450,27 +450,34 @@ Block InterpreterSelectQuery::getSampleBlock() BlockIO InterpreterSelectQuery::execute() { Pipeline pipeline; - executeImpl(pipeline, input); + BlockIO res; + executeImpl(pipeline, input, res.pipeline); executeUnion(pipeline, getSampleBlock()); - BlockIO res; res.in = pipeline.firstStream(); + res.pipeline.addInterpreterContext(context); + res.pipeline.addStorageHolder(storage); return res; } -BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams() +BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams(QueryPipeline & parent_pipeline) { + ///FIXME pipeline must be alive until query is finished Pipeline pipeline; - executeImpl(pipeline, input); + executeImpl(pipeline, input, parent_pipeline); unifyStreams(pipeline, getSampleBlock()); + parent_pipeline.addInterpreterContext(context); + parent_pipeline.addStorageHolder(storage); return pipeline.streams; } QueryPipeline InterpreterSelectQuery::executeWithProcessors() { QueryPipeline query_pipeline; - query_pipeline.setMaxThreads(context.getSettingsRef().max_threads); - executeImpl(query_pipeline, input); + query_pipeline.setMaxThreads(context->getSettingsRef().max_threads); + executeImpl(query_pipeline, input, query_pipeline); + query_pipeline.addInterpreterContext(context); + query_pipeline.addStorageHolder(storage); return query_pipeline; } @@ -478,13 +485,13 @@ QueryPipeline InterpreterSelectQuery::executeWithProcessors() Block InterpreterSelectQuery::getSampleBlockImpl() { auto & query = getSelectQuery(); - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); /// Do all AST changes here, because actions from analysis_result will be used later in readImpl. /// PREWHERE optimization. /// Turn off, if the table filter (row-level security) is applied. - if (storage && !context.hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) + if (storage && !context->hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) { query_analyzer->makeSetsForIndex(query.where()); query_analyzer->makeSetsForIndex(query.prewhere()); @@ -498,7 +505,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() /// Try transferring some condition from WHERE to PREWHERE if enabled and viable if (settings.optimize_move_to_prewhere && query.where() && !query.prewhere() && !query.final()) - MergeTreeWhereOptimizer{current_info, context, merge_tree, + MergeTreeWhereOptimizer{current_info, *context, merge_tree, syntax_analyzer_result->requiredSourceColumns(), log}; }; @@ -507,14 +514,14 @@ Block InterpreterSelectQuery::getSampleBlockImpl() } if (storage && !options.only_analyze) - from_stage = storage->getQueryProcessingStage(context); + from_stage = storage->getQueryProcessingStage(*context); analysis_result = analyzeExpressions( getSelectQuery(), *query_analyzer, from_stage, options.to_stage, - context, + *context, storage, options.only_analyze, filter_info, @@ -1007,7 +1014,7 @@ static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, cons template -void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input) +void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input, QueryPipeline & save_context_and_storage) { /** Streams of data. When the query is executed in parallel, we have several data streams. * If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then @@ -1023,14 +1030,14 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS /// Now we will compose block streams that perform the necessary actions. auto & query = getSelectQuery(); - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); auto & expressions = analysis_result; SortingInfoPtr sorting_info; if (settings.optimize_read_in_order && storage && query.orderBy() && !query_analyzer->hasAggregation() && !query.final() && !query.join()) { if (const auto * merge_tree_data = dynamic_cast(storage.get())) - sorting_info = optimizeReadInOrder(*merge_tree_data, query, context, syntax_analyzer_result); + sorting_info = optimizeReadInOrder(*merge_tree_data, query, *context, syntax_analyzer_result); } if (options.only_analyze) @@ -1090,7 +1097,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS throw Exception("PREWHERE is not supported if the table is filtered by row-level security expression", ErrorCodes::ILLEGAL_PREWHERE); /** Read the data from Storage. from_stage - to what stage the request was completed in Storage. */ - executeFetchColumns(from_stage, pipeline, sorting_info, expressions.prewhere_info, expressions.columns_to_remove_after_prewhere); + executeFetchColumns(from_stage, pipeline, sorting_info, expressions.prewhere_info, expressions.columns_to_remove_after_prewhere, save_context_and_storage); LOG_TRACE(log, QueryProcessingStage::toString(from_stage) << " -> " << QueryProcessingStage::toString(options.to_stage)); } @@ -1349,12 +1356,13 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS template void InterpreterSelectQuery::executeFetchColumns( QueryProcessingStage::Enum processing_stage, TPipeline & pipeline, - const SortingInfoPtr & sorting_info, const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere) + const SortingInfoPtr & sorting_info, const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere, + QueryPipeline & save_context_and_storage) { constexpr bool pipeline_with_processors = std::is_same::value; auto & query = getSelectQuery(); - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); /// Optimization for trivial query like SELECT count() FROM table. auto check_trivial_count_query = [&]() -> std::optional @@ -1418,11 +1426,11 @@ void InterpreterSelectQuery::executeFetchColumns( if (storage) { /// Append columns from the table filter to required - if (context.hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) + if (context->hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) { auto initial_required_columns = required_columns; ExpressionActionsPtr actions; - generateFilterActions(actions, storage, context, initial_required_columns); + generateFilterActions(actions, storage, *context, initial_required_columns); auto required_columns_from_filter = actions->getRequiredColumns(); for (const auto & column : required_columns_from_filter) @@ -1521,8 +1529,8 @@ void InterpreterSelectQuery::executeFetchColumns( = ext::map(required_columns_after_prewhere, [](const auto & it) { return it.name; }); } - auto syntax_result = SyntaxAnalyzer(context).analyze(required_columns_all_expr, required_columns_after_prewhere, {}, storage); - alias_actions = ExpressionAnalyzer(required_columns_all_expr, syntax_result, context).getActions(true); + auto syntax_result = SyntaxAnalyzer(*context).analyze(required_columns_all_expr, required_columns_after_prewhere, {}, storage); + alias_actions = ExpressionAnalyzer(required_columns_all_expr, syntax_result, *context).getActions(true); /// The set of required columns could be added as a result of adding an action to calculate ALIAS. required_columns = alias_actions->getRequiredColumns(); @@ -1542,7 +1550,7 @@ void InterpreterSelectQuery::executeFetchColumns( if (prewhere_info) { /// Don't remove columns which are needed to be aliased. - auto new_actions = std::make_shared(prewhere_info->prewhere_actions->getRequiredColumnsWithTypes(), context); + auto new_actions = std::make_shared(prewhere_info->prewhere_actions->getRequiredColumnsWithTypes(), *context); for (const auto & action : prewhere_info->prewhere_actions->getActions()) { if (action.type != ExpressionAction::REMOVE_COLUMN @@ -1552,9 +1560,9 @@ void InterpreterSelectQuery::executeFetchColumns( prewhere_info->prewhere_actions = std::move(new_actions); auto analyzed_result - = SyntaxAnalyzer(context).analyze(required_columns_from_prewhere_expr, storage->getColumns().getAllPhysical()); + = SyntaxAnalyzer(*context).analyze(required_columns_from_prewhere_expr, storage->getColumns().getAllPhysical()); prewhere_info->alias_actions - = ExpressionAnalyzer(required_columns_from_prewhere_expr, analyzed_result, context).getActions(true, false); + = ExpressionAnalyzer(required_columns_from_prewhere_expr, analyzed_result, *context).getActions(true, false); /// Add (physical?) columns required by alias actions. auto required_columns_from_alias = prewhere_info->alias_actions->getRequiredColumns(); @@ -1597,7 +1605,7 @@ void InterpreterSelectQuery::executeFetchColumns( UInt64 max_block_size = settings.max_block_size; - auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); + auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context); /** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY, WITH TIES but LIMIT is specified, and limit + offset < max_block_size, * then as the block size we will use limit + offset (not to read more from the table than requested), @@ -1638,7 +1646,7 @@ void InterpreterSelectQuery::executeFetchColumns( throw Exception("Subquery expected", ErrorCodes::LOGICAL_ERROR); interpreter_subquery = std::make_unique( - subquery, getSubqueryContext(context), + subquery, getSubqueryContext(*context), options.copy().subquery().noModify(), required_columns); if (query_analyzer->hasAggregation()) @@ -1649,7 +1657,7 @@ void InterpreterSelectQuery::executeFetchColumns( /// Just use pipeline from subquery. pipeline = interpreter_subquery->executeWithProcessors(); else - pipeline.streams = interpreter_subquery->executeWithMultipleStreams(); + pipeline.streams = interpreter_subquery->executeWithMultipleStreams(save_context_and_storage); } else if (storage) { @@ -1676,9 +1684,9 @@ void InterpreterSelectQuery::executeFetchColumns( bool use_pipes = pipeline_with_processors && storage->supportProcessorsPipeline(); if (use_pipes) - pipes = storage->readWithProcessors(required_columns, query_info, context, processing_stage, max_block_size, max_streams); + pipes = storage->readWithProcessors(required_columns, query_info, *context, processing_stage, max_block_size, max_streams); else - streams = storage->read(required_columns, query_info, context, processing_stage, max_block_size, max_streams); + streams = storage->read(required_columns, query_info, *context, processing_stage, max_block_size, max_streams); if (streams.empty() && !use_pipes) { @@ -1757,7 +1765,7 @@ void InterpreterSelectQuery::executeFetchColumns( limits.speed_limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed; } - QuotaForIntervals & quota = context.getQuota(); + QuotaForIntervals & quota = context->getQuota(); for (auto & stream : streams) { @@ -1798,7 +1806,7 @@ void InterpreterSelectQuery::executeFetchColumns( auto header = stream->getHeader(); auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; if (!blocksHaveEqualStructure(first_header, header)) - stream = std::make_shared(context, stream, first_header, mode); + stream = std::make_shared(*context, stream, first_header, mode); } } @@ -1886,7 +1894,7 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre for (const auto & name : descr.argument_names) descr.arguments.push_back(header.getPositionByName(name)); - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); /** Two-level aggregation is useful in two cases: * 1. Parallel aggregation is done, and the results should be merged in parallel. @@ -1899,7 +1907,7 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); + context->getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); /// If there are several sources, then we perform parallel aggregation if (pipeline.streams.size() > 1) @@ -1952,7 +1960,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const for (const auto & name : descr.argument_names) descr.arguments.push_back(header_before_aggregation.getPositionByName(name)); - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); /** Two-level aggregation is useful in two cases: * 1. Parallel aggregation is done, and the results should be merged in parallel. @@ -1965,7 +1973,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); + context->getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); auto transform_params = std::make_shared(params, final); @@ -2029,7 +2037,7 @@ void InterpreterSelectQuery::executeMergeAggregated(Pipeline & pipeline, bool ov * but it can work more slowly. */ - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); Aggregator::Params params(header, keys, aggregates, overflow_row, settings.max_threads); @@ -2080,7 +2088,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPipeline & pipeline, bo * but it can work more slowly. */ - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); Aggregator::Params params(header_before_merge, keys, aggregates, overflow_row, settings.max_threads); @@ -2140,7 +2148,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha { executeUnion(pipeline, {}); - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); pipeline.firstStream() = std::make_shared( pipeline.firstStream(), @@ -2154,7 +2162,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha void InterpreterSelectQuery::executeTotalsAndHaving(QueryPipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final) { - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); auto totals_having = std::make_shared( pipeline.getHeader(), overflow_row, expression, @@ -2180,13 +2188,13 @@ void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificato for (const auto & name : key_names) keys.push_back(header.getPositionByName(name)); - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); Aggregator::Params params(header, keys, aggregates, false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, SettingUInt64(0), SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); + context->getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); if (modificator == Modificator::ROLLUP) pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); @@ -2209,13 +2217,13 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPipeline & pipeline, Modif for (const auto & name : key_names) keys.push_back(header_before_transform.getPositionByName(name)); - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); Aggregator::Params params(header_before_transform, keys, aggregates, false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, SettingUInt64(0), SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); + context->getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); auto transform_params = std::make_shared(params, true); @@ -2251,9 +2259,9 @@ void InterpreterSelectQuery::executeExpression(QueryPipeline & pipeline, const E void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr sorting_info) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, context); - const Settings & settings = context.getSettingsRef(); - UInt64 limit = getLimitForSorting(query, context); + SortDescription order_descr = getSortDescription(query, *context); + const Settings & settings = context->getSettingsRef(); + UInt64 limit = getLimitForSorting(query, *context); if (sorting_info) { @@ -2305,17 +2313,17 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr so pipeline.firstStream() = std::make_shared( pipeline.firstStream(), order_descr, settings.max_block_size, limit, settings.max_bytes_before_remerge_sort, - settings.max_bytes_before_external_sort, context.getTemporaryPath(), settings.min_free_disk_space_for_temporary_data); + settings.max_bytes_before_external_sort, context->getTemporaryPath(), settings.min_free_disk_space_for_temporary_data); } } void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoPtr sorting_info) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, context); - UInt64 limit = getLimitForSorting(query, context); + SortDescription order_descr = getSortDescription(query, *context); + UInt64 limit = getLimitForSorting(query, *context); - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); /// TODO: Limits on sorting // IBlockInputStream::LocalLimits limits; @@ -2385,7 +2393,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoP return std::make_shared( header, order_descr, settings.max_block_size, limit, settings.max_bytes_before_remerge_sort, - settings.max_bytes_before_external_sort, context.getTemporaryPath(), settings.min_free_disk_space_for_temporary_data); + settings.max_bytes_before_external_sort, context->getTemporaryPath(), settings.min_free_disk_space_for_temporary_data); }); } @@ -2393,8 +2401,8 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoP void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, context); - UInt64 limit = getLimitForSorting(query, context); + SortDescription order_descr = getSortDescription(query, *context); + UInt64 limit = getLimitForSorting(query, *context); /// If there are several streams, then we merge them into one if (pipeline.hasMoreThanOneStream()) @@ -2409,7 +2417,7 @@ void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline, const SortD { if (pipeline.hasMoreThanOneStream()) { - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); /** MergingSortedBlockInputStream reads the sources sequentially. * To make the data on the remote servers prepared in parallel, we wrap it in AsynchronousBlockInputStream. @@ -2428,8 +2436,8 @@ void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline, const SortD void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, context); - UInt64 limit = getLimitForSorting(query, context); + SortDescription order_descr = getSortDescription(query, *context); + UInt64 limit = getLimitForSorting(query, *context); executeMergeSorted(pipeline, order_descr, limit); } @@ -2439,7 +2447,7 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline, const /// If there are several streams, then we merge them into one if (pipeline.getNumStreams() > 1) { - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); auto transform = std::make_shared( pipeline.getHeader(), @@ -2474,9 +2482,9 @@ void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_or auto & query = getSelectQuery(); if (query.distinct) { - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); - auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); + auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context); UInt64 limit_for_distinct = 0; /// If after this stage of DISTINCT ORDER BY is not executed, then you can get no more than limit_length + limit_offset of different rows. @@ -2496,9 +2504,9 @@ void InterpreterSelectQuery::executeDistinct(QueryPipeline & pipeline, bool befo auto & query = getSelectQuery(); if (query.distinct) { - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); - auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); + auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context); UInt64 limit_for_distinct = 0; /// If after this stage of DISTINCT ORDER BY is not executed, then you can get no more than limit_length + limit_offset of different rows. @@ -2548,13 +2556,13 @@ void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline) /// If there is LIMIT if (query.limitLength()) { - auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); + auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context); SortDescription sort_descr; if (query.limit_with_ties) { if (!query.orderBy()) throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); - sort_descr = getSortDescription(query, context); + sort_descr = getSortDescription(query, *context); } pipeline.transform([&, limit = limit_length + limit_offset](auto & stream) { @@ -2570,7 +2578,7 @@ void InterpreterSelectQuery::executePreLimit(QueryPipeline & pipeline) /// If there is LIMIT if (query.limitLength()) { - auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); + auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context); pipeline.addSimpleTransform([&, limit = limit_length + limit_offset](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr { if (stream_type == QueryPipeline::StreamType::Totals) @@ -2591,8 +2599,8 @@ void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline) Names columns; for (const auto & elem : query.limitBy()->children) columns.emplace_back(elem->getColumnName()); - UInt64 length = getLimitUIntValue(query.limitByLength(), context); - UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), context) : 0); + UInt64 length = getLimitUIntValue(query.limitByLength(), *context); + UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), *context) : 0); pipeline.transform([&](auto & stream) { @@ -2610,8 +2618,8 @@ void InterpreterSelectQuery::executeLimitBy(QueryPipeline & pipeline) for (const auto & elem : query.limitBy()->children) columns.emplace_back(elem->getColumnName()); - UInt64 length = getLimitUIntValue(query.limitByLength(), context); - UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), context) : 0); + UInt64 length = getLimitUIntValue(query.limitByLength(), *context); + UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), *context) : 0); pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr { @@ -2675,12 +2683,12 @@ void InterpreterSelectQuery::executeLimit(Pipeline & pipeline) { if (!query.orderBy()) throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); - order_descr = getSortDescription(query, context); + order_descr = getSortDescription(query, *context); } UInt64 limit_length; UInt64 limit_offset; - std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, context); + std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, *context); pipeline.transform([&](auto & stream) { @@ -2695,7 +2703,7 @@ void InterpreterSelectQuery::executeWithFill(Pipeline & pipeline) auto & query = getSelectQuery(); if (query.orderBy()) { - SortDescription order_descr = getSortDescription(query, context); + SortDescription order_descr = getSortDescription(query, *context); SortDescription fill_descr; for (auto & desc : order_descr) { @@ -2718,7 +2726,7 @@ void InterpreterSelectQuery::executeWithFill(QueryPipeline & pipeline) auto & query = getSelectQuery(); if (query.orderBy()) { - SortDescription order_descr = getSortDescription(query, context); + SortDescription order_descr = getSortDescription(query, *context); SortDescription fill_descr; for (auto & desc : order_descr) { @@ -2762,14 +2770,14 @@ void InterpreterSelectQuery::executeLimit(QueryPipeline & pipeline) UInt64 limit_length; UInt64 limit_offset; - std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, context); + std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, *context); SortDescription order_descr; if (query.limit_with_ties) { if (!query.orderBy()) throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); - order_descr = getSortDescription(query, context); + order_descr = getSortDescription(query, *context); } pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr @@ -2786,7 +2794,7 @@ void InterpreterSelectQuery::executeLimit(QueryPipeline & pipeline) void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline) { - if (!context.getSettingsRef().extremes) + if (!context->getSettingsRef().extremes) return; pipeline.transform([&](auto & stream) @@ -2797,7 +2805,7 @@ void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline) void InterpreterSelectQuery::executeExtremes(QueryPipeline & pipeline) { - if (!context.getSettingsRef().extremes) + if (!context->getSettingsRef().extremes) return; auto transform = std::make_shared(pipeline.getHeader()); @@ -2818,7 +2826,7 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline executeUnion(pipeline, {}); pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), subqueries_for_sets, context); + pipeline.firstStream(), subqueries_for_sets, *context); } void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, SubqueriesForSets & subqueries_for_sets) @@ -2830,12 +2838,12 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pip executeMergeSorted(pipeline, query_info.sorting_info->prefix_order_descr, 0); } - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); auto creating_sets = std::make_shared( pipeline.getHeader(), subqueries_for_sets, SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), - context); + *context); pipeline.addCreatingSetsTransform(std::move(creating_sets)); } @@ -2856,7 +2864,7 @@ void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline, Block header) auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; if (!blocksHaveEqualStructure(header, stream_header)) - stream = std::make_shared(context, stream, header, mode); + stream = std::make_shared(*context, stream, header, mode); } } @@ -2871,7 +2879,7 @@ void InterpreterSelectQuery::initSettings() { auto & query = getSelectQuery(); if (query.settings()) - InterpreterSetQuery(query.settings(), context).executeForCurrentContext(); + InterpreterSetQuery(query.settings(), *context).executeForCurrentContext(); } } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 6d922c6b079..239f4364d94 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -69,7 +69,7 @@ public: BlockIO execute() override; /// Execute the query and return multuple streams for parallel processing. - BlockInputStreams executeWithMultipleStreams(); + BlockInputStreams executeWithMultipleStreams(QueryPipeline & parent_pipeline); QueryPipeline executeWithProcessors() override; bool canExecuteWithProcessors() const override { return true; } @@ -137,7 +137,7 @@ private: }; template - void executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input); + void executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input, QueryPipeline & save_context_and_storage); struct AnalysisResult { @@ -199,7 +199,8 @@ private: template void executeFetchColumns(QueryProcessingStage::Enum processing_stage, TPipeline & pipeline, const SortingInfoPtr & sorting_info, const PrewhereInfoPtr & prewhere_info, - const Names & columns_to_remove_after_prewhere); + const Names & columns_to_remove_after_prewhere, + QueryPipeline & save_context_and_storage); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter); void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final); @@ -261,7 +262,7 @@ private: const SelectQueryOptions options; ASTPtr query_ptr; - Context context; + std::shared_ptr context; SyntaxAnalyzerResultPtr syntax_analyzer_result; std::unique_ptr query_analyzer; SelectQueryInfo query_info; diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index e9671eb7358..1a2762a6daf 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -165,13 +165,13 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock( } -BlockInputStreams InterpreterSelectWithUnionQuery::executeWithMultipleStreams() +BlockInputStreams InterpreterSelectWithUnionQuery::executeWithMultipleStreams(QueryPipeline & parent_pipeline) { BlockInputStreams nested_streams; for (auto & interpreter : nested_interpreters) { - BlockInputStreams streams = interpreter->executeWithMultipleStreams(); + BlockInputStreams streams = interpreter->executeWithMultipleStreams(parent_pipeline); nested_streams.insert(nested_streams.end(), streams.begin(), streams.end()); } @@ -188,7 +188,8 @@ BlockIO InterpreterSelectWithUnionQuery::execute() { const Settings & settings = context.getSettingsRef(); - BlockInputStreams nested_streams = executeWithMultipleStreams(); + BlockIO res; + BlockInputStreams nested_streams = executeWithMultipleStreams(res.pipeline); BlockInputStreamPtr result_stream; if (nested_streams.empty()) @@ -206,7 +207,6 @@ BlockIO InterpreterSelectWithUnionQuery::execute() nested_streams.clear(); } - BlockIO res; res.in = result_stream; return res; } diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h index 9f2a4a96494..0e07313f5b9 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -29,7 +29,7 @@ public: BlockIO execute() override; /// Execute the query without union of streams. - BlockInputStreams executeWithMultipleStreams(); + BlockInputStreams executeWithMultipleStreams(QueryPipeline & parent_pipeline); QueryPipeline executeWithProcessors() override; bool canExecuteWithProcessors() const override { return true; } @@ -47,7 +47,7 @@ public: private: const SelectQueryOptions options; ASTPtr query_ptr; - Context context; + const Context & context; std::vector> nested_interpreters; diff --git a/dbms/src/Processors/QueryPipeline.cpp b/dbms/src/Processors/QueryPipeline.cpp index 55829338b07..525cc898cc1 100644 --- a/dbms/src/Processors/QueryPipeline.cpp +++ b/dbms/src/Processors/QueryPipeline.cpp @@ -486,6 +486,10 @@ void QueryPipeline::unitePipelines( processors.insert(processors.end(), pipeline.processors.begin(), pipeline.processors.end()); streams.insert(streams.end(), pipeline.streams.begin(), pipeline.streams.end()); + + interpreter_context.insert(interpreter_context.end(), pipeline.interpreter_context.begin(), pipeline.interpreter_context.end()); + storage_holder.insert(storage_holder.end(), pipeline.storage_holder.begin(), pipeline.storage_holder.end()); + /// Do we need also copy table_locks? } if (!extremes.empty()) diff --git a/dbms/src/Processors/QueryPipeline.h b/dbms/src/Processors/QueryPipeline.h index dbeb5166ea2..e32ed6a0abe 100644 --- a/dbms/src/Processors/QueryPipeline.h +++ b/dbms/src/Processors/QueryPipeline.h @@ -6,6 +6,7 @@ #include #include +#include namespace DB { @@ -75,6 +76,8 @@ public: const Block & getHeader() const { return current_header; } void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); } + void addInterpreterContext(std::shared_ptr context) { interpreter_context.emplace_back(std::move(context)); } + void addStorageHolder(StoragePtr storage) { storage_holder.emplace_back(std::move(storage)); } /// For compatibility with IBlockInputStream. void setProgressCallback(const ProgressCallback & callback); @@ -109,6 +112,12 @@ private: TableStructureReadLocks table_locks; + /// Some Streams (or Processors) may implicitly use Context or temporary Storage created by Interpreter. + /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, + /// because QueryPipeline is alive until query is finished. + std::vector> interpreter_context; + std::vector storage_holder; + IOutputFormat * output_format = nullptr; size_t max_threads = 0; diff --git a/dbms/src/Storages/StorageView.cpp b/dbms/src/Storages/StorageView.cpp index 912e358e3ae..824856dfc4e 100644 --- a/dbms/src/Storages/StorageView.cpp +++ b/dbms/src/Storages/StorageView.cpp @@ -66,7 +66,9 @@ BlockInputStreams StorageView::read( current_inner_query = new_inner_query; } - res = InterpreterSelectWithUnionQuery(current_inner_query, context, {}, column_names).executeWithMultipleStreams(); + QueryPipeline pipeline; + /// FIXME res may implicitly use some objects owned be pipeline, but them will be destructed after return + res = InterpreterSelectWithUnionQuery(current_inner_query, context, {}, column_names).executeWithMultipleStreams(pipeline); /// It's expected that the columns read from storage are not constant. /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. diff --git a/dbms/tests/integration/test_mysql_database_engine/test.py b/dbms/tests/integration/test_mysql_database_engine/test.py index de996c791dc..9c3fe4da250 100644 --- a/dbms/tests/integration/test_mysql_database_engine/test.py +++ b/dbms/tests/integration/test_mysql_database_engine/test.py @@ -112,10 +112,10 @@ def test_table_function(started_cluster): mysql_connection = get_mysql_conn() create_normal_mysql_table(mysql_connection, 'table_function') table_function = get_mysql_table_function_expr('table_function') + assert node1.query("SELECT count() FROM {}".format(table_function)).rstrip() == '0' node1.query("INSERT INTO {} (id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000)".format('TABLE FUNCTION ' + table_function)) - node1.query("SELECT * FROM {}".format(table_function)) # should fail with asan assert node1.query("SELECT count() FROM {}".format(table_function)).rstrip() == '10000' - assert node1.query("SELECT sum(`float`) FROM {}".format(table_function)).rstrip() == '30000' + assert node1.query("SELECT sum(`money`) FROM {}".format(table_function)).rstrip() == '30000' mysql_connection.close() From 3f5ac643cd15a145e41020474ba5f0fb3cf89560 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 15 Nov 2019 22:10:42 +0300 Subject: [PATCH 41/90] more complex test --- dbms/tests/integration/test_mysql_database_engine/test.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/tests/integration/test_mysql_database_engine/test.py b/dbms/tests/integration/test_mysql_database_engine/test.py index 9c3fe4da250..5e0cd7f62e2 100644 --- a/dbms/tests/integration/test_mysql_database_engine/test.py +++ b/dbms/tests/integration/test_mysql_database_engine/test.py @@ -115,6 +115,10 @@ def test_table_function(started_cluster): assert node1.query("SELECT count() FROM {}".format(table_function)).rstrip() == '0' node1.query("INSERT INTO {} (id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000)".format('TABLE FUNCTION ' + table_function)) assert node1.query("SELECT count() FROM {}".format(table_function)).rstrip() == '10000' + assert node1.query("SELECT sum(c) FROM (" + "SELECT count() as c FROM {} WHERE id % 3 == 0" + " UNION ALL SELECT count() as c FROM {} WHERE id % 3 == 1" + " UNION ALL SELECT count() as c FROM {} WHERE id % 3 == 2)".format(table_function, table_function, table_function)).rstrip() == '10000' assert node1.query("SELECT sum(`money`) FROM {}".format(table_function)).rstrip() == '30000' mysql_connection.close() From fffda9c53076d134af3db8ef3ddebdd204e7f047 Mon Sep 17 00:00:00 2001 From: Viktor Taranenko Date: Sun, 17 Nov 2019 18:37:53 +0000 Subject: [PATCH 42/90] document arrayFlatten (flatten) in en, ru --- .../query_language/functions/array_functions.md | 16 ++++++++++++++++ .../query_language/functions/array_functions.md | 16 ++++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index 1772ec88b94..2454df4042e 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -772,6 +772,22 @@ SELECT arrayReduce('uniqUpTo(3)', [1, 2, 3, 4, 5, 6, 7, 8, 9, 10]) └─────────────────────────────────────────────────────────────┘ ``` +## arrayFlatten(arr) {#array_functions-arrayflatten} + +The `arrayFlatten` (or `flatten` alias) method will collapse the elements of an array to create a single array. + +Example: + +```sql +SELECT arrayFlatten([[1, 2, 3], [4, 5]]) +``` + +```text +┌─arrayFlatten([[1, 2, 3], [4, 5]])─┐ +│ [1,2,3,4,5] │ +└───────────────────────────────────┘ +``` + ## arrayReverse(arr) {#array_functions-arrayreverse} Returns an array of the same size as the original array containing the elements in reverse order. diff --git a/docs/ru/query_language/functions/array_functions.md b/docs/ru/query_language/functions/array_functions.md index 93c75ac3525..377750c99c1 100644 --- a/docs/ru/query_language/functions/array_functions.md +++ b/docs/ru/query_language/functions/array_functions.md @@ -778,6 +778,22 @@ SELECT arrayReduce('uniqUpTo(3)', [1, 2, 3, 4, 5, 6, 7, 8, 9, 10]) └─────────────────────────────────────────────────────────────┘ ``` +## arrayFlatten(arr) {#array_functions-arrayflatten} + +Функция `arrayFlatten` (или `flatten`) соеденит вложенные массивы и вернет массив из их элементов. + +Пример: + +```sql +SELECT arrayFlatten([[1, 2, 3], [4, 5]]) +``` + +```text +┌─arrayFlatten([[1, 2, 3], [4, 5]])─┐ +│ [1,2,3,4,5] │ +└───────────────────────────────────┘ +``` + ## arrayReverse(arr) {#array_functions-arrayreverse} Возвращает массив того же размера, что и исходный массив, содержащий элементы в обратном порядке. From 83030b98a2ad0823deab915a01acebb16519cbba Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 18 Nov 2019 16:10:14 +0300 Subject: [PATCH 43/90] remove MemoryExt<> --- .../DataStreams/ParallelParsingBlockInputStream.cpp | 9 ++++----- dbms/src/DataStreams/ParallelParsingBlockInputStream.h | 10 ++-------- dbms/src/Formats/FormatFactory.h | 1 - .../Formats/Impl/TabSeparatedRowInputFormat.cpp | 4 +--- 4 files changed, 7 insertions(+), 17 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index 5e3bebf1de9..403f0d01114 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -28,15 +28,14 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() assert(unit.status == READY_TO_INSERT); // Segmentating the original input. - unit.segment.used_size = 0; - unit.segment.memory.resize(0); + unit.segment.resize(0); const bool have_more_data = file_segmentation_engine(original_buffer, - unit.segment.memory, unit.segment.used_size, min_chunk_size); + unit.segment, min_chunk_size); // Creating buffer from the segment of data. - auto new_buffer = BufferBase::Buffer(unit.segment.memory.data(), - unit.segment.memory.data() + unit.segment.used_size); + auto new_buffer = BufferBase::Buffer(unit.segment.data(), + unit.segment.data() + unit.segment.size()); unit.readbuffer->buffer().swap(new_buffer); unit.readbuffer->position() = unit.readbuffer->buffer().begin(); diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index f08e7103011..866705dfc44 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -178,12 +178,6 @@ private: READY_TO_READ }; - struct MemoryExt - { - Memory<> memory; - size_t used_size{0}; - }; - struct BlockExt { std::vector block; @@ -194,7 +188,7 @@ private: { explicit ProcessingUnit(const Builder & builder) : status(ProcessingUnitStatus::READY_TO_INSERT) { - readbuffer = std::make_unique(segment.memory.data(), segment.used_size, 0); + readbuffer = std::make_unique(segment.data(), segment.size(), 0); parser = std::make_unique( builder.input_processor_creator(*readbuffer, builder.input_creator_params.sample, @@ -205,7 +199,7 @@ private: BlockExt block_ext; std::unique_ptr readbuffer; - MemoryExt segment; + Memory<> segment; std::unique_ptr parser; std::atomic status; bool is_last{false}; diff --git a/dbms/src/Formats/FormatFactory.h b/dbms/src/Formats/FormatFactory.h index b3359a36003..71acd10a0c8 100644 --- a/dbms/src/Formats/FormatFactory.h +++ b/dbms/src/Formats/FormatFactory.h @@ -49,7 +49,6 @@ public: using FileSegmentationEngine = std::function & memory, - size_t & used_size, size_t min_chunk_size)>; /// This callback allows to perform some additional actions after writing a single row. diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index db2aa052d72..fbe8ba58c9f 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -384,13 +384,12 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) } } -bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t & used_size, size_t min_chunk_bytes) +bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_bytes) { for (;;) { if (in.eof()) { - used_size = memory.size(); // remove me return false; } @@ -448,7 +447,6 @@ bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memo { // Found the separator and the chunk big enough so that we can // return it. - used_size = memory.size(); //FIXME return true; } // Didn't find the separator, or the chunk is not big enough. Read more From 4ab7ac14bdf2be6aa968f0f727f8666e993478d0 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 18 Nov 2019 16:12:28 +0300 Subject: [PATCH 44/90] Remove all segmentation engines except TSV --- dbms/src/Formats/FormatFactory.cpp | 7 -- dbms/src/IO/ReadHelpers.cpp | 22 ------ .../Formats/Impl/CSVRowInputFormat.cpp | 63 ----------------- .../Impl/JSONEachRowRowInputFormat.cpp | 67 ------------------- .../Formats/Impl/TSKVRowInputFormat.cpp | 36 ---------- 5 files changed, 195 deletions(-) diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index d54cff43551..8eaad2ad6bd 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -291,11 +291,7 @@ void registerOutputFormatProcessorTemplate(FormatFactory &factory); /// File Segmentation Engines for parallel reading -void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); -//void registerFileSegmentationEngineValues(FormatFactory & factory); -void registerFileSegmentationEngineCSV(FormatFactory & factory); -void registerFileSegmentationEngineTSKV(FormatFactory & factory); /// Output only (presentational) formats. @@ -347,10 +343,7 @@ FormatFactory::FormatFactory() registerInputFormatProcessorTemplate(*this); registerOutputFormatProcessorTemplate(*this); - registerFileSegmentationEngineJSONEachRow(*this); registerFileSegmentationEngineTabSeparated(*this); - registerFileSegmentationEngineCSV(*this); - registerFileSegmentationEngineTSKV(*this); registerOutputFormatNull(*this); diff --git a/dbms/src/IO/ReadHelpers.cpp b/dbms/src/IO/ReadHelpers.cpp index e85c40ee84d..7c0c2301c28 100644 --- a/dbms/src/IO/ReadHelpers.cpp +++ b/dbms/src/IO/ReadHelpers.cpp @@ -1053,26 +1053,4 @@ void skipToUnescapedNextLineOrEOF(ReadBuffer & buf) } } -bool eofWithSavingBufferState(ReadBuffer & buf, DB::Memory<> & memory, size_t & used_size, char * & begin_pos, bool force_saving_buffer_state) -{ - /// If there is some pending data - no need to copy data from buffer to memory. - if (force_saving_buffer_state || !buf.hasPendingData()) - { - const auto capacity = memory.size(); - const auto block_size = static_cast(buf.position() - begin_pos); - - /// To avoid calling a function when not needed. - if (capacity <= block_size + used_size) - { - memory.resize(used_size + block_size); - } - memcpy(memory.data() + used_size, begin_pos, buf.position() - begin_pos); - used_size += block_size; - bool res = buf.eof(); - begin_pos = buf.position(); - return res; - } - return false; -} - } diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index ef50e585d71..648956a14ee 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -422,68 +422,5 @@ void registerInputFormatProcessorCSV(FormatFactory & factory) } } -bool fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t & used_size, size_t min_chunk_size) -{ - if (in.eof()) - return false; - - skipWhitespacesAndTabs(in); - char * begin_pos = in.position(); - bool quotes = false; - bool need_more_data = true; - memory.resize(min_chunk_size); - while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && need_more_data) - { - if (quotes) - { - in.position() = find_first_symbols<'"'>(in.position(), in.buffer().end()); - if (in.position() == in.buffer().end()) - continue; - if (*in.position() == '"') - { - ++in.position(); - if (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && *in.position() == '"') - ++in.position(); - else - quotes = false; - } - } - else - { - in.position() = find_first_symbols<'"','\r', '\n'>(in.position(), in.buffer().end()); - if (in.position() == in.buffer().end()) - continue; - if (*in.position() == '"') - { - quotes = true; - ++in.position(); - } - else if (*in.position() == '\n') - { - if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) - need_more_data = false; - ++in.position(); - if (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && *in.position() == '\r') - ++in.position(); - } - else if (*in.position() == '\r') - { - if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) - need_more_data = false; - ++in.position(); - if (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && *in.position() == '\n') - ++in.position(); - } - } - } - eofWithSavingBufferState(in, memory, used_size, begin_pos, true); - return true; -} - -void registerFileSegmentationEngineCSV(FormatFactory & factory) -{ - factory.registerFileSegmentationEngine("CSV", &fileSegmentationEngineCSVImpl); -} - } diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 2529ee9f74f..a4a6911f8eb 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -270,71 +270,4 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory) }); } -bool fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t & used_size, size_t min_chunk_size) -{ - if (in.eof()) - return false; - - skipWhitespaceIfAny(in); - char * begin_pos = in.position(); - size_t balance = 0; - bool quotes = false; - memory.resize(min_chunk_size); - while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) - && (balance || used_size + static_cast(in.position() - begin_pos) < min_chunk_size)) - { - if (quotes) - { - in.position() = find_first_symbols<'\\', '"'>(in.position(), in.buffer().end()); - if (in.position() == in.buffer().end()) - continue; - if (*in.position() == '\\') - { - ++in.position(); - if (!eofWithSavingBufferState(in, memory, used_size, begin_pos)) - ++in.position(); - } - else if (*in.position() == '"') - { - ++in.position(); - quotes = false; - } - } - else - { - in.position() = find_first_symbols<'{', '}', '\\', '"'>(in.position(), in.buffer().end()); - if (in.position() == in.buffer().end()) - continue; - if (*in.position() == '{') - { - ++balance; - ++in.position(); - } - else if (*in.position() == '}') - { - --balance; - ++in.position(); - } - else if (*in.position() == '\\') - { - ++in.position(); - if (!eofWithSavingBufferState(in, memory, used_size, begin_pos)) - ++in.position(); - } - else if (*in.position() == '"') - { - quotes = true; - ++in.position(); - } - } - } - eofWithSavingBufferState(in, memory, used_size, begin_pos, true); - return true; -} - -void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory) -{ - factory.registerFileSegmentationEngine("JSONEachRow", &fileSegmentationEngineJSONEachRowImpl); -} - } diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index 7bb5f2eeadd..0b1b7e3e44b 100644 --- a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -210,40 +210,4 @@ void registerInputFormatProcessorTSKV(FormatFactory & factory) }); } -bool fileSegmentationEngineTSKVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t & used_size, size_t min_chunk_size) -{ - if (in.eof()) - return false; - - char * begin_pos = in.position(); - bool need_more_data = true; - memory.resize(min_chunk_size); - while (!eofWithSavingBufferState(in, memory, used_size, begin_pos) && need_more_data) - { - in.position() = find_first_symbols<'\\','\r', '\n'>(in.position(), in.buffer().end()); - if (in.position() == in.buffer().end()) - continue; - if (*in.position() == '\\') - { - ++in.position(); - if (!eofWithSavingBufferState(in, memory, used_size, begin_pos)) - ++in.position(); - } - else if (*in.position() == '\n' || *in.position() == '\r') - { - if (used_size + static_cast(in.position() - begin_pos) >= min_chunk_size) - need_more_data = false; - ++in.position(); - } - } - eofWithSavingBufferState(in, memory, used_size, begin_pos, true); - return true; -} - -void registerFileSegmentationEngineTSKV(FormatFactory & factory) -{ - factory.registerFileSegmentationEngine("TSKV", &fileSegmentationEngineTSKVImpl); -} - - } From bb98328ad59e0f4a35acaac3ca978a3300790f6b Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 18 Nov 2019 16:41:40 +0300 Subject: [PATCH 45/90] Make parser and read buffer local to parser thread --- .../ParallelParsingBlockInputStream.cpp | 32 ++++++++----------- .../ParallelParsingBlockInputStream.h | 27 ++++++++-------- 2 files changed, 27 insertions(+), 32 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index 403f0d01114..aeeb4ff399e 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -33,22 +33,7 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() const bool have_more_data = file_segmentation_engine(original_buffer, unit.segment, min_chunk_size); - // Creating buffer from the segment of data. - auto new_buffer = BufferBase::Buffer(unit.segment.data(), - unit.segment.data() + unit.segment.size()); - - unit.readbuffer->buffer().swap(new_buffer); - unit.readbuffer->position() = unit.readbuffer->buffer().begin(); - - unit.parser = std::make_unique( - input_processor_creator(*unit.readbuffer, header, context, row_input_format_params, format_settings) - ); - - if (!have_more_data) - { - unit.is_last = true; - } - + unit.is_last = !have_more_data; unit.status = READY_TO_PARSE; scheduleParserThreadForUnitWithNumber(current_unit_number); ++segmentator_ticket_number; @@ -73,16 +58,27 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n auto & unit = processing_units[current_unit_number]; + /* + * This is kind of suspicious -- the input_process_creator contract with + * respect to multithreaded use is not clear, but we hope that it is + * just a 'normal' factory class that doesn't have any state, and so we + * can use it from multiple threads simultaneously. + */ + ReadBuffer read_buffer(unit.segment.data(), unit.segment.size(), 0); + auto parser = std::make_unique( + input_processor_creator(read_buffer, header, context, + row_input_format_params, format_settings)); + unit.block_ext.block.clear(); unit.block_ext.block_missing_values.clear(); // We don't know how many blocks will be. So we have to read them all // until an empty block occured. Block block; - while (!finished && (block = unit.parser->read()) != Block()) + while (!finished && (block = parser->read()) != Block()) { unit.block_ext.block.emplace_back(block); - unit.block_ext.block_missing_values.emplace_back(unit.parser->getMissingValues()); + unit.block_ext.block_missing_values.emplace_back(parser->getMissingValues()); } if (!finished) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 866705dfc44..9e470c2fa18 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -91,8 +91,7 @@ public: { // Allocate more units than threads to decrease segmentator // waiting on reader on wraparound. The number is random. - for (size_t i = 0; i < builder.max_threads_to_use + 4; ++i) - processing_units.emplace_back(builder); + processing_units.resize(builder.max_threads_to_use + 4); segmentator_thread = ThreadFromGlobalPool([this] { segmentatorThreadFunction(); }); } @@ -114,8 +113,16 @@ public: is_killed = true; is_cancelled = true; - for (auto& unit: processing_units) - unit.parser->cancel(kill); + /* + * The format parsers themselves are not being cancelled here, so we'll + * have to wait until they process the current block. Given that the + * chunk size is on the order of megabytes, this should't be too long. + * We can't call IInputFormat->cancel here, because the parser object is + * local to the parser thread, and we don't want to introduce any + * synchronization between parser threads and the other threads to get + * better performance. An ideal solution would be to add a callback to + * IInputFormat that checks whether it was cancelled. + */ finishAndWait(); } @@ -186,21 +193,13 @@ private: struct ProcessingUnit { - explicit ProcessingUnit(const Builder & builder) : status(ProcessingUnitStatus::READY_TO_INSERT) + explicit ProcessingUnit() + : status(ProcessingUnitStatus::READY_TO_INSERT) { - readbuffer = std::make_unique(segment.data(), segment.size(), 0); - parser = std::make_unique( - builder.input_processor_creator(*readbuffer, - builder.input_creator_params.sample, - builder.input_creator_params.context, - builder.input_creator_params.row_input_format_params, - builder.input_creator_params.settings)); } BlockExt block_ext; - std::unique_ptr readbuffer; Memory<> segment; - std::unique_ptr parser; std::atomic status; bool is_last{false}; }; From 2f708c72be2f420f7f80358720ad42cd978479f6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 18 Nov 2019 19:01:36 +0300 Subject: [PATCH 46/90] add debug assert for Context --- dbms/src/Interpreters/Context.cpp | 163 +++++++++++++++++++++++++++++- dbms/src/Interpreters/Context.h | 59 ++++++++--- 2 files changed, 201 insertions(+), 21 deletions(-) diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 3c89d029784..b1e6c3c088c 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -331,19 +331,20 @@ Context Context::createGlobal() Context::~Context() = default; -InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; } +InterserverIOHandler & Context::getInterserverIOHandler() { assert_context_exists(); return shared->interserver_io_handler; } std::unique_lock Context::getLock() const { + assert_context_exists(); ProfileEvents::increment(ProfileEvents::ContextLock); CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; return std::unique_lock(shared->mutex); } -ProcessList & Context::getProcessList() { return shared->process_list; } -const ProcessList & Context::getProcessList() const { return shared->process_list; } -MergeList & Context::getMergeList() { return shared->merge_list; } -const MergeList & Context::getMergeList() const { return shared->merge_list; } +ProcessList & Context::getProcessList() { assert_context_exists(); return shared->process_list; } +const ProcessList & Context::getProcessList() const { assert_context_exists(); return shared->process_list; } +MergeList & Context::getMergeList() { assert_context_exists(); return shared->merge_list; } +const MergeList & Context::getMergeList() const { assert_context_exists(); return shared->merge_list; } const Databases Context::getDatabases() const @@ -361,6 +362,7 @@ Databases Context::getDatabases() Context::SessionKey Context::getSessionKey(const String & session_id) const { + assert_context_exists(); auto & user_name = client_info.current_user; if (user_name.empty()) @@ -372,6 +374,7 @@ Context::SessionKey Context::getSessionKey(const String & session_id) const void Context::scheduleCloseSession(const Context::SessionKey & key, std::chrono::steady_clock::duration timeout) { + assert_context_exists(); const UInt64 close_index = timeout / shared->close_interval + 1; const auto new_close_cycle = shared->close_cycle + close_index; @@ -648,6 +651,7 @@ void Context::calculateUserSettings() void Context::setProfile(const String & profile) { + assert_context_exists(); settings.setProfile(profile, *shared->users_config); auto new_constraints @@ -658,6 +662,7 @@ void Context::setProfile(const String & profile) std::shared_ptr Context::getUser(const String & user_name) { + assert_context_exists(); return shared->users_manager->getUser(user_name); } @@ -715,6 +720,7 @@ bool Context::hasDictionaryAccessRights(const String & dictionary_name) const void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) const { + assert_context_exists(); if (client_info.current_user.empty() || (database_name == "system")) { /// An unnamed user, i.e. server, has access to all databases. @@ -727,6 +733,7 @@ void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) c void Context::addDependencyUnsafe(const DatabaseAndTableName & from, const DatabaseAndTableName & where) { + assert_context_exists(); checkDatabaseAccessRightsImpl(from.first); checkDatabaseAccessRightsImpl(where.first); shared->view_dependencies[from].insert(where); @@ -745,6 +752,7 @@ void Context::addDependency(const DatabaseAndTableName & from, const DatabaseAnd void Context::removeDependencyUnsafe(const DatabaseAndTableName & from, const DatabaseAndTableName & where) { + assert_context_exists(); checkDatabaseAccessRightsImpl(from.first); checkDatabaseAccessRightsImpl(where.first); shared->view_dependencies[from].erase(where); @@ -816,6 +824,7 @@ bool Context::isDatabaseExist(const String & database_name) const bool Context::isExternalTableExist(const String & table_name) const { + assert_context_exists(); return external_tables.end() != external_tables.find(table_name); } @@ -861,12 +870,14 @@ void Context::assertDatabaseDoesntExist(const String & database_name) const const Scalars & Context::getScalars() const { + assert_context_exists(); return scalars; } const Block & Context::getScalar(const String & name) const { + assert_context_exists(); auto it = scalars.find(name); if (scalars.end() == it) throw Exception("Scalar " + backQuoteIfNeed(name) + " doesn't exist (internal bug)", ErrorCodes::UNKNOWN_SCALAR); @@ -898,6 +909,7 @@ Tables Context::getExternalTables() const StoragePtr Context::tryGetExternalTable(const String & table_name) const { + assert_context_exists(); TableAndCreateASTs::const_iterator jt = external_tables.find(table_name); if (external_tables.end() == jt) return StoragePtr(); @@ -924,6 +936,7 @@ StoragePtr Context::tryGetTable(const String & database_name, const String & tab StoragePtr Context::getTableImpl(const String & database_name, const String & table_name, Exception * exception) const { + assert_context_exists(); String db; DatabasePtr database; @@ -965,6 +978,7 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta void Context::addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast) { + assert_context_exists(); if (external_tables.end() != external_tables.find(table_name)) throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); @@ -974,18 +988,21 @@ void Context::addExternalTable(const String & table_name, const StoragePtr & sto void Context::addScalar(const String & name, const Block & block) { + assert_context_exists(); scalars[name] = block; } bool Context::hasScalar(const String & name) const { + assert_context_exists(); return scalars.count(name); } StoragePtr Context::tryRemoveExternalTable(const String & table_name) { + assert_context_exists(); TableAndCreateASTs::const_iterator it = external_tables.find(table_name); if (external_tables.end() == it) @@ -999,6 +1016,7 @@ StoragePtr Context::tryRemoveExternalTable(const String & table_name) StoragePtr Context::executeTableFunction(const ASTPtr & table_expression) { + assert_context_exists(); /// Slightly suboptimal. auto hash = table_expression->getTreeHash(); String key = toString(hash.first) + '_' + toString(hash.second); @@ -1019,6 +1037,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression) void Context::addViewSource(const StoragePtr & storage) { + assert_context_exists(); if (view_source) throw Exception( "Temporary view source storage " + backQuoteIfNeed(view_source->getName()) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); @@ -1028,6 +1047,7 @@ void Context::addViewSource(const StoragePtr & storage) StoragePtr Context::getViewSource() { + assert_context_exists(); return view_source; } @@ -1054,6 +1074,7 @@ DDLGuard::~DDLGuard() std::unique_ptr Context::getDDLGuard(const String & database, const String & table) const { + assert_context_exists(); std::unique_lock lock(shared->ddl_guards_mutex); return std::make_unique(shared->ddl_guards[database], std::move(lock), table); } @@ -1102,6 +1123,7 @@ ASTPtr Context::getCreateDictionaryQuery(const String & database_name, const Str ASTPtr Context::getCreateExternalTableQuery(const String & table_name) const { + assert_context_exists(); TableAndCreateASTs::const_iterator jt = external_tables.find(table_name); if (external_tables.end() == jt) throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " doesn't exist", ErrorCodes::UNKNOWN_TABLE); @@ -1121,12 +1143,14 @@ ASTPtr Context::getCreateDatabaseQuery(const String & database_name) const Settings Context::getSettings() const { + assert_context_exists(); return settings; } void Context::setSettings(const Settings & settings_) { + assert_context_exists(); settings = settings_; } @@ -1157,6 +1181,7 @@ void Context::setSetting(const String & name, const Field & value) void Context::applySettingChange(const SettingChange & change) { + assert_context_exists(); setSetting(change.name, change.value); } @@ -1171,6 +1196,7 @@ void Context::applySettingsChanges(const SettingsChanges & changes) void Context::checkSettingsConstraints(const SettingChange & change) { + assert_context_exists(); if (settings_constraints) settings_constraints->check(settings, change); } @@ -1178,6 +1204,7 @@ void Context::checkSettingsConstraints(const SettingChange & change) void Context::checkSettingsConstraints(const SettingsChanges & changes) { + assert_context_exists(); if (settings_constraints) settings_constraints->check(settings, changes); } @@ -1185,18 +1212,21 @@ void Context::checkSettingsConstraints(const SettingsChanges & changes) String Context::getCurrentDatabase() const { + assert_context_exists(); return current_database; } String Context::getCurrentQueryId() const { + assert_context_exists(); return client_info.current_query_id; } String Context::getInitialQueryId() const { + assert_context_exists(); return client_info.initial_query_id; } @@ -1211,6 +1241,7 @@ void Context::setCurrentDatabase(const String & name) void Context::setCurrentQueryId(const String & query_id) { + assert_context_exists(); if (!client_info.current_query_id.empty()) throw Exception("Logical error: attempt to set query_id twice", ErrorCodes::LOGICAL_ERROR); @@ -1251,6 +1282,7 @@ void Context::setCurrentQueryId(const String & query_id) void Context::killCurrentQuery() { + assert_context_exists(); if (process_list_elem) { process_list_elem->cancelQuery(true); @@ -1259,27 +1291,32 @@ void Context::killCurrentQuery() String Context::getDefaultFormat() const { + assert_context_exists(); return default_format.empty() ? "TabSeparated" : default_format; } void Context::setDefaultFormat(const String & name) { + assert_context_exists(); default_format = name; } MultiVersion::Version Context::getMacros() const { + assert_context_exists(); return shared->macros.get(); } void Context::setMacros(std::unique_ptr && macros) { + assert_context_exists(); shared->macros.set(std::move(macros)); } const Context & Context::getQueryContext() const { + assert_context_exists(); if (!query_context) throw Exception("There is no query", ErrorCodes::THERE_IS_NO_QUERY); return *query_context; @@ -1287,6 +1324,7 @@ const Context & Context::getQueryContext() const Context & Context::getQueryContext() { + assert_context_exists(); if (!query_context) throw Exception("There is no query", ErrorCodes::THERE_IS_NO_QUERY); return *query_context; @@ -1294,6 +1332,7 @@ Context & Context::getQueryContext() const Context & Context::getSessionContext() const { + assert_context_exists(); if (!session_context) throw Exception("There is no session", ErrorCodes::THERE_IS_NO_SESSION); return *session_context; @@ -1301,6 +1340,7 @@ const Context & Context::getSessionContext() const Context & Context::getSessionContext() { + assert_context_exists(); if (!session_context) throw Exception("There is no session", ErrorCodes::THERE_IS_NO_SESSION); return *session_context; @@ -1308,6 +1348,7 @@ Context & Context::getSessionContext() const Context & Context::getGlobalContext() const { + assert_context_exists(); if (!global_context) throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR); return *global_context; @@ -1315,6 +1356,7 @@ const Context & Context::getGlobalContext() const Context & Context::getGlobalContext() { + assert_context_exists(); if (!global_context) throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR); return *global_context; @@ -1323,17 +1365,20 @@ Context & Context::getGlobalContext() const EmbeddedDictionaries & Context::getEmbeddedDictionaries() const { + assert_context_exists(); return getEmbeddedDictionariesImpl(false); } EmbeddedDictionaries & Context::getEmbeddedDictionaries() { + assert_context_exists(); return getEmbeddedDictionariesImpl(false); } const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() const { + assert_context_exists(); std::lock_guard lock(shared->external_dictionaries_mutex); if (!shared->external_dictionaries_loader) { @@ -1347,12 +1392,14 @@ const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() cons ExternalDictionariesLoader & Context::getExternalDictionariesLoader() { + assert_context_exists(); return const_cast(const_cast(this)->getExternalDictionariesLoader()); } const ExternalModelsLoader & Context::getExternalModelsLoader() const { + assert_context_exists(); std::lock_guard lock(shared->external_models_mutex); if (!shared->external_models_loader) { @@ -1366,12 +1413,14 @@ const ExternalModelsLoader & Context::getExternalModelsLoader() const ExternalModelsLoader & Context::getExternalModelsLoader() { + assert_context_exists(); return const_cast(const_cast(this)->getExternalModelsLoader()); } EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_error) const { + assert_context_exists(); std::lock_guard lock(shared->embedded_dictionaries_mutex); if (!shared->embedded_dictionaries) @@ -1390,30 +1439,35 @@ EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_ void Context::tryCreateEmbeddedDictionaries() const { + assert_context_exists(); static_cast(getEmbeddedDictionariesImpl(true)); } void Context::setProgressCallback(ProgressCallback callback) { + assert_context_exists(); /// Callback is set to a session or to a query. In the session, only one query is processed at a time. Therefore, the lock is not needed. progress_callback = callback; } ProgressCallback Context::getProgressCallback() const { + assert_context_exists(); return progress_callback; } void Context::setProcessListElement(ProcessList::Element * elem) { + assert_context_exists(); /// Set to a session or query. In the session, only one query is processed at a time. Therefore, the lock is not needed. process_list_elem = elem; } ProcessList::Element * Context::getProcessListElement() const { + assert_context_exists(); return process_list_elem; } @@ -1515,6 +1569,7 @@ DDLWorker & Context::getDDLWorker() const zkutil::ZooKeeperPtr Context::getZooKeeper() const { + assert_context_exists(); std::lock_guard lock(shared->zookeeper_mutex); if (!shared->zookeeper) @@ -1527,24 +1582,28 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const void Context::resetZooKeeper() const { + assert_context_exists(); std::lock_guard lock(shared->zookeeper_mutex); shared->zookeeper.reset(); } bool Context::hasZooKeeper() const { + assert_context_exists(); return getConfigRef().has("zookeeper"); } void Context::setInterserverIOAddress(const String & host, UInt16 port) { + assert_context_exists(); shared->interserver_io_host = host; shared->interserver_io_port = port; } std::pair Context::getInterserverIOAddress() const { + assert_context_exists(); if (shared->interserver_io_host.empty() || shared->interserver_io_port == 0) throw Exception("Parameter 'interserver_http(s)_port' required for replication is not specified in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); @@ -1554,22 +1613,26 @@ std::pair Context::getInterserverIOAddress() const void Context::setInterserverCredentials(const String & user, const String & password) { + assert_context_exists(); shared->interserver_io_user = user; shared->interserver_io_password = password; } std::pair Context::getInterserverCredentials() const { + assert_context_exists(); return { shared->interserver_io_user, shared->interserver_io_password }; } void Context::setInterserverScheme(const String & scheme) { + assert_context_exists(); shared->interserver_scheme = scheme; } String Context::getInterserverScheme() const { + assert_context_exists(); return shared->interserver_scheme; } @@ -1593,6 +1656,7 @@ std::optional Context::getTCPPortSecure() const std::shared_ptr Context::getCluster(const std::string & cluster_name) const { + assert_context_exists(); auto res = getClusters().getCluster(cluster_name); if (!res) @@ -1604,12 +1668,14 @@ std::shared_ptr Context::getCluster(const std::string & cluster_name) c std::shared_ptr Context::tryGetCluster(const std::string & cluster_name) const { + assert_context_exists(); return getClusters().getCluster(cluster_name); } void Context::reloadClusterConfig() { + assert_context_exists(); while (true) { ConfigurationPtr cluster_config; @@ -1637,6 +1703,7 @@ void Context::reloadClusterConfig() Clusters & Context::getClusters() const { + assert_context_exists(); std::lock_guard lock(shared->clusters_mutex); if (!shared->clusters) { @@ -1651,6 +1718,7 @@ Clusters & Context::getClusters() const /// On repeating calls updates existing clusters and adds new clusters, doesn't delete old clusters void Context::setClustersConfig(const ConfigurationPtr & config, const String & config_name) { + assert_context_exists(); std::lock_guard lock(shared->clusters_mutex); shared->clusters_config = config; @@ -1664,6 +1732,7 @@ void Context::setClustersConfig(const ConfigurationPtr & config, const String & void Context::setCluster(const String & cluster_name, const std::shared_ptr & cluster) { + assert_context_exists(); std::lock_guard lock(shared->clusters_mutex); if (!shared->clusters) @@ -1681,11 +1750,13 @@ void Context::initializeSystemLogs() bool Context::hasTraceCollector() { + assert_context_exists(); return shared->hasTraceCollector(); } void Context::initializeTraceCollector() { + assert_context_exists(); shared->initializeTraceCollector(getTraceLog()); } @@ -1850,6 +1921,7 @@ const MergeTreeSettings & Context::getMergeTreeSettings() const void Context::checkCanBeDropped(const String & database, const String & table, const size_t & size, const size_t & max_size_to_drop) const { + assert_context_exists(); if (!max_size_to_drop || size <= max_size_to_drop) return; @@ -1891,6 +1963,7 @@ void Context::checkCanBeDropped(const String & database, const String & table, c void Context::setMaxTableSizeToDrop(size_t max_size) { + assert_context_exists(); // Is initialized at server startup shared->max_table_size_to_drop = max_size; } @@ -1898,6 +1971,7 @@ void Context::setMaxTableSizeToDrop(size_t max_size) void Context::checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const { + assert_context_exists(); size_t max_table_size_to_drop = shared->max_table_size_to_drop; checkCanBeDropped(database, table, table_size, max_table_size_to_drop); @@ -1906,6 +1980,7 @@ void Context::checkTableCanBeDropped(const String & database, const String & tab void Context::setMaxPartitionSizeToDrop(size_t max_size) { + assert_context_exists(); // Is initialized at server startup shared->max_partition_size_to_drop = max_size; } @@ -1913,6 +1988,7 @@ void Context::setMaxPartitionSizeToDrop(size_t max_size) void Context::checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const { + assert_context_exists(); size_t max_partition_size_to_drop = shared->max_partition_size_to_drop; checkCanBeDropped(database, table, partition_size, max_partition_size_to_drop); @@ -1921,16 +1997,19 @@ void Context::checkPartitionCanBeDropped(const String & database, const String & BlockInputStreamPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size) const { + assert_context_exists(); return FormatFactory::instance().getInput(name, buf, sample, *this, max_block_size); } BlockOutputStreamPtr Context::getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const { + assert_context_exists(); return FormatFactory::instance().getOutput(name, buf, sample, *this); } OutputFormatPtr Context::getOutputFormatProcessor(const String & name, WriteBuffer & buf, const Block & sample) const { + assert_context_exists(); return FormatFactory::instance().getOutputFormat(name, buf, sample, *this); } @@ -1944,12 +2023,14 @@ time_t Context::getUptimeSeconds() const void Context::setConfigReloadCallback(ConfigReloadCallback && callback) { + assert_context_exists(); /// Is initialized at server startup, so lock isn't required. Otherwise use mutex. shared->config_reload_callback = std::move(callback); } void Context::reloadConfig() const { + assert_context_exists(); /// Use mutex if callback may be changed after startup. if (!shared->config_reload_callback) throw Exception("Can't reload config beacuse config_reload_callback is not set.", ErrorCodes::LOGICAL_ERROR); @@ -1960,23 +2041,27 @@ void Context::reloadConfig() const void Context::shutdown() { + assert_context_exists(); shared->shutdown(); } Context::ApplicationType Context::getApplicationType() const { + assert_context_exists(); return shared->application_type; } void Context::setApplicationType(ApplicationType type) { + assert_context_exists(); /// Lock isn't required, you should set it at start shared->application_type = type; } void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & config) { + assert_context_exists(); shared->default_profile_name = config.getString("default_profile", "default"); shared->system_profile_name = config.getString("system_profile", shared->default_profile_name); setSetting("profile", shared->system_profile_name); @@ -1984,44 +2069,52 @@ void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & confi String Context::getDefaultProfileName() const { + assert_context_exists(); return shared->default_profile_name; } String Context::getSystemProfileName() const { + assert_context_exists(); return shared->system_profile_name; } String Context::getFormatSchemaPath() const { + assert_context_exists(); return shared->format_schema_path; } void Context::setFormatSchemaPath(const String & path) { + assert_context_exists(); shared->format_schema_path = path; } Context::SampleBlockCache & Context::getSampleBlockCache() const { + assert_context_exists(); return getQueryContext().sample_block_cache; } bool Context::hasQueryParameters() const { + assert_context_exists(); return !query_parameters.empty(); } const NameToNameMap & Context::getQueryParameters() const { + assert_context_exists(); return query_parameters; } void Context::setQueryParameter(const String & name, const String & value) { + assert_context_exists(); if (!query_parameters.emplace(name, value).second) throw Exception("Duplicate name " + backQuote(name) + " of query parameter", ErrorCodes::BAD_ARGUMENTS); } @@ -2065,12 +2158,14 @@ void Context::addXDBCBridgeCommand(std::unique_ptr cmd) const IHostContextPtr & Context::getHostContext() { + assert_context_exists(); return host_context; } const IHostContextPtr & Context::getHostContext() const { + assert_context_exists(); return host_context; } @@ -2088,6 +2183,7 @@ std::shared_ptr Context::getActionLocksManager() void Context::setExternalTablesInitializer(ExternalTablesInitializer && initializer) { + assert_context_exists(); if (external_tables_initializer_callback) throw Exception("External tables initializer is already set", ErrorCodes::LOGICAL_ERROR); @@ -2096,6 +2192,7 @@ void Context::setExternalTablesInitializer(ExternalTablesInitializer && initiali void Context::initializeExternalTablesIfSet() { + assert_context_exists(); if (external_tables_initializer_callback) { external_tables_initializer_callback(*this); @@ -2107,6 +2204,7 @@ void Context::initializeExternalTablesIfSet() void Context::setInputInitializer(InputInitializer && initializer) { + assert_context_exists(); if (input_initializer_callback) throw Exception("Input initializer is already set", ErrorCodes::LOGICAL_ERROR); @@ -2116,6 +2214,7 @@ void Context::setInputInitializer(InputInitializer && initializer) void Context::initializeInput(const StoragePtr & input_storage) { + assert_context_exists(); if (!input_initializer_callback) throw Exception("Input initializer is not set", ErrorCodes::LOGICAL_ERROR); @@ -2127,6 +2226,7 @@ void Context::initializeInput(const StoragePtr & input_storage) void Context::setInputBlocksReaderCallback(InputBlocksReader && reader) { + assert_context_exists(); if (input_blocks_reader) throw Exception("Input blocks reader is already set", ErrorCodes::LOGICAL_ERROR); @@ -2136,12 +2236,14 @@ void Context::setInputBlocksReaderCallback(InputBlocksReader && reader) InputBlocksReader Context::getInputBlocksReaderCallback() const { + assert_context_exists(); return input_blocks_reader; } void Context::resetInputCallbacks() { + assert_context_exists(); if (input_initializer_callback) input_initializer_callback = {}; @@ -2184,5 +2286,56 @@ void SessionCleaner::run() } } +#ifndef NDEBUG +std::unordered_map __ContextDebug::__contexts; +std::list> __ContextDebug::__destroyed_contexts; +std::mutex __ContextDebug::__contexts_lock; + +__ContextDebug::__ContextDebug() +{ + std::unique_lock lock{__contexts_lock}; + __contexts.insert({this, StackTrace()}); +} + +__ContextDebug::__ContextDebug(const __ContextDebug &) +{ + std::unique_lock lock{__contexts_lock}; + __contexts.insert({this, StackTrace()}); +} + +__ContextDebug & __ContextDebug::operator=(const __ContextDebug &) +{ + assert_context_exists(); + return *this; +} + +__ContextDebug::~__ContextDebug() +{ + std::unique_lock lock{__contexts_lock}; + auto it = __contexts.find(this); + __destroyed_contexts.emplace_front(it->first, std::move(it->second), StackTrace()); + if (__max_traces < __destroyed_contexts.size()) + __destroyed_contexts.pop_back(); + __contexts.erase(it); +} + +void __ContextDebug::assert_context_exists() const +{ + std::unique_lock lock{__contexts_lock}; + if (__contexts.find(static_cast(this)) == __contexts.end()) + { + String msg; + auto it = std::find_if(__destroyed_contexts.begin(), __destroyed_contexts.end(), [=](auto & tuple) { return static_cast(this) == std::get<0>(tuple); }); + if (it != __destroyed_contexts.end()) + { + msg += "\nContext was created here:\n" + std::get<1>(*it).toString(); + msg += "\nAnd was destroyed here:\n" + std::get<2>(*it).toString(); + } + throw Exception("\n================================\n" + "Assertion failed: trying to access Context, which doesn't exist. Most likely it's because of dangling reference." + msg + + "\n================================\n", ErrorCodes::LOGICAL_ERROR); + } +} +#endif } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index da4566e6b2c..0e1cafe640c 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -117,6 +117,27 @@ struct IHostContext using IHostContextPtr = std::shared_ptr; +#ifndef NDEBUG +/// Unfortunately, AddressSanitizer cannot find all usages of dangling references to Context +class __ContextDebug +{ + static std::unordered_map __contexts; + static std::list> __destroyed_contexts; + static const size_t __max_traces = 4096; + static std::mutex __contexts_lock; +protected: + __ContextDebug(); + __ContextDebug(const __ContextDebug &); + __ContextDebug & operator=(const __ContextDebug &); + __ContextDebug(__ContextDebug &&) = delete; + __ContextDebug & operator=(__ContextDebug &&) = delete; + ~__ContextDebug(); + +public: + void assert_context_exists() const; +}; +#endif + /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) * and copied part (which can be its own for each session or query). @@ -124,8 +145,14 @@ using IHostContextPtr = std::shared_ptr; * Everything is encapsulated for all sorts of checks and locks. */ class Context +#ifndef NDEBUG + : public __ContextDebug +#endif { private: +#ifdef NDEBUG + void assert_context_exists() const {}; +#endif using Shared = std::shared_ptr; Shared shared; @@ -235,8 +262,8 @@ public: InputBlocksReader getInputBlocksReaderCallback() const; void resetInputCallbacks(); - ClientInfo & getClientInfo() { return client_info; } - const ClientInfo & getClientInfo() const { return client_info; } + ClientInfo & getClientInfo() { assert_context_exists(); return client_info; } + const ClientInfo & getClientInfo() const { assert_context_exists(); return client_info; } void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address); QuotaForIntervals & getQuota(); @@ -301,8 +328,8 @@ public: void killCurrentQuery(); - void setInsertionTable(std::pair && db_and_table) { insertion_table = db_and_table; } - const std::pair & getInsertionTable() const { return insertion_table; } + void setInsertionTable(std::pair && db_and_table) { assert_context_exists(); insertion_table = db_and_table; } + const std::pair & getInsertionTable() const { assert_context_exists(); return insertion_table; } String getDefaultFormat() const; /// If default_format is not specified, some global default format is returned. void setDefaultFormat(const String & name); @@ -324,7 +351,7 @@ public: void checkSettingsConstraints(const SettingsChanges & changes); /// Returns the current constraints (can return null). - std::shared_ptr getSettingsConstraints() const { return settings_constraints; } + std::shared_ptr getSettingsConstraints() const { assert_context_exists(); return settings_constraints; } const EmbeddedDictionaries & getEmbeddedDictionaries() const; const ExternalDictionariesLoader & getExternalDictionariesLoader() const; @@ -384,25 +411,25 @@ public: const Context & getQueryContext() const; Context & getQueryContext(); - bool hasQueryContext() const { return query_context != nullptr; } + bool hasQueryContext() const { assert_context_exists(); return query_context != nullptr; } const Context & getSessionContext() const; Context & getSessionContext(); - bool hasSessionContext() const { return session_context != nullptr; } + bool hasSessionContext() const { assert_context_exists(); return session_context != nullptr; } const Context & getGlobalContext() const; Context & getGlobalContext(); - bool hasGlobalContext() const { return global_context != nullptr; } + bool hasGlobalContext() const { assert_context_exists(); return global_context != nullptr; } - void setQueryContext(Context & context_) { query_context = &context_; } - void setSessionContext(Context & context_) { session_context = &context_; } + void setQueryContext(Context & context_) { assert_context_exists(); query_context = &context_; } + void setSessionContext(Context & context_) { assert_context_exists(); session_context = &context_; } - void makeQueryContext() { query_context = this; } - void makeSessionContext() { session_context = this; } - void makeGlobalContext() { global_context = this; } + void makeQueryContext() { assert_context_exists(); query_context = this; } + void makeSessionContext() { assert_context_exists(); session_context = this; } + void makeGlobalContext() { assert_context_exists(); global_context = this; } - const Settings & getSettingsRef() const { return settings; } - Settings & getSettingsRef() { return settings; } + const Settings & getSettingsRef() const { assert_context_exists(); return settings; } + Settings & getSettingsRef() { assert_context_exists(); return settings; } void setProgressCallback(ProgressCallback callback); @@ -544,7 +571,7 @@ public: bool hasQueryParameters() const; const NameToNameMap & getQueryParameters() const; void setQueryParameter(const String & name, const String & value); - void setQueryParameters(const NameToNameMap & parameters) { query_parameters = parameters; } + void setQueryParameters(const NameToNameMap & parameters) { assert_context_exists(); query_parameters = parameters; } #if USE_EMBEDDED_COMPILER std::shared_ptr getCompiledExpressionCache() const; From a4d557f998784cd52a21bb3d5442e3bf9107f2c4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 18 Nov 2019 19:13:43 +0300 Subject: [PATCH 47/90] fix Context in interpretSubquery --- .../InterpreterSelectWithUnionQuery.cpp | 20 ++++++++++++------- .../InterpreterSelectWithUnionQuery.h | 2 +- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 1a2762a6daf..a876633609a 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -34,7 +34,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( const Names & required_result_column_names) : options(options_), query_ptr(query_ptr_), - context(context_) + context(std::make_shared(context_)) { const auto & ast = query_ptr->as(); @@ -57,7 +57,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( /// We use it to determine positions of 'required_result_column_names' in SELECT clause. Block full_result_header = InterpreterSelectQuery( - ast.list_of_selects->children.at(0), context, options.copy().analyze().noModify()).getSampleBlock(); + ast.list_of_selects->children.at(0), *context, options.copy().analyze().noModify()).getSampleBlock(); std::vector positions_of_required_result_columns(required_result_column_names.size()); for (size_t required_result_num = 0, size = required_result_column_names.size(); required_result_num < size; ++required_result_num) @@ -66,7 +66,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( for (size_t query_num = 1; query_num < num_selects; ++query_num) { Block full_result_header_for_current_select = InterpreterSelectQuery( - ast.list_of_selects->children.at(query_num), context, options.copy().analyze().noModify()).getSampleBlock(); + ast.list_of_selects->children.at(query_num), *context, options.copy().analyze().noModify()).getSampleBlock(); if (full_result_header_for_current_select.columns() != full_result_header.columns()) throw Exception("Different number of columns in UNION ALL elements:\n" @@ -88,7 +88,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( nested_interpreters.emplace_back(std::make_unique( ast.list_of_selects->children.at(query_num), - context, + *context, options, current_required_result_column_names)); } @@ -177,8 +177,11 @@ BlockInputStreams InterpreterSelectWithUnionQuery::executeWithMultipleStreams(Qu /// Unify data structure. if (nested_interpreters.size() > 1) + { for (auto & stream : nested_streams) - stream = std::make_shared(context, stream, result_header, ConvertingBlockInputStream::MatchColumnsMode::Position); + stream = std::make_shared(*context, stream, result_header,ConvertingBlockInputStream::MatchColumnsMode::Position); + parent_pipeline.addInterpreterContext(context); + } return nested_streams; } @@ -186,7 +189,7 @@ BlockInputStreams InterpreterSelectWithUnionQuery::executeWithMultipleStreams(Qu BlockIO InterpreterSelectWithUnionQuery::execute() { - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); BlockIO res; BlockInputStreams nested_streams = executeWithMultipleStreams(res.pipeline); @@ -208,6 +211,7 @@ BlockIO InterpreterSelectWithUnionQuery::execute() } res.in = result_stream; + res.pipeline.addInterpreterContext(context); return res; } @@ -242,9 +246,11 @@ QueryPipeline InterpreterSelectWithUnionQuery::executeWithProcessors() if (!pipelines.empty()) { auto common_header = getCommonHeaderForUnion(headers); - main_pipeline.unitePipelines(std::move(pipelines), common_header, context); + main_pipeline.unitePipelines(std::move(pipelines), common_header, *context); } + main_pipeline.addInterpreterContext(context); + return main_pipeline; } diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h index 0e07313f5b9..4e425d260e6 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -47,7 +47,7 @@ public: private: const SelectQueryOptions options; ASTPtr query_ptr; - const Context & context; + std::shared_ptr context; std::vector> nested_interpreters; From b7e9c69768b9028f81be06a6ca4ce867a18283aa Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 18 Nov 2019 19:49:23 +0300 Subject: [PATCH 48/90] distinguish table function with list of columns from an aggregate function with list of parameters --- dbms/src/Parsers/ExpressionElementParsers.cpp | 2 +- dbms/src/Parsers/ExpressionElementParsers.h | 3 +++ dbms/src/Parsers/ParserInsertQuery.cpp | 2 +- dbms/tests/integration/test_mysql_database_engine/test.py | 3 +-- 4 files changed, 6 insertions(+), 4 deletions(-) diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 89793a5042d..c26f9363797 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -252,7 +252,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } /// The parametric aggregate function has two lists (parameters and arguments) in parentheses. Example: quantile(0.9)(x). - if (pos->type == TokenType::OpeningRoundBracket) + if (allow_function_parameters && pos->type == TokenType::OpeningRoundBracket) { ++pos; diff --git a/dbms/src/Parsers/ExpressionElementParsers.h b/dbms/src/Parsers/ExpressionElementParsers.h index aab2cefbc2a..63ed1348b13 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.h +++ b/dbms/src/Parsers/ExpressionElementParsers.h @@ -90,9 +90,12 @@ protected: */ class ParserFunction : public IParserBase { +public: + ParserFunction(bool allow_function_parameters_ = true) : allow_function_parameters(allow_function_parameters_) {} protected: const char * getName() const { return "function"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); + bool allow_function_parameters; }; class ParserCodecDeclarationList : public IParserBase diff --git a/dbms/src/Parsers/ParserInsertQuery.cpp b/dbms/src/Parsers/ParserInsertQuery.cpp index c6434899a0f..62f9f57930c 100644 --- a/dbms/src/Parsers/ParserInsertQuery.cpp +++ b/dbms/src/Parsers/ParserInsertQuery.cpp @@ -35,7 +35,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserToken s_rparen(TokenType::ClosingRoundBracket); ParserIdentifier name_p; ParserList columns_p(std::make_unique(), std::make_unique(TokenType::Comma), false); - ParserFunction table_function_p; + ParserFunction table_function_p{false}; ASTPtr database; ASTPtr table; diff --git a/dbms/tests/integration/test_mysql_database_engine/test.py b/dbms/tests/integration/test_mysql_database_engine/test.py index 5e0cd7f62e2..0bac6010300 100644 --- a/dbms/tests/integration/test_mysql_database_engine/test.py +++ b/dbms/tests/integration/test_mysql_database_engine/test.py @@ -128,8 +128,7 @@ def get_mysql_conn(): return conn def get_mysql_table_function_expr(table_name): - # FIXME We need extra round brackets after "mysql" to distinguish table function with list of columns from an aggregate function with list of parameters (see ParserInsertQuery and ParserFunction) - return "mysql()('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format(table_name) + return "mysql('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format(table_name) def create_mysql_db(conn, name): with conn.cursor() as cursor: From 17b4565c4842118cfdb02be948303e333ebc794d Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 18 Nov 2019 22:16:41 +0300 Subject: [PATCH 49/90] Cleanup --- .../ParallelParsingBlockInputStream.cpp | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index aeeb4ff399e..86c86653a27 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -81,12 +81,13 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n unit.block_ext.block_missing_values.emplace_back(parser->getMissingValues()); } - if (!finished) - { - std::unique_lock lock(mutex); - unit.status = READY_TO_READ; - reader_condvar.notify_all(); - } + // We suppose we will get at least some blocks for a non-empty buffer, + // except at the end of file. Also see a matching assert in readImpl(). + assert(unit.is_last || unit.block_ext.block.size() > 0); + + std::unique_lock lock(mutex); + unit.status = READY_TO_READ; + reader_condvar.notify_all(); } catch (...) { @@ -157,6 +158,12 @@ Block ParallelParsingBlockInputStream::readImpl() if (unit.block_ext.block.size() == 0) { + /* + * Can we get zero blocks for an entire segment, when the format parser + * skips it entire content and does not create any blocks? Probably not, + * but if we ever do, we should add a loop around the above if, to skip + * these. Also see a matching assert in the parser thread. + */ assert(unit.is_last); finished = true; return Block{}; @@ -169,7 +176,7 @@ Block ParallelParsingBlockInputStream::readImpl() next_block_in_current_unit.value() += 1; - if (next_block_in_current_unit.value() == unit.block_ext.block.size()) + if (*next_block_in_current_unit == unit.block_ext.block.size()) { // Finished reading this Processing Unit, move to the next one. next_block_in_current_unit.reset(); From 168e15b2aed8f59230e3f438b1620edb63d9ff57 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 18 Nov 2019 22:25:17 +0300 Subject: [PATCH 50/90] chunk size -> chunk bytes --- dbms/src/Core/Settings.h | 6 +++--- .../DataStreams/ParallelParsingBlockInputStream.cpp | 2 +- dbms/src/DataStreams/ParallelParsingBlockInputStream.h | 8 ++++---- dbms/src/Formats/FormatFactory.cpp | 2 +- dbms/src/Formats/FormatFactory.h | 4 ++-- .../Processors/Formats/Impl/ValuesBlockInputFormat.cpp | 4 ++-- .../Processors/Formats/Impl/ValuesBlockInputFormat.h | 2 +- docs/en/operations/settings/settings.md | 10 +++++----- 8 files changed, 19 insertions(+), 19 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index b9f8a390a06..3a706881d44 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -108,9 +108,9 @@ struct Settings : public SettingsCollection M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.") \ M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.") \ \ - M(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for several data formats (JSONEachRow, TSV, TKSV, CSV).") \ - M(SettingUInt64, max_threads_for_parallel_parsing, 10, "The maximum number of threads to parallel parsing.") \ - M(SettingUInt64, min_chunk_size_for_parallel_parsing, (1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.") \ + M(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.") \ + M(SettingUInt64, max_threads_for_parallel_parsing, 0, "The maximum number of threads to parallel parsing.") \ + M(SettingUInt64, min_chunk_bytes_for_parallel_parsing, (1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.") \ \ M(SettingUInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.") \ M(SettingUInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.") \ diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index 86c86653a27..21233da9327 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -31,7 +31,7 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() unit.segment.resize(0); const bool have_more_data = file_segmentation_engine(original_buffer, - unit.segment, min_chunk_size); + unit.segment, min_chunk_bytes); unit.is_last = !have_more_data; unit.status = READY_TO_PARSE; diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 9e470c2fa18..47baadfeb92 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -17,7 +17,7 @@ namespace DB * ORDER-PRESERVING parallel parsing of data formats. * It splits original data into chunks. Then each chunk is parsed by different thread. * The number of chunks equals to max_threads_for_parallel_parsing setting. - * The size of chunk is equal to min_chunk_size_for_parallel_parsing setting. + * The size of chunk is equal to min_chunk_bytes_for_parallel_parsing setting. * * This stream has three kinds of threads: one segmentator, multiple parsers * (max_threads_for_parallel_parsing) and one reader thread -- that is, the one @@ -75,7 +75,7 @@ public: const InputCreatorParams &input_creator_params; FormatFactory::FileSegmentationEngine file_segmentation_engine; size_t max_threads_to_use; - size_t min_chunk_size; + size_t min_chunk_bytes; }; explicit ParallelParsingBlockInputStream(const Builder & builder) @@ -84,7 +84,7 @@ public: 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), - min_chunk_size(builder.min_chunk_size), + min_chunk_bytes(builder.min_chunk_bytes), original_buffer(builder.read_buffer), pool(builder.max_threads_to_use), file_segmentation_engine(builder.file_segmentation_engine) @@ -148,7 +148,7 @@ private: const FormatSettings format_settings; const InputProcessorCreator input_processor_creator; - const size_t min_chunk_size; + const size_t min_chunk_bytes; /* * This is declared as atomic to avoid UB, because parser threads access it diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 8eaad2ad6bd..75213f6799e 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -132,7 +132,7 @@ BlockInputStreamPtr FormatFactory::getInput( //LOG_TRACE(&Poco::Logger::get("FormatFactory::getInput()"), "Will use " << max_threads_to_use << " threads for parallel parsing."); auto params = ParallelParsingBlockInputStream::InputCreatorParams{sample, context, row_input_format_params, format_settings}; - ParallelParsingBlockInputStream::Builder builder{buf, input_getter, params, file_segmentation_engine, max_threads_to_use, settings.min_chunk_size_for_parallel_parsing}; + ParallelParsingBlockInputStream::Builder builder{buf, input_getter, params, file_segmentation_engine, max_threads_to_use, settings.min_chunk_bytes_for_parallel_parsing}; return std::make_shared(builder); } diff --git a/dbms/src/Formats/FormatFactory.h b/dbms/src/Formats/FormatFactory.h index 71acd10a0c8..46b19a55870 100644 --- a/dbms/src/Formats/FormatFactory.h +++ b/dbms/src/Formats/FormatFactory.h @@ -43,13 +43,13 @@ public: using ReadCallback = std::function; /** Fast reading data from buffer and save result to memory. - * Reads at least min_chunk_size bytes and some more until the end of the chunk, depends on the format. + * Reads at least min_chunk_bytes and some more until the end of the chunk, depends on the format. * Used in ParallelParsingBlockInputStream. */ using FileSegmentationEngine = std::function & memory, - size_t min_chunk_size)>; + size_t min_chunk_bytes)>; /// This callback allows to perform some additional actions after writing a single row. /// It's initial purpose was to flush Kafka message for each row. diff --git a/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 05bde62d902..861a2320dd6 100644 --- a/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -320,7 +320,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx } /// Can be used in fileSegmentationEngine for parallel parsing of Values -bool ValuesBlockInputFormat::skipToNextRow(size_t min_chunk_size, int balance) +bool ValuesBlockInputFormat::skipToNextRow(size_t min_chunk_bytes, int balance) { skipWhitespaceIfAny(buf); if (buf.eof() || *buf.position() == ';') @@ -328,7 +328,7 @@ bool ValuesBlockInputFormat::skipToNextRow(size_t min_chunk_size, int balance) bool quoted = false; size_t chunk_begin_buf_count = buf.count(); - while (!buf.eof() && (balance || buf.count() - chunk_begin_buf_count < min_chunk_size)) + while (!buf.eof() && (balance || buf.count() - chunk_begin_buf_count < min_chunk_bytes)) { buf.position() = find_first_symbols<'\\', '\'', ')', '('>(buf.position(), buf.buffer().end()); if (buf.position() == buf.buffer().end()) diff --git a/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index cf2c4efb1cf..ee0462f1eb4 100644 --- a/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -60,7 +60,7 @@ private: void readSuffix(); - bool skipToNextRow(size_t min_chunk_size = 0, int balance = 0); + bool skipToNextRow(size_t min_chunk_bytes = 0, int balance = 0); private: PeekableReadBuffer buf; diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c48142ca209..f912350e960 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1004,15 +1004,15 @@ Enable order-preserving parallel parsing of data formats such as JSONEachRow, TS ## max_threads_for_parallel_parsing - Type: unsigned int -- Default value: 10 +- Default value: 0 -The maximum number of threads for order-preserving parallel parsing of data formats. +The maximum number of threads for order-preserving parallel parsing of data formats. 0 means use global maximum. -## min_chunk_size_for_parallel_parsing +## min_chunk_bytes_for_parallel_parsing - Type: unsigned int -- Default value: 1024 * 1024 +- Default value: 1 MiB -The minimum chunk size in bytes, which each thread will parse in parallel. By default it equals to one megabyte. +The minimum chunk size in bytes, which each thread will parse in parallel. [Original article](https://clickhouse.yandex/docs/en/operations/settings/settings/) From f8f6de836a61a5db501764d7b8db024b17a4aa2c Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 18 Nov 2019 22:27:29 +0300 Subject: [PATCH 51/90] remove separate control for threads --- dbms/src/Core/Settings.h | 1 - dbms/src/DataStreams/ParallelParsingBlockInputStream.h | 7 +++---- dbms/src/Formats/FormatFactory.cpp | 9 +-------- docs/en/operations/settings/settings.md | 5 ----- 4 files changed, 4 insertions(+), 18 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 3a706881d44..a14afdc2a5e 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -109,7 +109,6 @@ struct Settings : public SettingsCollection M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.") \ \ M(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.") \ - M(SettingUInt64, max_threads_for_parallel_parsing, 0, "The maximum number of threads to parallel parsing.") \ M(SettingUInt64, min_chunk_bytes_for_parallel_parsing, (1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.") \ \ M(SettingUInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.") \ diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 47baadfeb92..9cf194b7408 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -16,12 +16,11 @@ namespace DB /** * ORDER-PRESERVING parallel parsing of data formats. * It splits original data into chunks. Then each chunk is parsed by different thread. - * The number of chunks equals to max_threads_for_parallel_parsing setting. + * The number of chunks equals to the number or parser threads. * The size of chunk is equal to min_chunk_bytes_for_parallel_parsing setting. * - * This stream has three kinds of threads: one segmentator, multiple parsers - * (max_threads_for_parallel_parsing) and one reader thread -- that is, the one - * from which readImpl() is called. + * This stream has three kinds of threads: one segmentator, multiple parsers, + * and one reader thread -- that is, the one from which readImpl() is called. * They operate one after another on parts of data called "processing units". * One unit consists of buffer with raw data from file, filled by segmentator * thread. This raw data is then parsed by a parser thread to form a number of diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 75213f6799e..c090c29eaad 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -124,15 +124,8 @@ BlockInputStreamPtr FormatFactory::getInput( row_input_format_params.max_execution_time = settings.max_execution_time; row_input_format_params.timeout_overflow_mode = settings.timeout_overflow_mode; - //The number of threads for parallel parsing must be less or equal settings.max_threads. - const size_t global_max_threads = settings.max_threads; - const size_t max_threads_for_parallel_parsing = settings.max_threads_for_parallel_parsing; - const size_t max_threads_to_use = max_threads_for_parallel_parsing == 0 ? global_max_threads : std::min(max_threads_for_parallel_parsing, global_max_threads); - - //LOG_TRACE(&Poco::Logger::get("FormatFactory::getInput()"), "Will use " << max_threads_to_use << " threads for parallel parsing."); - auto params = ParallelParsingBlockInputStream::InputCreatorParams{sample, context, row_input_format_params, format_settings}; - ParallelParsingBlockInputStream::Builder builder{buf, input_getter, params, file_segmentation_engine, max_threads_to_use, settings.min_chunk_bytes_for_parallel_parsing}; + ParallelParsingBlockInputStream::Builder builder{buf, input_getter, params, file_segmentation_engine, settings.max_threads, settings.min_chunk_bytes_for_parallel_parsing}; return std::make_shared(builder); } diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f912350e960..cf9c6619b74 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1001,11 +1001,6 @@ Default value: 0. Enable order-preserving parallel parsing of data formats such as JSONEachRow, TSV, TKSV and CSV. Reading will be single threaded and parsing will be multithreaded. -## max_threads_for_parallel_parsing - -- Type: unsigned int -- Default value: 0 - The maximum number of threads for order-preserving parallel parsing of data formats. 0 means use global maximum. ## min_chunk_bytes_for_parallel_parsing From 85627890590ca227609b6edd72e0b0e9a722a0a2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 19 Nov 2019 01:36:03 +0300 Subject: [PATCH 52/90] Revert "add debug assert for Context" This reverts commit 2f708c72be2f420f7f80358720ad42cd978479f6. --- dbms/src/Interpreters/Context.cpp | 163 +----------------------------- dbms/src/Interpreters/Context.h | 59 +++-------- 2 files changed, 21 insertions(+), 201 deletions(-) diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index b1e6c3c088c..3c89d029784 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -331,20 +331,19 @@ Context Context::createGlobal() Context::~Context() = default; -InterserverIOHandler & Context::getInterserverIOHandler() { assert_context_exists(); return shared->interserver_io_handler; } +InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; } std::unique_lock Context::getLock() const { - assert_context_exists(); ProfileEvents::increment(ProfileEvents::ContextLock); CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; return std::unique_lock(shared->mutex); } -ProcessList & Context::getProcessList() { assert_context_exists(); return shared->process_list; } -const ProcessList & Context::getProcessList() const { assert_context_exists(); return shared->process_list; } -MergeList & Context::getMergeList() { assert_context_exists(); return shared->merge_list; } -const MergeList & Context::getMergeList() const { assert_context_exists(); return shared->merge_list; } +ProcessList & Context::getProcessList() { return shared->process_list; } +const ProcessList & Context::getProcessList() const { return shared->process_list; } +MergeList & Context::getMergeList() { return shared->merge_list; } +const MergeList & Context::getMergeList() const { return shared->merge_list; } const Databases Context::getDatabases() const @@ -362,7 +361,6 @@ Databases Context::getDatabases() Context::SessionKey Context::getSessionKey(const String & session_id) const { - assert_context_exists(); auto & user_name = client_info.current_user; if (user_name.empty()) @@ -374,7 +372,6 @@ Context::SessionKey Context::getSessionKey(const String & session_id) const void Context::scheduleCloseSession(const Context::SessionKey & key, std::chrono::steady_clock::duration timeout) { - assert_context_exists(); const UInt64 close_index = timeout / shared->close_interval + 1; const auto new_close_cycle = shared->close_cycle + close_index; @@ -651,7 +648,6 @@ void Context::calculateUserSettings() void Context::setProfile(const String & profile) { - assert_context_exists(); settings.setProfile(profile, *shared->users_config); auto new_constraints @@ -662,7 +658,6 @@ void Context::setProfile(const String & profile) std::shared_ptr Context::getUser(const String & user_name) { - assert_context_exists(); return shared->users_manager->getUser(user_name); } @@ -720,7 +715,6 @@ bool Context::hasDictionaryAccessRights(const String & dictionary_name) const void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) const { - assert_context_exists(); if (client_info.current_user.empty() || (database_name == "system")) { /// An unnamed user, i.e. server, has access to all databases. @@ -733,7 +727,6 @@ void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) c void Context::addDependencyUnsafe(const DatabaseAndTableName & from, const DatabaseAndTableName & where) { - assert_context_exists(); checkDatabaseAccessRightsImpl(from.first); checkDatabaseAccessRightsImpl(where.first); shared->view_dependencies[from].insert(where); @@ -752,7 +745,6 @@ void Context::addDependency(const DatabaseAndTableName & from, const DatabaseAnd void Context::removeDependencyUnsafe(const DatabaseAndTableName & from, const DatabaseAndTableName & where) { - assert_context_exists(); checkDatabaseAccessRightsImpl(from.first); checkDatabaseAccessRightsImpl(where.first); shared->view_dependencies[from].erase(where); @@ -824,7 +816,6 @@ bool Context::isDatabaseExist(const String & database_name) const bool Context::isExternalTableExist(const String & table_name) const { - assert_context_exists(); return external_tables.end() != external_tables.find(table_name); } @@ -870,14 +861,12 @@ void Context::assertDatabaseDoesntExist(const String & database_name) const const Scalars & Context::getScalars() const { - assert_context_exists(); return scalars; } const Block & Context::getScalar(const String & name) const { - assert_context_exists(); auto it = scalars.find(name); if (scalars.end() == it) throw Exception("Scalar " + backQuoteIfNeed(name) + " doesn't exist (internal bug)", ErrorCodes::UNKNOWN_SCALAR); @@ -909,7 +898,6 @@ Tables Context::getExternalTables() const StoragePtr Context::tryGetExternalTable(const String & table_name) const { - assert_context_exists(); TableAndCreateASTs::const_iterator jt = external_tables.find(table_name); if (external_tables.end() == jt) return StoragePtr(); @@ -936,7 +924,6 @@ StoragePtr Context::tryGetTable(const String & database_name, const String & tab StoragePtr Context::getTableImpl(const String & database_name, const String & table_name, Exception * exception) const { - assert_context_exists(); String db; DatabasePtr database; @@ -978,7 +965,6 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta void Context::addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast) { - assert_context_exists(); if (external_tables.end() != external_tables.find(table_name)) throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); @@ -988,21 +974,18 @@ void Context::addExternalTable(const String & table_name, const StoragePtr & sto void Context::addScalar(const String & name, const Block & block) { - assert_context_exists(); scalars[name] = block; } bool Context::hasScalar(const String & name) const { - assert_context_exists(); return scalars.count(name); } StoragePtr Context::tryRemoveExternalTable(const String & table_name) { - assert_context_exists(); TableAndCreateASTs::const_iterator it = external_tables.find(table_name); if (external_tables.end() == it) @@ -1016,7 +999,6 @@ StoragePtr Context::tryRemoveExternalTable(const String & table_name) StoragePtr Context::executeTableFunction(const ASTPtr & table_expression) { - assert_context_exists(); /// Slightly suboptimal. auto hash = table_expression->getTreeHash(); String key = toString(hash.first) + '_' + toString(hash.second); @@ -1037,7 +1019,6 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression) void Context::addViewSource(const StoragePtr & storage) { - assert_context_exists(); if (view_source) throw Exception( "Temporary view source storage " + backQuoteIfNeed(view_source->getName()) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); @@ -1047,7 +1028,6 @@ void Context::addViewSource(const StoragePtr & storage) StoragePtr Context::getViewSource() { - assert_context_exists(); return view_source; } @@ -1074,7 +1054,6 @@ DDLGuard::~DDLGuard() std::unique_ptr Context::getDDLGuard(const String & database, const String & table) const { - assert_context_exists(); std::unique_lock lock(shared->ddl_guards_mutex); return std::make_unique(shared->ddl_guards[database], std::move(lock), table); } @@ -1123,7 +1102,6 @@ ASTPtr Context::getCreateDictionaryQuery(const String & database_name, const Str ASTPtr Context::getCreateExternalTableQuery(const String & table_name) const { - assert_context_exists(); TableAndCreateASTs::const_iterator jt = external_tables.find(table_name); if (external_tables.end() == jt) throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " doesn't exist", ErrorCodes::UNKNOWN_TABLE); @@ -1143,14 +1121,12 @@ ASTPtr Context::getCreateDatabaseQuery(const String & database_name) const Settings Context::getSettings() const { - assert_context_exists(); return settings; } void Context::setSettings(const Settings & settings_) { - assert_context_exists(); settings = settings_; } @@ -1181,7 +1157,6 @@ void Context::setSetting(const String & name, const Field & value) void Context::applySettingChange(const SettingChange & change) { - assert_context_exists(); setSetting(change.name, change.value); } @@ -1196,7 +1171,6 @@ void Context::applySettingsChanges(const SettingsChanges & changes) void Context::checkSettingsConstraints(const SettingChange & change) { - assert_context_exists(); if (settings_constraints) settings_constraints->check(settings, change); } @@ -1204,7 +1178,6 @@ void Context::checkSettingsConstraints(const SettingChange & change) void Context::checkSettingsConstraints(const SettingsChanges & changes) { - assert_context_exists(); if (settings_constraints) settings_constraints->check(settings, changes); } @@ -1212,21 +1185,18 @@ void Context::checkSettingsConstraints(const SettingsChanges & changes) String Context::getCurrentDatabase() const { - assert_context_exists(); return current_database; } String Context::getCurrentQueryId() const { - assert_context_exists(); return client_info.current_query_id; } String Context::getInitialQueryId() const { - assert_context_exists(); return client_info.initial_query_id; } @@ -1241,7 +1211,6 @@ void Context::setCurrentDatabase(const String & name) void Context::setCurrentQueryId(const String & query_id) { - assert_context_exists(); if (!client_info.current_query_id.empty()) throw Exception("Logical error: attempt to set query_id twice", ErrorCodes::LOGICAL_ERROR); @@ -1282,7 +1251,6 @@ void Context::setCurrentQueryId(const String & query_id) void Context::killCurrentQuery() { - assert_context_exists(); if (process_list_elem) { process_list_elem->cancelQuery(true); @@ -1291,32 +1259,27 @@ void Context::killCurrentQuery() String Context::getDefaultFormat() const { - assert_context_exists(); return default_format.empty() ? "TabSeparated" : default_format; } void Context::setDefaultFormat(const String & name) { - assert_context_exists(); default_format = name; } MultiVersion::Version Context::getMacros() const { - assert_context_exists(); return shared->macros.get(); } void Context::setMacros(std::unique_ptr && macros) { - assert_context_exists(); shared->macros.set(std::move(macros)); } const Context & Context::getQueryContext() const { - assert_context_exists(); if (!query_context) throw Exception("There is no query", ErrorCodes::THERE_IS_NO_QUERY); return *query_context; @@ -1324,7 +1287,6 @@ const Context & Context::getQueryContext() const Context & Context::getQueryContext() { - assert_context_exists(); if (!query_context) throw Exception("There is no query", ErrorCodes::THERE_IS_NO_QUERY); return *query_context; @@ -1332,7 +1294,6 @@ Context & Context::getQueryContext() const Context & Context::getSessionContext() const { - assert_context_exists(); if (!session_context) throw Exception("There is no session", ErrorCodes::THERE_IS_NO_SESSION); return *session_context; @@ -1340,7 +1301,6 @@ const Context & Context::getSessionContext() const Context & Context::getSessionContext() { - assert_context_exists(); if (!session_context) throw Exception("There is no session", ErrorCodes::THERE_IS_NO_SESSION); return *session_context; @@ -1348,7 +1308,6 @@ Context & Context::getSessionContext() const Context & Context::getGlobalContext() const { - assert_context_exists(); if (!global_context) throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR); return *global_context; @@ -1356,7 +1315,6 @@ const Context & Context::getGlobalContext() const Context & Context::getGlobalContext() { - assert_context_exists(); if (!global_context) throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR); return *global_context; @@ -1365,20 +1323,17 @@ Context & Context::getGlobalContext() const EmbeddedDictionaries & Context::getEmbeddedDictionaries() const { - assert_context_exists(); return getEmbeddedDictionariesImpl(false); } EmbeddedDictionaries & Context::getEmbeddedDictionaries() { - assert_context_exists(); return getEmbeddedDictionariesImpl(false); } const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() const { - assert_context_exists(); std::lock_guard lock(shared->external_dictionaries_mutex); if (!shared->external_dictionaries_loader) { @@ -1392,14 +1347,12 @@ const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() cons ExternalDictionariesLoader & Context::getExternalDictionariesLoader() { - assert_context_exists(); return const_cast(const_cast(this)->getExternalDictionariesLoader()); } const ExternalModelsLoader & Context::getExternalModelsLoader() const { - assert_context_exists(); std::lock_guard lock(shared->external_models_mutex); if (!shared->external_models_loader) { @@ -1413,14 +1366,12 @@ const ExternalModelsLoader & Context::getExternalModelsLoader() const ExternalModelsLoader & Context::getExternalModelsLoader() { - assert_context_exists(); return const_cast(const_cast(this)->getExternalModelsLoader()); } EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_error) const { - assert_context_exists(); std::lock_guard lock(shared->embedded_dictionaries_mutex); if (!shared->embedded_dictionaries) @@ -1439,35 +1390,30 @@ EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_ void Context::tryCreateEmbeddedDictionaries() const { - assert_context_exists(); static_cast(getEmbeddedDictionariesImpl(true)); } void Context::setProgressCallback(ProgressCallback callback) { - assert_context_exists(); /// Callback is set to a session or to a query. In the session, only one query is processed at a time. Therefore, the lock is not needed. progress_callback = callback; } ProgressCallback Context::getProgressCallback() const { - assert_context_exists(); return progress_callback; } void Context::setProcessListElement(ProcessList::Element * elem) { - assert_context_exists(); /// Set to a session or query. In the session, only one query is processed at a time. Therefore, the lock is not needed. process_list_elem = elem; } ProcessList::Element * Context::getProcessListElement() const { - assert_context_exists(); return process_list_elem; } @@ -1569,7 +1515,6 @@ DDLWorker & Context::getDDLWorker() const zkutil::ZooKeeperPtr Context::getZooKeeper() const { - assert_context_exists(); std::lock_guard lock(shared->zookeeper_mutex); if (!shared->zookeeper) @@ -1582,28 +1527,24 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const void Context::resetZooKeeper() const { - assert_context_exists(); std::lock_guard lock(shared->zookeeper_mutex); shared->zookeeper.reset(); } bool Context::hasZooKeeper() const { - assert_context_exists(); return getConfigRef().has("zookeeper"); } void Context::setInterserverIOAddress(const String & host, UInt16 port) { - assert_context_exists(); shared->interserver_io_host = host; shared->interserver_io_port = port; } std::pair Context::getInterserverIOAddress() const { - assert_context_exists(); if (shared->interserver_io_host.empty() || shared->interserver_io_port == 0) throw Exception("Parameter 'interserver_http(s)_port' required for replication is not specified in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); @@ -1613,26 +1554,22 @@ std::pair Context::getInterserverIOAddress() const void Context::setInterserverCredentials(const String & user, const String & password) { - assert_context_exists(); shared->interserver_io_user = user; shared->interserver_io_password = password; } std::pair Context::getInterserverCredentials() const { - assert_context_exists(); return { shared->interserver_io_user, shared->interserver_io_password }; } void Context::setInterserverScheme(const String & scheme) { - assert_context_exists(); shared->interserver_scheme = scheme; } String Context::getInterserverScheme() const { - assert_context_exists(); return shared->interserver_scheme; } @@ -1656,7 +1593,6 @@ std::optional Context::getTCPPortSecure() const std::shared_ptr Context::getCluster(const std::string & cluster_name) const { - assert_context_exists(); auto res = getClusters().getCluster(cluster_name); if (!res) @@ -1668,14 +1604,12 @@ std::shared_ptr Context::getCluster(const std::string & cluster_name) c std::shared_ptr Context::tryGetCluster(const std::string & cluster_name) const { - assert_context_exists(); return getClusters().getCluster(cluster_name); } void Context::reloadClusterConfig() { - assert_context_exists(); while (true) { ConfigurationPtr cluster_config; @@ -1703,7 +1637,6 @@ void Context::reloadClusterConfig() Clusters & Context::getClusters() const { - assert_context_exists(); std::lock_guard lock(shared->clusters_mutex); if (!shared->clusters) { @@ -1718,7 +1651,6 @@ Clusters & Context::getClusters() const /// On repeating calls updates existing clusters and adds new clusters, doesn't delete old clusters void Context::setClustersConfig(const ConfigurationPtr & config, const String & config_name) { - assert_context_exists(); std::lock_guard lock(shared->clusters_mutex); shared->clusters_config = config; @@ -1732,7 +1664,6 @@ void Context::setClustersConfig(const ConfigurationPtr & config, const String & void Context::setCluster(const String & cluster_name, const std::shared_ptr & cluster) { - assert_context_exists(); std::lock_guard lock(shared->clusters_mutex); if (!shared->clusters) @@ -1750,13 +1681,11 @@ void Context::initializeSystemLogs() bool Context::hasTraceCollector() { - assert_context_exists(); return shared->hasTraceCollector(); } void Context::initializeTraceCollector() { - assert_context_exists(); shared->initializeTraceCollector(getTraceLog()); } @@ -1921,7 +1850,6 @@ const MergeTreeSettings & Context::getMergeTreeSettings() const void Context::checkCanBeDropped(const String & database, const String & table, const size_t & size, const size_t & max_size_to_drop) const { - assert_context_exists(); if (!max_size_to_drop || size <= max_size_to_drop) return; @@ -1963,7 +1891,6 @@ void Context::checkCanBeDropped(const String & database, const String & table, c void Context::setMaxTableSizeToDrop(size_t max_size) { - assert_context_exists(); // Is initialized at server startup shared->max_table_size_to_drop = max_size; } @@ -1971,7 +1898,6 @@ void Context::setMaxTableSizeToDrop(size_t max_size) void Context::checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const { - assert_context_exists(); size_t max_table_size_to_drop = shared->max_table_size_to_drop; checkCanBeDropped(database, table, table_size, max_table_size_to_drop); @@ -1980,7 +1906,6 @@ void Context::checkTableCanBeDropped(const String & database, const String & tab void Context::setMaxPartitionSizeToDrop(size_t max_size) { - assert_context_exists(); // Is initialized at server startup shared->max_partition_size_to_drop = max_size; } @@ -1988,7 +1913,6 @@ void Context::setMaxPartitionSizeToDrop(size_t max_size) void Context::checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const { - assert_context_exists(); size_t max_partition_size_to_drop = shared->max_partition_size_to_drop; checkCanBeDropped(database, table, partition_size, max_partition_size_to_drop); @@ -1997,19 +1921,16 @@ void Context::checkPartitionCanBeDropped(const String & database, const String & BlockInputStreamPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size) const { - assert_context_exists(); return FormatFactory::instance().getInput(name, buf, sample, *this, max_block_size); } BlockOutputStreamPtr Context::getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const { - assert_context_exists(); return FormatFactory::instance().getOutput(name, buf, sample, *this); } OutputFormatPtr Context::getOutputFormatProcessor(const String & name, WriteBuffer & buf, const Block & sample) const { - assert_context_exists(); return FormatFactory::instance().getOutputFormat(name, buf, sample, *this); } @@ -2023,14 +1944,12 @@ time_t Context::getUptimeSeconds() const void Context::setConfigReloadCallback(ConfigReloadCallback && callback) { - assert_context_exists(); /// Is initialized at server startup, so lock isn't required. Otherwise use mutex. shared->config_reload_callback = std::move(callback); } void Context::reloadConfig() const { - assert_context_exists(); /// Use mutex if callback may be changed after startup. if (!shared->config_reload_callback) throw Exception("Can't reload config beacuse config_reload_callback is not set.", ErrorCodes::LOGICAL_ERROR); @@ -2041,27 +1960,23 @@ void Context::reloadConfig() const void Context::shutdown() { - assert_context_exists(); shared->shutdown(); } Context::ApplicationType Context::getApplicationType() const { - assert_context_exists(); return shared->application_type; } void Context::setApplicationType(ApplicationType type) { - assert_context_exists(); /// Lock isn't required, you should set it at start shared->application_type = type; } void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & config) { - assert_context_exists(); shared->default_profile_name = config.getString("default_profile", "default"); shared->system_profile_name = config.getString("system_profile", shared->default_profile_name); setSetting("profile", shared->system_profile_name); @@ -2069,52 +1984,44 @@ void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & confi String Context::getDefaultProfileName() const { - assert_context_exists(); return shared->default_profile_name; } String Context::getSystemProfileName() const { - assert_context_exists(); return shared->system_profile_name; } String Context::getFormatSchemaPath() const { - assert_context_exists(); return shared->format_schema_path; } void Context::setFormatSchemaPath(const String & path) { - assert_context_exists(); shared->format_schema_path = path; } Context::SampleBlockCache & Context::getSampleBlockCache() const { - assert_context_exists(); return getQueryContext().sample_block_cache; } bool Context::hasQueryParameters() const { - assert_context_exists(); return !query_parameters.empty(); } const NameToNameMap & Context::getQueryParameters() const { - assert_context_exists(); return query_parameters; } void Context::setQueryParameter(const String & name, const String & value) { - assert_context_exists(); if (!query_parameters.emplace(name, value).second) throw Exception("Duplicate name " + backQuote(name) + " of query parameter", ErrorCodes::BAD_ARGUMENTS); } @@ -2158,14 +2065,12 @@ void Context::addXDBCBridgeCommand(std::unique_ptr cmd) const IHostContextPtr & Context::getHostContext() { - assert_context_exists(); return host_context; } const IHostContextPtr & Context::getHostContext() const { - assert_context_exists(); return host_context; } @@ -2183,7 +2088,6 @@ std::shared_ptr Context::getActionLocksManager() void Context::setExternalTablesInitializer(ExternalTablesInitializer && initializer) { - assert_context_exists(); if (external_tables_initializer_callback) throw Exception("External tables initializer is already set", ErrorCodes::LOGICAL_ERROR); @@ -2192,7 +2096,6 @@ void Context::setExternalTablesInitializer(ExternalTablesInitializer && initiali void Context::initializeExternalTablesIfSet() { - assert_context_exists(); if (external_tables_initializer_callback) { external_tables_initializer_callback(*this); @@ -2204,7 +2107,6 @@ void Context::initializeExternalTablesIfSet() void Context::setInputInitializer(InputInitializer && initializer) { - assert_context_exists(); if (input_initializer_callback) throw Exception("Input initializer is already set", ErrorCodes::LOGICAL_ERROR); @@ -2214,7 +2116,6 @@ void Context::setInputInitializer(InputInitializer && initializer) void Context::initializeInput(const StoragePtr & input_storage) { - assert_context_exists(); if (!input_initializer_callback) throw Exception("Input initializer is not set", ErrorCodes::LOGICAL_ERROR); @@ -2226,7 +2127,6 @@ void Context::initializeInput(const StoragePtr & input_storage) void Context::setInputBlocksReaderCallback(InputBlocksReader && reader) { - assert_context_exists(); if (input_blocks_reader) throw Exception("Input blocks reader is already set", ErrorCodes::LOGICAL_ERROR); @@ -2236,14 +2136,12 @@ void Context::setInputBlocksReaderCallback(InputBlocksReader && reader) InputBlocksReader Context::getInputBlocksReaderCallback() const { - assert_context_exists(); return input_blocks_reader; } void Context::resetInputCallbacks() { - assert_context_exists(); if (input_initializer_callback) input_initializer_callback = {}; @@ -2286,56 +2184,5 @@ void SessionCleaner::run() } } -#ifndef NDEBUG -std::unordered_map __ContextDebug::__contexts; -std::list> __ContextDebug::__destroyed_contexts; -std::mutex __ContextDebug::__contexts_lock; - -__ContextDebug::__ContextDebug() -{ - std::unique_lock lock{__contexts_lock}; - __contexts.insert({this, StackTrace()}); -} - -__ContextDebug::__ContextDebug(const __ContextDebug &) -{ - std::unique_lock lock{__contexts_lock}; - __contexts.insert({this, StackTrace()}); -} - -__ContextDebug & __ContextDebug::operator=(const __ContextDebug &) -{ - assert_context_exists(); - return *this; -} - -__ContextDebug::~__ContextDebug() -{ - std::unique_lock lock{__contexts_lock}; - auto it = __contexts.find(this); - __destroyed_contexts.emplace_front(it->first, std::move(it->second), StackTrace()); - if (__max_traces < __destroyed_contexts.size()) - __destroyed_contexts.pop_back(); - __contexts.erase(it); -} - -void __ContextDebug::assert_context_exists() const -{ - std::unique_lock lock{__contexts_lock}; - if (__contexts.find(static_cast(this)) == __contexts.end()) - { - String msg; - auto it = std::find_if(__destroyed_contexts.begin(), __destroyed_contexts.end(), [=](auto & tuple) { return static_cast(this) == std::get<0>(tuple); }); - if (it != __destroyed_contexts.end()) - { - msg += "\nContext was created here:\n" + std::get<1>(*it).toString(); - msg += "\nAnd was destroyed here:\n" + std::get<2>(*it).toString(); - } - throw Exception("\n================================\n" - "Assertion failed: trying to access Context, which doesn't exist. Most likely it's because of dangling reference." + msg + - "\n================================\n", ErrorCodes::LOGICAL_ERROR); - } -} -#endif } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 0e1cafe640c..da4566e6b2c 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -117,27 +117,6 @@ struct IHostContext using IHostContextPtr = std::shared_ptr; -#ifndef NDEBUG -/// Unfortunately, AddressSanitizer cannot find all usages of dangling references to Context -class __ContextDebug -{ - static std::unordered_map __contexts; - static std::list> __destroyed_contexts; - static const size_t __max_traces = 4096; - static std::mutex __contexts_lock; -protected: - __ContextDebug(); - __ContextDebug(const __ContextDebug &); - __ContextDebug & operator=(const __ContextDebug &); - __ContextDebug(__ContextDebug &&) = delete; - __ContextDebug & operator=(__ContextDebug &&) = delete; - ~__ContextDebug(); - -public: - void assert_context_exists() const; -}; -#endif - /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) * and copied part (which can be its own for each session or query). @@ -145,14 +124,8 @@ public: * Everything is encapsulated for all sorts of checks and locks. */ class Context -#ifndef NDEBUG - : public __ContextDebug -#endif { private: -#ifdef NDEBUG - void assert_context_exists() const {}; -#endif using Shared = std::shared_ptr; Shared shared; @@ -262,8 +235,8 @@ public: InputBlocksReader getInputBlocksReaderCallback() const; void resetInputCallbacks(); - ClientInfo & getClientInfo() { assert_context_exists(); return client_info; } - const ClientInfo & getClientInfo() const { assert_context_exists(); return client_info; } + ClientInfo & getClientInfo() { return client_info; } + const ClientInfo & getClientInfo() const { return client_info; } void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address); QuotaForIntervals & getQuota(); @@ -328,8 +301,8 @@ public: void killCurrentQuery(); - void setInsertionTable(std::pair && db_and_table) { assert_context_exists(); insertion_table = db_and_table; } - const std::pair & getInsertionTable() const { assert_context_exists(); return insertion_table; } + void setInsertionTable(std::pair && db_and_table) { insertion_table = db_and_table; } + const std::pair & getInsertionTable() const { return insertion_table; } String getDefaultFormat() const; /// If default_format is not specified, some global default format is returned. void setDefaultFormat(const String & name); @@ -351,7 +324,7 @@ public: void checkSettingsConstraints(const SettingsChanges & changes); /// Returns the current constraints (can return null). - std::shared_ptr getSettingsConstraints() const { assert_context_exists(); return settings_constraints; } + std::shared_ptr getSettingsConstraints() const { return settings_constraints; } const EmbeddedDictionaries & getEmbeddedDictionaries() const; const ExternalDictionariesLoader & getExternalDictionariesLoader() const; @@ -411,25 +384,25 @@ public: const Context & getQueryContext() const; Context & getQueryContext(); - bool hasQueryContext() const { assert_context_exists(); return query_context != nullptr; } + bool hasQueryContext() const { return query_context != nullptr; } const Context & getSessionContext() const; Context & getSessionContext(); - bool hasSessionContext() const { assert_context_exists(); return session_context != nullptr; } + bool hasSessionContext() const { return session_context != nullptr; } const Context & getGlobalContext() const; Context & getGlobalContext(); - bool hasGlobalContext() const { assert_context_exists(); return global_context != nullptr; } + bool hasGlobalContext() const { return global_context != nullptr; } - void setQueryContext(Context & context_) { assert_context_exists(); query_context = &context_; } - void setSessionContext(Context & context_) { assert_context_exists(); session_context = &context_; } + void setQueryContext(Context & context_) { query_context = &context_; } + void setSessionContext(Context & context_) { session_context = &context_; } - void makeQueryContext() { assert_context_exists(); query_context = this; } - void makeSessionContext() { assert_context_exists(); session_context = this; } - void makeGlobalContext() { assert_context_exists(); global_context = this; } + void makeQueryContext() { query_context = this; } + void makeSessionContext() { session_context = this; } + void makeGlobalContext() { global_context = this; } - const Settings & getSettingsRef() const { assert_context_exists(); return settings; } - Settings & getSettingsRef() { assert_context_exists(); return settings; } + const Settings & getSettingsRef() const { return settings; } + Settings & getSettingsRef() { return settings; } void setProgressCallback(ProgressCallback callback); @@ -571,7 +544,7 @@ public: bool hasQueryParameters() const; const NameToNameMap & getQueryParameters() const; void setQueryParameter(const String & name, const String & value); - void setQueryParameters(const NameToNameMap & parameters) { assert_context_exists(); query_parameters = parameters; } + void setQueryParameters(const NameToNameMap & parameters) { query_parameters = parameters; } #if USE_EMBEDDED_COMPILER std::shared_ptr getCompiledExpressionCache() const; From 5504e3d76836f7265158ab299e4e44438a588267 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 19 Nov 2019 13:04:42 +0100 Subject: [PATCH 53/90] Disable DwarfFDECache in libunwind --- contrib/libunwind | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libunwind b/contrib/libunwind index 5afe6d87ae9..96f5f17125a 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 5afe6d87ae9e66485c7fcb106d2f7c2c0359c8f6 +Subproject commit 96f5f17125a3d8eed890e365868ccaa96bcd2301 From 8cbc52e711e376800a3aceded019c3ef7686fcf5 Mon Sep 17 00:00:00 2001 From: Andrei Bodrov Date: Tue, 19 Nov 2019 15:46:07 +0300 Subject: [PATCH 54/90] add gzip read/write to file/s3/url/hdfs --- dbms/programs/server/HTTPHandler.cpp | 4 +- dbms/src/DataStreams/IBlockInputStream.h | 13 +++++ dbms/src/DataStreams/IBlockOutputStream.h | 15 ++++++ dbms/src/IO/CompressionMethod.h | 1 + dbms/src/IO/WriteBuffer.h | 3 ++ .../IO/WriteBufferFromHTTPServerResponse.cpp | 12 ++--- .../IO/WriteBufferFromHTTPServerResponse.h | 2 +- dbms/src/IO/ZlibDeflatingWriteBuffer.cpp | 24 ++++----- dbms/src/IO/ZlibDeflatingWriteBuffer.h | 4 +- dbms/src/IO/ZlibInflatingReadBuffer.cpp | 14 ++--- dbms/src/IO/ZlibInflatingReadBuffer.h | 4 +- dbms/src/IO/tests/zlib_buffers.cpp | 8 +-- dbms/src/Storages/IStorage.cpp | 18 +++++++ dbms/src/Storages/IStorage.h | 3 ++ dbms/src/Storages/StorageFile.cpp | 43 +++++++++------ dbms/src/Storages/StorageFile.h | 4 +- dbms/src/Storages/StorageHDFS.cpp | 39 +++++++++----- dbms/src/Storages/StorageHDFS.h | 4 +- dbms/src/Storages/StorageS3.cpp | 37 ++++++++----- dbms/src/Storages/StorageS3.h | 4 +- dbms/src/Storages/StorageURL.cpp | 45 +++++++++++----- dbms/src/Storages/StorageURL.h | 9 ++-- dbms/src/Storages/StorageXDBC.cpp | 3 +- .../TableFunctions/ITableFunctionFileLike.cpp | 14 +++-- .../TableFunctions/ITableFunctionFileLike.h | 2 +- dbms/src/TableFunctions/TableFunctionFile.cpp | 5 +- dbms/src/TableFunctions/TableFunctionFile.h | 2 +- dbms/src/TableFunctions/TableFunctionHDFS.cpp | 5 +- dbms/src/TableFunctions/TableFunctionHDFS.h | 2 +- dbms/src/TableFunctions/TableFunctionS3.cpp | 4 +- dbms/src/TableFunctions/TableFunctionS3.h | 3 +- dbms/src/TableFunctions/TableFunctionURL.cpp | 4 +- dbms/src/TableFunctions/TableFunctionURL.h | 2 +- dbms/tests/integration/helpers/hdfs_api.py | 13 ++++- .../integration/test_storage_hdfs/test.py | 54 ++++++++++++++++++- .../0_stateless/01030_storage_hdfs_syntax.sql | 2 +- .../0_stateless/01030_storage_s3_syntax.sql | 2 +- .../0_stateless/01030_storage_url_syntax.sql | 2 +- 38 files changed, 311 insertions(+), 118 deletions(-) diff --git a/dbms/programs/server/HTTPHandler.cpp b/dbms/programs/server/HTTPHandler.cpp index 7ab1105e453..676ab05949e 100644 --- a/dbms/programs/server/HTTPHandler.cpp +++ b/dbms/programs/server/HTTPHandler.cpp @@ -407,11 +407,11 @@ void HTTPHandler::processQuery( { if (http_request_compression_method_str == "gzip") { - in_post = std::make_unique(*in_post_raw, CompressionMethod::Gzip); + in_post = std::make_unique(std::move(in_post_raw), CompressionMethod::Gzip); } else if (http_request_compression_method_str == "deflate") { - in_post = std::make_unique(*in_post_raw, CompressionMethod::Zlib); + in_post = std::make_unique(std::move(in_post_raw), CompressionMethod::Zlib); } #if USE_BROTLI else if (http_request_compression_method_str == "br") diff --git a/dbms/src/DataStreams/IBlockInputStream.h b/dbms/src/DataStreams/IBlockInputStream.h index dfa9194a6f9..73098163833 100644 --- a/dbms/src/DataStreams/IBlockInputStream.h +++ b/dbms/src/DataStreams/IBlockInputStream.h @@ -5,7 +5,9 @@ #include #include #include +#include #include +#include #include #include @@ -228,6 +230,17 @@ public: /// Enable calculation of minimums and maximums by the result columns. void enableExtremes() { enabled_extremes = true; } + template + std::unique_ptr getBuffer(const DB::CompressionMethod method, Types... args) + { + if (method == DB::CompressionMethod::Gzip) + { + auto read_buf = std::make_unique(args...); + return std::make_unique(std::move(read_buf), method); + } + return std::make_unique(args...); + } + protected: /// Order is important: `table_locks` must be destroyed after `children` so that tables from /// which child streams read are protected by the locks during the lifetime of the child streams. diff --git a/dbms/src/DataStreams/IBlockOutputStream.h b/dbms/src/DataStreams/IBlockOutputStream.h index 060438ba457..de8895c850e 100644 --- a/dbms/src/DataStreams/IBlockOutputStream.h +++ b/dbms/src/DataStreams/IBlockOutputStream.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include @@ -63,6 +65,19 @@ public: */ void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); } + template + std::unique_ptr getBuffer(const DB::CompressionMethod method, Types... args) + { + if (method == DB::CompressionMethod::Gzip) + { + auto write_buf = std::make_unique(args...); + return std::make_unique(std::move(write_buf), method, (int) 1 /* compression level */); + } + return std::make_unique(args...); + } + + virtual void finalize() {} + private: std::vector table_locks; }; diff --git a/dbms/src/IO/CompressionMethod.h b/dbms/src/IO/CompressionMethod.h index 96b9d41305e..c54d2b581fd 100644 --- a/dbms/src/IO/CompressionMethod.h +++ b/dbms/src/IO/CompressionMethod.h @@ -12,6 +12,7 @@ enum class CompressionMethod /// This option corresponds to HTTP Content-Encoding: deflate. Zlib, Brotli, + None }; } diff --git a/dbms/src/IO/WriteBuffer.h b/dbms/src/IO/WriteBuffer.h index 18c6c8c6640..0777062716f 100644 --- a/dbms/src/IO/WriteBuffer.h +++ b/dbms/src/IO/WriteBuffer.h @@ -90,6 +90,9 @@ public: ++pos; } + void virtual sync() {} + void virtual finalize() {} + private: /** Write the data in the buffer (from the beginning of the buffer to the current position). * Throw an exception if something is wrong. diff --git a/dbms/src/IO/WriteBufferFromHTTPServerResponse.cpp b/dbms/src/IO/WriteBufferFromHTTPServerResponse.cpp index 11eb9831d58..f8bd166a4dd 100644 --- a/dbms/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/dbms/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -113,8 +113,8 @@ void WriteBufferFromHTTPServerResponse::nextImpl() response.set("Content-Encoding", "gzip"); response_body_ostr = &(response.send()); #endif - out_raw.emplace(*response_body_ostr); - deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin()); + out_raw = std::make_unique(*response_body_ostr); + deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin()); out = &*deflating_buf; } else if (compression_method == CompressionMethod::Zlib) @@ -125,8 +125,8 @@ void WriteBufferFromHTTPServerResponse::nextImpl() response.set("Content-Encoding", "deflate"); response_body_ostr = &(response.send()); #endif - out_raw.emplace(*response_body_ostr); - deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin()); + out_raw = std::make_unique(*response_body_ostr); + deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin()); out = &*deflating_buf; } #if USE_BROTLI @@ -138,7 +138,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl() response.set("Content-Encoding", "br"); response_body_ostr = &(response.send()); #endif - out_raw.emplace(*response_body_ostr); + out_raw = std::make_unique(*response_body_ostr); brotli_buf.emplace(*out_raw, compression_level, working_buffer.size(), working_buffer.begin()); out = &*brotli_buf; } @@ -155,7 +155,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl() response_body_ostr = &(response.send()); #endif - out_raw.emplace(*response_body_ostr, working_buffer.size(), working_buffer.begin()); + out_raw = std::make_unique(*response_body_ostr, working_buffer.size(), working_buffer.begin()); out = &*out_raw; } } diff --git a/dbms/src/IO/WriteBufferFromHTTPServerResponse.h b/dbms/src/IO/WriteBufferFromHTTPServerResponse.h index 8a47cf0660d..fdbf6969aeb 100644 --- a/dbms/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/dbms/src/IO/WriteBufferFromHTTPServerResponse.h @@ -60,7 +60,7 @@ private: std::ostream * response_header_ostr = nullptr; #endif - std::optional out_raw; + std::unique_ptr out_raw; std::optional deflating_buf; #if USE_BROTLI std::optional brotli_buf; diff --git a/dbms/src/IO/ZlibDeflatingWriteBuffer.cpp b/dbms/src/IO/ZlibDeflatingWriteBuffer.cpp index dce103fd252..df2bb29c99f 100644 --- a/dbms/src/IO/ZlibDeflatingWriteBuffer.cpp +++ b/dbms/src/IO/ZlibDeflatingWriteBuffer.cpp @@ -6,14 +6,14 @@ namespace DB { ZlibDeflatingWriteBuffer::ZlibDeflatingWriteBuffer( - WriteBuffer & out_, + std::unique_ptr out_, CompressionMethod compression_method, int compression_level, size_t buf_size, char * existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment) - , out(out_) + , out(std::move(out_)) { zstr.zalloc = nullptr; zstr.zfree = nullptr; @@ -64,18 +64,18 @@ void ZlibDeflatingWriteBuffer::nextImpl() do { - out.nextIfAtEnd(); - zstr.next_out = reinterpret_cast(out.position()); - zstr.avail_out = out.buffer().end() - out.position(); + out->nextIfAtEnd(); + zstr.next_out = reinterpret_cast(out->position()); + zstr.avail_out = out->buffer().end() - out->position(); int rc = deflate(&zstr, Z_NO_FLUSH); - out.position() = out.buffer().end() - zstr.avail_out; + out->position() = out->buffer().end() - zstr.avail_out; // Unpoison the result of deflate explicitly. It uses some custom SSE algo // for computing CRC32, and it looks like msan is unable to comprehend // it fully, so it complains about the resulting value depending on the // uninitialized padding of the input buffer. - __msan_unpoison(out.position(), zstr.avail_out); + __msan_unpoison(out->position(), zstr.avail_out); if (rc != Z_OK) throw Exception(std::string("deflate failed: ") + zError(rc), ErrorCodes::ZLIB_DEFLATE_FAILED); @@ -92,18 +92,18 @@ void ZlibDeflatingWriteBuffer::finish() while (true) { - out.nextIfAtEnd(); - zstr.next_out = reinterpret_cast(out.position()); - zstr.avail_out = out.buffer().end() - out.position(); + out->nextIfAtEnd(); + zstr.next_out = reinterpret_cast(out->position()); + zstr.avail_out = out->buffer().end() - out->position(); int rc = deflate(&zstr, Z_FINISH); - out.position() = out.buffer().end() - zstr.avail_out; + out->position() = out->buffer().end() - zstr.avail_out; // Unpoison the result of deflate explicitly. It uses some custom SSE algo // for computing CRC32, and it looks like msan is unable to comprehend // it fully, so it complains about the resulting value depending on the // uninitialized padding of the input buffer. - __msan_unpoison(out.position(), zstr.avail_out); + __msan_unpoison(out->position(), zstr.avail_out); if (rc == Z_STREAM_END) { diff --git a/dbms/src/IO/ZlibDeflatingWriteBuffer.h b/dbms/src/IO/ZlibDeflatingWriteBuffer.h index 19f2770ef05..86eee1cffe5 100644 --- a/dbms/src/IO/ZlibDeflatingWriteBuffer.h +++ b/dbms/src/IO/ZlibDeflatingWriteBuffer.h @@ -20,7 +20,7 @@ class ZlibDeflatingWriteBuffer : public BufferWithOwnMemory { public: ZlibDeflatingWriteBuffer( - WriteBuffer & out_, + std::unique_ptr out_, CompressionMethod compression_method, int compression_level, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, @@ -37,7 +37,7 @@ public: private: void nextImpl() override; - WriteBuffer & out; + std::unique_ptr out; z_stream zstr; bool finished = false; }; diff --git a/dbms/src/IO/ZlibInflatingReadBuffer.cpp b/dbms/src/IO/ZlibInflatingReadBuffer.cpp index efa06678424..d3c0d37d17c 100644 --- a/dbms/src/IO/ZlibInflatingReadBuffer.cpp +++ b/dbms/src/IO/ZlibInflatingReadBuffer.cpp @@ -5,13 +5,13 @@ namespace DB { ZlibInflatingReadBuffer::ZlibInflatingReadBuffer( - ReadBuffer & in_, + std::unique_ptr in_, CompressionMethod compression_method, size_t buf_size, char * existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment) - , in(in_) + , in(std::move(in_)) , eof(false) { zstr.zalloc = nullptr; @@ -49,21 +49,21 @@ bool ZlibInflatingReadBuffer::nextImpl() if (!zstr.avail_in) { - in.nextIfAtEnd(); - zstr.next_in = reinterpret_cast(in.position()); - zstr.avail_in = in.buffer().end() - in.position(); + in->nextIfAtEnd(); + zstr.next_in = reinterpret_cast(in->position()); + zstr.avail_in = in->buffer().end() - in->position(); } zstr.next_out = reinterpret_cast(internal_buffer.begin()); zstr.avail_out = internal_buffer.size(); int rc = inflate(&zstr, Z_NO_FLUSH); - in.position() = in.buffer().end() - zstr.avail_in; + in->position() = in->buffer().end() - zstr.avail_in; working_buffer.resize(internal_buffer.size() - zstr.avail_out); if (rc == Z_STREAM_END) { - if (in.eof()) + if (in->eof()) { eof = true; return working_buffer.size() != 0; diff --git a/dbms/src/IO/ZlibInflatingReadBuffer.h b/dbms/src/IO/ZlibInflatingReadBuffer.h index 02ed443aa60..4ba152cdfdc 100644 --- a/dbms/src/IO/ZlibInflatingReadBuffer.h +++ b/dbms/src/IO/ZlibInflatingReadBuffer.h @@ -21,7 +21,7 @@ class ZlibInflatingReadBuffer : public BufferWithOwnMemory { public: ZlibInflatingReadBuffer( - ReadBuffer & in_, + std::unique_ptr in_, CompressionMethod compression_method, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, @@ -32,7 +32,7 @@ public: private: bool nextImpl() override; - ReadBuffer & in; + std::unique_ptr in; z_stream zstr; bool eof; }; diff --git a/dbms/src/IO/tests/zlib_buffers.cpp b/dbms/src/IO/tests/zlib_buffers.cpp index ff7aa8c5d26..3c7af125a5a 100644 --- a/dbms/src/IO/tests/zlib_buffers.cpp +++ b/dbms/src/IO/tests/zlib_buffers.cpp @@ -22,8 +22,8 @@ try Stopwatch stopwatch; { - DB::WriteBufferFromFile buf("test_zlib_buffers.gz", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); - DB::ZlibDeflatingWriteBuffer deflating_buf(buf, DB::CompressionMethod::Gzip, /* compression_level = */ 3); + auto buf = std::make_unique("test_zlib_buffers.gz", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); + DB::ZlibDeflatingWriteBuffer deflating_buf(std::move(buf), DB::CompressionMethod::Gzip, /* compression_level = */ 3); stopwatch.restart(); for (size_t i = 0; i < n; ++i) @@ -40,8 +40,8 @@ try } { - DB::ReadBufferFromFile buf("test_zlib_buffers.gz"); - DB::ZlibInflatingReadBuffer inflating_buf(buf, DB::CompressionMethod::Gzip); + auto buf = std::make_unique("test_zlib_buffers.gz"); + DB::ZlibInflatingReadBuffer inflating_buf(std::move(buf), DB::CompressionMethod::Gzip); stopwatch.restart(); for (size_t i = 0; i < n; ++i) diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 7a2960a1335..f5c34587fb2 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -445,4 +446,21 @@ BlockInputStreams IStorage::read( return res; } +DB::CompressionMethod IStorage::chooseCompressionMethod(const String & uri, const String & compression_method) +{ + if (compression_method == "auto" || compression_method == "") + { + if (endsWith(uri, ".gz")) + return DB::CompressionMethod::Gzip; + else + return DB::CompressionMethod::None; + } + else if (compression_method == "gzip") + return DB::CompressionMethod::Gzip; + else if (compression_method == "none") + return DB::CompressionMethod::None; + else + throw Exception("Only auto, none, gzip supported as compression method", ErrorCodes::NOT_IMPLEMENTED); +} + } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 684e20efcd5..1ca5860fda1 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -434,6 +435,8 @@ public: return {}; } + static DB::CompressionMethod chooseCompressionMethod(const String & uri, const String & compression_method); + private: /// You always need to take the next three locks in this order. diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 7258cf1c518..23b12cb7efb 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -127,9 +127,10 @@ StorageFile::StorageFile( const std::string & format_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - Context & context_) + Context & context_, + const String & compression_method_ = "") : - table_name(table_name_), database_name(database_name_), format_name(format_name_), context_global(context_), table_fd(table_fd_) + table_name(table_name_), database_name(database_name_), format_name(format_name_), context_global(context_), table_fd(table_fd_), compression_method(compression_method_) { setColumns(columns_); setConstraints(constraints_); @@ -178,7 +179,10 @@ StorageFile::StorageFile( class StorageFileBlockInputStream : public IBlockInputStream { public: - StorageFileBlockInputStream(std::shared_ptr storage_, const Context & context, UInt64 max_block_size, std::string file_path) + StorageFileBlockInputStream(std::shared_ptr storage_, + const Context & context, UInt64 max_block_size, + std::string file_path, + const CompressionMethod compression_method) : storage(std::move(storage_)) { if (storage->use_table_fd) @@ -199,12 +203,12 @@ public: } storage->table_fd_was_used = true; - read_buf = std::make_unique(storage->table_fd); + read_buf = getBuffer(compression_method, storage->table_fd); } else { shared_lock = std::shared_lock(storage->rwlock); - read_buf = std::make_unique(file_path); + read_buf = getBuffer(compression_method, file_path); } reader = FormatFactory::instance().getInput(storage->format_name, *read_buf, storage->getSampleBlock(), context, max_block_size); @@ -235,7 +239,7 @@ public: private: std::shared_ptr storage; Block sample_block; - std::unique_ptr read_buf; + std::unique_ptr read_buf; BlockInputStreamPtr reader; std::shared_lock shared_lock; @@ -260,7 +264,7 @@ BlockInputStreams StorageFile::read( for (const auto & file_path : paths) { BlockInputStreamPtr cur_block = std::make_shared( - std::static_pointer_cast(shared_from_this()), context, max_block_size, file_path); + std::static_pointer_cast(shared_from_this()), context, max_block_size, file_path, IStorage::chooseCompressionMethod(file_path, compression_method)); blocks_input.push_back(column_defaults.empty() ? cur_block : std::make_shared(cur_block, column_defaults, context)); } return blocks_input; @@ -270,7 +274,8 @@ BlockInputStreams StorageFile::read( class StorageFileBlockOutputStream : public IBlockOutputStream { public: - explicit StorageFileBlockOutputStream(StorageFile & storage_) + explicit StorageFileBlockOutputStream(StorageFile & storage_, + const CompressionMethod compression_method) : storage(storage_), lock(storage.rwlock) { if (storage.use_table_fd) @@ -280,13 +285,13 @@ public: * INSERT data; SELECT *; last SELECT returns only insert_data */ storage.table_fd_was_used = true; - write_buf = std::make_unique(storage.table_fd); + write_buf = getBuffer(compression_method, storage.table_fd); } else { if (storage.paths.size() != 1) throw Exception("Table '" + storage.table_name + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED); - write_buf = std::make_unique(storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); + write_buf = getBuffer(compression_method, storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); } writer = FormatFactory::instance().getOutput(storage.format_name, *write_buf, storage.getSampleBlock(), storage.context_global); @@ -317,7 +322,7 @@ public: private: StorageFile & storage; std::unique_lock lock; - std::unique_ptr write_buf; + std::unique_ptr write_buf; BlockOutputStreamPtr writer; }; @@ -325,7 +330,8 @@ BlockOutputStreamPtr StorageFile::write( const ASTPtr & /*query*/, const Context & /*context*/) { - return std::make_shared(*this); + return std::make_shared(*this, + IStorage::chooseCompressionMethod(paths[0], compression_method)); } Strings StorageFile::getDataPaths() const @@ -361,9 +367,9 @@ void registerStorageFile(StorageFactory & factory) { ASTs & engine_args = args.engine_args; - if (!(engine_args.size() == 1 || engine_args.size() == 2)) + if (!(engine_args.size() >= 1 && engine_args.size() <= 3)) throw Exception( - "Storage File requires 1 or 2 arguments: name of used format and source.", + "Storage File requires from 1 to 3 arguments: name of used format, source and compression_method.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); @@ -371,6 +377,7 @@ void registerStorageFile(StorageFactory & factory) int source_fd = -1; String source_path; + String compression_method; if (engine_args.size() >= 2) { /// Will use FD if engine_args[1] is int literal or identifier with std* name @@ -397,13 +404,19 @@ void registerStorageFile(StorageFactory & factory) else if (type == Field::Types::String) source_path = literal->value.get(); } + if (engine_args.size() == 3) + { + engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context); + compression_method = engine_args[2]->as().value.safeGet(); + } else compression_method = "auto"; } return StorageFile::create( source_path, source_fd, args.data_path, args.database_name, args.table_name, format_name, args.columns, args.constraints, - args.context); + args.context, + compression_method); }); } diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index 0d9854f75cf..f37c42ae59f 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -60,7 +60,8 @@ protected: const std::string & format_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - Context & context_); + Context & context_, + const String & compression_method_); private: std::string table_name; @@ -69,6 +70,7 @@ private: Context & context_global; int table_fd = -1; + String compression_method; std::vector paths; diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 4b68bed871c..58b59dbeb16 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include #include @@ -36,12 +35,14 @@ StorageHDFS::StorageHDFS(const String & uri_, const String & format_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - Context & context_) + Context & context_, + const String & compression_method_ = "") : uri(uri_) , format_name(format_name_) , table_name(table_name_) , database_name(database_name_) , context(context_) + , compression_method(compression_method_) { setColumns(columns_); setConstraints(constraints_); @@ -57,9 +58,11 @@ public: const String & format, const Block & sample_block, const Context & context, - UInt64 max_block_size) + UInt64 max_block_size, + const CompressionMethod compression_method) { - std::unique_ptr read_buf = std::make_unique(uri); + auto read_buf = getBuffer(compression_method, uri); + auto input_stream = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); reader = std::make_shared>(input_stream, std::move(read_buf)); } @@ -99,10 +102,11 @@ public: HDFSBlockOutputStream(const String & uri, const String & format, const Block & sample_block_, - const Context & context) + const Context & context, + const CompressionMethod compression_method) : sample_block(sample_block_) { - write_buf = std::make_unique(uri); + write_buf = getBuffer(compression_method, uri); writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); } @@ -130,7 +134,7 @@ public: private: Block sample_block; - std::unique_ptr write_buf; + std::unique_ptr write_buf; BlockOutputStreamPtr writer; }; @@ -203,7 +207,7 @@ BlockInputStreams StorageHDFS::read( for (const auto & res_path : res_paths) { result.push_back(std::make_shared(uri_without_path + res_path, format_name, getSampleBlock(), context_, - max_block_size)); + max_block_size, IStorage::chooseCompressionMethod(res_path, compression_method))); } return result; @@ -217,7 +221,11 @@ void StorageHDFS::rename(const String & /*new_path_to_db*/, const String & new_d BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const Context & /*context*/) { - return std::make_shared(uri, format_name, getSampleBlock(), context); + return std::make_shared(uri, + format_name, + getSampleBlock(), + context, + IStorage::chooseCompressionMethod(uri, compression_method)); } void registerStorageHDFS(StorageFactory & factory) @@ -226,9 +234,9 @@ void registerStorageHDFS(StorageFactory & factory) { ASTs & engine_args = args.engine_args; - if (engine_args.size() != 2) + if (engine_args.size() != 2 && engine_args.size() != 3) throw Exception( - "Storage HDFS requires exactly 2 arguments: url and name of used format.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + "Storage HDFS requires 2 or 3 arguments: url, name of used format and optional compression method.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); @@ -238,7 +246,14 @@ void registerStorageHDFS(StorageFactory & factory) String format_name = engine_args[1]->as().value.safeGet(); - return StorageHDFS::create(url, args.database_name, args.table_name, format_name, args.columns, args.constraints, args.context); + String compression_method; + if (engine_args.size() == 3) + { + engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context); + compression_method = engine_args[2]->as().value.safeGet(); + } else compression_method = "auto"; + + return StorageHDFS::create(url, args.database_name, args.table_name, format_name, args.columns, args.constraints, args.context, compression_method); }); } diff --git a/dbms/src/Storages/StorageHDFS.h b/dbms/src/Storages/StorageHDFS.h index 8361916e0e2..5c02793f781 100644 --- a/dbms/src/Storages/StorageHDFS.h +++ b/dbms/src/Storages/StorageHDFS.h @@ -39,7 +39,8 @@ protected: const String & format_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - Context & context_); + Context & context_, + const String & compression_method_); private: String uri; @@ -47,6 +48,7 @@ private: String table_name; String database_name; Context & context; + String compression_method; Logger * log = &Logger::get("StorageHDFS"); }; diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp index 488a38c3e83..f574ad5130d 100644 --- a/dbms/src/Storages/StorageS3.cpp +++ b/dbms/src/Storages/StorageS3.cpp @@ -35,11 +35,11 @@ namespace const Block & sample_block, const Context & context, UInt64 max_block_size, - const ConnectionTimeouts & timeouts) + const ConnectionTimeouts & timeouts, + const CompressionMethod compression_method) : name(name_) { - read_buf = std::make_unique(uri, timeouts); - + read_buf = getBuffer(compression_method, uri, timeouts); reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); } @@ -70,7 +70,7 @@ namespace private: String name; - std::unique_ptr read_buf; + std::unique_ptr read_buf; BlockInputStreamPtr reader; }; @@ -82,10 +82,11 @@ namespace UInt64 min_upload_part_size, const Block & sample_block_, const Context & context, - const ConnectionTimeouts & timeouts) + const ConnectionTimeouts & timeouts, + const CompressionMethod compression_method) : sample_block(sample_block_) { - write_buf = std::make_unique(uri, min_upload_part_size, timeouts); + write_buf = getBuffer(compression_method, uri, min_upload_part_size, timeouts); writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); } @@ -113,7 +114,7 @@ namespace private: Block sample_block; - std::unique_ptr write_buf; + std::unique_ptr write_buf; BlockOutputStreamPtr writer; }; } @@ -127,7 +128,8 @@ StorageS3::StorageS3( UInt64 min_upload_part_size_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - Context & context_) + Context & context_, + const String & compression_method_ = "") : IStorage(columns_) , uri(uri_) , context_global(context_) @@ -135,6 +137,7 @@ StorageS3::StorageS3( , database_name(database_name_) , table_name(table_name_) , min_upload_part_size(min_upload_part_size_) + , compression_method(compression_method_) { setColumns(columns_); setConstraints(constraints_); @@ -156,7 +159,8 @@ BlockInputStreams StorageS3::read( getHeaderBlock(column_names), context, max_block_size, - ConnectionTimeouts::getHTTPTimeouts(context)); + ConnectionTimeouts::getHTTPTimeouts(context), + IStorage::chooseCompressionMethod(uri.toString(), compression_method)); auto column_defaults = getColumns().getDefaults(); if (column_defaults.empty()) @@ -173,7 +177,9 @@ void StorageS3::rename(const String & /*new_path_to_db*/, const String & new_dat BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const Context & /*context*/) { return std::make_shared( - uri, format_name, min_upload_part_size, getSampleBlock(), context_global, ConnectionTimeouts::getHTTPTimeouts(context_global)); + uri, format_name, min_upload_part_size, getSampleBlock(), context_global, + ConnectionTimeouts::getHTTPTimeouts(context_global), + IStorage::chooseCompressionMethod(uri.toString(), compression_method)); } void registerStorageS3(StorageFactory & factory) @@ -182,9 +188,9 @@ void registerStorageS3(StorageFactory & factory) { ASTs & engine_args = args.engine_args; - if (engine_args.size() != 2) + if (engine_args.size() != 2 && engine_args.size() != 3) throw Exception( - "Storage S3 requires exactly 2 arguments: url and name of used format.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + "Storage S3 requires 2 or 3 arguments: url, name of used format and compression_method.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); @@ -197,6 +203,13 @@ void registerStorageS3(StorageFactory & factory) UInt64 min_upload_part_size = args.local_context.getSettingsRef().s3_min_upload_part_size; + String compression_method; + if (engine_args.size() == 3) + { + engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context); + compression_method = engine_args[2]->as().value.safeGet(); + } else compression_method = "auto"; + return StorageS3::create(uri, args.database_name, args.table_name, format_name, min_upload_part_size, args.columns, args.constraints, args.context); }); } diff --git a/dbms/src/Storages/StorageS3.h b/dbms/src/Storages/StorageS3.h index 65cd65458c6..88b470ac2ac 100644 --- a/dbms/src/Storages/StorageS3.h +++ b/dbms/src/Storages/StorageS3.h @@ -24,7 +24,8 @@ public: UInt64 min_upload_part_size_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - Context & context_); + Context & context_, + const String & compression_method_); String getName() const override { @@ -61,6 +62,7 @@ private: String database_name; String table_name; UInt64 min_upload_part_size; + String compression_method; }; } diff --git a/dbms/src/Storages/StorageURL.cpp b/dbms/src/Storages/StorageURL.cpp index 70c401ba417..bff4d115ead 100644 --- a/dbms/src/Storages/StorageURL.cpp +++ b/dbms/src/Storages/StorageURL.cpp @@ -31,8 +31,9 @@ IStorageURLBase::IStorageURLBase( const std::string & table_name_, const String & format_name_, const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_) - : uri(uri_), context_global(context_), format_name(format_name_), table_name(table_name_), database_name(database_name_) + const ConstraintsDescription & constraints_, + const String & compression_method_) + : uri(uri_), context_global(context_), compression_method(compression_method_), format_name(format_name_), table_name(table_name_), database_name(database_name_) { setColumns(columns_); setConstraints(constraints_); @@ -51,10 +52,11 @@ namespace const Block & sample_block, const Context & context, UInt64 max_block_size, - const ConnectionTimeouts & timeouts) + const ConnectionTimeouts & timeouts, + const CompressionMethod compression_method) : name(name_) { - read_buf = std::make_unique(uri, method, callback, timeouts, context.getSettingsRef().max_http_get_redirects); + read_buf = getBuffer(compression_method, uri, method, callback, timeouts, context.getSettingsRef().max_http_get_redirects); reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); } @@ -85,7 +87,7 @@ namespace private: String name; - std::unique_ptr read_buf; + std::unique_ptr read_buf; BlockInputStreamPtr reader; }; @@ -96,10 +98,11 @@ namespace const String & format, const Block & sample_block_, const Context & context, - const ConnectionTimeouts & timeouts) + const ConnectionTimeouts & timeouts, + const CompressionMethod compression_method) : sample_block(sample_block_) { - write_buf = std::make_unique(uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts); + write_buf = getBuffer(compression_method, uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts); writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); } @@ -127,7 +130,7 @@ namespace private: Block sample_block; - std::unique_ptr write_buf; + std::unique_ptr write_buf; BlockOutputStreamPtr writer; }; } @@ -177,8 +180,8 @@ BlockInputStreams IStorageURLBase::read(const Names & column_names, getHeaderBlock(column_names), context, max_block_size, - ConnectionTimeouts::getHTTPTimeouts(context)); - + ConnectionTimeouts::getHTTPTimeouts(context), + IStorage::chooseCompressionMethod(request_uri.toString(), compression_method)); auto column_defaults = getColumns().getDefaults(); if (column_defaults.empty()) @@ -195,7 +198,9 @@ void IStorageURLBase::rename(const String & /*new_path_to_db*/, const String & n BlockOutputStreamPtr IStorageURLBase::write(const ASTPtr & /*query*/, const Context & /*context*/) { return std::make_shared( - uri, format_name, getSampleBlock(), context_global, ConnectionTimeouts::getHTTPTimeouts(context_global)); + uri, format_name, getSampleBlock(), context_global, + ConnectionTimeouts::getHTTPTimeouts(context_global), + IStorage::chooseCompressionMethod(uri.toString(), compression_method)); } void registerStorageURL(StorageFactory & factory) @@ -204,9 +209,9 @@ void registerStorageURL(StorageFactory & factory) { ASTs & engine_args = args.engine_args; - if (engine_args.size() != 2) + if (engine_args.size() != 2 && engine_args.size() != 3) throw Exception( - "Storage URL requires exactly 2 arguments: url and name of used format.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + "Storage URL requires 2 or 3 arguments: url, name of used format and optional compression method.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); @@ -217,7 +222,19 @@ void registerStorageURL(StorageFactory & factory) String format_name = engine_args[1]->as().value.safeGet(); - return StorageURL::create(uri, args.database_name, args.table_name, format_name, args.columns, args.constraints, args.context); + String compression_method; + if (engine_args.size() == 3) + { + engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context); + compression_method = engine_args[2]->as().value.safeGet(); + } else compression_method = "auto"; + + return StorageURL::create( + uri, + args.database_name, args.table_name, + format_name, + args.columns, args.constraints, args.context, + compression_method); }); } } diff --git a/dbms/src/Storages/StorageURL.h b/dbms/src/Storages/StorageURL.h index cdd78c7b60f..a518aed71f7 100644 --- a/dbms/src/Storages/StorageURL.h +++ b/dbms/src/Storages/StorageURL.h @@ -39,10 +39,12 @@ protected: const std::string & table_name_, const String & format_name_, const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_); + const ConstraintsDescription & constraints_, + const String & compression_method_); Poco::URI uri; const Context & context_global; + String compression_method; private: String format_name; @@ -80,8 +82,9 @@ public: const String & format_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - Context & context_) - : IStorageURLBase(uri_, context_, database_name_, table_name_, format_name_, columns_, constraints_) + Context & context_, + const String & compression_method_) + : IStorageURLBase(uri_, context_, database_name_, table_name_, format_name_, columns_, constraints_, compression_method_) { } diff --git a/dbms/src/Storages/StorageXDBC.cpp b/dbms/src/Storages/StorageXDBC.cpp index fc9bb776da0..bab751e4f36 100644 --- a/dbms/src/Storages/StorageXDBC.cpp +++ b/dbms/src/Storages/StorageXDBC.cpp @@ -7,6 +7,7 @@ #include #include +#include #include #include #include @@ -31,7 +32,7 @@ StorageXDBC::StorageXDBC( const Context & context_, const BridgeHelperPtr bridge_helper_) /// Please add support for constraints as soon as StorageODBC or JDBC will support insertion. - : IStorageURLBase(Poco::URI(), context_, database_name_, table_name_, IXDBCBridgeHelper::DEFAULT_FORMAT, columns_, ConstraintsDescription{}) + : IStorageURLBase(Poco::URI(), context_, database_name_, table_name_, IXDBCBridgeHelper::DEFAULT_FORMAT, columns_, ConstraintsDescription{}, "" /* CompressionMethod */) , bridge_helper(bridge_helper_) , remote_database_name(remote_database_name_) , remote_table_name(remote_table_name_) diff --git a/dbms/src/TableFunctions/ITableFunctionFileLike.cpp b/dbms/src/TableFunctions/ITableFunctionFileLike.cpp index 51c95d3a1be..44f7e02dd4b 100644 --- a/dbms/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/dbms/src/TableFunctions/ITableFunctionFileLike.cpp @@ -32,21 +32,27 @@ StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, cons ASTs & args = args_func.at(0)->children; - if (args.size() != 3) - throw Exception("Table function '" + getName() + "' requires exactly 3 arguments: filename, format and structure.", + if (args.size() != 3 && args.size() != 4) + throw Exception("Table function '" + getName() + "' requires 3 or 4 arguments: filename, format, structure and compression method (default none).", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - for (size_t i = 0; i < 3; ++i) + for (size_t i = 0; i < args.size(); ++i) args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(args[i], context); std::string filename = args[0]->as().value.safeGet(); std::string format = args[1]->as().value.safeGet(); std::string structure = args[2]->as().value.safeGet(); + std::string compression_method; + + if (args.size() == 4) + { + compression_method = args[3]->as().value.safeGet(); + } else compression_method = "auto"; ColumnsDescription columns = parseColumnsListFromString(structure, context); /// Create table - StoragePtr storage = getStorage(filename, format, columns, const_cast(context), table_name); + StoragePtr storage = getStorage(filename, format, columns, const_cast(context), table_name, compression_method); storage->startup(); diff --git a/dbms/src/TableFunctions/ITableFunctionFileLike.h b/dbms/src/TableFunctions/ITableFunctionFileLike.h index 1e4febc935b..df0338231ba 100644 --- a/dbms/src/TableFunctions/ITableFunctionFileLike.h +++ b/dbms/src/TableFunctions/ITableFunctionFileLike.h @@ -16,6 +16,6 @@ class ITableFunctionFileLike : public ITableFunction private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; virtual StoragePtr getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const = 0; + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const = 0; }; } diff --git a/dbms/src/TableFunctions/TableFunctionFile.cpp b/dbms/src/TableFunctions/TableFunctionFile.cpp index 7cf2c500f1e..1adea8d60ff 100644 --- a/dbms/src/TableFunctions/TableFunctionFile.cpp +++ b/dbms/src/TableFunctions/TableFunctionFile.cpp @@ -6,7 +6,7 @@ namespace DB { StoragePtr TableFunctionFile::getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method) const { return StorageFile::create(source, -1, @@ -16,7 +16,8 @@ StoragePtr TableFunctionFile::getStorage( format, columns, ConstraintsDescription{}, - global_context); + global_context, + compression_method); } void registerTableFunctionFile(TableFunctionFactory & factory) diff --git a/dbms/src/TableFunctions/TableFunctionFile.h b/dbms/src/TableFunctions/TableFunctionFile.h index d5e54c1113f..8d3e5593db0 100644 --- a/dbms/src/TableFunctions/TableFunctionFile.h +++ b/dbms/src/TableFunctions/TableFunctionFile.h @@ -23,6 +23,6 @@ public: private: StoragePtr getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const override; + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method) const override; }; } diff --git a/dbms/src/TableFunctions/TableFunctionHDFS.cpp b/dbms/src/TableFunctions/TableFunctionHDFS.cpp index 4ff999d31dd..3e8a3d6b954 100644 --- a/dbms/src/TableFunctions/TableFunctionHDFS.cpp +++ b/dbms/src/TableFunctions/TableFunctionHDFS.cpp @@ -9,7 +9,7 @@ namespace DB { StoragePtr TableFunctionHDFS::getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const { return StorageHDFS::create(source, getDatabaseName(), @@ -17,7 +17,8 @@ StoragePtr TableFunctionHDFS::getStorage( format, columns, ConstraintsDescription{}, - global_context); + global_context, + compression_method); } void registerTableFunctionHDFS(TableFunctionFactory & factory) diff --git a/dbms/src/TableFunctions/TableFunctionHDFS.h b/dbms/src/TableFunctions/TableFunctionHDFS.h index ffe7eb58a10..417c1121955 100644 --- a/dbms/src/TableFunctions/TableFunctionHDFS.h +++ b/dbms/src/TableFunctions/TableFunctionHDFS.h @@ -24,7 +24,7 @@ public: private: StoragePtr getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const override; + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const override; }; } diff --git a/dbms/src/TableFunctions/TableFunctionS3.cpp b/dbms/src/TableFunctions/TableFunctionS3.cpp index 849836b0498..a9ee5ebf691 100644 --- a/dbms/src/TableFunctions/TableFunctionS3.cpp +++ b/dbms/src/TableFunctions/TableFunctionS3.cpp @@ -7,11 +7,11 @@ namespace DB { StoragePtr TableFunctionS3::getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const { Poco::URI uri(source); UInt64 min_upload_part_size = global_context.getSettingsRef().s3_min_upload_part_size; - return StorageS3::create(uri, getDatabaseName(), table_name, format, min_upload_part_size, columns, ConstraintsDescription{}, global_context); + return StorageS3::create(uri, getDatabaseName(), table_name, format, min_upload_part_size, columns, ConstraintsDescription{}, global_context, compression_method); } void registerTableFunctionS3(TableFunctionFactory & factory) diff --git a/dbms/src/TableFunctions/TableFunctionS3.h b/dbms/src/TableFunctions/TableFunctionS3.h index ecb9ea03197..2f14e0319d4 100644 --- a/dbms/src/TableFunctions/TableFunctionS3.h +++ b/dbms/src/TableFunctions/TableFunctionS3.h @@ -25,7 +25,8 @@ private: const String & format, const ColumnsDescription & columns, Context & global_context, - const std::string & table_name) const override; + const std::string & table_name, + const String & compression_method) const override; }; } diff --git a/dbms/src/TableFunctions/TableFunctionURL.cpp b/dbms/src/TableFunctions/TableFunctionURL.cpp index 1f89264a422..adb930efa8c 100644 --- a/dbms/src/TableFunctions/TableFunctionURL.cpp +++ b/dbms/src/TableFunctions/TableFunctionURL.cpp @@ -8,10 +8,10 @@ namespace DB { StoragePtr TableFunctionURL::getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const { Poco::URI uri(source); - return StorageURL::create(uri, getDatabaseName(), table_name, format, columns, ConstraintsDescription{}, global_context); + return StorageURL::create(uri, getDatabaseName(), table_name, format, columns, ConstraintsDescription{}, global_context, compression_method); } void registerTableFunctionURL(TableFunctionFactory & factory) diff --git a/dbms/src/TableFunctions/TableFunctionURL.h b/dbms/src/TableFunctions/TableFunctionURL.h index fefd3ec072c..0816bed4bab 100644 --- a/dbms/src/TableFunctions/TableFunctionURL.h +++ b/dbms/src/TableFunctions/TableFunctionURL.h @@ -19,6 +19,6 @@ public: private: StoragePtr getStorage( - const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const override; + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const override; }; } diff --git a/dbms/tests/integration/helpers/hdfs_api.py b/dbms/tests/integration/helpers/hdfs_api.py index 989d66ee1e3..97df8a13aeb 100644 --- a/dbms/tests/integration/helpers/hdfs_api.py +++ b/dbms/tests/integration/helpers/hdfs_api.py @@ -1,4 +1,6 @@ #-*- coding: utf-8 -*- +import StringIO +import gzip import requests import subprocess from tempfile import NamedTemporaryFile @@ -19,7 +21,7 @@ class HDFSApi(object): if response_data.status_code != 200: response_data.raise_for_status() - return response_data.text + return response_data.content # Requests can't put file def _curl_to_put(self, filename, path, params): @@ -44,3 +46,12 @@ class HDFSApi(object): output = self._curl_to_put(fpath, path, additional_params) if "201 Created" not in output: raise Exception("Can't create file on hdfs:\n {}".format(output)) + + def write_gzip_data(self, path, content): + out = StringIO.StringIO() + with gzip.GzipFile(fileobj=out, mode="w") as f: + f.write(content) + self.write_data(path, out.getvalue()) + + def read_gzip_data(self, path): + return gzip.GzipFile(fileobj=StringIO.StringIO(self.read_data(path))).read() diff --git a/dbms/tests/integration/test_storage_hdfs/test.py b/dbms/tests/integration/test_storage_hdfs/test.py index cf4205115ff..575b7593ca0 100644 --- a/dbms/tests/integration/test_storage_hdfs/test.py +++ b/dbms/tests/integration/test_storage_hdfs/test.py @@ -133,4 +133,56 @@ def test_globs_in_read_table(started_cluster): ("?", 0)] for pattern, value in test_requests: - assert node1.query("select * from hdfs('hdfs://hdfs1:9000" + globs_dir + pattern + "', 'TSV', 'id UInt64, text String, number Float64')") == value * some_data \ No newline at end of file + assert node1.query("select * from hdfs('hdfs://hdfs1:9000" + globs_dir + pattern + "', 'TSV', 'id UInt64, text String, number Float64')") == value * some_data + +def test_read_write_gzip_table(started_cluster): + hdfs_api = HDFSApi("root") + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_gzip_data("/simple_table_function.gz", data) + + assert hdfs_api.read_gzip_data("/simple_table_function.gz") == data + + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64')") == data + +def test_read_write_gzip_table_with_parameter_gzip(started_cluster): + hdfs_api = HDFSApi("root") + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_gzip_data("/simple_table_function", data) + + assert hdfs_api.read_gzip_data("/simple_table_function") == data + + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64', 'gzip')") == data + +def test_read_write_table_with_parameter_none(started_cluster): + hdfs_api = HDFSApi("root") + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_data("/simple_table_function.gz", data) + + assert hdfs_api.read_data("/simple_table_function.gz") == data + + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64', 'none')") == data + +def test_read_write_gzip_table_with_parameter_auto_gz(started_cluster): + hdfs_api = HDFSApi("root") + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_gzip_data("/simple_table_function.gz", data) + + assert hdfs_api.read_gzip_data("/simple_table_function.gz") == data + + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64', 'auto')") == data + +def test_write_gz_storage(started_cluster): + hdfs_api = HDFSApi("root") + + node1.query("create table GZHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage.gz', 'TSV')") + node1.query("insert into GZHDFSStorage values (1, 'Mark', 72.53)") + assert hdfs_api.read_gzip_data("/storage.gz") == "1\tMark\t72.53\n" + assert node1.query("select * from GZHDFSStorage") == "1\tMark\t72.53\n" + +def test_write_gzip_storage(started_cluster): + hdfs_api = HDFSApi("root") + + node1.query("create table GZIPHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/gzip_storage', 'TSV', 'gzip')") + node1.query("insert into GZIPHDFSStorage values (1, 'Mark', 72.53)") + assert hdfs_api.read_gzip_data("/gzip_storage") == "1\tMark\t72.53\n" + assert node1.query("select * from GZIPHDFSStorage") == "1\tMark\t72.53\n" diff --git a/dbms/tests/queries/0_stateless/01030_storage_hdfs_syntax.sql b/dbms/tests/queries/0_stateless/01030_storage_hdfs_syntax.sql index 9b16141338c..807889a935a 100644 --- a/dbms/tests/queries/0_stateless/01030_storage_hdfs_syntax.sql +++ b/dbms/tests/queries/0_stateless/01030_storage_hdfs_syntax.sql @@ -2,7 +2,7 @@ drop table if exists test_table_hdfs_syntax ; create table test_table_hdfs_syntax (id UInt32) ENGINE = HDFS('') ; -- { serverError 42 } -create table test_table_hdfs_syntax (id UInt32) ENGINE = HDFS('','','') +create table test_table_hdfs_syntax (id UInt32) ENGINE = HDFS('','','', '') ; -- { serverError 42 } drop table if exists test_table_hdfs_syntax ; diff --git a/dbms/tests/queries/0_stateless/01030_storage_s3_syntax.sql b/dbms/tests/queries/0_stateless/01030_storage_s3_syntax.sql index 504b5375b38..6579984f57d 100644 --- a/dbms/tests/queries/0_stateless/01030_storage_s3_syntax.sql +++ b/dbms/tests/queries/0_stateless/01030_storage_s3_syntax.sql @@ -2,7 +2,7 @@ drop table if exists test_table_s3_syntax ; create table test_table_s3_syntax (id UInt32) ENGINE = S3('') ; -- { serverError 42 } -create table test_table_s3_syntax (id UInt32) ENGINE = S3('','','') +create table test_table_s3_syntax (id UInt32) ENGINE = S3('','','','') ; -- { serverError 42 } drop table if exists test_table_s3_syntax ; diff --git a/dbms/tests/queries/0_stateless/01030_storage_url_syntax.sql b/dbms/tests/queries/0_stateless/01030_storage_url_syntax.sql index 11c4b01f1ca..0efb121eda5 100644 --- a/dbms/tests/queries/0_stateless/01030_storage_url_syntax.sql +++ b/dbms/tests/queries/0_stateless/01030_storage_url_syntax.sql @@ -2,7 +2,7 @@ drop table if exists test_table_url_syntax ; create table test_table_url_syntax (id UInt32) ENGINE = URL('') ; -- { serverError 42 } -create table test_table_url_syntax (id UInt32) ENGINE = URL('','','') +create table test_table_url_syntax (id UInt32) ENGINE = URL('','','','') ; -- { serverError 42 } drop table if exists test_table_url_syntax ; From 0d1933cb44d1ec2c00ab80b8f12479cb9f0396b7 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 19 Nov 2019 16:11:04 +0300 Subject: [PATCH 55/90] Calculate the number of parsing theads correctly --- .../ParallelParsingBlockInputStream.h | 37 +++++++++++-------- dbms/src/Formats/FormatFactory.cpp | 15 ++++++-- 2 files changed, 33 insertions(+), 19 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 9cf194b7408..4b5e091cfc9 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -67,30 +67,37 @@ public: const FormatSettings &settings; }; - struct Builder + struct Params { ReadBuffer & read_buffer; const InputProcessorCreator &input_processor_creator; const InputCreatorParams &input_creator_params; FormatFactory::FileSegmentationEngine file_segmentation_engine; - size_t max_threads_to_use; + int max_threads; size_t min_chunk_bytes; }; - 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), - min_chunk_bytes(builder.min_chunk_bytes), - original_buffer(builder.read_buffer), - pool(builder.max_threads_to_use), - file_segmentation_engine(builder.file_segmentation_engine) + explicit ParallelParsingBlockInputStream(const Params & params) + : header(params.input_creator_params.sample), + context(params.input_creator_params.context), + row_input_format_params(params.input_creator_params.row_input_format_params), + format_settings(params.input_creator_params.settings), + input_processor_creator(params.input_processor_creator), + min_chunk_bytes(params.min_chunk_bytes), + original_buffer(params.read_buffer), + // Subtract one thread that we use for segmentation and one for + // reading. After that, must have at least two threads left for + // parsing. See the assertion below. + pool(std::max(2, params.max_threads - 2)), + file_segmentation_engine(params.file_segmentation_engine) { - // Allocate more units than threads to decrease segmentator - // waiting on reader on wraparound. The number is random. - processing_units.resize(builder.max_threads_to_use + 4); + // See comment above. + assert(params.max_threads >= 4); + + // One unit for each thread, including segmentator and reader, plus a + // couple more units so that the segmentation thread doesn't spuriously + // bump into reader thread on wraparound. + processing_units.resize(params.max_threads + 2); segmentator_thread = ThreadFromGlobalPool([this] { segmentatorThreadFunction(); }); } diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index c090c29eaad..bd2544d7afd 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -108,7 +108,11 @@ BlockInputStreamPtr FormatFactory::getInput( const Settings & settings = context.getSettingsRef(); const auto & file_segmentation_engine = getCreators(name).file_segmentation_engine; - if (settings.input_format_parallel_parsing && file_segmentation_engine) + // Doesn't make sense to use parallel parsing with less than four threads + // (segmentator + two parsers + reader). + if (settings.input_format_parallel_parsing + && file_segmentation_engine + && settings.max_threads >= 4) { const auto & input_getter = getCreators(name).input_processor_creator; if (!input_getter) @@ -124,9 +128,12 @@ BlockInputStreamPtr FormatFactory::getInput( row_input_format_params.max_execution_time = settings.max_execution_time; row_input_format_params.timeout_overflow_mode = settings.timeout_overflow_mode; - auto params = ParallelParsingBlockInputStream::InputCreatorParams{sample, context, row_input_format_params, format_settings}; - ParallelParsingBlockInputStream::Builder builder{buf, input_getter, params, file_segmentation_engine, settings.max_threads, settings.min_chunk_bytes_for_parallel_parsing}; - return std::make_shared(builder); + auto input_creator_params = ParallelParsingBlockInputStream::InputCreatorParams{sample, context, row_input_format_params, format_settings}; + ParallelParsingBlockInputStream::Params params{buf, input_getter, + input_creator_params, file_segmentation_engine, + static_cast(settings.max_threads), + settings.min_chunk_bytes_for_parallel_parsing}; + return std::make_shared(params); } auto format = getInputFormat(name, buf, sample, context, max_block_size, std::move(callback)); From a2b598917779142e49b3efd41edeb571cfcb0876 Mon Sep 17 00:00:00 2001 From: Andrei Bodrov Date: Tue, 19 Nov 2019 16:57:54 +0300 Subject: [PATCH 56/90] BrotliReadBuffer refactoring --- dbms/programs/server/HTTPHandler.cpp | 2 +- dbms/src/IO/BrotliReadBuffer.cpp | 16 ++++++++-------- dbms/src/IO/BrotliReadBuffer.h | 4 ++-- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/dbms/programs/server/HTTPHandler.cpp b/dbms/programs/server/HTTPHandler.cpp index 676ab05949e..cefa3712997 100644 --- a/dbms/programs/server/HTTPHandler.cpp +++ b/dbms/programs/server/HTTPHandler.cpp @@ -416,7 +416,7 @@ void HTTPHandler::processQuery( #if USE_BROTLI else if (http_request_compression_method_str == "br") { - in_post = std::make_unique(*in_post_raw); + in_post = std::make_unique(std::move(in_post_raw)); } #endif else diff --git a/dbms/src/IO/BrotliReadBuffer.cpp b/dbms/src/IO/BrotliReadBuffer.cpp index 145d096f0d6..9a781ea6f0b 100644 --- a/dbms/src/IO/BrotliReadBuffer.cpp +++ b/dbms/src/IO/BrotliReadBuffer.cpp @@ -32,9 +32,9 @@ public: BrotliDecoderResult result; }; -BrotliReadBuffer::BrotliReadBuffer(ReadBuffer &in_, size_t buf_size, char *existing_memory, size_t alignment) +BrotliReadBuffer::BrotliReadBuffer(std::unique_ptr in_, size_t buf_size, char *existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment) - , in(in_) + , in(std::move(in_)) , brotli(std::make_unique()) , in_available(0) , in_data(nullptr) @@ -55,12 +55,12 @@ bool BrotliReadBuffer::nextImpl() if (!in_available) { - in.nextIfAtEnd(); - in_available = in.buffer().end() - in.position(); - in_data = reinterpret_cast(in.position()); + in->nextIfAtEnd(); + in_available = in->buffer().end() - in->position(); + in_data = reinterpret_cast(in->position()); } - if (brotli->result == BROTLI_DECODER_RESULT_NEEDS_MORE_INPUT && (!in_available || in.eof())) + if (brotli->result == BROTLI_DECODER_RESULT_NEEDS_MORE_INPUT && (!in_available || in->eof())) { throw Exception("brotli decode error", ErrorCodes::BROTLI_READ_FAILED); } @@ -70,12 +70,12 @@ bool BrotliReadBuffer::nextImpl() brotli->result = BrotliDecoderDecompressStream(brotli->state, &in_available, &in_data, &out_capacity, &out_data, nullptr); - in.position() = in.buffer().end() - in_available; + in->position() = in->buffer().end() - in_available; working_buffer.resize(internal_buffer.size() - out_capacity); if (brotli->result == BROTLI_DECODER_RESULT_SUCCESS) { - if (in.eof()) + if (in->eof()) { eof = true; return working_buffer.size() != 0; diff --git a/dbms/src/IO/BrotliReadBuffer.h b/dbms/src/IO/BrotliReadBuffer.h index 960999cc937..0fa999d1de5 100644 --- a/dbms/src/IO/BrotliReadBuffer.h +++ b/dbms/src/IO/BrotliReadBuffer.h @@ -11,7 +11,7 @@ class BrotliReadBuffer : public BufferWithOwnMemory { public: BrotliReadBuffer( - ReadBuffer & in_, + std::unique_ptr in_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0); @@ -21,7 +21,7 @@ public: private: bool nextImpl() override; - ReadBuffer & in; + std::unique_ptr in; class BrotliStateWrapper; std::unique_ptr brotli; From 9296f652d2a92310c6de66b7c5111276d5514d06 Mon Sep 17 00:00:00 2001 From: Andrei Bodrov Date: Tue, 19 Nov 2019 17:35:03 +0300 Subject: [PATCH 57/90] old-style cast --- dbms/src/DataStreams/IBlockOutputStream.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/IBlockOutputStream.h b/dbms/src/DataStreams/IBlockOutputStream.h index de8895c850e..c90bd82215f 100644 --- a/dbms/src/DataStreams/IBlockOutputStream.h +++ b/dbms/src/DataStreams/IBlockOutputStream.h @@ -71,7 +71,7 @@ public: if (method == DB::CompressionMethod::Gzip) { auto write_buf = std::make_unique(args...); - return std::make_unique(std::move(write_buf), method, (int) 1 /* compression level */); + return std::make_unique(std::move(write_buf), method, 1 /* compression level */); } return std::make_unique(args...); } From 245c3511f6d018b7e1776929bc3083d3729cb6a8 Mon Sep 17 00:00:00 2001 From: Andrei Bodrov Date: Tue, 19 Nov 2019 20:11:13 +0300 Subject: [PATCH 58/90] clang fix --- dbms/src/IO/WriteBufferAIO.h | 2 +- dbms/src/IO/WriteBufferFromHDFS.h | 2 +- dbms/src/IO/WriteBufferFromHTTPServerResponse.h | 2 +- dbms/src/IO/WriteBufferFromS3.h | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/IO/WriteBufferAIO.h b/dbms/src/IO/WriteBufferAIO.h index ce63bd7325c..d51da73f906 100644 --- a/dbms/src/IO/WriteBufferAIO.h +++ b/dbms/src/IO/WriteBufferAIO.h @@ -51,7 +51,7 @@ private: /// Prepare an asynchronous request. void prepare(); /// - void finalize(); + void finalize() override; private: /// Buffer for asynchronous data writes. diff --git a/dbms/src/IO/WriteBufferFromHDFS.h b/dbms/src/IO/WriteBufferFromHDFS.h index 35aafc00921..22eab5c4d5d 100644 --- a/dbms/src/IO/WriteBufferFromHDFS.h +++ b/dbms/src/IO/WriteBufferFromHDFS.h @@ -25,7 +25,7 @@ public: ~WriteBufferFromHDFS() override; - void sync(); + void sync() override; }; } #endif diff --git a/dbms/src/IO/WriteBufferFromHTTPServerResponse.h b/dbms/src/IO/WriteBufferFromHTTPServerResponse.h index fdbf6969aeb..642e59e4921 100644 --- a/dbms/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/dbms/src/IO/WriteBufferFromHTTPServerResponse.h @@ -109,7 +109,7 @@ public: /// Use after the data has possibly been sent and no error happened (and thus you do not plan /// to change response HTTP code. /// This method is idempotent. - void finalize(); + void finalize() override; /// Turn compression on or off. /// The setting has any effect only if HTTP headers haven't been sent yet. diff --git a/dbms/src/IO/WriteBufferFromS3.h b/dbms/src/IO/WriteBufferFromS3.h index 9afda1d14e2..9a619f8c8bc 100644 --- a/dbms/src/IO/WriteBufferFromS3.h +++ b/dbms/src/IO/WriteBufferFromS3.h @@ -43,7 +43,7 @@ public: void nextImpl() override; /// Receives response from the server after sending all data. - void finalize(); + void finalize() override; ~WriteBufferFromS3() override; From 30ddf9599824aa454306be59b0d5a7151e8b3edc Mon Sep 17 00:00:00 2001 From: Andrei Bodrov Date: Tue, 19 Nov 2019 20:27:24 +0300 Subject: [PATCH 59/90] another clang fix --- dbms/src/DataStreams/SquashingBlockOutputStream.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/SquashingBlockOutputStream.h b/dbms/src/DataStreams/SquashingBlockOutputStream.h index 7828ad7e96d..2a8fb857457 100644 --- a/dbms/src/DataStreams/SquashingBlockOutputStream.h +++ b/dbms/src/DataStreams/SquashingBlockOutputStream.h @@ -31,7 +31,7 @@ private: SquashingTransform transform; bool all_written = false; - void finalize(); + void finalize() override; bool disable_flush = false; }; From 9a2b864940f947b9f66455ca5faf855dd7d80559 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 19 Nov 2019 21:42:51 +0300 Subject: [PATCH 60/90] Cleanup --- dbms/programs/client/Client.cpp | 7 +++++-- dbms/src/IO/ReadHelpers.h | 14 -------------- .../Processors/Formats/Impl/CSVRowInputFormat.cpp | 1 - docs/en/operations/settings/settings.md | 4 +--- 4 files changed, 6 insertions(+), 20 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index f969f99393e..32395ff9101 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -1113,11 +1113,14 @@ private: auto packet_type = connection->checkPacket(); if (packet_type && *packet_type == Protocol::Server::Exception) { - async_block_input->cancel(false); + /* + * We're exiting with error, so it makes sense to kill the + * input stream without waiting for it to complete. + */ + async_block_input->cancel(true); return; } - connection->sendData(block); processed_rows += block.rows(); diff --git a/dbms/src/IO/ReadHelpers.h b/dbms/src/IO/ReadHelpers.h index e6b3e7e91e3..c3935e1092d 100644 --- a/dbms/src/IO/ReadHelpers.h +++ b/dbms/src/IO/ReadHelpers.h @@ -24,7 +24,6 @@ #include #include -#include #include #include @@ -912,17 +911,4 @@ void skipToNextLineOrEOF(ReadBuffer & buf); /// Skip to next character after next unescaped \n. If no \n in stream, skip to end. Does not throw on invalid escape sequences. void skipToUnescapedNextLineOrEOF(ReadBuffer & buf); -/** Returns buffer eof() result. - * And saves data if there is no pending data in buffer or it was explicitly asked. - * Why we have to use this strange function? Consider we have begin_pos in the middle of our buffer - * and the cursor in the end of the buffer. When we call eof() it calls next(). - * And this function can fill the buffer with new data, so we will lose the data from previous buffer state. - * @param buf - original buffer to read from. - * memory - where to put data from buf - * used_size - special parameter not to do useless reallocations - * begin_pos - defines from which position we will copy the data. - * forse_saving_buffer_state - allows to explicitly copy all the data from begin_pos to current_position. - */ -bool eofWithSavingBufferState(ReadBuffer & buf, DB::Memory<> & memory, size_t & used_size, char * & begin_pos, bool force_saving_buffer_state = false); - } diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 648956a14ee..a13bb365192 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -422,5 +422,4 @@ void registerInputFormatProcessorCSV(FormatFactory & factory) } } - } diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index cf9c6619b74..4437ba79f7f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -999,9 +999,7 @@ Default value: 0. - Type: bool - Default value: True -Enable order-preserving parallel parsing of data formats such as JSONEachRow, TSV, TKSV and CSV. Reading will be single threaded and parsing will be multithreaded. - -The maximum number of threads for order-preserving parallel parsing of data formats. 0 means use global maximum. +Enable order-preserving parallel parsing of data formats. Supported only for TSV format. ## min_chunk_bytes_for_parallel_parsing From 2c37e31d2e44023c3edc077f4f2c53279d341c39 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Metehan=20=C3=87etinkaya?= Date: Wed, 20 Nov 2019 09:12:42 +0300 Subject: [PATCH 61/90] Added comprehension to improve performance --- docs/tools/concatenate.py | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/docs/tools/concatenate.py b/docs/tools/concatenate.py index 4eb8fcf9562..4ff9f9214df 100755 --- a/docs/tools/concatenate.py +++ b/docs/tools/concatenate.py @@ -11,11 +11,8 @@ def concatenate(lang, docs_path, single_page_file): az_re = re.compile(r'[a-z]') with open(proj_config) as cfg_file: - files_to_concatenate = [] - for l in cfg_file: - if '.md' in l and 'single_page' not in l: - path = (l[l.index(':') + 1:]).strip(" '\n") - files_to_concatenate.append(path) + files_to_concatenate = [(l[l.index(':') + 1:]).strip(" '\n") for l in cfg_file + if '.md' in l and 'single_page' not in l] logging.info( str(len(files_to_concatenate)) + From db1a86a94dd2123e82213e059863ed9f8d20753f Mon Sep 17 00:00:00 2001 From: Andrei Bodrov Date: Wed, 20 Nov 2019 11:27:59 +0300 Subject: [PATCH 62/90] typo --- dbms/src/TableFunctions/ITableFunctionFileLike.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/TableFunctions/ITableFunctionFileLike.cpp b/dbms/src/TableFunctions/ITableFunctionFileLike.cpp index 44f7e02dd4b..3e0ddafaa90 100644 --- a/dbms/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/dbms/src/TableFunctions/ITableFunctionFileLike.cpp @@ -33,7 +33,7 @@ StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, cons ASTs & args = args_func.at(0)->children; if (args.size() != 3 && args.size() != 4) - throw Exception("Table function '" + getName() + "' requires 3 or 4 arguments: filename, format, structure and compression method (default none).", + throw Exception("Table function '" + getName() + "' requires 3 or 4 arguments: filename, format, structure and compression method (default auto).", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (size_t i = 0; i < args.size(); ++i) From 53b02227adbd21eeb7c3cdd3bc08fc5007bb43cc Mon Sep 17 00:00:00 2001 From: Andrei Bodrov Date: Wed, 20 Nov 2019 13:11:26 +0300 Subject: [PATCH 63/90] virtual void --- dbms/src/IO/WriteBuffer.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/IO/WriteBuffer.h b/dbms/src/IO/WriteBuffer.h index 0777062716f..25c93b227e3 100644 --- a/dbms/src/IO/WriteBuffer.h +++ b/dbms/src/IO/WriteBuffer.h @@ -90,8 +90,8 @@ public: ++pos; } - void virtual sync() {} - void virtual finalize() {} + virtual void sync() {} + virtual void finalize() {} private: /** Write the data in the buffer (from the beginning of the buffer to the current position). From 7bbbd62353255525ed55657148b96d6674f0c827 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 20 Nov 2019 14:34:23 +0300 Subject: [PATCH 64/90] Added descriptions to some tasks --- docs/ru/extended_roadmap.md | 328 +++++++++++++++++++++++++++++++++++- 1 file changed, 327 insertions(+), 1 deletion(-) diff --git a/docs/ru/extended_roadmap.md b/docs/ru/extended_roadmap.md index 56bcd3ba5d9..4674f77092e 100644 --- a/docs/ru/extended_roadmap.md +++ b/docs/ru/extended_roadmap.md @@ -20,6 +20,10 @@ Делает [Александр Токмаков](https://github.com/tavplubix), первый рабочий вариант в декабре 2019. Нужно для DataLens и Яндекс.Метрики. +Манипуляции с каталогом баз данных: запросы CREATE TABLE, DROP TABLE, RENAME TABLE и DATABASE, требуют синхронизации с помощью блокировок. Эта синхронизация становится весьма сложной, так как на неё полагается много внутренних структур данных. + +Предлагается реализовать альтернативный подход, в котором таблицы и базы данных являются всего лишь ссылками на persistent объекты. Подробное описание задачи: https://github.com/yandex/ClickHouse/issues/6787 + ### 1.3. Неблокирующие ALTER. И полностью immutable куски. Делает [Александр Сапин](https://github.com/alesapin). Готов приступить к задаче в конце ноября 2019. Нужно для Яндекс.Метрики. @@ -36,6 +40,12 @@ Делает [Антон Попов](https://github.com/CurtizJ), первый рабочий вариант в декабре. Пререквизит чтобы снизить сложность мелких INSERT, что в свою очередь нужно для 1.12, иначе задача 1.12 не сможет нормально работать. Особенно нужно для Яндекс.Облака. +Данные в таблицах типа MergeTree в ClickHouse хранятся в виде набора независимых "кусков". Внутри куска, каждый столбец, а также индекс, хранится в отдельных файлах. Это сделано для возможности быстрых манипуляций со столбцами (пример - запрос ALTER DROP COLUMN). При вставке данных (INSERT), создаётся новый кусок. Для таблиц с большим количеством столбцов, запросы INSERT с маленьким количеством строк являются неэффективными, так как требуют создания большого количества файлов в файловой системе. Это является врождённой особенностью ClickHouse - одной из первой проблем, с которыми сталкиваются пользователи. Пользователям приходится буферизовывать данные и собирать их в более крупные пачки перед вставкой в ClickHouse. + +Для смягчения эффекта от этой проблемы, в ClickHouse существуют таблицы типа Buffer. Они накапливают данные в оперативке перед записью в другую таблицу. Впрочем, таблицы Buffer не являются полноценным решением проблемы из-за: - наличия блокировок при вставке; - переупорядочивание вставляемых данных; - неатомарность перекладывания данных из Buffer в результирующую таблицу. + +Вместо этого предлагается разрешить кускам таблиц типа MergeTree располагать данные в разных форматах. А именно: - в оперативной памяти; - на диске со всеми столбцами в одном файле; - на диске со столбцами в отдельных файлах: в зависимости от размера куска и прошедшего времени. Для размещения кусков в оперативной памяти, придётся также реализовать опциональную поддержку write-ahead log с настраиваемыми правилами по сбросу на диск. Это позволит избавиться от проблем с мелкими вставками для MergeTree таблиц. Для ReplicatedMergeTree таблиц, это решит проблему лишь частично. + ### 1.7. Буферизация и WAL в MergeTree. Требует 1.6. @@ -48,14 +58,40 @@ В очереди. +Сейчас пользователь может задать в таблице выражение, которое определяет, сколько времени хранятся данные. Обычно это выражение задаётся относительно значения столбца с датой - например: удалять данные через три месяца. https://clickhouse.yandex/docs/ru/operations/table_engines/mergetree/#table_engine-mergetree-ttl + +Это может быть задано для всей таблицы (тогда строки целиком удаляются после указанного времени) или для отдельных столбцов (тогда данные столбца физически удаляются с диска, а строки в таблице остаются; при чтении значений столбца, они читаются как значения по-умолчанию). + +Но пользователи также хотят более продвинутый вариант этой функциональности: не удалять строки или столбцы целиком, а прореживать их - оставлять меньшее количество строк. + +И тут есть несколько вариантов: +1. По прошествии времени, оставлять каждую N-ую строку. +2. По прошествии времени, выполнять агрегацию данных, заменяя значения некоторых столбцов на значения агрегатных функций от множества значений в нескольких строках. + +Пункт 1 не представляет интереса, так как уже реализован с помощью TTL выражений для удаления данных. В качестве этого выражения можно прописать, например, cityHash64(*) % 10 = 0 ? now() : event_time + INTERVAL 3 MONTH. Правда как-то неудобно получается. + +А вот пункт 2 требуется продумать. Не очевидно даже, какой лучше использовать синтаксис для этого при создании таблицы. Но мы придумаем - сразу видно несколько вариантов. + +Частный случай такой задачи уже есть в https://clickhouse.yandex/docs/ru/operations/table_engines/graphitemergetree/ Но это было сделано для конкретной задачи. А надо обобщить. + ### 1.10. Пережатие старых данных в фоне. Будет делать Кирилл Барухов, ВШЭ, экспериментальная реализация к весне 2020. Нужно для Яндекс.Метрики. +Алгоритмы сжатия типа LZ77 позволяют потратить больше времени на сжатие данных, чтобы сжать данные сильнее, но при этом без проигрыша по скорости разжатия данных. В частности, этим свойством обладает LZ4 и ZSTD, которые используются в ClickHouse. Это позволяет использовать свободные ресурсы CPU, когда сервер не нагружен, для пережатия данных, чтобы данные занимали меньше места на дисках, и при этом сохранить или даже улучшить скорость обработки запросов. + +В то же время, ClickHouse обычно используется для "импульсного" сценария нагрузки. Запрос от пользователя обрабатывается максимально быстро, используя все ресурсы CPU, но в среднем по времени, сервер недостаточно нагружен. + +Предлагается добавить в ClickHouse настройки по пережатию данных и фоновые потоки, выполняющие эту задачу. + ### 1.11. Виртуальная файловая система. Нужно для Яндекс.Облака. Делает Александр Бурмак, Яндекс.Облако, а также Олег Ершов, ВШЭ и Яндекс. +ClickHouse использует для хранения данных локальную файловую систему. Существует сценарий работы, в котором размещение старых (архивных) данных было бы выгодно на удалённой файловой системе. Если файловая система POSIX совместимая, то это не составляет проблем: ClickHouse успешно работает с Ceph, GlusterFS, MooseFS. Также востребованным является сценарий использования S3 (из-за доступности в облаке) или HDFS (для интеграции с Hadoop). Но эти файловые системы не являются POSIX совместимыми. Хотя для них существуют FUSE драйверы, но скорость работы сильно страдает и поддержка неполная. + +ClickHouse использует небольшое подмножество функций ФС, но в то же время, и некоторые специфические части: симлинки и хардлинки, O_DIRECT. Предлагается выделить всё взаимодействие с файловой системой в отдельный интерфейс. + ### 1.12. Экспериментальная реализация VFS поверх S3 и HDFS. Нужно для Яндекс.Облака. Требует 1.11. Желательно 1.6 и 1.18. @@ -182,6 +218,12 @@ [Никита Лапков](https://github.com/laplab), весна 2020. Нужно для Яндекс.Метрики. Требует 4.1. +Если распределённый запрос затрагивает большое количество серверов, то время выполнения запросов часто становится большим из-за tail latencies - случайных редких замедлений отдельных серверов. Эту проблему можно избежать, отправляя один и тот же запрос сразу на несколько реплик, и используя данные с наиболее быстрой. + +Задача скрывает в себе много тонкостей, связанных с обработкой стадий выполнения запроса (соединение, обмен handshake, отправка запроса, получение заголовка результата, получение пакетов прогресса, получение данных), правильной возможностью настройки таймаутов, правильной отменой запросов. + +Сейчас для распределённых запросов используется по потоку на соединение. Это позволяет хорошо распараллелить вычисления над полученными данными и утилизировать сеть, но становится сильно избыточным для больших кластеров. Для примера, создание 1000 потоков для чтения данных из 1000 серверов кластера - лишь расходует ресурсы и увеличивает время выполнения запроса. Вместо этого необходимо использовать количество потоков не большее количества процессорных ядер, и мультиплексировать в одном потоке общение с серверами. Реализация нетривиальна, так как мультиплексировать необходимо каждую стадию общения по сети, включая установку соединения и обмен handshake. + ### 4.3. Ограничение числа одновременных скачиваний с реплик. Изначально делал Олег Алексеенков, но решение оказалось неудачным, хотя там не так уж много доделывать. @@ -236,6 +278,8 @@ Делает Андрей Скобцов, ВШЭ. +В Linux существует возможность получать в программе информацию о счётчиках производительности и событиях, относящихся к CPU и ядру ОС. Подробнее смотрите `man perf_event_open`. Предлагается добавить эти метрики в ClickHouse для инструментирования запросов. + ### 6.5. Эксперименты с LLVM X-Ray. Требует 2.2. @@ -319,6 +363,31 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э Тагир Кускаров, ВШЭ. Посмотрим на https://github.com/AmokHuginnsson/replxx +Для ввода запросов в интерактивном режиме в клиенте командной строки clickhouse-client используется библиотека readline или libedit. + +Библиотеки readline и libedit обладает следующими недостатками: +- (исправлено в новых версиях readline) Очень низкая производительность вставки больших кусков текста. Вставка каждого следующего символа имеет сложность O(n = количество предыдущих символов) и при вставке 1 МБ текста, скорость падает до десятков байт в секунду. +- Крайне сложно или невозможно реализовать подсветку синтаксиса по мере набора текста, а также autocomplete без нажатия дополнительных клавиш для вызова. +- Лицензия GPL (для readline) препятствует её включению в кодовую базу продукта. +- Плохо работает навигация по истории, если история вкючает запросы, не помещающиеся на экран. +- История сохраняется лишь при завершении работы клиента. +- При параллельной работе нескольких клиентов с одним файлом истории, сохраняется история только одного из клиентов. +- Плохо работает история для многострочных запросов. +- Излишняя экономия пересылаемых данных, что часто приводит к остаткам мусора в терминале. + +Кроме того, имеются следующие сложно достижимые достоинства: +- Поддержка right-to-left текста; +- Поддержка editrc конфигураций. + +В качестве альтернатив можно рассмотреть следующие варианты: +- Linenoise от Salvatore Sanfilippo. Достоинства: простота и компактность кода; высокая скорость работы. Недостатки: отсутствует поддержка Unicode; отсутствует автоматический перенос текста, что затрудняет работу с многострочными запросами. +- Linenoise с патчами для поддержки Unicode. Недостаток: теряется преимущество по скорости работы. +- Fish shell. Не является библиотекой, но представляет собой отличный пример, как можно реализовать подстветку синтаксиса и удобный autocomplete. Поддерживает Unicode, но работает весьма медленно. +- Python Prompt Toolkit. Не является подходящим решением для интеграции в C++ проект. Хорошие возможности по подсветке синтаксиса и autocomplete. + +Вместо этого предлагается в качестве примера изучить прототип текстового редактора Kilo: https://viewsourcecode.org/snaptoken/kilo/ и реализовать всю необходимую функциональность. + + ### 7.15. Замена libressl обратно на openssl. Поводом использования libressl послужило желание Константина podshumok Игнатова из QRator и то, что тогда openssl был опозорен и libressl считалась адекватной альтернативой. Но сейчас ситуация изменилась - openssl продолжает развиваться, а libressl не особо, и можно спокойно менять обратно. @@ -359,14 +428,46 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э Дарья Петрова, УрФУ. +Над ClickHouse одновременно работает большое количество разработчиков, которые оформляют свои изменения в виде pull requests. Когда непомерженных pull requests много, то возникает сложность с организацией работы - непонятно, на какой pull request смотреть в первую очередь. + +Предлагается реализовать простое одностраничное веб-приложение, в котором отображается список pull requests со следующей информацией: +- размер diff - количество изменённых строк; +- как давно было последнее обновление; +- типы изменённых файлов: C++, документация, скрипты сборки; +- наличие добавленных тестов; +- есть ли описание для changelog; +- изменены ли submodules; +- был ли разрешён запуск проверок CI; +- статусы проверок CI; +- количество approve от ревьюеров; + +Статусы проверок - наиболее важная часть. Так как для каждого PR выполняется несколько десятков проверок и наиболее медленные работают до нескольких часов, придётся: +- отображать сразу все проверки для каждого PR в виде красивой разноцветной матрицы с информацией по наведению мыши; +- отсортировать проверки по важности: например, если у внешнего разработчика проходят все проверки кроме стиля кода, то мы можем взять это в работу сами; +- если для предыдущего коммита проверка была завершена, а для последнего коммита ещё только идёт - то можно отображать в таблице статус предыдущей проверки более блёклым цветом. + +Предлагается реализовать несколько вариантов сортировок. Очевидное - по времени обновления, более интересно - некое ранжирование с целью выяснить, "что лучше взять в работу прямо сейчас". + +Похожие продукты уже есть, например: http://prs.mozilla.io/yandex:ClickHouse К сожалению, этот продукт заброшен, да и делает не совсем то, что нужно. По своему усмотрению, можно взять из него что-нибудь полезное. + ### 7.23. Функции для fuzzing. Андрей Некрашевич, ВШЭ. +Fuzzing тестирование - это тестирование случайными данными. Мы рассмотрим несколько подходов к этой задачи: + +1. Добавление в SQL диалект ClickHouse функций для генерации случайных данных (пример - случайные бинарные строки заданной длины, случайные валидные UTF-8 строки) и "порчи" данных (например, поменять значения случайных бит с заданной частотой). Это будет использовано для тестирования SQL-функций ClickHouse. + ### 7.24. Fuzzing лексера и парсера запросов; кодеков и форматов. Андрей Некрашевич, ВШЭ. +Продолжение 7.23. + +2. Использование AFL или LibFuzzer для тестирования отдельных частей кодовой базы ClickHouse. + +3. Генерация и выполнение случайных синтаксически корректных запросов на случайных данных. + ### 7.25. Синхронизация релизов в Аркадию. Изначально занимался Олег Алексеенков. Сейчас он перешёл работать в Яндекс.Морду, но обещает продолжать синхронизацию. @@ -396,6 +497,12 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э Роман Ильговский. Нужно для Яндекс.Метрики. +Имея SQL запрос, требуется вывести структуру таблиц, на которых этот запрос будет выполнен, и заполнить эти таблицы случайными данными, такими, что результат этого запроса зависит от выбора подмножества данных. + +Для примера, если есть запрос `SELECT SearchPhrase, count(*) FROM table WHERE CounterID = 34 AND SearchPhrase LIKE '%ClickHouse%'`, то мы можем сделать вывод, что CounterID имеет числовой тип, а SearchPhrase - строковый. Заполнить таблицу данными, на которых отдельные условия `CounterID = 34` и `SearchPhrase LIKE '%ClickHouse%'` для некоторых строк выполнены, а для некоторых строк не выполнены. + +Обфускация запросов: имея секретные запросы и структуру таблиц, заменить имена полей и константы, чтобы запросы можно было использовать в качестве публично доступных тестов. + ### 7.33. Выкладывать патч релизы в репозиторий автоматически. [Александр Сапин](https://github.com/alesapin). Может делегировать эту задачу кому угодно. @@ -471,16 +578,26 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э Павел Круглов, ВШЭ и Яндекс. +Формат Apache Avro является компактным структурированным построчным бинарным форматом данных с внешней схемой. Этот формат часто используется совместно с Kafka и поддержка его в качестве одного из форматов ввода-вывода в ClickHouse является востребованной пользователями. + ### 8.17. ClickHouse как MySQL реплика. Ильяс Адюгамов, ВШЭ. +Реализовать возможность подписаться на row-based репликацию MySQL и сохранять полученные данные в CollapsingMergeTree или ReplacingMergeTree таблицы. Сторонние решения для этой задачи уже существуют: https://www.altinity.com/blog/2018/6/30/realtime-mysql-clickhouse-replication-in-practice Также существует стороннее решение для PostgreSQL: https://github.com/mkabilov/pg2ch + +Встроенная в ClickHouse возможность работать в качестве реплики MySQL даст преимущества для дальнейшего развития. + ### 8.18. ClickHouse как Federated MySQL. ### 8.19. Интеграция с RabbitMQ. Ксения Сумарокова, ВШЭ. +В ClickHouse часто используется потоковый импорт данных из распределённой очереди. Наиболее популярно использование совместно с Kafka. Эта возможность уже есть. + +Следующей по востребованности является система очередей RabbitMQ. Её поддержка в ClickHouse отсутствует. + ### 8.20. Интеграция с SQS. ### 8.21. Поддержка произвольного количества языков для имён регионов. @@ -498,6 +615,10 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э Валерий Батурин, ВШЭ. +ClickHouse предоставляет возможность обратиться к внешней базе данных из языка запросов. Это реализовано в виде табличных функций. В параметрах к табличной функции указывается адрес удалённой базы данных (хост, порт), а также аутентификационные данные (имя пользователя, пароль). Аутентификационные данные указываются в запросе в открытом виде и, таким образом, попадают в историю запросов и в логи, что компрометирует безопасность системы. + +Вместо этого предлагается описывать необходимые данные в конфигурационном файле сервера или в отдельном сервисе и ссылаться на них по именам. + ### 9.3. Поддержка TLS для ZooKeeper. @@ -563,10 +684,16 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э Никита Васильев, ВШЭ и Яндекс. +Реализовать в ClickHouse специализированный движок таблиц, подходящий для быстрых key-value запросов и оптимизированный для расположения данных на SSD. Это может быть: реализация на основе RocksDB; сериализованные RowBinary данные с индексом в оперативке; секретная очень эффективная структура данных, о которой я расскажу. + +Использовать эту структуру данных как отдельный вид словарей, как источник для cache словарей или как дополнительный уровень кэширования для cache словарей. + ### 10.17. Локальный дамп состояния словаря для быстрого старта сервера. ### 10.18. Таблица Join или словарь на удалённом сервере как key-value БД для cache словаря. +### 10.19. Возможность зарегистрировать некоторые функции, использующие словари, под пользовательскими именами. + ## 11. Интерфейсы. @@ -574,7 +701,7 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э ### 11.2. Возможность использовать движок JDBC из коробки. -Нужно разобраться, как упаковывать Java в статический бинарник, возможно AppImage. +Нужно разобраться, как упаковывать Java в статический бинарник, возможно AppImage. Или предоставить максимально простую инструкцию по установке jdbc-bridge. Может быть будет заинтересован Александр Крашенинников, Badoo, так как он разработал jdbc-bridge. ### 11.3. Интеграционные тесты ODBC драйвера путём подключения ClickHouse к самому себе через ODBC. @@ -594,6 +721,8 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э Элбакян Мовсес Андраникович, ВШЭ. +В ClickHouse в прошлом году добавили поддержку wire-протокола MySQL. PostgreSQL, так же как MySQL, использует несложный протокол общения между клиентом и сервером, но свой собственный. Поддержка этого протокола является востребованной и откроет новые возможности для ClickHouse. + ### 11.9. Доработки ODBC драйвера. Денис Глазачев, Altinity. @@ -602,6 +731,8 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э zhang2014 +Возможность описать в конфигурационном файле handler (путь в URL) для HTTP запросов к серверу, которому соответствует некоторый параметризованный запрос. Пользователь может вызвать этот обработчик и не должен передавать SQL запрос. + ## 12. Управление пользователями и доступом. @@ -673,6 +804,10 @@ zhang2014 Софья Борзенкова, ВШЭ. +В ClickHouse поддерживается вычисление COUNT(DISTINCT x). Предлагается добавить возможность использования модификатора DISTINCT для всех агрегатных функций. Например, AVG(DISTINCT x) - вычислить среднее значение для всех различных значений x. Под вопросом вариант, в котором фильтрация уникальных значений выполняется по одному выражению, а агрегация по другому. + +Результат некоторых агрегатных функций зависит от порядка данных. Предлагается реализовать модификатор ORDER BY, задающий порядок явно. Пример: groupArray(x ORDER BY y, z). + ### 14.9. Поддержка запроса EXPLAIN. Требует 2.1. [Николай Кочетов](https://github.com/KochetovNicolai). @@ -685,6 +820,10 @@ zhang2014 Сложная задача, так как вводит новый класс функций и требует его обработку в оптимизаторе запросов. +В time-series СУБД нужны функции, которые зависят от последовательности значений. Или даже от последовательности значений и их меток времени. Примеры: moving average, exponential smoothing, derivative, Holt-Winters forecast. Вычисление таких функций поддерживается в ClickHouse лишь частично. Так, ClickHouse поддерживает тип данных "массив" и позволяет реализовать эти функции как функции, принимающие массивы. Но гораздо удобнее для пользователя было бы иметь возможность применить такие функции к таблице (промежуточному результату запроса после сортировки). + +Это требует введение нового класса функций (помимо обычных и агрегатных функций) - такие функции будут иметь в коде ClickHouse свой собственный интерфейс, и их вычисление придётся отдельно учитывать в конвейере выполнения запросов. Для примера, вычисление обычных функций тривиально распараллеливается по процессорным ядрам и по серверам; вычисление агрегатных функций распараллеливается с некоторыми особенностями (работа с промежуточными состояниями вычислений, операция merge); а для функций по обработке временных рядов этот вопрос остаётся открытым - возможно, их придётся вычислять на одном сервере и в одном потоке. + ### 14.13. Применимость функций высшего порядка для кортежей и Nested. ### 14.14. Неявные преобразования типов констант. @@ -770,10 +909,16 @@ zhang2014 [Андрей Чулков](https://github.com/achulkov2), Антон Кваша, Артур Петуховский, ВШЭ. Будет основано на коде от Арслана Урташева. +ClickHouse не является geospatial СУБД. Тем не менее, в ClickHouse есть несколько функций для таких задач. Например, функция `pointInPolygon` позволяет быстро проверить попадание точек в полигон на плоскости. При этом, полигон задаётся в явном виде и должен быть константным для вызова функции (то есть - проверяется принадлежность многих точек одному полигону). Эта функциональность нужна, например, для рекламного таргетинга мобильных устройств по координатам. + +Похожая, но более сложная задача, которую ClickHouse пока не умеет решать - определение полигона среди множества полигонов, в которые попадают точки. Для примера: определение района города по географическим координатам. Для решения этой задачи нужно будет реализовать поддержку словарей с полигонами, в которых данные проиндексированы для быстрого поиска. + ### 17.2. GIS типы данных и операции. Алексей Коряков, Алексей Илюхов, ВШЭ, Яндекс.Карты. +Реализовать в ClickHouse типы данных для задач обработки геоинформационных данных: Point, Line, MultiLine, Polygon и операции над ними - проверка вхождения, пересечения. Вариантом минимум будет реализация этих операций в евклидовой системе координат. Дополнительно - на сфере и WGS84. + ### 17.3. Ускорение greatCircleDistance. [Ольга Хвостикова](https://github.com/stavrolia), основано на коде Андрея Аксёнова, получено разрешение на использование кода. @@ -797,6 +942,8 @@ zhang2014 Артём Цыганов, Руденский Константин Игоревич, Семёнов Денис, ВШЭ. +Предлагается реализовать в ClickHouse статистические тесты (Analysis of Variance, тесты нормальности распределения и т. п.) в виде агрегатных функций. Пример: `welchTTest(value, sample_idx)`. + ### 18.3. Инфраструктура для тренировки моделей в ClickHouse. В очереди. Возможно, Александр Кожихов. У него сначала идёт задача 24.26. @@ -808,10 +955,24 @@ zhang2014 Александра Латышева, ВШЭ и Яндекс. +Репликация данных в ClickHouse по-умолчанию является асинхронной без выделенного мастера. Это значит, что клиент, осуществляющий вставку данных, получает успешный ответ после того, как данные попали на один сервер; репликация данных по остальным серверам осуществляется в другой момент времени. Это ненадёжно, потому что допускает потерю только что вставленных данных при потере лишь одного сервера. + +Для решения этой проблемы, в ClickHouse есть возможность включить "кворумную" вставку. Это значит, что клиент, осуществляющий вставку данных, получает успешный ответ после того, как данные попали на несколько (кворум) серверов. Обеспечивается линеаризуемость: клиент, получает успешный ответ после того, как данные попали на несколько реплик, *которые содержат все предыдущие данные, вставленные с кворумом* (такие реплики можно называть "синхронными"), и при запросе SELECT можно выставить настройку, разрешающую только чтение с синхронных реплик. + +Если бы свойства линеаризуемости не было, то для трёх серверов A, B, C, значения кворума = 2, и для трёх вставок данных 1, 2, 3, возможна ситуация, что первая вставка прошла на серверы A и B, вторая прошла на серверы B и C, а третья - на серверы A и C, и теперь ни один из серверов не содержит полный набор данных 1, 2, 3. + +Как ни странно, такое свойство не нужно большинству пользователей. Оно запрещает параллельно выполняющиеся вставки. А пользователи хотят вставлять данные надёжно (на более чем одну реплику), но не важно, в каком порядке. Предлагается сделать опцию, которая отключает линеаризуемость. + +Иногда пользователь хочет реализовать кворумную вставку вручную: просто соединиться с несколькими репликами и вставть на них одинаковые данные (чтобы обеспечить надёжную вставку, не ориентируясь на то, как работает механизм репликации). Сейчас ожидания пользователя не оправдываются. В ClickHouse есть механизм дедупликации для обеспечения идемпотентности вставок. Вторая вставка с такими же данными (пусть даже на другую реплику) будет проигнорирована. Надо сделать так, чтобы вместо этого, вставка одинаковых данных на другую реплику, имела такой же эффект, как если бы эти данные были получены с помощью механизма репликации. + ### 19.2. Подключение Etcd или Consul как альтернативы ZooKeeper. Алексей Лёвушкин, ВШЭ и Яндекс. +Для координации реплик в ClickHouse используется ZooKeeper. Многие пользователи ClickHouse хотели бы иметь возможность использовать для координации некоторые другие системы вместо ZooKeeper. Рассматриваемыми вариантами таких систем являются Etcd, Consul, FoundationDB. Это весьма проблематично, так как эти системы существенно отличаются по интерфейсам и возможностям. Тем не менее, для того, чтобы эта задача стала возможной, в ClickHouse обобщён интерфейс взаимодействия с ZooKeeper, и теперь на его место можно подставлять другие реализации. + +В прошлом году, Алексей добавил модельную реализацию (mock) интерфейса ZooKeeper для тестирования. Сейчас предлагается сделать реализацию поверх Etcd, а также расширить возможности тестовой реализации. + ### 19.3. Подключение YT Cypress или YDB как альтернативы ZooKeeper. Hold. Полезно для Яндекс.Облака и БК, но есть риски, что будет вредно, а не полезно. @@ -822,6 +983,10 @@ Hold. Полезно для Яндекс.Облака и БК, но есть р В очереди, возможно Валерий Батурин, ВШЭ. +Репликация в ClickHouse работает на уровне отдельных таблиц. Это является очень гибким решением: на одном сервере одна из таблиц может быть не реплицирована, другая иметь двухкратную репликацию, а третья - реплицирована по всем серверам. Но если все таблицы в базе данных реплицированы одинаковым образом. то это затрудняет управление кластером. Например, при восстановлени сервера, требуется отдельно создавать реплику для каждой таблицы. + +Предлагается реализовать "движок баз данных", который осуществляет репликацию метаданных (множество имеющихся таблиц и лог DDL операций над ними: CREATE, DROP, RENAME, ALTER). Пользователь сможет создать реплицируемую базу данных; при её создании или восстановлении на другом сервере, все реплицируемые таблицы будут созданы автоматически. + ### 19.6. Одновременный выбор кусков для слияния многими репликами, отказ от leader election в ZK. ### 19.7. Возможность записи данных при недоступности ZK и отказ от линейного порядка кусков в большинстве случаев. @@ -866,6 +1031,12 @@ Hold. Полезно для Яндекс.Облака и БК, но есть р Дмитрий Рубашкин, ВШЭ. Помогает Антон Попов. +Если таблица имеет ключ сортировки, то возможно эффективное чтение упорядоченных данных. Если запрос содержит операцию GROUP BY, содержащую по крайней мере префикс от ключа сортировки таблицы, либо инъективные функции от него, то возможно более эффективное выполнение GROUP BY: промежуточный результат агрегации финализируется и отправляется клиенту как только в потоке данных при чтении из таблицы встретился следующий ключ. + +Аналогичную оптимизацию следует реализовать для DISTINCT и LIMIT BY. + +В прошлом году, аналогичное решение сделали для операции ORDER BY. + ### 21.5. Распараллеливание INSERT при INSERT SELECT, если это необходимо. ### 21.6. Уменьшение числа потоков для SELECT в случае тривиального INSERT SELECT. @@ -878,6 +1049,12 @@ Hold. Полезно для Яндекс.Облака и БК, но есть р Михаил Кот, ВШЭ. Задача сложная и рискованная. +Для выделения памяти, аллокаторы запрашивают её у операционной системы (`mmap`). Это возможно только для достаточно крупных кусков памяти является довольно медленной операцией. Поэтому, современные аллокаторы кэшируют крупные куски памяти в программе. При вызове free, кусок памяти, как правило, не отдаётся ОС, а остаётся для последующего переиспользования. Для выделения мелких кусков памяти, крупные куски разбиваются с помощью специальных структур данных (free-list, heap, bitmap). Для уменьшения contention в многопоточных программах, эти структуры также делаются thread-локальными. + +Часто в программе есть кэши некоторых данных. Например - кэш данных после разжатия, использующийся чтобы сэкономить на повторных запросах одних и тех же данных. При вытеснении из кэша, блок данных освобождается (`free`) и данные, бывшие в кэше, становятся недоступными для переиспользования. Но если принимать во внимание то, как работает аллокатор памяти, то оказывается, что после освобождения памяти, данные всё ещё остаются доступными в программе. И если этот кусок памяти не будет выделен аллокатором снова, его можно было бы продолжить использовать в качестве кэша. Иными словами, в программе есть domain-specific кэш, а аллокатор имеет свой кэш, и они не знают друг о друге. + +Для domain-specific кэшей (как например, кэш разжатых данных) выгодно, чтобы они использовали как можно больший объём свободной памяти. Но в этом случае, памяти может не хватить для других структур данных в программе. Если аллокатор памяти знает про кэш, то выделение памяти можно было бы делать путём вытеснения данных из кэша. + ### 21.8.1. Отдельный аллокатор для кэшей с ASLR. В прошлом году задачу пытался сделать Данила Кутенин с помощью lfalloc из Аркадии и mimalloc из Microsoft, но оба решения оказались неудачными. Успешная реализация задачи 21.8 отменит необходимость в этой задаче, поэтому холд. @@ -894,10 +1071,25 @@ Amos Bird. Руслан Камалов, Михаил Малафеев, Виктор Гришанин, ВШЭ +Реализовать в ClickHouse оптимизации запросов, основанные на упрощении отдельных небольших кусков выражений (так называемые "peephole" оптимизации). Примеры: +- Замена цепочек if на multiIf. +- Удаление min/max/any-агрегатов от выражений от ключей GROUP BY. +- Вынесение арифметических операций из агрегатных функций; +- Вынесение любых функций наружу any, anyLast. +- При GROUP BY по transform или if по строкам, замена строк на Enum. + ### 21.12. Алгебраические оптимизации запросов. Руслан Камалов, Михаил Малафеев, Виктор Гришанин, ВШЭ +Реализовать в ClickHouse оптимизации запросов, основанные на алгебраических свойствах функций. Примеры: +- Обращение инъективных функций в сравнениях на равенство. +- Вынесение инъективных функцию наружу uniq. +- Удаление монотонных функций из ORDER BY. +- Удаление избыточных выражений из ORDER BY. +- Удаление из GROUP BY функций от других ключей GROUP BY. +- Удаление дублирующихся DISTINCT, ORDER BY из подзапросов. + ### 21.13. Fusion агрегатных функций. После или совместно с 21.11. @@ -906,6 +1098,10 @@ Amos Bird. Мария Нефедова, ВШЭ. +Constraints позволяют задать выражение, истинность которого проверяется при вставке данных в таблицу. Предположение о том, что выражение истинно, может использоваться и для оптимизации запросов. Например, встретив в запросе точно такое же выражение, можно заменить его на константу 1. + +Если выражение содержит равенство, то встретив в запросе одну из частей равенства, её можно заменить на другую часть равенства, если это сделает проще чтение данных или вычисление выражения. Например, задан constraint: `URLDomain = domain(URL)`. Значит, выражение `domain(URL)` можно заменить на `URLDomain`. + ### 21.15. Многоступенчатое чтение данных вместо PREWHERE. Требует 2.1 и 21.10. @@ -925,6 +1121,28 @@ Amos Bird. Василий Морозов, Арслан Гумеров, Альберт Кидрачев, ВШЭ. В прошлом году задачу начинал делать Евгений Правда, ВШЭ, но почти полностью не сделал её. +1. Оптимизация top sort. + +В ClickHouse используется неоптимальный вариант top sort. Суть его в том, что из каждого блока достаётся top N записей, а затем, все блоки мержатся. Но доставание top N записей у каждого следующего блока бессмысленно, если мы знаем, что из них в глобальный top N войдёт меньше. Конечно нужно реализовать вариацию на тему priority queue (heap) с быстрым пропуском целых блоков, если ни одна строка не попадёт в накопленный top. + +2. Рекурсивный вариант сортировки по кортежам. + +Для сортировки по кортежам используется обычная сортировка с компаратором, который в цикле по элементам кортежа делает виртуальные вызовы `IColumn::compareAt`. Это неоптимально - как из-за короткого цикла по неизвестному в compile-time количеству элементов, так и из-за виртуальных вызовов. Чтобы обойтись без виртуальных вызовов, есть метод `IColumn::getPermutation`. Он используется в случае сортировки по одному столбцу. Есть вариант, что в случае сортировки по кортежу, что-то похожее тоже можно применить... например, сделать метод `updatePermutation`, принимающий аргументы offset и limit, и допереставляющий перестановку в диапазоне значений, в которых предыдущий столбец имел равные значения. + +3. RadixSort для сортировки. + +Один наш знакомый начал делать задачу по попытке использования RadixSort для сортировки столбцов. Был сделан вариант indirect сортировки (для `getPermutation`), но не оптимизирован до конца - есть лишние ненужные перекладывания элементов. Для того, чтобы его оптимизировать, придётся добавить немного шаблонной магии (на последнем шаге что-то не копировать, вместо перекладывания индексов - складывать их в готовое место). Также этот человек добавил метод MSD Radix Sort для реализации radix partial sort. Но даже не проверил производительность. + +Наиболее содержательная часть задачи может состоять в применении Radix Sort для сортировки кортежей, расположенных в оперативке в виде Structure Of Arrays неизвестного в compile-time размера. Это может работать хуже, чем то, что описано в пункте 2... Но попробовать не помешает. + +4. Three-way comparison sort. + +Виртуальный метод `compareAt` возвращает -1, 0, 1. Но алгоритмы сортировки сравнениями обычно рассчитаны на `operator<` и не могут получить преимущества от three-way comparison. А можно ли написать так, чтобы преимущество было? + +5. pdq partial sort + +Хороший алгоритм сортировки сравнениями `pdqsort` не имеет варианта partial sort. Заметим, что на практике, почти все сортировки в запросах ClickHouse являются partial_sort, так как `ORDER BY` почти всегда идёт с `LIMIT`. Кстати, Данила Кутенин уже попробовал это и показал, что в тривиальном случае преимущества нет. Но не очевидно, что нельзя сделать лучше. + ### 21.20. Использование материализованных представлений для оптимизации запросов. В ByteDance есть готовая реализация, но они её боятся из-за, возможно, низкого качества кода. @@ -1099,38 +1317,89 @@ zhang2014. Антон Мамонов, УрФУ, Яндекс. +Внутри ClickHouse есть богатые возможности по интроспекции и профилированию. Эти возможности доступны через системные таблицы и использовать их приходится путём формулирования SQL запросов. Это неудобно. + +Вместо этого предлагается сделать, чтобы ClickHouse отдавал HTML страницу, реализующую интерактивный web-интерфейс со следующими возможностями: +- отображение состояния кластеров (какие кластеры известны, статус каждого сервера); +- графики нагрузки текущего сервера или выбранного сервера кластера; +- обновляемый список запросов; +- просмотр лога запросов с наиболее востребованными фильтрациями по одной кнопке; +- просмотр лога на кластере, например - последние ошибки; +- просмотр метрик использования ресурсов, flame graph и pprof-граф для выбранных запросов; +- отчёт по использованию кластера (пример: количество ядер CPU по пользователям за сегодня). + ### 24.2. Экспериментальные алгоритмы сжатия. Анастасия Наумова, ВШЭ. +ClickHouse поддерживает LZ4 и ZSTD для сжатия данных. Эти алгоритмы являются парето-оптимальными по соотношению скорости и коэффициентам сжатия среди достаточно известных. Тем не менее, существуют менее известные алгоритмы сжатия, которые могут превзойти их по какому-либо критерию. Из потенциально более быстрых по сравнимом коэффициенте сжатия: Lizard, LZSSE, density. Из более сильных: bsc и csc. Необходимо изучить эти алгоритмы, добавить их поддержку в ClickHouse и исследовать их работу на тестовых датасетах. + ### 24.3. Экспериментальные кодеки. Вероника Фалчикова, Лада Торчик, ВШЭ. +Существуют специализированные алгоритмы кодирования числовых последовательностей: Group VarInt, MaskedVByte, PFOR. Необходимо изучить наиболее эффективные реализации этих алгоритмов. Примеры вы сможете найти на https://github.com/lemire и https://github.com/powturbo/ а также https://github.com/schizofreny/middle-out + +Внедрить их в ClickHouse в виде кодеков и изучить их работу на тестовых датасетах. + ### 24.4. Шифрование в ClickHouse на уровне кусков данных. Yuchen Dong, ICS. +Данные в ClickHouse хранятся без шифрования. При наличии доступа к дискам, злоумышленник может прочитать данные. Предлагается реализовать два подхода к шифрованию: + +1. Шифрование блоков данных. +Шифрование данных столбцов на диске требуется реализовать в виде кодеков. Это позволит применять шифрование к отдельным столбцам; применять его после сжатия данных (эффективно, но менее безопасно) или без сжатия. Потребуется проработать работу с ключами: получение ключей из отдельного сервиса, правильная работа с ключами в оперативке. Отдельным вопросом стоит шифрование индексов. + ### 24.5. Поддержка функций шифрования для отдельных значений. Yuchen Dong, ICS. +Смотрите также 24.5. + +2. Шифрование отдельных значений. +Для этого требуется реализовать функции шифрования и расшифрования, доступные из SQL. Для шифрования реализовать возможность добавления нужного количества случайных бит для исключения одинаковых зашифрованных значений на одинаковых данных. Это позволит реализовать возможность "забывания" данных без удаления строк таблицы: можно шифровать данные разных клиентов разными ключами, и для того, чтобы забыть данные одного клиента, потребуется всего лишь удалить ключ. + ### 24.6. Userspace RAID. Глеб Новиков, ВШЭ. +RAID позволяет одновременно увеличить надёжность хранения данных на дисках и увеличить скорость работы дискового массива. Обычно RAID настраивается с помощью встроенных возможностей ядра Linux (mdraid) или с помощью hardware контроллера. У этого есть следующие ограничения: + +1. Иногда (в облачной инфраструктуре некоторых компаний) сервер предоставляется с отдельными дисками, подмонтированными в виде отдельных разделов (JBOD), без возможности создания RAID. + +2. В ClickHouse для обеспечения избыточности обычно используется репликация между серверами. Но при восстановлении одного из дисков RAID не используются данные с реплик, а в случае отказа одного из дисков в RAID-0, приходится передавать с реплики все данные, а не только данные, соответствующие одному из дисков. Это происходит, потому что RAID не интегрирован в ClickHouse и "не знает" про его особенности. + +3. Отсутствуют продвинутые варианты обеспечения избыточности, как например, LRC. + +Для преодоления этих ограничений, предлагается реализовать в ClickHouse встроенный алгоритм расположения данных на дисках. + ### 24.7. Вероятностные структуры данных для фильтрации по подзапросам. Рузель Ибрагимов, ВШЭ и Яндекс. +Частой задачей является выполнение запроса с фильтрацией по множеству, полученному по подзапросу. Пример: найти пользователей, которые заходили на сайт сегодня и заходили неделю назад. Это выражается в виде запроса: `SELECT UserID FROM table WHERE EventDate = today() AND UserID IN (SELECT ...)`. При выполнении этого запроса, сначала выполняется подзапрос в правой части `IN` и формируется хэш-таблица в оперативке; затем эта хэш-таблица используется для фильтрации. + +Иногда объём данных достаточно большой, и хэш-таблица не помещается в оперативку. В этом случае можно рассмотреть в качестве варианта приближённый рассчёт: найти пользователей, которые заходили на сайт сегодня и наверное заходили неделю назад. Для этого можно вместо хэш-таблицы использовать Bloom Filter. Другая задача: найти пользователей, которые встречались, скорее всего, не менее некоторого количества раз. Для этого можно использовать Counting Bloom Filter. Также следует изучить структуры данных Quotient Filter и Cuckoo Filer, а ещё - секретный алгоритм Chaotic Map от Андрея Плахова. + +Предлагается реализовать это в языке запросов ClickHouse с помощью специального синтаксиса, например `x IN BLOOM FILTER (n, m) (SELECT ...)`. + ### 24.8. Специализация векторизованного кода для AVX/AVX2/AVX512 и ARM NEON. Дмитрий Ковальков, ВШЭ и Яндекс. +Подавляющее большинство кода ClickHouse написана для x86_64 с набором инструкций до SSE 4.2 включительно. Лишь отдельные редкие функции поддерживают AVX/AVX2/AVX512 с динамической диспетчеризацией. + +В первой части задачи, следует добавить в ClickHouse реализации некоторых примитивов, оптимизированные под более новый набор инструкций. Например, AVX2 реализацию генератора случайных чисел pcg: https://github.com/lemire/simdpcg + +Во второй части задачи, предлагается адаптировать существующие куски кода, использующие SSE intrinsics на AVX/AVX2 и сравнить производительность. Также рассматривается оптимизация под ARM NEON. + ### 24.9. Общий подход к CPU dispatching в фабрике функций. Дмитрий Ковальков, ВШЭ и Яндекс. +Продолжение 24.8. + ### 24.10. Поддержка типов half/bfloat16/unum. Рустам Гусейн-заде, ВШЭ. @@ -1139,12 +1408,30 @@ Yuchen Dong, ICS. Игорь Минеев, ВШЭ. +ClickHouse предоставляет достаточно богатый набор встроенных функций языка запросов, но не позволяет пользователю добавлять свои функции без редактировния исходников и перекомпиляции системы. Это мотивировано следующими потенциальными проблемами: + +1. ClickHouse является array-oriented системой, и все функции внутри кода принимают для обработки целые массивы, а не отдельные значения. Это усложняет внутренний интерфейс и делает его менее удобным для пользователя. +2. Предоставление возможности подключения UDF в виде shared библиотек, потребовало бы фиксировать этот интерфейс или поддерживать обратную совместимость, тогда как мы бы хотели, при разработке ClickHouse, менять этот интерфейс по своему усмотрению без оглядки. +3. Сложность внутренних структур данных повышает вероятность ошибок типа buffer overflow и повреждения памяти, что сильно затруднит сопровождение ClickHouse с пользовательскими функциями. + +Тем не менее, можно выбрать более аккуратный подход, избегающий непосредственной линковки с shared библиотеками. + +Сначала можно реализовать поддержку UDF в виде выражений, составленных из простых функций ClickHouse. В ClickHouse есть встроенная кодогенерация на LLVM, что позволит таким функциям работать весьма эффективно. Но этот подход весьма ограничен и поэтому не является исчерпывающим. + +Затем предлагается реализовать поддержку UDF в виде исходников на C++, которые компилируются в runtime, с использованием заголовочных файлов ClickHouse. Требование компиляции из исходников вместо shared библиотек, позволит ослабить необходимость в поддержке совместимости ABI. + +Для безопасности, потребуется исследовать возможность размещения буферов данных в shared memory для выполнения UDF в отдельных процессах с изоляцией по памяти. Возможно, для этого пригодится интеграция с Apache Arrow. + +Также рассматривается возможность написания UDF на Rust, а также использование Web Assembly. Отдельно можно рассмотреть подключение NumPy и R и других технологий, которые предоставляют операции над целыми массивами. + ### 24.12. GPU offloading. Риск состоит в том, что даже известные GPU базы, такие как OmniSci, работают медленнее, чем ClickHouse. Преимущество возможно только на полной сортировке и JOIN. Алексей Соловей, nVidia и Рита Коннова, ВШЭ. +В компании nVidia сделали прототип offloading вычисления GROUP BY с некоторыми из агрегатных функций в ClickHouse и обещат предоставить исходники в публичный доступ для дальнейшего развития. Предлагается изучить этот прототип и расширить его применимость для более широкого сценария использования. В качестве альтернативы, предлагается изучить исходные коды системы `OmniSci` или `Alenka` или библиотеку `CUB` https://nvlabs.github.io/cub/ и применить некоторые из алгоритмов в ClickHouse. + ### 24.13. Stream запросы. Пререквизит для ClickHouse как CEP-системы. @@ -1181,14 +1468,32 @@ Yuchen Dong, ICS. Эльмир Марданов, ВШЭ. +ClickHouse является строго типизированной системой. Для того, чтобы прочитать данные в каком либо формате (например, CSV), требуется заранее указать типы данных. Если при чтении формата выясняется, что данные не могут быть прочитаны в рамках заданных типов, то кидается исключение. + +ClickHouse также может использоваться для быстрой аналитики по локальным файлам, без загрузки их в базу данных (программа `clickhouse-local`). В этом случае, его использование может заменить `awk`, `sed`, `grep`. Но остаётся неудобство - необходимость указания типов данных. + +Предлагается реализовать функциональность вывода типов по первому блоку данных путём применения эвристик и постепенного расширения типов. + +Другая экспериментальная задача - реализация эвристик для обработки данных в неизвестном построчном текстовом формате. Детектирование CSV, TSV, JSON, детектирование разделителей и форматов значений. + ### 24.23. Минимальная поддержка транзакций для множества вставок/чтений. Максим Кузнецов, ВШЭ. +Таблицы типа MergeTree состоят из набора независимых неизменяемых "кусков" данных. При вставках данных (INSERT), формируются новые куски. При модификациях данных (слияние кусков), формируются новые куски, а старые - становятся неактивными и перестают использоваться следующими запросами. Чтение данных (SELECT) производится из снэпшота множества кусков на некоторый момент времени. Таким образом, чтения и вставки не блокируют друг друга. + +Если же выполняется несколько запросов SELECT, то чтение данных может осуществляться из снэпшотов по состоянию на несколько разных моментов времени и быть неконсистентным. Пример: пользователю отображается отчёт из нескольких графиков и таблиц, но из-за того, что между разными запросами, данные успели обновиться, отображаемые данные не соответствуют друг другу. + +Пример с другой стороны - пользователь хочет осуществить несколько вставок (INSERT) в одну или несколько таблиц, но так, чтобы данные появились в них атомарно с точки зрения других запросов (SELECT). + +Для решения этих проблем, предлагается ввести глобальные метки времени для кусков данных (сейчас уже есть инкрементальные номера кусков, но они выделяются в рамках одной таблицы). Первым шагом сделаем эти метки времени в рамках сервера. Вторым шагом сделаем метки времени в рамках всех серверов, но неточные на основе локальных часов. Третьим шагом сделаем метки времени, выдаваемые сервисом координации. + ### 24.24. Реализация алгоритмов differential privacy. Артём Вишняков, ВШЭ. +https://github.com/yandex/ClickHouse/issues/6874 + ### 24.25. Интеграция в ClickHouse функциональности обработки HTTP User Agent. Есть хороший код в Яндекс.Метрике. Получено согласие от руководства. @@ -1202,6 +1507,8 @@ Yuchen Dong, ICS. ucasFL, ICS. +Алгоритмы min-hash и sim-hash позволяют вычислить для текста несколько хэш-значений таких, что при небольшом изменении текста, по крайней мере один из хэшей не меняется. Вычисления можно реализовать на n-грамах и словарных шинглах. Предлагается добавить поддержку этих алгоритмов в виде функций в ClickHouse и изучить их применимость для задачи нечёткого поиска полудубликатов. + ### 24.28. Другой sketch для квантилей. Похоже на quantileTiming, но с логарифмическими корзинами. @@ -1221,6 +1528,12 @@ Amos Bird, но его решение слишком громоздкое и п Мария Конькова, ВШЭ и Яндекс. Также смотрите 24.29. +В ClickHouse есть два основных протокола: родной протокол общения между серверами и HTTP/1.1 протокол. HTTP/1.1 протокол удобен для работы из самых разных языков программирования, но, в отличие от родного протокола, не поддерживает двусторонний обмен информацией во время запроса: +- передачу информации о прогрессе во время выполнения запроса; +- передачу логов во время выполнения запроса; +- отмену выполнения запроса в тот момент как данные ещё не начали передаваться; + +Рассматривается вариант - поддержка GRPC в ClickHouse. Здесь есть неочевидные моменты, такие как - эффективная передача массивов данных в column-oriented формате - насколько удобно будет обернуть это в GRPC. ## 25. DevRel @@ -1287,6 +1600,17 @@ Amos Bird, но его решение слишком громоздкое и п ### 25.16. Сайт play.clickhouse. +Цель состоит в реализации сайта, на котором можно попробовать задавать произвольные запросы к временному экземпляру ClickHouse и изучать его поведение. Из похожих проектов можно отметить: [Compiler Explorer](https://godbolt.org/), http://ideone.com/, [SQLFiddle](http://sqlfiddle.com/), [DB-Fiddle](https://www.db-fiddle.com/). + +С помощью такого сайта можно решать следующие задачи: +- ознакомление с языком запросов ClickHouse; +- демонстрация примеров из документации; +- демонстрация скорости работы на тестовых датасетах; +- сравнение поведения разных версий ClickHouse друг с другом; +- демонстрация неожиданного поведения или багов; + +Требуется проработать вопрос безопасности и изоляции инстансов (поднятие в контейнерах с ограничениями по сети), подключение тестовых датасетов с помощью copy-on-write файловой системы; органичения ресурсов. + ### 25.17. Взаимодействие с ВУЗами: ВШЭ, УрФУ, ICS Beijing. Алексей Миловидов и вся группа разработки @@ -1301,6 +1625,8 @@ Amos Bird, но его решение слишком громоздкое и п Матвей Бубнов, УрФУ +Существуют мало известные специализированные СУБД, способные конкурировать с ClickHouse по скорости обработки некоторых классов запросов. Пример: `TDEngine` и `DolphinDB`, `VictoriaMetrics`, а также `Apache Doris` и `LocustDB`. Предлагается изучить и классифицировать архитектурные особенности этих систем - их особенности и преимущества. Установить эти системы, загрузить тестовые данные, изучить производительность. Проанализировать, за счёт чего достигаются преимущества. + ### 25.21. Повторное награждение контрибьюторов в Китае. ### 25.22. On-site помощь с ClickHouse компаниям в дни рядом с мероприятиями. From ea13ed1949752e2c63a7973d7871c43651b05cb1 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Wed, 20 Nov 2019 08:04:00 -0400 Subject: [PATCH 65/90] Doc change. Partition key instead of old month (#7828) * Update insert_into.md Partition key instead of old month * Update insert_into.md Partition key instead of old month * Update insert_into.md Partition key instead of old month --- docs/en/query_language/insert_into.md | 4 ++-- docs/ru/query_language/insert_into.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/query_language/insert_into.md b/docs/en/query_language/insert_into.md index 2b361fd5a18..3383e90620e 100644 --- a/docs/en/query_language/insert_into.md +++ b/docs/en/query_language/insert_into.md @@ -61,10 +61,10 @@ However, you can delete old data using `ALTER TABLE ... DROP PARTITION`. ### Performance Considerations -`INSERT` sorts the input data by primary key and splits them into partitions by month. If you insert data for mixed months, it can significantly reduce the performance of the `INSERT` query. To avoid this: +`INSERT` sorts the input data by primary key and splits them into partitions by a partition key. If you insert data into several partitions at once, it can significantly reduce the performance of the `INSERT` query. To avoid this: - Add data in fairly large batches, such as 100,000 rows at a time. -- Group data by month before uploading it to ClickHouse. +- Group data by a partition key before uploading it to ClickHouse. Performance will not decrease if: diff --git a/docs/ru/query_language/insert_into.md b/docs/ru/query_language/insert_into.md index 88c548d394c..4cd14e21871 100644 --- a/docs/ru/query_language/insert_into.md +++ b/docs/ru/query_language/insert_into.md @@ -62,10 +62,10 @@ INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... ### Замечания о производительности -`INSERT` сортирует входящие данные по первичному ключу и разбивает их на партиции по месяцам. Если вы вставляете данные за разные месяцы вперемешку, то это может значительно снизить производительность запроса `INSERT`. Чтобы избежать этого: +`INSERT` сортирует входящие данные по первичному ключу и разбивает их на партиции по ключу партиционирования. Если вы вставляете данные в несколько партиций одновременно, то это может значительно снизить производительность запроса `INSERT`. Чтобы избежать этого: - Добавляйте данные достаточно большими пачками. Например, по 100 000 строк. -- Группируйте данные по месяцам самостоятельно перед загрузкой в ClickHouse. +- Группируйте данные по ключу партиционирования самостоятельно перед загрузкой в ClickHouse. Снижения производительности не будет, если: From 6ae19e69f24c3dbdc61c512f5c48632ce969b8b3 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Wed, 20 Nov 2019 08:07:53 -0400 Subject: [PATCH 66/90] Doc change. ATTACH PARTITION FROM (#7827) * Update alter.md ATTACH PARTITION FROM * Update alter.md ATTACH PARTITION FROM --- docs/en/query_language/alter.md | 18 ++++++++++++++++-- docs/ru/query_language/alter.md | 21 +++++++++++++++++++-- 2 files changed, 35 insertions(+), 4 deletions(-) diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index eee707fd176..e6667f3534a 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -189,7 +189,8 @@ The following operations with [partitions](../operations/table_engines/custom_pa - [DETACH PARTITION](#alter_detach-partition) – Moves a partition to the `detached` directory and forget it. - [DROP PARTITION](#alter_drop-partition) – Deletes a partition. - [ATTACH PART|PARTITION](#alter_attach-partition) – Adds a part or partition from the `detached` directory to the table. -- [REPLACE PARTITION](#alter_replace-partition) - Copies the data partition from one table to another. +- [ATTACH PARTITION FROM](#alter_attach-partition-from) – Copies the data partition from one table to another and adds. +- [REPLACE PARTITION](#alter_replace-partition) - Copies the data partition from one table to another and replaces. - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) - Resets the value of a specified column in a partition. - [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - Resets the specified secondary index in a partition. - [FREEZE PARTITION](#alter_freeze-partition) – Creates a backup of a partition. @@ -256,13 +257,26 @@ This query is replicated. Each replica checks whether there is data in the `deta So you can put data to the `detached` directory on one replica, and use the `ALTER ... ATTACH` query to add it to the table on all replicas. +#### ATTACH PARTITION FROM {#alter_attach-partition-from} + +```sql +ALTER TABLE table2 ATTACH PARTITION partition_expr FROM table1 +``` + +This query copies the data partition from the `table1` to `table2` adds data to exsisting in the `table2`. Note that data won't be deleted from `table1`. + +For the query to run successfully, the following conditions must be met: + +- Both tables must have the same structure. +- Both tables must have the same partition key. + #### REPLACE PARTITION {#alter_replace-partition} ```sql ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1 ``` -This query copies the data partition from the `table1` to `table2`. Note that data won't be deleted from `table1`. +This query copies the data partition from the `table1` to `table2` and replaces existing partition in the `table2`. Note that data won't be deleted from `table1`. For the query to run successfully, the following conditions must be met: diff --git a/docs/ru/query_language/alter.md b/docs/ru/query_language/alter.md index b4f1622e2dd..4c99b5152c1 100644 --- a/docs/ru/query_language/alter.md +++ b/docs/ru/query_language/alter.md @@ -188,7 +188,8 @@ ALTER TABLE [db].name DROP CONSTRAINT constraint_name; - [DETACH PARTITION](#alter_detach-partition) – перенести партицию в директорию `detached`; - [DROP PARTITION](#alter_drop-partition) – удалить партицию; - [ATTACH PARTITION|PART](#alter_attach-partition) – добавить партицию/кусок в таблицу из директории `detached`; -- [REPLACE PARTITION](#alter_replace-partition) – скопировать партицию из другой таблицы; +- [ATTACH PARTITION FROM](#alter_attach-partition-from) – скопировать партицию из другой таблицы; +- [REPLACE PARTITION](#alter_replace-partition) – скопировать партицию из другой таблицы с заменой; - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) – удалить все значения в столбце для заданной партиции; - [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - очистить построенные вторичные индексы для заданной партиции; - [FREEZE PARTITION](#alter_freeze-partition) – создать резервную копию партиции; @@ -255,13 +256,29 @@ ALTER TABLE visits ATTACH PART 201901_2_2_0; Это означает, что вы можете разместить данные в директории `detached` на одной реплике и с помощью запроса `ALTER ... ATTACH` добавить их в таблицу на всех репликах. +#### ATTACH PARTITION FROM {#alter_attach-partition-from} + +```sql +ALTER TABLE table2 ATTACH PARTITION partition_expr FROM table1 +``` + +Копирует партицию из таблицы `table1` в таблицу `table2` и добавляет к существующим данным `table2`. Данные из `table1` не удаляются. + +Следует иметь в виду: + +- Таблицы должны иметь одинаковую структуру. +- Для таблиц должен быть задан одинаковый ключ партиционирования. + +Подробнее о том, как корректно задать имя партиции, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr). + + #### REPLACE PARTITION {#alter_replace-partition} ```sql ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1 ``` -Копирует партицию из таблицы `table1` в таблицу `table2`. Данные из `table1` не удаляются. +Копирует партицию из таблицы `table1` в таблицу `table2` с заменой существующих данных в `table2`. Данные из `table1` не удаляются. Следует иметь в виду: From 68946cfd09c7e01cb4b905899732be56a59cec79 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 20 Nov 2019 15:17:25 +0300 Subject: [PATCH 67/90] Changed wording --- docs/ru/extended_roadmap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/extended_roadmap.md b/docs/ru/extended_roadmap.md index 4674f77092e..68caa508047 100644 --- a/docs/ru/extended_roadmap.md +++ b/docs/ru/extended_roadmap.md @@ -257,7 +257,7 @@ ClickHouse использует небольшое подмножество фу ### 5.5. LTS релизы. -Требует 7.5. Задачу хочет Метрика, Облако, БК, Маркет и Altinity. Задача полностью обречена на провал с практической точки зрения, но пользователям важен сам факт того, что она будет, а мы уважаем чувства пользователей. Первой LTS версией уже стала версия 19.14. +Требует 7.5. Задачу хочет Метрика, Облако, БК, Маркет и Altinity. Первой LTS версией уже стала версия 19.14. Хотя мы не рекомендуем использовать LTS версии в продакшене, пользователи найдут их полезными для себя. ## 6. Инструментирование. From c2849d0864802a8e1d249503a1453593f19e209d Mon Sep 17 00:00:00 2001 From: Andrei Bodrov Date: Wed, 20 Nov 2019 17:48:01 +0300 Subject: [PATCH 68/90] move to helpers --- dbms/src/DataStreams/IBlockInputStream.h | 13 ------------- dbms/src/DataStreams/IBlockOutputStream.h | 13 ------------- dbms/src/IO/ReadHelpers.h | 13 +++++++++++++ dbms/src/IO/WriteHelpers.h | 14 ++++++++++++++ dbms/src/Storages/StorageFile.cpp | 9 +++++---- dbms/src/Storages/StorageHDFS.cpp | 6 ++++-- dbms/src/Storages/StorageS3.cpp | 6 ++++-- dbms/src/Storages/StorageURL.cpp | 6 ++++-- 8 files changed, 44 insertions(+), 36 deletions(-) diff --git a/dbms/src/DataStreams/IBlockInputStream.h b/dbms/src/DataStreams/IBlockInputStream.h index 73098163833..dfa9194a6f9 100644 --- a/dbms/src/DataStreams/IBlockInputStream.h +++ b/dbms/src/DataStreams/IBlockInputStream.h @@ -5,9 +5,7 @@ #include #include #include -#include #include -#include #include #include @@ -230,17 +228,6 @@ public: /// Enable calculation of minimums and maximums by the result columns. void enableExtremes() { enabled_extremes = true; } - template - std::unique_ptr getBuffer(const DB::CompressionMethod method, Types... args) - { - if (method == DB::CompressionMethod::Gzip) - { - auto read_buf = std::make_unique(args...); - return std::make_unique(std::move(read_buf), method); - } - return std::make_unique(args...); - } - protected: /// Order is important: `table_locks` must be destroyed after `children` so that tables from /// which child streams read are protected by the locks during the lifetime of the child streams. diff --git a/dbms/src/DataStreams/IBlockOutputStream.h b/dbms/src/DataStreams/IBlockOutputStream.h index c90bd82215f..4d627d49698 100644 --- a/dbms/src/DataStreams/IBlockOutputStream.h +++ b/dbms/src/DataStreams/IBlockOutputStream.h @@ -3,8 +3,6 @@ #include #include #include -#include -#include #include @@ -65,17 +63,6 @@ public: */ void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); } - template - std::unique_ptr getBuffer(const DB::CompressionMethod method, Types... args) - { - if (method == DB::CompressionMethod::Gzip) - { - auto write_buf = std::make_unique(args...); - return std::make_unique(std::move(write_buf), method, 1 /* compression level */); - } - return std::make_unique(args...); - } - virtual void finalize() {} private: diff --git a/dbms/src/IO/ReadHelpers.h b/dbms/src/IO/ReadHelpers.h index c3935e1092d..affd0df8349 100644 --- a/dbms/src/IO/ReadHelpers.h +++ b/dbms/src/IO/ReadHelpers.h @@ -23,9 +23,11 @@ #include +#include #include #include #include +#include #ifdef __clang__ #pragma clang diagnostic push @@ -911,4 +913,15 @@ void skipToNextLineOrEOF(ReadBuffer & buf); /// Skip to next character after next unescaped \n. If no \n in stream, skip to end. Does not throw on invalid escape sequences. void skipToUnescapedNextLineOrEOF(ReadBuffer & buf); +template +std::unique_ptr getReadBuffer(const DB::CompressionMethod method, Types... args) +{ +if (method == DB::CompressionMethod::Gzip) +{ + auto read_buf = std::make_unique(args...); + return std::make_unique(std::move(read_buf), method); +} +return std::make_unique(args...); +} + } diff --git a/dbms/src/IO/WriteHelpers.h b/dbms/src/IO/WriteHelpers.h index cd21a379a75..02bb17c1b96 100644 --- a/dbms/src/IO/WriteHelpers.h +++ b/dbms/src/IO/WriteHelpers.h @@ -20,11 +20,13 @@ #include #include +#include #include #include #include #include #include +#include #include @@ -905,4 +907,16 @@ inline String toString(const T & x) writeText(x, buf); return buf.str(); } + +template +std::unique_ptr getWriteBuffer(const DB::CompressionMethod method, Types... args) +{ + if (method == DB::CompressionMethod::Gzip) + { + auto write_buf = std::make_unique(args...); + return std::make_unique(std::move(write_buf), method, 1 /* compression level */); + } + return std::make_unique(args...); +} + } diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 23b12cb7efb..097557acad3 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -8,6 +8,7 @@ #include #include +#include #include #include @@ -203,12 +204,12 @@ public: } storage->table_fd_was_used = true; - read_buf = getBuffer(compression_method, storage->table_fd); + read_buf = getReadBuffer(compression_method, storage->table_fd); } else { shared_lock = std::shared_lock(storage->rwlock); - read_buf = getBuffer(compression_method, file_path); + read_buf = getReadBuffer(compression_method, file_path); } reader = FormatFactory::instance().getInput(storage->format_name, *read_buf, storage->getSampleBlock(), context, max_block_size); @@ -285,13 +286,13 @@ public: * INSERT data; SELECT *; last SELECT returns only insert_data */ storage.table_fd_was_used = true; - write_buf = getBuffer(compression_method, storage.table_fd); + write_buf = getWriteBuffer(compression_method, storage.table_fd); } else { if (storage.paths.size() != 1) throw Exception("Table '" + storage.table_name + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED); - write_buf = getBuffer(compression_method, storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); + write_buf = getWriteBuffer(compression_method, storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); } writer = FormatFactory::instance().getOutput(storage.format_name, *write_buf, storage.getSampleBlock(), storage.context_global); diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 58b59dbeb16..00867134068 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -7,8 +7,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -61,7 +63,7 @@ public: UInt64 max_block_size, const CompressionMethod compression_method) { - auto read_buf = getBuffer(compression_method, uri); + auto read_buf = getReadBuffer(compression_method, uri); auto input_stream = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); reader = std::make_shared>(input_stream, std::move(read_buf)); @@ -106,7 +108,7 @@ public: const CompressionMethod compression_method) : sample_block(sample_block_) { - write_buf = getBuffer(compression_method, uri); + write_buf = getWriteBuffer(compression_method, uri); writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); } diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp index f574ad5130d..ed9173c52ec 100644 --- a/dbms/src/Storages/StorageS3.cpp +++ b/dbms/src/Storages/StorageS3.cpp @@ -6,7 +6,9 @@ #include #include +#include #include +#include #include @@ -39,7 +41,7 @@ namespace const CompressionMethod compression_method) : name(name_) { - read_buf = getBuffer(compression_method, uri, timeouts); + read_buf = getReadBuffer(compression_method, uri, timeouts); reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); } @@ -86,7 +88,7 @@ namespace const CompressionMethod compression_method) : sample_block(sample_block_) { - write_buf = getBuffer(compression_method, uri, min_upload_part_size, timeouts); + write_buf = getWriteBuffer(compression_method, uri, min_upload_part_size, timeouts); writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); } diff --git a/dbms/src/Storages/StorageURL.cpp b/dbms/src/Storages/StorageURL.cpp index bff4d115ead..4e6bc697f4e 100644 --- a/dbms/src/Storages/StorageURL.cpp +++ b/dbms/src/Storages/StorageURL.cpp @@ -5,8 +5,10 @@ #include #include +#include #include #include +#include #include @@ -56,7 +58,7 @@ namespace const CompressionMethod compression_method) : name(name_) { - read_buf = getBuffer(compression_method, uri, method, callback, timeouts, context.getSettingsRef().max_http_get_redirects); + read_buf = getReadBuffer(compression_method, uri, method, callback, timeouts, context.getSettingsRef().max_http_get_redirects); reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); } @@ -102,7 +104,7 @@ namespace const CompressionMethod compression_method) : sample_block(sample_block_) { - write_buf = getBuffer(compression_method, uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts); + write_buf = getWriteBuffer(compression_method, uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts); writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); } From decdd7d4761f058338340d55ffae3e37822a8714 Mon Sep 17 00:00:00 2001 From: Andrei Bodrov Date: Wed, 20 Nov 2019 18:40:42 +0300 Subject: [PATCH 69/90] fix typos --- dbms/src/DataStreams/IBlockOutputStream.h | 2 -- dbms/src/DataStreams/SquashingBlockOutputStream.h | 2 +- dbms/src/Storages/StorageHDFS.cpp | 1 + 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/dbms/src/DataStreams/IBlockOutputStream.h b/dbms/src/DataStreams/IBlockOutputStream.h index 4d627d49698..060438ba457 100644 --- a/dbms/src/DataStreams/IBlockOutputStream.h +++ b/dbms/src/DataStreams/IBlockOutputStream.h @@ -63,8 +63,6 @@ public: */ void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); } - virtual void finalize() {} - private: std::vector table_locks; }; diff --git a/dbms/src/DataStreams/SquashingBlockOutputStream.h b/dbms/src/DataStreams/SquashingBlockOutputStream.h index 2a8fb857457..7828ad7e96d 100644 --- a/dbms/src/DataStreams/SquashingBlockOutputStream.h +++ b/dbms/src/DataStreams/SquashingBlockOutputStream.h @@ -31,7 +31,7 @@ private: SquashingTransform transform; bool all_written = false; - void finalize() override; + void finalize(); bool disable_flush = false; }; diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 00867134068..15734dbfa14 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include From 6e2f9f6f6df88f06d1e2d2a8a26c9b3a50934354 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 20 Nov 2019 19:33:49 +0300 Subject: [PATCH 70/90] Invert flags in Settings.h: IGNORABLE flag is replaced with IMPORTANT flag. --- dbms/src/Core/Settings.h | 18 +++++++++--------- dbms/src/Core/SettingsCollection.h | 4 ++-- dbms/src/Core/SettingsCollectionImpl.h | 18 ++++++++++-------- 3 files changed, 21 insertions(+), 19 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 310860d3130..58d8e185adc 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -41,8 +41,8 @@ struct Settings : public SettingsCollection * Note: as an alternative, we could implement settings to be completely dynamic in form of map: String -> Field, * but we are not going to do it, because settings is used everywhere as static struct fields. * - * `flags` can be either 0 or IGNORABLE. - * A setting is "IGNORABLE" if it doesn't affects the results of the queries and can be ignored without exception. + * `flags` can be either 0 or IMPORTANT. + * A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions. */ #define LIST_OF_SETTINGS(M) \ @@ -72,7 +72,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.", 0) \ M(SettingUInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \ M(SettingUInt64, s3_min_upload_part_size, 512*1024*1024, "The mininum size of part to upload during multipart upload to S3.", 0) \ - M(SettingBool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", 0) \ + M(SettingBool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.", 0) \ M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \ M(SettingUInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \ @@ -90,7 +90,7 @@ struct Settings : public SettingsCollection \ M(SettingLoadBalancing, load_balancing, LoadBalancing::RANDOM, "Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing.", 0) \ \ - M(SettingTotalsMode, totals_mode, TotalsMode::AFTER_HAVING_EXCLUSIVE, "How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = ‘any’ are present.", 0) \ + M(SettingTotalsMode, totals_mode, TotalsMode::AFTER_HAVING_EXCLUSIVE, "How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = ‘any’ are present.", IMPORTANT) \ M(SettingFloat, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.", 0) \ \ M(SettingBool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \ @@ -146,7 +146,7 @@ struct Settings : public SettingsCollection \ M(SettingUInt64, log_queries_cut_to_length, 100000, "If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.", 0) \ \ - M(SettingDistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?", 0) \ + M(SettingDistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?", IMPORTANT) \ \ M(SettingUInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.", 0) \ \ @@ -179,7 +179,7 @@ struct Settings : public SettingsCollection M(SettingBool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \ M(SettingBool, input_format_with_names_use_header, false, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \ M(SettingBool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \ - M(SettingBool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", 0) \ + M(SettingBool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", IMPORTANT) \ M(SettingBool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ M(SettingBool, input_format_null_as_default, false, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \ \ @@ -209,10 +209,10 @@ struct Settings : public SettingsCollection M(SettingUInt64, input_format_allow_errors_num, 0, "Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \ M(SettingFloat, input_format_allow_errors_ratio, 0, "Maximum relative amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \ \ - M(SettingBool, join_use_nulls, 0, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", 0) \ + M(SettingBool, join_use_nulls, 0, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \ \ M(SettingJoinStrictness, join_default_strictness, JoinStrictness::ALL, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \ - M(SettingBool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys maping to be consistent with LEFT one.", 0) \ + M(SettingBool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys maping to be consistent with LEFT one.", IMPORTANT) \ \ M(SettingUInt64, preferred_block_size_bytes, 1000000, "", 0) \ \ @@ -307,7 +307,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, max_rows_in_join, 0, "Maximum size of the hash table for JOIN (in number of rows).", 0) \ M(SettingUInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).", 0) \ M(SettingOverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ - M(SettingBool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", 0) \ + M(SettingBool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", IMPORTANT) \ M(SettingBool, partial_merge_join, false, "Use partial merge join instead of hash join for LEFT and INNER JOINs.", 0) \ M(SettingBool, partial_merge_join_optimizations, false, "Enable optimizations in partial merge join", 0) \ M(SettingUInt64, default_max_bytes_in_join, 100000000, "Maximum size of right-side table if limit's required but max_bytes_in_join is not set.", 0) \ diff --git a/dbms/src/Core/SettingsCollection.h b/dbms/src/Core/SettingsCollection.h index 57bd42bc7b3..a7a28fef847 100644 --- a/dbms/src/Core/SettingsCollection.h +++ b/dbms/src/Core/SettingsCollection.h @@ -286,7 +286,7 @@ enum class SettingsBinaryFormat * { * # define APPLY_FOR_MYSETTINGS(M) \ * M(SettingUInt64, a, 100, "Description of a", 0) \ - * M(SettingFloat, f, 3.11, "Description of f", IGNORABLE) // IGNORABLE - means the setting can be ignored by older versions) \ + * M(SettingFloat, f, 3.11, "Description of f", IMPORTANT) // IMPORTANT - means the setting can't be ignored by older versions) \ * M(SettingString, s, "default", "Description of s", 0) * * DECLARE_SETTINGS_COLLECTION(MySettings, APPLY_FOR_MYSETTINGS) @@ -316,7 +316,7 @@ private: StringRef name; StringRef description; - bool is_ignorable; + bool is_important; IsChangedFunction is_changed; GetStringFunction get_string; GetFieldFunction get_field; diff --git a/dbms/src/Core/SettingsCollectionImpl.h b/dbms/src/Core/SettingsCollectionImpl.h index d6ff9573a4c..a97de0f48af 100644 --- a/dbms/src/Core/SettingsCollectionImpl.h +++ b/dbms/src/Core/SettingsCollectionImpl.h @@ -254,7 +254,7 @@ void SettingsCollection::serialize(WriteBuffer & buf, SettingsBinaryFor { details::SettingsCollectionUtils::serializeName(member.name, buf); if (format >= SettingsBinaryFormat::STRINGS) - details::SettingsCollectionUtils::serializeFlag(member.is_ignorable, buf); + details::SettingsCollectionUtils::serializeFlag(member.is_important, buf); member.serialize(castToDerived(), buf, format); } } @@ -272,18 +272,20 @@ void SettingsCollection::deserialize(ReadBuffer & buf, SettingsBinaryFo if (name.empty() /* empty string is a marker of the end of settings */) break; auto * member = the_members.find(name); - bool is_ignorable = (format >= SettingsBinaryFormat::STRINGS) ? details::SettingsCollectionUtils::deserializeFlag(buf) : false; + bool is_important = (format >= SettingsBinaryFormat::STRINGS) ? details::SettingsCollectionUtils::deserializeFlag(buf) : true; if (member) { member->deserialize(castToDerived(), buf, format); } - else if (is_ignorable) + else if (is_important) + { + details::SettingsCollectionUtils::throwNameNotFound(name); + } + else { details::SettingsCollectionUtils::warningNameNotFound(name); details::SettingsCollectionUtils::skipValue(buf); } - else - details::SettingsCollectionUtils::throwNameNotFound(name); } } @@ -298,8 +300,8 @@ void SettingsCollection::deserialize(ReadBuffer & buf, SettingsBinaryFo { \ LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_) \ }; \ - constexpr int IGNORABLE = 1; \ - UNUSED(IGNORABLE); \ + constexpr int IMPORTANT = 1; \ + UNUSED(IMPORTANT); \ LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_) \ } \ /** \ @@ -322,7 +324,7 @@ void SettingsCollection::deserialize(ReadBuffer & buf, SettingsBinaryFo #define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ add({StringRef(#NAME, strlen(#NAME)), StringRef(DESCRIPTION, strlen(DESCRIPTION)), \ - FLAGS & IGNORABLE, \ + FLAGS & IMPORTANT, \ [](const Derived & d) { return d.NAME.changed; }, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \ From c8def1b29efd1dc65b64e69129ffeee0be327c05 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 20 Nov 2019 17:38:42 +0100 Subject: [PATCH 71/90] Updated libunwind and protobuf to fix gcc-9 build and #7114 --- contrib/libunwind | 2 +- contrib/protobuf | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/libunwind b/contrib/libunwind index 96f5f17125a..68cffcbbd18 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 96f5f17125a3d8eed890e365868ccaa96bcd2301 +Subproject commit 68cffcbbd1840e14664a5f7f19c5e43f65c525b5 diff --git a/contrib/protobuf b/contrib/protobuf index 12735370922..0795fa6bc44 160000 --- a/contrib/protobuf +++ b/contrib/protobuf @@ -1 +1 @@ -Subproject commit 12735370922a35f03999afff478e1c6d7aa917a4 +Subproject commit 0795fa6bc443666068bec56bf700e1f488f592f1 From c913155d955ba0f6935931b45b08a323be04d4a5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 20 Nov 2019 20:24:44 +0300 Subject: [PATCH 72/90] Fix TSV segmentation. --- .../Impl/TabSeparatedRowInputFormat.cpp | 127 +++++++++--------- 1 file changed, 61 insertions(+), 66 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index fbe8ba58c9f..537fc6f690f 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -384,79 +384,74 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) } } -bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_bytes) +void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * ptr) { - for (;;) + assert(ptr >= in.position()); + assert(ptr <= in.buffer().end()); + + const int old_bytes = memory.size(); + const int additional_bytes = ptr - in.position(); + const int new_bytes = old_bytes + additional_bytes; + memory.resize(new_bytes); + memcpy(memory.data() + old_bytes, in.position(), additional_bytes); + in.position() = ptr; +} + +bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & ptr) +{ + assert(ptr <= in.buffer().end()); + + if (ptr < in.buffer().end()) { - if (in.eof()) - { - return false; - } - - const auto old_total_bytes = memory.size(); - - // Calculate the minimal amount of bytes we must read for this chunk. - // The chunk size may be already bigger than the required minimum, if - // we have a giant row and still haven't read up to the separator. - const auto min_bytes_needed = (min_chunk_bytes >= old_total_bytes) - ? min_chunk_bytes - old_total_bytes : 0; - - // The start position might be over the in.buffer().end(), it's OK -- - // find_first_symbols will process this correctly and return - // in.buffer().end(). - //char * next_separator = in.position() + min_bytes_needed; - bool found_separator = false; - char * chunk_end = in.position() + min_bytes_needed; - // Loop to skip the escaped line separators. - for (;;) - { - const auto next_separator = find_first_symbols<'\r', '\n'>(chunk_end, - in.buffer().end()); - assert(next_separator <= in.buffer().end()); - - if (next_separator == in.buffer().end()) - { - // Got to end of buffer, return it. - chunk_end = in.buffer().end(); - break; - } - - chunk_end = next_separator + 1; - - // We found a line separator character, check whether it is escaped by - // checking if there is a '\' to the left. The previous character may - // have been read on the previous loop, in this case we read it from - // 'memory' buffer. - if ((next_separator > in.position() && *(next_separator - 1) != '\\') - || (next_separator == in.position() && memory[memory.size() - 1] != '\\')) - { - found_separator = true; - break; - } - // This is an escaped separator, loop further. - } - - const auto bytes_read_now = chunk_end - in.position(); - const auto new_total_bytes = old_total_bytes + bytes_read_now; - memory.resize(new_total_bytes); - memcpy(memory.data() + old_total_bytes, in.position(), bytes_read_now); - - in.position() = chunk_end; - - if (found_separator && new_total_bytes >= min_chunk_bytes) - { - // Found the separator and the chunk big enough so that we can - // return it. - return true; - } - // Didn't find the separator, or the chunk is not big enough. Read more - // from the file. + return true; } + + saveUpToPosition(in, memory, in.buffer().end()); + bool loaded_more = !in.eof(); + assert(in.position() == in.buffer().begin()); + ptr = in.position(); + return loaded_more; +} + +bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) +{ + bool need_more_data = true; + char * pos = in.position(); + while (loadAtPosition(in, memory, pos) && need_more_data) + { + pos = find_first_symbols<'\\', '\r', '\n'>(pos, in.buffer().end()); + if (pos == in.buffer().end()) + { + continue; + } + + if (*pos == '\\') + { + ++pos; + if (loadAtPosition(in, memory, pos)) + { + ++pos; + } + } + else if (*pos == '\n' || *pos == '\r') + { + if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) + { + need_more_data = false; + } + + ++pos; + } + } + saveUpToPosition(in, memory, pos); + + return loadAtPosition(in, memory, pos); } void registerFileSegmentationEngineTabSeparated(FormatFactory & factory) { - for (auto name : {"TabSeparated", "TSV"}) + // We can use the same segmentation engine for TSKV. + for (auto name : {"TabSeparated", "TSV", "TSKV"}) { factory.registerFileSegmentationEngine(name, &fileSegmentationEngineTabSeparatedImpl); } From b8438de5048ade0f9b2ec7aabc523a8293b659f9 Mon Sep 17 00:00:00 2001 From: pufit <19871338+pufit@users.noreply.github.com> Date: Wed, 20 Nov 2019 21:17:49 +0300 Subject: [PATCH 73/90] typo MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit неприемлимо -> неприемлемо --- docs/ru/extended_roadmap.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/extended_roadmap.md b/docs/ru/extended_roadmap.md index 68caa508047..42399f8b589 100644 --- a/docs/ru/extended_roadmap.md +++ b/docs/ru/extended_roadmap.md @@ -180,7 +180,7 @@ ClickHouse использует небольшое подмножество фу ### 2.14. Все функции с состоянием переделать на FunctionBuilder. -Долг [Николай Кочетов](https://github.com/KochetovNicolai). Сейчас код находится в переходном состоянии, что неприемлимо. +Долг [Николай Кочетов](https://github.com/KochetovNicolai). Сейчас код находится в переходном состоянии, что неприемлемо. ### 2.15. Функция subscribe для IStorage. @@ -199,7 +199,7 @@ ClickHouse использует небольшое подмножество фу Требует 3.1. -### 3.3. Исправить катастрофически отвратительно неприемлимый поиск по документации. +### 3.3. Исправить катастрофически отвратительно неприемлемый поиск по документации. [Иван Блинков](https://github.com/blinkov/) - очень хороший человек. Сам сайт документации основан на треш-технологиях, которые трудно исправить. From d2c39b40d577bf8ff74a791f6b8726bb00c294a5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 20 Nov 2019 23:55:26 +0300 Subject: [PATCH 74/90] Updated roadmap --- docs/ru/extended_roadmap.md | 89 +++++++++++++++++++------------------ 1 file changed, 46 insertions(+), 43 deletions(-) diff --git a/docs/ru/extended_roadmap.md b/docs/ru/extended_roadmap.md index 68caa508047..9089b291c9f 100644 --- a/docs/ru/extended_roadmap.md +++ b/docs/ru/extended_roadmap.md @@ -1,8 +1,8 @@ # Планы разработки ClickHouse 2020. -Здесь собраны важные задачи на 2020 год. Также многие из них присутствуют в GitHub Issues или (редко) в Яндекс.Трекере, а также (очень редко) в Яндекс.Goals. Мы предпочитаем использовать Яндекс.Трекер только для секретных задач, а все публичные задачи размещать в GitHub Issues для того, чтобы сделать разработку более открытой наружу. В любом случае, для работы над задачами, таск-трекеры уступают в удобстве plaintext файлу. +Здесь собраны важные задачи на 2020 год. Многие из них присутствуют в GitHub Issues. Данный текст следует рассматривать как рабочий черновик со сводкой и кратким описанием задач, ссылками и материалами для быстрого доступа на одной странице. Классификация задач условная. -Классификация задач условная и за её основу взята известная [классификация животных](https://ru.wikipedia.org/wiki/%D0%9A%D0%BB%D0%B0%D1%81%D1%81%D0%B8%D1%84%D0%B8%D0%BA%D0%B0%D1%86%D0%B8%D1%8F_%D0%B6%D0%B8%D0%B2%D0%BE%D1%82%D0%BD%D1%8B%D1%85_(%D0%91%D0%BE%D1%80%D1%85%D0%B5%D1%81)). +Так как ClickHouse - open-source продукт, мы хотим, чтобы рабочий процесс был также максимально открытым. В связи с этим, вам следует ожидать наличия на данной странице несколько больше деталей описания рабочего процесса, чем вы могли бы предположить - настолько близко, насколько рабочий процесс видят разработчики. Так как неотъемлимой частью процесса разработки является исправление недостатков продукта и улучшение качества кода, на данной странице вы найдёте весьма подробные описания таких деталей. Для других open-source продуктов такой подход к разработке обычно нехарактерен. Благодаря тому, что для каждой задачи указаны её зависимости, вы сможете понять, какие подготовительные работы требуются, что позволяет более точно понимать сроки реализации. ## 1. Хранение данных, индексация. @@ -14,7 +14,7 @@ Задача "normalized z-Order curve" в перспективе может быть полезна для БК и Метрики, так как позволяет смешивать OrderID и PageID и избежать дублирования данных. В задаче также вводится способ индексации путём обращения функции нескольких аргументов на интервале, что имеет смысл для дальнейшего развития. -Изначально делал [Андрей Чулков](https://github.com/achulkov2), ВШЭ, теперь доделывает [Ольга Хвостикова](https://github.com/stavrolia), но задача прокрастинирована из-за задачи 25.9. Сейчас оказалось, что задача 25.9 тоже не готова. Будем надеятся на реализацию к концу ноября. Впрочем, [Андрей Чулков](https://github.com/achulkov2) скоро выйдет на работу стажёром на пол ставки и сможет помочь её доделать. +Изначально делал [Андрей Чулков](https://github.com/achulkov2), ВШЭ, теперь доделывает [Ольга Хвостикова](https://github.com/stavrolia), но сроки немного сдвинуты из-за задачи 25.9. Будем надеятся на реализацию к концу ноября. Впрочем, [Андрей Чулков](https://github.com/achulkov2) скоро выйдет на работу стажёром на пол ставки и сможет помочь её доделать. ### 1.2. Wait-free каталог баз данных. @@ -86,7 +86,7 @@ ### 1.11. Виртуальная файловая система. -Нужно для Яндекс.Облака. Делает Александр Бурмак, Яндекс.Облако, а также Олег Ершов, ВШЭ и Яндекс. +Нужно для Яндекс.Облака. Делает Александр, Яндекс.Облако, а также Олег Ершов, ВШЭ и Яндекс. ClickHouse использует для хранения данных локальную файловую систему. Существует сценарий работы, в котором размещение старых (архивных) данных было бы выгодно на удалённой файловой системе. Если файловая система POSIX совместимая, то это не составляет проблем: ClickHouse успешно работает с Ceph, GlusterFS, MooseFS. Также востребованным является сценарий использования S3 (из-за доступности в облаке) или HDFS (для интеграции с Hadoop). Но эти файловые системы не являются POSIX совместимыми. Хотя для них существуют FUSE драйверы, но скорость работы сильно страдает и поддержка неполная. @@ -95,7 +95,7 @@ ClickHouse использует небольшое подмножество фу ### 1.12. Экспериментальная реализация VFS поверх S3 и HDFS. Нужно для Яндекс.Облака. Требует 1.11. Желательно 1.6 и 1.18. -Делает Александр Бурмак, Яндекс.Облако (сначала часть для S3), а также Олег Ершов, ВШЭ и Яндекс. +Делает Александр, Яндекс.Облако (сначала часть для S3), а также Олег Ершов, ВШЭ и Яндекс. ### 1.13. Ускорение запросов с FINAL. @@ -156,7 +156,7 @@ ClickHouse использует небольшое подмножество фу ### 2.8. Декларативный парсер запросов. -Низкий приоритет. Задачу хочет сделать [Иван Лежанкин](https://github.com/abyss7) в свободное время, но за последний год никаких поползновений не видно. +Низкий приоритет. Задачу хочет сделать [Иван Лежанкин](https://github.com/abyss7) в свободное время, но пока ничего нет. ### 2.9. Логгировние в format-стиле. @@ -201,7 +201,7 @@ ClickHouse использует небольшое подмножество фу ### 3.3. Исправить катастрофически отвратительно неприемлимый поиск по документации. -[Иван Блинков](https://github.com/blinkov/) - очень хороший человек. Сам сайт документации основан на треш-технологиях, которые трудно исправить. +[Иван Блинков](https://github.com/blinkov/) - очень хороший человек. Сам сайт документации основан на технологиях, не удовлетворяющих требованиям задачи, и эти технологии трудно исправить. ### 3.4. Добавить японский язык в документацию. @@ -226,7 +226,7 @@ ClickHouse использует небольшое подмножество фу ### 4.3. Ограничение числа одновременных скачиваний с реплик. -Изначально делал Олег Алексеенков, но решение оказалось неудачным, хотя там не так уж много доделывать. +Изначально делал Олег Алексеенков, но пока решение не готово, хотя там не так уж много доделывать. ### 4.4. Ограничение сетевой полосы при репликации. @@ -257,14 +257,14 @@ ClickHouse использует небольшое подмножество фу ### 5.5. LTS релизы. -Требует 7.5. Задачу хочет Метрика, Облако, БК, Маркет и Altinity. Первой LTS версией уже стала версия 19.14. Хотя мы не рекомендуем использовать LTS версии в продакшене, пользователи найдут их полезными для себя. +Требует 7.5. Задачу хочет Метрика, Облако, БК, Маркет и Altinity. Первой LTS версией уже стала версия 19.14. ## 6. Инструментирование. ### 6.1. Исправления сэмплирующего профайлера запросов. -[Никита Лапков](https://github.com/laplab), ноябрь 2019. Может не сделать, тогда будет делать Алексей Миловидов. +Михаил Филимонов, Altinity. Ноябрь 2019. ### 6.2. Добавление memory profiler. @@ -311,7 +311,7 @@ ClickHouse использует небольшое подмножество фу ### 7.2. LLVM в submodules. Уже добавлено, но старой версии, и поэтому не используется. Надо обновить. -Георгий Кондратьев. Возможно, он не сможет сделать эту задачу, тогда будет делать Алексей Миловидов. +Георгий - очень опытный разработчик, либо будет делать Алексей Миловидов. ### 7.3. Обновление Poco. @@ -328,7 +328,7 @@ ClickHouse использует небольшое подмножество фу ### 7.6. Правильный статистический тест для comparison mode в clickhouse-performance-test. -Задачу начал делать Дмитрий Рубашкин. Сейчас продолжает [Александр Кузьменков](https://github.com/akuzm). +Задачу начал делать Дмитрий Рубашкин (ВШЭ). Сейчас продолжает [Александр Кузьменков](https://github.com/akuzm). ### 7.7. Доделать тесты под MSan. @@ -337,7 +337,7 @@ ClickHouse использует небольшое подмножество фу ### 7.8. Добавить clang-tidy. -Уже есть PVS-Studio, но этого недостаточно. +Уже есть PVS-Studio. Мы очень довольны, но этого недостаточно. ### 7.9. Проверки на стиль имён с помощью clang-tidy. @@ -390,7 +390,7 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э ### 7.15. Замена libressl обратно на openssl. -Поводом использования libressl послужило желание Константина podshumok Игнатова из QRator и то, что тогда openssl был опозорен и libressl считалась адекватной альтернативой. Но сейчас ситуация изменилась - openssl продолжает развиваться, а libressl не особо, и можно спокойно менять обратно. +Поводом использования libressl послужило желание нашего хорошего друга из известной компании несколько лет назад. Но сейчас ситуация состоит в том, что openssl продолжает развиваться, а libressl не особо, и можно спокойно менять обратно. ### 7.16. tzdata внутри бинарника. @@ -470,12 +470,12 @@ Fuzzing тестирование - это тестирование случай ### 7.25. Синхронизация релизов в Аркадию. -Изначально занимался Олег Алексеенков. Сейчас он перешёл работать в Яндекс.Морду, но обещает продолжать синхронизацию. -Затем, возможно, [Иван Лежанкин](https://github.com/abyss7). Но сейчас приостановлено, так как Максим Ахмедов должен исправить регрессию производительности в анализе индекса. +Изначально занимался Олег Алексеенков. Сейчас он перешёл работать в дружественный отдел, но обещает продолжать синхронизацию. +Затем, возможно, [Иван Лежанкин](https://github.com/abyss7). Но сейчас приостановлено, так как Максим из YT должен исправить регрессию производительности в анализе индекса. ### 7.26. Побайтовая идентичность репозитория с Аркадией. -Команда DevTools. Фактически никто ничего не делает. +Команда DevTools. Прогресс по задаче под вопросом. ### 7.27. Запуск автотестов в Аркадии. @@ -523,8 +523,7 @@ Fuzzing тестирование - это тестирование случай ### 7.37. Разобраться с repo.yandex.ru. -Есть жалобы на скорость загрузки. Подозрение, что repo.yandex.ru не является нормальным CDN. Отсутствует простой доступ к мониторингу и логам. -Очень редко бывает нужно удалить пакет, но сделать это можно только через одного человека. +Есть жалобы на скорость загрузки и неудобство maintenance, operations, visibility. ## 8. Интеграция с внешними системами. @@ -626,12 +625,12 @@ ClickHouse предоставляет возможность обратитьс ### 10.1. Исправление зависания в библиотеке доступа к YT. -Библиотека для доступа к YT обладает катастрофически отвратительно неприемлимым поведением и не переживает учения. +Библиотека для доступа к YT не переживает учения. Нужно для БК и Метрики. Поиск причин - [Александр Сапин](https://github.com/alesapin). Дальшейшее исправление возможно на стороне YT. ### 10.2. Исправление SIGILL в библиотеке доступа к YT. -Код YT зачем-то использует SIGILL вместо abort. Это, опять же, происходит при учениях. +Код YT использует SIGILL вместо abort. Это, опять же, происходит при учениях. Нужно для БК и Метрики. Поиск причин - [Александр Сапин](https://github.com/alesapin). Дальшейшее исправление возможно на стороне YT. ### 10.3. Возможность чтения данных из статических таблиц в YT словарях. @@ -640,7 +639,7 @@ ClickHouse предоставляет возможность обратитьс ### 10.4. Словарь из YDB (KikiMR). -Нужно для Метрики, а делать будет таинственный незнакомец из команды KikiMR. Или он сейчас это прочитает и скажет "я никогда не буду делать эту задачу". +Нужно для Метрики, а делать будет таинственный незнакомец из команды KikiMR (под вопросом). ### 10.5. Закрытие соединений и уменьшение числа соединений для MySQL и ODBC. @@ -658,7 +657,7 @@ ClickHouse предоставляет возможность обратитьс ### 10.9. Уменьшение блокировок для cache словарей за счёт одновременных запросов одного и того же. -Нужно для БК, но на самом деле они так просто думают, а все проблемы из-за низко-качественной библиотеки для доступа в YT. +Нужно для БК, но мотивация задачи находится под вопросом, так как есть рабочее предположение о том, что данная задача не устраняет причину проблемы. ### 10.10. Возможность использования старых значений из cache словаря пока они перезапрашиваются. @@ -711,7 +710,7 @@ ClickHouse предоставляет возможность обратитьс ### 11.5. Поддержка TLS в clickhouse-cpp. -Библиотеку clickhouse-cpp разрабатывал Павел Артёмкин в свободное время. +А знаете ли вы, что библиотеку clickhouse-cpp разрабатывал один хороший человек в свободное время? ### 11.6. Интеграционные тесты clickhouse-cpp. @@ -792,7 +791,7 @@ zhang2014 ### 14.5. Поддержка задания множества как массива в правой части секции IN. -Василий Немков, Altinity, делал эту задачу, но зачем-то забросил её. +Василий Немков, Altinity, делал эту задачу, но временно приостановил работу над ней в пользу других задач. ### 14.6. Глобальный scope для WITH. @@ -885,7 +884,7 @@ zhang2014 ### 16.1. DateTime64. -Василий Немков. +Василий Немков, Altinity, декабрь 2019. ### 16.2. Тип данных для JSON. @@ -975,7 +974,7 @@ ClickHouse не является geospatial СУБД. Тем не менее, в ### 19.3. Подключение YT Cypress или YDB как альтернативы ZooKeeper. -Hold. Полезно для Яндекс.Облака и БК, но есть риски, что будет вредно, а не полезно. +Hold. Полезно для заказчиков внутри Яндекса, но есть риски. ### 19.4. internal_replication = 'auto'. @@ -1057,7 +1056,7 @@ Hold. Полезно для Яндекс.Облака и БК, но есть р ### 21.8.1. Отдельный аллокатор для кэшей с ASLR. -В прошлом году задачу пытался сделать Данила Кутенин с помощью lfalloc из Аркадии и mimalloc из Microsoft, но оба решения оказались неудачными. Успешная реализация задачи 21.8 отменит необходимость в этой задаче, поэтому холд. +В прошлом году задачу пытался сделать Данила Кутенин с помощью lfalloc из Аркадии и mimalloc из Microsoft, но оба решения не были квалифицированы для использования в продакшене. Успешная реализация задачи 21.8 отменит необходимость в этой задаче, поэтому холд. ### 21.9. Исправить push-down выражений с помощью Processors. @@ -1119,7 +1118,7 @@ Constraints позволяют задать выражение, истиннос ### 21.19. Оптимизация сортировки. Василий Морозов, Арслан Гумеров, Альберт Кидрачев, ВШЭ. -В прошлом году задачу начинал делать Евгений Правда, ВШЭ, но почти полностью не сделал её. +В прошлом году задачу начинал делать другой человек, но не добился достаточного прогресса. 1. Оптимизация top sort. @@ -1204,10 +1203,12 @@ zhang2014. [Виталий Баранов](https://github.com/vitlibar), почти всё готово. -### 22.12. Исправление катастрофически отвратительно неприемлимо низкой производительности чтения из Kafka. +### 22.12. Исправление низкой производительности чтения из Kafka. [Иван Лежанкин](https://github.com/abyss7). +Для ClickHouse нехарактерно наличие кода, обладающего столь низкой производительностью. Практики разработки не подразумевают, что такой код должен попасть в продакшен без надлежащего тестирования производительности. + ### 22.13. Посмотреть, почему не работают некоторые collations. [Иван Лежанкин](https://github.com/abyss7), совмещается с 7.1. @@ -1218,9 +1219,11 @@ zhang2014. [Иван Лежанкин](https://github.com/abyss7), если он не сдастся. -### 22.16. Исправление катастрофически отвратительно неприемлимо низкой производительности кодека DoubleDelta. +### 22.16. Исправление низкой производительности кодека DoubleDelta. -Василий Немков, Altinity - сейчас старательно динамит эту задачу. +Василий Немков, Altinity - временно приостановлено, но намерения остаются в силе. + +Мы считаем важным, что код в ClickHouse содержит разумные оптимизации, основанные на анализе производительности. Но иногда бывают досадные исключения. ### 22.17. Консистентно работающий POPULATE для MaterializedView. @@ -1264,7 +1267,7 @@ zhang2014. ### 22.28. Изучить и исправить поведение работы с Kafka при ребалансировке. -[Иван Лежанкин](https://github.com/abyss7), если он не сдастся. +[Иван Лежанкин](https://github.com/abyss7). ## 23. Default Festival. @@ -1288,7 +1291,7 @@ zhang2014. ### 23.5. Включение compile_expressions. -Требует 7.2. Задачу изначально делал Денис Скоробогатов, ВШЭ и Яндекс, затем доделывал Алексей Миловидов, а затем [Александр Сапин](https://github.com/alesapin). +Требует 7.2. Задачу изначально на 99% сделал Денис Скоробогатов, ВШЭ и Яндекс. Остальной процент доделывал Алексей Миловидов, а затем [Александр Сапин](https://github.com/alesapin). ### 23.6. Включение учёта порядка столбцов в CSV. @@ -1308,7 +1311,7 @@ zhang2014. ### 23.10. Включение mlock бинарника. -Возможность mlock бинарника сделал Олег Алексеенков. Поможет, когда на серверах кроме ClickHouse работает много треш-программ. +Возможность mlock бинарника сделал Олег Алексеенков. Поможет, когда на серверах кроме ClickHouse работает много посторонних программ (мы иногда называем их в шутку "треш-программами"). ## 24. Экспериментальные задачи. @@ -1544,25 +1547,25 @@ Amos Bird, но его решение слишком громоздкое и п ### 25.2. Вычитка и выкладка статьи про обфускацию данных на английском. -Эми Жанель Кришниевски, Александр Казаков, Алексей Миловидов, ноябрь 2019. +Эми, Александр Казаков, Алексей Миловидов, ноябрь 2019. ### 25.3. Подготовка статьи "Секреты оптимизации производительности ClickHouse". -Алексей Миловидов, Леонид Клюев. +Алексей Миловидов, Леонид. ### 25.4. Подготовка статьи "Профайлер запросов: трудный путь". -Алексей Миловидов, Леонид Клюев. +Алексей Миловидов, Леонид. ### 25.5. Подготовка статьи "Тестирование ClickHouse, которое мы заслужили". ### 25.6. Перевод этих статей на английский. -Требует 25.3, 25.4, 25.5. Эми Жанель Кришниевски +Требует 25.3, 25.4, 25.5. Эми ### 25.7. Перевод статьи Данилы Кутенина на английский. -Эми Жанель Кришниевски +Эми ### 25.8. Выступление keynote на BDTC. @@ -1576,7 +1579,7 @@ Amos Bird, но его решение слишком громоздкое и п ### 25.10. Митапы в России и Беларуси: Москва x2 + митап для разработчиков или хакатон, Санкт-Петербург, Минск, Нижний Новгород, Екатеринбург, Новосибирск и/или Академгородок, Иннополис или Казань. -Екатерина Миназова - организация +Екатерина - организация ### 25.11. Митапы зарубежные: восток США (Нью Йорк, возможно Raleigh), возможно северо-запад (Сиэтл), Китай (Пекин снова, возможно митап для разработчиков или хакатон), Лондон. @@ -1594,7 +1597,7 @@ Amos Bird, но его решение слишком громоздкое и п Алексей Миловидов и все подготовленные докладчики -### 25.15. Конференции зарубежные: Percona, DataOps, возможно Big Data Warsaw, попытка попасть на более крупные. +### 25.15. Конференции зарубежные: Percona, DataOps, попытка попасть на более крупные. Алексей Миловидов и все подготовленные докладчики @@ -1649,4 +1652,4 @@ Amos Bird, но его решение слишком громоздкое и п ### 25.27. Обновить сайт ClickHouse. -Иван Блинков. Есть риск, что станет хуже. +Иван Блинков. Есть риски. From 533cb088e6e120e3d571a06b01f44120630864e2 Mon Sep 17 00:00:00 2001 From: Andrei Bodrov Date: Wed, 20 Nov 2019 23:57:34 +0300 Subject: [PATCH 75/90] std::forward in template --- dbms/src/IO/ReadHelpers.h | 4 ++-- dbms/src/IO/WriteHelpers.h | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/IO/ReadHelpers.h b/dbms/src/IO/ReadHelpers.h index affd0df8349..e53346c606e 100644 --- a/dbms/src/IO/ReadHelpers.h +++ b/dbms/src/IO/ReadHelpers.h @@ -914,11 +914,11 @@ void skipToNextLineOrEOF(ReadBuffer & buf); void skipToUnescapedNextLineOrEOF(ReadBuffer & buf); template -std::unique_ptr getReadBuffer(const DB::CompressionMethod method, Types... args) +std::unique_ptr getReadBuffer(const DB::CompressionMethod method, Types&&... args) { if (method == DB::CompressionMethod::Gzip) { - auto read_buf = std::make_unique(args...); + auto read_buf = std::make_unique(std::forward(args)...); return std::make_unique(std::move(read_buf), method); } return std::make_unique(args...); diff --git a/dbms/src/IO/WriteHelpers.h b/dbms/src/IO/WriteHelpers.h index 02bb17c1b96..0163a3c3740 100644 --- a/dbms/src/IO/WriteHelpers.h +++ b/dbms/src/IO/WriteHelpers.h @@ -909,11 +909,11 @@ inline String toString(const T & x) } template -std::unique_ptr getWriteBuffer(const DB::CompressionMethod method, Types... args) +std::unique_ptr getWriteBuffer(const DB::CompressionMethod method, Types&&... args) { if (method == DB::CompressionMethod::Gzip) { - auto write_buf = std::make_unique(args...); + auto write_buf = std::make_unique(std::forward(args)...); return std::make_unique(std::move(write_buf), method, 1 /* compression level */); } return std::make_unique(args...); From 5f507f43bf2cef4835b55f5cf6f4db442679bcb4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 20 Nov 2019 23:57:38 +0300 Subject: [PATCH 76/90] Updated roadmap --- docs/ru/extended_roadmap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/extended_roadmap.md b/docs/ru/extended_roadmap.md index 9089b291c9f..184454aa071 100644 --- a/docs/ru/extended_roadmap.md +++ b/docs/ru/extended_roadmap.md @@ -2,7 +2,7 @@ Здесь собраны важные задачи на 2020 год. Многие из них присутствуют в GitHub Issues. Данный текст следует рассматривать как рабочий черновик со сводкой и кратким описанием задач, ссылками и материалами для быстрого доступа на одной странице. Классификация задач условная. -Так как ClickHouse - open-source продукт, мы хотим, чтобы рабочий процесс был также максимально открытым. В связи с этим, вам следует ожидать наличия на данной странице несколько больше деталей описания рабочего процесса, чем вы могли бы предположить - настолько близко, насколько рабочий процесс видят разработчики. Так как неотъемлимой частью процесса разработки является исправление недостатков продукта и улучшение качества кода, на данной странице вы найдёте весьма подробные описания таких деталей. Для других open-source продуктов такой подход к разработке обычно нехарактерен. Благодаря тому, что для каждой задачи указаны её зависимости, вы сможете понять, какие подготовительные работы требуются, что позволяет более точно понимать сроки реализации. +Так как ClickHouse - open-source продукт, мы хотим, чтобы рабочий процесс был также максимально открытым. В связи с этим, вам следует ожидать наличия на данной странице несколько большего количества деталей описания рабочего процесса, чем вы могли бы предположить - настолько близко, насколько рабочий процесс видят разработчики. Так как неотъемлимой частью процесса разработки является исправление недостатков продукта и улучшение качества кода, на данной странице вы найдёте весьма подробные описания таких деталей. Для других open-source продуктов такой подход к разработке обычно нехарактерен. Благодаря тому, что для каждой задачи указаны её зависимости, вы сможете понять, какие подготовительные работы требуются, что позволяет более точно понимать сроки реализации. ## 1. Хранение данных, индексация. From d61507801ce742ced6124d45a65e4076a2037813 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 20 Nov 2019 23:58:38 +0300 Subject: [PATCH 77/90] Updated roadmap --- docs/ru/extended_roadmap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/extended_roadmap.md b/docs/ru/extended_roadmap.md index 184454aa071..5c89eb0284a 100644 --- a/docs/ru/extended_roadmap.md +++ b/docs/ru/extended_roadmap.md @@ -14,7 +14,7 @@ Задача "normalized z-Order curve" в перспективе может быть полезна для БК и Метрики, так как позволяет смешивать OrderID и PageID и избежать дублирования данных. В задаче также вводится способ индексации путём обращения функции нескольких аргументов на интервале, что имеет смысл для дальнейшего развития. -Изначально делал [Андрей Чулков](https://github.com/achulkov2), ВШЭ, теперь доделывает [Ольга Хвостикова](https://github.com/stavrolia), но сроки немного сдвинуты из-за задачи 25.9. Будем надеятся на реализацию к концу ноября. Впрочем, [Андрей Чулков](https://github.com/achulkov2) скоро выйдет на работу стажёром на пол ставки и сможет помочь её доделать. +Изначально делал [Андрей Чулков](https://github.com/achulkov2), ВШЭ, теперь доделывает [Ольга Хвостикова](https://github.com/stavrolia), но сроки немного сдвинуты из-за задачи 25.9. Будем надеятся на реализацию к концу ноября. Впрочем, [Андрей Чулков](https://github.com/achulkov2) скоро сможет помочь её доделать. ### 1.2. Wait-free каталог баз данных. From 61783e1e1719cd002445e5fbe55f2d087a3d4ceb Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Nov 2019 00:10:32 +0300 Subject: [PATCH 78/90] fix bytes count in PeekableReadBuffer --- dbms/src/IO/PeekableReadBuffer.cpp | 5 ----- .../01034_values_parse_float_bug.reference | 1 + .../0_stateless/01034_values_parse_float_bug.sh | 16 ++++++++++++++++ 3 files changed, 17 insertions(+), 5 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01034_values_parse_float_bug.reference create mode 100755 dbms/tests/queries/0_stateless/01034_values_parse_float_bug.sh diff --git a/dbms/src/IO/PeekableReadBuffer.cpp b/dbms/src/IO/PeekableReadBuffer.cpp index eff935a9de5..7c60bb252ad 100644 --- a/dbms/src/IO/PeekableReadBuffer.cpp +++ b/dbms/src/IO/PeekableReadBuffer.cpp @@ -19,7 +19,6 @@ bool PeekableReadBuffer::peekNext() { checkStateCorrect(); - size_t bytes_read = 0; Position copy_from = pos; size_t bytes_to_copy = sub_buf.available(); if (useSubbufferOnly()) @@ -27,11 +26,9 @@ bool PeekableReadBuffer::peekNext() /// Don't have to copy all data from sub-buffer if there is no data in own memory (checkpoint and pos are in sub-buffer) if (checkpoint) copy_from = checkpoint; - bytes_read = copy_from - sub_buf.buffer().begin(); bytes_to_copy = sub_buf.buffer().end() - copy_from; if (!bytes_to_copy) { - bytes += bytes_read; sub_buf.position() = copy_from; /// Both checkpoint and pos are at the end of sub-buffer. Just load next part of data. @@ -50,7 +47,6 @@ bool PeekableReadBuffer::peekNext() if (useSubbufferOnly()) { - bytes += bytes_read; sub_buf.position() = copy_from; } @@ -198,7 +194,6 @@ void PeekableReadBuffer::resizeOwnMemoryIfNecessary(size_t bytes_to_append) /// Move unread data to the beginning of own memory instead of resize own memory peeked_size -= offset; memmove(memory.data(), memory.data() + offset, peeked_size); - bytes += offset; if (need_update_checkpoint) checkpoint -= offset; diff --git a/dbms/tests/queries/0_stateless/01034_values_parse_float_bug.reference b/dbms/tests/queries/0_stateless/01034_values_parse_float_bug.reference new file mode 100644 index 00000000000..ae7a0f09264 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01034_values_parse_float_bug.reference @@ -0,0 +1 @@ +-160.32605 37.705841 diff --git a/dbms/tests/queries/0_stateless/01034_values_parse_float_bug.sh b/dbms/tests/queries/0_stateless/01034_values_parse_float_bug.sh new file mode 100755 index 00000000000..8e06e126f56 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01034_values_parse_float_bug.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS values_floats" + +${CLICKHOUSE_CLIENT} --query="CREATE TABLE values_floats (a Float32, b Float64) ENGINE = Memory" + +${CLICKHOUSE_CLIENT} --query="SELECT '(-160.32605134916085,37.70584056842162),' FROM numbers(1000000)" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO values_floats FORMAT Values" + +${CLICKHOUSE_CLIENT} --query="SELECT DISTINCT round(a, 6), round(b, 6) FROM values_floats" + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS values_floats" + From 2b4f6af736aaf10b2395be091f787ef7eb3d93ce Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 21 Nov 2019 12:09:43 +0300 Subject: [PATCH 79/90] Fixed errors with space reservation introduced in #7558 and #7602. --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 5 +++++ dbms/src/Storages/StorageReplicatedMergeTree.cpp | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 728f69f477e..19d8d8e4299 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -3269,6 +3269,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( String tmp_dst_part_name = tmp_part_prefix + dst_part_name; auto reservation = src_part->disk->reserve(src_part->bytes_on_disk); + if (!reservation) + { + throw Exception("Cannot reserve " + formatReadableSizeWithBinarySuffix(src_part->bytes_on_disk) + ", not enough space", + ErrorCodes::NOT_ENOUGH_SPACE); + } String dst_part_path = getFullPathOnDisk(reservation->getDisk()); Poco::Path dst_part_absolute_path = Poco::Path(dst_part_path + tmp_dst_part_name).absolute(); Poco::Path src_part_absolute_path = Poco::Path(src_part->getFullPath()).absolute(); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index f5b299edc9f..7f19b91ff28 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -1141,6 +1141,11 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM /// Can throw an exception. /// Once we mutate part, we must reserve space on the same disk, because mutations can possibly create hardlinks. DiskSpace::ReservationPtr reserved_space = source_part->disk->reserve(estimated_space_for_result); + if (!reserved_space) + { + throw Exception("Cannot reserve " + formatReadableSizeWithBinarySuffix(estimated_space_for_result) + ", not enough space", + ErrorCodes::NOT_ENOUGH_SPACE); + } auto table_lock = lockStructureForShare(false, RWLockImpl::NO_QUERY); From 970ad9d84a5bfe2b1de48e1e8cd4227a6a717e0c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 21 Nov 2019 12:23:03 +0300 Subject: [PATCH 80/90] Update metrica.md --- docs/ru/getting_started/example_datasets/metrica.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting_started/example_datasets/metrica.md b/docs/ru/getting_started/example_datasets/metrica.md index 3aaa4db952a..4bb3dc9e4c6 100644 --- a/docs/ru/getting_started/example_datasets/metrica.md +++ b/docs/ru/getting_started/example_datasets/metrica.md @@ -49,4 +49,4 @@ $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" ``` ## Запросы -Примеры запросов к этим таблицам (они называются `test.hits` и `test.visits`) можно найти среди [stateful тестов](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/queries/1_stateful) и в некоторых [performance тестах](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/performance/test_hits) ClickHouse. +Примеры запросов к этим таблицам (они называются `test.hits` и `test.visits`) можно найти среди [stateful тестов](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/queries/1_stateful) и в некоторых [performance тестах](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/performance) ClickHouse. From cf39761ddf21ded94a19349d8ace3324ea04c3d5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 21 Nov 2019 12:24:53 +0300 Subject: [PATCH 81/90] Update metrica.md --- docs/en/getting_started/example_datasets/metrica.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting_started/example_datasets/metrica.md b/docs/en/getting_started/example_datasets/metrica.md index 34d4e0c9d75..19947273338 100644 --- a/docs/en/getting_started/example_datasets/metrica.md +++ b/docs/en/getting_started/example_datasets/metrica.md @@ -48,4 +48,4 @@ $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" ``` ## Queries -Examples of queries to these tables (they are named `test.hits` and `test.visits`) can be found among [stateful tests](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/queries/1_stateful) and in some [performance tests](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/performance/test_hits) of ClickHouse. +Examples of queries to these tables (they are named `test.hits` and `test.visits`) can be found among [stateful tests](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/queries/1_stateful) and in some [performance tests](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/performance) of ClickHouse. From 01236a3e232415f9f062d4a6a7d4aca3909197e3 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 21 Nov 2019 15:10:06 +0300 Subject: [PATCH 82/90] Prepare integration test runner to use Minio. --- dbms/tests/integration/image/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/integration/image/Dockerfile b/dbms/tests/integration/image/Dockerfile index 0ebecb43681..ec66be91a06 100644 --- a/dbms/tests/integration/image/Dockerfile +++ b/dbms/tests/integration/image/Dockerfile @@ -34,7 +34,7 @@ RUN apt-get update \ ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone -RUN pip install pytest docker-compose==1.22.0 docker dicttoxml kazoo PyMySQL psycopg2==2.7.5 pymongo tzlocal kafka-python protobuf redis aerospike pytest-timeout +RUN pip install urllib3==1.23 pytest docker-compose==1.22.0 docker dicttoxml kazoo PyMySQL psycopg2==2.7.5 pymongo tzlocal kafka-python protobuf redis aerospike pytest-timeout minio ENV DOCKER_CHANNEL stable ENV DOCKER_VERSION 17.09.1-ce From 05c39e5432f3cfbba314b6ec5f19f5d09c83a9e7 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 21 Nov 2019 15:24:42 +0300 Subject: [PATCH 83/90] Update settings.md --- docs/ru/operations/server_settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/server_settings/settings.md b/docs/ru/operations/server_settings/settings.md index 50bbb6b1a5d..aca2fed8063 100644 --- a/docs/ru/operations/server_settings/settings.md +++ b/docs/ru/operations/server_settings/settings.md @@ -702,12 +702,12 @@ ClickHouse использует ZooKeeper для хранения метадан Например: - ```xml +```xml example_host 2181 - ``` +``` Атрибут `index` задает порядок опроса нод при попытках подключиться к кластеру ZooKeeper. From 6621fb852f8f5ec0f48da87433e09d8c46c5c3c7 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 21 Nov 2019 15:25:07 +0300 Subject: [PATCH 84/90] Update settings.md --- docs/en/operations/server_settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index 7bcd183c6c3..4158cad3440 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -753,12 +753,12 @@ This section contains the following parameters: For example: - ```xml +```xml example_host 2181 - ``` +``` The `index` attribute specifies the node order when trying to connect to the ZooKeeper cluster. From 31cb69269b5c343e29671c26cbc43e475c117e65 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 21 Nov 2019 16:44:11 +0300 Subject: [PATCH 85/90] small fix --- .../Impl/TabSeparatedRowInputFormat.cpp | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 537fc6f690f..39f93b290db 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -384,32 +384,36 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) } } -void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * ptr) +void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current) { assert(ptr >= in.position()); assert(ptr <= in.buffer().end()); const int old_bytes = memory.size(); - const int additional_bytes = ptr - in.position(); + const int additional_bytes = current - in.position(); const int new_bytes = old_bytes + additional_bytes; + /// There are no new bytes to add to memory. + /// No need to do extra stuff. + if (new_bytes == 0) + return; memory.resize(new_bytes); memcpy(memory.data() + old_bytes, in.position(), additional_bytes); - in.position() = ptr; + in.position() = current; } -bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & ptr) +bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current) { assert(ptr <= in.buffer().end()); - if (ptr < in.buffer().end()) + if (current < in.buffer().end()) { return true; } - saveUpToPosition(in, memory, in.buffer().end()); + saveUpToPosition(in, memory, current); bool loaded_more = !in.eof(); assert(in.position() == in.buffer().begin()); - ptr = in.position(); + current = in.position(); return loaded_more; } From 83af9abccd1ef3b19677655af8e8ced6aa19357c Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Thu, 21 Nov 2019 17:30:36 +0300 Subject: [PATCH 86/90] Update supported versions of compilers --- dbms/tests/instructions/developer_instruction_en.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/tests/instructions/developer_instruction_en.md b/dbms/tests/instructions/developer_instruction_en.md index 58500dd2627..463f186fb3b 100644 --- a/dbms/tests/instructions/developer_instruction_en.md +++ b/dbms/tests/instructions/developer_instruction_en.md @@ -109,13 +109,13 @@ However, these libraries are optional and ClickHouse can well be built without t # C++ Compiler -Compilers GCC starting from version 7 and Clang version 7 or above are supported for building ClickHouse. +Compilers GCC starting from version 9 and Clang version 8 or above are supported for building ClickHouse. Official Yandex builds currently use GCC because it generates machine code of slightly better performance (yielding a difference of up to several percent according to our benchmarks). And Clang is more convenient for development usually. Though, our continuous integration (CI) platform runs checks for about a dozen of build combinations. To install GCC on Ubuntu run: `sudo apt install gcc g++` -Check the version of gcc: `gcc --version`. If it is below 7, then follow the instruction here: https://clickhouse.yandex/docs/en/development/build/#install-gcc-7. +Check the version of gcc: `gcc --version`. If it is below 9, then follow the instruction here: https://clickhouse.yandex/docs/en/development/build/#install-gcc-9. To install GCC on Mac OS X run: `brew install gcc`. @@ -131,9 +131,9 @@ cd build ``` You can have several different directories (build_release, build_debug, etc.) for different types of build. -While inside the `build` directory, configure your build by running CMake. Before the first run you need to define environment variables that specify compiler (version 7 gcc compiler in this example). +While inside the `build` directory, configure your build by running CMake. Before the first run you need to define environment variables that specify compiler (version 9 gcc compiler in this example). ``` -export CC=gcc-7 CXX=g++-7 +export CC=gcc-9 CXX=g++-9 cmake .. ``` The `CC` variable specifies the compiler for C (short for C Compiler), and `CXX` variable instructs which C++ compiler is to be used for building. @@ -265,4 +265,4 @@ Testing will commence as soon as Yandex employees label your PR with a tag "can The system will prepare ClickHouse binary builds for your pull request individually. To retrieve these builds click the "Details" link next to "ClickHouse build check" entry in the list of checks. There you will find direct links to the built .deb packages of ClickHouse which you can deploy even on your production servers (if you have no fear). -Most probably some of the builds will fail at first times. This is due to the fact that we check builds both with gcc as well as with clang, with almost all of existing warnings (always with the `-Werror` flag) enabled for clang. On that same page you can find all of the build logs so that you do not have to build ClickHouse in all of the possible ways. \ No newline at end of file +Most probably some of the builds will fail at first times. This is due to the fact that we check builds both with gcc as well as with clang, with almost all of existing warnings (always with the `-Werror` flag) enabled for clang. On that same page you can find all of the build logs so that you do not have to build ClickHouse in all of the possible ways. From d3456b1a8cc535ee3ad2328a1f88eb20188a8965 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Thu, 21 Nov 2019 17:31:53 +0300 Subject: [PATCH 87/90] Update supported versions of compilers --- dbms/tests/instructions/developer_instruction_ru.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/tests/instructions/developer_instruction_ru.md b/dbms/tests/instructions/developer_instruction_ru.md index 7aa110b1b94..ad03c192d52 100644 --- a/dbms/tests/instructions/developer_instruction_ru.md +++ b/dbms/tests/instructions/developer_instruction_ru.md @@ -109,13 +109,13 @@ Mac OS X: `brew install icu4c readline` # Компилятор C++ -В качестве компилятора C++ поддерживается GCC начиная с версии 7 или Clang начиная с версии 7. +В качестве компилятора C++ поддерживается GCC начиная с версии 9 или Clang начиная с версии 8. Официальные сборки от Яндекса, на данный момент, используют GCC, так как он генерирует слегка более производительный машинный код (разница в среднем до нескольких процентов по нашим бенчмаркам). Clang обычно более удобен для разработки. Впрочем, наша среда continuous integration проверяет около десятка вариантов сборки. Для установки GCC под Ubuntu, выполните: `sudo apt install gcc g++`. -Проверьте версию gcc: `gcc --version`. Если версия меньше 7, то следуйте инструкции: https://clickhouse.yandex/docs/en/development/build/#install-gcc-7 +Проверьте версию gcc: `gcc --version`. Если версия меньше 9, то следуйте инструкции: https://clickhouse.yandex/docs/en/development/build/#install-gcc-9 Для установки GCC под Mac OS X, выполните `brew install gcc`. @@ -132,9 +132,9 @@ cd build Вы можете иметь несколько разных директорий (build_release, build_debug) для разных вариантов сборки. Находясь в директории build, выполните конфигурацию сборки с помощью CMake. -Перед первым запуском необходимо выставить переменные окружения, отвечающие за выбор компилятора (в данном примере это - gcc версии 7). +Перед первым запуском необходимо выставить переменные окружения, отвечающие за выбор компилятора (в данном примере это - gcc версии 9). ``` -export CC=gcc-7 CXX=g++-7 +export CC=gcc-9 CXX=g++-9 cmake .. ``` Переменная CC отвечает за компилятор C (сокращение от слов C Compiler), переменная CXX отвечает за выбор компилятора C++ (символ X - это как плюс, но положенный набок, ради того, чтобы превратить его в букву). From ba3969527ff37f48e59856da605065e0e5162f83 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Nov 2019 18:37:59 +0300 Subject: [PATCH 88/90] move table locks to main pipeline --- dbms/src/Processors/QueryPipeline.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Processors/QueryPipeline.cpp b/dbms/src/Processors/QueryPipeline.cpp index 525cc898cc1..fd75d7f57cf 100644 --- a/dbms/src/Processors/QueryPipeline.cpp +++ b/dbms/src/Processors/QueryPipeline.cpp @@ -487,9 +487,9 @@ void QueryPipeline::unitePipelines( processors.insert(processors.end(), pipeline.processors.begin(), pipeline.processors.end()); streams.insert(streams.end(), pipeline.streams.begin(), pipeline.streams.end()); + table_locks.insert(table_locks.end(), std::make_move_iterator(pipeline.table_locks.begin()), std::make_move_iterator(pipeline.table_locks.end())); interpreter_context.insert(interpreter_context.end(), pipeline.interpreter_context.begin(), pipeline.interpreter_context.end()); storage_holder.insert(storage_holder.end(), pipeline.storage_holder.begin(), pipeline.storage_holder.end()); - /// Do we need also copy table_locks? } if (!extremes.empty()) From 1d74aecda885b60a4d245e34523e95ecece5206a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 21 Nov 2019 21:14:47 +0300 Subject: [PATCH 89/90] fix typo in asserts --- .../Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 39f93b290db..3f8c64344f9 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -386,8 +386,8 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current) { - assert(ptr >= in.position()); - assert(ptr <= in.buffer().end()); + assert(current >= in.position()); + assert(current <= in.buffer().end()); const int old_bytes = memory.size(); const int additional_bytes = current - in.position(); @@ -403,7 +403,7 @@ void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current) bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current) { - assert(ptr <= in.buffer().end()); + assert(current <= in.buffer().end()); if (current < in.buffer().end()) { From f592594c2ebc514fca250fa347200a42641eb3a0 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 22 Nov 2019 06:48:34 +0300 Subject: [PATCH 90/90] Remove link to past meetup --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 6f5c6f02bc7..32f6eba9bab 100644 --- a/README.md +++ b/README.md @@ -14,7 +14,6 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events -* [ClickHouse Meetup in Ankara](https://www.eventbrite.com/e/clickhouse-meetup-ankara-create-blazing-fast-experiences-w-clickhouse-tickets-73100530655) on November 21. * [ClickHouse Meetup in Singapore](https://www.meetup.com/Singapore-Clickhouse-Meetup-Group/events/265085331/) on November 23. * [ClickHouse Meetup in San Francisco](https://www.eventbrite.com/e/clickhouse-december-meetup-registration-78642047481) on December 3.