From 13be98f3366df19fb48dbde0d8f85c18630cd242 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=B8=87=E5=BA=B7?= Date: Wed, 16 Jun 2021 20:59:02 +0800 Subject: [PATCH 001/659] add setting exact_rows_before_limit --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 7 ++++--- src/Interpreters/InterpreterSelectWithUnionQuery.cpp | 2 +- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2aed174c088..59201b7541f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -559,6 +559,7 @@ class IColumn; M(Bool, output_format_pretty_row_numbers, false, "Add row numbers before each row for pretty output format", 0) \ M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \ M(Bool, cross_to_inner_join_rewrite, true, "Use inner join instead of comma/cross join if possible", 0) \ + M(Bool, exact_rows_before_limit, false, "When enabled, LimitBlockInputStream will always_read_till_end and the field rows_before_limit_at_least will have precise value", 0) \ // End of FORMAT_FACTORY_SETTINGS diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 85b9026c642..57961bdf474 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2442,8 +2442,8 @@ void InterpreterSelectQuery::executePreLimit(QueryPlan & query_plan, bool do_not limit_length += limit_offset; limit_offset = 0; } - - auto limit = std::make_unique(query_plan.getCurrentDataStream(), limit_length, limit_offset); + const Settings & settings = context->getSettingsRef(); + auto limit = std::make_unique(query_plan.getCurrentDataStream(), limit_length, limit_offset, settings.exact_rows_before_limit); limit->setStepDescription("preliminary LIMIT"); query_plan.addStep(std::move(limit)); } @@ -2504,7 +2504,8 @@ void InterpreterSelectQuery::executeLimit(QueryPlan & query_plan) * if there is WITH TOTALS and there is no ORDER BY, then read the data to the end, * otherwise TOTALS is counted according to incomplete data. */ - bool always_read_till_end = false; + const Settings & settings = context->getSettingsRef(); + bool always_read_till_end = settings.exact_rows_before_limit; if (query.group_by_with_totals && !query.orderBy()) always_read_till_end = true; diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 3cf4a905d38..b897d7e688d 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -289,7 +289,7 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) { if (settings.limit > 0) { - auto limit = std::make_unique(query_plan.getCurrentDataStream(), settings.limit, settings.offset); + auto limit = std::make_unique(query_plan.getCurrentDataStream(), settings.limit, settings.offset, settings.exact_rows_before_limit); limit->setStepDescription("LIMIT OFFSET for SETTINGS"); query_plan.addStep(std::move(limit)); } From 6ae91c3e8a1aecb9302f2d8ade94d2801916455c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=B8=87=E5=BA=B7?= Date: Thu, 17 Jun 2021 23:55:06 +0800 Subject: [PATCH 002/659] add tests --- .../01913_exact_rows_before_limit.reference | 36 +++++++++++++++++++ .../01913_exact_rows_before_limit.sql | 7 ++++ 2 files changed, 43 insertions(+) create mode 100644 tests/queries/0_stateless/01913_exact_rows_before_limit.reference create mode 100644 tests/queries/0_stateless/01913_exact_rows_before_limit.sql diff --git a/tests/queries/0_stateless/01913_exact_rows_before_limit.reference b/tests/queries/0_stateless/01913_exact_rows_before_limit.reference new file mode 100644 index 00000000000..d4fa14a28f2 --- /dev/null +++ b/tests/queries/0_stateless/01913_exact_rows_before_limit.reference @@ -0,0 +1,36 @@ +{ + "meta": + [ + { + "name": "0", + "type": "UInt8" + } + ], + + "data": + [ + [0] + ], + + "rows": 1, + + "rows_before_limit_at_least": 10000 +} +{ + "meta": + [ + { + "name": "0", + "type": "UInt8" + } + ], + + "data": + [ + [0] + ], + + "rows": 1, + + "rows_before_limit_at_least": 20000 +} diff --git a/tests/queries/0_stateless/01913_exact_rows_before_limit.sql b/tests/queries/0_stateless/01913_exact_rows_before_limit.sql new file mode 100644 index 00000000000..cb26942bb39 --- /dev/null +++ b/tests/queries/0_stateless/01913_exact_rows_before_limit.sql @@ -0,0 +1,7 @@ +drop table if exists test_rows; +create table test_rows(f1 int,f2 int) engine=MergeTree partition by f1 order by f2; +insert into test_rows select 0,arrayJoin(range(10000)); +insert into test_rows select 1,arrayJoin(range(10000)); +select 0 from test_rows limit 1 FORMAT JSONCompact settings exact_rows_before_limit = 0,output_format_write_statistics = 0; +select 0 from test_rows limit 1 FORMAT JSONCompact settings exact_rows_before_limit = 1, output_format_write_statistics = 0; +drop table if exists test_rows; From 9896d6f40ee2c3803ff4c30ada729df9e86333c9 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 23 Dec 2021 11:37:20 +0300 Subject: [PATCH 003/659] Update Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 75a7ad9cc5b..05570f976f4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -658,7 +658,7 @@ class IColumn; M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \ \ M(Bool, cross_to_inner_join_rewrite, true, "Use inner join instead of comma/cross join if possible", 0) \ - M(Bool, exact_rows_before_limit, false, "When enabled, LimitBlockInputStream will always_read_till_end and the field rows_before_limit_at_least will have precise value", 0) \ + M(Bool, exact_rows_before_limit, false, "When enabled, ClickHouse will provide exact value for rows_before_limit_at_least statistic, but with the cost that the data before limit will have to be read completely", 0) \ \ M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ \ From 49414a590aa00639d34f8decfd1874d17db8f78a Mon Sep 17 00:00:00 2001 From: MaxWk <610379995@qq.com> Date: Wed, 6 Apr 2022 01:27:23 +0800 Subject: [PATCH 004/659] add different result between compact and wide part --- .../01913_exact_rows_before_limit.reference | 36 +++++++++++++++++++ .../01913_exact_rows_before_limit.sql | 24 +++++++++---- 2 files changed, 53 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/01913_exact_rows_before_limit.reference b/tests/queries/0_stateless/01913_exact_rows_before_limit.reference index d4fa14a28f2..af808adf83d 100644 --- a/tests/queries/0_stateless/01913_exact_rows_before_limit.reference +++ b/tests/queries/0_stateless/01913_exact_rows_before_limit.reference @@ -34,3 +34,39 @@ "rows_before_limit_at_least": 20000 } +{ + "meta": + [ + { + "name": "0", + "type": "UInt8" + } + ], + + "data": + [ + [0] + ], + + "rows": 1, + + "rows_before_limit_at_least": 1 +} +{ + "meta": + [ + { + "name": "0", + "type": "UInt8" + } + ], + + "data": + [ + [0] + ], + + "rows": 1, + + "rows_before_limit_at_least": 20000 +} diff --git a/tests/queries/0_stateless/01913_exact_rows_before_limit.sql b/tests/queries/0_stateless/01913_exact_rows_before_limit.sql index cb26942bb39..f6b02d0a510 100644 --- a/tests/queries/0_stateless/01913_exact_rows_before_limit.sql +++ b/tests/queries/0_stateless/01913_exact_rows_before_limit.sql @@ -1,7 +1,17 @@ -drop table if exists test_rows; -create table test_rows(f1 int,f2 int) engine=MergeTree partition by f1 order by f2; -insert into test_rows select 0,arrayJoin(range(10000)); -insert into test_rows select 1,arrayJoin(range(10000)); -select 0 from test_rows limit 1 FORMAT JSONCompact settings exact_rows_before_limit = 0,output_format_write_statistics = 0; -select 0 from test_rows limit 1 FORMAT JSONCompact settings exact_rows_before_limit = 1, output_format_write_statistics = 0; -drop table if exists test_rows; +-- Tags: no-parallel +drop table if exists test_rows_compact_part; +create table test_rows_compact_part(f1 int,f2 int) engine=MergeTree partition by f1 order by f2 settings min_bytes_for_wide_part=10485760; +insert into test_rows_compact_part select 0,arrayJoin(range(10000)) ; +insert into test_rows_compact_part select 1,arrayJoin(range(10000)); +select 0 from test_rows_compact_part limit 1 FORMAT JSONCompact settings exact_rows_before_limit = 0,output_format_write_statistics = 0; +select 0 from test_rows_compact_part limit 1 FORMAT JSONCompact settings exact_rows_before_limit = 1, output_format_write_statistics = 0; +drop table if exists test_rows_compact_part; + + +drop table if exists test_rows_wide_part; +create table test_rows_wide_part(f1 int,f2 int) engine=MergeTree partition by f1 order by f2 settings min_bytes_for_wide_part=0; +insert into test_rows_wide_part select 0,arrayJoin(range(10000)) ; +insert into test_rows_wide_part select 1,arrayJoin(range(10000)); +select 0 from test_rows_wide_part limit 1 FORMAT JSONCompact settings exact_rows_before_limit = 0,output_format_write_statistics = 0; +select 0 from test_rows_wide_part limit 1 FORMAT JSONCompact settings exact_rows_before_limit = 1, output_format_write_statistics = 0; +drop table if exists test_rows_compact_part; \ No newline at end of file From 9f06ec1f14da208248b10f66ed7c16a52961ab72 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 26 Apr 2022 13:52:35 -0400 Subject: [PATCH 005/659] prepare cmake and submodule link for new compression solution named intel IAA deflate --- .gitmodules | 3 +++ contrib/CMakeLists.txt | 1 + contrib/qpl-cmake/CMakeLists.txt | 17 +++++++++++++++++ src/CMakeLists.txt | 5 +++++ 4 files changed, 26 insertions(+) create mode 100644 contrib/qpl-cmake/CMakeLists.txt diff --git a/.gitmodules b/.gitmodules index 6c9e66f9cbc..f423dc1d31a 100644 --- a/.gitmodules +++ b/.gitmodules @@ -262,3 +262,6 @@ [submodule "contrib/minizip-ng"] path = contrib/minizip-ng url = https://github.com/zlib-ng/minizip-ng +[submodule "contrib/qpl"] + path = contrib/qpl + url = https://github.com/intel/qpl.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 1f03c0fd341..85bd4658454 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -151,6 +151,7 @@ endif() add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) +add_contrib (qpl-cmake qpl) # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt new file mode 100644 index 00000000000..605b2e8cfcf --- /dev/null +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -0,0 +1,17 @@ +# The Intel® QPL provides high performance implementations of data processing functions for existing hardware accelerator, and/or software path in case if hardware accelerator is not available. + +set (QPL_INCLUDE_DIRS + "${ClickHouse_SOURCE_DIR}/contrib/qpl/include" + "${ClickHouse_BINARY_DIR}/contrib/qpl/include" +) +set (LOG_HW_INIT ON) +set (EFFICIENT_WAIT ON) + +set (QPL_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl") +set (QPL_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/qpl") +add_subdirectory ("${QPL_SOURCE_DIR}" "${QPL_BINARY_DIR}") + +add_library (_qpl INTERFACE) +target_link_libraries (_qpl INTERFACE qpl) +target_include_directories (_qpl SYSTEM BEFORE INTERFACE ${QPL_INCLUDE_DIRS}) +add_library(ch_contrib::qpl ALIAS _qpl) \ No newline at end of file diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index e8e1153e2b4..b0242bd2bc5 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -524,6 +524,11 @@ if (TARGET ch_contrib::datasketches) endif () target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::lz4) +target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::qpl) +target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/qpl/include) +set_source_files_properties( + Compression/CompressionCodecDeflate.cpp + PROPERTIES COMPILE_FLAGS "-mwaitpkg") dbms_target_link_libraries(PRIVATE _boost_context) From 68a7e4b9b552870ef7d0cdd32240ea237d0da6f6 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 26 Apr 2022 14:14:09 -0400 Subject: [PATCH 006/659] preliminary patch for iaa deflate solution --- programs/compressor/Compressor.cpp | 6 +- src/Compression/CompressedReadBufferBase.cpp | 90 +++- src/Compression/CompressedReadBufferBase.h | 9 +- .../CompressedReadBufferFromFile.cpp | 57 ++- src/Compression/CompressionCodecDeflate.cpp | 448 ++++++++++++++++++ src/Compression/CompressionCodecDeflate.h | 332 +++++++++++++ src/Compression/CompressionFactory.cpp | 4 +- src/Compression/CompressionInfo.h | 3 +- src/Compression/ICompressionCodec.cpp | 46 +- src/Compression/ICompressionCodec.h | 48 +- 10 files changed, 1015 insertions(+), 28 deletions(-) create mode 100644 src/Compression/CompressionCodecDeflate.cpp create mode 100644 src/Compression/CompressionCodecDeflate.h diff --git a/programs/compressor/Compressor.cpp b/programs/compressor/Compressor.cpp index d47372631fe..86658010310 100644 --- a/programs/compressor/Compressor.cpp +++ b/programs/compressor/Compressor.cpp @@ -77,6 +77,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) ("block-size,b", po::value()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size") ("hc", "use LZ4HC instead of LZ4") ("zstd", "use ZSTD instead of LZ4") + ("deflate", "use deflate instead of LZ4") ("codec", po::value>()->multitoken(), "use codecs combination instead of LZ4") ("level", po::value(), "compression level for codecs specified via flags") ("none", "use no compression instead of LZ4") @@ -103,6 +104,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) bool decompress = options.count("decompress"); bool use_lz4hc = options.count("hc"); bool use_zstd = options.count("zstd"); + bool use_deflate = options.count("deflate"); bool stat_mode = options.count("stat"); bool use_none = options.count("none"); unsigned block_size = options["block-size"].as(); @@ -110,7 +112,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) if (options.count("codec")) codecs = options["codec"].as>(); - if ((use_lz4hc || use_zstd || use_none) && !codecs.empty()) + if ((use_deflate || use_lz4hc || use_zstd || use_none) && !codecs.empty()) throw Exception("Wrong options, codec flags like --zstd and --codec options are mutually exclusive", ErrorCodes::BAD_ARGUMENTS); if (!codecs.empty() && options.count("level")) @@ -122,6 +124,8 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) method_family = "LZ4HC"; else if (use_zstd) method_family = "ZSTD"; + else if (use_deflate) + method_family = "DEFLATE"; else if (use_none) method_family = "NONE"; diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 81e49e445a7..f169aad6da4 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -187,6 +187,79 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, return size_compressed_without_checksum + sizeof(Checksum); } +size_t CompressedReadBufferBase::readCompressedDataBlockHold(size_t & size_decompressed, size_t & size_compressed_without_checksum) +{ + UInt8 header_size = ICompressionCodec::getHeaderSize(); + if (compressed_in->eof() || (compressed_in->available() < (header_size + sizeof(Checksum)))) + return 0; + + own_compressed_buffer.resize(header_size + sizeof(Checksum)); + + compressed_in->readStrict(own_compressed_buffer.data(), sizeof(Checksum) + header_size); + char * compressed_header = own_compressed_buffer.data() + sizeof(Checksum); + + uint8_t method = ICompressionCodec::readMethod(compressed_header); + + if (!codec) + { + codec = CompressionCodecFactory::instance().get(method); + } + else if (method != codec->getMethodByte()) + { + if (allow_different_codecs) + { + codec = CompressionCodecFactory::instance().get(method); + } + else + { + throw Exception("Data compressed with different methods, given method byte 0x" + + getHexUIntLowercase(method) + + ", previous method byte 0x" + + getHexUIntLowercase(codec->getMethodByte()), + ErrorCodes::CANNOT_DECOMPRESS); + } + } + + size_compressed_without_checksum = ICompressionCodec::readCompressedBlockSize(compressed_header); + size_decompressed = ICompressionCodec::readDecompressedBlockSize(compressed_header); + + /// This is for clang static analyzer. + assert(size_decompressed > 0); + + if (size_compressed_without_checksum > DBMS_MAX_COMPRESSED_SIZE) + throw Exception("Too large size_compressed_without_checksum: " + + toString(size_compressed_without_checksum) + + ". Most likely corrupted data.", + ErrorCodes::TOO_LARGE_SIZE_COMPRESSED); + + if (size_compressed_without_checksum < header_size) + throw Exception("Can't decompress data: the compressed data size (" + toString(size_compressed_without_checksum) + + ", this should include header size) is less than the header size (" + toString(header_size) + ")", ErrorCodes::CORRUPTED_DATA); + + ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum)); + + auto additional_size_at_the_end_of_buffer = codec->getAdditionalSizeAtTheEndOfBuffer(); + + /// Is whole compressed block located in 'compressed_in->' buffer? + if (compressed_in->offset() >= header_size + sizeof(Checksum) && + compressed_in->available() >= (size_compressed_without_checksum - header_size) + additional_size_at_the_end_of_buffer + sizeof(Checksum)) + { + compressed_in->position() -= header_size; + compressed_buffer = compressed_in->position(); + compressed_in->position() += size_compressed_without_checksum; + if (!disable_checksum) + { + Checksum & checksum = *reinterpret_cast(own_compressed_buffer.data()); + validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum); + } + return size_compressed_without_checksum + sizeof(Checksum); + } + else + { + compressed_in->position() -= (sizeof(Checksum) + header_size); + return 0; + } +} static void readHeaderAndGetCodec(const char * compressed_buffer, size_t size_decompressed, CompressionCodecPtr & codec, bool allow_different_codecs) { @@ -217,14 +290,14 @@ static void readHeaderAndGetCodec(const char * compressed_buffer, size_t size_de } -void CompressedReadBufferBase::decompressTo(char * to, size_t size_decompressed, size_t size_compressed_without_checksum) +void CompressedReadBufferBase::decompressTo(char * to, size_t size_decompressed, size_t size_compressed_without_checksum, UInt8 req_type) { readHeaderAndGetCodec(compressed_buffer, size_decompressed, codec, allow_different_codecs); - codec->decompress(compressed_buffer, size_compressed_without_checksum, to); + codec->decompress(compressed_buffer, size_compressed_without_checksum, to, req_type); } -void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum) +void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum, UInt8 req_type) { readHeaderAndGetCodec(compressed_buffer, size_decompressed, codec, allow_different_codecs); @@ -242,9 +315,16 @@ void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_d to = BufferBase::Buffer(compressed_buffer + header_size, compressed_buffer + size_compressed_without_checksum); } else - codec->decompress(compressed_buffer, size_compressed_without_checksum, to.begin()); + codec->decompress(compressed_buffer, size_compressed_without_checksum, to.begin(), req_type); } +void CompressedReadBufferBase::decompressFlush(void) +{ + if (codec) + { + codec->decompressFlush(); + } +} /// 'compressed_in' could be initialized lazily, but before first call of 'readCompressedData'. CompressedReadBufferBase::CompressedReadBufferBase(ReadBuffer * in, bool allow_different_codecs_) @@ -253,7 +333,7 @@ CompressedReadBufferBase::CompressedReadBufferBase(ReadBuffer * in, bool allow_d } -CompressedReadBufferBase::~CompressedReadBufferBase() = default; /// Proper destruction of unique_ptr of forward-declared type. +CompressedReadBufferBase::~CompressedReadBufferBase() = default; /// Proper destruction of unique_ptr of forward-declared type. } diff --git a/src/Compression/CompressedReadBufferBase.h b/src/Compression/CompressedReadBufferBase.h index 152447c0b64..841c4a0bdd3 100644 --- a/src/Compression/CompressedReadBufferBase.h +++ b/src/Compression/CompressedReadBufferBase.h @@ -38,14 +38,15 @@ protected: /// /// Returns number of compressed bytes read. size_t readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum, bool always_copy); - + size_t readCompressedDataBlockHold(size_t & size_decompressed, size_t & size_compressed_without_checksum); /// Decompress into memory pointed by `to` - void decompressTo(char * to, size_t size_decompressed, size_t size_compressed_without_checksum); + void decompressTo(char * to, size_t size_decompressed, size_t size_compressed_without_checksum, UInt8 req_type=0); /// This method can change location of `to` to avoid unnecessary copy if data is uncompressed. /// It is more efficient for compression codec NONE but not suitable if you want to decompress into specific location. - void decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum); - + void decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum, UInt8 req_type=0); + /// Flush all asynchronous decompress request + void decompressFlush(void); public: /// 'compressed_in' could be initialized lazily, but before first call of 'readCompressedData'. explicit CompressedReadBufferBase(ReadBuffer * in = nullptr, bool allow_different_codecs_ = false); diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index 9efb3c92cde..c99c6690b39 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -91,6 +91,8 @@ void CompressedReadBufferFromFile::seek(size_t offset_in_compressed_file, size_t size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) { size_t bytes_read = 0; + UInt8 req_type = 0; + bool readTail = false; /// If there are unread bytes in the buffer, then we copy needed to `to`. if (pos < working_buffer.end()) @@ -102,10 +104,22 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) size_t size_decompressed = 0; size_t size_compressed_without_checksum = 0; - size_t new_size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum, false); + size_t new_size_compressed = readCompressedDataBlockHold(size_decompressed, size_compressed_without_checksum); + + if (new_size_compressed) + { + req_type = 1; + } + else + { + decompressFlush(); /// here switch to unhold block in compress_in, we must flush for previous blocks completely hold in compress_in + new_size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum, false); + req_type = 0; + } size_compressed = 0; /// file_in no longer points to the end of the block in working_buffer. + if (!new_size_compressed) - return bytes_read; + break; auto additional_size_at_the_end_of_buffer = codec->getAdditionalSizeAtTheEndOfBuffer(); @@ -113,10 +127,27 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) /// need to skip some bytes in decompressed data (seek happened before readBig call). if (nextimpl_working_buffer_offset == 0 && size_decompressed + additional_size_at_the_end_of_buffer <= n - bytes_read) { - decompressTo(to + bytes_read, size_decompressed, size_compressed_without_checksum); + decompressTo(to + bytes_read, size_decompressed, size_compressed_without_checksum, req_type); //Async req bytes_read += size_decompressed; bytes += size_decompressed; } + else if (nextimpl_working_buffer_offset > 0) + { + //Need to skip some bytes in decompressed data (seek happened before readBig call). + size_compressed = new_size_compressed; + bytes += offset(); + + /// This is for clang static analyzer. + assert(size_decompressed + additional_size_at_the_end_of_buffer > 0); + memory.resize(size_decompressed + additional_size_at_the_end_of_buffer); + working_buffer = Buffer(memory.data(), &memory[size_decompressed]); + decompress(working_buffer, size_decompressed, size_compressed_without_checksum); + + size_t size_partial = std::min((size_decompressed - nextimpl_working_buffer_offset), (n - bytes_read)); + pos = working_buffer.begin() + nextimpl_working_buffer_offset; + nextimpl_working_buffer_offset = 0; + bytes_read += read(to + bytes_read, size_partial); + } else { size_compressed = new_size_compressed; @@ -128,18 +159,22 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) memory.resize(size_decompressed + additional_size_at_the_end_of_buffer); working_buffer = Buffer(memory.data(), &memory[size_decompressed]); - decompress(working_buffer, size_decompressed, size_compressed_without_checksum); - - /// Manually take nextimpl_working_buffer_offset into account, because we don't use - /// nextImpl in this method. - pos = working_buffer.begin() + nextimpl_working_buffer_offset; - nextimpl_working_buffer_offset = 0; - - bytes_read += read(to + bytes_read, n - bytes_read); + decompress(working_buffer, size_decompressed, size_compressed_without_checksum, 1); + readTail = true; break; } } + decompressFlush(); + + if (readTail) + { + /// Manually take nextimpl_working_buffer_offset into account, because we don't use + /// nextImpl in this method. + pos = working_buffer.begin(); + bytes_read += read(to + bytes_read, n - bytes_read); + } + return bytes_read; } diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp new file mode 100644 index 00000000000..1e3faa2d44f --- /dev/null +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -0,0 +1,448 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_COMPRESS; + extern const int CANNOT_DECOMPRESS; +} + +qpl_job * DeflateJobHWPool::jobPool[jobPoolSize]; +std::atomic_bool DeflateJobHWPool::jobLock[jobPoolSize]; + +qpl_job * DeflateJobSWPool::jobSWPool[jobSWPoolSize]; +std::atomic_bool DeflateJobSWPool::jobSWLock[jobSWPoolSize]; + +DeflateJobHWPool & DeflateJobHWPool::instance() +{ + static DeflateJobHWPool ret; + return ret; +} + +DeflateJobHWPool::DeflateJobHWPool() +{ + if (initJobPool() < 0) + throw Exception("DeflateJobHWPool initializing fail!", ErrorCodes::CANNOT_COMPRESS); +} +DeflateJobHWPool::~DeflateJobHWPool() +{ + destroyJobPool(); +} + +DeflateJobSWPool & DeflateJobSWPool::instance() +{ + static DeflateJobSWPool ret; + return ret; +} + +DeflateJobSWPool::DeflateJobSWPool() +{ + if (initJobPool() < 0) + throw Exception("DeflateJobSWPool initializing fail!", ErrorCodes::CANNOT_COMPRESS); +} +DeflateJobSWPool::~DeflateJobSWPool() +{ + destroyJobPool(); +} + + +CompressionCodecDeflate::CompressionCodecDeflate() +{ + log = &Poco::Logger::get("CompressionCodecDeflate"); + setCodecDescription("DEFLATE"); +} + +CompressionCodecDeflate::~CompressionCodecDeflate() +{ + if (jobDecompAsyncMap.size() > 0) + { + LOG_ERROR(log, "Exception -> find un-released job when CompressionCodecDeflate destory"); + //doDecompressDataFlush(); + for (auto it : jobDecompAsyncMap) + { + DeflateJobHWPool::instance().releaseJob(it.first); + } + jobDecompAsyncMap.clear(); + } + if (jobCompAsyncList.size() > 0) + { + for (auto id : jobCompAsyncList) + { + DeflateJobHWPool::instance().releaseJob(id); + } + jobCompAsyncList.clear(); + } +} + +uint8_t CompressionCodecDeflate::getMethodByte() const +{ + return static_cast(CompressionMethodByte::Deflate); +} + +bool CompressionCodecDeflate::isAsyncSupported() const +{ + return true; +} + +void CompressionCodecDeflate::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); +} + +#define DEFLATE_COMPRESSBOUND(isize) ((isize) + ((isize) >> 12) + ((isize) >> 14) + ((isize) >> 25) + 13) //Aligned with ZLIB +uint32_t CompressionCodecDeflate::getMaxCompressedDataSize(uint32_t uncompressed_size) const +{ + return DEFLATE_COMPRESSBOUND(uncompressed_size); +} + +uint32_t CompressionCodecDeflate::doCompressDataSWNative(const char * source, uint32_t source_size, char * dest) const +{ + qpl_status status; + uint32_t size = 0; + + // Job initialization + status = qpl_get_job_size(DeflateJobSWPool::SW_PATH, &size); + if (status != QPL_STS_OK) + { + throw Exception("doCompressDataSWNative cannot compress: qpl_get_job_size fail", ErrorCodes::CANNOT_COMPRESS); + } + qpl_job * jobPtr = reinterpret_cast(new uint8_t[size]); + + status = qpl_init_job(DeflateJobSWPool::SW_PATH, jobPtr); + if (status != QPL_STS_OK) + { + throw Exception("doCompressDataSWNative cannot compress: qpl_init_job fail", ErrorCodes::CANNOT_COMPRESS); + } + + // Performing a compression operation + jobPtr->op = qpl_op_compress; + jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); + jobPtr->next_out_ptr = reinterpret_cast(dest); + jobPtr->available_in = source_size; + jobPtr->available_out = getMaxCompressedDataSize(source_size); + jobPtr->level = qpl_high_level; + jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; + + // Compression + status = qpl_execute_job(jobPtr); + if (status != QPL_STS_OK) + { + throw Exception("doCompressDataSWNative cannot compress: qpl_execute_job fail", ErrorCodes::CANNOT_COMPRESS); + } + + const uint32_t compressed_size = jobPtr->total_out; + // Freeing resources + status = qpl_fini_job(jobPtr); + if (status != QPL_STS_OK) + { + throw Exception("doCompressDataSWNative cannot compress: qpl_fini_job fail", ErrorCodes::CANNOT_COMPRESS); + } + + delete[] jobPtr; + return compressed_size; +} + +uint32_t CompressionCodecDeflate::doCompressDataSW(const char * source, uint32_t source_size, char * dest) const +{ + uint32_t jobID = 0; + qpl_job * jobPtr = DeflateJobSWPool::instance().acquireJob(&jobID); + if (jobPtr == nullptr) + { + DeflateJobSWPool::instance().releaseJob(jobID); + LOG_WARNING(log, "doCompressDataSW acquireJob fail! switch to SW native compress..."); + return doCompressDataSWNative(source, source_size, dest); + } + qpl_status status; + uint32_t compressed_size = 0; + + jobPtr->op = qpl_op_compress; + jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); + jobPtr->next_out_ptr = reinterpret_cast(dest); + jobPtr->available_in = source_size; + jobPtr->available_out = getMaxCompressedDataSize(source_size); + jobPtr->level = qpl_high_level; + jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; + // Compression + status = qpl_execute_job(jobPtr); + if (QPL_STS_OK != status) + { + throw Exception("doCompressDataSW Cannot compress", ErrorCodes::CANNOT_COMPRESS); + } + compressed_size = jobPtr->total_out; + DeflateJobSWPool::instance().releaseJob(jobID); + return compressed_size; +} + +uint32_t CompressionCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest) const +{ + uint32_t jobID = 0; + qpl_job * jobPtr = DeflateJobHWPool::instance().acquireJob(&jobID); + if (jobPtr == nullptr) + { + DeflateJobHWPool::instance().releaseJob(jobID); + LOG_WARNING(log, "doCompressData HW acquireJob fail! switch to SW compress..."); + return doCompressDataSW(source, source_size, dest); + } + qpl_status status; + uint32_t compressed_size = 0; + + jobPtr->op = qpl_op_compress; + jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); + jobPtr->next_out_ptr = reinterpret_cast(dest); + jobPtr->available_in = source_size; + jobPtr->level = qpl_default_level; + jobPtr->available_out = getMaxCompressedDataSize(source_size); + jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; + // Compression + status = qpl_execute_job(jobPtr); + if (QPL_STS_OK == status) + { + compressed_size = jobPtr->total_out; + } + else + { + LOG_WARNING(log, "doCompressData HW fail! switch to SW compress ->status: '{}' ", static_cast(status)); + compressed_size = doCompressDataSW(source, source_size, dest); + } + DeflateJobHWPool::instance().releaseJob(jobID); + return compressed_size; +} + +UInt32 CompressionCodecDeflate::doCompressDataReq(const char * source, UInt32 source_size, char * dest, UInt32 & req_id) +{ + uint32_t jobID = 0; + req_id = 0; + qpl_job * jobPtr = DeflateJobHWPool::instance().acquireJob(&jobID); + if (jobPtr == nullptr) + { + DeflateJobHWPool::instance().releaseJob(jobID); + LOG_WARNING(log, "doCompressDataReq HW acquireJob fail! switch to SW compress..."); + return doCompressDataSW(source, source_size, dest); + } + qpl_status status; + + jobPtr->op = qpl_op_compress; + jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); + jobPtr->next_out_ptr = reinterpret_cast(dest); + jobPtr->available_in = source_size; + jobPtr->level = qpl_default_level; + jobPtr->available_out = getMaxCompressedDataSize(source_size); + jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; + // Compression + status = qpl_submit_job(jobPtr); + if (QPL_STS_OK != status) + { + LOG_WARNING(log, "doCompressDataReq HW fail! switch to SW compress ->status: '{}' ", static_cast(status)); + DeflateJobHWPool::instance().releaseJob(jobID); + return doCompressDataSW(source, source_size, dest); + } + //LOG_WARNING(log, "doCompressDataReq ->jobID:{}, source_size:{}",jobID, source_size); + jobCompAsyncList.push_back(jobID); + req_id = jobID; + return 0; +} + +uint32_t CompressionCodecDeflate::doCompressDataFlush(uint32_t req_id) +{ + uint32_t compressed_size = 0; + qpl_job * jobPtr = DeflateJobHWPool::instance().getJobPtr(req_id); + while (QPL_STS_BEING_PROCESSED == qpl_check_job(jobPtr)) + { + _tpause(1, __rdtsc() + 1000); + } + compressed_size = jobPtr->total_out; + DeflateJobHWPool::instance().releaseJob(req_id); + return compressed_size; +} + +void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const +{ + uint32_t jobID = 0; + qpl_job * jobPtr = DeflateJobHWPool::instance().acquireJob(&jobID); + if (jobPtr == nullptr) + { + DeflateJobHWPool::instance().releaseJob(jobID); + LOG_WARNING(log, "doDecompressData HW acquireJob fail! switch to SW decompress"); + return doDecompressDataSW(source, source_size, dest, uncompressed_size); + } + qpl_status status; + + // Performing a decompression operation + jobPtr->op = qpl_op_decompress; + jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); + jobPtr->next_out_ptr = reinterpret_cast(dest); + jobPtr->available_in = source_size; + jobPtr->available_out = uncompressed_size; + jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; + + // Decompression + status = qpl_execute_job(jobPtr); + if (status != QPL_STS_OK) + { + LOG_WARNING( + log, + "doDecompressData HW fail! switch to SW decompress ->status: '{}' ,source_size: '{}' ,uncompressed_size: '{}' ", + static_cast(status), + source_size, + uncompressed_size); + doDecompressDataSW(source, source_size, dest, uncompressed_size); + } + DeflateJobHWPool::instance().releaseJob(jobID); +} + +void CompressionCodecDeflate::doDecompressDataSWNative( + const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const +{ + qpl_status status; + uint32_t size = 0; + + // Job initialization + status = qpl_get_job_size(DeflateJobSWPool::SW_PATH, &size); + if (status != QPL_STS_OK) + { + throw Exception("doDecompressDataSWNative cannot decompress: qpl_get_job_size fail", ErrorCodes::CANNOT_DECOMPRESS); + } + qpl_job * jobPtr = reinterpret_cast(new uint8_t[size]); + + status = qpl_init_job(DeflateJobSWPool::SW_PATH, jobPtr); + if (status != QPL_STS_OK) + { + throw Exception("doDecompressDataSWNative cannot decompress: qpl_init_job fail", ErrorCodes::CANNOT_DECOMPRESS); + } + + // Performing a decompression operation + jobPtr->op = qpl_op_decompress; + jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); + jobPtr->next_out_ptr = reinterpret_cast(dest); + jobPtr->available_in = source_size; + jobPtr->available_out = uncompressed_size; + jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; + + // Decompression + status = qpl_execute_job(jobPtr); + if (status != QPL_STS_OK) + { + throw Exception("doDecompressDataSWNative cannot decompress: qpl_execute_job fail", ErrorCodes::CANNOT_DECOMPRESS); + } + // Freeing resources + status = qpl_fini_job(jobPtr); + if (status != QPL_STS_OK) + { + throw Exception("doDecompressDataSWNative cannot decompress: qpl_fini_job fail", ErrorCodes::CANNOT_DECOMPRESS); + } + delete[] jobPtr; +} + +void CompressionCodecDeflate::doDecompressDataSW(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const +{ + uint32_t jobID = 0; + qpl_job * jobPtr = DeflateJobSWPool::instance().acquireJob(&jobID); + if (jobPtr == nullptr) + { + DeflateJobSWPool::instance().releaseJob(jobID); + LOG_WARNING(log, "doDecompressDataSW acquireJob fail! switch to SW native decompress..."); + return doDecompressDataSWNative(source, source_size, dest, uncompressed_size); + } + qpl_status status; + + // Performing a decompression operation + jobPtr->op = qpl_op_decompress; + jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); + jobPtr->next_out_ptr = reinterpret_cast(dest); + jobPtr->available_in = source_size; + jobPtr->available_out = uncompressed_size; + jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; + + // Decompression + status = qpl_execute_job(jobPtr); + + if (QPL_STS_OK != status) + { + throw Exception("doDecompressDataSW cannot decompress", ErrorCodes::CANNOT_DECOMPRESS); + } + DeflateJobSWPool::instance().releaseJob(jobID); +} + +void CompressionCodecDeflate::doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) +{ + uint32_t jobID = 0; + qpl_job * jobPtr = DeflateJobHWPool::instance().acquireJob(&jobID); + if (jobPtr == nullptr) + { + DeflateJobHWPool::instance().releaseJob(jobID); + LOG_WARNING(log, "doDecompressDataReq acquireJob fail! switch to SW decompress"); + doDecompressDataSW(source, source_size, dest, uncompressed_size); + return; + } + qpl_status status; + + // Performing a decompression operation + jobPtr->op = qpl_op_decompress; + jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); + jobPtr->next_out_ptr = reinterpret_cast(dest); + jobPtr->available_in = source_size; + jobPtr->available_out = uncompressed_size; + jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; + + // Decompression + status = qpl_submit_job(jobPtr); + if (QPL_STS_OK == status) + { + jobDecompAsyncMap.insert(std::make_pair(jobID, jobPtr)); + } + else + { + DeflateJobHWPool::instance().releaseJob(jobID); + LOG_WARNING(log, "doDecompressDataReq HW fail! switch to SW decompress... ->status: '{}' ", static_cast(status)); + doDecompressDataSW(source, source_size, dest, uncompressed_size); + } +} + +void CompressionCodecDeflate::doDecompressDataFlush(void) +{ + uint32_t jobID = 0; + qpl_job * jobPtr = nullptr; + + + std::map::iterator it; + uint32_t nJobsProcessing = jobDecompAsyncMap.size(); + it = jobDecompAsyncMap.begin(); + + while (nJobsProcessing) + { + jobID = it->first; + jobPtr = it->second; + + if (QPL_STS_BEING_PROCESSED == qpl_check_job(jobPtr)) + { + it++; + } + else + { + DeflateJobHWPool::instance().releaseJob(jobID); + it = jobDecompAsyncMap.erase(it); + nJobsProcessing--; + } + if (it == jobDecompAsyncMap.end()) + { + it = jobDecompAsyncMap.begin(); + _tpause(1, __rdtsc() + 1000); + } + } +} + +void registerCodecDeflate(CompressionCodecFactory & factory) +{ + factory.registerSimpleCompressionCodec( + "DEFLATE", static_cast(CompressionMethodByte::Deflate), [&]() { return std::make_shared(); }); +} + +} diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h new file mode 100644 index 00000000000..ce8d3c1b8db --- /dev/null +++ b/src/Compression/CompressionCodecDeflate.h @@ -0,0 +1,332 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +namespace Poco +{ +class Logger; +} + +namespace DB +{ + +class DeflateJobHWPool +{ +public: + DeflateJobHWPool(); + ~DeflateJobHWPool(); + static DeflateJobHWPool & instance(); + static constexpr auto jobPoolSize = 1024; + static constexpr qpl_path_t PATH = qpl_path_hardware; + static qpl_job * jobPool[jobPoolSize]; + static std::atomic_bool jobLock[jobPoolSize]; + + qpl_job * ALWAYS_INLINE acquireJob(uint32_t * job_id) + { + uint32_t retry = 0; + auto index = random(jobPoolSize); + while (tryLockJob(index) == false) + { + index = random(jobPoolSize); + retry++; + if (retry > jobPoolSize) + { + return nullptr; + } + } + *job_id = jobPoolSize - index; + return jobPool[index]; + } + qpl_job * ALWAYS_INLINE releaseJob(uint32_t job_id) + { + uint32_t index = jobPoolSize - job_id; + ReleaseJobObjectGuard _(index); + return jobPool[index]; + } + qpl_job * ALWAYS_INLINE getJobPtr(uint32_t job_id) + { + uint32_t index = jobPoolSize - job_id; + return jobPool[index]; + } + +private: + size_t ALWAYS_INLINE random(uint32_t pool_size) + { + size_t tsc = 0; + unsigned lo, hi; + __asm__ volatile("rdtsc" : "=a"(lo), "=d"(hi) : :); + tsc = (((static_cast(hi)) << 32) | (static_cast(lo))); + return (static_cast((tsc * 44485709377909ULL) >> 4)) % pool_size; + } + + int32_t ALWAYS_INLINE get_job_size_helper() + { + static uint32_t size = 0; + if (size == 0) + { + const auto status = qpl_get_job_size(PATH, &size); + if (status != QPL_STS_OK) + { + return -1; + } + } + return size; + } + + int32_t ALWAYS_INLINE init_job_helper(qpl_job * qpl_job_ptr) + { + if (qpl_job_ptr == nullptr) + { + return -1; + } + auto status = qpl_init_job(PATH, qpl_job_ptr); + if (status != QPL_STS_OK) + { + return -1; + } + return 0; + } + + int32_t ALWAYS_INLINE initJobPool() + { + static bool initialized = false; + + if (initialized == false) + { + const int32_t size = get_job_size_helper(); + if (size < 0) + return -1; + for (int i = 0; i < jobPoolSize; ++i) + { + jobPool[i] = nullptr; + qpl_job * qpl_job_ptr = reinterpret_cast(new uint8_t[size]); + if (init_job_helper(qpl_job_ptr) < 0) + return -1; + jobPool[i] = qpl_job_ptr; + jobLock[i].store(false); + } + initialized = true; + } + return 0; + } + + bool ALWAYS_INLINE tryLockJob(size_t index) + { + bool expected = false; + return jobLock[index].compare_exchange_strong(expected, true); + } + + void ALWAYS_INLINE destroyJobPool() + { + const uint32_t size = get_job_size_helper(); + for (uint32_t i = 0; i < jobPoolSize && size > 0; ++i) + { + while (tryLockJob(i) == false) + { + } + if (jobPool[i]) + { + qpl_fini_job(jobPool[i]); + delete[] jobPool[i]; + } + jobPool[i] = nullptr; + jobLock[i].store(false); + } + } + + struct ReleaseJobObjectGuard + { + uint32_t index; + ReleaseJobObjectGuard() = delete; + + public: + ALWAYS_INLINE ReleaseJobObjectGuard(const uint32_t i) : index(i) + { + } + ALWAYS_INLINE ~ReleaseJobObjectGuard() + { + jobLock[index].store(false); + } + }; +}; + +class DeflateJobSWPool +{ +public: + DeflateJobSWPool(); + ~DeflateJobSWPool(); + static DeflateJobSWPool & instance(); + + static constexpr auto jobSWPoolSize = 128; + static constexpr qpl_path_t SW_PATH = qpl_path_software; + static qpl_job * jobSWPool[jobSWPoolSize]; + static std::atomic_bool jobSWLock[jobSWPoolSize]; + + qpl_job * ALWAYS_INLINE acquireJob(uint32_t * job_id) + { + uint32_t retry = 0; + auto index = random(jobSWPoolSize); + while (tryLockJob(index) == false) + { + index = random(jobSWPoolSize); + retry++; + if (retry > jobSWPoolSize) + { + return nullptr; + } + } + *job_id = jobSWPoolSize - index; + return jobSWPool[index]; + } + qpl_job * ALWAYS_INLINE releaseJob(uint32_t job_id) + { + uint32_t index = jobSWPoolSize - job_id; + ReleaseJobObjectGuard _(index); + return jobSWPool[index]; + } + +private: + size_t ALWAYS_INLINE random(uint32_t pool_size) + { + size_t tsc = 0; + unsigned lo, hi; + __asm__ volatile("rdtsc" : "=a"(lo), "=d"(hi) : :); + tsc = (((static_cast(hi)) << 32) | (static_cast(lo))); + return (static_cast((tsc * 44485709377909ULL) >> 4)) % pool_size; + } + + int32_t ALWAYS_INLINE get_job_size_helper() + { + static uint32_t size = 0; + if (size == 0) + { + const auto status = qpl_get_job_size(SW_PATH, &size); + if (status != QPL_STS_OK) + { + return -1; + } + } + return size; + } + + int32_t ALWAYS_INLINE init_job_helper(qpl_job * qpl_job_ptr) + { + if (qpl_job_ptr == nullptr) + { + return -1; + } + auto status = qpl_init_job(SW_PATH, qpl_job_ptr); + if (status != QPL_STS_OK) + { + return -1; + } + return 0; + } + + int32_t ALWAYS_INLINE initJobPool() + { + static bool initialized = false; + + if (initialized == false) + { + const int32_t size = get_job_size_helper(); + if (size < 0) + return -1; + for (int i = 0; i < jobSWPoolSize; ++i) + { + jobSWPool[i] = nullptr; + qpl_job * qpl_job_ptr = reinterpret_cast(new uint8_t[size]); + if (init_job_helper(qpl_job_ptr) < 0) + return -1; + jobSWPool[i] = qpl_job_ptr; + jobSWLock[i].store(false); + } + initialized = true; + } + return 0; + } + + bool ALWAYS_INLINE tryLockJob(size_t index) + { + bool expected = false; + return jobSWLock[index].compare_exchange_strong(expected, true); + } + + void ALWAYS_INLINE destroyJobPool() + { + const uint32_t size = get_job_size_helper(); + for (uint32_t i = 0; i < jobSWPoolSize && size > 0; ++i) + { + while (tryLockJob(i) == false) + { + } + if (jobSWPool[i]) + { + qpl_fini_job(jobSWPool[i]); + delete[] jobSWPool[i]; + } + jobSWPool[i] = nullptr; + jobSWLock[i].store(false); + } + } + + struct ReleaseJobObjectGuard + { + uint32_t index; + ReleaseJobObjectGuard() = delete; + + public: + ALWAYS_INLINE ReleaseJobObjectGuard(const uint32_t i) : index(i) + { + } + ALWAYS_INLINE ~ReleaseJobObjectGuard() + { + jobSWLock[index].store(false); + } + }; +}; +class CompressionCodecDeflate : public ICompressionCodec +{ +public: + CompressionCodecDeflate(); + ~CompressionCodecDeflate() override; + uint8_t getMethodByte() const override; + void updateHash(SipHash & hash) const override; + bool isAsyncSupported() const override; + +protected: + bool isCompression() const override + { + return true; + } + bool isGenericCompression() const override + { + return true; + } + + uint32_t doCompressData(const char * source, uint32_t source_size, char * dest) const override; + uint32_t doCompressDataSW(const char * source, uint32_t source_size, char * dest) const; + uint32_t doCompressDataSWNative(const char * source, uint32_t source_size, char * dest) const; + UInt32 doCompressDataReq(const char * source, UInt32 source_size, char * dest, uint32_t & req_id) override; + uint32_t doCompressDataFlush(uint32_t req_id) override; + + void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; + void doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) override; + void doDecompressDataSW(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; + void doDecompressDataSWNative(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const; + void doDecompressDataFlush(void) override; + +private: + uint32_t getMaxCompressedDataSize(uint32_t uncompressed_size) const override; + std::map jobDecompAsyncMap; + std::vector jobCompAsyncList; + Poco::Logger * log; +}; + +} diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index abf5e38a8c3..a0d79b3dc9c 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -166,7 +166,7 @@ void registerCodecLZ4(CompressionCodecFactory & factory); void registerCodecLZ4HC(CompressionCodecFactory & factory); void registerCodecZSTD(CompressionCodecFactory & factory); void registerCodecMultiple(CompressionCodecFactory & factory); - +void registerCodecDeflate(CompressionCodecFactory & factory); /// Keeper use only general-purpose codecs, so we don't need these special codecs /// in standalone build @@ -187,7 +187,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecZSTD(*this); registerCodecLZ4HC(*this); registerCodecMultiple(*this); - + registerCodecDeflate(*this); #ifndef KEEPER_STANDALONE_BUILD registerCodecDelta(*this); registerCodecT64(*this); diff --git a/src/Compression/CompressionInfo.h b/src/Compression/CompressionInfo.h index bbe8315f3ea..40b9b4edba7 100644 --- a/src/Compression/CompressionInfo.h +++ b/src/Compression/CompressionInfo.h @@ -44,7 +44,8 @@ enum class CompressionMethodByte : uint8_t DoubleDelta = 0x94, Gorilla = 0x95, AES_128_GCM_SIV = 0x96, - AES_256_GCM_SIV = 0x97 + AES_256_GCM_SIV = 0x97, + Deflate = 0x98, }; } diff --git a/src/Compression/ICompressionCodec.cpp b/src/Compression/ICompressionCodec.cpp index ba52aee69f8..c99358bd6cd 100644 --- a/src/Compression/ICompressionCodec.cpp +++ b/src/Compression/ICompressionCodec.cpp @@ -90,9 +90,35 @@ UInt32 ICompressionCodec::compress(const char * source, UInt32 source_size, char unalignedStore(&dest[5], source_size); return header_size + compressed_bytes_written; } +UInt32 ICompressionCodec::compressReq(const char * source, UInt32 source_size, char * dest, UInt32 & req_id) +{ + assert(source != nullptr && dest != nullptr); + dest[0] = getMethodByte(); + UInt8 header_size = getHeaderSize(); + UInt32 res = doCompressDataReq(source, source_size, &dest[header_size], req_id); + if (res > 0) + { + unalignedStore(&dest[1], res + header_size); + unalignedStore(&dest[5], source_size); + return header_size + res; + } + else + { + unalignedStore(&dest[5], source_size); + return 0; + } +} -UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, char * dest) const +UInt32 ICompressionCodec::compressFlush(UInt32 req_id, char * dest) +{ + UInt32 compressed_bytes_written = doCompressDataFlush(req_id); + UInt8 header_size = getHeaderSize(); + unalignedStore(&dest[1], compressed_bytes_written + header_size); + return header_size + compressed_bytes_written; +} + +UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, char * dest, UInt8 req_type) { assert(source != nullptr && dest != nullptr); @@ -106,11 +132,27 @@ UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, ch throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Can't decompress data with codec byte {} using codec with byte {}", method, our_method); UInt32 decompressed_size = readDecompressedBlockSize(source); - doDecompressData(&source[header_size], source_size - header_size, dest, decompressed_size); + switch (req_type) + { + case 0: + doDecompressData(&source[header_size], source_size - header_size, dest, decompressed_size); + break; + case 1: + doDecompressDataReq(&source[header_size], source_size - header_size, dest, decompressed_size); + break; + case 2: + doDecompressDataSW(&source[header_size], source_size - header_size, dest, decompressed_size); + break; + } return decompressed_size; } +void ICompressionCodec::decompressFlush(void) +{ + doDecompressDataFlush(); +} + UInt32 ICompressionCodec::readCompressedBlockSize(const char * source) { UInt32 compressed_block_size = unalignedLoad(&source[1]); diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index a741e65dfdd..16c6a6e04d9 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -44,9 +44,24 @@ public: /// Compressed bytes from uncompressed source to dest. Dest should preallocate memory UInt32 compress(const char * source, UInt32 source_size, char * dest) const; + UInt32 compressReq(const char * source, UInt32 source_size, char * dest, UInt32 & req_id); + // Flush all asychronous request for compression + UInt32 compressFlush(UInt32 req_id, char * dest); + /// Decompress bytes from compressed source to dest. Dest should preallocate memory; + // reqType is specific for HW decompressor: + //0 means sychronous request by default; + //1 means asychronous request, must be used in pair with decompressFlush; + //2 means SW decompressor instead of HW + UInt32 decompress(const char * source, UInt32 source_size, char * dest, UInt8 req_type = 0); - /// Decompress bytes from compressed source to dest. Dest should preallocate memory - UInt32 decompress(const char * source, UInt32 source_size, char * dest) const; + /// Flush all asychronous request for decompression + void decompressFlush(void); + + /// Some codecs (QPL_deflate, for example) support asychronous request + virtual bool isAsyncSupported() const + { + return false; + } /// Number of bytes, that will be used to compress uncompressed_size bytes with current codec virtual UInt32 getCompressedReserveSize(UInt32 uncompressed_size) const @@ -95,9 +110,38 @@ protected: /// Actually compress data, without header virtual UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const = 0; + /// Asynchronous compression request to HW decompressor + virtual UInt32 doCompressDataReq(const char * source, UInt32 source_size, char * dest, UInt32 & req_id) + { + req_id = 0; + return doCompressData(source, source_size, dest); + } + + /// Flush asynchronous request for compression + virtual UInt32 doCompressDataFlush(UInt32 req_id = 0) + { + return req_id; + } + /// Actually decompress data without header virtual void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const = 0; + /// Asynchronous decompression request to HW decompressor + virtual void doDecompressDataReq(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) + { + doDecompressData(source, source_size, dest, uncompressed_size); + } + + /// SW decompressor instead of HW + virtual void doDecompressDataSW(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const + { + doDecompressData(source, source_size, dest, uncompressed_size); + } + + /// Flush asynchronous request for decompression + virtual void doDecompressDataFlush(void) + { + } /// Construct and set codec description from codec name and arguments. Must be called in codec constructor. void setCodecDescription(const String & name, const ASTs & arguments = {}); From 53f29f837d14afcd9d6046a74d74086d1d31f432 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 26 Apr 2022 14:51:31 -0400 Subject: [PATCH 007/659] fixed submodule:qpl build issue --- contrib/qpl | 1 + contrib/qpl-cmake/CMakeLists.txt | 1 + 2 files changed, 2 insertions(+) create mode 160000 contrib/qpl diff --git a/contrib/qpl b/contrib/qpl new file mode 160000 index 00000000000..cdc8442f7a5 --- /dev/null +++ b/contrib/qpl @@ -0,0 +1 @@ +Subproject commit cdc8442f7a5e7a6ff6eea39c69665e0c5034d85d diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 605b2e8cfcf..03a722943fd 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -6,6 +6,7 @@ set (QPL_INCLUDE_DIRS ) set (LOG_HW_INIT ON) set (EFFICIENT_WAIT ON) +set (LIB_FUZZING_ENGINE ON) set (QPL_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl") set (QPL_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/qpl") From 484b08b8157f5ea96127aaba56594f20dcfa724b Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 26 Apr 2022 21:37:38 -0400 Subject: [PATCH 008/659] improve qpl-cmake to build qpl independently --- contrib/qpl-cmake/CMakeLists.txt | 346 +++++++++++++++++++++++++++++-- contrib/qpl-cmake/uuid/uuid.h | 104 ++++++++++ 2 files changed, 437 insertions(+), 13 deletions(-) create mode 100644 contrib/qpl-cmake/uuid/uuid.h diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 03a722943fd..bafef8c44fc 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -1,18 +1,338 @@ -# The Intel® QPL provides high performance implementations of data processing functions for existing hardware accelerator, and/or software path in case if hardware accelerator is not available. +## The Intel® QPL provides high performance implementations of data processing functions for existing hardware accelerator, and/or software path in case if hardware accelerator is not available. + +set (QPL_VERSION 0.1.21) +set (QPL_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl") +set (QPL_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl/sources") +set (QPL_BINARY_DIR "${ClickHouse_SOURCE_DIR}/build/contrib/qpl") +set (UUID_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl-cmake") -set (QPL_INCLUDE_DIRS - "${ClickHouse_SOURCE_DIR}/contrib/qpl/include" - "${ClickHouse_BINARY_DIR}/contrib/qpl/include" -) set (LOG_HW_INIT ON) set (EFFICIENT_WAIT ON) -set (LIB_FUZZING_ENGINE ON) +set (BLOCK_ON_FAULT ON) -set (QPL_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl") -set (QPL_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/qpl") -add_subdirectory ("${QPL_SOURCE_DIR}" "${QPL_BINARY_DIR}") +message(STATUS "Intel QPL version: ${QPL_VERSION}") -add_library (_qpl INTERFACE) -target_link_libraries (_qpl INTERFACE qpl) -target_include_directories (_qpl SYSTEM BEFORE INTERFACE ${QPL_INCLUDE_DIRS}) -add_library(ch_contrib::qpl ALIAS _qpl) \ No newline at end of file +include("${QPL_PROJECT_DIR}/cmake/CompileOptions.cmake") + +# [SUBDIR]isal +enable_language(ASM_NASM) + +set(ISAL_C_SRC ${QPL_SRC_DIR}/isal/igzip/adler32_base.c + ${QPL_SRC_DIR}/isal/igzip/huff_codes.c + ${QPL_SRC_DIR}/isal/igzip/hufftables_c.c + ${QPL_SRC_DIR}/isal/igzip/igzip.c + ${QPL_SRC_DIR}/isal/igzip/igzip_base.c + ${QPL_SRC_DIR}/isal/igzip/flatten_ll.c + ${QPL_SRC_DIR}/isal/igzip/encode_df.c + ${QPL_SRC_DIR}/isal/igzip/igzip_icf_base.c + ${QPL_SRC_DIR}/isal/igzip/igzip_inflate.c + ${QPL_SRC_DIR}/isal/igzip/igzip_icf_body.c + ${QPL_SRC_DIR}/isal/crc/crc_base.c + ${QPL_SRC_DIR}/isal/crc/crc64_base.c) + +set(ISAL_ASM_SRC ${QPL_SRC_DIR}/isal/igzip/igzip_body.asm + ${QPL_SRC_DIR}/isal/igzip/igzip_gen_icf_map_lh1_04.asm + ${QPL_SRC_DIR}/isal/igzip/igzip_gen_icf_map_lh1_06.asm + ${QPL_SRC_DIR}/isal/igzip/igzip_decode_block_stateless_04.asm + ${QPL_SRC_DIR}/isal/igzip/igzip_finish.asm + ${QPL_SRC_DIR}/isal/igzip/encode_df_04.asm + ${QPL_SRC_DIR}/isal/igzip/encode_df_06.asm + ${QPL_SRC_DIR}/isal/igzip/igzip_decode_block_stateless_01.asm + ${QPL_SRC_DIR}/isal/igzip/proc_heap.asm + ${QPL_SRC_DIR}/isal/igzip/igzip_icf_body_h1_gr_bt.asm + ${QPL_SRC_DIR}/isal/igzip/igzip_icf_finish.asm + ${QPL_SRC_DIR}/isal/igzip/igzip_inflate_multibinary.asm + ${QPL_SRC_DIR}/isal/igzip/igzip_update_histogram_01.asm + ${QPL_SRC_DIR}/isal/igzip/igzip_update_histogram_04.asm + ${QPL_SRC_DIR}/isal/igzip/rfc1951_lookup.asm + ${QPL_SRC_DIR}/isal/igzip/adler32_sse.asm + ${QPL_SRC_DIR}/isal/igzip/adler32_avx2_4.asm + ${QPL_SRC_DIR}/isal/igzip/igzip_deflate_hash.asm + ${QPL_SRC_DIR}/isal/igzip/igzip_set_long_icf_fg_04.asm + ${QPL_SRC_DIR}/isal/igzip/igzip_set_long_icf_fg_06.asm + ${QPL_SRC_DIR}/isal/igzip/igzip_multibinary.asm + ${QPL_SRC_DIR}/isal/igzip/stdmac.asm + ${QPL_SRC_DIR}/isal/crc/crc_multibinary.asm + ${QPL_SRC_DIR}/isal/crc/crc32_gzip_refl_by8.asm + ${QPL_SRC_DIR}/isal/crc/crc32_gzip_refl_by8_02.asm + ${QPL_SRC_DIR}/isal/crc/crc32_gzip_refl_by16_10.asm + ${QPL_SRC_DIR}/isal/crc/crc32_ieee_01.asm + ${QPL_SRC_DIR}/isal/crc/crc32_ieee_02.asm + ${QPL_SRC_DIR}/isal/crc/crc32_ieee_by4.asm + ${QPL_SRC_DIR}/isal/crc/crc32_ieee_by16_10.asm + ${QPL_SRC_DIR}/isal/crc/crc32_iscsi_00.asm + ${QPL_SRC_DIR}/isal/crc/crc32_iscsi_01.asm + ${QPL_SRC_DIR}/isal/crc/crc32_iscsi_by16_10.asm) + +set(DISABLED_WARNINGS_FLAGS /wd4244 /wd4101 /wd4267 /wd4018 /wd4146 /wd4334 /wd4308) +set(FEATURE_FLAGS -DHAVE_AS_KNOWS_AVX512 -DAS_FEATURE_LEVEL=10) + +# Adding ISA-L library target +add_library(isal OBJECT ${ISAL_C_SRC}) +add_library(isal_asm OBJECT ${ISAL_ASM_SRC}) + +# Setting external and internal interfaces for ISA-L library +target_include_directories(isal + PUBLIC $ + PRIVATE ${QPL_SRC_DIR}/isal/include + PUBLIC ${QPL_SRC_DIR}/isal/igzip) + +set_target_properties(isal PROPERTIES + CXX_STANDARD 11 + C_STANDARD 99) + +target_compile_options(isal PRIVATE + "$<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}>" + "$<$:/WX;${DISABLED_WARNINGS_FLAGS};${QPL_WINDOWS_TOOLCHAIN_REQUIRED_FLAGS}>" + "$<$:>" + "$<$:>") + +target_compile_options(isal_asm PUBLIC "-I${QPL_SRC_DIR}/isal/include/" + PUBLIC "-I${QPL_SRC_DIR}/isal/igzip/" + PUBLIC "-I${QPL_SRC_DIR}/isal/crc/" + PUBLIC "-DHAVE_AS_KNOWS_AVX512" + PUBLIC "-DAS_FEATURE_LEVEL=10" + PUBLIC "-DQPL_LIB") + +target_compile_definitions(isal PUBLIC + QPL_LIB + NDEBUG) + +# [SUBDIR]core-sw +enable_language(C) + +# +# Create avx512 library +# +#set(CMAKE_INCLUDE_CURRENT_DIR ON) + +# Find Core Sources +file(GLOB SOURCES + ${QPL_SRC_DIR}/core-sw/src/checksums/*.c + ${QPL_SRC_DIR}/core-sw/src/filtering/*.c + ${QPL_SRC_DIR}/core-sw/src/other/*.c + ${QPL_SRC_DIR}/core-sw/src/compression/*.c) + +file(GLOB DATA_SOURCES + ${QPL_SRC_DIR}/core-sw/src/data/*.c) + +# Create library +add_library(qplcore_avx512 OBJECT ${SOURCES}) + +target_compile_definitions(qplcore_avx512 PRIVATE PLATFORM=2) + +target_include_directories(qplcore_avx512 + PUBLIC $ + PUBLIC $ + PUBLIC $ + PRIVATE $) + +set_target_properties(qplcore_avx512 PROPERTIES + $<$:C_STANDARD 18> + $<$:C_STANDARD 17>) + +target_link_libraries(qplcore_avx512 ${CMAKE_DL_LIBS} isal) + +if (WIN32) + target_compile_options(qplcore_avx512 + PRIVATE ${QPL_WINDOWS_TOOLCHAIN_REQUIRED_FLAGS} + PRIVATE /arch:AVX512 + PRIVATE "$<$:>" + PRIVATE "$<$:-O2>") +else () + target_compile_options(qplcore_avx512 + PRIVATE ${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS} + PRIVATE -march=skylake-avx512 + PRIVATE "$<$:>" + PRIVATE "$<$:-O3;-D_FORTIFY_SOURCE=2>") +endif () + +target_compile_definitions(qplcore_avx512 PUBLIC QPL_BADARG_CHECK) + +# +# Create px library +# +#set(CMAKE_INCLUDE_CURRENT_DIR ON) + +# Create library +add_library(qplcore_px OBJECT ${SOURCES} ${DATA_SOURCES}) + +target_compile_definitions(qplcore_px PRIVATE PLATFORM=0) + +target_include_directories(qplcore_px + PUBLIC $ + PUBLIC $ + PUBLIC $ + PRIVATE $) + +set_target_properties(qplcore_px PROPERTIES + $<$:C_STANDARD 18> + $<$:C_STANDARD 17>) + +target_link_libraries(qplcore_px isal ${CMAKE_DL_LIBS}) + +if (WIN32) + target_compile_options(qplcore_px + PRIVATE "$<$:>" + PRIVATE ${QPL_WINDOWS_TOOLCHAIN_REQUIRED_FLAGS} + PRIVATE "$<$:-O2>") +else () + target_compile_options(qplcore_px + PRIVATE ${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS} + PRIVATE "$<$:>" + PRIVATE "$<$:-O3;-D_FORTIFY_SOURCE=2>") +endif () + +target_compile_definitions(qplcore_px PUBLIC QPL_BADARG_CHECK) + +# [SUBDIR]core-iaa +file(GLOB HW_PATH_SRC ${QPL_SRC_DIR}/core-iaa/sources/aecs/*.c + ${QPL_SRC_DIR}/core-iaa/sources/aecs/*.cpp + ${QPL_SRC_DIR}/core-iaa/sources/driver_loader/*.c + ${QPL_SRC_DIR}/core-iaa/sources/driver_loader/*.cpp + ${QPL_SRC_DIR}/core-iaa/sources/descriptors/*.c + ${QPL_SRC_DIR}/core-iaa/sources/descriptors/*.cpp + ${QPL_SRC_DIR}/core-iaa/sources/bit_rev.c) + +# Create library +add_library(core_iaa OBJECT ${HW_PATH_SRC}) + +target_include_directories(core_iaa + PRIVATE ${UUID_DIR} + PUBLIC $ + PRIVATE $ + PRIVATE $) + +set_target_properties(core_iaa PROPERTIES + $<$:C_STANDARD 18> + $<$:C_STANDARD 17> + CXX_STANDARD 17) + +if (WIN32) + modify_standard_language_flag(LANGUAGE_NAME "CXX" + FLAG_NAME "/GR" + NEW_FLAG_VALUE "-") +endif () + +target_compile_options(core_iaa + PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}; + $<$:-O3;-D_FORTIFY_SOURCE=2>> + PRIVATE $<$:${QPL_WINDOWS_TOOLCHAIN_REQUIRED_FLAGS}; + $<$:-O2>>) + +target_compile_features(core_iaa PRIVATE c_std_11) + +target_compile_definitions(core_iaa PRIVATE QPL_BADARG_CHECK + PRIVATE $<$: BLOCK_ON_FAULT_ENABLED> + PRIVATE $<$:LOG_HW_INIT>) + +# [SUBDIR]middle-layer +enable_language(CXX) + +generate_unpack_kernel_arrays(${QPL_BINARY_DIR}) + +file(GLOB MIDDLE_LAYER_SRC + ${QPL_SRC_DIR}/middle-layer/analytics/*.cpp + ${QPL_SRC_DIR}/middle-layer/c_wrapper/*.cpp + ${QPL_SRC_DIR}/middle-layer/checksum/*.cpp + ${QPL_SRC_DIR}/middle-layer/common/*.cpp + ${QPL_SRC_DIR}/middle-layer/compression/*.cpp + ${QPL_SRC_DIR}/middle-layer/compression/*/*.cpp + ${QPL_SRC_DIR}/middle-layer/compression/*/*/*.cpp + ${QPL_SRC_DIR}/middle-layer/dispatcher/*.cpp + ${QPL_SRC_DIR}/middle-layer/other/*.cpp + ${QPL_SRC_DIR}/middle-layer/util/*.cpp + ${QPL_SRC_DIR}/middle-layer/inflate/*.cpp + ${QPL_SRC_DIR}/core-iaa/sources/accelerator/*.cpp) # todo + +file(GLOB GENERATED_PX_TABLES_SRC ${QPL_BINARY_DIR}/generated/px_*.cpp) +file(GLOB GENERATED_AVX512_TABLES_SRC ${QPL_BINARY_DIR}/generated/avx512_*.cpp) + +add_library(middle_layer_lib OBJECT + ${GENERATED_PX_TABLES_SRC} + ${GENERATED_AVX512_TABLES_SRC} + ${MIDDLE_LAYER_SRC}) + +if (WIN32) + modify_standard_language_flag(LANGUAGE_NAME "CXX" + FLAG_NAME "/GR" + NEW_FLAG_VALUE "-") +endif () + +target_compile_options(middle_layer_lib + PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}; + ${QPL_LINUX_TOOLCHAIN_DYNAMIC_LIBRARY_FLAGS}; + $<$:-O3;-D_FORTIFY_SOURCE=2>> + PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}> + PRIVATE $<$:${QPL_WINDOWS_TOOLCHAIN_REQUIRED_FLAGS}; + ${QPL_WINDOWS_TOOLCHAIN_DYNAMIC_LIBRARY_FLAGS} + $<$:-O2>> + PRIVATE $<$:${QPL_WINDOWS_TOOLCHAIN_CPP_EMBEDDED_FLAGS}>) + +target_compile_definitions(middle_layer_lib + PUBLIC QPL_VERSION="${CMAKE_PROJECT_VERSION}" + PUBLIC $<$:_ENABLE_EXTENDED_ALIGNED_STORAGE> + PUBLIC $<$:LOG_HW_INIT> + PUBLIC $<$:QPL_EFFICIENT_WAIT> + PUBLIC QPL_BADARG_CHECK) + +set_target_properties(middle_layer_lib PROPERTIES CXX_STANDARD 17) +set_source_files_properties(${GENERATED_PX_TABLES_SRC} PROPERTIES COMPILE_DEFINITIONS PLATFORM=0) +set_source_files_properties(${GENERATED_AVX512_TABLES_SRC} PROPERTIES COMPILE_DEFINITIONS PLATFORM=2) + +target_include_directories(middle_layer_lib + PRIVATE ${UUID_DIR} + PUBLIC $ + PUBLIC $ + PUBLIC $ + PUBLIC $ + PUBLIC $ + PUBLIC $) + +target_compile_definitions(middle_layer_lib PUBLIC -DQPL_LIB) +# [SUBDIR]c_api +enable_language(C CXX) + +file(GLOB_RECURSE QPL_C_API_SRC + ${QPL_SRC_DIR}/c_api/*.c + ${QPL_SRC_DIR}/c_api/*.cpp) + +add_library(_qpl STATIC ${QPL_C_API_SRC} + $ + $ + $ + $ + $ + $ + $) + +target_include_directories(_qpl + PUBLIC $ + PRIVATE $ + PRIVATE $) + +set_target_properties(_qpl PROPERTIES + $<$:C_STANDARD 18> + $<$:C_STANDARD 17> + CXX_STANDARD 17) + +target_compile_options(_qpl + PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}; + ${QPL_LINUX_TOOLCHAIN_DYNAMIC_LIBRARY_FLAGS}; + $<$:-O3;-D_FORTIFY_SOURCE=2>> + PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}> + PRIVATE $<$:${QPL_WINDOWS_TOOLCHAIN_REQUIRED_FLAGS}; + ${QPL_WINDOWS_TOOLCHAIN_DYNAMIC_LIBRARY_FLAGS} + $<$:-O2>> + PRIVATE $<$:${QPL_WINDOWS_TOOLCHAIN_CPP_EMBEDDED_FLAGS}>) + +target_compile_definitions(_qpl + PUBLIC -DQPL_LIB + PUBLIC -DQPL_BADARG_CHECK + PUBLIC $<$:_ENABLE_EXTENDED_ALIGNED_STORAGE>) + +target_link_libraries(_qpl + PRIVATE ${CMAKE_DL_LIBS}) + +add_library (ch_contrib::qpl ALIAS _qpl) +target_include_directories(_qpl PUBLIC "${QPL_PROJECT_DIR}/include") diff --git a/contrib/qpl-cmake/uuid/uuid.h b/contrib/qpl-cmake/uuid/uuid.h new file mode 100644 index 00000000000..874d65a196d --- /dev/null +++ b/contrib/qpl-cmake/uuid/uuid.h @@ -0,0 +1,104 @@ +/* + * Public include file for the UUID library + * + * Copyright (C) 1996, 1997, 1998 Theodore Ts'o. + * + * %Begin-Header% + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * 1. Redistributions of source code must retain the above copyright + * notice, and the entire permission notice in its entirety, + * including the disclaimer of warranties. + * 2. Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in the + * documentation and/or other materials provided with the distribution. + * 3. The name of the author may not be used to endorse or promote + * products derived from this software without specific prior + * written permission. + * + * THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESS OR IMPLIED + * WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES + * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE, ALL OF + * WHICH ARE HEREBY DISCLAIMED. IN NO EVENT SHALL THE AUTHOR BE + * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT + * OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR + * BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF + * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE + * USE OF THIS SOFTWARE, EVEN IF NOT ADVISED OF THE POSSIBILITY OF SUCH + * DAMAGE. + * %End-Header% + */ + +#ifndef _UUID_UUID_H +#define _UUID_UUID_H + +#include +#ifndef _WIN32 +#include +#endif +#include + +typedef unsigned char uuid_t[16]; + +/* UUID Variant definitions */ +#define UUID_VARIANT_NCS 0 +#define UUID_VARIANT_DCE 1 +#define UUID_VARIANT_MICROSOFT 2 +#define UUID_VARIANT_OTHER 3 + +/* UUID Type definitions */ +#define UUID_TYPE_DCE_TIME 1 +#define UUID_TYPE_DCE_RANDOM 4 + +/* Allow UUID constants to be defined */ +#ifdef __GNUC__ +#define UUID_DEFINE(name,u0,u1,u2,u3,u4,u5,u6,u7,u8,u9,u10,u11,u12,u13,u14,u15) \ + static const uuid_t name __attribute__ ((unused)) = {u0,u1,u2,u3,u4,u5,u6,u7,u8,u9,u10,u11,u12,u13,u14,u15} +#else +#define UUID_DEFINE(name,u0,u1,u2,u3,u4,u5,u6,u7,u8,u9,u10,u11,u12,u13,u14,u15) \ + static const uuid_t name = {u0,u1,u2,u3,u4,u5,u6,u7,u8,u9,u10,u11,u12,u13,u14,u15} +#endif + +#ifdef __cplusplus +extern "C" { +#endif + +/* clear.c */ +void uuid_clear(uuid_t uu); + +/* compare.c */ +int uuid_compare(const uuid_t uu1, const uuid_t uu2); + +/* copy.c */ +void uuid_copy(uuid_t dst, const uuid_t src); + +/* gen_uuid.c */ +void uuid_generate(uuid_t out); +void uuid_generate_random(uuid_t out); +void uuid_generate_time(uuid_t out); +int uuid_generate_time_safe(uuid_t out); + +/* isnull.c */ +int uuid_is_null(const uuid_t uu); + +/* parse.c */ +int uuid_parse(const char *in, uuid_t uu); + +/* unparse.c */ +void uuid_unparse(const uuid_t uu, char *out); +void uuid_unparse_lower(const uuid_t uu, char *out); +void uuid_unparse_upper(const uuid_t uu, char *out); + +/* uuid_time.c */ +time_t uuid_time(const uuid_t uu, struct timeval *ret_tv); +int uuid_type(const uuid_t uu); +int uuid_variant(const uuid_t uu); + +#ifdef __cplusplus +} +#endif + +#endif /* _UUID_UUID_H */ From 8788185e82c3192881504af7d2211e08905c74ed Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 27 Apr 2022 10:52:44 -0400 Subject: [PATCH 009/659] fixed typos issue in code --- src/Compression/CompressionCodecDeflate.cpp | 2 +- src/Compression/ICompressionCodec.h | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 1e3faa2d44f..a6b4daf5d3a 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -64,7 +64,7 @@ CompressionCodecDeflate::~CompressionCodecDeflate() { if (jobDecompAsyncMap.size() > 0) { - LOG_ERROR(log, "Exception -> find un-released job when CompressionCodecDeflate destory"); + LOG_ERROR(log, "Exception -> find un-released job when CompressionCodecDeflate destroy"); //doDecompressDataFlush(); for (auto it : jobDecompAsyncMap) { diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 16c6a6e04d9..594e24364ff 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -45,19 +45,19 @@ public: /// Compressed bytes from uncompressed source to dest. Dest should preallocate memory UInt32 compress(const char * source, UInt32 source_size, char * dest) const; UInt32 compressReq(const char * source, UInt32 source_size, char * dest, UInt32 & req_id); - // Flush all asychronous request for compression + // Flush all asynchronous request for compression UInt32 compressFlush(UInt32 req_id, char * dest); /// Decompress bytes from compressed source to dest. Dest should preallocate memory; // reqType is specific for HW decompressor: - //0 means sychronous request by default; - //1 means asychronous request, must be used in pair with decompressFlush; + //0 means synchronous request by default; + //1 means asynchronous request, must be used in pair with decompressFlush; //2 means SW decompressor instead of HW UInt32 decompress(const char * source, UInt32 source_size, char * dest, UInt8 req_type = 0); - /// Flush all asychronous request for decompression + /// Flush all asynchronous request for decompression void decompressFlush(void); - /// Some codecs (QPL_deflate, for example) support asychronous request + /// Some codecs (QPL_deflate, for example) support asynchronous request virtual bool isAsyncSupported() const { return false; From a74e8dbc841241c9b5087457087aca29443160fc Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 27 Apr 2022 13:54:57 -0400 Subject: [PATCH 010/659] add definition for qpl version --- contrib/qpl-cmake/CMakeLists.txt | 16 +++++++++++++--- contrib/sysroot | 2 +- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index bafef8c44fc..8af0230a4f8 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -1,14 +1,24 @@ ## The Intel® QPL provides high performance implementations of data processing functions for existing hardware accelerator, and/or software path in case if hardware accelerator is not available. -set (QPL_VERSION 0.1.21) set (QPL_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl") set (QPL_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl/sources") set (QPL_BINARY_DIR "${ClickHouse_SOURCE_DIR}/build/contrib/qpl") set (UUID_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl-cmake") -set (LOG_HW_INIT ON) set (EFFICIENT_WAIT ON) set (BLOCK_ON_FAULT ON) +set (LOG_HW_INIT OFF) +set (SANITIZE_MEMORY OFF) +set (SANITIZE_THREADS OFF) +set (LIB_FUZZING_ENGINE OFF) + +function(GetLibraryVersion _content _outputVar) + string(REGEX MATCHALL ".*Qpl VERSION+.* (.+).*LANGUAGES" VERSION_REGEX "${_content}") + SET(${_outputVar} ${CMAKE_MATCH_1} PARENT_SCOPE) +endfunction() + +FILE(READ "${QPL_PROJECT_DIR}/CMakeLists.txt" HEADER_CONTENT) +GetLibraryVersion("${HEADER_CONTENT}" QPL_VERSION) message(STATUS "Intel QPL version: ${QPL_VERSION}") @@ -270,7 +280,7 @@ target_compile_options(middle_layer_lib PRIVATE $<$:${QPL_WINDOWS_TOOLCHAIN_CPP_EMBEDDED_FLAGS}>) target_compile_definitions(middle_layer_lib - PUBLIC QPL_VERSION="${CMAKE_PROJECT_VERSION}" + PUBLIC QPL_VERSION="${QPL_VERSION}" PUBLIC $<$:_ENABLE_EXTENDED_ALIGNED_STORAGE> PUBLIC $<$:LOG_HW_INIT> PUBLIC $<$:QPL_EFFICIENT_WAIT> diff --git a/contrib/sysroot b/contrib/sysroot index e9fb375d0a1..bbcac834526 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit e9fb375d0a1e5ebfd74c043f088f2342552103f8 +Subproject commit bbcac834526d90d1e764164b861be426891d1743 From 34934b80a995e9c3f092e086b63af23afe0b2b39 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 29 Apr 2022 11:40:08 -0400 Subject: [PATCH 011/659] update qpl deflate supporting test script and docs --- docker/test/fasttest/run.sh | 1 + docs/en/development/contrib.md | 1 + docs/en/operations/server-configuration-parameters/settings.md | 2 +- docs/ru/development/contrib.md | 1 + docs/ru/operations/server-configuration-parameters/settings.md | 2 +- docs/zh/development/contrib.md | 1 + programs/keeper/CMakeLists.txt | 1 + 7 files changed, 7 insertions(+), 2 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index d9a5bb23a80..df227dfbcad 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -176,6 +176,7 @@ function clone_submodules contrib/NuRaft contrib/jemalloc contrib/replxx + contrib/qpl ) git submodule sync diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index 7cbe32fdd8b..6abd27707ba 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -85,6 +85,7 @@ The list of third-party libraries: | xz | [Public Domain](https://github.com/xz-mirror/xz/blob/869b9d1b4edd6df07f819d360d306251f8147353/COPYING) | | zlib-ng | [zLib](https://github.com/ClickHouse-Extras/zlib-ng/blob/6a5e93b9007782115f7f7e5235dedc81c4f1facb/LICENSE.md) | | zstd | [BSD](https://github.com/facebook/zstd/blob/a488ba114ec17ea1054b9057c26a046fc122b3b6/LICENSE) | +| qpl | [MIT](https://github.com/intel/qpl/blob/cdc8442f7a5e7a6ff6eea39c69665e0c5034d85d/LICENSE) | The list of third-party libraries can be obtained by the following query: diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 32697cdfba0..344075d2e03 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -45,7 +45,7 @@ Configuration template: - `min_part_size` – The minimum size of a data part. - `min_part_size_ratio` – The ratio of the data part size to the table size. -- `method` – Compression method. Acceptable values: `lz4`, `lz4hc`, `zstd`. +- `method` – Compression method. Acceptable values: `lz4`, `lz4hc`, `zstd`,`deflate`. - `level` – Compression level. See [Codecs](../../sql-reference/statements/create/table.md#create-query-general-purpose-codecs). You can configure multiple `` sections. diff --git a/docs/ru/development/contrib.md b/docs/ru/development/contrib.md index b98ed847a0b..a7b6f7dc5c9 100644 --- a/docs/ru/development/contrib.md +++ b/docs/ru/development/contrib.md @@ -85,6 +85,7 @@ sidebar_label: "Используемые сторонние библиотеки | xz | [Public Domain](https://github.com/xz-mirror/xz/blob/869b9d1b4edd6df07f819d360d306251f8147353/COPYING) | | zlib-ng | [zLib](https://github.com/ClickHouse-Extras/zlib-ng/blob/6a5e93b9007782115f7f7e5235dedc81c4f1facb/LICENSE.md) | | zstd | [BSD](https://github.com/facebook/zstd/blob/a488ba114ec17ea1054b9057c26a046fc122b3b6/LICENSE) | +| qpl | [MIT](https://github.com/intel/qpl/blob/cdc8442f7a5e7a6ff6eea39c69665e0c5034d85d/LICENSE) | Список всех сторонних библиотек можно получить с помощью запроса: diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 269c00fb1eb..fcb6b16e3a6 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -44,7 +44,7 @@ ClickHouse перезагружает встроенные словари с з - `min_part_size` - Минимальный размер части таблицы. - `min_part_size_ratio` - Отношение размера минимальной части таблицы к полному размеру таблицы. -- `method` - Метод сжатия. Возможные значения: `lz4`, `lz4hc`, `zstd`. +- `method` - Метод сжатия. Возможные значения: `lz4`, `lz4hc`, `zstd`,`deflate`. - `level` – Уровень сжатия. См. [Кодеки](../../sql-reference/statements/create/table/#create-query-common-purpose-codecs). Можно сконфигурировать несколько разделов ``. diff --git a/docs/zh/development/contrib.md b/docs/zh/development/contrib.md index 8e8efc3c04e..0f6f17de29e 100644 --- a/docs/zh/development/contrib.md +++ b/docs/zh/development/contrib.md @@ -31,3 +31,4 @@ | UnixODBC | [LGPL v2.1](https://github.com/ClickHouse-Extras/UnixODBC/tree/b0ad30f7f6289c12b76f04bfb9d466374bb32168) | | zlib-ng | [Zlib许可证](https://github.com/ClickHouse-Extras/zlib-ng/blob/develop/LICENSE.md) | | zstd | [BSD3-条款许可](https://github.com/facebook/zstd/blob/dev/LICENSE) | +| deflate | [MIT](https://github.com/intel/qpl/blob/develop/LICENSE) \ No newline at end of file diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index b82b13d9607..6d6eeac476b 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -114,6 +114,7 @@ if (BUILD_STANDALONE_KEEPER) ch_contrib::nuraft ch_contrib::lz4 ch_contrib::zstd + ch_contrib::qpl ch_contrib::cityhash common ch_contrib::double_conversion ch_contrib::dragonbox_to_chars From 4e8c6cb123e902e12387ea8ac17c6d427a858c2a Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 29 Apr 2022 13:41:09 -0400 Subject: [PATCH 012/659] modify uuid content --- contrib/qpl-cmake/uuid/uuid.h | 106 +--------------------------------- 1 file changed, 3 insertions(+), 103 deletions(-) diff --git a/contrib/qpl-cmake/uuid/uuid.h b/contrib/qpl-cmake/uuid/uuid.h index 874d65a196d..bf108ba0d29 100644 --- a/contrib/qpl-cmake/uuid/uuid.h +++ b/contrib/qpl-cmake/uuid/uuid.h @@ -1,104 +1,4 @@ -/* - * Public include file for the UUID library - * - * Copyright (C) 1996, 1997, 1998 Theodore Ts'o. - * - * %Begin-Header% - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions - * are met: - * 1. Redistributions of source code must retain the above copyright - * notice, and the entire permission notice in its entirety, - * including the disclaimer of warranties. - * 2. Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * 3. The name of the author may not be used to endorse or promote - * products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESS OR IMPLIED - * WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES - * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE, ALL OF - * WHICH ARE HEREBY DISCLAIMED. IN NO EVENT SHALL THE AUTHOR BE - * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT - * OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR - * BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF - * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE - * USE OF THIS SOFTWARE, EVEN IF NOT ADVISED OF THE POSSIBILITY OF SUCH - * DAMAGE. - * %End-Header% - */ - -#ifndef _UUID_UUID_H -#define _UUID_UUID_H - -#include -#ifndef _WIN32 -#include -#endif -#include - +#ifndef _QPL_UUID_UUID_H +#define _QPL_UUID_UUID_H typedef unsigned char uuid_t[16]; - -/* UUID Variant definitions */ -#define UUID_VARIANT_NCS 0 -#define UUID_VARIANT_DCE 1 -#define UUID_VARIANT_MICROSOFT 2 -#define UUID_VARIANT_OTHER 3 - -/* UUID Type definitions */ -#define UUID_TYPE_DCE_TIME 1 -#define UUID_TYPE_DCE_RANDOM 4 - -/* Allow UUID constants to be defined */ -#ifdef __GNUC__ -#define UUID_DEFINE(name,u0,u1,u2,u3,u4,u5,u6,u7,u8,u9,u10,u11,u12,u13,u14,u15) \ - static const uuid_t name __attribute__ ((unused)) = {u0,u1,u2,u3,u4,u5,u6,u7,u8,u9,u10,u11,u12,u13,u14,u15} -#else -#define UUID_DEFINE(name,u0,u1,u2,u3,u4,u5,u6,u7,u8,u9,u10,u11,u12,u13,u14,u15) \ - static const uuid_t name = {u0,u1,u2,u3,u4,u5,u6,u7,u8,u9,u10,u11,u12,u13,u14,u15} -#endif - -#ifdef __cplusplus -extern "C" { -#endif - -/* clear.c */ -void uuid_clear(uuid_t uu); - -/* compare.c */ -int uuid_compare(const uuid_t uu1, const uuid_t uu2); - -/* copy.c */ -void uuid_copy(uuid_t dst, const uuid_t src); - -/* gen_uuid.c */ -void uuid_generate(uuid_t out); -void uuid_generate_random(uuid_t out); -void uuid_generate_time(uuid_t out); -int uuid_generate_time_safe(uuid_t out); - -/* isnull.c */ -int uuid_is_null(const uuid_t uu); - -/* parse.c */ -int uuid_parse(const char *in, uuid_t uu); - -/* unparse.c */ -void uuid_unparse(const uuid_t uu, char *out); -void uuid_unparse_lower(const uuid_t uu, char *out); -void uuid_unparse_upper(const uuid_t uu, char *out); - -/* uuid_time.c */ -time_t uuid_time(const uuid_t uu, struct timeval *ret_tv); -int uuid_type(const uuid_t uu); -int uuid_variant(const uuid_t uu); - -#ifdef __cplusplus -} -#endif - -#endif /* _UUID_UUID_H */ +#endif /* _QPL_UUID_UUID_H */ From c4bce5234ecef134cd2d69fd201ce77e6998273d Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 5 May 2022 11:56:11 -0400 Subject: [PATCH 013/659] add nasm source into contrib for nasm compiler --- .gitmodules | 3 +++ contrib/qpl-cmake/CMakeLists.txt | 17 +++++++++++++++++ docker/test/fasttest/run.sh | 1 + 3 files changed, 21 insertions(+) diff --git a/.gitmodules b/.gitmodules index f423dc1d31a..13345c90481 100644 --- a/.gitmodules +++ b/.gitmodules @@ -265,3 +265,6 @@ [submodule "contrib/qpl"] path = contrib/qpl url = https://github.com/intel/qpl.git +[submodule "contrib/nasm"] + path = contrib/nasm + url = https://github.com/netwide-assembler/nasm.git diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 8af0230a4f8..b4c3176fe54 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -24,6 +24,23 @@ message(STATUS "Intel QPL version: ${QPL_VERSION}") include("${QPL_PROJECT_DIR}/cmake/CompileOptions.cmake") +# check nasm compiler +include(CheckLanguage) +check_language(ASM_NASM) +if(NOT CMAKE_ASM_NASM_COMPILER) + set (NASM_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/nasm") + set (NASM_BINARY_DIR "${ClickHouse_SOURCE_DIR}/build/contrib/nasm") + message(WARNING "Trying to build NASM from ${NASM_SOURCE_DIR} because NASM could not be found on local machine") + + execute_process(COMMAND + bash -c "./autogen.sh && ./configure --prefix=${NASM_BINARY_DIR} && make install" + WORKING_DIRECTORY ${NASM_SOURCE_DIR} + RESULT_VARIABLE variable_RESULT) + message(STATUS "NASM build result:${variable_RESULT}") + + set(CMAKE_ASM_NASM_COMPILER ${NASM_BINARY_DIR}/bin/nasm) +endif() + # [SUBDIR]isal enable_language(ASM_NASM) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index df227dfbcad..4ade347997d 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -177,6 +177,7 @@ function clone_submodules contrib/jemalloc contrib/replxx contrib/qpl + contrib/nasm ) git submodule sync From 05fc3914c1cb1b802bbadfd381429296304dec15 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sun, 8 May 2022 10:40:17 -0400 Subject: [PATCH 014/659] add folder for contrib/nasm --- contrib/nasm | 1 + 1 file changed, 1 insertion(+) create mode 160000 contrib/nasm diff --git a/contrib/nasm b/contrib/nasm new file mode 160000 index 00000000000..3f9fc2a3a71 --- /dev/null +++ b/contrib/nasm @@ -0,0 +1 @@ +Subproject commit 3f9fc2a3a7134936cbbae5780beb4319694f702a From 930973501653e3a2821f3997d22ef44f50e0135b Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 10 May 2022 16:46:50 -0400 Subject: [PATCH 015/659] remove unused job release call --- src/Compression/CompressionCodecDeflate.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index a6b4daf5d3a..f55ca89d8d5 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -156,7 +156,6 @@ uint32_t CompressionCodecDeflate::doCompressDataSW(const char * source, uint32_t qpl_job * jobPtr = DeflateJobSWPool::instance().acquireJob(&jobID); if (jobPtr == nullptr) { - DeflateJobSWPool::instance().releaseJob(jobID); LOG_WARNING(log, "doCompressDataSW acquireJob fail! switch to SW native compress..."); return doCompressDataSWNative(source, source_size, dest); } @@ -187,7 +186,6 @@ uint32_t CompressionCodecDeflate::doCompressData(const char * source, uint32_t s qpl_job * jobPtr = DeflateJobHWPool::instance().acquireJob(&jobID); if (jobPtr == nullptr) { - DeflateJobHWPool::instance().releaseJob(jobID); LOG_WARNING(log, "doCompressData HW acquireJob fail! switch to SW compress..."); return doCompressDataSW(source, source_size, dest); } @@ -223,7 +221,6 @@ UInt32 CompressionCodecDeflate::doCompressDataReq(const char * source, UInt32 so qpl_job * jobPtr = DeflateJobHWPool::instance().acquireJob(&jobID); if (jobPtr == nullptr) { - DeflateJobHWPool::instance().releaseJob(jobID); LOG_WARNING(log, "doCompressDataReq HW acquireJob fail! switch to SW compress..."); return doCompressDataSW(source, source_size, dest); } @@ -269,7 +266,6 @@ void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t sou qpl_job * jobPtr = DeflateJobHWPool::instance().acquireJob(&jobID); if (jobPtr == nullptr) { - DeflateJobHWPool::instance().releaseJob(jobID); LOG_WARNING(log, "doDecompressData HW acquireJob fail! switch to SW decompress"); return doDecompressDataSW(source, source_size, dest, uncompressed_size); } @@ -347,7 +343,6 @@ void CompressionCodecDeflate::doDecompressDataSW(const char * source, uint32_t s qpl_job * jobPtr = DeflateJobSWPool::instance().acquireJob(&jobID); if (jobPtr == nullptr) { - DeflateJobSWPool::instance().releaseJob(jobID); LOG_WARNING(log, "doDecompressDataSW acquireJob fail! switch to SW native decompress..."); return doDecompressDataSWNative(source, source_size, dest, uncompressed_size); } @@ -377,7 +372,6 @@ void CompressionCodecDeflate::doDecompressDataReq(const char * source, uint32_t qpl_job * jobPtr = DeflateJobHWPool::instance().acquireJob(&jobID); if (jobPtr == nullptr) { - DeflateJobHWPool::instance().releaseJob(jobID); LOG_WARNING(log, "doDecompressDataReq acquireJob fail! switch to SW decompress"); doDecompressDataSW(source, source_size, dest, uncompressed_size); return; From f413b49364ba3bb12481bb89ba72a195bbda8881 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 16 May 2022 14:59:16 -0400 Subject: [PATCH 016/659] fixed logger header file path --- src/Compression/CompressionCodecDeflate.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index f55ca89d8d5..101d05ce150 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include namespace DB From 4857d6e2c30de82e94a391b3b2112c6e15b4ad77 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 30 May 2022 17:45:26 -0400 Subject: [PATCH 017/659] add nasm into docker configuration --- contrib/qpl-cmake/CMakeLists.txt | 12 +----------- docker/test/base/Dockerfile | 1 + 2 files changed, 2 insertions(+), 11 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index b4c3176fe54..c397c5734a2 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -28,17 +28,7 @@ include("${QPL_PROJECT_DIR}/cmake/CompileOptions.cmake") include(CheckLanguage) check_language(ASM_NASM) if(NOT CMAKE_ASM_NASM_COMPILER) - set (NASM_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/nasm") - set (NASM_BINARY_DIR "${ClickHouse_SOURCE_DIR}/build/contrib/nasm") - message(WARNING "Trying to build NASM from ${NASM_SOURCE_DIR} because NASM could not be found on local machine") - - execute_process(COMMAND - bash -c "./autogen.sh && ./configure --prefix=${NASM_BINARY_DIR} && make install" - WORKING_DIRECTORY ${NASM_SOURCE_DIR} - RESULT_VARIABLE variable_RESULT) - message(STATUS "NASM build result:${variable_RESULT}") - - set(CMAKE_ASM_NASM_COMPILER ${NASM_BINARY_DIR}/bin/nasm) + message(FATAL_ERROR "Please install NASM from 'https://github.com/netwide-assembler/nasm' because NASM compiler could not be found on local machine!") endif() # [SUBDIR]isal diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index 6beab2e5bb7..c89ca065141 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -55,6 +55,7 @@ RUN apt-get update \ pkg-config \ tzdata \ pv \ + nasm \ --yes --no-install-recommends # Sanitizer options for services (clickhouse-server) From 822cef10b37510296a4dde5f69b0c60a19a0fe10 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 30 May 2022 21:16:06 -0400 Subject: [PATCH 018/659] add nasm into more docker configuration --- docker/packager/binary/Dockerfile | 1 + docker/test/fasttest/Dockerfile | 1 + 2 files changed, 2 insertions(+) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index d4e824838c2..acfeecd5a7e 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -51,6 +51,7 @@ RUN apt-get update \ rename \ software-properties-common \ tzdata \ + nasm \ --yes --no-install-recommends \ && apt-get clean diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 46b74d89e13..591d90c6610 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -71,6 +71,7 @@ RUN apt-get update \ software-properties-common \ tzdata \ unixodbc \ + nasm \ --yes --no-install-recommends RUN pip3 install numpy scipy pandas Jinja2 From 330a46c4184c0ca42e136ff624e0bf5e23625e1a Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 31 May 2022 09:55:06 -0400 Subject: [PATCH 019/659] Remove eigen library according to master commit: 7d0ed7e51a364e6932ad93ab2d0458a3818203a4 --- contrib/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 657690ef7b5..3b79aadfbb9 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -154,7 +154,6 @@ endif() add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) add_contrib (qpl-cmake qpl) -add_contrib (eigen-cmake eigen) # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear From 53be92453741f94a3b344f879763a20057dee81b Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 31 May 2022 15:40:31 -0400 Subject: [PATCH 020/659] fixed invalid case style --- src/Compression/CompressedReadBufferBase.cpp | 2 +- src/Compression/CompressedReadBufferBase.h | 2 +- .../CompressedReadBufferFromFile.cpp | 6 +- src/Compression/CompressionCodecDeflate.cpp | 232 +++++++++--------- src/Compression/CompressionCodecDeflate.h | 4 +- src/Compression/ICompressionCodec.cpp | 2 +- src/Compression/ICompressionCodec.h | 2 +- 7 files changed, 125 insertions(+), 125 deletions(-) diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index f169aad6da4..c05610d09c7 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -318,7 +318,7 @@ void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_d codec->decompress(compressed_buffer, size_compressed_without_checksum, to.begin(), req_type); } -void CompressedReadBufferBase::decompressFlush(void) +void CompressedReadBufferBase::decompressFlush() { if (codec) { diff --git a/src/Compression/CompressedReadBufferBase.h b/src/Compression/CompressedReadBufferBase.h index 841c4a0bdd3..b86ba97555b 100644 --- a/src/Compression/CompressedReadBufferBase.h +++ b/src/Compression/CompressedReadBufferBase.h @@ -46,7 +46,7 @@ protected: /// It is more efficient for compression codec NONE but not suitable if you want to decompress into specific location. void decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum, UInt8 req_type=0); /// Flush all asynchronous decompress request - void decompressFlush(void); + void decompressFlush(); public: /// 'compressed_in' could be initialized lazily, but before first call of 'readCompressedData'. explicit CompressedReadBufferBase(ReadBuffer * in = nullptr, bool allow_different_codecs_ = false); diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index d063a4f0c66..997ee7ec7e4 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -92,7 +92,7 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) { size_t bytes_read = 0; UInt8 req_type = 0; - bool readTail = false; + bool read_tail = false; /// If there are unread bytes in the buffer, then we copy needed to `to`. if (pos < working_buffer.end()) @@ -162,14 +162,14 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) working_buffer = Buffer(memory.data(), &memory[size_decompressed]); decompress(working_buffer, size_decompressed, size_compressed_without_checksum, 1); - readTail = true; + read_tail = true; break; } } decompressFlush(); - if (readTail) + if (read_tail) { /// Manually take nextimpl_working_buffer_offset into account, because we don't use /// nextImpl in this method. diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 101d05ce150..985e6f71c33 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -62,7 +62,7 @@ CompressionCodecDeflate::CompressionCodecDeflate() CompressionCodecDeflate::~CompressionCodecDeflate() { - if (jobDecompAsyncMap.size() > 0) + if (!jobDecompAsyncMap.empty()) { LOG_ERROR(log, "Exception -> find un-released job when CompressionCodecDeflate destroy"); //doDecompressDataFlush(); @@ -72,7 +72,7 @@ CompressionCodecDeflate::~CompressionCodecDeflate() } jobDecompAsyncMap.clear(); } - if (jobCompAsyncList.size() > 0) + if (!jobCompAsyncList.empty()) { for (auto id : jobCompAsyncList) { @@ -114,47 +114,47 @@ uint32_t CompressionCodecDeflate::doCompressDataSWNative(const char * source, ui { throw Exception("doCompressDataSWNative cannot compress: qpl_get_job_size fail", ErrorCodes::CANNOT_COMPRESS); } - qpl_job * jobPtr = reinterpret_cast(new uint8_t[size]); + qpl_job * job_ptr = reinterpret_cast(new uint8_t[size]); - status = qpl_init_job(DeflateJobSWPool::SW_PATH, jobPtr); + status = qpl_init_job(DeflateJobSWPool::SW_PATH, job_ptr); if (status != QPL_STS_OK) { throw Exception("doCompressDataSWNative cannot compress: qpl_init_job fail", ErrorCodes::CANNOT_COMPRESS); } // Performing a compression operation - jobPtr->op = qpl_op_compress; - jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); - jobPtr->next_out_ptr = reinterpret_cast(dest); - jobPtr->available_in = source_size; - jobPtr->available_out = getMaxCompressedDataSize(source_size); - jobPtr->level = qpl_high_level; - jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; + job_ptr->op = qpl_op_compress; + job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); + job_ptr->next_out_ptr = reinterpret_cast(dest); + job_ptr->available_in = source_size; + job_ptr->available_out = getMaxCompressedDataSize(source_size); + job_ptr->level = qpl_high_level; + job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; // Compression - status = qpl_execute_job(jobPtr); + status = qpl_execute_job(job_ptr); if (status != QPL_STS_OK) { throw Exception("doCompressDataSWNative cannot compress: qpl_execute_job fail", ErrorCodes::CANNOT_COMPRESS); } - const uint32_t compressed_size = jobPtr->total_out; + const uint32_t compressed_size = job_ptr->total_out; // Freeing resources - status = qpl_fini_job(jobPtr); + status = qpl_fini_job(job_ptr); if (status != QPL_STS_OK) { throw Exception("doCompressDataSWNative cannot compress: qpl_fini_job fail", ErrorCodes::CANNOT_COMPRESS); } - delete[] jobPtr; + delete[] job_ptr; return compressed_size; } uint32_t CompressionCodecDeflate::doCompressDataSW(const char * source, uint32_t source_size, char * dest) const { - uint32_t jobID = 0; - qpl_job * jobPtr = DeflateJobSWPool::instance().acquireJob(&jobID); - if (jobPtr == nullptr) + uint32_t job_id = 0; + qpl_job * job_ptr = DeflateJobSWPool::instance().acquireJob(&job_id); + if (job_ptr == nullptr) { LOG_WARNING(log, "doCompressDataSW acquireJob fail! switch to SW native compress..."); return doCompressDataSWNative(source, source_size, dest); @@ -162,29 +162,29 @@ uint32_t CompressionCodecDeflate::doCompressDataSW(const char * source, uint32_t qpl_status status; uint32_t compressed_size = 0; - jobPtr->op = qpl_op_compress; - jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); - jobPtr->next_out_ptr = reinterpret_cast(dest); - jobPtr->available_in = source_size; - jobPtr->available_out = getMaxCompressedDataSize(source_size); - jobPtr->level = qpl_high_level; - jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; + job_ptr->op = qpl_op_compress; + job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); + job_ptr->next_out_ptr = reinterpret_cast(dest); + job_ptr->available_in = source_size; + job_ptr->available_out = getMaxCompressedDataSize(source_size); + job_ptr->level = qpl_high_level; + job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; // Compression - status = qpl_execute_job(jobPtr); + status = qpl_execute_job(job_ptr); if (QPL_STS_OK != status) { throw Exception("doCompressDataSW Cannot compress", ErrorCodes::CANNOT_COMPRESS); } - compressed_size = jobPtr->total_out; - DeflateJobSWPool::instance().releaseJob(jobID); + compressed_size = job_ptr->total_out; + DeflateJobSWPool::instance().releaseJob(job_id); return compressed_size; } uint32_t CompressionCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest) const { - uint32_t jobID = 0; - qpl_job * jobPtr = DeflateJobHWPool::instance().acquireJob(&jobID); - if (jobPtr == nullptr) + uint32_t job_id = 0; + qpl_job * job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); + if (job_ptr == nullptr) { LOG_WARNING(log, "doCompressData HW acquireJob fail! switch to SW compress..."); return doCompressDataSW(source, source_size, dest); @@ -192,79 +192,79 @@ uint32_t CompressionCodecDeflate::doCompressData(const char * source, uint32_t s qpl_status status; uint32_t compressed_size = 0; - jobPtr->op = qpl_op_compress; - jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); - jobPtr->next_out_ptr = reinterpret_cast(dest); - jobPtr->available_in = source_size; - jobPtr->level = qpl_default_level; - jobPtr->available_out = getMaxCompressedDataSize(source_size); - jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; + job_ptr->op = qpl_op_compress; + job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); + job_ptr->next_out_ptr = reinterpret_cast(dest); + job_ptr->available_in = source_size; + job_ptr->level = qpl_default_level; + job_ptr->available_out = getMaxCompressedDataSize(source_size); + job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; // Compression - status = qpl_execute_job(jobPtr); + status = qpl_execute_job(job_ptr); if (QPL_STS_OK == status) { - compressed_size = jobPtr->total_out; + compressed_size = job_ptr->total_out; } else { LOG_WARNING(log, "doCompressData HW fail! switch to SW compress ->status: '{}' ", static_cast(status)); compressed_size = doCompressDataSW(source, source_size, dest); } - DeflateJobHWPool::instance().releaseJob(jobID); + DeflateJobHWPool::instance().releaseJob(job_id); return compressed_size; } UInt32 CompressionCodecDeflate::doCompressDataReq(const char * source, UInt32 source_size, char * dest, UInt32 & req_id) { - uint32_t jobID = 0; + uint32_t job_id = 0; req_id = 0; - qpl_job * jobPtr = DeflateJobHWPool::instance().acquireJob(&jobID); - if (jobPtr == nullptr) + qpl_job * job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); + if (job_ptr == nullptr) { LOG_WARNING(log, "doCompressDataReq HW acquireJob fail! switch to SW compress..."); return doCompressDataSW(source, source_size, dest); } qpl_status status; - jobPtr->op = qpl_op_compress; - jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); - jobPtr->next_out_ptr = reinterpret_cast(dest); - jobPtr->available_in = source_size; - jobPtr->level = qpl_default_level; - jobPtr->available_out = getMaxCompressedDataSize(source_size); - jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; + job_ptr->op = qpl_op_compress; + job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); + job_ptr->next_out_ptr = reinterpret_cast(dest); + job_ptr->available_in = source_size; + job_ptr->level = qpl_default_level; + job_ptr->available_out = getMaxCompressedDataSize(source_size); + job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; // Compression - status = qpl_submit_job(jobPtr); + status = qpl_submit_job(job_ptr); if (QPL_STS_OK != status) { LOG_WARNING(log, "doCompressDataReq HW fail! switch to SW compress ->status: '{}' ", static_cast(status)); - DeflateJobHWPool::instance().releaseJob(jobID); + DeflateJobHWPool::instance().releaseJob(job_id); return doCompressDataSW(source, source_size, dest); } - //LOG_WARNING(log, "doCompressDataReq ->jobID:{}, source_size:{}",jobID, source_size); - jobCompAsyncList.push_back(jobID); - req_id = jobID; + //LOG_WARNING(log, "doCompressDataReq ->job_id:{}, source_size:{}",job_id, source_size); + jobCompAsyncList.push_back(job_id); + req_id = job_id; return 0; } uint32_t CompressionCodecDeflate::doCompressDataFlush(uint32_t req_id) { uint32_t compressed_size = 0; - qpl_job * jobPtr = DeflateJobHWPool::instance().getJobPtr(req_id); - while (QPL_STS_BEING_PROCESSED == qpl_check_job(jobPtr)) + qpl_job * job_ptr = DeflateJobHWPool::instance().getJobPtr(req_id); + while (QPL_STS_BEING_PROCESSED == qpl_check_job(job_ptr)) { _tpause(1, __rdtsc() + 1000); } - compressed_size = jobPtr->total_out; + compressed_size = job_ptr->total_out; DeflateJobHWPool::instance().releaseJob(req_id); return compressed_size; } void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const { - uint32_t jobID = 0; - qpl_job * jobPtr = DeflateJobHWPool::instance().acquireJob(&jobID); - if (jobPtr == nullptr) + uint32_t job_id = 0; + qpl_job * job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); + if (job_ptr == nullptr) { LOG_WARNING(log, "doDecompressData HW acquireJob fail! switch to SW decompress"); return doDecompressDataSW(source, source_size, dest, uncompressed_size); @@ -272,15 +272,15 @@ void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t sou qpl_status status; // Performing a decompression operation - jobPtr->op = qpl_op_decompress; - jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); - jobPtr->next_out_ptr = reinterpret_cast(dest); - jobPtr->available_in = source_size; - jobPtr->available_out = uncompressed_size; - jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; + job_ptr->op = qpl_op_decompress; + job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); + job_ptr->next_out_ptr = reinterpret_cast(dest); + job_ptr->available_in = source_size; + job_ptr->available_out = uncompressed_size; + job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; // Decompression - status = qpl_execute_job(jobPtr); + status = qpl_execute_job(job_ptr); if (status != QPL_STS_OK) { LOG_WARNING( @@ -291,11 +291,11 @@ void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t sou uncompressed_size); doDecompressDataSW(source, source_size, dest, uncompressed_size); } - DeflateJobHWPool::instance().releaseJob(jobID); + DeflateJobHWPool::instance().releaseJob(job_id); } void CompressionCodecDeflate::doDecompressDataSWNative( - const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const + const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) { qpl_status status; uint32_t size = 0; @@ -306,42 +306,42 @@ void CompressionCodecDeflate::doDecompressDataSWNative( { throw Exception("doDecompressDataSWNative cannot decompress: qpl_get_job_size fail", ErrorCodes::CANNOT_DECOMPRESS); } - qpl_job * jobPtr = reinterpret_cast(new uint8_t[size]); + qpl_job * job_ptr = reinterpret_cast(new uint8_t[size]); - status = qpl_init_job(DeflateJobSWPool::SW_PATH, jobPtr); + status = qpl_init_job(DeflateJobSWPool::SW_PATH, job_ptr); if (status != QPL_STS_OK) { throw Exception("doDecompressDataSWNative cannot decompress: qpl_init_job fail", ErrorCodes::CANNOT_DECOMPRESS); } // Performing a decompression operation - jobPtr->op = qpl_op_decompress; - jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); - jobPtr->next_out_ptr = reinterpret_cast(dest); - jobPtr->available_in = source_size; - jobPtr->available_out = uncompressed_size; - jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; + job_ptr->op = qpl_op_decompress; + job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); + job_ptr->next_out_ptr = reinterpret_cast(dest); + job_ptr->available_in = source_size; + job_ptr->available_out = uncompressed_size; + job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; // Decompression - status = qpl_execute_job(jobPtr); + status = qpl_execute_job(job_ptr); if (status != QPL_STS_OK) { throw Exception("doDecompressDataSWNative cannot decompress: qpl_execute_job fail", ErrorCodes::CANNOT_DECOMPRESS); } // Freeing resources - status = qpl_fini_job(jobPtr); + status = qpl_fini_job(job_ptr); if (status != QPL_STS_OK) { throw Exception("doDecompressDataSWNative cannot decompress: qpl_fini_job fail", ErrorCodes::CANNOT_DECOMPRESS); } - delete[] jobPtr; + delete[] job_ptr; } void CompressionCodecDeflate::doDecompressDataSW(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const { - uint32_t jobID = 0; - qpl_job * jobPtr = DeflateJobSWPool::instance().acquireJob(&jobID); - if (jobPtr == nullptr) + uint32_t job_id = 0; + qpl_job * job_ptr = DeflateJobSWPool::instance().acquireJob(&job_id); + if (job_ptr == nullptr) { LOG_WARNING(log, "doDecompressDataSW acquireJob fail! switch to SW native decompress..."); return doDecompressDataSWNative(source, source_size, dest, uncompressed_size); @@ -349,28 +349,28 @@ void CompressionCodecDeflate::doDecompressDataSW(const char * source, uint32_t s qpl_status status; // Performing a decompression operation - jobPtr->op = qpl_op_decompress; - jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); - jobPtr->next_out_ptr = reinterpret_cast(dest); - jobPtr->available_in = source_size; - jobPtr->available_out = uncompressed_size; - jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; + job_ptr->op = qpl_op_decompress; + job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); + job_ptr->next_out_ptr = reinterpret_cast(dest); + job_ptr->available_in = source_size; + job_ptr->available_out = uncompressed_size; + job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; // Decompression - status = qpl_execute_job(jobPtr); + status = qpl_execute_job(job_ptr); if (QPL_STS_OK != status) { throw Exception("doDecompressDataSW cannot decompress", ErrorCodes::CANNOT_DECOMPRESS); } - DeflateJobSWPool::instance().releaseJob(jobID); + DeflateJobSWPool::instance().releaseJob(job_id); } void CompressionCodecDeflate::doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) { - uint32_t jobID = 0; - qpl_job * jobPtr = DeflateJobHWPool::instance().acquireJob(&jobID); - if (jobPtr == nullptr) + uint32_t job_id = 0; + qpl_job * job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); + if (job_ptr == nullptr) { LOG_WARNING(log, "doDecompressDataReq acquireJob fail! switch to SW decompress"); doDecompressDataSW(source, source_size, dest, uncompressed_size); @@ -379,51 +379,51 @@ void CompressionCodecDeflate::doDecompressDataReq(const char * source, uint32_t qpl_status status; // Performing a decompression operation - jobPtr->op = qpl_op_decompress; - jobPtr->next_in_ptr = reinterpret_cast(const_cast(source)); - jobPtr->next_out_ptr = reinterpret_cast(dest); - jobPtr->available_in = source_size; - jobPtr->available_out = uncompressed_size; - jobPtr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; + job_ptr->op = qpl_op_decompress; + job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); + job_ptr->next_out_ptr = reinterpret_cast(dest); + job_ptr->available_in = source_size; + job_ptr->available_out = uncompressed_size; + job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; // Decompression - status = qpl_submit_job(jobPtr); + status = qpl_submit_job(job_ptr); if (QPL_STS_OK == status) { - jobDecompAsyncMap.insert(std::make_pair(jobID, jobPtr)); + jobDecompAsyncMap.insert(std::make_pair(job_id, job_ptr)); } else { - DeflateJobHWPool::instance().releaseJob(jobID); + DeflateJobHWPool::instance().releaseJob(job_id); LOG_WARNING(log, "doDecompressDataReq HW fail! switch to SW decompress... ->status: '{}' ", static_cast(status)); doDecompressDataSW(source, source_size, dest, uncompressed_size); } } -void CompressionCodecDeflate::doDecompressDataFlush(void) +void CompressionCodecDeflate::doDecompressDataFlush() { - uint32_t jobID = 0; - qpl_job * jobPtr = nullptr; + uint32_t job_id = 0; + qpl_job * job_ptr = nullptr; std::map::iterator it; - uint32_t nJobsProcessing = jobDecompAsyncMap.size(); + uint32_t n_jobs_processing = jobDecompAsyncMap.size(); it = jobDecompAsyncMap.begin(); - while (nJobsProcessing) + while (n_jobs_processing) { - jobID = it->first; - jobPtr = it->second; + job_id = it->first; + job_ptr = it->second; - if (QPL_STS_BEING_PROCESSED == qpl_check_job(jobPtr)) + if (QPL_STS_BEING_PROCESSED == qpl_check_job(job_ptr)) { it++; } else { - DeflateJobHWPool::instance().releaseJob(jobID); + DeflateJobHWPool::instance().releaseJob(job_id); it = jobDecompAsyncMap.erase(it); - nJobsProcessing--; + n_jobs_processing--; } if (it == jobDecompAsyncMap.end()) { diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index ce8d3c1b8db..588709e7df4 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -319,8 +319,8 @@ protected: void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; void doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) override; void doDecompressDataSW(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; - void doDecompressDataSWNative(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const; - void doDecompressDataFlush(void) override; + static void doDecompressDataSWNative(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); + void doDecompressDataFlush() override; private: uint32_t getMaxCompressedDataSize(uint32_t uncompressed_size) const override; diff --git a/src/Compression/ICompressionCodec.cpp b/src/Compression/ICompressionCodec.cpp index c99358bd6cd..933eceb7859 100644 --- a/src/Compression/ICompressionCodec.cpp +++ b/src/Compression/ICompressionCodec.cpp @@ -148,7 +148,7 @@ UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, ch return decompressed_size; } -void ICompressionCodec::decompressFlush(void) +void ICompressionCodec::decompressFlush() { doDecompressDataFlush(); } diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 594e24364ff..134862e37c8 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -139,7 +139,7 @@ protected: } /// Flush asynchronous request for decompression - virtual void doDecompressDataFlush(void) + virtual void doDecompressDataFlush() { } /// Construct and set codec description from codec name and arguments. Must be called in codec constructor. From 1d3ae2d88c2edb19df2e77bc87bba80f2d2ce7eb Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 31 May 2022 17:14:08 -0400 Subject: [PATCH 021/659] fixed unable to find library -ldl in docker build --- docker/packager/binary/Dockerfile | 1 + docker/test/base/Dockerfile | 1 + docker/test/fasttest/Dockerfile | 1 + 3 files changed, 3 insertions(+) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index acfeecd5a7e..5f45596c62f 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -52,6 +52,7 @@ RUN apt-get update \ software-properties-common \ tzdata \ nasm \ + libc6-dev \ --yes --no-install-recommends \ && apt-get clean diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index c89ca065141..af28ccb6ef9 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -56,6 +56,7 @@ RUN apt-get update \ tzdata \ pv \ nasm \ + libc6-dev \ --yes --no-install-recommends # Sanitizer options for services (clickhouse-server) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 591d90c6610..fe2f4a6f38d 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -72,6 +72,7 @@ RUN apt-get update \ tzdata \ unixodbc \ nasm \ + libc6-dev \ --yes --no-install-recommends RUN pip3 install numpy scipy pandas Jinja2 From e1426c7e9d23e79743cc1817ecded401f9a7bffa Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 1 Jun 2022 11:47:43 -0400 Subject: [PATCH 022/659] add build option for iaa compression --- CMakeLists.txt | 6 ++++++ contrib/CMakeLists.txt | 3 ++- contrib/qpl-cmake/CMakeLists.txt | 2 +- src/CMakeLists.txt | 27 +++++++++++++++++++++----- src/Compression/CompressionFactory.cpp | 2 ++ 5 files changed, 33 insertions(+), 7 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index abe263834ed..9c1f0957424 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -122,6 +122,12 @@ if (ENABLE_FUZZING) set (ENABLE_PROTOBUF 1) endif() +option (ENABLE_QPL "Enable Intel® Query Processing Library" ON) +if (ENABLE_QPL) + set (ENABLE_QPL_COMPRESSION 1) + set (ENABLE_QPL_ANALYTIC 1) +endif() + # Global libraries # See: # - default_libs.cmake diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 3b79aadfbb9..ccda0e3d00f 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -153,8 +153,9 @@ endif() add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) +if (ENABLE_QPL) add_contrib (qpl-cmake qpl) - +endif() # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear # in "contrib/..." as originally planned, so we workaround this by fixing FOLDER properties of all targets manually, diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index c397c5734a2..afaf62e7698 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -28,7 +28,7 @@ include("${QPL_PROJECT_DIR}/cmake/CompileOptions.cmake") include(CheckLanguage) check_language(ASM_NASM) if(NOT CMAKE_ASM_NASM_COMPILER) - message(FATAL_ERROR "Please install NASM from 'https://github.com/netwide-assembler/nasm' because NASM compiler could not be found on local machine!") + message(FATAL_ERROR "Please install NASM from 'https://www.nasm.us/' because NASM compiler can not be found!") endif() # [SUBDIR]isal diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 29f9abf40af..01219b35a7c 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -228,11 +228,28 @@ macro(add_object_library name common_path) endif () endmacro() +macro(remove_item_object_library name common_file) + if (USE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) + list (REMOVE_ITEM dbms_sources ${common_file}) + else () + list (REMOVE_ITEM ${${name}_sources} ${common_file}) + endif () +endmacro() + add_object_library(clickhouse_access Access) add_object_library(clickhouse_backups Backups) add_object_library(clickhouse_core Core) add_object_library(clickhouse_core_mysql Core/MySQL) add_object_library(clickhouse_compression Compression) +if (NOT ENABLE_QPL_COMPRESSION) +add_definitions(-DENABLE_QPL_COMPRESSION=0) +remove_item_object_library(clickhouse_compression Compression/CompressionCodecDeflate.cpp) +else () +add_definitions(-DENABLE_QPL_COMPRESSION=1) +set_source_files_properties( + Compression/CompressionCodecDeflate.cpp + PROPERTIES COMPILE_FLAGS "-mwaitpkg") +endif () add_object_library(clickhouse_querypipeline QueryPipeline) add_object_library(clickhouse_datatypes DataTypes) add_object_library(clickhouse_datatypes_serializations DataTypes/Serializations) @@ -518,11 +535,11 @@ if (TARGET ch_contrib::datasketches) endif () target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::lz4) -target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::qpl) -target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/qpl/include) -set_source_files_properties( - Compression/CompressionCodecDeflate.cpp - PROPERTIES COMPILE_FLAGS "-mwaitpkg") + +if (ENABLE_QPL_COMPRESSION) +dbms_target_link_libraries(PUBLIC ch_contrib::qpl) +dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/qpl/include) +endif () dbms_target_link_libraries(PRIVATE _boost_context) diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index a0d79b3dc9c..40054ef9393 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -187,7 +187,9 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecZSTD(*this); registerCodecLZ4HC(*this); registerCodecMultiple(*this); +#if ENABLE_QPL_COMPRESSION registerCodecDeflate(*this); +#endif #ifndef KEEPER_STANDALONE_BUILD registerCodecDelta(*this); registerCodecT64(*this); From 041a8e42caec9941666c05a636125d3a9c8a40a0 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 2 Jun 2022 10:34:11 -0400 Subject: [PATCH 023/659] revise code style --- contrib/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index ccda0e3d00f..16c5b9811c1 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -154,7 +154,7 @@ endif() add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) if (ENABLE_QPL) -add_contrib (qpl-cmake qpl) + add_contrib (qpl-cmake qpl) endif() # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear From 2170a99484da88285d2c8513dd0310b798a6357a Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 3 Jun 2022 21:06:55 -0400 Subject: [PATCH 024/659] revise ENABLE_QPL_COMPRESSION build option --- contrib/qpl-cmake/CMakeLists.txt | 1 + src/CMakeLists.txt | 2 -- src/Compression/CompressedReadBufferBase.cpp | 2 +- src/Compression/CompressedReadBufferBase.h | 2 +- src/Compression/CompressionFactory.cpp | 2 +- 5 files changed, 4 insertions(+), 5 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index afaf62e7698..73aba26a607 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -346,6 +346,7 @@ target_compile_options(_qpl target_compile_definitions(_qpl PUBLIC -DQPL_LIB PUBLIC -DQPL_BADARG_CHECK + PUBLIC -DENABLE_QPL_COMPRESSION PUBLIC $<$:_ENABLE_EXTENDED_ALIGNED_STORAGE>) target_link_libraries(_qpl diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 01219b35a7c..0489cc7e527 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -242,10 +242,8 @@ add_object_library(clickhouse_core Core) add_object_library(clickhouse_core_mysql Core/MySQL) add_object_library(clickhouse_compression Compression) if (NOT ENABLE_QPL_COMPRESSION) -add_definitions(-DENABLE_QPL_COMPRESSION=0) remove_item_object_library(clickhouse_compression Compression/CompressionCodecDeflate.cpp) else () -add_definitions(-DENABLE_QPL_COMPRESSION=1) set_source_files_properties( Compression/CompressionCodecDeflate.cpp PROPERTIES COMPILE_FLAGS "-mwaitpkg") diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index c05610d09c7..da2edccb776 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -318,7 +318,7 @@ void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_d codec->decompress(compressed_buffer, size_compressed_without_checksum, to.begin(), req_type); } -void CompressedReadBufferBase::decompressFlush() +void CompressedReadBufferBase::decompressFlush() const { if (codec) { diff --git a/src/Compression/CompressedReadBufferBase.h b/src/Compression/CompressedReadBufferBase.h index b86ba97555b..3b822e540b9 100644 --- a/src/Compression/CompressedReadBufferBase.h +++ b/src/Compression/CompressedReadBufferBase.h @@ -46,7 +46,7 @@ protected: /// It is more efficient for compression codec NONE but not suitable if you want to decompress into specific location. void decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum, UInt8 req_type=0); /// Flush all asynchronous decompress request - void decompressFlush(); + void decompressFlush() const; public: /// 'compressed_in' could be initialized lazily, but before first call of 'readCompressedData'. explicit CompressedReadBufferBase(ReadBuffer * in = nullptr, bool allow_different_codecs_ = false); diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 40054ef9393..05595418115 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -187,7 +187,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecZSTD(*this); registerCodecLZ4HC(*this); registerCodecMultiple(*this); -#if ENABLE_QPL_COMPRESSION +#ifdef ENABLE_QPL_COMPRESSION registerCodecDeflate(*this); #endif #ifndef KEEPER_STANDALONE_BUILD From 0a063820c7980005920bcbf713d1fe170d5ffaf5 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 4 Jun 2022 01:35:41 -0400 Subject: [PATCH 025/659] WITH FILL of date/time type must be equal to type of ORDER BY column --- src/Interpreters/InterpreterSelectQuery.cpp | 38 ++++++++++++++------- 1 file changed, 25 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index fbafb98e0d8..5d9baadbfdf 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -802,10 +803,14 @@ Block InterpreterSelectQuery::getSampleBlockImpl() return analysis_result.final_projection->getResultColumns(); } -static Field getWithFillFieldValue(const ASTPtr & node, ContextPtr context) +static Field getWithFillFieldValue(DataTypePtr col_type, const ASTPtr & node, ContextPtr context) { auto [field, type] = evaluateConstantExpression(node, context); + WhichDataType which(col_type); + if ((which.isDateOrDate32() || which.isDateTime() || which.isDateTime64()) && !col_type->equals(*type)) + throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be same as column " + col_type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + if (!isColumnedAsNumber(type)) throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); @@ -825,14 +830,14 @@ static std::pair> getWithFillStep(const ASTPt throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); } -static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, ContextPtr context) +static FillColumnDescription getWithFillDescription(DataTypePtr type, const ASTOrderByElement & order_by_elem, ContextPtr context) { FillColumnDescription descr; if (order_by_elem.fill_from) - descr.fill_from = getWithFillFieldValue(order_by_elem.fill_from, context); + descr.fill_from = getWithFillFieldValue(type, order_by_elem.fill_from, context); if (order_by_elem.fill_to) - descr.fill_to = getWithFillFieldValue(order_by_elem.fill_to, context); + descr.fill_to = getWithFillFieldValue(type, order_by_elem.fill_to, context); if (order_by_elem.fill_step) std::tie(descr.fill_step, descr.step_kind) = getWithFillStep(order_by_elem.fill_step, context); @@ -872,7 +877,7 @@ static FillColumnDescription getWithFillDescription(const ASTOrderByElement & or return descr; } -static SortDescription getSortDescription(const ASTSelectQuery & query, ContextPtr context) +static SortDescription getSortDescription(const ASTSelectQuery & query, const Block & result_block, const Aliases & aliases, ContextPtr context) { SortDescription order_descr; order_descr.reserve(query.orderBy()->children.size()); @@ -886,7 +891,14 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP collator = std::make_shared(order_by_elem.collation->as().value.get()); if (order_by_elem.with_fill) { - FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context); + auto column = result_block.findByName(name); + if (!column) + for (auto &[alias, ast] : aliases) + if (name == ast->getColumnName()) + if ((column = result_block.findByName(alias))) + break; + + FillColumnDescription fill_desc = getWithFillDescription(column->type, order_by_elem, context); order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, true, fill_desc); } else @@ -1492,7 +1504,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

aliases, context); for (auto & desc : order_descr) if (desc.with_fill) { @@ -2105,7 +2117,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc // TODO Do we need a projection variant for this field? query, analysis_result.order_by_elements_actions, - getSortDescription(query, context), + getSortDescription(query, result_header, syntax_analyzer_result->aliases, context), query_info.syntax_analyzer_result); } else @@ -2113,7 +2125,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc query_info.order_optimizer = std::make_shared( query, analysis_result.order_by_elements_actions, - getSortDescription(query, context), + getSortDescription(query, result_header, syntax_analyzer_result->aliases, context), query_info.syntax_analyzer_result); } } @@ -2542,7 +2554,7 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPlan & query_plan, Input void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfoPtr input_sorting_info) { auto & query = getSelectQuery(); - SortDescription output_order_descr = getSortDescription(query, context); + SortDescription output_order_descr = getSortDescription(query, result_header, syntax_analyzer_result->aliases, context); UInt64 limit = getLimitForSorting(query, context); if (input_sorting_info) @@ -2580,7 +2592,7 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const std::string & description) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, context); + SortDescription order_descr = getSortDescription(query, result_header, syntax_analyzer_result->aliases, context); UInt64 limit = getLimitForSorting(query, context); executeMergeSorted(query_plan, order_descr, limit, description); @@ -2683,7 +2695,7 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) auto & query = getSelectQuery(); if (query.orderBy()) { - SortDescription order_descr = getSortDescription(query, context); + SortDescription order_descr = getSortDescription(query, result_header, syntax_analyzer_result->aliases, context); SortDescription fill_descr; for (auto & desc : order_descr) { @@ -2734,7 +2746,7 @@ void InterpreterSelectQuery::executeLimit(QueryPlan & query_plan) { if (!query.orderBy()) throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); - order_descr = getSortDescription(query, context); + order_descr = getSortDescription(query, result_header, syntax_analyzer_result->aliases, context); } auto limit = std::make_unique( From adf305e1dd08ab2f453e223c9e00b3b93e385da6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 4 Jun 2022 11:44:20 -0400 Subject: [PATCH 026/659] search source header for columns --- src/Interpreters/InterpreterSelectQuery.cpp | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 5d9baadbfdf..038d52d63c2 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -877,7 +877,7 @@ static FillColumnDescription getWithFillDescription(DataTypePtr type, const ASTO return descr; } -static SortDescription getSortDescription(const ASTSelectQuery & query, const Block & result_block, const Aliases & aliases, ContextPtr context) +static SortDescription getSortDescription(const ASTSelectQuery & query, const Block & source_block, const Block & result_block, const Aliases & aliases, ContextPtr context) { SortDescription order_descr; order_descr.reserve(query.orderBy()->children.size()); @@ -892,6 +892,8 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, const Bl if (order_by_elem.with_fill) { auto column = result_block.findByName(name); + if (!column) + column = source_block.findByName(name); if (!column) for (auto &[alias, ast] : aliases) if (name == ast->getColumnName()) @@ -1504,7 +1506,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

aliases, context); + SortDescription order_descr = getSortDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context); for (auto & desc : order_descr) if (desc.with_fill) { @@ -2117,7 +2119,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc // TODO Do we need a projection variant for this field? query, analysis_result.order_by_elements_actions, - getSortDescription(query, result_header, syntax_analyzer_result->aliases, context), + getSortDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context), query_info.syntax_analyzer_result); } else @@ -2125,7 +2127,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc query_info.order_optimizer = std::make_shared( query, analysis_result.order_by_elements_actions, - getSortDescription(query, result_header, syntax_analyzer_result->aliases, context), + getSortDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context), query_info.syntax_analyzer_result); } } @@ -2554,7 +2556,7 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPlan & query_plan, Input void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfoPtr input_sorting_info) { auto & query = getSelectQuery(); - SortDescription output_order_descr = getSortDescription(query, result_header, syntax_analyzer_result->aliases, context); + SortDescription output_order_descr = getSortDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context); UInt64 limit = getLimitForSorting(query, context); if (input_sorting_info) @@ -2592,7 +2594,7 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const std::string & description) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, result_header, syntax_analyzer_result->aliases, context); + SortDescription order_descr = getSortDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context); UInt64 limit = getLimitForSorting(query, context); executeMergeSorted(query_plan, order_descr, limit, description); @@ -2695,7 +2697,7 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) auto & query = getSelectQuery(); if (query.orderBy()) { - SortDescription order_descr = getSortDescription(query, result_header, syntax_analyzer_result->aliases, context); + SortDescription order_descr = getSortDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context); SortDescription fill_descr; for (auto & desc : order_descr) { @@ -2746,7 +2748,7 @@ void InterpreterSelectQuery::executeLimit(QueryPlan & query_plan) { if (!query.orderBy()) throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); - order_descr = getSortDescription(query, result_header, syntax_analyzer_result->aliases, context); + order_descr = getSortDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context); } auto limit = std::make_unique( From 07009c27c234e0061f2015f6dfb6d887c9963690 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 4 Jun 2022 17:24:48 -0400 Subject: [PATCH 027/659] tidy build suggestions --- src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 038d52d63c2..27d866c976a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -891,11 +891,11 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, const Bl collator = std::make_shared(order_by_elem.collation->as().value.get()); if (order_by_elem.with_fill) { - auto column = result_block.findByName(name); + const auto *column = result_block.findByName(name); if (!column) column = source_block.findByName(name); if (!column) - for (auto &[alias, ast] : aliases) + for (const auto &[alias, ast] : aliases) if (name == ast->getColumnName()) if ((column = result_block.findByName(alias))) break; From c7ca746994c964605e793ede63e381a3923498c4 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sun, 5 Jun 2022 14:46:28 -0400 Subject: [PATCH 028/659] fixed build issues in clickhouse-keeper --- CMakeLists.txt | 10 ++++++---- contrib/qpl-cmake/CMakeLists.txt | 6 ++++++ docker/packager/binary/Dockerfile | 1 - docker/test/base/Dockerfile | 1 - docker/test/fasttest/Dockerfile | 1 - src/Compression/CompressionFactory.cpp | 2 +- 6 files changed, 13 insertions(+), 8 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 9c1f0957424..0252347178f 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -122,10 +122,12 @@ if (ENABLE_FUZZING) set (ENABLE_PROTOBUF 1) endif() -option (ENABLE_QPL "Enable Intel® Query Processing Library" ON) -if (ENABLE_QPL) - set (ENABLE_QPL_COMPRESSION 1) - set (ENABLE_QPL_ANALYTIC 1) +if (OS_LINUX AND ARCH_AMD64 AND NOT BUILD_STANDALONE_KEEPER) + option (ENABLE_QPL "Enable Intel® Query Processing Library" ON) + if (ENABLE_QPL) + set (ENABLE_QPL_COMPRESSION 1) + set (ENABLE_QPL_ANALYTIC 1) + endif() endif() # Global libraries diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 73aba26a607..8b21c9db360 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -111,6 +111,12 @@ target_compile_options(isal_asm PUBLIC "-I${QPL_SRC_DIR}/isal/include/" PUBLIC "-DAS_FEATURE_LEVEL=10" PUBLIC "-DQPL_LIB") +if (SANITIZE STREQUAL "undefined") + get_target_property(target_options isal_asm COMPILE_OPTIONS) + list(REMOVE_ITEM target_options "no-sanitize=undefined") + set_property(TARGET isal_asm PROPERTY COMPILE_OPTIONS ${target_options}) +endif() + target_compile_definitions(isal PUBLIC QPL_LIB NDEBUG) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 5f45596c62f..acfeecd5a7e 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -52,7 +52,6 @@ RUN apt-get update \ software-properties-common \ tzdata \ nasm \ - libc6-dev \ --yes --no-install-recommends \ && apt-get clean diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index af28ccb6ef9..c89ca065141 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -56,7 +56,6 @@ RUN apt-get update \ tzdata \ pv \ nasm \ - libc6-dev \ --yes --no-install-recommends # Sanitizer options for services (clickhouse-server) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index fe2f4a6f38d..591d90c6610 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -72,7 +72,6 @@ RUN apt-get update \ tzdata \ unixodbc \ nasm \ - libc6-dev \ --yes --no-install-recommends RUN pip3 install numpy scipy pandas Jinja2 diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 05595418115..4892a49628c 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -187,10 +187,10 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecZSTD(*this); registerCodecLZ4HC(*this); registerCodecMultiple(*this); +#ifndef KEEPER_STANDALONE_BUILD #ifdef ENABLE_QPL_COMPRESSION registerCodecDeflate(*this); #endif -#ifndef KEEPER_STANDALONE_BUILD registerCodecDelta(*this); registerCodecT64(*this); registerCodecDoubleDelta(*this); From 52ae3f0e9262700e5a057bd1f5226de2f15022a0 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 5 Jun 2022 22:39:21 -0400 Subject: [PATCH 029/659] full refactoring - move type check to transform --- src/Core/SortDescription.h | 2 + src/Interpreters/InterpreterSelectQuery.cpp | 48 +++++++------------ .../Transforms/FillingTransform.cpp | 9 ++++ 3 files changed, 28 insertions(+), 31 deletions(-) diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index 3d4e3b665ee..75a4afe4ef0 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -28,7 +28,9 @@ struct FillColumnDescription /// All missed values in range [FROM, TO) will be filled /// Range [FROM, TO) respects sorting direction Field fill_from; /// Fill value >= FILL_FROM + DataTypePtr fill_from_type; Field fill_to; /// Fill value + STEP < FILL_TO + DataTypePtr fill_to_type; Field fill_step; /// Default = +1 or -1 according to direction std::optional step_kind; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 27d866c976a..791bcd5562f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include @@ -803,18 +802,14 @@ Block InterpreterSelectQuery::getSampleBlockImpl() return analysis_result.final_projection->getResultColumns(); } -static Field getWithFillFieldValue(DataTypePtr col_type, const ASTPtr & node, ContextPtr context) +static std::pair getWithFillFieldValue(const ASTPtr & node, ContextPtr context) { - auto [field, type] = evaluateConstantExpression(node, context); + auto field_type = evaluateConstantExpression(node, context); - WhichDataType which(col_type); - if ((which.isDateOrDate32() || which.isDateTime() || which.isDateTime64()) && !col_type->equals(*type)) - throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be same as column " + col_type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + if (!isColumnedAsNumber(field_type.second)) + throw Exception("Illegal type " + field_type.second->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); - if (!isColumnedAsNumber(type)) - throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); - - return field; + return field_type; } static std::pair> getWithFillStep(const ASTPtr & node, ContextPtr context) @@ -830,14 +825,14 @@ static std::pair> getWithFillStep(const ASTPt throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); } -static FillColumnDescription getWithFillDescription(DataTypePtr type, const ASTOrderByElement & order_by_elem, ContextPtr context) +static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, ContextPtr context) { FillColumnDescription descr; if (order_by_elem.fill_from) - descr.fill_from = getWithFillFieldValue(type, order_by_elem.fill_from, context); + std::tie(descr.fill_from, descr.fill_from_type) = getWithFillFieldValue(order_by_elem.fill_from, context); if (order_by_elem.fill_to) - descr.fill_to = getWithFillFieldValue(type, order_by_elem.fill_to, context); + std::tie(descr.fill_to, descr.fill_to_type) = getWithFillFieldValue(order_by_elem.fill_to, context); if (order_by_elem.fill_step) std::tie(descr.fill_step, descr.step_kind) = getWithFillStep(order_by_elem.fill_step, context); @@ -877,7 +872,7 @@ static FillColumnDescription getWithFillDescription(DataTypePtr type, const ASTO return descr; } -static SortDescription getSortDescription(const ASTSelectQuery & query, const Block & source_block, const Block & result_block, const Aliases & aliases, ContextPtr context) +static SortDescription getSortDescription(const ASTSelectQuery & query, ContextPtr context) { SortDescription order_descr; order_descr.reserve(query.orderBy()->children.size()); @@ -891,16 +886,7 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, const Bl collator = std::make_shared(order_by_elem.collation->as().value.get()); if (order_by_elem.with_fill) { - const auto *column = result_block.findByName(name); - if (!column) - column = source_block.findByName(name); - if (!column) - for (const auto &[alias, ast] : aliases) - if (name == ast->getColumnName()) - if ((column = result_block.findByName(alias))) - break; - - FillColumnDescription fill_desc = getWithFillDescription(column->type, order_by_elem, context); + FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context); order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, true, fill_desc); } else @@ -1506,7 +1492,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

aliases, context); + SortDescription order_descr = getSortDescription(query, context); for (auto & desc : order_descr) if (desc.with_fill) { @@ -2119,7 +2105,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc // TODO Do we need a projection variant for this field? query, analysis_result.order_by_elements_actions, - getSortDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context), + getSortDescription(query, context), query_info.syntax_analyzer_result); } else @@ -2127,7 +2113,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc query_info.order_optimizer = std::make_shared( query, analysis_result.order_by_elements_actions, - getSortDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context), + getSortDescription(query, context), query_info.syntax_analyzer_result); } } @@ -2556,7 +2542,7 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPlan & query_plan, Input void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfoPtr input_sorting_info) { auto & query = getSelectQuery(); - SortDescription output_order_descr = getSortDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context); + SortDescription output_order_descr = getSortDescription(query, context); UInt64 limit = getLimitForSorting(query, context); if (input_sorting_info) @@ -2594,7 +2580,7 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const std::string & description) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context); + SortDescription order_descr = getSortDescription(query, context); UInt64 limit = getLimitForSorting(query, context); executeMergeSorted(query_plan, order_descr, limit, description); @@ -2697,7 +2683,7 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) auto & query = getSelectQuery(); if (query.orderBy()) { - SortDescription order_descr = getSortDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context); + SortDescription order_descr = getSortDescription(query, context); SortDescription fill_descr; for (auto & desc : order_descr) { @@ -2748,7 +2734,7 @@ void InterpreterSelectQuery::executeLimit(QueryPlan & query_plan) { if (!query.orderBy()) throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); - order_descr = getSortDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context); + order_descr = getSortDescription(query, context); } auto limit = std::make_unique( diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index a41b5660e0d..153f962a8f4 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -54,6 +54,15 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & WhichDataType which(type); DataTypePtr to_type; + WhichDataType which_from(descr.fill_from_type); + bool is_from_date = which_from.isDateOrDate32() || which_from.isDateTime() || which_from.isDateTime64(); + WhichDataType which_to(descr.fill_to_type); + bool is_to_date = which_to.isDateOrDate32() || which_to.isDateTime() || which_to.isDateTime64(); + + if ((is_from_date || is_to_date) && + (!descr.fill_from_type->equals(*descr.fill_to_type) || !descr.fill_from_type->equals(*type))) + return false; + /// TODO Wrong results for big integers. if (isInteger(type) || which.isDate() || which.isDate32() || which.isDateTime()) { From c9ff52f2e9829d3079d60e992a72bd63d192eb2e Mon Sep 17 00:00:00 2001 From: Wangyang Guo Date: Fri, 22 Apr 2022 16:15:23 +0800 Subject: [PATCH 030/659] Add VBMI optimized copyOverlap32Shuffle --- src/Compression/LZ4_decompress_faster.cpp | 53 +++++++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 32d21a37f18..7a7c7001297 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -15,6 +15,10 @@ #include #endif +#if defined(__AVX512VBMI__) +#include +#endif + #ifdef __aarch64__ #include #endif @@ -403,10 +407,59 @@ inline void copyOverlap32(UInt8 * op, const UInt8 *& match, const size_t offset) match += shift4[offset]; } +inline void copyOverlap32Shuffle(UInt8 * op, const UInt8 *& match, const size_t offset) +{ +#if defined(__AVX512VBMI__) && !defined(MEMORY_SANITIZER) + static constexpr UInt8 __attribute__((__aligned__(32))) masks[] = + { + 0, 1, 2, 2, 4, 2, 2, 4, 8, 5, 2, 10, 8, 6, 4, 2, 16, 15, 14, 13, 12, 11, 10, 9, 8, 7, 6, 5, 4, 3, 2, 1, /* offset=0, shift amount index. */ + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, /* offset=1 */ + 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, + 0, 1, 2, 0, 1, 2, 0, 1, 2, 0, 1, 2, 0, 1, 2, 0, 1, 2, 0, 1, 2, 0, 1, 2, 0, 1, 2, 0, 1, 2, 0, 1, + 0, 1, 2, 3, 0, 1, 2, 3, 0, 1, 2, 3, 0, 1, 2, 3, 0, 1, 2, 3, 0, 1, 2, 3, 0, 1, 2, 3, 0, 1, 2, 3, + 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, + 0, 1, 2, 3, 4, 5, 0, 1, 2, 3, 4, 5, 0, 1, 2, 3, 4, 5, 0, 1, 2, 3, 4, 5, 0, 1, 2, 3, 4, 5, 0, 1, + 0, 1, 2, 3, 4, 5, 6, 0, 1, 2, 3, 4, 5, 6, 0, 1, 2, 3, 4, 5, 6, 0, 1, 2, 3, 4, 5, 6, 0, 1, 2, 3, + 0, 1, 2, 3, 4, 5, 6, 7, 0, 1, 2, 3, 4, 5, 6, 7, 0, 1, 2, 3, 4, 5, 6, 7, 0, 1, 2, 3, 4, 5, 6, 7, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 0, 1, 2, 3, 4, 5, 6, 7, 8, 0, 1, 2, 3, 4, 5, 6, 7, 8, 0, 1, 2, 3, 4, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 0, 1, 2, 3, 4, 5, 6, 7, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 0, 1, 2, 3, 4, 5, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 0, 1, 2, 3, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 0, 1, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 0, 1, 2, 3, 4, 5, 6, 7, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 0, 1, 2, 3, 4, 5, 6, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 0, 1, 2, 3, 4, 5, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 0, 1, 2, 3, 4, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 0, 1, 2, 3, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 0, 1, 2, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 0, 1, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 0, + }; + + _mm256_storeu_si256(reinterpret_cast<__m256i *>(op), + _mm256_permutexvar_epi8( + _mm256_load_si256(reinterpret_cast(masks) + offset), + _mm256_loadu_si256(reinterpret_cast(match)))); +#else + copyOverlap32(op, match, offset); +#endif +} + template <> void inline copy<32>(UInt8 * dst, const UInt8 * src) { copy32(dst, src); } template <> void inline wildCopy<32>(UInt8 * dst, const UInt8 * src, UInt8 * dst_end) { wildCopy32(dst, src, dst_end); } template <> void inline copyOverlap<32, false>(UInt8 * op, const UInt8 *& match, const size_t offset) { copyOverlap32(op, match, offset); } +template <> void inline copyOverlap<32, true>(UInt8 * op, const UInt8 *& match, const size_t offset) { copyOverlap32Shuffle(op, match, offset); } /// See also https://stackoverflow.com/a/30669632 From 50ac01c9b86d368c7714acbcf76bb1c7db09e70e Mon Sep 17 00:00:00 2001 From: Wangyang Guo Date: Mon, 25 Apr 2022 15:44:30 +0800 Subject: [PATCH 031/659] Enable copyOverlap32Shuffle --- src/Compression/LZ4_decompress_faster.cpp | 17 +++++++++++++---- src/Compression/LZ4_decompress_faster.h | 2 +- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 7a7c7001297..4b5eac71428 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -446,10 +447,16 @@ inline void copyOverlap32Shuffle(UInt8 * op, const UInt8 *& match, const size_t 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 0, }; - _mm256_storeu_si256(reinterpret_cast<__m256i *>(op), - _mm256_permutexvar_epi8( - _mm256_load_si256(reinterpret_cast(masks) + offset), - _mm256_loadu_si256(reinterpret_cast(match)))); + if (DB::Cpu::CpuFlagsCache::have_AVX512VBMI) + { + _mm256_storeu_si256(reinterpret_cast<__m256i *>(op), + _mm256_permutexvar_epi8( + _mm256_load_si256(reinterpret_cast(masks) + offset), + _mm256_loadu_si256(reinterpret_cast(match)))); + match += masks[offset]; + } else { + copyOverlap32(op, match, offset); + } #else copyOverlap32(op, match, offset); #endif @@ -645,6 +652,8 @@ bool decompress( success = decompressImpl<8, true>(source, dest, source_size, dest_size); if (best_variant == 3) success = decompressImpl<32, false>(source, dest, source_size, dest_size); + if (best_variant == 4) + success = decompressImpl<32, true>(source, dest, source_size, dest_size); watch.stop(); diff --git a/src/Compression/LZ4_decompress_faster.h b/src/Compression/LZ4_decompress_faster.h index c596ea6364b..7c6224e939f 100644 --- a/src/Compression/LZ4_decompress_faster.h +++ b/src/Compression/LZ4_decompress_faster.h @@ -88,7 +88,7 @@ struct PerformanceStatistics }; /// Number of different algorithms to select from. - static constexpr size_t NUM_ELEMENTS = 4; + static constexpr size_t NUM_ELEMENTS = 5; /// Cold invocations may be affected by additional memory latencies. Don't take first invocations into account. static constexpr double NUM_INVOCATIONS_TO_THROW_OFF = 2; From 0cd2a6b9c38d9824c9f2f610ee110ca486782773 Mon Sep 17 00:00:00 2001 From: Wangyang Guo Date: Mon, 25 Apr 2022 16:24:39 +0800 Subject: [PATCH 032/659] Add ENABLE_AVX512_VBMI build flag --- cmake/cpu_features.cmake | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index fd4cc51b6f9..da4a35f4fc5 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -19,6 +19,7 @@ option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1) option (ENABLE_AVX "Use AVX instructions on x86_64" 0) option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0) option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 0) +option (ENABLE_AVX512_VBMI "Use AVX512_VBMI instruction on x86_64 (depends on ENABLE_AVX512)" 0) option (ENABLE_BMI "Use BMI instructions on x86_64" 0) option (ENABLE_AVX2_FOR_SPEC_OP "Use avx2 instructions for specific operations on x86_64" 0) option (ENABLE_AVX512_FOR_SPEC_OP "Use avx512 instructions for specific operations on x86_64" 0) @@ -151,6 +152,20 @@ elseif (ARCH_AMD64) set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () + set (TEST_FLAG "-mavx512vbmi") + set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") + check_cxx_source_compiles(" + #include + int main() { + auto a = _mm512_permutexvar_epi8(__m512i(), __m512i()); + (void)a; + return 0; + } + " HAVE_AVX512_VBMI) + if (HAVE_AVX512 AND ENABLE_AVX512 AND HAVE_AVX512_VBMI AND ENABLE_AVX512_VBMI) + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () + set (TEST_FLAG "-mbmi") set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") check_cxx_source_compiles(" From ea5e37abefb6c2e4a78f8c7f81ab254545ad9b41 Mon Sep 17 00:00:00 2001 From: Wangyang Guo Date: Mon, 16 May 2022 15:01:22 +0800 Subject: [PATCH 033/659] Align branches within a 32B boundary for AMD64 arch to make benchmark more stable --- CMakeLists.txt | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index a6a09afc489..5f8ad137795 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -319,6 +319,22 @@ if (COMPILER_GCC OR COMPILER_CLANG) set(COMPILER_FLAGS "${COMPILER_FLAGS} -falign-functions=32") endif () +if (ARCH_AMD64) + # align branches within a 32-Byte boundary to avoid the potential performance loss when code layout change, + # which makes benchmark results more stable. + set(BRANCHES_WITHIN_32B_BOUNDARIES "-mbranches-within-32B-boundaries") + if (COMPILER_GCC) + # gcc is in assembler, need to add "-Wa," prefix + set(BRANCHES_WITHIN_32B_BOUNDARIES "-Wa,${BRANCHES_WITHIN_32B_BOUNDARIES}") + endif() + + include(CheckCXXCompilerFlag) + check_cxx_compiler_flag("${BRANCHES_WITHIN_32B_BOUNDARIES}" HAS_BRANCHES_WITHIN_32B_BOUNDARIES) + if (HAS_BRANCHES_WITHIN_32B_BOUNDARIES) + set(COMPILER_FLAGS "${COMPILER_FLAGS} ${BRANCHES_WITHIN_32B_BOUNDARIES}") + endif() +endif() + if (COMPILER_GCC) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fcoroutines") endif () From addefa202037c847c9a91f55fd278395f5f5ce47 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 6 Jun 2022 16:36:03 -0400 Subject: [PATCH 034/659] from or to statement can be absent --- .../Transforms/FillingTransform.cpp | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 153f962a8f4..1303b9affff 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -54,14 +54,21 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & WhichDataType which(type); DataTypePtr to_type; - WhichDataType which_from(descr.fill_from_type); - bool is_from_date = which_from.isDateOrDate32() || which_from.isDateTime() || which_from.isDateTime64(); - WhichDataType which_to(descr.fill_to_type); - bool is_to_date = which_to.isDateOrDate32() || which_to.isDateTime() || which_to.isDateTime64(); + if (descr.fill_from_type) + { + WhichDataType which_from(descr.fill_from_type); + if ((which_from.isDateOrDate32() || which_from.isDateTime() || which_from.isDateTime64()) && + !descr.fill_from_type->equals(*type)) + return false; + } - if ((is_from_date || is_to_date) && - (!descr.fill_from_type->equals(*descr.fill_to_type) || !descr.fill_from_type->equals(*type))) - return false; + if (descr.fill_to_type) + { + WhichDataType which_to(descr.fill_to_type); + if ((which_to.isDateOrDate32() || which_to.isDateTime() || which_to.isDateTime64()) && + !descr.fill_to_type->equals(*type)) + return false; + } /// TODO Wrong results for big integers. if (isInteger(type) || which.isDate() || which.isDate32() || which.isDateTime()) From 2b684ff40e10a43b301aca0da61b6c24f9b3e466 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 8 Jun 2022 09:28:35 -0400 Subject: [PATCH 035/659] remove sw job pool --- src/Compression/CompressionCodecDeflate.cpp | 165 ++++------------- src/Compression/CompressionCodecDeflate.h | 191 ++++---------------- 2 files changed, 77 insertions(+), 279 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 985e6f71c33..6d724738361 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -18,9 +18,6 @@ namespace ErrorCodes qpl_job * DeflateJobHWPool::jobPool[jobPoolSize]; std::atomic_bool DeflateJobHWPool::jobLock[jobPoolSize]; -qpl_job * DeflateJobSWPool::jobSWPool[jobSWPoolSize]; -std::atomic_bool DeflateJobSWPool::jobSWLock[jobSWPoolSize]; - DeflateJobHWPool & DeflateJobHWPool::instance() { static DeflateJobHWPool ret; @@ -29,35 +26,27 @@ DeflateJobHWPool & DeflateJobHWPool::instance() DeflateJobHWPool::DeflateJobHWPool() { + log = &Poco::Logger::get("DeflateJobHWPool"); if (initJobPool() < 0) - throw Exception("DeflateJobHWPool initializing fail!", ErrorCodes::CANNOT_COMPRESS); + { + jobPoolAvailable = false; + LOG_WARNING(log, "DeflateJobHWPool initializing fail! Please check if IAA hardware support."); + } + else + { + jobPoolAvailable = true; + } } DeflateJobHWPool::~DeflateJobHWPool() { destroyJobPool(); } -DeflateJobSWPool & DeflateJobSWPool::instance() -{ - static DeflateJobSWPool ret; - return ret; -} - -DeflateJobSWPool::DeflateJobSWPool() -{ - if (initJobPool() < 0) - throw Exception("DeflateJobSWPool initializing fail!", ErrorCodes::CANNOT_COMPRESS); -} -DeflateJobSWPool::~DeflateJobSWPool() -{ - destroyJobPool(); -} - - CompressionCodecDeflate::CompressionCodecDeflate() { log = &Poco::Logger::get("CompressionCodecDeflate"); setCodecDescription("DEFLATE"); + jobSWPtr = initSoftwareJobCodecPtr(); } CompressionCodecDeflate::~CompressionCodecDeflate() @@ -65,7 +54,6 @@ CompressionCodecDeflate::~CompressionCodecDeflate() if (!jobDecompAsyncMap.empty()) { LOG_ERROR(log, "Exception -> find un-released job when CompressionCodecDeflate destroy"); - //doDecompressDataFlush(); for (auto it : jobDecompAsyncMap) { DeflateJobHWPool::instance().releaseJob(it.first); @@ -103,25 +91,35 @@ uint32_t CompressionCodecDeflate::getMaxCompressedDataSize(uint32_t uncompressed return DEFLATE_COMPRESSBOUND(uncompressed_size); } -uint32_t CompressionCodecDeflate::doCompressDataSWNative(const char * source, uint32_t source_size, char * dest) const +qpl_job * CompressionCodecDeflate::initSoftwareJobCodecPtr() { + qpl_job * job_ptr; qpl_status status; uint32_t size = 0; + std::unique_ptr job_buffer; // Job initialization - status = qpl_get_job_size(DeflateJobSWPool::SW_PATH, &size); + status = qpl_get_job_size(qpl_path_software, &size); if (status != QPL_STS_OK) { - throw Exception("doCompressDataSWNative cannot compress: qpl_get_job_size fail", ErrorCodes::CANNOT_COMPRESS); + throw Exception("initSoftwareJobCodecPtr: qpl_get_job_size fail:"+ std::to_string(status), ErrorCodes::CANNOT_COMPRESS); } - qpl_job * job_ptr = reinterpret_cast(new uint8_t[size]); - status = qpl_init_job(DeflateJobSWPool::SW_PATH, job_ptr); + job_buffer = std::make_unique(size); + job_ptr = reinterpret_cast(job_buffer.get()); + + status = qpl_init_job(qpl_path_software, job_ptr); if (status != QPL_STS_OK) { - throw Exception("doCompressDataSWNative cannot compress: qpl_init_job fail", ErrorCodes::CANNOT_COMPRESS); + throw Exception("initSoftwareJobCodecPtr: qpl_init_job fail:"+ std::to_string(status), ErrorCodes::CANNOT_COMPRESS); } + return job_ptr; +} +uint32_t CompressionCodecDeflate::doCompressDataSW(const char * source, uint32_t source_size, char * dest)const +{ + qpl_status status; + qpl_job * job_ptr = jobSWPtr; // Performing a compression operation job_ptr->op = qpl_op_compress; job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); @@ -135,49 +133,10 @@ uint32_t CompressionCodecDeflate::doCompressDataSWNative(const char * source, ui status = qpl_execute_job(job_ptr); if (status != QPL_STS_OK) { - throw Exception("doCompressDataSWNative cannot compress: qpl_execute_job fail", ErrorCodes::CANNOT_COMPRESS); + throw Exception("doCompressDataSW cannot compress: qpl_execute_job fail:" + std::to_string(status), ErrorCodes::CANNOT_COMPRESS); } - const uint32_t compressed_size = job_ptr->total_out; - // Freeing resources - status = qpl_fini_job(job_ptr); - if (status != QPL_STS_OK) - { - throw Exception("doCompressDataSWNative cannot compress: qpl_fini_job fail", ErrorCodes::CANNOT_COMPRESS); - } - - delete[] job_ptr; - return compressed_size; -} - -uint32_t CompressionCodecDeflate::doCompressDataSW(const char * source, uint32_t source_size, char * dest) const -{ - uint32_t job_id = 0; - qpl_job * job_ptr = DeflateJobSWPool::instance().acquireJob(&job_id); - if (job_ptr == nullptr) - { - LOG_WARNING(log, "doCompressDataSW acquireJob fail! switch to SW native compress..."); - return doCompressDataSWNative(source, source_size, dest); - } - qpl_status status; - uint32_t compressed_size = 0; - - job_ptr->op = qpl_op_compress; - job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); - job_ptr->next_out_ptr = reinterpret_cast(dest); - job_ptr->available_in = source_size; - job_ptr->available_out = getMaxCompressedDataSize(source_size); - job_ptr->level = qpl_high_level; - job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; - // Compression - status = qpl_execute_job(job_ptr); - if (QPL_STS_OK != status) - { - throw Exception("doCompressDataSW Cannot compress", ErrorCodes::CANNOT_COMPRESS); - } - compressed_size = job_ptr->total_out; - DeflateJobSWPool::instance().releaseJob(job_id); - return compressed_size; + return job_ptr->total_out; } uint32_t CompressionCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest) const @@ -251,12 +210,15 @@ uint32_t CompressionCodecDeflate::doCompressDataFlush(uint32_t req_id) { uint32_t compressed_size = 0; qpl_job * job_ptr = DeflateJobHWPool::instance().getJobPtr(req_id); - while (QPL_STS_BEING_PROCESSED == qpl_check_job(job_ptr)) + if(nullptr != job_ptr) { - _tpause(1, __rdtsc() + 1000); + while (QPL_STS_BEING_PROCESSED == qpl_check_job(job_ptr)) + { + _tpause(1, __rdtsc() + 1000); + } + compressed_size = job_ptr->total_out; + DeflateJobHWPool::instance().releaseJob(req_id); } - compressed_size = job_ptr->total_out; - DeflateJobHWPool::instance().releaseJob(req_id); return compressed_size; } @@ -294,25 +256,10 @@ void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t sou DeflateJobHWPool::instance().releaseJob(job_id); } -void CompressionCodecDeflate::doDecompressDataSWNative( - const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) +void CompressionCodecDeflate::doDecompressDataSW(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size)const { qpl_status status; - uint32_t size = 0; - - // Job initialization - status = qpl_get_job_size(DeflateJobSWPool::SW_PATH, &size); - if (status != QPL_STS_OK) - { - throw Exception("doDecompressDataSWNative cannot decompress: qpl_get_job_size fail", ErrorCodes::CANNOT_DECOMPRESS); - } - qpl_job * job_ptr = reinterpret_cast(new uint8_t[size]); - - status = qpl_init_job(DeflateJobSWPool::SW_PATH, job_ptr); - if (status != QPL_STS_OK) - { - throw Exception("doDecompressDataSWNative cannot decompress: qpl_init_job fail", ErrorCodes::CANNOT_DECOMPRESS); - } + qpl_job * job_ptr = jobSWPtr; // Performing a decompression operation job_ptr->op = qpl_op_decompress; @@ -326,44 +273,8 @@ void CompressionCodecDeflate::doDecompressDataSWNative( status = qpl_execute_job(job_ptr); if (status != QPL_STS_OK) { - throw Exception("doDecompressDataSWNative cannot decompress: qpl_execute_job fail", ErrorCodes::CANNOT_DECOMPRESS); + throw Exception("doDecompressDataSW cannot decompress: qpl_execute_job fail:"+ std::to_string(status), ErrorCodes::CANNOT_DECOMPRESS); } - // Freeing resources - status = qpl_fini_job(job_ptr); - if (status != QPL_STS_OK) - { - throw Exception("doDecompressDataSWNative cannot decompress: qpl_fini_job fail", ErrorCodes::CANNOT_DECOMPRESS); - } - delete[] job_ptr; -} - -void CompressionCodecDeflate::doDecompressDataSW(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const -{ - uint32_t job_id = 0; - qpl_job * job_ptr = DeflateJobSWPool::instance().acquireJob(&job_id); - if (job_ptr == nullptr) - { - LOG_WARNING(log, "doDecompressDataSW acquireJob fail! switch to SW native decompress..."); - return doDecompressDataSWNative(source, source_size, dest, uncompressed_size); - } - qpl_status status; - - // Performing a decompression operation - job_ptr->op = qpl_op_decompress; - job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); - job_ptr->next_out_ptr = reinterpret_cast(dest); - job_ptr->available_in = source_size; - job_ptr->available_out = uncompressed_size; - job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; - - // Decompression - status = qpl_execute_job(job_ptr); - - if (QPL_STS_OK != status) - { - throw Exception("doDecompressDataSW cannot decompress", ErrorCodes::CANNOT_DECOMPRESS); - } - DeflateJobSWPool::instance().releaseJob(job_id); } void CompressionCodecDeflate::doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index 588709e7df4..76de3dd0d3f 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -29,33 +29,56 @@ public: qpl_job * ALWAYS_INLINE acquireJob(uint32_t * job_id) { - uint32_t retry = 0; - auto index = random(jobPoolSize); - while (tryLockJob(index) == false) + if(jobPoolAvailable) { - index = random(jobPoolSize); - retry++; - if (retry > jobPoolSize) + uint32_t retry = 0; + auto index = random(jobPoolSize); + while (tryLockJob(index) == false) { - return nullptr; + index = random(jobPoolSize); + retry++; + if (retry > jobPoolSize) + { + return nullptr; + } } + *job_id = jobPoolSize - index; + return jobPool[index]; + } + else + { + return nullptr; } - *job_id = jobPoolSize - index; - return jobPool[index]; } qpl_job * ALWAYS_INLINE releaseJob(uint32_t job_id) { - uint32_t index = jobPoolSize - job_id; - ReleaseJobObjectGuard _(index); - return jobPool[index]; + if(jobPoolAvailable) + { + uint32_t index = jobPoolSize - job_id; + ReleaseJobObjectGuard _(index); + return jobPool[index]; + } + else + { + return nullptr; + } } qpl_job * ALWAYS_INLINE getJobPtr(uint32_t job_id) { - uint32_t index = jobPoolSize - job_id; - return jobPool[index]; + if(jobPoolAvailable) + { + uint32_t index = jobPoolSize - job_id; + return jobPool[index]; + } + else + { + return nullptr; + } } private: + bool jobPoolAvailable; + Poco::Logger * log; size_t ALWAYS_INLINE random(uint32_t pool_size) { size_t tsc = 0; @@ -156,141 +179,6 @@ private: }; }; -class DeflateJobSWPool -{ -public: - DeflateJobSWPool(); - ~DeflateJobSWPool(); - static DeflateJobSWPool & instance(); - - static constexpr auto jobSWPoolSize = 128; - static constexpr qpl_path_t SW_PATH = qpl_path_software; - static qpl_job * jobSWPool[jobSWPoolSize]; - static std::atomic_bool jobSWLock[jobSWPoolSize]; - - qpl_job * ALWAYS_INLINE acquireJob(uint32_t * job_id) - { - uint32_t retry = 0; - auto index = random(jobSWPoolSize); - while (tryLockJob(index) == false) - { - index = random(jobSWPoolSize); - retry++; - if (retry > jobSWPoolSize) - { - return nullptr; - } - } - *job_id = jobSWPoolSize - index; - return jobSWPool[index]; - } - qpl_job * ALWAYS_INLINE releaseJob(uint32_t job_id) - { - uint32_t index = jobSWPoolSize - job_id; - ReleaseJobObjectGuard _(index); - return jobSWPool[index]; - } - -private: - size_t ALWAYS_INLINE random(uint32_t pool_size) - { - size_t tsc = 0; - unsigned lo, hi; - __asm__ volatile("rdtsc" : "=a"(lo), "=d"(hi) : :); - tsc = (((static_cast(hi)) << 32) | (static_cast(lo))); - return (static_cast((tsc * 44485709377909ULL) >> 4)) % pool_size; - } - - int32_t ALWAYS_INLINE get_job_size_helper() - { - static uint32_t size = 0; - if (size == 0) - { - const auto status = qpl_get_job_size(SW_PATH, &size); - if (status != QPL_STS_OK) - { - return -1; - } - } - return size; - } - - int32_t ALWAYS_INLINE init_job_helper(qpl_job * qpl_job_ptr) - { - if (qpl_job_ptr == nullptr) - { - return -1; - } - auto status = qpl_init_job(SW_PATH, qpl_job_ptr); - if (status != QPL_STS_OK) - { - return -1; - } - return 0; - } - - int32_t ALWAYS_INLINE initJobPool() - { - static bool initialized = false; - - if (initialized == false) - { - const int32_t size = get_job_size_helper(); - if (size < 0) - return -1; - for (int i = 0; i < jobSWPoolSize; ++i) - { - jobSWPool[i] = nullptr; - qpl_job * qpl_job_ptr = reinterpret_cast(new uint8_t[size]); - if (init_job_helper(qpl_job_ptr) < 0) - return -1; - jobSWPool[i] = qpl_job_ptr; - jobSWLock[i].store(false); - } - initialized = true; - } - return 0; - } - - bool ALWAYS_INLINE tryLockJob(size_t index) - { - bool expected = false; - return jobSWLock[index].compare_exchange_strong(expected, true); - } - - void ALWAYS_INLINE destroyJobPool() - { - const uint32_t size = get_job_size_helper(); - for (uint32_t i = 0; i < jobSWPoolSize && size > 0; ++i) - { - while (tryLockJob(i) == false) - { - } - if (jobSWPool[i]) - { - qpl_fini_job(jobSWPool[i]); - delete[] jobSWPool[i]; - } - jobSWPool[i] = nullptr; - jobSWLock[i].store(false); - } - } - - struct ReleaseJobObjectGuard - { - uint32_t index; - ReleaseJobObjectGuard() = delete; - - public: - ALWAYS_INLINE ReleaseJobObjectGuard(const uint32_t i) : index(i) - { - } - ALWAYS_INLINE ~ReleaseJobObjectGuard() - { - jobSWLock[index].store(false); - } - }; -}; class CompressionCodecDeflate : public ICompressionCodec { public: @@ -309,17 +197,15 @@ protected: { return true; } - + qpl_job * initSoftwareJobCodecPtr(); uint32_t doCompressData(const char * source, uint32_t source_size, char * dest) const override; uint32_t doCompressDataSW(const char * source, uint32_t source_size, char * dest) const; - uint32_t doCompressDataSWNative(const char * source, uint32_t source_size, char * dest) const; UInt32 doCompressDataReq(const char * source, UInt32 source_size, char * dest, uint32_t & req_id) override; uint32_t doCompressDataFlush(uint32_t req_id) override; void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; void doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) override; void doDecompressDataSW(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; - static void doDecompressDataSWNative(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); void doDecompressDataFlush() override; private: @@ -327,6 +213,7 @@ private: std::map jobDecompAsyncMap; std::vector jobCompAsyncList; Poco::Logger * log; + qpl_job * jobSWPtr; //Software Job Codec Ptr }; } From 1236d74ae7e18d8973376b8ebf59e234c7be4866 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 8 Jun 2022 11:47:44 -0400 Subject: [PATCH 036/659] restructure code design to be compatible both of hardware and software codec --- src/Compression/CompressionCodecDeflate.cpp | 379 ++++++++++++-------- src/Compression/CompressionCodecDeflate.h | 56 ++- 2 files changed, 274 insertions(+), 161 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 6d724738361..de5a1c30486 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -1,11 +1,11 @@ -#include #include +#include #include #include #include #include -#include #include +#include namespace DB { @@ -14,7 +14,7 @@ namespace ErrorCodes extern const int CANNOT_COMPRESS; extern const int CANNOT_DECOMPRESS; } - +//DeflateJobHWPool qpl_job * DeflateJobHWPool::jobPool[jobPoolSize]; std::atomic_bool DeflateJobHWPool::jobLock[jobPoolSize]; @@ -29,31 +29,28 @@ DeflateJobHWPool::DeflateJobHWPool() log = &Poco::Logger::get("DeflateJobHWPool"); if (initJobPool() < 0) { - jobPoolAvailable = false; - LOG_WARNING(log, "DeflateJobHWPool initializing fail! Please check if IAA hardware support."); + jobPoolEnabled = false; + LOG_WARNING(log, "DeflateJobHWPool is not ready. Please check if IAA hardware support.Auto switch to deflate software codec here"); } else - { - jobPoolAvailable = true; - } + jobPoolEnabled = true; } DeflateJobHWPool::~DeflateJobHWPool() { destroyJobPool(); } - -CompressionCodecDeflate::CompressionCodecDeflate() +//HardwareCodecDeflate +HardwareCodecDeflate::HardwareCodecDeflate() { - log = &Poco::Logger::get("CompressionCodecDeflate"); - setCodecDescription("DEFLATE"); - jobSWPtr = initSoftwareJobCodecPtr(); + log = &Poco::Logger::get("HardwareCodecDeflate"); + hwEnabled = DeflateJobHWPool::instance().jobPoolReady(); } -CompressionCodecDeflate::~CompressionCodecDeflate() +HardwareCodecDeflate::~HardwareCodecDeflate() { if (!jobDecompAsyncMap.empty()) { - LOG_ERROR(log, "Exception -> find un-released job when CompressionCodecDeflate destroy"); + LOG_WARNING(log, "Find un-released job when HardwareCodecDeflate destroy"); for (auto it : jobDecompAsyncMap) { DeflateJobHWPool::instance().releaseJob(it.first); @@ -69,84 +66,14 @@ CompressionCodecDeflate::~CompressionCodecDeflate() jobCompAsyncList.clear(); } } - -uint8_t CompressionCodecDeflate::getMethodByte() const -{ - return static_cast(CompressionMethodByte::Deflate); -} - -bool CompressionCodecDeflate::isAsyncSupported() const -{ - return true; -} - -void CompressionCodecDeflate::updateHash(SipHash & hash) const -{ - getCodecDesc()->updateTreeHash(hash); -} - -#define DEFLATE_COMPRESSBOUND(isize) ((isize) + ((isize) >> 12) + ((isize) >> 14) + ((isize) >> 25) + 13) //Aligned with ZLIB -uint32_t CompressionCodecDeflate::getMaxCompressedDataSize(uint32_t uncompressed_size) const -{ - return DEFLATE_COMPRESSBOUND(uncompressed_size); -} - -qpl_job * CompressionCodecDeflate::initSoftwareJobCodecPtr() -{ - qpl_job * job_ptr; - qpl_status status; - uint32_t size = 0; - std::unique_ptr job_buffer; - - // Job initialization - status = qpl_get_job_size(qpl_path_software, &size); - if (status != QPL_STS_OK) - { - throw Exception("initSoftwareJobCodecPtr: qpl_get_job_size fail:"+ std::to_string(status), ErrorCodes::CANNOT_COMPRESS); - } - - job_buffer = std::make_unique(size); - job_ptr = reinterpret_cast(job_buffer.get()); - - status = qpl_init_job(qpl_path_software, job_ptr); - if (status != QPL_STS_OK) - { - throw Exception("initSoftwareJobCodecPtr: qpl_init_job fail:"+ std::to_string(status), ErrorCodes::CANNOT_COMPRESS); - } - return job_ptr; -} - -uint32_t CompressionCodecDeflate::doCompressDataSW(const char * source, uint32_t source_size, char * dest)const -{ - qpl_status status; - qpl_job * job_ptr = jobSWPtr; - // Performing a compression operation - job_ptr->op = qpl_op_compress; - job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); - job_ptr->next_out_ptr = reinterpret_cast(dest); - job_ptr->available_in = source_size; - job_ptr->available_out = getMaxCompressedDataSize(source_size); - job_ptr->level = qpl_high_level; - job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; - - // Compression - status = qpl_execute_job(job_ptr); - if (status != QPL_STS_OK) - { - throw Exception("doCompressDataSW cannot compress: qpl_execute_job fail:" + std::to_string(status), ErrorCodes::CANNOT_COMPRESS); - } - - return job_ptr->total_out; -} - -uint32_t CompressionCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest) const +uint32_t HardwareCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const { uint32_t job_id = 0; qpl_job * job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); if (job_ptr == nullptr) { - LOG_WARNING(log, "doCompressData HW acquireJob fail! switch to SW compress..."); - return doCompressDataSW(source, source_size, dest); + LOG_WARNING(log, "HardwareCodecDeflate::doCompressData acquireJob fail!"); + return 0; } qpl_status status; uint32_t compressed_size = 0; @@ -156,32 +83,28 @@ uint32_t CompressionCodecDeflate::doCompressData(const char * source, uint32_t s job_ptr->next_out_ptr = reinterpret_cast(dest); job_ptr->available_in = source_size; job_ptr->level = qpl_default_level; - job_ptr->available_out = getMaxCompressedDataSize(source_size); + job_ptr->available_out = dest_size; job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; - // Compression status = qpl_execute_job(job_ptr); if (QPL_STS_OK == status) { compressed_size = job_ptr->total_out; } else - { - LOG_WARNING(log, "doCompressData HW fail! switch to SW compress ->status: '{}' ", static_cast(status)); - compressed_size = doCompressDataSW(source, source_size, dest); - } + LOG_WARNING(log, "HardwareCodecDeflate::doCompressData fail ->status: '{}' ", static_cast(status)); + DeflateJobHWPool::instance().releaseJob(job_id); return compressed_size; } -UInt32 CompressionCodecDeflate::doCompressDataReq(const char * source, UInt32 source_size, char * dest, UInt32 & req_id) +uint32_t HardwareCodecDeflate::doCompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) { uint32_t job_id = 0; - req_id = 0; qpl_job * job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); if (job_ptr == nullptr) { - LOG_WARNING(log, "doCompressDataReq HW acquireJob fail! switch to SW compress..."); - return doCompressDataSW(source, source_size, dest); + LOG_WARNING(log, "HardwareCodecDeflate::doCompressDataReq acquireJob fail!"); + return 0; } qpl_status status; @@ -190,27 +113,27 @@ UInt32 CompressionCodecDeflate::doCompressDataReq(const char * source, UInt32 so job_ptr->next_out_ptr = reinterpret_cast(dest); job_ptr->available_in = source_size; job_ptr->level = qpl_default_level; - job_ptr->available_out = getMaxCompressedDataSize(source_size); + job_ptr->available_out = dest_size; job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; - // Compression status = qpl_submit_job(job_ptr); - if (QPL_STS_OK != status) + if (QPL_STS_OK == status) { - LOG_WARNING(log, "doCompressDataReq HW fail! switch to SW compress ->status: '{}' ", static_cast(status)); - DeflateJobHWPool::instance().releaseJob(job_id); - return doCompressDataSW(source, source_size, dest); + jobCompAsyncList.push_back(job_id); + return job_id; + } + else + { + LOG_WARNING(log, "HardwareCodecDeflate::doCompressDataReq fail ->status: '{}' ", static_cast(status)); + DeflateJobHWPool::instance().releaseJob(job_id); + return 0; } - //LOG_WARNING(log, "doCompressDataReq ->job_id:{}, source_size:{}",job_id, source_size); - jobCompAsyncList.push_back(job_id); - req_id = job_id; - return 0; } -uint32_t CompressionCodecDeflate::doCompressDataFlush(uint32_t req_id) +uint32_t HardwareCodecDeflate::doCompressDataFlush(uint32_t req_id) { uint32_t compressed_size = 0; qpl_job * job_ptr = DeflateJobHWPool::instance().getJobPtr(req_id); - if(nullptr != job_ptr) + if (nullptr != job_ptr) { while (QPL_STS_BEING_PROCESSED == qpl_check_job(job_ptr)) { @@ -222,14 +145,14 @@ uint32_t CompressionCodecDeflate::doCompressDataFlush(uint32_t req_id) return compressed_size; } -void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const +uint32_t HardwareCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const { uint32_t job_id = 0; qpl_job * job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); if (job_ptr == nullptr) { - LOG_WARNING(log, "doDecompressData HW acquireJob fail! switch to SW decompress"); - return doDecompressDataSW(source, source_size, dest, uncompressed_size); + LOG_WARNING(log, "HardwareCodecDeflate::doDecompressData acquireJob fail!"); + return 0; } qpl_status status; @@ -243,49 +166,28 @@ void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t sou // Decompression status = qpl_execute_job(job_ptr); - if (status != QPL_STS_OK) + + if (status == QPL_STS_OK) { - LOG_WARNING( - log, - "doDecompressData HW fail! switch to SW decompress ->status: '{}' ,source_size: '{}' ,uncompressed_size: '{}' ", - static_cast(status), - source_size, - uncompressed_size); - doDecompressDataSW(source, source_size, dest, uncompressed_size); + DeflateJobHWPool::instance().releaseJob(job_id); + return job_ptr->total_out; } - DeflateJobHWPool::instance().releaseJob(job_id); -} - -void CompressionCodecDeflate::doDecompressDataSW(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size)const -{ - qpl_status status; - qpl_job * job_ptr = jobSWPtr; - - // Performing a decompression operation - job_ptr->op = qpl_op_decompress; - job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); - job_ptr->next_out_ptr = reinterpret_cast(dest); - job_ptr->available_in = source_size; - job_ptr->available_out = uncompressed_size; - job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; - - // Decompression - status = qpl_execute_job(job_ptr); - if (status != QPL_STS_OK) + else { - throw Exception("doDecompressDataSW cannot decompress: qpl_execute_job fail:"+ std::to_string(status), ErrorCodes::CANNOT_DECOMPRESS); + LOG_WARNING(log, "HardwareCodecDeflate::doDecompressData fail ->status: '{}' ", static_cast(status)); + DeflateJobHWPool::instance().releaseJob(job_id); + return 0; } } -void CompressionCodecDeflate::doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) +uint32_t HardwareCodecDeflate::doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) { uint32_t job_id = 0; qpl_job * job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); if (job_ptr == nullptr) { - LOG_WARNING(log, "doDecompressDataReq acquireJob fail! switch to SW decompress"); - doDecompressDataSW(source, source_size, dest, uncompressed_size); - return; + LOG_WARNING(log, "HardwareCodecDeflate::doDecompressDataReq acquireJob fail!"); + return 0; } qpl_status status; @@ -302,16 +204,17 @@ void CompressionCodecDeflate::doDecompressDataReq(const char * source, uint32_t if (QPL_STS_OK == status) { jobDecompAsyncMap.insert(std::make_pair(job_id, job_ptr)); + return job_id; } else { DeflateJobHWPool::instance().releaseJob(job_id); - LOG_WARNING(log, "doDecompressDataReq HW fail! switch to SW decompress... ->status: '{}' ", static_cast(status)); - doDecompressDataSW(source, source_size, dest, uncompressed_size); + LOG_WARNING(log, "HardwareCodecDeflate::doDecompressDataReq fail ->status: '{}' ", static_cast(status)); + return 0; } } -void CompressionCodecDeflate::doDecompressDataFlush() +void HardwareCodecDeflate::doDecompressDataFlush() { uint32_t job_id = 0; qpl_job * job_ptr = nullptr; @@ -335,6 +238,8 @@ void CompressionCodecDeflate::doDecompressDataFlush() DeflateJobHWPool::instance().releaseJob(job_id); it = jobDecompAsyncMap.erase(it); n_jobs_processing--; + if (n_jobs_processing <= 0) + break; } if (it == jobDecompAsyncMap.end()) { @@ -343,11 +248,187 @@ void CompressionCodecDeflate::doDecompressDataFlush() } } } +//SoftwareCodecDeflate +SoftwareCodecDeflate::SoftwareCodecDeflate() +{ + jobSWPtr = nullptr; +} +SoftwareCodecDeflate::~SoftwareCodecDeflate() +{ + if (nullptr != jobSWPtr) + qpl_fini_job(jobSWPtr); +} + +qpl_job * SoftwareCodecDeflate::getJobCodecPtr() +{ + if (nullptr == jobSWPtr) + { + qpl_status status; + uint32_t size = 0; + // Job initialization + status = qpl_get_job_size(qpl_path_software, &size); + if (status != QPL_STS_OK) + { + throw Exception( + "SoftwareCodecDeflate::getJobCodecPtr -> qpl_get_job_size fail:" + std::to_string(status), ErrorCodes::CANNOT_COMPRESS); + } + + jobSWbuffer = std::make_unique(size); + jobSWPtr = reinterpret_cast(jobSWbuffer.get()); + + status = qpl_init_job(qpl_path_software, jobSWPtr); + if (status != QPL_STS_OK) + { + throw Exception( + "SoftwareCodecDeflate::getJobCodecPtr -> qpl_init_job fail:" + std::to_string(status), ErrorCodes::CANNOT_COMPRESS); + } + } + return jobSWPtr; +} + +uint32_t SoftwareCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) +{ + qpl_status status; + qpl_job * job_ptr = getJobCodecPtr(); + // Performing a compression operation + job_ptr->op = qpl_op_compress; + job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); + job_ptr->next_out_ptr = reinterpret_cast(dest); + job_ptr->available_in = source_size; + job_ptr->available_out = dest_size; + job_ptr->level = qpl_default_level; + job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; + + // Compression + status = qpl_execute_job(job_ptr); + if (status != QPL_STS_OK) + { + throw Exception( + "SoftwareCodecDeflate::doCompressData -> qpl_execute_job fail:" + std::to_string(status), ErrorCodes::CANNOT_COMPRESS); + } + + return job_ptr->total_out; +} + +void SoftwareCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) +{ + qpl_status status; + qpl_job * job_ptr = getJobCodecPtr(); + + // Performing a decompression operation + job_ptr->op = qpl_op_decompress; + job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); + job_ptr->next_out_ptr = reinterpret_cast(dest); + job_ptr->available_in = source_size; + job_ptr->available_out = uncompressed_size; + job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; + + // Decompression + status = qpl_execute_job(job_ptr); + if (status != QPL_STS_OK) + { + throw Exception( + "SoftwareCodecDeflate::doDecompressData -> qpl_execute_job fail:" + std::to_string(status), ErrorCodes::CANNOT_DECOMPRESS); + } +} + +//CompressionCodecDeflate +CompressionCodecDeflate::CompressionCodecDeflate() +{ + setCodecDescription("DEFLATE"); + hwCodec = std::make_unique(); + swCodec = std::make_unique(); +} + +uint8_t CompressionCodecDeflate::getMethodByte() const +{ + return static_cast(CompressionMethodByte::Deflate); +} + +bool CompressionCodecDeflate::isAsyncSupported() const +{ + return hwCodec->hwEnabled; +} + +void CompressionCodecDeflate::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); +} + +#define DEFLATE_COMPRESSBOUND(isize) ((isize) + ((isize) >> 12) + ((isize) >> 14) + ((isize) >> 25) + 13) //Aligned with ZLIB +uint32_t CompressionCodecDeflate::getMaxCompressedDataSize(uint32_t uncompressed_size) const +{ + return DEFLATE_COMPRESSBOUND(uncompressed_size); +} + +uint32_t CompressionCodecDeflate::doCompressDataSW(const char * source, uint32_t source_size, char * dest) const +{ + return swCodec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); +} + +uint32_t CompressionCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest) const +{ + uint32_t res = 0; + if (hwCodec->hwEnabled) + res = hwCodec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); + if (0 == res) + res = swCodec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); + return res; +} + +uint32_t CompressionCodecDeflate::doCompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t & req_id) +{ + if (hwCodec->hwEnabled) + req_id = hwCodec->doCompressDataReq(source, source_size, dest, getMaxCompressedDataSize(source_size)); + else + req_id = 0; + + if (0 == req_id) + return swCodec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); + else + return 0; +} + +uint32_t CompressionCodecDeflate::doCompressDataFlush(uint32_t req_id) +{ + if (hwCodec->hwEnabled) + return hwCodec->doCompressDataFlush(req_id); + else + return 0; +} + +void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const +{ + uint32_t res = 0; + if (hwCodec->hwEnabled) + res = hwCodec->doDecompressData(source, source_size, dest, uncompressed_size); + if (0 == res) + swCodec->doDecompressData(source, source_size, dest, uncompressed_size); +} + +void CompressionCodecDeflate::doDecompressDataSW(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const +{ + return swCodec->doDecompressData(source, source_size, dest, uncompressed_size); +} + +void CompressionCodecDeflate::doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) +{ + uint32_t res = 0; + if (hwCodec->hwEnabled) + res = hwCodec->doDecompressDataReq(source, source_size, dest, uncompressed_size); + if (0 == res) + swCodec->doDecompressData(source, source_size, dest, uncompressed_size); +} + +void CompressionCodecDeflate::doDecompressDataFlush() +{ + if (hwCodec->hwEnabled) + hwCodec->doDecompressDataFlush(); +} void registerCodecDeflate(CompressionCodecFactory & factory) { factory.registerSimpleCompressionCodec( "DEFLATE", static_cast(CompressionMethodByte::Deflate), [&]() { return std::make_shared(); }); } - } diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index 76de3dd0d3f..d3fb06ab10b 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -26,10 +26,15 @@ public: static constexpr qpl_path_t PATH = qpl_path_hardware; static qpl_job * jobPool[jobPoolSize]; static std::atomic_bool jobLock[jobPoolSize]; + bool jobPoolEnabled; + bool ALWAYS_INLINE jobPoolReady() + { + return jobPoolEnabled; + } qpl_job * ALWAYS_INLINE acquireJob(uint32_t * job_id) { - if(jobPoolAvailable) + if (jobPoolEnabled) { uint32_t retry = 0; auto index = random(jobPoolSize); @@ -52,7 +57,7 @@ public: } qpl_job * ALWAYS_INLINE releaseJob(uint32_t job_id) { - if(jobPoolAvailable) + if (jobPoolEnabled) { uint32_t index = jobPoolSize - job_id; ReleaseJobObjectGuard _(index); @@ -65,7 +70,7 @@ public: } qpl_job * ALWAYS_INLINE getJobPtr(uint32_t job_id) { - if(jobPoolAvailable) + if (jobPoolEnabled) { uint32_t index = jobPoolSize - job_id; return jobPool[index]; @@ -77,8 +82,6 @@ public: } private: - bool jobPoolAvailable; - Poco::Logger * log; size_t ALWAYS_INLINE random(uint32_t pool_size) { size_t tsc = 0; @@ -177,13 +180,45 @@ private: jobLock[index].store(false); } }; + Poco::Logger * log; +}; +class SoftwareCodecDeflate +{ +public: + SoftwareCodecDeflate(); + ~SoftwareCodecDeflate(); + uint32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size); + void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); + +private: + qpl_job * jobSWPtr; //Software Job Codec Ptr + std::unique_ptr jobSWbuffer; + qpl_job * getJobCodecPtr(); }; +class HardwareCodecDeflate +{ +public: + bool hwEnabled; + HardwareCodecDeflate(); + ~HardwareCodecDeflate(); + uint32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const; + uint32_t doCompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t dest_size); + uint32_t doCompressDataFlush(uint32_t req_id); + uint32_t doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const; + uint32_t doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); + void doDecompressDataFlush(); + +private: + std::map jobDecompAsyncMap; + std::vector jobCompAsyncList; + Poco::Logger * log; +}; class CompressionCodecDeflate : public ICompressionCodec { public: CompressionCodecDeflate(); - ~CompressionCodecDeflate() override; + //~CompressionCodecDeflate() ; uint8_t getMethodByte() const override; void updateHash(SipHash & hash) const override; bool isAsyncSupported() const override; @@ -197,10 +232,9 @@ protected: { return true; } - qpl_job * initSoftwareJobCodecPtr(); uint32_t doCompressData(const char * source, uint32_t source_size, char * dest) const override; uint32_t doCompressDataSW(const char * source, uint32_t source_size, char * dest) const; - UInt32 doCompressDataReq(const char * source, UInt32 source_size, char * dest, uint32_t & req_id) override; + uint32_t doCompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t & req_id) override; uint32_t doCompressDataFlush(uint32_t req_id) override; void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; @@ -210,10 +244,8 @@ protected: private: uint32_t getMaxCompressedDataSize(uint32_t uncompressed_size) const override; - std::map jobDecompAsyncMap; - std::vector jobCompAsyncList; - Poco::Logger * log; - qpl_job * jobSWPtr; //Software Job Codec Ptr + std::unique_ptr hwCodec; + std::unique_ptr swCodec; }; } From b537e91487c6b4f1eeed87a67d538e88c8173853 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Jun 2022 12:18:45 +0000 Subject: [PATCH 037/659] parse additional filetrs. --- src/Interpreters/InterpreterSelectQuery.cpp | 53 +++++++++++++++++++++ src/Parsers/ASTIdentifier.h | 2 +- 2 files changed, 54 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 94ac7c26183..e7158ec4070 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -266,6 +266,59 @@ static void checkAccessRightsForSelect( context->checkAccess(AccessType::SELECT, table_id, syntax_analyzer_result.requiredSourceColumnsForAccessCheck()); } +static ASTPtr parseAdditionalFilterConditionForTable( + const char * data, size_t size, + const DatabaseAndTableWithAlias & target, + const Context & context) +{ + size_t pos = 0; + for (; pos < size; ++pos) + if (data[pos] == ':') + break; + + if (pos == size) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "No table is specified for additional filter {}. Expected syntax: 'table:condition'", + std::string_view(data, size)); + + std::string_view table(data, pos); + if ((target.database == context.getCurrentDatabase() && target.table == table) || + (target.database + '.' + target.table == table)) + { + /// Try to parse expression + ParserExpression parser; + const auto & settings = context.getSettingsRef(); + return parseQuery(parser, data + pos, data + size, "additional filter", settings.max_query_size, settings.max_parser_depth); + } + + return nullptr; +} + +static ASTPtr parseAdditionalFilterConditionForTable( + const std::string & setting, + const DatabaseAndTableWithAlias & target, + const Context & context) +{ + if (setting.empty()) + return nullptr; + + const char delimiter = ';'; + + const auto size = setting.size(); + const char * data = setting.data(); + const char * end = data + setting.size(); + for (const char * pos = data; pos < end; ++pos) + { + if (setting[pos] == delimiter) + { + if (auto ast = parseAdditionalFilterConditionForTable(data, pos, target, context)) + return ast; + + data = + } + } +} + /// Returns true if we should ignore quotas and limits for a specified table in the system database. static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) { diff --git a/src/Parsers/ASTIdentifier.h b/src/Parsers/ASTIdentifier.h index 14e2fcef39d..846a9266fe9 100644 --- a/src/Parsers/ASTIdentifier.h +++ b/src/Parsers/ASTIdentifier.h @@ -68,7 +68,7 @@ private: void resetFullName(); }; -class ASTTableIdentifier : public ASTIdentifier +class ZASTTableIdentifier : public ASTIdentifier { public: explicit ASTTableIdentifier(const String & table_name, std::vector && name_params = {}); From 543782436dcf51a2dc03a0341a6b5e75a0c3668c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 21 Jun 2022 11:24:46 +0000 Subject: [PATCH 038/659] Process additional filters. --- src/Core/Settings.h | 2 + src/Interpreters/InterpreterSelectQuery.cpp | 118 ++++++++++++++------ src/Interpreters/InterpreterSelectQuery.h | 5 +- src/Parsers/ASTIdentifier.h | 2 +- src/Storages/SelectQueryInfo.h | 4 + 5 files changed, 95 insertions(+), 36 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ec24eb871b3..e24c69ef287 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -587,6 +587,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) \ M(Bool, allow_unrestricted_reads_from_keeper, false, "Allow unrestricted (without condition on path) reads from system.zookeeper table, can be handy, but is not safe for zookeeper", 0) \ \ + M(String, additional_filters, "", "Additional filter expression which would be applied after reading from specified table. Syntax : 'table1:expression1;database.table2:expression2'", 0) \ + \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e7158ec4070..913028afcd2 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -108,8 +108,17 @@ namespace ErrorCodes } /// Assumes `storage` is set and the table filter (row-level security) is not empty. -String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, const Names & prerequisite_columns) const +FilterDAGInfoPtr generateFilterActions( + const StorageID & table_id, + const ASTPtr & row_policy_filter, + const ContextPtr & context, + const StoragePtr & storage, + const StorageSnapshotPtr & storage_snapshot, + const StorageMetadataPtr & metadata_snapshot, + Names & prerequisite_columns) { + auto filter_info = std::make_shared(); + const auto & db_name = table_id.getDatabaseName(); const auto & table_name = table_id.getTableName(); @@ -145,16 +154,24 @@ String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, co /// Using separate expression analyzer to prevent any possible alias injection auto syntax_result = TreeRewriter(context).analyzeSelect(query_ast, TreeRewriterResult({}, storage, storage_snapshot)); SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot); - actions = analyzer.simpleSelectActions(); + filter_info->actions = analyzer.simpleSelectActions(); - auto column_name = expr_list->children.at(0)->getColumnName(); - actions->removeUnusedActions(NameSet{column_name}); - actions->projectInput(false); + filter_info->column_name = expr_list->children.at(0)->getColumnName(); + filter_info->actions->removeUnusedActions(NameSet{filter_info->column_name}); + filter_info->actions->projectInput(false); - for (const auto * node : actions->getInputs()) - actions->getIndex().push_back(node); + for (const auto * node : filter_info->actions->getInputs()) + filter_info->actions->getIndex().push_back(node); - return column_name; + auto required_columns_from_filter = filter_info->actions->getRequiredColumns(); + + for (const auto & column : required_columns_from_filter) + { + if (prerequisite_columns.end() == std::find(prerequisite_columns.begin(), prerequisite_columns.end(), column.name)) + prerequisite_columns.push_back(column.name); + } + + return filter_info; } InterpreterSelectQuery::InterpreterSelectQuery( @@ -266,29 +283,37 @@ static void checkAccessRightsForSelect( context->checkAccess(AccessType::SELECT, table_id, syntax_analyzer_result.requiredSourceColumnsForAccessCheck()); } +/// Parse additional filter for table in format 'table.name:expression' +/// Examples: 'default.hits:UserID=12345', 'visits:UserID = 0 ? 1 : 0' static ASTPtr parseAdditionalFilterConditionForTable( - const char * data, size_t size, + const char * start, + const char * end, const DatabaseAndTableWithAlias & target, const Context & context) { - size_t pos = 0; - for (; pos < size; ++pos) - if (data[pos] == ':') + const char delimiter = ':'; + + const char * pos = start; + for (; pos < end; ++pos) + if (*pos == delimiter) break; - if (pos == size) + std::string_view table(start, pos - start); + + if (pos == end) throw Exception(ErrorCodes::BAD_ARGUMENTS, "No table is specified for additional filter {}. Expected syntax: 'table:condition'", - std::string_view(data, size)); + table); - std::string_view table(data, pos); - if ((target.database == context.getCurrentDatabase() && target.table == table) || - (target.database + '.' + target.table == table)) + trim(table); + + if ((table == target.table && context.getCurrentDatabase() == target.database) || + (table == target.database + '.' + target.table)) { /// Try to parse expression ParserExpression parser; const auto & settings = context.getSettingsRef(); - return parseQuery(parser, data + pos, data + size, "additional filter", settings.max_query_size, settings.max_parser_depth); + return parseQuery(parser, pos + 1, end, "additional filter", settings.max_query_size, settings.max_parser_depth); } return nullptr; @@ -304,19 +329,23 @@ static ASTPtr parseAdditionalFilterConditionForTable( const char delimiter = ';'; - const auto size = setting.size(); - const char * data = setting.data(); - const char * end = data + setting.size(); - for (const char * pos = data; pos < end; ++pos) + const char * start = setting.data(); + const char * end = start + setting.size(); + for (const char * pos = start; pos < end; ++pos) { - if (setting[pos] == delimiter) + if (*pos == delimiter) { - if (auto ast = parseAdditionalFilterConditionForTable(data, pos, target, context)) + if (auto ast = parseAdditionalFilterConditionForTable(start, pos, target, context)) return ast; - data = + start = pos + 1; } } + + if (start < end) + return parseAdditionalFilterConditionForTable(start, end, target, context); + + return nullptr; } /// Returns true if we should ignore quotas and limits for a specified table in the system database. @@ -498,6 +527,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (storage) view = dynamic_cast(storage.get()); + if (!settings.additional_filters.value.empty() && storage && !joined_tables.tablesWithColumns().empty()) + query_info.additional_filter_ast = parseAdditionalFilterConditionForTable( + settings.additional_filters, joined_tables.tablesWithColumns().front().table, *context); + auto analyze = [&] (bool try_move_to_prewhere) { /// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it. @@ -616,16 +649,16 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Fix source_header for filter actions. if (row_policy_filter) { - filter_info = std::make_shared(); - filter_info->column_name = generateFilterActions(filter_info->actions, required_columns); + filter_info = generateFilterActions( + table_id, row_policy_filter, context, storage, storage_snapshot, metadata_snapshot, required_columns); + } - auto required_columns_from_filter = filter_info->actions->getRequiredColumns(); + if (query_info.additional_filter_ast) + { + additional_filter_info = generateFilterActions( + table_id, query_info.additional_filter_ast, context, storage, storage_snapshot, metadata_snapshot, required_columns); - for (const auto & column : required_columns_from_filter) - { - if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column.name)) - required_columns.push_back(column.name); - } + additional_filter_info->do_remove_column = true; } source_header = storage_snapshot->getSampleBlockForColumns(required_columns); @@ -1339,6 +1372,18 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( + query_plan.getCurrentDataStream(), + additional_filter_info->actions, + additional_filter_info->column_name, + additional_filter_info->do_remove_column); + + additional_filter_step->setStepDescription("Additional filter"); + query_plan.addStep(std::move(additional_filter_step)); + } + if (expressions.before_array_join) { QueryPlanStepPtr before_array_join_step @@ -1950,6 +1995,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc && storage && storage->getName() != "MaterializedMySQL" && !row_policy_filter + && !query_info.additional_filter_ast && processing_stage == QueryProcessingStage::FetchColumns && query_analyzer->hasAggregation() && (query_analyzer->aggregates().size() == 1) @@ -2048,6 +2094,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc && !query.limit_with_ties && !query.prewhere() && !query.where() + && !query_info.additional_filter_ast && !query.groupBy() && !query.having() && !query.orderBy() @@ -2207,6 +2254,11 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc table_aliases->setStepDescription("Add table aliases"); query_plan.addStep(std::move(table_aliases)); } + + if (processing_stage == QueryProcessingStage::FetchColumns && query_info.additional_filter_ast) + { + + } } void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter) diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 40afaaaeed0..9992c8a54e8 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -186,8 +186,6 @@ private: void executeMergeSorted(QueryPlan & query_plan, const SortDescription & sort_description, UInt64 limit, const std::string & description); - String generateFilterActions(ActionsDAGPtr & actions, const Names & prerequisite_columns = {}) const; - enum class Modificator { ROLLUP = 0, @@ -214,6 +212,9 @@ private: ASTPtr row_policy_filter; FilterDAGInfoPtr filter_info; + /// For additional_filter setting. + FilterDAGInfoPtr additional_filter_info; + QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; /// List of columns to read to execute the query. diff --git a/src/Parsers/ASTIdentifier.h b/src/Parsers/ASTIdentifier.h index 846a9266fe9..14e2fcef39d 100644 --- a/src/Parsers/ASTIdentifier.h +++ b/src/Parsers/ASTIdentifier.h @@ -68,7 +68,7 @@ private: void resetFullName(); }; -class ZASTTableIdentifier : public ASTIdentifier +class ASTTableIdentifier : public ASTIdentifier { public: explicit ASTTableIdentifier(const String & table_name, std::vector && name_params = {}); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index bdb4c392c48..5046a0b6fe0 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -156,6 +156,10 @@ struct SelectQueryInfoBase PrewhereInfoPtr prewhere_info; + /// This is an additional filer applied to current table. + /// It is needed only for additional PK filtering. + ASTPtr additional_filter_ast; + ReadInOrderOptimizerPtr order_optimizer; /// Can be modified while reading from storage InputOrderInfoPtr input_order_info; From 1a23dfa827ad378aeebe1e53c516f165e36d129d Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Tue, 21 Jun 2022 06:50:02 -0700 Subject: [PATCH 039/659] Update thrift to 0.13.0 --- contrib/thrift | 2 +- contrib/thrift-cmake/CMakeLists.txt | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/contrib/thrift b/contrib/thrift index 010ccf0a0c7..cecee50308f 160000 --- a/contrib/thrift +++ b/contrib/thrift @@ -1 +1 @@ -Subproject commit 010ccf0a0c7023fea0f6bf4e4078ebdff7e61982 +Subproject commit cecee50308fc7e6f77f55b3fd906c1c6c471fa2f diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index 6f94c1ebdc0..33c15a700be 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -15,7 +15,6 @@ set(thriftcpp_SOURCES "${LIBRARY_DIR}/src/thrift/async/TConcurrentClientSyncInfo.cpp" "${LIBRARY_DIR}/src/thrift/concurrency/ThreadManager.cpp" "${LIBRARY_DIR}/src/thrift/concurrency/TimerManager.cpp" - "${LIBRARY_DIR}/src/thrift/concurrency/Util.cpp" "${LIBRARY_DIR}/src/thrift/processor/PeekProcessor.cpp" "${LIBRARY_DIR}/src/thrift/protocol/TBase64Utils.cpp" "${LIBRARY_DIR}/src/thrift/protocol/TDebugProtocol.cpp" From 4deddbf9005546a54e8c69af7d54de4a057c825e Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Tue, 21 Jun 2022 12:28:40 -0700 Subject: [PATCH 040/659] Update thrift to 0.16.0(latest) --- contrib/thrift | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/thrift b/contrib/thrift index cecee50308f..2a93df80f27 160000 --- a/contrib/thrift +++ b/contrib/thrift @@ -1 +1 @@ -Subproject commit cecee50308fc7e6f77f55b3fd906c1c6c471fa2f +Subproject commit 2a93df80f27739ccabb5b885cb12a8dc7595ecdf From 45f94c3d2392135c549357f56281b9dc51063913 Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Thu, 23 Jun 2022 14:01:45 -0700 Subject: [PATCH 041/659] Add new files to makefile for thrift 0.16.0 --- contrib/thrift-cmake/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index 33c15a700be..e4ed25b27b8 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -32,6 +32,8 @@ set(thriftcpp_SOURCES "${LIBRARY_DIR}/src/thrift/transport/TServerSocket.cpp" "${LIBRARY_DIR}/src/thrift/transport/TTransportUtils.cpp" "${LIBRARY_DIR}/src/thrift/transport/TBufferTransports.cpp" + "${LIBRARY_DIR}/src/thrift/transport/SocketCommon.cpp" + "${LIBRARY_DIR}/src/thrift/transport/TWebSocketServer.cpp" "${LIBRARY_DIR}/src/thrift/server/TConnectedClient.cpp" "${LIBRARY_DIR}/src/thrift/server/TServerFramework.cpp" "${LIBRARY_DIR}/src/thrift/server/TSimpleServer.cpp" From 1d5ba77c876e1992ae81eddc9686a4278a63e078 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 27 Jun 2022 17:41:55 +0000 Subject: [PATCH 042/659] Fix test. --- src/Interpreters/ExpressionAnalyzer.cpp | 8 + src/Interpreters/ExpressionAnalyzer.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 6 +- .../02346_additional_filters.reference | 219 ++++++++++++++++++ .../0_stateless/02346_additional_filters.sql | 76 ++++++ 5 files changed, 309 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02346_additional_filters.reference create mode 100644 tests/queries/0_stateless/02346_additional_filters.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 00333503db1..29990be9e0f 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1707,6 +1707,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( bool second_stage_, bool only_types, const FilterDAGInfoPtr & filter_info_, + const FilterDAGInfoPtr & additional_filter, const Block & source_header) : first_stage(first_stage_) , second_stage(second_stage_) @@ -1758,6 +1759,13 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( columns_for_final.begin(), columns_for_final.end()); } + if (storage && additional_filter) + { + Names columns_for_additional_filter = additional_filter->actions->getRequiredColumnsNames(); + additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), + columns_for_additional_filter.begin(), columns_for_additional_filter.end()); + } + if (storage && filter_info_) { filter_info = filter_info_; diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 80c664832e5..2817843ce17 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -278,6 +278,7 @@ struct ExpressionAnalysisResult bool second_stage, bool only_types, const FilterDAGInfoPtr & filter_info, + const FilterDAGInfoPtr & additional_filter, /// for setting additional_filters const Block & source_header); /// Filter for row-level security. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 913028afcd2..f69de5dfbe0 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -167,10 +167,14 @@ FilterDAGInfoPtr generateFilterActions( for (const auto & column : required_columns_from_filter) { + std::cerr << "========= required : " << column.name << std::endl; if (prerequisite_columns.end() == std::find(prerequisite_columns.begin(), prerequisite_columns.end(), column.name)) prerequisite_columns.push_back(column.name); } + for (auto & col : prerequisite_columns) + std::cerr << "====== col " << col << std::endl; + return filter_info; } @@ -818,7 +822,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() && options.to_stage > QueryProcessingStage::WithMergeableState; analysis_result = ExpressionAnalysisResult( - *query_analyzer, metadata_snapshot, first_stage, second_stage, options.only_analyze, filter_info, source_header); + *query_analyzer, metadata_snapshot, first_stage, second_stage, options.only_analyze, filter_info, additional_filter_info, source_header); if (options.to_stage == QueryProcessingStage::Enum::FetchColumns) { diff --git a/tests/queries/0_stateless/02346_additional_filters.reference b/tests/queries/0_stateless/02346_additional_filters.reference new file mode 100644 index 00000000000..7b74a7a9488 --- /dev/null +++ b/tests/queries/0_stateless/02346_additional_filters.reference @@ -0,0 +1,219 @@ +-- { echoOn } + +select * from table_1; +1 aa +2 bb +3 ccc +4 dddd +select * from table_1 settings additional_filters='table_1 : x != 2'; +1 aa +3 ccc +4 dddd +select * from table_1 settings additional_filters='table_1 : x != 2 and x != 3'; +1 aa +4 dddd +select x from table_1 settings additional_filters='table_1 : x != 2'; +1 +3 +4 +select y from table_1 settings additional_filters='table_1 : x != 2'; +aa +ccc +dddd +select * from table_1 where x != 3 settings additional_filters='table_1 : x != 2'; +1 aa +4 dddd +select * from table_1 prewhere x != 4 settings additional_filters='table_1 : x != 2'; +1 aa +3 ccc +select * from table_1 prewhere x != 4 where x != 3 settings additional_filters='table_1 : x != 2'; +1 aa +select x from table_1 where x != 3 settings additional_filters='table_1 : x != 2'; +1 +4 +select x from table_1 prewhere x != 4 settings additional_filters='table_1 : x != 2'; +1 +3 +select x from table_1 prewhere x != 4 where x != 3 settings additional_filters='table_1 : x != 2'; +1 +select y from table_1 where x != 3 settings additional_filters='table_1 : x != 2'; +aa +dddd +select y from table_1 prewhere x != 4 settings additional_filters='table_1 : x != 2'; +aa +ccc +select y from table_1 prewhere x != 4 where x != 3 settings additional_filters='table_1 : x != 2'; +aa +select x from table_1 where x != 2 settings additional_filters='table_1 : x != 2'; +1 +3 +4 +select x from table_1 prewhere x != 2 settings additional_filters='table_1 : x != 2'; +1 +3 +4 +select x from table_1 prewhere x != 2 where x != 2 settings additional_filters='table_1 : x != 2'; +1 +3 +4 +select * from system.numbers limit 5; +0 +1 +2 +3 +4 +select * from system.numbers limit 5 settings additional_filters='system.numbers : number != 3'; +0 +1 +2 +4 +5 +select * from system.numbers limit 5 settings additional_filters='system.numbers:number != 3;table_1:x!=2'; +0 +1 +2 +4 +5 +select * from (select number from system.numbers limit 5 union all select x from table_1) order by number settings additional_filters='system.numbers:number != 3;table_1:x!=2'; +0 +1 +1 +2 +3 +4 +4 +5 +select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x settings additional_filters='system.numbers : number != 3; table_1 : x != 2'; +0 0 +1 1 aa +2 0 +4 4 dddd +5 0 +select b + 1 as c from (select a + 1 as b from (select x + 1 as a from table_1)) settings additional_filters='table_1 : x != 2 and x != 3'; +4 +7 +-- { echoOn } +select * from v_numbers; +1 +2 +3 +4 +5 +select * from v_numbers settings additional_filters='system.numbers : number != 3'; +1 +2 +3 +5 +6 +select * from v_numbers settings additional_filters='v_numbers : x != 3'; +1 +2 +4 +5 +select * from v_numbers settings additional_filters='system.numbers : number != 3; v_numbers : x != 3'; +1 +2 +5 +6 +-- { echoOn } +-- additional filter for inner tables for Materialized View does not work because it does not create internal interpreter +-- probably it is expected +select * from mv_table; +4 dddd +5 eeeee +6 ffffff +7 ggggggg +select * from mv_table settings additional_filters='mv_table : x != 5'; +4 dddd +6 ffffff +7 ggggggg +select * from mv_table settings additional_filters='table_1 : x != 5'; +4 dddd +5 eeeee +6 ffffff +7 ggggggg +select * from mv_table settings additional_filters='table_2 : x != 5'; +4 dddd +5 eeeee +6 ffffff +7 ggggggg +-- { echoOn } +-- additional filter for inner tables for Merge does not work because it does not create internal interpreter +-- probably it is expected +select * from m_table order by x; +1 aa +2 bb +3 ccc +4 dddd +4 dddd +5 eeeee +6 ffffff +7 ggggggg +select * from m_table order by x settings additional_filters='table_1 : x != 2'; +1 aa +2 bb +3 ccc +4 dddd +4 dddd +5 eeeee +6 ffffff +7 ggggggg +select * from m_table order by x settings additional_filters='table_2 : x != 5'; +1 aa +2 bb +3 ccc +4 dddd +4 dddd +5 eeeee +6 ffffff +7 ggggggg +select * from m_table order by x settings additional_filters='table_1 : x != 2; table_2 : x != 5'; +1 aa +2 bb +3 ccc +4 dddd +4 dddd +5 eeeee +6 ffffff +7 ggggggg +select * from m_table order by x settings additional_filters='table_1 : x != 4'; +1 aa +2 bb +3 ccc +4 dddd +4 dddd +5 eeeee +6 ffffff +7 ggggggg +select * from m_table order by x settings additional_filters='table_2 : x != 4'; +1 aa +2 bb +3 ccc +4 dddd +4 dddd +5 eeeee +6 ffffff +7 ggggggg +select * from m_table order by x settings additional_filters='table_1 : x != 4; table_2 : x != 4'; +1 aa +2 bb +3 ccc +4 dddd +4 dddd +5 eeeee +6 ffffff +7 ggggggg +select * from m_table order by x settings additional_filters='m_table : x != 4'; +1 aa +2 bb +3 ccc +5 eeeee +6 ffffff +7 ggggggg +select * from m_table order by x settings additional_filters='m_table : x != 4; table_1 : x != 2; table_2 : x != 5'; +1 aa +2 bb +3 ccc +5 eeeee +6 ffffff +7 ggggggg diff --git a/tests/queries/0_stateless/02346_additional_filters.sql b/tests/queries/0_stateless/02346_additional_filters.sql new file mode 100644 index 00000000000..9c8185ff549 --- /dev/null +++ b/tests/queries/0_stateless/02346_additional_filters.sql @@ -0,0 +1,76 @@ +drop table if exists table_1; +drop table if exists table_2; +drop table if exists v_numbers; +drop table if exists mv_table; + +create table table_1 (x UInt32, y String) engine = MergeTree order by x; +insert into table_1 values (1, 'aa'), (2, 'bb'), (3, 'ccc'), (4, 'dddd'); + +-- { echoOn } + +select * from table_1; +select * from table_1 settings additional_filters='table_1 : x != 2'; +select * from table_1 settings additional_filters='table_1 : x != 2 and x != 3'; +select x from table_1 settings additional_filters='table_1 : x != 2'; +select y from table_1 settings additional_filters='table_1 : x != 2'; +select * from table_1 where x != 3 settings additional_filters='table_1 : x != 2'; +select * from table_1 prewhere x != 4 settings additional_filters='table_1 : x != 2'; +select * from table_1 prewhere x != 4 where x != 3 settings additional_filters='table_1 : x != 2'; +select x from table_1 where x != 3 settings additional_filters='table_1 : x != 2'; +select x from table_1 prewhere x != 4 settings additional_filters='table_1 : x != 2'; +select x from table_1 prewhere x != 4 where x != 3 settings additional_filters='table_1 : x != 2'; +select y from table_1 where x != 3 settings additional_filters='table_1 : x != 2'; +select y from table_1 prewhere x != 4 settings additional_filters='table_1 : x != 2'; +select y from table_1 prewhere x != 4 where x != 3 settings additional_filters='table_1 : x != 2'; +select x from table_1 where x != 2 settings additional_filters='table_1 : x != 2'; +select x from table_1 prewhere x != 2 settings additional_filters='table_1 : x != 2'; +select x from table_1 prewhere x != 2 where x != 2 settings additional_filters='table_1 : x != 2'; + +select * from system.numbers limit 5; +select * from system.numbers limit 5 settings additional_filters='system.numbers : number != 3'; +select * from system.numbers limit 5 settings additional_filters='system.numbers:number != 3;table_1:x!=2'; +select * from (select number from system.numbers limit 5 union all select x from table_1) order by number settings additional_filters='system.numbers:number != 3;table_1:x!=2'; +select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x settings additional_filters='system.numbers : number != 3; table_1 : x != 2'; +select b + 1 as c from (select a + 1 as b from (select x + 1 as a from table_1)) settings additional_filters='table_1 : x != 2 and x != 3'; + +-- { echoOff } + +create view v_numbers as select number + 1 as x from system.numbers limit 5; + +-- { echoOn } +select * from v_numbers; +select * from v_numbers settings additional_filters='system.numbers : number != 3'; +select * from v_numbers settings additional_filters='v_numbers : x != 3'; +select * from v_numbers settings additional_filters='system.numbers : number != 3; v_numbers : x != 3'; + +-- { echoOff } + +create table table_2 (x UInt32, y String) engine = MergeTree order by x; +insert into table_2 values (4, 'dddd'), (5, 'eeeee'), (6, 'ffffff'), (7, 'ggggggg'); + +create materialized view mv_table to table_2 (x UInt32, y String) as select * from table_1; + +-- { echoOn } +-- additional filter for inner tables for Materialized View does not work because it does not create internal interpreter +-- probably it is expected +select * from mv_table; +select * from mv_table settings additional_filters='mv_table : x != 5'; +select * from mv_table settings additional_filters='table_1 : x != 5'; +select * from mv_table settings additional_filters='table_2 : x != 5'; + +-- { echoOff } + +create table m_table (x UInt32, y String) engine = Merge(currentDatabase(), '^table_'); + +-- { echoOn } +-- additional filter for inner tables for Merge does not work because it does not create internal interpreter +-- probably it is expected +select * from m_table order by x; +select * from m_table order by x settings additional_filters='table_1 : x != 2'; +select * from m_table order by x settings additional_filters='table_2 : x != 5'; +select * from m_table order by x settings additional_filters='table_1 : x != 2; table_2 : x != 5'; +select * from m_table order by x settings additional_filters='table_1 : x != 4'; +select * from m_table order by x settings additional_filters='table_2 : x != 4'; +select * from m_table order by x settings additional_filters='table_1 : x != 4; table_2 : x != 4'; +select * from m_table order by x settings additional_filters='m_table : x != 4'; +select * from m_table order by x settings additional_filters='m_table : x != 4; table_1 : x != 2; table_2 : x != 5'; From 79cf4c4f7e32a4396e89fc515a175ef236b17c81 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 27 Jun 2022 17:44:30 +0000 Subject: [PATCH 043/659] Remove commented code. --- src/Interpreters/InterpreterSelectQuery.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f69de5dfbe0..c5a38593ef5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -167,14 +167,10 @@ FilterDAGInfoPtr generateFilterActions( for (const auto & column : required_columns_from_filter) { - std::cerr << "========= required : " << column.name << std::endl; if (prerequisite_columns.end() == std::find(prerequisite_columns.begin(), prerequisite_columns.end(), column.name)) prerequisite_columns.push_back(column.name); } - for (auto & col : prerequisite_columns) - std::cerr << "====== col " << col << std::endl; - return filter_info; } From 99436ab1dad935c04781fdf85ab3ef8e6a8c9cad Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 27 Jun 2022 17:46:09 +0000 Subject: [PATCH 044/659] Remove unneeded code. --- src/Interpreters/InterpreterSelectQuery.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c5a38593ef5..9d28fdfcfd6 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2254,11 +2254,6 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc table_aliases->setStepDescription("Add table aliases"); query_plan.addStep(std::move(table_aliases)); } - - if (processing_stage == QueryProcessingStage::FetchColumns && query_info.additional_filter_ast) - { - - } } void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter) From 6824570ea3875683beac3392c0095185d99cebde Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 27 Jun 2022 18:10:29 +0000 Subject: [PATCH 045/659] Fix style. --- src/Interpreters/InterpreterSelectQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 9d28fdfcfd6..90ad364256f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -105,6 +105,7 @@ namespace ErrorCodes extern const int INVALID_WITH_FILL_EXPRESSION; extern const int ACCESS_DENIED; extern const int UNKNOWN_IDENTIFIER; + extern const int BAD_ARGUMENTS; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. From 1d15d72211f19273793dea91d1c750333f7bf366 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 27 Jun 2022 18:42:26 +0000 Subject: [PATCH 046/659] Support NULLs in ROLLUP --- src/Core/Settings.h | 2 + src/DataTypes/IDataType.h | 6 ++ src/Interpreters/ExpressionAnalyzer.cpp | 40 +++++++++++++- src/Interpreters/ExpressionAnalyzer.h | 3 + src/Interpreters/InterpreterSelectQuery.cpp | 1 + src/Processors/QueryPlan/AggregatingStep.cpp | 55 +++++++++++++++---- src/Processors/QueryPlan/AggregatingStep.h | 3 + src/Processors/QueryPlan/RollupStep.cpp | 2 +- src/Processors/Transforms/RollupTransform.cpp | 31 ++++++++--- src/Processors/Transforms/RollupTransform.h | 7 ++- .../02343_group_by_use_nulls.reference | 21 +++++++ .../0_stateless/02343_group_by_use_nulls.sql | 4 ++ 12 files changed, 152 insertions(+), 23 deletions(-) create mode 100644 tests/queries/0_stateless/02343_group_by_use_nulls.reference create mode 100644 tests/queries/0_stateless/02343_group_by_use_nulls.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 756e6eb651e..5972ca4a9a3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -129,6 +129,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.", 0) \ M(Bool, enable_positional_arguments, false, "Enable positional arguments in ORDER BY, GROUP BY and LIMIT BY", 0) \ \ + M(Bool, group_by_use_nulls, false, "Treat columns mentioned in ROLLUP, CUBE or GROUPING SETS as Nullable", 0) \ + \ M(UInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \ M(UInt64, parallel_replicas_count, 0, "", 0) \ M(UInt64, parallel_replica_offset, 0, "", 0) \ diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 420ef61a13f..08c8fd74f3e 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -532,6 +532,12 @@ inline bool isBool(const DataTypePtr & data_type) return data_type->getName() == "Bool"; } +inline bool isAggregateFunction(const DataTypePtr & data_type) +{ + WhichDataType which(data_type); + return which.isAggregateFunction(); +} + template constexpr bool IsDataTypeDecimal = false; template constexpr bool IsDataTypeNumber = false; template constexpr bool IsDataTypeDateOrDateTime = false; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index cfe1167c36c..07b2a1ce1f9 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -41,8 +41,12 @@ #include +#include "Common/logger_useful.h" #include #include +#include "Columns/ColumnNullable.h" +#include "Core/ColumnsWithTypeAndName.h" +#include "DataTypes/IDataType.h" #include #include #include @@ -64,6 +68,7 @@ #include #include #include +#include namespace DB { @@ -393,7 +398,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) } } - NameAndTypePair key{column_name, node->result_type}; + NameAndTypePair key{column_name, makeNullable(node->result_type)}; grouping_set_list.push_back(key); @@ -447,7 +452,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) } } - NameAndTypePair key{column_name, node->result_type}; + NameAndTypePair key = select_query->group_by_with_rollup || select_query->group_by_with_cube ? NameAndTypePair{ column_name, makeNullable(node->result_type) } : NameAndTypePair{column_name, node->result_type}; /// Aggregation keys are uniqued. if (!unique_keys.contains(key.name)) @@ -1418,6 +1423,28 @@ void SelectQueryExpressionAnalyzer::appendExpressionsAfterWindowFunctions(Expres } } +void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAGPtr & before_aggregation, ExpressionActionsChain & chain, bool /* only_types */) +{ + const auto * select_query = getAggregatingQuery(); + + if (!select_query->groupBy() || !(select_query->group_by_with_rollup || select_query->group_by_with_cube)) + return; + + auto source_columns = before_aggregation->getResultColumns(); + ColumnsWithTypeAndName result_columns; + + for (const auto & source_column : source_columns) + { + if (isAggregateFunction(source_column.type)) + result_columns.push_back(source_column); + else + result_columns.emplace_back(makeNullable(source_column.type), source_column.name); + } + ExpressionActionsChain::Step & step = chain.lastStep(before_aggregation->getNamesAndTypesList()); + + step.actions() = ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position); +} + bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types) { const auto * select_query = getAggregatingQuery(); @@ -1597,6 +1624,8 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActio ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); + LOG_DEBUG(&Poco::Logger::get("SelectQueryExpressionAnalyzer"), "Before output: {}", step.actions()->getNamesAndTypesList().toString()); + NamesWithAliases result_columns; ASTs asts = select_query->select()->children; @@ -1638,7 +1667,11 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActio } auto actions = chain.getLastActions(); + LOG_DEBUG(&Poco::Logger::get("SelectQueryExpressionAnalyzer"), "Before projection: {}", actions->getNamesAndTypesList().toString()); + actions->project(result_columns); + LOG_DEBUG(&Poco::Logger::get("SelectQueryExpressionAnalyzer"), "After projection: {}", actions->getNamesAndTypesList().toString()); + return actions; } @@ -1862,6 +1895,9 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( query_analyzer.appendAggregateFunctionsArguments(chain, only_types || !first_stage); before_aggregation = chain.getLastActions(); + before_aggregation_with_nullable = chain.getLastActions(); + query_analyzer.appendGroupByModifiers(before_aggregation, chain, only_types); + finalize_chain(chain); if (query_analyzer.appendHaving(chain, only_types || !second_stage)) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 6c27d8c6760..7a183e865c0 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -245,6 +245,7 @@ struct ExpressionAnalysisResult JoinPtr join; ActionsDAGPtr before_where; ActionsDAGPtr before_aggregation; + ActionsDAGPtr before_aggregation_with_nullable; ActionsDAGPtr before_having; String having_column_name; bool remove_having_filter = false; @@ -410,6 +411,8 @@ private: void appendExpressionsAfterWindowFunctions(ExpressionActionsChain & chain, bool only_types); + void appendGroupByModifiers(ActionsDAGPtr & before_aggregation, ExpressionActionsChain & chain, bool only_types); + /// After aggregation: bool appendHaving(ExpressionActionsChain & chain, bool only_types); /// appendSelect diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ec7c3878b06..feae7ac6a21 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -582,6 +582,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Calculate structure of the result. result_header = getSampleBlockImpl(); + LOG_DEBUG(&Poco::Logger::get("InterpreterSelectQuery"), "Result header: {}", result_header.dumpStructure()); }; analyze(shouldMoveToPrewhere()); diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 17a0498fb7e..a0f5fce908b 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -11,10 +11,13 @@ #include #include #include +#include #include #include #include #include +#include "Core/ColumnNumbers.h" +#include "DataTypes/IDataType.h" namespace DB { @@ -46,22 +49,44 @@ Block appendGroupingSetColumn(Block header) return res; } +Block generateOutputHeader(const Block & input_header) +{ + auto header = appendGroupingSetColumn(input_header); + for (size_t i = 1; i < header.columns(); ++i) + { + auto & column = header.getByPosition(i); + + if (!isAggregateFunction(column.type)) + { + column.type = makeNullable(column.type); + column.column = makeNullable(column.column); + } + } + return header; +} + +Block generateOutputHeader(const Block & input_header, const ColumnNumbers & keys) +{ + auto header = appendGroupingSetColumn(input_header); + for (auto key : keys) + { + auto & column = header.getByPosition(key + 1); + + if (!isAggregateFunction(column.type)) + { + column.type = makeNullable(column.type); + column.column = makeNullable(column.column); + } + } + return header; +} + static Block appendGroupingColumn(Block block, const GroupingSetsParamsList & params) { if (params.empty()) return block; - Block res; - - size_t rows = block.rows(); - auto column = ColumnUInt64::create(rows); - - res.insert({ColumnPtr(std::move(column)), std::make_shared(), "__grouping_set"}); - - for (auto & col : block) - res.insert(std::move(col)); - - return res; + return generateOutputHeader(block); } AggregatingStep::AggregatingStep( @@ -249,7 +274,13 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B index.push_back(node); } else - index.push_back(dag->getIndex()[header.getPositionByName(col.name)]); + { + const auto * column_node = dag->getIndex()[header.getPositionByName(col.name)]; + // index.push_back(dag->getIndex()[header.getPositionByName(col.name)]); + + const auto * node = &dag->addFunction(FunctionFactory::instance().get("toNullable", nullptr), { column_node }, col.name); + index.push_back(node); + } } dag->getIndex().swap(index); diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 4dd3d956350..3d024a99063 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -3,6 +3,7 @@ #include #include #include +#include "Core/ColumnNumbers.h" namespace DB { @@ -26,6 +27,8 @@ struct GroupingSetsParams using GroupingSetsParamsList = std::vector; Block appendGroupingSetColumn(Block header); +Block generateOutputHeader(const Block & input_header); +Block generateOutputHeader(const Block & input_header, const ColumnNumbers & keys); /// Aggregation. See AggregatingTransform. class AggregatingStep : public ITransformingStep diff --git a/src/Processors/QueryPlan/RollupStep.cpp b/src/Processors/QueryPlan/RollupStep.cpp index 3b061f9c246..5109a5ce169 100644 --- a/src/Processors/QueryPlan/RollupStep.cpp +++ b/src/Processors/QueryPlan/RollupStep.cpp @@ -23,7 +23,7 @@ static ITransformingStep::Traits getTraits() } RollupStep::RollupStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_) - : ITransformingStep(input_stream_, appendGroupingSetColumn(params_->getHeader()), getTraits()) + : ITransformingStep(input_stream_, generateOutputHeader(params_->getHeader(), params_->params.keys), getTraits()) , params(std::move(params_)) , keys_size(params->params.keys_size) { diff --git a/src/Processors/Transforms/RollupTransform.cpp b/src/Processors/Transforms/RollupTransform.cpp index b69a691323c..6ac5ae35fa2 100644 --- a/src/Processors/Transforms/RollupTransform.cpp +++ b/src/Processors/Transforms/RollupTransform.cpp @@ -1,16 +1,24 @@ #include #include #include +#include +#include "Common/logger_useful.h" +#include "Columns/ColumnNullable.h" namespace DB { RollupTransform::RollupTransform(Block header, AggregatingTransformParamsPtr params_) - : IAccumulatingTransform(std::move(header), appendGroupingSetColumn(params_->getHeader())) + : IAccumulatingTransform(std::move(header), generateOutputHeader(params_->getHeader(), params_->params.keys)) , params(std::move(params_)) , keys(params->params.keys) , aggregates_mask(getAggregatesMask(params->getHeader(), params->params.aggregates)) { + auto output_aggregator_params = params->params; + intermediate_header = getOutputPort().getHeader(); + intermediate_header.erase(0); + output_aggregator_params.src_header = intermediate_header; + output_aggregator = std::make_unique(output_aggregator_params); } void RollupTransform::consume(Chunk chunk) @@ -18,13 +26,14 @@ void RollupTransform::consume(Chunk chunk) consumed_chunks.emplace_back(std::move(chunk)); } -Chunk RollupTransform::merge(Chunks && chunks, bool final) +Chunk RollupTransform::merge(Chunks && chunks, bool is_input, bool final) { BlocksList rollup_blocks; + auto header = is_input ? getInputPort().getHeader() : intermediate_header; for (auto & chunk : chunks) - rollup_blocks.emplace_back(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); + rollup_blocks.emplace_back(header.cloneWithColumns(chunk.detachColumns())); - auto rollup_block = params->aggregator.mergeBlocks(rollup_blocks, final); + auto rollup_block = is_input ? params->aggregator.mergeBlocks(rollup_blocks, final) : output_aggregator->mergeBlocks(rollup_blocks, final); auto num_rows = rollup_block.rows(); return Chunk(rollup_block.getColumns(), num_rows); } @@ -42,9 +51,16 @@ Chunk RollupTransform::generate() if (!consumed_chunks.empty()) { if (consumed_chunks.size() > 1) - rollup_chunk = merge(std::move(consumed_chunks), false); + rollup_chunk = merge(std::move(consumed_chunks), true, false); else rollup_chunk = std::move(consumed_chunks.front()); + + size_t rows = rollup_chunk.getNumRows(); + auto columns = rollup_chunk.getColumns(); + for (auto key : keys) + columns[key] = makeNullable(columns[key]); + rollup_chunk = Chunk{ columns, rows }; + LOG_DEBUG(&Poco::Logger::get("RollupTransform"), "Chunk source: {}", rollup_chunk.dumpStructure()); consumed_chunks.clear(); last_removed_key = keys.size(); @@ -59,11 +75,12 @@ Chunk RollupTransform::generate() auto num_rows = gen_chunk.getNumRows(); auto columns = gen_chunk.getColumns(); - columns[key] = getColumnWithDefaults(getInputPort().getHeader(), key, num_rows); + columns[key] = getColumnWithDefaults(intermediate_header, key, num_rows); Chunks chunks; chunks.emplace_back(std::move(columns), num_rows); - rollup_chunk = merge(std::move(chunks), false); + rollup_chunk = merge(std::move(chunks), false, false); + LOG_DEBUG(&Poco::Logger::get("RollupTransform"), "Chunk generated: {}", rollup_chunk.dumpStructure()); } finalizeChunk(gen_chunk, aggregates_mask); diff --git a/src/Processors/Transforms/RollupTransform.h b/src/Processors/Transforms/RollupTransform.h index 8fd27e3e6a2..8b66c85e0b5 100644 --- a/src/Processors/Transforms/RollupTransform.h +++ b/src/Processors/Transforms/RollupTransform.h @@ -1,4 +1,5 @@ #pragma once +#include #include #include #include @@ -23,12 +24,16 @@ private: const ColumnNumbers keys; const ColumnsMask aggregates_mask; + std::unique_ptr output_aggregator; + + Block intermediate_header; + Chunks consumed_chunks; Chunk rollup_chunk; size_t last_removed_key = 0; size_t set_counter = 0; - Chunk merge(Chunks && chunks, bool final); + Chunk merge(Chunks && chunks, bool is_input, bool final); }; } diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.reference b/tests/queries/0_stateless/02343_group_by_use_nulls.reference new file mode 100644 index 00000000000..0d7fa8f3a3b --- /dev/null +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.reference @@ -0,0 +1,21 @@ +0 0 0 +0 \N 0 +1 1 1 +1 \N 1 +2 0 2 +2 \N 2 +3 1 3 +3 \N 3 +4 0 4 +4 \N 4 +5 1 5 +5 \N 5 +6 0 6 +6 \N 6 +7 1 7 +7 \N 7 +8 0 8 +8 \N 8 +9 1 9 +9 \N 9 +\N \N 45 diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.sql b/tests/queries/0_stateless/02343_group_by_use_nulls.sql new file mode 100644 index 00000000000..1107ae79244 --- /dev/null +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.sql @@ -0,0 +1,4 @@ +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY ROLLUP(number, number % 2) +ORDER BY (number, number % 2, val); From fe451a4317f1159d76c4639636f4120405e1fac1 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 28 Jun 2022 22:52:11 -0400 Subject: [PATCH 047/659] remove compress sync interface --- contrib/qpl-cmake/CMakeLists.txt | 2 +- programs/keeper/CMakeLists.txt | 1 - src/Compression/CompressionCodecDeflate.cpp | 77 --------------------- src/Compression/CompressionCodecDeflate.h | 6 -- src/Compression/ICompressionCodec.cpp | 27 -------- src/Compression/ICompressionCodec.h | 17 +---- 6 files changed, 2 insertions(+), 128 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 8b21c9db360..e6714c671e1 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -113,7 +113,7 @@ target_compile_options(isal_asm PUBLIC "-I${QPL_SRC_DIR}/isal/include/" if (SANITIZE STREQUAL "undefined") get_target_property(target_options isal_asm COMPILE_OPTIONS) - list(REMOVE_ITEM target_options "no-sanitize=undefined") + list(REMOVE_ITEM target_options "-fno-sanitize=undefined") set_property(TARGET isal_asm PROPERTY COMPILE_OPTIONS ${target_options}) endif() diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 9a4266aa9db..c77b335b615 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -114,7 +114,6 @@ if (BUILD_STANDALONE_KEEPER) ch_contrib::nuraft ch_contrib::lz4 ch_contrib::zstd - ch_contrib::qpl ch_contrib::cityhash common ch_contrib::double_conversion ch_contrib::dragonbox_to_chars diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index de5a1c30486..2eca32714e6 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -57,14 +57,6 @@ HardwareCodecDeflate::~HardwareCodecDeflate() } jobDecompAsyncMap.clear(); } - if (!jobCompAsyncList.empty()) - { - for (auto id : jobCompAsyncList) - { - DeflateJobHWPool::instance().releaseJob(id); - } - jobCompAsyncList.clear(); - } } uint32_t HardwareCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const { @@ -97,54 +89,6 @@ uint32_t HardwareCodecDeflate::doCompressData(const char * source, uint32_t sour return compressed_size; } -uint32_t HardwareCodecDeflate::doCompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) -{ - uint32_t job_id = 0; - qpl_job * job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); - if (job_ptr == nullptr) - { - LOG_WARNING(log, "HardwareCodecDeflate::doCompressDataReq acquireJob fail!"); - return 0; - } - qpl_status status; - - job_ptr->op = qpl_op_compress; - job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); - job_ptr->next_out_ptr = reinterpret_cast(dest); - job_ptr->available_in = source_size; - job_ptr->level = qpl_default_level; - job_ptr->available_out = dest_size; - job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; - status = qpl_submit_job(job_ptr); - if (QPL_STS_OK == status) - { - jobCompAsyncList.push_back(job_id); - return job_id; - } - else - { - LOG_WARNING(log, "HardwareCodecDeflate::doCompressDataReq fail ->status: '{}' ", static_cast(status)); - DeflateJobHWPool::instance().releaseJob(job_id); - return 0; - } -} - -uint32_t HardwareCodecDeflate::doCompressDataFlush(uint32_t req_id) -{ - uint32_t compressed_size = 0; - qpl_job * job_ptr = DeflateJobHWPool::instance().getJobPtr(req_id); - if (nullptr != job_ptr) - { - while (QPL_STS_BEING_PROCESSED == qpl_check_job(job_ptr)) - { - _tpause(1, __rdtsc() + 1000); - } - compressed_size = job_ptr->total_out; - DeflateJobHWPool::instance().releaseJob(req_id); - } - return compressed_size; -} - uint32_t HardwareCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const { uint32_t job_id = 0; @@ -377,27 +321,6 @@ uint32_t CompressionCodecDeflate::doCompressData(const char * source, uint32_t s return res; } -uint32_t CompressionCodecDeflate::doCompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t & req_id) -{ - if (hwCodec->hwEnabled) - req_id = hwCodec->doCompressDataReq(source, source_size, dest, getMaxCompressedDataSize(source_size)); - else - req_id = 0; - - if (0 == req_id) - return swCodec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); - else - return 0; -} - -uint32_t CompressionCodecDeflate::doCompressDataFlush(uint32_t req_id) -{ - if (hwCodec->hwEnabled) - return hwCodec->doCompressDataFlush(req_id); - else - return 0; -} - void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const { uint32_t res = 0; diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index d3fb06ab10b..70a4a9cb529 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -203,15 +203,12 @@ public: HardwareCodecDeflate(); ~HardwareCodecDeflate(); uint32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const; - uint32_t doCompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t dest_size); - uint32_t doCompressDataFlush(uint32_t req_id); uint32_t doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const; uint32_t doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); void doDecompressDataFlush(); private: std::map jobDecompAsyncMap; - std::vector jobCompAsyncList; Poco::Logger * log; }; class CompressionCodecDeflate : public ICompressionCodec @@ -234,9 +231,6 @@ protected: } uint32_t doCompressData(const char * source, uint32_t source_size, char * dest) const override; uint32_t doCompressDataSW(const char * source, uint32_t source_size, char * dest) const; - uint32_t doCompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t & req_id) override; - uint32_t doCompressDataFlush(uint32_t req_id) override; - void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; void doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) override; void doDecompressDataSW(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; diff --git a/src/Compression/ICompressionCodec.cpp b/src/Compression/ICompressionCodec.cpp index 933eceb7859..06bd7cc7558 100644 --- a/src/Compression/ICompressionCodec.cpp +++ b/src/Compression/ICompressionCodec.cpp @@ -90,33 +90,6 @@ UInt32 ICompressionCodec::compress(const char * source, UInt32 source_size, char unalignedStore(&dest[5], source_size); return header_size + compressed_bytes_written; } -UInt32 ICompressionCodec::compressReq(const char * source, UInt32 source_size, char * dest, UInt32 & req_id) -{ - assert(source != nullptr && dest != nullptr); - dest[0] = getMethodByte(); - UInt8 header_size = getHeaderSize(); - - UInt32 res = doCompressDataReq(source, source_size, &dest[header_size], req_id); - if (res > 0) - { - unalignedStore(&dest[1], res + header_size); - unalignedStore(&dest[5], source_size); - return header_size + res; - } - else - { - unalignedStore(&dest[5], source_size); - return 0; - } -} - -UInt32 ICompressionCodec::compressFlush(UInt32 req_id, char * dest) -{ - UInt32 compressed_bytes_written = doCompressDataFlush(req_id); - UInt8 header_size = getHeaderSize(); - unalignedStore(&dest[1], compressed_bytes_written + header_size); - return header_size + compressed_bytes_written; -} UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, char * dest, UInt8 req_type) { diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 134862e37c8..03b5ca2b4e5 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -44,9 +44,7 @@ public: /// Compressed bytes from uncompressed source to dest. Dest should preallocate memory UInt32 compress(const char * source, UInt32 source_size, char * dest) const; - UInt32 compressReq(const char * source, UInt32 source_size, char * dest, UInt32 & req_id); - // Flush all asynchronous request for compression - UInt32 compressFlush(UInt32 req_id, char * dest); + /// Decompress bytes from compressed source to dest. Dest should preallocate memory; // reqType is specific for HW decompressor: //0 means synchronous request by default; @@ -110,19 +108,6 @@ protected: /// Actually compress data, without header virtual UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const = 0; - /// Asynchronous compression request to HW decompressor - virtual UInt32 doCompressDataReq(const char * source, UInt32 source_size, char * dest, UInt32 & req_id) - { - req_id = 0; - return doCompressData(source, source_size, dest); - } - - /// Flush asynchronous request for compression - virtual UInt32 doCompressDataFlush(UInt32 req_id = 0) - { - return req_id; - } - /// Actually decompress data without header virtual void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const = 0; From e32a0838d1b42beca1538f741f0d38ec09631ab4 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Wed, 29 Jun 2022 15:52:08 +0800 Subject: [PATCH 048/659] fix bug for mapUpdate --- src/Functions/map.cpp | 23 +++++++++++++++---- .../0_stateless/02169_map_functions.reference | 10 ++++++++ .../0_stateless/02169_map_functions.sql | 11 +++++++++ 3 files changed, 40 insertions(+), 4 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 8c891fdec81..036e242cfc2 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -561,15 +561,23 @@ public: { const ColumnMap * col_map_left = typeid_cast(arguments[0].column.get()); const auto * col_const_map_left = checkAndGetColumnConst(arguments[0].column.get()); + bool col_const_map_left_flag = false; if (col_const_map_left) + { + col_const_map_left_flag = true; col_map_left = typeid_cast(&col_const_map_left->getDataColumn()); + } if (!col_map_left) return nullptr; const ColumnMap * col_map_right = typeid_cast(arguments[1].column.get()); const auto * col_const_map_right = checkAndGetColumnConst(arguments[1].column.get()); + bool col_const_map_right_flag = false; if (col_const_map_right) + { + col_const_map_right_flag = true; col_map_right = typeid_cast(&col_const_map_right->getDataColumn()); + } if (!col_map_right) return nullptr; @@ -591,13 +599,18 @@ public: MutableColumnPtr offsets = DataTypeNumber().createColumn(); IColumn::Offset current_offset = 0; - for (size_t idx = 0; idx < input_rows_count; ++idx) + for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) { - for (size_t i = offsets_left[idx - 1]; i < offsets_left[idx]; ++i) + size_t left_it_begin = col_const_map_left_flag ? 0 : offsets_left[row_idx - 1]; + size_t left_it_end = col_const_map_left_flag ? offsets_left.size() : offsets_left[row_idx]; + size_t right_it_begin = col_const_map_right_flag ? 0 : offsets_right[row_idx - 1]; + size_t right_it_end = col_const_map_right_flag ? offsets_right.size() : offsets_right[row_idx]; + + for (size_t i = left_it_begin; i < left_it_end; ++i) { bool matched = false; auto key = keys_data_left.getDataAt(i); - for (size_t j = offsets_right[idx - 1]; j < offsets_right[idx]; ++j) + for (size_t j = right_it_begin; j < right_it_end; ++j) { if (keys_data_right.getDataAt(j).toString() == key.toString()) { @@ -612,12 +625,14 @@ public: ++current_offset; } } - for (size_t j = offsets_right[idx - 1]; j < offsets_right[idx]; ++j) + + for (size_t j = right_it_begin; j < right_it_end; ++j) { keys_data->insertFrom(keys_data_right, j); values_data->insertFrom(values_data_right, j); ++current_offset; } + offsets->insert(current_offset); } diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index 160aebbc852..16fda0bb840 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -31,3 +31,13 @@ {1:2,2:3} {'x':'y','x':'y'} {'x':'y','x':'y'} +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index 4cccaa56722..9e93412739f 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -37,3 +37,14 @@ SELECT mapUpdate(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } DROP TABLE table_map; + +DROP TABLE IF EXISTS map_test; +CREATE TABLE map_test(`tags` Map(String, String)) ENGINE = MergeTree PRIMARY KEY tags ORDER BY tags SETTINGS index_granularity = 8192; +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +SELECT mapUpdate(mapFilter((k, v) -> (k in ('fruit')), tags), map('season', 'autumn')) FROM map_test; +SELECT mapUpdate(map('season','autumn'), mapFilter((k, v) -> (k in ('fruit')), tags)) FROM map_test; +DROP TABLE map_test; From 98e9bc84d5a95259112ecff815228c62ce08c9da Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 30 Jun 2022 10:13:58 +0000 Subject: [PATCH 049/659] Refector ROLLUP and CUBE --- src/Interpreters/InterpreterSelectQuery.cpp | 4 +- src/Processors/QueryPlan/AggregatingStep.cpp | 21 +++-- src/Processors/QueryPlan/AggregatingStep.h | 2 +- src/Processors/QueryPlan/CubeStep.cpp | 2 +- src/Processors/QueryPlan/RollupStep.cpp | 2 +- .../Transforms/AggregatingTransform.h | 4 +- src/Processors/Transforms/CubeTransform.cpp | 41 ++------- src/Processors/Transforms/CubeTransform.h | 10 +-- src/Processors/Transforms/RollupTransform.cpp | 88 ++++++++++++------- src/Processors/Transforms/RollupTransform.h | 40 ++++++--- .../02343_group_by_use_nulls.reference | 88 +++++++++++++++++++ .../0_stateless/02343_group_by_use_nulls.sql | 22 ++++- 12 files changed, 221 insertions(+), 103 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index feae7ac6a21..a29684288cf 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1618,7 +1618,7 @@ static void executeMergeAggregatedImpl( Aggregator::Params params(header_before_merge, keys, aggregates, overflow_row, settings.max_threads); - auto transform_params = std::make_shared(params, final); + auto transform_params = std::make_shared(params, final, false); auto merging_aggregated = std::make_unique( query_plan.getCurrentDataStream(), @@ -2363,7 +2363,7 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modific keys.push_back(header_before_transform.getPositionByName(key.name)); auto params = getAggregatorParams(query_ptr, *query_analyzer, *context, header_before_transform, keys, query_analyzer->aggregates(), false, settings, 0, 0); - auto transform_params = std::make_shared(std::move(params), true); + auto transform_params = std::make_shared(std::move(params), true, settings.group_by_use_nulls); QueryPlanStepPtr step; if (modificator == Modificator::ROLLUP) diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index a0f5fce908b..a038b5cf302 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -65,17 +65,20 @@ Block generateOutputHeader(const Block & input_header) return header; } -Block generateOutputHeader(const Block & input_header, const ColumnNumbers & keys) +Block generateOutputHeader(const Block & input_header, const ColumnNumbers & keys, bool use_nulls) { auto header = appendGroupingSetColumn(input_header); - for (auto key : keys) + if (use_nulls) { - auto & column = header.getByPosition(key + 1); - - if (!isAggregateFunction(column.type)) + for (auto key : keys) { - column.type = makeNullable(column.type); - column.column = makeNullable(column.column); + auto & column = header.getByPosition(key + 1); + + if (!isAggregateFunction(column.type)) + { + column.type = makeNullable(column.type); + column.column = makeNullable(column.column); + } } } return header; @@ -144,7 +147,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B * 1. Parallel aggregation is done, and the results should be merged in parallel. * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. */ - auto transform_params = std::make_shared(std::move(params), final); + auto transform_params = std::make_shared(std::move(params), final, false); if (!grouping_sets_params.empty()) { @@ -194,7 +197,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B transform_params->params.intermediate_header, transform_params->params.stats_collecting_params }; - auto transform_params_for_set = std::make_shared(std::move(params_for_set), final); + auto transform_params_for_set = std::make_shared(std::move(params_for_set), final, false); if (streams > 1) { diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 3d024a99063..c0be99403e9 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -28,7 +28,7 @@ using GroupingSetsParamsList = std::vector; Block appendGroupingSetColumn(Block header); Block generateOutputHeader(const Block & input_header); -Block generateOutputHeader(const Block & input_header, const ColumnNumbers & keys); +Block generateOutputHeader(const Block & input_header, const ColumnNumbers & keys, bool use_nulls); /// Aggregation. See AggregatingTransform. class AggregatingStep : public ITransformingStep diff --git a/src/Processors/QueryPlan/CubeStep.cpp b/src/Processors/QueryPlan/CubeStep.cpp index 91c85a08412..9ea0d1c9995 100644 --- a/src/Processors/QueryPlan/CubeStep.cpp +++ b/src/Processors/QueryPlan/CubeStep.cpp @@ -25,7 +25,7 @@ static ITransformingStep::Traits getTraits() } CubeStep::CubeStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_) - : ITransformingStep(input_stream_, appendGroupingSetColumn(params_->getHeader()), getTraits()) + : ITransformingStep(input_stream_, generateOutputHeader(params_->getHeader(), params_->params.keys, params_->use_nulls), getTraits()) , keys_size(params_->params.keys_size) , params(std::move(params_)) { diff --git a/src/Processors/QueryPlan/RollupStep.cpp b/src/Processors/QueryPlan/RollupStep.cpp index 5109a5ce169..130525bfacb 100644 --- a/src/Processors/QueryPlan/RollupStep.cpp +++ b/src/Processors/QueryPlan/RollupStep.cpp @@ -23,7 +23,7 @@ static ITransformingStep::Traits getTraits() } RollupStep::RollupStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_) - : ITransformingStep(input_stream_, generateOutputHeader(params_->getHeader(), params_->params.keys), getTraits()) + : ITransformingStep(input_stream_, generateOutputHeader(params_->getHeader(), params_->params.keys, params_->use_nulls), getTraits()) , params(std::move(params_)) , keys_size(params->params.keys_size) { diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index bfc3904e5d8..5438bc32ed9 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -35,12 +35,14 @@ struct AggregatingTransformParams Aggregator & aggregator; bool final; bool only_merge = false; + bool use_nulls = false; - AggregatingTransformParams(const Aggregator::Params & params_, bool final_) + AggregatingTransformParams(const Aggregator::Params & params_, bool final_, bool use_nulls_) : params(params_) , aggregator_list_ptr(std::make_shared()) , aggregator(*aggregator_list_ptr->emplace(aggregator_list_ptr->end(), params)) , final(final_) + , use_nulls(use_nulls_) { } diff --git a/src/Processors/Transforms/CubeTransform.cpp b/src/Processors/Transforms/CubeTransform.cpp index 83ed346dabe..40b096e88f4 100644 --- a/src/Processors/Transforms/CubeTransform.cpp +++ b/src/Processors/Transforms/CubeTransform.cpp @@ -1,6 +1,7 @@ #include #include #include +#include "Processors/Transforms/RollupTransform.h" namespace DB { @@ -10,57 +11,31 @@ namespace ErrorCodes } CubeTransform::CubeTransform(Block header, AggregatingTransformParamsPtr params_) - : IAccumulatingTransform(std::move(header), appendGroupingSetColumn(params_->getHeader())) - , params(std::move(params_)) - , keys(params->params.keys) + : GroupByModifierTransform(std::move(header), params_) , aggregates_mask(getAggregatesMask(params->getHeader(), params->params.aggregates)) { if (keys.size() >= 8 * sizeof(mask)) throw Exception("Too many keys are used for CubeTransform.", ErrorCodes::LOGICAL_ERROR); } -Chunk CubeTransform::merge(Chunks && chunks, bool final) -{ - BlocksList rollup_blocks; - for (auto & chunk : chunks) - rollup_blocks.emplace_back(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); - - auto rollup_block = params->aggregator.mergeBlocks(rollup_blocks, final); - auto num_rows = rollup_block.rows(); - return Chunk(rollup_block.getColumns(), num_rows); -} - -void CubeTransform::consume(Chunk chunk) -{ - consumed_chunks.emplace_back(std::move(chunk)); -} - -MutableColumnPtr getColumnWithDefaults(Block const & header, size_t key, size_t n); - Chunk CubeTransform::generate() { if (!consumed_chunks.empty()) { - if (consumed_chunks.size() > 1) - cube_chunk = merge(std::move(consumed_chunks), false); - else - cube_chunk = std::move(consumed_chunks.front()); + mergeConsumed(); - consumed_chunks.clear(); - - auto num_rows = cube_chunk.getNumRows(); + auto num_rows = current_chunk.getNumRows(); mask = (static_cast(1) << keys.size()) - 1; - current_columns = cube_chunk.getColumns(); + current_columns = current_chunk.getColumns(); current_zero_columns.clear(); current_zero_columns.reserve(keys.size()); - auto const & input_header = getInputPort().getHeader(); for (auto key : keys) - current_zero_columns.emplace_back(getColumnWithDefaults(input_header, key, num_rows)); + current_zero_columns.emplace_back(getColumnWithDefaults(key, num_rows)); } - auto gen_chunk = std::move(cube_chunk); + auto gen_chunk = std::move(current_chunk); if (mask) { @@ -75,7 +50,7 @@ Chunk CubeTransform::generate() Chunks chunks; chunks.emplace_back(std::move(columns), current_columns.front()->size()); - cube_chunk = merge(std::move(chunks), false); + current_chunk = merge(std::move(chunks), !params->use_nulls, false); } finalizeChunk(gen_chunk, aggregates_mask); diff --git a/src/Processors/Transforms/CubeTransform.h b/src/Processors/Transforms/CubeTransform.h index 4575a01935d..ac51a28bd0e 100644 --- a/src/Processors/Transforms/CubeTransform.h +++ b/src/Processors/Transforms/CubeTransform.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include #include @@ -9,30 +10,23 @@ namespace DB /// Takes blocks after grouping, with non-finalized aggregate functions. /// Calculates all subsets of columns and aggregates over them. -class CubeTransform : public IAccumulatingTransform +class CubeTransform : public GroupByModifierTransform { public: CubeTransform(Block header, AggregatingTransformParamsPtr params); String getName() const override { return "CubeTransform"; } protected: - void consume(Chunk chunk) override; Chunk generate() override; private: - AggregatingTransformParamsPtr params; - const ColumnNumbers keys; const ColumnsMask aggregates_mask; - Chunks consumed_chunks; - Chunk cube_chunk; Columns current_columns; Columns current_zero_columns; UInt64 mask = 0; UInt64 grouping_set = 0; - - Chunk merge(Chunks && chunks, bool final); }; } diff --git a/src/Processors/Transforms/RollupTransform.cpp b/src/Processors/Transforms/RollupTransform.cpp index 6ac5ae35fa2..2fbde7abf0e 100644 --- a/src/Processors/Transforms/RollupTransform.cpp +++ b/src/Processors/Transforms/RollupTransform.cpp @@ -8,36 +8,71 @@ namespace DB { -RollupTransform::RollupTransform(Block header, AggregatingTransformParamsPtr params_) - : IAccumulatingTransform(std::move(header), generateOutputHeader(params_->getHeader(), params_->params.keys)) +GroupByModifierTransform::GroupByModifierTransform(Block header, AggregatingTransformParamsPtr params_) + : IAccumulatingTransform(std::move(header), generateOutputHeader(params_->getHeader(), params_->params.keys, params_->use_nulls)) , params(std::move(params_)) , keys(params->params.keys) - , aggregates_mask(getAggregatesMask(params->getHeader(), params->params.aggregates)) { - auto output_aggregator_params = params->params; intermediate_header = getOutputPort().getHeader(); - intermediate_header.erase(0); - output_aggregator_params.src_header = intermediate_header; - output_aggregator = std::make_unique(output_aggregator_params); + if (params->use_nulls) + { + auto output_aggregator_params = params->params; + intermediate_header.erase(0); + output_aggregator_params.src_header = intermediate_header; + output_aggregator = std::make_unique(output_aggregator_params); + } } -void RollupTransform::consume(Chunk chunk) +void GroupByModifierTransform::consume(Chunk chunk) { consumed_chunks.emplace_back(std::move(chunk)); } -Chunk RollupTransform::merge(Chunks && chunks, bool is_input, bool final) +void GroupByModifierTransform::mergeConsumed() { - BlocksList rollup_blocks; - auto header = is_input ? getInputPort().getHeader() : intermediate_header; - for (auto & chunk : chunks) - rollup_blocks.emplace_back(header.cloneWithColumns(chunk.detachColumns())); + if (consumed_chunks.size() > 1) + current_chunk = merge(std::move(consumed_chunks), true, false); + else + current_chunk = std::move(consumed_chunks.front()); - auto rollup_block = is_input ? params->aggregator.mergeBlocks(rollup_blocks, final) : output_aggregator->mergeBlocks(rollup_blocks, final); - auto num_rows = rollup_block.rows(); - return Chunk(rollup_block.getColumns(), num_rows); + size_t rows = current_chunk.getNumRows(); + auto columns = current_chunk.getColumns(); + if (params->use_nulls) + { + for (auto key : keys) + columns[key] = makeNullable(columns[key]); + } + current_chunk = Chunk{ columns, rows }; + + consumed_chunks.clear(); } +Chunk GroupByModifierTransform::merge(Chunks && chunks, bool is_input, bool final) +{ + auto header = is_input ? getInputPort().getHeader() : intermediate_header; + + BlocksList blocks; + for (auto & chunk : chunks) + blocks.emplace_back(header.cloneWithColumns(chunk.detachColumns())); + + auto current_block = is_input ? params->aggregator.mergeBlocks(blocks, final) : output_aggregator->mergeBlocks(blocks, final); + auto num_rows = current_block.rows(); + return Chunk(current_block.getColumns(), num_rows); +} + +MutableColumnPtr GroupByModifierTransform::getColumnWithDefaults(size_t key, size_t n) const +{ + auto const & col = intermediate_header.getByPosition(key); + auto result_column = col.column->cloneEmpty(); + col.type->insertManyDefaultsInto(*result_column, n); + return result_column; +} + +RollupTransform::RollupTransform(Block header, AggregatingTransformParamsPtr params_) + : GroupByModifierTransform(std::move(header), params_) + , aggregates_mask(getAggregatesMask(params->getHeader(), params->params.aggregates)) +{} + MutableColumnPtr getColumnWithDefaults(Block const & header, size_t key, size_t n) { auto const & col = header.getByPosition(key); @@ -50,23 +85,11 @@ Chunk RollupTransform::generate() { if (!consumed_chunks.empty()) { - if (consumed_chunks.size() > 1) - rollup_chunk = merge(std::move(consumed_chunks), true, false); - else - rollup_chunk = std::move(consumed_chunks.front()); - - size_t rows = rollup_chunk.getNumRows(); - auto columns = rollup_chunk.getColumns(); - for (auto key : keys) - columns[key] = makeNullable(columns[key]); - rollup_chunk = Chunk{ columns, rows }; - LOG_DEBUG(&Poco::Logger::get("RollupTransform"), "Chunk source: {}", rollup_chunk.dumpStructure()); - - consumed_chunks.clear(); + mergeConsumed(); last_removed_key = keys.size(); } - auto gen_chunk = std::move(rollup_chunk); + auto gen_chunk = std::move(current_chunk); if (last_removed_key) { @@ -75,12 +98,11 @@ Chunk RollupTransform::generate() auto num_rows = gen_chunk.getNumRows(); auto columns = gen_chunk.getColumns(); - columns[key] = getColumnWithDefaults(intermediate_header, key, num_rows); + columns[key] = getColumnWithDefaults(key, num_rows); Chunks chunks; chunks.emplace_back(std::move(columns), num_rows); - rollup_chunk = merge(std::move(chunks), false, false); - LOG_DEBUG(&Poco::Logger::get("RollupTransform"), "Chunk generated: {}", rollup_chunk.dumpStructure()); + current_chunk = merge(std::move(chunks), !params->use_nulls, false); } finalizeChunk(gen_chunk, aggregates_mask); diff --git a/src/Processors/Transforms/RollupTransform.h b/src/Processors/Transforms/RollupTransform.h index 8b66c85e0b5..cbed7705628 100644 --- a/src/Processors/Transforms/RollupTransform.h +++ b/src/Processors/Transforms/RollupTransform.h @@ -7,33 +7,47 @@ namespace DB { -/// Takes blocks after grouping, with non-finalized aggregate functions. -/// Calculates subtotals and grand totals values for a set of columns. -class RollupTransform : public IAccumulatingTransform +struct GroupByModifierTransform : public IAccumulatingTransform { -public: - RollupTransform(Block header, AggregatingTransformParamsPtr params); - String getName() const override { return "RollupTransform"; } + GroupByModifierTransform(Block header, AggregatingTransformParamsPtr params_); protected: void consume(Chunk chunk) override; - Chunk generate() override; -private: + void mergeConsumed(); + + Chunk merge(Chunks && chunks, bool is_input, bool final); + + MutableColumnPtr getColumnWithDefaults(size_t key, size_t n) const; + AggregatingTransformParamsPtr params; - const ColumnNumbers keys; - const ColumnsMask aggregates_mask; + + const ColumnNumbers & keys; std::unique_ptr output_aggregator; Block intermediate_header; Chunks consumed_chunks; - Chunk rollup_chunk; + Chunk current_chunk; +}; + +/// Takes blocks after grouping, with non-finalized aggregate functions. +/// Calculates subtotals and grand totals values for a set of columns. +class RollupTransform : public GroupByModifierTransform +{ +public: + RollupTransform(Block header, AggregatingTransformParamsPtr params); + String getName() const override { return "RollupTransform"; } + +protected: + Chunk generate() override; + +private: + const ColumnsMask aggregates_mask; + size_t last_removed_key = 0; size_t set_counter = 0; - - Chunk merge(Chunks && chunks, bool is_input, bool final); }; } diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.reference b/tests/queries/0_stateless/02343_group_by_use_nulls.reference index 0d7fa8f3a3b..769324efec3 100644 --- a/tests/queries/0_stateless/02343_group_by_use_nulls.reference +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.reference @@ -1,3 +1,9 @@ +-- { echoOn } +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY ROLLUP(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; 0 0 0 0 \N 0 1 1 1 @@ -19,3 +25,85 @@ 9 1 9 9 \N 9 \N \N 45 +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY ROLLUP(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=0; +0 0 0 +0 0 0 +0 0 45 +1 0 1 +1 1 1 +2 0 2 +2 0 2 +3 0 3 +3 1 3 +4 0 4 +4 0 4 +5 0 5 +5 1 5 +6 0 6 +6 0 6 +7 0 7 +7 1 7 +8 0 8 +8 0 8 +9 0 9 +9 1 9 +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY CUBE(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; +0 0 0 +0 \N 0 +1 1 1 +1 \N 1 +2 0 2 +2 \N 2 +3 1 3 +3 \N 3 +4 0 4 +4 \N 4 +5 1 5 +5 \N 5 +6 0 6 +6 \N 6 +7 1 7 +7 \N 7 +8 0 8 +8 \N 8 +9 1 9 +9 \N 9 +\N 0 20 +\N 1 25 +\N \N 45 +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY CUBE(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=0; +0 0 0 +0 0 0 +0 0 20 +0 0 45 +0 1 25 +1 0 1 +1 1 1 +2 0 2 +2 0 2 +3 0 3 +3 1 3 +4 0 4 +4 0 4 +5 0 5 +5 1 5 +6 0 6 +6 0 6 +7 0 7 +7 1 7 +8 0 8 +8 0 8 +9 0 9 +9 1 9 diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.sql b/tests/queries/0_stateless/02343_group_by_use_nulls.sql index 1107ae79244..ac63e9feebc 100644 --- a/tests/queries/0_stateless/02343_group_by_use_nulls.sql +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.sql @@ -1,4 +1,24 @@ +-- { echoOn } SELECT number, number % 2, sum(number) AS val FROM numbers(10) GROUP BY ROLLUP(number, number % 2) -ORDER BY (number, number % 2, val); +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; + +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY ROLLUP(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=0; + +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY CUBE(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; + +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY CUBE(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=0; From 33f601ec0a82133b0c9cef734cff025169172e1f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 30 Jun 2022 15:14:26 +0000 Subject: [PATCH 050/659] Commit support use_nulls for GS --- src/Interpreters/InterpreterSelectQuery.cpp | 1 + src/Processors/QueryPlan/AggregatingStep.cpp | 34 ++++++++----- src/Processors/QueryPlan/AggregatingStep.h | 3 +- .../02343_group_by_use_nulls.reference | 48 +++++++++++++++++++ .../0_stateless/02343_group_by_use_nulls.sql | 26 ++++++++++ 5 files changed, 99 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a29684288cf..797fa700c9b 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2295,6 +2295,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac merge_threads, temporary_data_merge_threads, storage_has_evenly_distributed_read, + settings.group_by_use_nulls, std::move(group_by_info), std::move(group_by_sort_description)); query_plan.addStep(std::move(aggregating_step)); diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index a038b5cf302..48fb3729940 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -16,6 +16,8 @@ #include #include #include +#include +#include "Common/logger_useful.h" #include "Core/ColumnNumbers.h" #include "DataTypes/IDataType.h" @@ -49,12 +51,11 @@ Block appendGroupingSetColumn(Block header) return res; } -Block generateOutputHeader(const Block & input_header) +static inline void convertToNullable(Block & header, const ColumnNumbers & keys) { - auto header = appendGroupingSetColumn(input_header); - for (size_t i = 1; i < header.columns(); ++i) + for (auto key : keys) { - auto & column = header.getByPosition(i); + auto & column = header.getByPosition(key); if (!isAggregateFunction(column.type)) { @@ -62,7 +63,6 @@ Block generateOutputHeader(const Block & input_header) column.column = makeNullable(column.column); } } - return header; } Block generateOutputHeader(const Block & input_header, const ColumnNumbers & keys, bool use_nulls) @@ -84,12 +84,12 @@ Block generateOutputHeader(const Block & input_header, const ColumnNumbers & key return header; } -static Block appendGroupingColumn(Block block, const GroupingSetsParamsList & params) +static Block appendGroupingColumn(Block block, const ColumnNumbers & keys, const GroupingSetsParamsList & params, bool use_nulls) { if (params.empty()) return block; - return generateOutputHeader(block); + return generateOutputHeader(block, keys, use_nulls); } AggregatingStep::AggregatingStep( @@ -102,9 +102,10 @@ AggregatingStep::AggregatingStep( size_t merge_threads_, size_t temporary_data_merge_threads_, bool storage_has_evenly_distributed_read_, + bool group_by_use_nulls_, InputOrderInfoPtr group_by_info_, SortDescription group_by_sort_description_) - : ITransformingStep(input_stream_, appendGroupingColumn(params_.getHeader(final_), grouping_sets_params_), getTraits(), false) + : ITransformingStep(input_stream_, appendGroupingColumn(params_.getHeader(final_), params_.keys, grouping_sets_params_, group_by_use_nulls_), getTraits(), false) , params(std::move(params_)) , grouping_sets_params(std::move(grouping_sets_params_)) , final(std::move(final_)) @@ -113,6 +114,7 @@ AggregatingStep::AggregatingStep( , merge_threads(merge_threads_) , temporary_data_merge_threads(temporary_data_merge_threads_) , storage_has_evenly_distributed_read(storage_has_evenly_distributed_read_) + , group_by_use_nulls(group_by_use_nulls_) , group_by_info(std::move(group_by_info_)) , group_by_sort_description(std::move(group_by_sort_description_)) { @@ -243,6 +245,8 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B assert(ports.size() == grouping_sets_size); auto output_header = transform_params->getHeader(); + if (group_by_use_nulls) + convertToNullable(output_header, params.keys); for (size_t set_counter = 0; set_counter < grouping_sets_size; ++set_counter) { @@ -279,16 +283,22 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B else { const auto * column_node = dag->getIndex()[header.getPositionByName(col.name)]; - // index.push_back(dag->getIndex()[header.getPositionByName(col.name)]); - - const auto * node = &dag->addFunction(FunctionFactory::instance().get("toNullable", nullptr), { column_node }, col.name); - index.push_back(node); + if (isAggregateFunction(column_node->result_type) || !group_by_use_nulls) + { + index.push_back(column_node); + } + else + { + const auto * node = &dag->addFunction(FunctionFactory::instance().get("toNullable", nullptr), { column_node }, col.name); + index.push_back(node); + } } } dag->getIndex().swap(index); auto expression = std::make_shared(dag, settings.getActionsSettings()); auto transform = std::make_shared(header, expression); + LOG_DEBUG(&Poco::Logger::get("AggregatingStep"), "Header for GROUPING SET #{}: {}", set_counter, transform->getOutputPort().getHeader().dumpStructure()); connect(*ports[set_counter], transform->getInputPort()); processors.emplace_back(std::move(transform)); diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index c0be99403e9..c40994e018d 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -27,7 +27,6 @@ struct GroupingSetsParams using GroupingSetsParamsList = std::vector; Block appendGroupingSetColumn(Block header); -Block generateOutputHeader(const Block & input_header); Block generateOutputHeader(const Block & input_header, const ColumnNumbers & keys, bool use_nulls); /// Aggregation. See AggregatingTransform. @@ -44,6 +43,7 @@ public: size_t merge_threads_, size_t temporary_data_merge_threads_, bool storage_has_evenly_distributed_read_, + bool group_by_use_nulls_, InputOrderInfoPtr group_by_info_, SortDescription group_by_sort_description_); @@ -68,6 +68,7 @@ private: size_t temporary_data_merge_threads; bool storage_has_evenly_distributed_read; + bool group_by_use_nulls; InputOrderInfoPtr group_by_info; SortDescription group_by_sort_description; diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.reference b/tests/queries/0_stateless/02343_group_by_use_nulls.reference index 769324efec3..92d36c1a894 100644 --- a/tests/queries/0_stateless/02343_group_by_use_nulls.reference +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.reference @@ -107,3 +107,51 @@ SETTINGS group_by_use_nulls=0; 8 0 8 9 0 9 9 1 9 +SELECT + number, + number % 2, + sum(number) AS val +FROM numbers(10) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls = 1; +0 \N 0 +1 \N 1 +2 \N 2 +3 \N 3 +4 \N 4 +5 \N 5 +6 \N 6 +7 \N 7 +8 \N 8 +9 \N 9 +\N 0 20 +\N 1 25 +SELECT + number, + number % 2, + sum(number) AS val +FROM numbers(10) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls = 0; +0 0 0 +0 0 20 +0 1 25 +1 0 1 +2 0 2 +3 0 3 +4 0 4 +5 0 5 +6 0 6 +7 0 7 +8 0 8 +9 0 9 diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.sql b/tests/queries/0_stateless/02343_group_by_use_nulls.sql index ac63e9feebc..5256c6bda75 100644 --- a/tests/queries/0_stateless/02343_group_by_use_nulls.sql +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.sql @@ -22,3 +22,29 @@ FROM numbers(10) GROUP BY CUBE(number, number % 2) ORDER BY (number, number % 2, val) SETTINGS group_by_use_nulls=0; + +SELECT + number, + number % 2, + sum(number) AS val +FROM numbers(10) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls = 1; + +SELECT + number, + number % 2, + sum(number) AS val +FROM numbers(10) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls = 0; From d66154e697882f1d007e20d2370558973276dcd3 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 2 Jun 2022 22:54:14 -0300 Subject: [PATCH 051/659] Test host_regexp against all PTR records instead of only one --- .gitmodules | 6 + contrib/CMakeLists.txt | 2 + contrib/cpp-dns | 1 + contrib/cpp-dns-cmake/CMakeLists.txt | 31 + contrib/udns-cmake/CMakeLists.txt | 54 + contrib/udns/COPYING.LGPL | 502 ++++++++ contrib/udns/Makefile.in | 197 ++++ contrib/udns/NEWS | 136 +++ contrib/udns/NOTES | 226 ++++ contrib/udns/README.md | 1 + contrib/udns/TODO | 59 + contrib/udns/config.h | 7 + contrib/udns/configure | 166 +++ contrib/udns/configure.lib | 268 +++++ contrib/udns/conftest.c | 3 + contrib/udns/conftest.out | 7 + contrib/udns/dnsget.1 | 195 ++++ contrib/udns/dnsget.c | 759 ++++++++++++ contrib/udns/ex-rdns.c | 114 ++ contrib/udns/getopt.c | 165 +++ contrib/udns/inet_XtoX.c | 327 ++++++ contrib/udns/rblcheck.1 | 151 +++ contrib/udns/rblcheck.c | 378 ++++++ contrib/udns/udns.3 | 1352 ++++++++++++++++++++++ contrib/udns/udns.h | 778 +++++++++++++ contrib/udns/udns_XtoX.c | 50 + contrib/udns/udns_bl.c | 160 +++ contrib/udns/udns_codes.c | 199 ++++ contrib/udns/udns_dn.c | 379 ++++++ contrib/udns/udns_dntosp.c | 30 + contrib/udns/udns_init.c | 231 ++++ contrib/udns/udns_jran.c | 52 + contrib/udns/udns_misc.c | 67 ++ contrib/udns/udns_parse.c | 169 +++ contrib/udns/udns_resolver.c | 1323 +++++++++++++++++++++ contrib/udns/udns_rr_a.c | 123 ++ contrib/udns/udns_rr_mx.c | 91 ++ contrib/udns/udns_rr_naptr.c | 128 ++ contrib/udns/udns_rr_ptr.c | 109 ++ contrib/udns/udns_rr_srv.c | 155 +++ contrib/udns/udns_rr_txt.c | 98 ++ src/Access/Common/AllowedClientHosts.cpp | 41 +- src/CMakeLists.txt | 10 + src/Common/DNSResolver.cpp | 31 +- src/Common/DNSResolver.h | 4 +- 45 files changed, 9309 insertions(+), 26 deletions(-) create mode 160000 contrib/cpp-dns create mode 100644 contrib/cpp-dns-cmake/CMakeLists.txt create mode 100644 contrib/udns-cmake/CMakeLists.txt create mode 100644 contrib/udns/COPYING.LGPL create mode 100644 contrib/udns/Makefile.in create mode 100644 contrib/udns/NEWS create mode 100644 contrib/udns/NOTES create mode 100644 contrib/udns/README.md create mode 100644 contrib/udns/TODO create mode 100644 contrib/udns/config.h create mode 100755 contrib/udns/configure create mode 100644 contrib/udns/configure.lib create mode 100644 contrib/udns/conftest.c create mode 100644 contrib/udns/conftest.out create mode 100644 contrib/udns/dnsget.1 create mode 100644 contrib/udns/dnsget.c create mode 100644 contrib/udns/ex-rdns.c create mode 100644 contrib/udns/getopt.c create mode 100644 contrib/udns/inet_XtoX.c create mode 100644 contrib/udns/rblcheck.1 create mode 100644 contrib/udns/rblcheck.c create mode 100644 contrib/udns/udns.3 create mode 100644 contrib/udns/udns.h create mode 100644 contrib/udns/udns_XtoX.c create mode 100644 contrib/udns/udns_bl.c create mode 100644 contrib/udns/udns_codes.c create mode 100644 contrib/udns/udns_dn.c create mode 100644 contrib/udns/udns_dntosp.c create mode 100644 contrib/udns/udns_init.c create mode 100644 contrib/udns/udns_jran.c create mode 100644 contrib/udns/udns_misc.c create mode 100644 contrib/udns/udns_parse.c create mode 100644 contrib/udns/udns_resolver.c create mode 100644 contrib/udns/udns_rr_a.c create mode 100644 contrib/udns/udns_rr_mx.c create mode 100644 contrib/udns/udns_rr_naptr.c create mode 100644 contrib/udns/udns_rr_ptr.c create mode 100644 contrib/udns/udns_rr_srv.c create mode 100644 contrib/udns/udns_rr_txt.c diff --git a/.gitmodules b/.gitmodules index c46b1c736fc..7d0ca61ebe2 100644 --- a/.gitmodules +++ b/.gitmodules @@ -274,3 +274,9 @@ [submodule "contrib/base-x"] path = contrib/base-x url = https://github.com/ClickHouse/base-x.git +[submodule "contrib/cpp-dns"] + path = contrib/cpp-dns + url = git@github.com:YukiWorkshop/cpp-dns.git +[submodule "contrib/udns"] + path = contrib/udns + url = https://github.com/arthurpassos/udns.git \ No newline at end of file diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 1dd28fa90ff..06e90dedb5b 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -156,6 +156,8 @@ endif() add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) add_contrib (base-x-cmake base-x) +add_contrib (udns-cmake udns) +add_contrib (cpp-dns-cmake cpp-dns) # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear diff --git a/contrib/cpp-dns b/contrib/cpp-dns new file mode 160000 index 00000000000..0ce2dbd40dc --- /dev/null +++ b/contrib/cpp-dns @@ -0,0 +1 @@ +Subproject commit 0ce2dbd40dce456bde1732c4d8f70008f0079db0 diff --git a/contrib/cpp-dns-cmake/CMakeLists.txt b/contrib/cpp-dns-cmake/CMakeLists.txt new file mode 100644 index 00000000000..342e189ff36 --- /dev/null +++ b/contrib/cpp-dns-cmake/CMakeLists.txt @@ -0,0 +1,31 @@ +set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/cpp-dns") + +# If not, ASIO standalone mode. +#find_path(LIBUDNS_INCLUDE_DIR +# NAME udns.h +# HINTS /opt/local/include) +# +#find_library(lib_udns udns +# PATHS /opt/local/lib) + +#if (lib_udns) +# message("Found libudns!") +#else() +# message(FATAL_ERROR "udns not found") +#endif() + +#find_package(Boost 1.78 COMPONENTS system REQUIRED)#[[]] + +set(SRCS "${LIBRARY_DIR}/DNSResolver.cpp") + +set(HDRS "${LIBRARY_DIR}/DNSResolver.hpp") + +add_library(_cpp-dns ${SRCS} ${HDRS}) + +#message("Libraries: lb: ${LIBRARY_DIR} bid: ${Boost_INCLUDE_DIRS} bl: ${Boost_LIBRARIES} libudns: ${lib_udns}") + +target_link_libraries(_cpp-dns boost::system boost::headers_only ch_contrib::udns) + +target_include_directories(_cpp-dns SYSTEM BEFORE PUBLIC ${LIBRARY_DIR}) + +add_library(ch_contrib::cpp-dns ALIAS _cpp-dns) \ No newline at end of file diff --git a/contrib/udns-cmake/CMakeLists.txt b/contrib/udns-cmake/CMakeLists.txt new file mode 100644 index 00000000000..d56be90e11f --- /dev/null +++ b/contrib/udns-cmake/CMakeLists.txt @@ -0,0 +1,54 @@ +set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/udns") + +#add_library(udns +# udns_dn.c +# udns_dntosp.c +# udns_parse.c +# udns_resolver.c +# udns_init.c +# udns_misc.c +# udns_XtoX.c +# udns_rr_a.c +# udns_rr_ptr.c +# udns_rr_mx.c +# udns_rr_txt.c +# udns_bl.c +# udns_rr_srv.c +# udns_rr_naptr.c +# udns_codes.c +# udns_jran.c +# ) + +#add_library(_udns +# "${LIBRARY_DIR}/udns_dn.c" +# "${LIBRARY_DIR}/dnsget.c" "${LIBRARY_DIR}/ex-rdns.c" "${LIBRARY_DIR}/getopt.c" "${LIBRARY_DIR}/inet_XtoX.c" +# "${LIBRARY_DIR}/rblcheck.c" "${LIBRARY_DIR}/udns_bl.c" "${LIBRARY_DIR}/udns_dntosp.c" +# "${LIBRARY_DIR}/udns_init.c" "${LIBRARY_DIR}/udns_jran.c" "${LIBRARY_DIR}/udns_misc.c" "${LIBRARY_DIR}/udns_parse.c" +# "${LIBRARY_DIR}/udns_resolver.c" "${LIBRARY_DIR}/udns_rr_a.c" "${LIBRARY_DIR}/udns_rr_mx.c" +# "${LIBRARY_DIR}/udns_rr_naptr.c" "${LIBRARY_DIR}/udns_rr_ptr.c" "${LIBRARY_DIR}/udns_rr_srv.c" "${LIBRARY_DIR}/udns_rr_txt.c" +# "${LIBRARY_DIR}/udns_XtoX.c" ) + +add_library(_udns + "${LIBRARY_DIR}/udns_dn.c" + "${LIBRARY_DIR}/udns_dntosp.c" + "${LIBRARY_DIR}/udns_parse.c" + "${LIBRARY_DIR}/udns_resolver.c" + "${LIBRARY_DIR}/udns_init.c" + "${LIBRARY_DIR}/udns_misc.c" + "${LIBRARY_DIR}/udns_XtoX.c" + "${LIBRARY_DIR}/udns_rr_a.c" + "${LIBRARY_DIR}/udns_rr_ptr.c" + "${LIBRARY_DIR}/udns_rr_mx.c" + "${LIBRARY_DIR}/udns_rr_txt.c" + "${LIBRARY_DIR}/udns_bl.c" + "${LIBRARY_DIR}/udns_rr_srv.c" + "${LIBRARY_DIR}/udns_rr_naptr.c" + "${LIBRARY_DIR}/udns_codes.c" + "${LIBRARY_DIR}/udns_jran.c" + ) + +target_compile_definitions(_udns PRIVATE -DHAVE_CONFIG_H) + +target_include_directories(_udns SYSTEM BEFORE PUBLIC ${LIBRARY_DIR}) + +add_library(ch_contrib::udns ALIAS _udns) \ No newline at end of file diff --git a/contrib/udns/COPYING.LGPL b/contrib/udns/COPYING.LGPL new file mode 100644 index 00000000000..4362b49151d --- /dev/null +++ b/contrib/udns/COPYING.LGPL @@ -0,0 +1,502 @@ + GNU LESSER GENERAL PUBLIC LICENSE + Version 2.1, February 1999 + + Copyright (C) 1991, 1999 Free Software Foundation, Inc. + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA + Everyone is permitted to copy and distribute verbatim copies + of this license document, but changing it is not allowed. + +[This is the first released version of the Lesser GPL. It also counts + as the successor of the GNU Library Public License, version 2, hence + the version number 2.1.] + + Preamble + + The licenses for most software are designed to take away your +freedom to share and change it. By contrast, the GNU General Public +Licenses are intended to guarantee your freedom to share and change +free software--to make sure the software is free for all its users. + + This license, the Lesser General Public License, applies to some +specially designated software packages--typically libraries--of the +Free Software Foundation and other authors who decide to use it. You +can use it too, but we suggest you first think carefully about whether +this license or the ordinary General Public License is the better +strategy to use in any particular case, based on the explanations below. + + When we speak of free software, we are referring to freedom of use, +not price. Our General Public Licenses are designed to make sure that +you have the freedom to distribute copies of free software (and charge +for this service if you wish); that you receive source code or can get +it if you want it; that you can change the software and use pieces of +it in new free programs; and that you are informed that you can do +these things. + + To protect your rights, we need to make restrictions that forbid +distributors to deny you these rights or to ask you to surrender these +rights. These restrictions translate to certain responsibilities for +you if you distribute copies of the library or if you modify it. + + For example, if you distribute copies of the library, whether gratis +or for a fee, you must give the recipients all the rights that we gave +you. You must make sure that they, too, receive or can get the source +code. If you link other code with the library, you must provide +complete object files to the recipients, so that they can relink them +with the library after making changes to the library and recompiling +it. And you must show them these terms so they know their rights. + + We protect your rights with a two-step method: (1) we copyright the +library, and (2) we offer you this license, which gives you legal +permission to copy, distribute and/or modify the library. + + To protect each distributor, we want to make it very clear that +there is no warranty for the free library. Also, if the library is +modified by someone else and passed on, the recipients should know +that what they have is not the original version, so that the original +author's reputation will not be affected by problems that might be +introduced by others. + + Finally, software patents pose a constant threat to the existence of +any free program. We wish to make sure that a company cannot +effectively restrict the users of a free program by obtaining a +restrictive license from a patent holder. Therefore, we insist that +any patent license obtained for a version of the library must be +consistent with the full freedom of use specified in this license. + + Most GNU software, including some libraries, is covered by the +ordinary GNU General Public License. This license, the GNU Lesser +General Public License, applies to certain designated libraries, and +is quite different from the ordinary General Public License. We use +this license for certain libraries in order to permit linking those +libraries into non-free programs. + + When a program is linked with a library, whether statically or using +a shared library, the combination of the two is legally speaking a +combined work, a derivative of the original library. The ordinary +General Public License therefore permits such linking only if the +entire combination fits its criteria of freedom. The Lesser General +Public License permits more lax criteria for linking other code with +the library. + + We call this license the "Lesser" General Public License because it +does Less to protect the user's freedom than the ordinary General +Public License. It also provides other free software developers Less +of an advantage over competing non-free programs. These disadvantages +are the reason we use the ordinary General Public License for many +libraries. However, the Lesser license provides advantages in certain +special circumstances. + + For example, on rare occasions, there may be a special need to +encourage the widest possible use of a certain library, so that it becomes +a de-facto standard. To achieve this, non-free programs must be +allowed to use the library. A more frequent case is that a free +library does the same job as widely used non-free libraries. In this +case, there is little to gain by limiting the free library to free +software only, so we use the Lesser General Public License. + + In other cases, permission to use a particular library in non-free +programs enables a greater number of people to use a large body of +free software. For example, permission to use the GNU C Library in +non-free programs enables many more people to use the whole GNU +operating system, as well as its variant, the GNU/Linux operating +system. + + Although the Lesser General Public License is Less protective of the +users' freedom, it does ensure that the user of a program that is +linked with the Library has the freedom and the wherewithal to run +that program using a modified version of the Library. + + The precise terms and conditions for copying, distribution and +modification follow. Pay close attention to the difference between a +"work based on the library" and a "work that uses the library". The +former contains code derived from the library, whereas the latter must +be combined with the library in order to run. + + GNU LESSER GENERAL PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. This License Agreement applies to any software library or other +program which contains a notice placed by the copyright holder or +other authorized party saying it may be distributed under the terms of +this Lesser General Public License (also called "this License"). +Each licensee is addressed as "you". + + A "library" means a collection of software functions and/or data +prepared so as to be conveniently linked with application programs +(which use some of those functions and data) to form executables. + + The "Library", below, refers to any such software library or work +which has been distributed under these terms. A "work based on the +Library" means either the Library or any derivative work under +copyright law: that is to say, a work containing the Library or a +portion of it, either verbatim or with modifications and/or translated +straightforwardly into another language. (Hereinafter, translation is +included without limitation in the term "modification".) + + "Source code" for a work means the preferred form of the work for +making modifications to it. For a library, complete source code means +all the source code for all modules it contains, plus any associated +interface definition files, plus the scripts used to control compilation +and installation of the library. + + Activities other than copying, distribution and modification are not +covered by this License; they are outside its scope. The act of +running a program using the Library is not restricted, and output from +such a program is covered only if its contents constitute a work based +on the Library (independent of the use of the Library in a tool for +writing it). Whether that is true depends on what the Library does +and what the program that uses the Library does. + + 1. You may copy and distribute verbatim copies of the Library's +complete source code as you receive it, in any medium, provided that +you conspicuously and appropriately publish on each copy an +appropriate copyright notice and disclaimer of warranty; keep intact +all the notices that refer to this License and to the absence of any +warranty; and distribute a copy of this License along with the +Library. + + You may charge a fee for the physical act of transferring a copy, +and you may at your option offer warranty protection in exchange for a +fee. + + 2. You may modify your copy or copies of the Library or any portion +of it, thus forming a work based on the Library, and copy and +distribute such modifications or work under the terms of Section 1 +above, provided that you also meet all of these conditions: + + a) The modified work must itself be a software library. + + b) You must cause the files modified to carry prominent notices + stating that you changed the files and the date of any change. + + c) You must cause the whole of the work to be licensed at no + charge to all third parties under the terms of this License. + + d) If a facility in the modified Library refers to a function or a + table of data to be supplied by an application program that uses + the facility, other than as an argument passed when the facility + is invoked, then you must make a good faith effort to ensure that, + in the event an application does not supply such function or + table, the facility still operates, and performs whatever part of + its purpose remains meaningful. + + (For example, a function in a library to compute square roots has + a purpose that is entirely well-defined independent of the + application. Therefore, Subsection 2d requires that any + application-supplied function or table used by this function must + be optional: if the application does not supply it, the square + root function must still compute square roots.) + +These requirements apply to the modified work as a whole. If +identifiable sections of that work are not derived from the Library, +and can be reasonably considered independent and separate works in +themselves, then this License, and its terms, do not apply to those +sections when you distribute them as separate works. But when you +distribute the same sections as part of a whole which is a work based +on the Library, the distribution of the whole must be on the terms of +this License, whose permissions for other licensees extend to the +entire whole, and thus to each and every part regardless of who wrote +it. + +Thus, it is not the intent of this section to claim rights or contest +your rights to work written entirely by you; rather, the intent is to +exercise the right to control the distribution of derivative or +collective works based on the Library. + +In addition, mere aggregation of another work not based on the Library +with the Library (or with a work based on the Library) on a volume of +a storage or distribution medium does not bring the other work under +the scope of this License. + + 3. You may opt to apply the terms of the ordinary GNU General Public +License instead of this License to a given copy of the Library. To do +this, you must alter all the notices that refer to this License, so +that they refer to the ordinary GNU General Public License, version 2, +instead of to this License. (If a newer version than version 2 of the +ordinary GNU General Public License has appeared, then you can specify +that version instead if you wish.) Do not make any other change in +these notices. + + Once this change is made in a given copy, it is irreversible for +that copy, so the ordinary GNU General Public License applies to all +subsequent copies and derivative works made from that copy. + + This option is useful when you wish to copy part of the code of +the Library into a program that is not a library. + + 4. You may copy and distribute the Library (or a portion or +derivative of it, under Section 2) in object code or executable form +under the terms of Sections 1 and 2 above provided that you accompany +it with the complete corresponding machine-readable source code, which +must be distributed under the terms of Sections 1 and 2 above on a +medium customarily used for software interchange. + + If distribution of object code is made by offering access to copy +from a designated place, then offering equivalent access to copy the +source code from the same place satisfies the requirement to +distribute the source code, even though third parties are not +compelled to copy the source along with the object code. + + 5. A program that contains no derivative of any portion of the +Library, but is designed to work with the Library by being compiled or +linked with it, is called a "work that uses the Library". Such a +work, in isolation, is not a derivative work of the Library, and +therefore falls outside the scope of this License. + + However, linking a "work that uses the Library" with the Library +creates an executable that is a derivative of the Library (because it +contains portions of the Library), rather than a "work that uses the +library". The executable is therefore covered by this License. +Section 6 states terms for distribution of such executables. + + When a "work that uses the Library" uses material from a header file +that is part of the Library, the object code for the work may be a +derivative work of the Library even though the source code is not. +Whether this is true is especially significant if the work can be +linked without the Library, or if the work is itself a library. The +threshold for this to be true is not precisely defined by law. + + If such an object file uses only numerical parameters, data +structure layouts and accessors, and small macros and small inline +functions (ten lines or less in length), then the use of the object +file is unrestricted, regardless of whether it is legally a derivative +work. (Executables containing this object code plus portions of the +Library will still fall under Section 6.) + + Otherwise, if the work is a derivative of the Library, you may +distribute the object code for the work under the terms of Section 6. +Any executables containing that work also fall under Section 6, +whether or not they are linked directly with the Library itself. + + 6. As an exception to the Sections above, you may also combine or +link a "work that uses the Library" with the Library to produce a +work containing portions of the Library, and distribute that work +under terms of your choice, provided that the terms permit +modification of the work for the customer's own use and reverse +engineering for debugging such modifications. + + You must give prominent notice with each copy of the work that the +Library is used in it and that the Library and its use are covered by +this License. You must supply a copy of this License. If the work +during execution displays copyright notices, you must include the +copyright notice for the Library among them, as well as a reference +directing the user to the copy of this License. Also, you must do one +of these things: + + a) Accompany the work with the complete corresponding + machine-readable source code for the Library including whatever + changes were used in the work (which must be distributed under + Sections 1 and 2 above); and, if the work is an executable linked + with the Library, with the complete machine-readable "work that + uses the Library", as object code and/or source code, so that the + user can modify the Library and then relink to produce a modified + executable containing the modified Library. (It is understood + that the user who changes the contents of definitions files in the + Library will not necessarily be able to recompile the application + to use the modified definitions.) + + b) Use a suitable shared library mechanism for linking with the + Library. A suitable mechanism is one that (1) uses at run time a + copy of the library already present on the user's computer system, + rather than copying library functions into the executable, and (2) + will operate properly with a modified version of the library, if + the user installs one, as long as the modified version is + interface-compatible with the version that the work was made with. + + c) Accompany the work with a written offer, valid for at + least three years, to give the same user the materials + specified in Subsection 6a, above, for a charge no more + than the cost of performing this distribution. + + d) If distribution of the work is made by offering access to copy + from a designated place, offer equivalent access to copy the above + specified materials from the same place. + + e) Verify that the user has already received a copy of these + materials or that you have already sent this user a copy. + + For an executable, the required form of the "work that uses the +Library" must include any data and utility programs needed for +reproducing the executable from it. However, as a special exception, +the materials to be distributed need not include anything that is +normally distributed (in either source or binary form) with the major +components (compiler, kernel, and so on) of the operating system on +which the executable runs, unless that component itself accompanies +the executable. + + It may happen that this requirement contradicts the license +restrictions of other proprietary libraries that do not normally +accompany the operating system. Such a contradiction means you cannot +use both them and the Library together in an executable that you +distribute. + + 7. You may place library facilities that are a work based on the +Library side-by-side in a single library together with other library +facilities not covered by this License, and distribute such a combined +library, provided that the separate distribution of the work based on +the Library and of the other library facilities is otherwise +permitted, and provided that you do these two things: + + a) Accompany the combined library with a copy of the same work + based on the Library, uncombined with any other library + facilities. This must be distributed under the terms of the + Sections above. + + b) Give prominent notice with the combined library of the fact + that part of it is a work based on the Library, and explaining + where to find the accompanying uncombined form of the same work. + + 8. You may not copy, modify, sublicense, link with, or distribute +the Library except as expressly provided under this License. Any +attempt otherwise to copy, modify, sublicense, link with, or +distribute the Library is void, and will automatically terminate your +rights under this License. However, parties who have received copies, +or rights, from you under this License will not have their licenses +terminated so long as such parties remain in full compliance. + + 9. You are not required to accept this License, since you have not +signed it. However, nothing else grants you permission to modify or +distribute the Library or its derivative works. These actions are +prohibited by law if you do not accept this License. Therefore, by +modifying or distributing the Library (or any work based on the +Library), you indicate your acceptance of this License to do so, and +all its terms and conditions for copying, distributing or modifying +the Library or works based on it. + + 10. Each time you redistribute the Library (or any work based on the +Library), the recipient automatically receives a license from the +original licensor to copy, distribute, link with or modify the Library +subject to these terms and conditions. You may not impose any further +restrictions on the recipients' exercise of the rights granted herein. +You are not responsible for enforcing compliance by third parties with +this License. + + 11. If, as a consequence of a court judgment or allegation of patent +infringement or for any other reason (not limited to patent issues), +conditions are imposed on you (whether by court order, agreement or +otherwise) that contradict the conditions of this License, they do not +excuse you from the conditions of this License. If you cannot +distribute so as to satisfy simultaneously your obligations under this +License and any other pertinent obligations, then as a consequence you +may not distribute the Library at all. For example, if a patent +license would not permit royalty-free redistribution of the Library by +all those who receive copies directly or indirectly through you, then +the only way you could satisfy both it and this License would be to +refrain entirely from distribution of the Library. + +If any portion of this section is held invalid or unenforceable under any +particular circumstance, the balance of the section is intended to apply, +and the section as a whole is intended to apply in other circumstances. + +It is not the purpose of this section to induce you to infringe any +patents or other property right claims or to contest validity of any +such claims; this section has the sole purpose of protecting the +integrity of the free software distribution system which is +implemented by public license practices. Many people have made +generous contributions to the wide range of software distributed +through that system in reliance on consistent application of that +system; it is up to the author/donor to decide if he or she is willing +to distribute software through any other system and a licensee cannot +impose that choice. + +This section is intended to make thoroughly clear what is believed to +be a consequence of the rest of this License. + + 12. If the distribution and/or use of the Library is restricted in +certain countries either by patents or by copyrighted interfaces, the +original copyright holder who places the Library under this License may add +an explicit geographical distribution limitation excluding those countries, +so that distribution is permitted only in or among countries not thus +excluded. In such case, this License incorporates the limitation as if +written in the body of this License. + + 13. The Free Software Foundation may publish revised and/or new +versions of the Lesser General Public License from time to time. +Such new versions will be similar in spirit to the present version, +but may differ in detail to address new problems or concerns. + +Each version is given a distinguishing version number. If the Library +specifies a version number of this License which applies to it and +"any later version", you have the option of following the terms and +conditions either of that version or of any later version published by +the Free Software Foundation. If the Library does not specify a +license version number, you may choose any version ever published by +the Free Software Foundation. + + 14. If you wish to incorporate parts of the Library into other free +programs whose distribution conditions are incompatible with these, +write to the author to ask for permission. For software which is +copyrighted by the Free Software Foundation, write to the Free +Software Foundation; we sometimes make exceptions for this. Our +decision will be guided by the two goals of preserving the free status +of all derivatives of our free software and of promoting the sharing +and reuse of software generally. + + NO WARRANTY + + 15. BECAUSE THE LIBRARY IS LICENSED FREE OF CHARGE, THERE IS NO +WARRANTY FOR THE LIBRARY, TO THE EXTENT PERMITTED BY APPLICABLE LAW. +EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR +OTHER PARTIES PROVIDE THE LIBRARY "AS IS" WITHOUT WARRANTY OF ANY +KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR +PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE +LIBRARY IS WITH YOU. SHOULD THE LIBRARY PROVE DEFECTIVE, YOU ASSUME +THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION. + + 16. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN +WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY +AND/OR REDISTRIBUTE THE LIBRARY AS PERMITTED ABOVE, BE LIABLE TO YOU +FOR DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR +CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE +LIBRARY (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING +RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A +FAILURE OF THE LIBRARY TO OPERATE WITH ANY OTHER SOFTWARE), EVEN IF +SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH +DAMAGES. + + END OF TERMS AND CONDITIONS + + How to Apply These Terms to Your New Libraries + + If you develop a new library, and you want it to be of the greatest +possible use to the public, we recommend making it free software that +everyone can redistribute and change. You can do so by permitting +redistribution under these terms (or, alternatively, under the terms of the +ordinary General Public License). + + To apply these terms, attach the following notices to the library. It is +safest to attach them to the start of each source file to most effectively +convey the exclusion of warranty; and each file should have at least the +"copyright" line and a pointer to where the full notice is found. + + + Copyright (C) + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library; if not, write to the Free Software + Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA + +Also add information on how to contact you by electronic and paper mail. + +You should also get your employer (if you work as a programmer) or your +school, if any, to sign a "copyright disclaimer" for the library, if +necessary. Here is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the + library `Frob' (a library for tweaking knobs) written by James Random Hacker. + + , 1 April 1990 + Ty Coon, President of Vice + +That's all there is to it! diff --git a/contrib/udns/Makefile.in b/contrib/udns/Makefile.in new file mode 100644 index 00000000000..ec085206655 --- /dev/null +++ b/contrib/udns/Makefile.in @@ -0,0 +1,197 @@ +#! /usr/bin/make -rf +# Makefile.in +# libudns Makefile +# +# Copyright (C) 2005 Michael Tokarev +# This file is part of UDNS library, an async DNS stub resolver. +# +# This library is free software; you can redistribute it and/or +# modify it under the terms of the GNU Lesser General Public +# License as published by the Free Software Foundation; either +# version 2.1 of the License, or (at your option) any later version. +# +# This library is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU +# Lesser General Public License for more details. +# +# You should have received a copy of the GNU Lesser General Public +# License along with this library, in file named COPYING.LGPL; if not, +# write to the Free Software Foundation, Inc., 59 Temple Place, +# Suite 330, Boston, MA 02111-1307 USA + +NAME = udns +VERS = 0.4 +SOVER = 0 + +SRCS = udns_dn.c udns_dntosp.c udns_parse.c udns_resolver.c udns_init.c \ + udns_misc.c udns_XtoX.c \ + udns_rr_a.c udns_rr_ptr.c udns_rr_mx.c udns_rr_txt.c udns_bl.c \ + udns_rr_srv.c udns_rr_naptr.c udns_codes.c udns_jran.c +USRCS = dnsget.c rblcheck.c ex-rdns.c +DIST = COPYING.LGPL udns.h udns.3 dnsget.1 rblcheck.1 $(SRCS) $(USRCS) \ + NEWS TODO NOTES Makefile.in configure configure.lib \ + inet_XtoX.c getopt.c + +OBJS = $(SRCS:.c=.o) $(GEN:.c=.o) +LIB = lib$(NAME).a +LIBFL = -L. -l$(NAME) + +SOBJS = $(OBJS:.o=.lo) +SOLIB = lib$(NAME)_s.so +SOLIBV = lib$(NAME).so.$(SOVER) +SOLIBFL= -L. -l$(NAME)_s + +UTILS = $(USRCS:.c=) +UOBJS = $(USRCS:.c=.o) +SOUTILS = $(USRCS:.c=_s) + +NAMEPFX = $(NAME)-$(VERS) + +CC = @CC@ +CFLAGS = @CFLAGS@ +CDEFS = @CDEFS@ +LD = @LD@ +LDFLAGS = @LDFLAGS@ +LIBS = @LIBS@ +LDSHARED = $(LD) -shared +PICFLAGS = -fPIC +AWK = awk +TAR = tar + +all: static + +.SUFFIXES: .c .o .lo + +static: $(LIB) $(UTILS) +staticlib: $(LIB) +$(LIB): $(OBJS) + -rm -f $@ + $(AR) rv $@ $(OBJS) +.c.o: + $(CC) $(CFLAGS) $(CDEFS) -c $< + +shared: $(SOLIBV) $(SOUTILS) +sharedlib: $(SOLIBV) + +$(SOLIBV): $(SOBJS) + $(LDSHARED) -Wl,--soname,$(SOLIBV) -o $@ $(SOBJS) $(LDFLAGS) $(LIBS) +$(SOLIB): $(SOLIBV) + rm -f $@ + ln -s $(SOLIBV) $@ +.c.lo: + $(CC) $(CFLAGS) $(PICFLAGS) $(CDEFS) -o $@ -c $< + +# udns_codes.c is generated from udns.h +udns_codes.c: udns.h + @echo Generating $@ + @set -e; exec >$@.tmp; \ + set T type C class R rcode; \ + echo "/* Automatically generated. */"; \ + echo "#include \"udns.h\""; \ + while [ "$$1" ]; do \ + echo; \ + echo "const struct dns_nameval dns_$${2}tab[] = {"; \ + $(AWK) "/^ DNS_$${1}_[A-Z0-9_]+[ ]*=/ \ + { printf \" {%s,\\\"%s\\\"},\\n\", \$$1, substr(\$$1,7) }" \ + udns.h ; \ + echo " {0,0}};"; \ + echo "const char *dns_$${2}name(enum dns_$${2} code) {"; \ + echo " static char nm[20];"; \ + echo " switch(code) {"; \ + $(AWK) "BEGIN{i=0} \ + /^ DNS_$${1}_[A-Z0-9_]+[ ]*=/ \ + {printf \" case %s: return dns_$${2}tab[%d].name;\\n\",\$$1,i++}\ + " udns.h ; \ + echo " }"; \ + echo " return _dns_format_code(nm,\"$$2\",code);"; \ + echo "}"; \ + shift 2; \ + done + @mv $@.tmp $@ + +udns.3.html: udns.3 + groff -man -Thtml udns.3 > $@.tmp + mv $@.tmp $@ + +dist: $(NAMEPFX).tar.gz +$(NAMEPFX).tar.gz: $(DIST) + $(TAR) -cv -f $@ -z --transform 's|^|$(NAMEPFX)/|' $(DIST) + +subdist: + cp -p $(DIST) $(TARGET)/ + +clean: + rm -f $(OBJS) + rm -f $(SOBJS) + rm -f $(UOBJS) + rm -f config.log +distclean: clean + rm -f $(LIB) $(SOLIB) $(SOLIBV) udns.3.html + rm -f $(UTILS) $(SOUTILS) + rm -f config.status config.h Makefile + + +Makefile: configure configure.lib Makefile.in + ./configure + @echo + @echo Please rerun make >&2 + @exit 1 + +.PHONY: all static staticlib shared sharedlib dist clean distclean subdist \ + depend dep deps + +depend dep deps: $(SRCS) $(USRC) + @echo Generating deps for: + @echo \ $(SRCS) + @echo \ $(USRCS) + @sed '/^# depend/q' Makefile.in > Makefile.tmp + @set -e; \ + for f in $(SRCS) $(USRCS); do \ + echo $${f%.c}.o $${f%.c}.lo: $$f \ + `sed -n 's/^#[ ]*include[ ]*"\(.*\)".*/\1/p' $$f`; \ + done >> Makefile.tmp; \ + for f in $(USRCS:.c=.o); do \ + echo "$${f%.?}: $$f \$$(LIB)"; \ + echo " \$$(LD) \$$(LDLAGS) -o \$$@ $$f \$$(LIBFL) \$$(LIBS)"; \ + echo "$${f%.?}_s: $$f \$$(SOLIB)"; \ + echo " \$$(LD) \$$(LDFLAGS) -o \$$@ $$f \$$(SOLIBFL)"; \ + done >> Makefile.tmp ; \ + if cmp Makefile.tmp Makefile.in >/dev/null 2>&1 ; then \ + echo Makefile.in unchanged; rm -f Makefile.tmp; \ + else \ + echo Updating Makfile.in; mv -f Makefile.tmp Makefile.in ; \ + fi + +# depend +udns_dn.o udns_dn.lo: udns_dn.c udns.h +udns_dntosp.o udns_dntosp.lo: udns_dntosp.c udns.h +udns_parse.o udns_parse.lo: udns_parse.c udns.h +udns_resolver.o udns_resolver.lo: udns_resolver.c config.h udns.h +udns_init.o udns_init.lo: udns_init.c config.h udns.h +udns_misc.o udns_misc.lo: udns_misc.c udns.h +udns_XtoX.o udns_XtoX.lo: udns_XtoX.c config.h udns.h inet_XtoX.c +udns_rr_a.o udns_rr_a.lo: udns_rr_a.c udns.h +udns_rr_ptr.o udns_rr_ptr.lo: udns_rr_ptr.c udns.h +udns_rr_mx.o udns_rr_mx.lo: udns_rr_mx.c udns.h +udns_rr_txt.o udns_rr_txt.lo: udns_rr_txt.c udns.h +udns_bl.o udns_bl.lo: udns_bl.c udns.h +udns_rr_srv.o udns_rr_srv.lo: udns_rr_srv.c udns.h +udns_rr_naptr.o udns_rr_naptr.lo: udns_rr_naptr.c udns.h +udns_codes.o udns_codes.lo: udns_codes.c udns.h +udns_jran.o udns_jran.lo: udns_jran.c udns.h +dnsget.o dnsget.lo: dnsget.c config.h udns.h getopt.c +rblcheck.o rblcheck.lo: rblcheck.c config.h udns.h getopt.c +ex-rdns.o ex-rdns.lo: ex-rdns.c udns.h +dnsget: dnsget.o $(LIB) + $(LD) $(LDLAGS) -o $@ dnsget.o $(LIBFL) $(LIBS) +dnsget_s: dnsget.o $(SOLIB) + $(LD) $(LDFLAGS) -o $@ dnsget.o $(SOLIBFL) +rblcheck: rblcheck.o $(LIB) + $(LD) $(LDLAGS) -o $@ rblcheck.o $(LIBFL) $(LIBS) +rblcheck_s: rblcheck.o $(SOLIB) + $(LD) $(LDFLAGS) -o $@ rblcheck.o $(SOLIBFL) +ex-rdns: ex-rdns.o $(LIB) + $(LD) $(LDLAGS) -o $@ ex-rdns.o $(LIBFL) $(LIBS) +ex-rdns_s: ex-rdns.o $(SOLIB) + $(LD) $(LDFLAGS) -o $@ ex-rdns.o $(SOLIBFL) diff --git a/contrib/udns/NEWS b/contrib/udns/NEWS new file mode 100644 index 00000000000..88aff6fa420 --- /dev/null +++ b/contrib/udns/NEWS @@ -0,0 +1,136 @@ +NEWS +User-visible changes in udns library. Recent changes on top. + +0.4 (Jan 2014) + + - bugfix: fix a bug in new list code introduced in 0.3 + - portability: use $(LD)/$(LDFLAGS)/$(LIBS) + +0.3 (Jan 2014) + + - bugfix: refactor double-linked list implementation in udns_resolver.c + (internal to the library) to be more strict-aliasing-friendly, because + old code were miscompiled by gcc. + + - bugfix: forgotten strdup() in rblcheck + +0.2 (Dec 2011) + + - bugfix: SRV RR handling: fix domain name parsing and crash in case + if no port is specified on input for SRV record query + + - (trivial api) dns_set_opts() now returns number of unrecognized + options instead of always returning 0 + + - dnsget: combine -f and -o options in dnsget (and stop documenting -f), + and report unknown/invalid -o options (and error out) + + - dnsget: pretty-print SSHFP RRs + + 0.1 (Dec 2010) + + - bugfix: udns_new(old) - when actually cloning another context - + makes the new context referencing memory from old, which leads + to crashes when old is modified later + + - use random queue IDs (the 16bit qID) in queries instead of sequentional + ones, based on simple pseudo-random RNG by Bob Jenkins (udns_jran.[ch]). + Some people believe that this improves security (CVE-2008-1447). I'm + still not convinced (see comments in udns_resolver.c), but it isn't + difficult to add after all. + + - deprecate dns_random16() function which was declared in udns.h + (not anymore) but never documented. In order to keep ABI compatible + it is still exported. + + - library has a way now to set query flags (DNS_SET_DO; DNS_SET_CD). + + - dnsget now prints non-printable chars in all strings in DNS RRs using + decimal escape sequences (\%03u) instead of hexadecimal (\%02x) when + before - other DNS software does it like this. + + - recognize a few more record types in dnsget, notable some DNSSEC RRs; + add -f option for dnsget to set query flags. + + - udns is not a Debian native package anymore (was a wrong idea) + +0.0.9 (16 Jan 2007) + + - incompat: minor API changes in dns_init() &friends. dns_init() + now requires extra `struct dns_ctx *' argument. Not bumped + soversion yet - I only expect one "release" with this change. + + - many small bugfixes, here and there + + - more robust FORMERR replies handling - not only such replies are now + recognized, but udns retries queries without EDNS0 extensions if tried + with, but server reported FORMERR + + - portability changes, udns now includes getopt() implementation fo + the systems lacking it (mostly windows), and dns_ntop()&dns_pton(), + which are either just wrappers for system functions or reimplementations. + + - build is now based on autoconf-like configuration + + - NAPTR (RFC3403) RR decoding support + + - new file NOTES which complements TODO somewhat, and includes some + important shortcomings + + - many internal cleanups, including some preparations for better error + recovery, security and robustness (and thus API changes) + + - removed some #defines which are now unused (like DNS_MAXSRCH) + + - changed WIN32 to WINDOWS everywhere in preprocessor tests, + to be able to build it on win64 as well + +0.0.8 (12 Sep 2005) + + - added SRV records (rfc2782) parsing, + thanks to Thadeu Lima de Souza Cascardo for implementation. + + - bugfixes: + o use uninitialized value when no reply, library died with assertion: + assert((status < 0 && result == 0) || (status >= 0 && result != 0)). + o on some OSes, struct sockaddr_in has additional fields, so + memcmp'ing two sockaddresses does not work. + + - rblcheck(.1) + +0.0.7 (20 Apr 2005) + + - dnsget.1 manpage and several enhancements to dnsget. + + - allow nameserver names for -n option of dnsget. + + - API change: all dns_submit*() routines now does not expect + last `now' argument, since requests aren't sent immediately + anymore. + + - API change: different application timer callback mechanism. + Udns now uses single per-context timer instead of per-query. + + - don't assume DNS replies only contain backward DN pointers, + allow forward pointers too. Change parsing API. + + - debianize + +0.0.6 (08 Apr 2005) + + - use double sorted list for requests (sorted by deadline). + This should significantly speed up timeout processing for + large number of requests. + + - changed debugging interface, so it is finally useable + (still not documented). + + - dnsget routine is now Officially Useable, and sometimes + even more useable than `host' from BIND distribution + (and sometimes not - dnsget does not have -C option + and TCP mode) + + - Debian packaging in debian/ -- udns is now maintained as a + native Debian package. + + - alot (and I really mean alot) of code cleanups all over. diff --git a/contrib/udns/NOTES b/contrib/udns/NOTES new file mode 100644 index 00000000000..b99a077bb22 --- /dev/null +++ b/contrib/udns/NOTES @@ -0,0 +1,226 @@ +Assorted notes about udns (library). + +UDP-only mode +~~~~~~~~~~~~~ + +First of all, since udns is (currently) UDP-only, there are some +shortcomings. + +It assumes that a reply will fit into a UDP buffer. With adoption of EDNS0, +and general robustness of IP stacks, in most cases it's not an issue. But +in some cases there may be problems: + + - if an RRset is "very large" so it does not fit even in buffer of size + requested by the library (current default is 4096; some servers limits + it further), we will not see the reply, or will only see "damaged" + reply (depending on the server). + + - many DNS servers ignores EDNS0 option requests. In this case, no matter + which buffer size udns library will request, such servers reply is limited + to 512 bytes (standard pre-EDNS0 DNS packet size). (Udns falls back to + non-EDNO0 query if EDNS0-enabled one received FORMERR or NOTIMPL error). + +The problem is that with this, udns currently will not consider replies with +TC (truncation) bit set, and will treat such replies the same way as it +treats SERVFAIL replies, thus trying next server, or temp-failing the query +if no more servers to try. In other words, if the reply is really large, or +if the servers you're using don't support EDNS0, your application will be +unable to resolve a given name. + +Yet it's not common situation - in practice, it's very rare. + +Implementing TCP mode isn't difficult, but it complicates API significantly. +Currently udns uses only single UDP socket (or - maybe in the future - two, +see below), but in case of TCP, it will need to open and close sockets for +TCP connections left and right, and that have to be integrated into an +application's event loop in an easy and efficient way. Plus all the +timeouts - different for connect(), write, and several stages of read. + +IPv6 vs IPv4 usage +~~~~~~~~~~~~~~~~~~ + +This is only relevant for nameservers reachable over IPv6, NOT for IPv6 +queries. I.e., if you've IPv6 addresses in 'nameservers' line in your +/etc/resolv.conf file. Even more: if you have BOTH IPv6 AND IPv4 addresses +there. Or pass them to udns initialization routines. + +Since udns uses a single UDP socket to communicate with all nameservers, +it should support both v4 and v6 communications. Most current platforms +supports this mode - using PF_INET6 socket and V4MAPPED addresses, i.e, +"tunnelling" IPv4 inside IPv6. But not all systems supports this. And +more, it has been said that such mode is deprecated. + +So, list only IPv4 or only IPv6 addresses, but don't mix them, in your +/etc/resolv.conf. + +An alternative is to use two sockets instead of 1 - one for IPv6 and one +for IPv4. For now I'm not sure if it's worth the complexity - again, of +the API, not the library itself (but this will not simplify library either). + +Single socket for all queries +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Using single UDP socket for sending queries to all nameservers has obvious +advantages. First it's, again, trivial, simple to use API. And simple +library too. Also, after sending queries to all nameservers (in case first +didn't reply in time), we will be able to receive late reply from first +nameserver and accept it. + +But this mode has disadvantages too. Most important is that it's much easier +to send fake reply to us, as the UDP port where we expects the reply to come +to is constant during the whole lifetime of an application. More secure +implementations uses random port for every single query. While port number +(16 bits integer) can not hold much randomness, it's still of some help. +Ok, udns is a stub resolver, so it expects sorta friendly environment, but +on LAN it's usually much easier to fire an attack, due to the speed of local +network, where a bad guy can generate alot of packets in a short time. + +Spoofing of replies (Kaminsky attack, CVE-2008-1447) +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +While udns uses random numbers for query IDs, it uses single UDP port for +all queries (see previous item). And even if it used random UDP port for +each query, the attack described in CVE-2008-1447 is still quite trivial. +This is not specific to udns library unfortunately - it is inherent property +of the protocol. Udns is designed to work in a LAN, it needs full recursive +resolver nearby, and modern LAN usually uses high-bandwidth equipment which +makes the Kaminsky attack trivial. The problem is that even with qID (16 +bits) and random UDP port (about 20 bits available to a regular process) +combined still can not hold enough randomness, so on a fast network it is +still easy to flood the target with fake replies and hit the "right" reply +before real reply comes. So random qIDs don't add much protection anyway, +even if this feature is implemented in udns, and using all available +techniques wont solve it either. + +See also long comment in udns_resolver.c, udns_newid(). + +Assumptions about RRs returned +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Currently udns processes records in the reply it received sequentially. +This means that order of the records is significant. For example, if +we asked for foo.bar A, but the server returned that foo.bar is a CNAME +(alias) for bar.baz, and bar.baz, in turn, has address 1.2.3.4, when +the CNAME should come first in reply, followed by A. While DNS specs +does not say anything about order of records - it's an rrSET - unordered, - +I think an implementation which returns the records in "wrong" order is +somewhat insane... + +CNAME recursion +~~~~~~~~~~~~~~~ + +Another interesting point is the handling of CNAMEs returned as replies +to non-CNAME queries. If we asked for foo.bar A, but it's a CNAME, udns +expects BOTH the CNAME itself and the target DN to be present in the reply. +In other words, udns DOES NOT RECURSE CNAMES. If we asked for foo.bar A, +but only record in reply was that foo.bar is a CNAME for bar.baz, udns will +return no records to an application (NXDOMAIN). Strictly speaking, udns +should repeat the query asking for bar.baz A, and recurse. But since it's +stub resolver, recursive resolver should recurse for us instead. + +It's not very difficult to implement, however. Probably with some (global?) +flag to en/dis-able the feature. Provided there's some demand for it. + +To clarify: udns handles CNAME recursion in a single reply packet just fine. + +Note also that standard gethostbyname() routine does not recurse in this +situation, too. + +Error reporting +~~~~~~~~~~~~~~~ + +Too many places in the code (various failure paths) sets generic "TEMPFAIL" +error condition. For example, if no nameserver replied to our query, an +application will get generic TEMPFAIL, instead of something like TIMEDOUT. +This probably should be fixed, but most applications don't care about the +exact reasons of failure - 4 common cases are already too much: + - query returned some valid data + - NXDOMAIN + - valid domain but no data of requested type - =NXDOMAIN in most cases + - temporary error - this one sometimes (incorrectly!) treated as NXDOMAIN + by (naive) applications. +DNS isn't yes/no, it's at least 3 variants, temp err being the 3rd important +case! And adding more variations for the temp error case is complicating things +even more - again, from an application writer standpoint. For diagnostics, +such more specific error cases are of good help. + +Planned API changes +~~~~~~~~~~~~~~~~~~~ + +At least one thing I want to change for some future version is a way how +queries are submitted and how replies are handled. + +I want to made dns_query object to be owned by an application. So that instead +of udns library allocating it for the lifetime of query, it will be pre- +allocated by an application. This simplifies and enhances query submitting +interface, and complicates it a bit too, in simplest cases. + +Currently, we have: + +dns_submit_dn(dn, cls, typ, flags, parse, cbck, data) +dns_submit_p(name, cls, typ, flags, parse, cbck, data) +dns_submit_a4(ctx, name, flags, cbck, data) + +and so on -- with many parameters missed for type-specific cases, but generic +cases being too complex for most common usage. + +Instead, with dns_query being owned by an app, we will be able to separately +set up various parts of the query - domain name (various forms), type&class, +parser, flags, callback... and even change them at runtime. And we will also +be able to reuse query structures, instead of allocating/freeing them every +time. So the whole thing will look something like: + + q = dns_alloc_query(); + dns_submit(dns_q_flags(dns_q_a4(q, name, cbck), DNS_F_NOSRCH), data); + +The idea is to have a set of functions accepting struct dns_query* and +returning it (so the calls can be "nested" like the above), to set up +relevant parts of the query - specific type of callback, conversion from +(type-specific) query parameters into a domain name (this is for type- +specific query initializers), and setting various flags and options and +type&class things. + +One example where this is almost essential - if we want to support +per-query set of nameservers (which isn't at all useless: imagine a +high-volume mail server, were we want to direct DNSBL queries to a separate +set of nameservers, and rDNS queries to their own set and so on). Adding +another argument (set of nameservers to use) to EVERY query submitting +routine is.. insane. Especially since in 99% cases it will be set to +default NULL. But with such "nesting" of query initializers, it becomes +trivial. + +This change (the way how queries gets submitted) will NOT break API/ABI +compatibility with old versions, since the new submitting API works in +parallel with current (and current will use the new one as building +blocks, instead of doing all work at once). + +Another way to do the same is to manipulate query object right after a +query has been submitted, but before any events processing (during this +time, query object is allocated and initialized, but no actual network +packets were sent - it will happen on the next event processing). But +this way it become impossible to perform syncronous resolver calls, since +those calls hide query objects they use internally. + +Speaking of replies handling - the planned change is to stop using dynamic +memory (malloc) inside the library. That is, instead of allocating a buffer +for a reply dynamically in a parsing routine (or memdup'ing the raw reply +packet if no parsing routine is specified), I want udns to return the packet +buffer it uses internally, and change parsing routines to expect a buffer +for result. When parsing, a routine will return true amount of memory it +will need to place the result, regardless of whenever it has enough room +or not, so that an application can (re)allocate properly sized buffer and +call a parsing routine again. + +This, in theory, also can be done without breaking current API/ABI, but in +that case we'll again need a parallel set of routines (parsing included), +which makes the library more complicated with too many ways of doing the +same thing. Still, code reuse is at good level. + +Another modification I plan to include is to have an ability to work in +terms of domain names (DNs) as used with on-wire DNS packets, not only +with asciiz representations of them. For this to work, the above two +changes (query submission and result passing) have to be completed first +(esp. the query submission part), so that it will be possible to specify +some additional query flags (for example) to request domain names instead +of the text strings, and to allow easy query submissions with either DNs +or text strings. diff --git a/contrib/udns/README.md b/contrib/udns/README.md new file mode 100644 index 00000000000..f5fcc68aa79 --- /dev/null +++ b/contrib/udns/README.md @@ -0,0 +1 @@ +# udns diff --git a/contrib/udns/TODO b/contrib/udns/TODO new file mode 100644 index 00000000000..0dc9b967c60 --- /dev/null +++ b/contrib/udns/TODO @@ -0,0 +1,59 @@ +TODO + +The following is mostly an internal, not user-visible stuff. + +* rearrange an API to make dns_query object owned by application, + so that it'll look like this: + struct dns_query *q; + q = dns_query_alloc(ctx); + dns_query_set(q, options, domain_name, flags, ...); + dns_query_submit(ctx, q); + For more information see NOTES file, section "Planned API changes". + +* allow NULL callbacks? Or provide separate resolver + context list of queries which are done but wich did not + have callback, and dns_pick() routine to retrieve results + from this query, i.e. allow non-callback usage? The + non-callback usage may be handy sometimes (any *good* + example?), but it will be difficult to provide type-safe + non-callback interface due to various RR-specific types + in use. + +* DNS_OPT_FLAGS should be DNS_OPT_ADDFLAGS and DNS_OPT_SETFLAGS. + Currently one can't add a single flag bit but preserve + existing bits... at least not without retrieving all current + flags before, which isn't that bad anyway. + +* dns_set_opts() may process flags too (such as aaonly etc) + +* a way to disable $NSCACHEIP et al processing? + (with now separate dns_init() and dns_reset(), it has finer + control, but still no way to init from system files but ignore + environment variables and the like) + +* initialize/open the context automatically, and be more + liberal about initialization in general? + +* dns_init(ctx, do_open) - make the parameter opposite, aka + dns_init(ctx, skip_open) ? + +* allow TCP queue? + +* more accurate error reporting. Currently, udns always returns TEMPFAIL, + but don't specify why it happened (ENOMEM, timeout, etc). + +* check the error value returned by recvfrom() and + sendto() and determine which errors to ignore. + +* maybe merge dns_timeouts() and dns_ioevent(), to have + only one entry point for everything? For traditional + select-loop-based eventloop it may be easier, but for + callback-driven event loops the two should be separate. + Provide an option, or a single dns_events() entry point + for select-loop approach, or just call dns_ioevent() + from within dns_timeouts() (probably after renaming + it to be dns_events()) ? + +* implement /etc/hosts lookup too, ala [c-]ares?? + +* sortlist support? diff --git a/contrib/udns/config.h b/contrib/udns/config.h new file mode 100644 index 00000000000..5d79367247f --- /dev/null +++ b/contrib/udns/config.h @@ -0,0 +1,7 @@ +#pragma once +/* automatically generated by configure. */ + +#define HAVE_GETOPT 1 +#define HAVE_INET_PTON_NTOP 1 +#define HAVE_IPv6 1 +#define HAVE_POLL 1 \ No newline at end of file diff --git a/contrib/udns/configure b/contrib/udns/configure new file mode 100755 index 00000000000..a3c0a0dd3d0 --- /dev/null +++ b/contrib/udns/configure @@ -0,0 +1,166 @@ +#! /bin/sh +# autoconf-style configuration script +# + +set -e + +name=udns + +if [ -f udns.h -a -f udns_resolver.c ] ; then : +else + echo "configure: error: sources not found at `pwd`" >&2 + exit 1 +fi + +options="ipv6" + +for opt in $options; do + eval enable_$opt= +done + +if [ -f config.status ]; then + . ./config.status +fi + +enable() { + opt=`echo "$1" | sed 's/^--[^-]*-//'` + case "$opt" in + ipv6) ;; + *) echo "configure: unrecognized option \`$1'" >&2; exit 1;; + esac + eval enable_$opt=$2 +} + +while [ $# -gt 0 ]; do + case "$1" in + --disable-*|--without-*|--no-*) enable "$1" n;; + --enable-*|--with-*) enable "$1" y;; + --help | --hel | --he | --h | -help | -hel | -he | -h ) + cat <&2; exit 1 ;; + esac + shift +done + +. ./configure.lib + +ac_msg "configure" +ac_result "$name package" + +ac_prog_c_compiler_v +ac_prog_ranlib_v + +ac_ign ac_yesno "for getopt()" ac_have GETOPT ac_link < +extern int optind; +extern char *optarg; +extern int getopt(int, char **, char *); +int main(int argc, char **argv) { + getopt(argc, argv, "abc"); + return optarg ? optind : 0; +} +EOF + +if ac_library_find_v 'socket and connect' "" "-lsocket -lnsl" < +int main() { int socket_fd = socket(); connect(socket_fd); return 0; } +EOF +then : +else + ac_fatal "cannot find libraries needed for sockets" +fi + +ac_ign \ + ac_yesno "for inet_pton() && inet_ntop()" \ + ac_have INET_PTON_NTOP \ + ac_link < +#include +#include +int main() { + char buf[64]; + long x = 0; + inet_pton(AF_INET, &x, buf); + return inet_ntop(AF_INET, &x, buf, sizeof(buf)); +} +EOF + +if ac_yesno "for socklen_t" ac_compile < +#include +int foo() { socklen_t len; len = 0; return len; } +EOF +then : +else + ac_define socklen_t int +fi + +if [ n != "$enable_ipv6" ]; then +if ac_yesno "for IPv6" ac_have IPv6 ac_compile < +#include +#include +int main() { + struct sockaddr_in6 sa; + sa.sin6_family = AF_INET6; + return 0; +} +EOF +then : +elif [ "$enable_ipv6" ]; then + ac_fatal "IPv6 is requested but not available" +fi +fi # !disable_ipv6? + +if ac_yesno "for poll()" ac_have POLL ac_link < +#include +int main() { + struct pollfd pfd[2]; + return poll(pfd, 2, 10); +} +EOF +then : +else + ac_ign ac_yesno "for sys/select.h" ac_have SYS_SELECT_H ac_cpp < +#include +EOF +fi + +ac_config_h +ac_output Makefile +ac_msg "creating config.status" +rm -f config.status +{ +echo "# automatically generated by configure to hold command-line options" +echo +found= +for opt in $options; do + eval val=\$enable_$opt + if [ -n "$val" ]; then + echo enable_$opt=$val + found=y + fi +done +if [ ! "$found" ]; then + echo "# (no options encountered)" +fi +} > config.status +ac_result ok + +ac_result "all done." +exit 0 diff --git a/contrib/udns/configure.lib b/contrib/udns/configure.lib new file mode 100644 index 00000000000..541177a095b --- /dev/null +++ b/contrib/udns/configure.lib @@ -0,0 +1,268 @@ +# configure.lib +# a library of shell routines for simple autoconf system +# + +set -e +ac_substitutes= +rm -f conftest* config.log +exec 5>config.log +cat <&5 +This file contains any messages produced by compilers etc while +running configure, to aid debugging if configure script makes a mistake. + +EOF + +case `echo "a\c"` in + *c*) ac_en=-n ac_ec= ;; + *) ac_en= ac_ec='\c' ;; +esac + +##### Messages +ac_msg() { + echo $ac_en "$*... $ac_ec" + echo ">>> $*" >&5 +} +ac_checking() { + echo $ac_en "checking $*... $ac_ec" + echo ">>> checking $*" >&5 +} +ac_result() { + echo "$1" + echo "=== $1" >&5 +} +ac_fatal() { + echo "configure: fatal: $*" >&2 + echo "=== FATAL: $*" >&5 + exit 1 +} +ac_warning() { + echo "configure: warning: $*" >&2 + echo "=== WARNING: $*" >&5 +} +ac_ign() { + "$@" || : +} + +# ac_run command... +# captures output in conftest.out +ac_run() { + # apparently UnixWare (for one) /bin/sh optimizes the following "if" + # "away", by checking if there's such a command BEFORE redirecting + # output. So error message (like "gcc: command not found") goes + # to stderr instead of to conftest.out, and `cat conftest.out' below + # fails. + if "$@" >conftest.out 2>&1; then + return 0 + else + echo "==== Command invocation failed. Command line was:" >&5 + echo "$*" >&5 + echo "==== compiler input was:" >&5 + cat conftest.c >&5 + echo "==== output was:" >&5 + cat conftest.out >&5 + echo "====" >&5 + return 1 + fi +} + +# common case for ac_verbose: yes/no result +ac_yesno() { + ac_checking "$1" + shift + if "$@"; then + ac_result yes + return 0 + else + ac_result no + return 1 + fi +} + +ac_subst() { + ac_substitutes="$ac_substitutes $*" +} + +ac_define() { + CDEFS="$CDEFS -D$1=${2:-1}" +} + +ac_have() { + ac_what=$1; shift + if "$@"; then + ac_define HAVE_$ac_what + eval ac_have_$ac_what=yes + return 0 + else + eval ac_have_$ac_what=no + return 1 + fi +} + +##### Compiling, linking + +# run a compiler +ac_run_compiler() { + rm -f conftest*; cat >conftest.c + ac_run $CC $CFLAGS $CDEFS "$@" conftest.c +} + +ac_compile() { + ac_run_compiler -c +} + +ac_link() { + ac_run_compiler -o conftest $LIBS "$@" +} + +ac_cpp() { + ac_run_compiler -E "$@" +} + +### check for C compiler. Set $CC, $CFLAGS etc +ac_prog_c_compiler_v() { + ac_checking "for C compiler" + rm -f conftest* + echo 'int main(int argc, char **argv) { return 0; }' >conftest.c + + if [ -n "$CC" ]; then + if ac_run $CC -o conftest conftest.c && ac_run ./conftest; then + ac_result "\$CC ($CC)" + else + ac_result no + ac_fatal "\$CC ($CC) is not a working compiler" + fi + else + for cc in gcc cc ; do + if ac_run $cc -o conftest conftest.c && ac_run ./conftest; then + ac_result "$cc" + CC=$cc + break + fi + done + if [ -z "$CC" ]; then + ac_result no + ac_fatal "no working C compiler found in \$PATH. please set \$CC variable" + fi + fi + if [ -z "$CFLAGS" ]; then + if ac_yesno "whenever C compiler ($CC) is GNU CC" \ + ac_grep_cpp yEs_mAsTeR <conftest.c + for lib in "$@"; do + if ac_run $CC $CFLAGS $LDFLAGS conftest.c -o conftest $LIBS $lib; then + found=y + break + fi + done + if [ ! "$found" ]; then + ac_result "not found" + return 1 + fi + if [ -z "$lib" ]; then + ac_result "ok (none needed)" + else + ac_result "ok ($lib)" + LIBS="$LIBS $lib" + fi +} + +ac_compile_run() { + ac_link "$@" && ac_run ./conftest +} + +ac_grep_cpp() { + pattern="$1"; shift + ac_cpp "$@" && grep "$pattern" conftest.out >/dev/null +} + +ac_output() { + for var in $ac_substitutes; do + eval echo "\"s|@$var@|\$$var|\"" + done >conftest.sed + for file in "$@"; do + ac_msg "creating $file" + if [ -f $file.in ]; then + sed -f conftest.sed $file.in > $file.tmp + mv -f $file.tmp $file + ac_result ok + else + ac_result failed + ac_fatal "$file.in not found" + fi + done + rm -f conftest* +} + +ac_config_h() { + h=${1:-config.h} + ac_msg "creating $h" + rm -f $1.tmp + echo "/* automatically generated by configure. */" > $h.tmp + echo "$CDEFS" | tr ' ' ' +' | sed -e 's/^-D/#define /' -e 's/=/ /' >> $h.tmp + if [ -f $h ] && cmp -s $h.tmp $h ; then + rm -f $h.tmp + ac_result unchanged + else + mv -f $h.tmp $h + ac_result ok + fi + CDEFS=-DHAVE_CONFIG_H +} diff --git a/contrib/udns/conftest.c b/contrib/udns/conftest.c new file mode 100644 index 00000000000..5bc7720cbaa --- /dev/null +++ b/contrib/udns/conftest.c @@ -0,0 +1,3 @@ +#include + +int main() { socket(); connect(); return 0; } diff --git a/contrib/udns/conftest.out b/contrib/udns/conftest.out new file mode 100644 index 00000000000..cfff425825a --- /dev/null +++ b/contrib/udns/conftest.out @@ -0,0 +1,7 @@ +conftest.c:1:14: error: implicit declaration of function 'socket' is invalid in C99 [-Werror,-Wimplicit-function-declaration] +int main() { socket(); connect(); return 0; } + ^ +conftest.c:1:24: error: implicit declaration of function 'connect' is invalid in C99 [-Werror,-Wimplicit-function-declaration] +int main() { socket(); connect(); return 0; } + ^ +2 errors generated. diff --git a/contrib/udns/dnsget.1 b/contrib/udns/dnsget.1 new file mode 100644 index 00000000000..200557fe947 --- /dev/null +++ b/contrib/udns/dnsget.1 @@ -0,0 +1,195 @@ +.\" dnsget.1: dnsget manpage +.\" +.\" Copyright (C) 2005-2014 Michael Tokarev +.\" This file is part of UDNS library, an async DNS stub resolver. +.\" +.\" This library is free software; you can redistribute it and/or +.\" modify it under the terms of the GNU Lesser General Public +.\" License as published by the Free Software Foundation; either +.\" version 2.1 of the License, or (at your option) any later version. +.\" +.\" This library is distributed in the hope that it will be useful, +.\" but WITHOUT ANY WARRANTY; without even the implied warranty of +.\" MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU +.\" Lesser General Public License for more details. +.\" +.\" You should have received a copy of the GNU Lesser General Public +.\" License along with this library, in file named COPYING.LGPL; if not, +.\" write to the Free Software Foundation, Inc., 59 Temple Place, +.\" Suite 330, Boston, MA 02111-1307 USA + +.TH dnsget 1 "Jan 2014" "User Utilities" + +.SH NAME +dnsget \- DNS lookup utility + +.SH SYNOPSYS +.B dnsget +.RB [\| \-v \||\| \-q \|] +.RB [\| \-c +.IR class \|] +.RB [\| \-t +.IR type \|] +.RB [\| \-o +.IR opt , opt ,...] +.IR name \|.\|.\|. + +.SH DESCRIPTION +.B dnsget +is a simple command-line to perform DNS lookups, similar to +.BR host (1) +and +.BR dig (1). +It is useable for both interactive/debugging scenarious and +in scripts. +The program is implemented using +.BR udns (3) +library. + +.PP +By default, +.B dnsget +produces a human-readable output, similar to +.RS +.nf +alias.example.com. CNAME www.example.com. +www.example.com. A 192.168.1.1 +www.example.com. MX 10 mx.example.com. +.fi +.RE +which is just sufficient to see how a given name resolves. +Output format is controllable with +.B \-v +and +.B \-q +options -- the former increases verbosity level up to printing +the whole DNS contents of all packets sent and received, which +is suitable for debugging DNS problems, while the latter reduces +the level, making output more quiet, up to bare result with no +error messages, which is good for scripts. + +.SH OPTIONS + +The following options are recognized by +.BR dnsget : + +.TP +.B \-v +produce more detailed output. More +.BR \-v 's +means more details will be produced. With single +.BR \-v , dnsget +will print contents of all received DNS packets (in a readable format), +while with +.BR \-vv , +it will output all outgoing DNS packets too. + +.TP +.B \-q +the opposite for \fB\-v\fR -- produce less detailed output. +With single +.BR \-q , dnsget +will only show (decoded) data from final DNS resource records (RR), +while +.B \-qq +also suppresses error messages. + +.TP +\fB\-t \fItype\fR +request record(s) of the given type \fItype\fR. By default, +.B dnsget +will ask for IPv4 address (A) record, or for PTR record if the +argument in question is an IPv4 or IPv6 address. Recognized +types include A, AAAA, MX, TXT, CNAME, PTR, NS, SOA, ANY and +others. + +.TP +\fB\-c \fIclass\fR +request DNS record(s) of the given class \fIclass\fR. By +default +.B dnsget +uses IN class. Valid classes include IN, CH, HS, ANY. + +.TP +.B \-a +(compatibility option). Equivalent to setting query type to +.B ANY +and increasing verbosity level +.RB ( \-v ). + +.TP +.B \-C +(planned) + +.TP +.B \-x +(planned) + +.TP +\fB\-o \fIopt\fR,\fIopt\fR,... +(may be specified several times). +Set resolver options (in a form \fIoption\fR:\fIvalue\fR) as if they +were set in +.RB $ RES_OPTIONS +environment variable, or set query flags: +.RS +.TP +\fBtimeout\fR:\fIsec\fR +Set initial query timeout to \fIsec\fR. +.TP +\fBattempts\fR:\fInum\fR +(re)try every query \fInum\fR times before failing. +.TP +\fBudpbuf\fR:\fIbytes\fR +set DNS UDP buffer size to \fIbytes\fR bytes. Valid values +are from 512 to 65535. If \fIbytes\fR is greather than 512, +EDNS0 (RFC 2671) extensions will be used. +.TP +\fBport\fR:\fInum\fR +Use given UDP port number \fInum\fR instead of the default port 53 (domain). +.TP +\fBaa\fR +set AA (auth only) query bit. +.TP +\fBnord\fR +do not set RD (recursion desired) query bit (set by default). +.TP +\fBdnssec\fR or \fBdo\fR +set DNSSEC OK (DO) query flag (\fBdnsget\fR does not verify DNSSEC signatures, +only displays them; this is set in EDNS RR). +.TP +\fBcd\fR +set CD (checking disabled) query bit. +.RE + +.TP +\fB\-n \fInameserver\fR +Use the given nameserver(s) (may be specified more than once) +instead of the default. Using this option has the same same effect as +.RB $ NSCACHEIP +or +.RB $ NAMESERVERS +environment variables, with the only difference that only IPv4 addresses +are recognized for now, and it is possible to specify names (which will +be resolved using default settings) instead of IP addresses. + +.TP +.B \-h +print short help and exit. + +.SH "RETURN VALUE" +When all names where resovled successefully, +.B dnsget +exits with zero exit status. If at least one name was not found, +.B dnsget +will exit with return code 100. If some other error occured during +name resolution, it will exit with code 99. In case of usage or +initialization error, +.B dnsget +will return 1. + +.SH "SEE ALSO" +.BR host (1) +.BR dig (1) +.BR resolv.conf (5) +.BR udns (3). diff --git a/contrib/udns/dnsget.c b/contrib/udns/dnsget.c new file mode 100644 index 00000000000..417e8d9743c --- /dev/null +++ b/contrib/udns/dnsget.c @@ -0,0 +1,759 @@ +/* dnsget.c + simple host/dig-like application using UDNS library + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#ifdef HAVE_CONFIG_H +# include "config.h" +#endif +#ifdef WINDOWS +#include +#include +#else +#include +#include +#include +#include +#include +#endif +#include +#include +#include +#include +#include +#include +#include "udns.h" + +#ifndef HAVE_GETOPT +# include "getopt.c" +#endif + +#ifndef AF_INET6 +# define AF_INET6 10 +#endif + +static char *progname; +static int verbose = 1; +static int errors; +static int notfound; + +/* verbosity level: + * <0 - bare result + * 0 - bare result and error messages + * 1 - readable result + * 2 - received packet contents and `trying ...' stuff + * 3 - sent and received packet contents + */ + +static void die(int errnum, const char *fmt, ...) { + va_list ap; + fprintf(stderr, "%s: ", progname); + va_start(ap, fmt); vfprintf(stderr, fmt, ap); va_end(ap); + if (errnum) fprintf(stderr, ": %s\n", strerror(errnum)); + else putc('\n', stderr); + fflush(stderr); + exit(1); +} + +static const char *dns_xntop(int af, const void *src) { + static char buf[6*5+4*4]; + return dns_ntop(af, src, buf, sizeof(buf)); +} + +struct query { + const char *name; /* original query string */ + unsigned char *dn; /* the DN being looked up */ + enum dns_type qtyp; /* type of the query */ +}; + +static void query_free(struct query *q) { + free(q->dn); + free(q); +} + +static struct query * +query_new(const char *name, const unsigned char *dn, enum dns_type qtyp) { + struct query *q = malloc(sizeof(*q)); + unsigned l = dns_dnlen(dn); + unsigned char *cdn = malloc(l); + if (!q || !cdn) die(0, "out of memory"); + memcpy(cdn, dn, l); + q->name = name; + q->dn = cdn; + q->qtyp = qtyp; + return q; +} + +static enum dns_class qcls = DNS_C_IN; + +static void +dnserror(struct query *q, int errnum) { + if (verbose >= 0) + fprintf(stderr, "%s: unable to lookup %s record for %s: %s\n", progname, + dns_typename(q->qtyp), dns_dntosp(q->dn), dns_strerror(errnum)); + if (errnum == DNS_E_NXDOMAIN || errnum == DNS_E_NODATA) + ++notfound; + else + ++errors; + query_free(q); +} + +static const unsigned char * +printtxt(const unsigned char *c) { + unsigned n = *c++; + const unsigned char *e = c + n; + if (verbose > 0) while(c < e) { + if (*c < ' ' || *c >= 127) printf("\\%03u", *c); + else if (*c == '\\' || *c == '"') printf("\\%c", *c); + else putchar(*c); + ++c; + } + else + fwrite(c, n, 1, stdout); + return e; +} + +static void +printhex(const unsigned char *c, const unsigned char *e) { + while(c < e) + printf("%02x", *c++); +} + +static unsigned char to_b64[] = +"ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/"; + +static void +printb64(const unsigned char *c, const unsigned char *e) { + while(c < e) { + putchar(to_b64[c[0] >> 2]); + if (c+1 < e) { + putchar(to_b64[(c[0] & 0x3) << 4 | c[1] >> 4]); + if (c+2 < e) { + putchar(to_b64[(c[1] & 0xf) << 2 | c[2] >> 6]); + putchar(to_b64[c[2] & 0x3f]); + } + else { + putchar(to_b64[(c[1] & 0xf) << 2]); + putchar('='); + break; + } + } + else { + putchar(to_b64[(c[0] & 0x3) << 4]); + putchar('='); + putchar('='); + break; + } + c += 3; + } +} + +static void +printdate(time_t time) { + struct tm *tm = gmtime(&time); + printf("%04d%02d%02d%02d%02d%02d", + tm->tm_year + 1900, tm->tm_mon + 1, tm->tm_mday, + tm->tm_hour, tm->tm_min, tm->tm_sec); +} + +static void +printrr(const struct dns_parse *p, struct dns_rr *rr) { + const unsigned char *pkt = p->dnsp_pkt; + const unsigned char *end = p->dnsp_end; + const unsigned char *dptr = rr->dnsrr_dptr; + const unsigned char *dend = rr->dnsrr_dend; + unsigned char *dn = rr->dnsrr_dn; + const unsigned char *c; + unsigned n; + + if (verbose > 0) { + if (verbose > 1) { + if (!p->dnsp_rrl && !rr->dnsrr_dn[0] && rr->dnsrr_typ == DNS_T_OPT) { + printf(";EDNS%d OPT record (UDPsize: %d, ERcode: %d, Flags: 0x%02x): %d bytes\n", + (rr->dnsrr_ttl>>16) & 0xff, /* version */ + rr->dnsrr_cls, /* udp size */ + (rr->dnsrr_ttl>>24) & 0xff, /* extended rcode */ + rr->dnsrr_ttl & 0xffff, /* flags */ + rr->dnsrr_dsz); + return; + } + n = printf("%s.", dns_dntosp(rr->dnsrr_dn)); + printf("%s%u\t%s\t%s\t", + n > 15 ? "\t" : n > 7 ? "\t\t" : "\t\t\t", + rr->dnsrr_ttl, + dns_classname(rr->dnsrr_cls), + dns_typename(rr->dnsrr_typ)); + } + else + printf("%s. %s ", dns_dntosp(rr->dnsrr_dn), dns_typename(rr->dnsrr_typ)); + } + + switch(rr->dnsrr_typ) { + + case DNS_T_CNAME: + case DNS_T_PTR: + case DNS_T_NS: + case DNS_T_MB: + case DNS_T_MD: + case DNS_T_MF: + case DNS_T_MG: + case DNS_T_MR: + if (dns_getdn(pkt, &dptr, end, dn, DNS_MAXDN) <= 0) goto xperr; + printf("%s.", dns_dntosp(dn)); + break; + + case DNS_T_A: + if (rr->dnsrr_dsz != 4) goto xperr; + printf("%d.%d.%d.%d", dptr[0], dptr[1], dptr[2], dptr[3]); + break; + + case DNS_T_AAAA: + if (rr->dnsrr_dsz != 16) goto xperr; + printf("%s", dns_xntop(AF_INET6, dptr)); + break; + + case DNS_T_MX: + c = dptr + 2; + if (dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0 || c != dend) goto xperr; + printf("%d %s.", dns_get16(dptr), dns_dntosp(dn)); + break; + + case DNS_T_TXT: + /* first verify it */ + for(c = dptr; c < dend; c += n) { + n = *c++; + if (c + n > dend) goto xperr; + } + c = dptr; n = 0; + while (c < dend) { + if (verbose > 0) printf(n++ ? "\" \"":"\""); + c = printtxt(c); + } + if (verbose > 0) putchar('"'); + break; + + case DNS_T_HINFO: /* CPU, OS */ + c = dptr; + n = *c++; if ((c += n) >= dend) goto xperr; + n = *c++; if ((c += n) != dend) goto xperr; + c = dptr; + if (verbose > 0) putchar('"'); + c = printtxt(c); + if (verbose > 0) printf("\" \""); else putchar(' '); + printtxt(c); + if (verbose > 0) putchar('"'); + break; + + case DNS_T_WKS: + c = dptr; + if (dptr + 4 + 2 >= end) goto xperr; + printf("%s %d", dns_xntop(AF_INET, dptr), dptr[4]); + c = dptr + 5; + for (n = 0; c < dend; ++c, n += 8) { + if (*c) { + unsigned b; + for (b = 0; b < 8; ++b) + if (*c & (1 << (7-b))) printf(" %d", n + b); + } + } + break; + + case DNS_T_SRV: /* prio weight port targetDN */ + c = dptr; + c += 2 + 2 + 2; + if (dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0 || c != dend) goto xperr; + c = dptr; + printf("%d %d %d %s.", + dns_get16(c+0), dns_get16(c+2), dns_get16(c+4), + dns_dntosp(dn)); + break; + + case DNS_T_NAPTR: /* order pref flags serv regexp repl */ + c = dptr; + c += 4; /* order, pref */ + for (n = 0; n < 3; ++n) + if (c >= dend) goto xperr; + else c += *c + 1; + if (dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0 || c != dend) goto xperr; + c = dptr; + printf("%u %u", dns_get16(c+0), dns_get16(c+2)); + c += 4; + for(n = 0; n < 3; ++n) { + putchar(' '); + if (verbose > 0) putchar('"'); + c = printtxt(c); + if (verbose > 0) putchar('"'); + } + printf(" %s.", dns_dntosp(dn)); + break; + + case DNS_T_KEY: + case DNS_T_DNSKEY: + /* flags(2) proto(1) algo(1) pubkey */ + case DNS_T_DS: + case DNS_T_DLV: + /* ktag(2) proto(1) algo(1) pubkey */ + c = dptr; + if (c + 2 + 1 + 1 > dend) goto xperr; + printf("%d %d %d", dns_get16(c), c[2], c[3]); + c += 2 + 1 + 1; + if (c < dend) { + putchar(' '); + printb64(c, dend); + } + break; + + case DNS_T_SIG: + case DNS_T_RRSIG: + /* type(2) algo(1) labels(1) ottl(4) sexp(4) sinc(4) tag(2) sdn sig */ + c = dptr; + c += 2 + 1 + 1 + 4 + 4 + 4 + 2; + if (dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0) goto xperr; + printf("%s %u %u %u ", + dns_typename(dns_get16(dptr)), dptr[2], dptr[3], dns_get32(dptr+4)); + printdate(dns_get32(dptr+8)); + putchar(' '); + printdate(dns_get32(dptr+12)); + printf(" %d %s. ", dns_get16(dptr+10), dns_dntosp(dn)); + printb64(c, dend); + break; + + case DNS_T_SSHFP: /* algo(1), fp type(1), fp... */ + if (dend < dptr + 3) goto xperr; + printf("%u %u ", dptr[0], dptr[1]); /* algo, fp type */ + printhex(dptr + 2, dend); + break; + +#if 0 /* unused RR types? */ + case DNS_T_NSEC: /* nextDN bitmaps */ + c = dptr; + if (dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0) goto xperr; + printf("%s.", dns_dntosp(dn)); + unfinished. + break; +#endif + + + case DNS_T_SOA: + c = dptr; + if (dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0 || + dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0 || + c + 4*5 != dend) + goto xperr; + dns_getdn(pkt, &dptr, end, dn, DNS_MAXDN); + printf("%s. ", dns_dntosp(dn)); + dns_getdn(pkt, &dptr, end, dn, DNS_MAXDN); + printf("%s. ", dns_dntosp(dn)); + printf("%u %u %u %u %u", + dns_get32(dptr), dns_get32(dptr+4), dns_get32(dptr+8), + dns_get32(dptr+12), dns_get32(dptr+16)); + break; + + case DNS_T_MINFO: + c = dptr; + if (dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0 || + dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0 || + c != dend) + goto xperr; + dns_getdn(pkt, &dptr, end, dn, DNS_MAXDN); + printf("%s. ", dns_dntosp(dn)); + dns_getdn(pkt, &dptr, end, dn, DNS_MAXDN); + printf("%s.", dns_dntosp(dn)); + break; + + case DNS_T_NULL: + default: + printhex(dptr, dend); + break; + } + putchar('\n'); + return; + +xperr: + printf("\n"); + ++errors; +} + +static int +printsection(struct dns_parse *p, int nrr, const char *sname) { + struct dns_rr rr; + int r; + if (!nrr) return 0; + if (verbose > 1) printf("\n;; %s section (%d):\n", sname, nrr); + + p->dnsp_rrl = nrr; + while((r = dns_nextrr(p, &rr)) > 0) + printrr(p, &rr); + if (r < 0) printf("<>\n"); + return r; +} + +/* dbgcb will only be called if verbose > 1 */ +static void +dbgcb(int code, const struct sockaddr *sa, unsigned slen, + const unsigned char *pkt, int r, + const struct dns_query *unused_q, void *unused_data) { + struct dns_parse p; + const unsigned char *cur, *end; + int numqd; + + if (code > 0) { + printf(";; trying %s.\n", dns_dntosp(dns_payload(pkt))); + printf(";; sending %d bytes query to ", r); + } + else + printf(";; received %d bytes response from ", r); + if (sa->sa_family == AF_INET && slen >= sizeof(struct sockaddr_in)) + printf("%s port %d\n", + dns_xntop(AF_INET, &((struct sockaddr_in*)sa)->sin_addr), + htons(((struct sockaddr_in*)sa)->sin_port)); +#ifdef HAVE_IPv6 + else if (sa->sa_family == AF_INET6 && slen >= sizeof(struct sockaddr_in6)) + printf("%s port %d\n", + dns_xntop(AF_INET6, &((struct sockaddr_in6*)sa)->sin6_addr), + htons(((struct sockaddr_in6*)sa)->sin6_port)); +#endif + else + printf("<>\n", sa->sa_family); + if (code > 0 && verbose < 3) { + putchar('\n'); + return; + } + + if (code == -2) printf(";; reply from unexpected source\n"); + if (code == -5) printf(";; reply to a query we didn't sent (or old)\n"); + if (r < DNS_HSIZE) { + printf(";; short packet (%d bytes)\n", r); + return; + } + if (dns_opcode(pkt) != 0) + printf(";; unexpected opcode %d\n", dns_opcode(pkt)); + if (dns_tc(pkt) != 0) + printf(";; warning: TC bit set, probably incomplete reply\n"); + + printf(";; ->>HEADER<<- opcode: "); + switch(dns_opcode(pkt)) { + case 0: printf("QUERY"); break; + case 1: printf("IQUERY"); break; + case 2: printf("STATUS"); break; + default: printf("UNKNOWN(%u)", dns_opcode(pkt)); break; + } + printf(", status: %s, id: %d, size: %d\n;; flags:", + dns_rcodename(dns_rcode(pkt)), dns_qid(pkt), r); + if (dns_qr(pkt)) printf(" qr"); + if (dns_aa(pkt)) printf(" aa"); + if (dns_tc(pkt)) printf(" tc"); + if (dns_rd(pkt)) printf(" rd"); + if (dns_ra(pkt)) printf(" ra"); + /* if (dns_z(pkt)) printf(" z"); only one reserved bit left */ + if (dns_ad(pkt)) printf(" ad"); + if (dns_cd(pkt)) printf(" cd"); + numqd = dns_numqd(pkt); + printf("; QUERY: %d, ANSWER: %d, AUTHORITY: %d, ADDITIONAL: %d\n", + numqd, dns_numan(pkt), dns_numns(pkt), dns_numar(pkt)); + if (numqd != 1) + printf(";; unexpected number of entries in QUERY section: %d\n", + numqd); + printf("\n;; QUERY SECTION (%d):\n", numqd); + cur = dns_payload(pkt); + end = pkt + r; + while(numqd--) { + if (dns_getdn(pkt, &cur, end, p.dnsp_dnbuf, DNS_MAXDN) <= 0 || + cur + 4 > end) { + printf("; invalid query section\n"); + return; + } + r = printf(";%s.", dns_dntosp(p.dnsp_dnbuf)); + printf("%s%s\t%s\n", + r > 23 ? "\t" : r > 15 ? "\t\t" : r > 7 ? "\t\t\t" : "\t\t\t\t", + dns_classname(dns_get16(cur+2)), dns_typename(dns_get16(cur))); + cur += 4; + } + + p.dnsp_pkt = pkt; + p.dnsp_cur = p.dnsp_ans = cur; + p.dnsp_end = end; + p.dnsp_qdn = NULL; + p.dnsp_qcls = p.dnsp_qtyp = 0; + p.dnsp_ttl = 0xffffffffu; + p.dnsp_nrr = 0; + + r = printsection(&p, dns_numan(pkt), "ANSWER"); + if (r == 0) + r = printsection(&p, dns_numns(pkt), "AUTHORITY"); + if (r == 0) + r = printsection(&p, dns_numar(pkt), "ADDITIONAL"); + putchar('\n'); +} + +static void dnscb(struct dns_ctx *ctx, void *result, void *data) { + int r = dns_status(ctx); + struct query *q = data; + struct dns_parse p; + struct dns_rr rr; + unsigned nrr; + unsigned char dn[DNS_MAXDN]; + const unsigned char *pkt, *cur, *end; + if (!result) { + dnserror(q, r); + return; + } + pkt = result; end = pkt + r; cur = dns_payload(pkt); + dns_getdn(pkt, &cur, end, dn, sizeof(dn)); + dns_initparse(&p, NULL, pkt, cur, end); + p.dnsp_qcls = p.dnsp_qtyp = 0; + nrr = 0; + while((r = dns_nextrr(&p, &rr)) > 0) { + if (!dns_dnequal(dn, rr.dnsrr_dn)) continue; + if ((qcls == DNS_C_ANY || qcls == rr.dnsrr_cls) && + (q->qtyp == DNS_T_ANY || q->qtyp == rr.dnsrr_typ)) + ++nrr; + else if (rr.dnsrr_typ == DNS_T_CNAME && !nrr) { + if (dns_getdn(pkt, &rr.dnsrr_dptr, end, + p.dnsp_dnbuf, sizeof(p.dnsp_dnbuf)) <= 0 || + rr.dnsrr_dptr != rr.dnsrr_dend) { + r = DNS_E_PROTOCOL; + break; + } + else { + if (verbose == 1) { + printf("%s.", dns_dntosp(dn)); + printf(" CNAME %s.\n", dns_dntosp(p.dnsp_dnbuf)); + } + dns_dntodn(p.dnsp_dnbuf, dn, sizeof(dn)); + } + } + } + if (!r && !nrr) + r = DNS_E_NODATA; + if (r < 0) { + dnserror(q, r); + free(result); + return; + } + if (verbose < 2) { /* else it is already printed by dbgfn */ + dns_rewind(&p, NULL); + p.dnsp_qtyp = q->qtyp == DNS_T_ANY ? 0 : q->qtyp; + p.dnsp_qcls = qcls == DNS_C_ANY ? 0 : qcls; + while(dns_nextrr(&p, &rr)) + printrr(&p, &rr); + } + free(result); + query_free(q); +} + +int main(int argc, char **argv) { + int i; + int fd; + fd_set fds; + struct timeval tv; + time_t now; + char *ns[DNS_MAXSERV]; + int nns = 0; + struct query *q; + enum dns_type qtyp = 0; + struct dns_ctx *nctx = NULL; + int flags = 0; + + if (!(progname = strrchr(argv[0], '/'))) progname = argv[0]; + else argv[0] = ++progname; + + if (argc <= 1) + die(0, "try `%s -h' for help", progname); + + if (dns_init(NULL, 0) < 0 || !(nctx = dns_new(NULL))) + die(errno, "unable to initialize dns library"); + /* we keep two dns contexts: one may be needed to resolve + * nameservers if given as names, using default options. + */ + + while((i = getopt(argc, argv, "vqt:c:an:o:f:h")) != EOF) switch(i) { + case 'v': ++verbose; break; + case 'q': --verbose; break; + case 't': + if (optarg[0] == '*' && !optarg[1]) + i = DNS_T_ANY; + else if ((i = dns_findtypename(optarg)) <= 0) + die(0, "unrecognized query type `%s'", optarg); + qtyp = i; + break; + case 'c': + if (optarg[0] == '*' && !optarg[1]) + i = DNS_C_ANY; + else if ((i = dns_findclassname(optarg)) < 0) + die(0, "unrecognized query class `%s'", optarg); + qcls = i; + break; + case 'a': + qtyp = DNS_T_ANY; + ++verbose; + break; + case 'n': + if (nns >= DNS_MAXSERV) + die(0, "too many nameservers, %d max", DNS_MAXSERV); + ns[nns++] = optarg; + break; + case 'o': + case 'f': { + char *opt; + const char *const delim = " \t,;"; + for(opt = strtok(optarg, delim); opt != NULL; opt = strtok(NULL, delim)) { + if (dns_set_opts(NULL, optarg) == 0) + ; + else if (strcmp(opt, "aa") == 0) flags |= DNS_AAONLY; + else if (strcmp(optarg, "nord") == 0) flags |= DNS_NORD; + else if (strcmp(optarg, "dnssec") == 0) flags |= DNS_SET_DO; + else if (strcmp(optarg, "do") == 0) flags |= DNS_SET_DO; + else if (strcmp(optarg, "cd") == 0) flags |= DNS_SET_CD; + else + die(0, "invalid option: `%s'", opt); + } + break; + } + case 'h': + printf( +"%s: simple DNS query tool (using udns version %s)\n" +"Usage: %s [options] domain-name...\n" +"where options are:\n" +" -h - print this help and exit\n" +" -v - be more verbose\n" +" -q - be less verbose\n" +" -t type - set query type (A, AAA, PTR etc)\n" +" -c class - set query class (IN (default), CH, HS, *)\n" +" -a - equivalent to -t ANY -v\n" +" -n ns - use given nameserver(s) instead of default\n" +" (may be specified multiple times)\n" +" -o opt,opt,... (comma- or space-separated list,\n" +" may be specified more than once):\n" +" set resovler options (the same as setting $RES_OPTIONS):\n" +" timeout:sec - initial query timeout\n" +" attempts:num - number of attempt to resovle a query\n" +" ndots:num - if name has more than num dots, lookup it before search\n" +" port:num - port number for queries instead of default 53\n" +" udpbuf:num - size of UDP buffer (use EDNS0 if >512)\n" +" or query flags:\n" +" aa,nord,dnssec,do,cd - set query flag (auth-only, no recursion,\n" +" enable DNSSEC (DNSSEC Ok), check disabled)\n" + , progname, dns_version(), progname); + return 0; + default: + die(0, "try `%s -h' for help", progname); + } + + argc -= optind; argv += optind; + if (!argc) + die(0, "no name(s) to query specified"); + + if (nns) { + /* if nameservers given as names, resolve them. + * We only allow IPv4 nameservers as names for now. + * Ok, it is easy enouth to try both AAAA and A, + * but the question is what to do by default. + */ + struct sockaddr_in sin; + int j, r = 0, opened = 0; + memset(&sin, 0, sizeof(sin)); + sin.sin_family = AF_INET; + sin.sin_port = htons(dns_set_opt(NULL, DNS_OPT_PORT, -1)); + dns_add_serv(NULL, NULL); + for(i = 0; i < nns; ++i) { + if (dns_pton(AF_INET, ns[i], &sin.sin_addr) <= 0) { + struct dns_rr_a4 *rr; + if (!opened) { + if (dns_open(nctx) < 0) + die(errno, "unable to initialize dns context"); + opened = 1; + } + rr = dns_resolve_a4(nctx, ns[i], 0); + if (!rr) + die(0, "unable to resolve nameserver %s: %s", + ns[i], dns_strerror(dns_status(nctx))); + for(j = 0; j < rr->dnsa4_nrr; ++j) { + sin.sin_addr = rr->dnsa4_addr[j]; + if ((r = dns_add_serv_s(NULL, (struct sockaddr *)&sin)) < 0) + break; + } + free(rr); + } + else + r = dns_add_serv_s(NULL, (struct sockaddr *)&sin); + if (r < 0) + die(errno, "unable to add nameserver %s", + dns_xntop(AF_INET, &sin.sin_addr)); + } + } + dns_free(nctx); + + fd = dns_open(NULL); + if (fd < 0) + die(errno, "unable to initialize dns context"); + + if (verbose > 1) + dns_set_dbgfn(NULL, dbgcb); + + if (flags) + dns_set_opt(NULL, DNS_OPT_FLAGS, flags); + + for (i = 0; i < argc; ++i) { + char *name = argv[i]; + union { + struct in_addr addr; + struct in6_addr addr6; + } a; + unsigned char dn[DNS_MAXDN]; + enum dns_type l_qtyp = 0; + int abs; + if (dns_pton(AF_INET, name, &a.addr) > 0) { + dns_a4todn(&a.addr, 0, dn, sizeof(dn)); + l_qtyp = DNS_T_PTR; + abs = 1; + } +#ifdef HAVE_IPv6 + else if (dns_pton(AF_INET6, name, &a.addr6) > 0) { + dns_a6todn(&a.addr6, 0, dn, sizeof(dn)); + l_qtyp = DNS_T_PTR; + abs = 1; + } +#endif + else if (!dns_ptodn(name, strlen(name), dn, sizeof(dn), &abs)) + die(0, "invalid name `%s'\n", name); + else + l_qtyp = DNS_T_A; + if (qtyp) l_qtyp = qtyp; + q = query_new(name, dn, l_qtyp); + if (abs) abs = DNS_NOSRCH; + if (!dns_submit_dn(NULL, dn, qcls, l_qtyp, abs, 0, dnscb, q)) + dnserror(q, dns_status(NULL)); + } + + FD_ZERO(&fds); + now = 0; + while((i = dns_timeouts(NULL, -1, now)) > 0) { + FD_SET(fd, &fds); + tv.tv_sec = i; + tv.tv_usec = 0; + i = select(fd+1, &fds, 0, 0, &tv); + now = time(NULL); + if (i > 0) dns_ioevent(NULL, now); + } + + return errors ? 1 : notfound ? 100 : 0; +} diff --git a/contrib/udns/ex-rdns.c b/contrib/udns/ex-rdns.c new file mode 100644 index 00000000000..1e1e90d4b1d --- /dev/null +++ b/contrib/udns/ex-rdns.c @@ -0,0 +1,114 @@ +/* ex-rdns.c + parallel rDNS resolver example - read IP addresses from stdin, + write domain names to stdout + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "udns.h" + +static int curq; + +static const char *n2ip(const unsigned char *c) { + static char b[sizeof("255.255.255.255")]; + sprintf(b, "%u.%u.%u.%u", c[0], c[1], c[2], c[3]); + return b; +} +static void dnscb(struct dns_ctx *ctx, struct dns_rr_ptr *rr, void *data) { + const char *ip = n2ip((unsigned char *)&data); + int i; + --curq; + if (rr) { + printf("%s", ip); + for(i = 0; i < rr->dnsptr_nrr; ++i) + printf(" %s", rr->dnsptr_ptr[i]); + putchar('\n'); + free(rr); + } + else + fprintf(stderr, "%s: %s\n", ip, dns_strerror(dns_status(ctx))); +} + +int main(int argc, char **argv) { + int c; + time_t now; + int maxq = 10; + struct pollfd pfd; + char linebuf[1024]; + char *eol; + int eof; + + if (dns_init(NULL, 1) < 0) { + fprintf(stderr, "unable to initialize dns library\n"); + return 1; + } + while((c = getopt(argc, argv, "m:r")) != EOF) switch(c) { + case 'm': maxq = atoi(optarg); break; + case 'r': + dns_set_opt(0, DNS_OPT_FLAGS, + dns_set_opt(0, DNS_OPT_FLAGS, -1) | DNS_NORD); + break; + default: return 1; + } + if (argc != optind) return 1; + + pfd.fd = dns_sock(0); + pfd.events = POLLIN; + now = time(NULL); + c = optind; + eof = 0; + while(curq || !eof) { + if (!eof && curq < maxq) { + union { struct in_addr a; void *p; } pa; + if (!fgets(linebuf, sizeof(linebuf), stdin)) { + eof = 1; + continue; + } + eol = strchr(linebuf, '\n'); + if (eol) *eol = '\0'; + if (!linebuf[0]) continue; + if (dns_pton(AF_INET, linebuf, &pa.a) <= 0) + fprintf(stderr, "%s: invalid address\n", linebuf); + else if (dns_submit_a4ptr(0, &pa.a, dnscb, pa.p) == 0) + fprintf(stderr, "%s: unable to submit query: %s\n", + linebuf, dns_strerror(dns_status(0))); + else + ++curq; + continue; + } + if (curq) { + c = dns_timeouts(0, -1, now); + c = poll(&pfd, 1, c < 0 ? -1 : c * 1000); + now = time(NULL); + if (c) + dns_ioevent(0, now); + } + } + return 0; +} diff --git a/contrib/udns/getopt.c b/contrib/udns/getopt.c new file mode 100644 index 00000000000..e15a7a0579e --- /dev/null +++ b/contrib/udns/getopt.c @@ -0,0 +1,165 @@ +/* getopt.c + * Simple getopt() implementation. + * + * Standard interface: + * extern int getopt(int argc, char *const *argv, const char *opts); + * extern int optind; current index in argv[] + * extern char *optarg; argument for the current option + * extern int optopt; the current option + * extern int opterr; to control error printing + * + * Some minor extensions: + * ignores leading `+' sign in opts[] (unemplemented GNU extension) + * handles optional arguments, in form "x::" in opts[] + * if opts[] starts with `:', will return `:' in case of missing required + * argument, instead of '?'. + * + * Compile with -DGETOPT_NO_OPTERR to never print errors internally. + * Compile with -DGETOPT_NO_STDIO to use write() calls instead of fprintf() for + * error reporting (ignored with -DGETOPT_NO_OPTERR). + * Compile with -DGETOPT_CLASS=static to get static linkage. + * Compile with -DGETOPT_MY to redefine all visible symbols to be prefixed + * with "my_", like my_getopt instead of getopt. + * Compile with -DTEST to get a test executable. + * + * Written by Michael Tokarev. Public domain. + */ + +#include + +#ifndef GETOPT_CLASS +# define GETOPT_CLASS +#endif +#ifdef GETOPT_MY +# define optarg my_optarg +# define optind my_optind +# define opterr my_opterr +# define optopt my_optopt +# define getopt my_getopt +#endif + +GETOPT_CLASS char *optarg /* = NULL */; +GETOPT_CLASS int optind = 1; +GETOPT_CLASS int opterr = 1; +GETOPT_CLASS int optopt; + +static char *nextc /* = NULL */; + +#if defined(GETOPT_NO_OPTERR) + +#define printerr(argv, msg) + +#elif defined(GETOPT_NO_STDIO) + +extern int write(int, void *, int); + +static void printerr(char *const *argv, const char *msg) { + if (opterr) { + char buf[64]; + unsigned pl = strlen(argv[0]); + unsigned ml = strlen(msg); + char *p; + if (pl + /*": "*/2 + ml + /*" -- c\n"*/6 > sizeof(buf)) { + write(2, argv[0], pl); + p = buf; + } + else { + memcpy(buf, argv[0], ml); + p = buf + pl; + } + *p++ = ':'; *p++ = ' '; + memcpy(p, msg, ml); p += ml; + *p++ = ' '; *p++ = '-'; *p++ = '-'; *p++ = ' '; + *p++ = optopt; + *p++ = '\n'; + write(2, buf, p - buf); + } +} + +#else + +#include +static void printerr(char *const *argv, const char *msg) { + if (opterr) + fprintf(stderr, "%s: %s -- %c\n", argv[0], msg, optopt); +} + +#endif + +GETOPT_CLASS int getopt(int argc, char *const *argv, const char *opts) { + char *p; + + optarg = 0; + if (*opts == '+') /* GNU extension (permutation) - isn't supported */ + ++opts; + + if (!optind) { /* a way to reset things */ + nextc = 0; + optind = 1; + } + + if (!nextc || !*nextc) { /* advance to the next argv element */ + /* done scanning? */ + if (optind >= argc) + return -1; + /* not an optional argument */ + if (argv[optind][0] != '-') + return -1; + /* bare `-' */ + if (argv[optind][1] == '\0') + return -1; + /* special case `--' argument */ + if (argv[optind][1] == '-' && argv[optind][2] == '\0') { + ++optind; + return -1; + } + nextc = argv[optind] + 1; + } + + optopt = *nextc++; + if (!*nextc) + ++optind; + p = strchr(opts, optopt); + if (!p || optopt == ':') { + printerr(argv, "illegal option"); + return '?'; + } + if (p[1] == ':') { + if (*nextc) { + optarg = nextc; + nextc = NULL; + ++optind; + } + else if (p[2] != ':') { /* required argument */ + if (optind >= argc) { + printerr(argv, "option requires an argument"); + return *opts == ':' ? ':' : '?'; + } + else + optarg = argv[optind++]; + } + } + return optopt; +} + +#ifdef TEST + +#include + +int main(int argc, char **argv) { + int c; + while((c = getopt(argc, argv, "ab:c::")) != -1) switch(c) { + case 'a': + case 'b': + case 'c': + printf("option %c %s\n", c, optarg ? optarg : "(none)"); + break; + default: + return -1; + } + for(c = optind; c < argc; ++c) + printf("non-opt: %s\n", argv[c]); + return 0; +} + +#endif diff --git a/contrib/udns/inet_XtoX.c b/contrib/udns/inet_XtoX.c new file mode 100644 index 00000000000..50b5f8e81f3 --- /dev/null +++ b/contrib/udns/inet_XtoX.c @@ -0,0 +1,327 @@ +/* inet_XtoX.c + * Simple implementation of the following functions: + * inet_ntop(), inet_ntoa(), inet_pton(), inet_aton(). + * + * Differences from traditional implementaitons: + * o modifies destination buffers even on error return. + * o no fancy (hex, or 1.2) input support in inet_aton() + * o inet_aton() does not accept junk after an IP address. + * o inet_ntop(AF_INET) requires at least 16 bytes in dest, + * and inet_ntop(AF_INET6) at least 40 bytes + * (traditional inet_ntop() will try to fit anyway) + * + * Compile with -Dinet_XtoX_prefix=pfx_ to have pfx_*() instead of inet_*() + * Compile with -Dinet_XtoX_no_ntop or -Dinet_XtoX_no_pton + * to disable net2str or str2net conversions. + * + * #define inet_XtoX_prototypes and #include "this_file.c" + * to get function prototypes only (but not for inet_ntoa()). + * #define inet_XtoX_decl to be `static' for static visibility, + * or use __declspec(dllexport) or somesuch... + * + * Compile with -DTEST to test against stock implementation. + * + * Written by Michael Tokarev. Public domain. + */ + +#ifdef inet_XtoX_prototypes + +struct in_addr; + +#else + +#include + +#ifdef TEST + +# include +# include +# include +# include +# include +# include +# include +# undef inet_XtoX_prefix +# define inet_XtoX_prefix mjt_inet_ +# undef inet_XtoX_no_ntop +# undef inet_XtoX_no_pton + +#else /* !TEST */ + +struct in_addr { /* declare it here to avoid messing with headers */ + unsigned char x[4]; +}; + +#endif /* TEST */ + +#endif /* inet_XtoX_prototypes */ + +#ifndef inet_XtoX_prefix +# define inet_XtoX_prefix inet_ +#endif +#ifndef inet_XtoX_decl +# define inet_XtoX_decl /*empty*/ +#endif + +#define cc2_(x,y) cc2__(x,y) +#define cc2__(x,y) x##y +#define fn(x) cc2_(inet_XtoX_prefix,x) + +#ifndef inet_XtoX_no_ntop + +inet_XtoX_decl const char * +fn(ntop)(int af, const void *src, char *dst, unsigned size); + +#ifndef inet_XtoX_prototypes + +static int mjt_ntop4(const void *_src, char *dst, int size) { + unsigned i, x, r; + char *p; + const unsigned char *s = _src; + if (size < 4*4) /* for simplicity, disallow non-max-size buffer */ + return 0; + for (i = 0, p = dst; i < 4; ++i) { + if (i) *p++ = '.'; + x = r = s[i]; + if (x > 99) { *p++ = (char)(r / 100 + '0'); r %= 100; } + if (x > 9) { *p++ = (char)(r / 10 + '0'); r %= 10; } + *p++ = (char)(r + '0'); + } + *p = '\0'; + return 1; +} + +static char *hexc(char *p, unsigned x) { + static char hex[16] = "0123456789abcdef"; + if (x > 0x0fff) *p++ = hex[(x >>12) & 15]; + if (x > 0x00ff) *p++ = hex[(x >> 8) & 15]; + if (x > 0x000f) *p++ = hex[(x >> 4) & 15]; + *p++ = hex[x & 15]; + return p; +} + +static int mjt_ntop6(const void *_src, char *dst, int size) { + unsigned i; + unsigned short w[8]; + unsigned bs = 0, cs = 0; + unsigned bl = 0, cl = 0; + char *p; + const unsigned char *s = _src; + + if (size < 40) /* for simplicity, disallow non-max-size buffer */ + return 0; + + for(i = 0; i < 8; ++i, s += 2) { + w[i] = (((unsigned short)(s[0])) << 8) | s[1]; + if (!w[i]) { + if (!cl++) cs = i; + } + else { + if (cl > bl) bl = cl, bs = cs; + } + } + if (cl > bl) bl = cl, bs = cs; + p = dst; + if (bl == 1) + bl = 0; + if (bl) { + for(i = 0; i < bs; ++i) { + if (i) *p++ = ':'; + p = hexc(p, w[i]); + } + *p++ = ':'; + i += bl; + if (i == 8) + *p++ = ':'; + } + else + i = 0; + for(; i < 8; ++i) { + if (i) *p++ = ':'; + if (i == 6 && !bs && (bl == 6 || (bl == 5 && w[5] == 0xffff))) + return mjt_ntop4(s - 4, p, size - (p - dst)); + p = hexc(p, w[i]); + } + *p = '\0'; + return 1; +} + +inet_XtoX_decl const char * +fn(ntop)(int af, const void *src, char *dst, unsigned size) { + switch(af) { + /* don't use AF_*: don't mess with headers */ + case 2: /* AF_INET */ if (mjt_ntop4(src, dst, size)) return dst; break; + case 10: /* AF_INET6 */ if (mjt_ntop6(src, dst, size)) return dst; break; + default: errno = EAFNOSUPPORT; return (char*)0; + } + errno = ENOSPC; + return (char*)0; +} + +inet_XtoX_decl const char * +fn(ntoa)(struct in_addr addr) { + static char buf[4*4]; + mjt_ntop4(&addr, buf, sizeof(buf)); + return buf; +} + +#endif /* inet_XtoX_prototypes */ +#endif /* inet_XtoX_no_ntop */ + +#ifndef inet_XtoX_no_pton + +inet_XtoX_decl int fn(pton)(int af, const char *src, void *dst); +inet_XtoX_decl int fn(aton)(const char *src, struct in_addr *addr); + +#ifndef inet_XtoX_prototypes + +static int mjt_pton4(const char *c, void *dst) { + unsigned char *a = dst; + unsigned n, o; + for (n = 0; n < 4; ++n) { + if (*c < '0' || *c > '9') + return 0; + o = *c++ - '0'; + while(*c >= '0' && *c <= '9') + if ((o = o * 10 + (*c++ - '0')) > 255) + return 0; + if (*c++ != (n == 3 ? '\0' : '.')) + return 0; + *a++ = (unsigned char)o; + } + return 1; +} + +static int mjt_pton6(const char *c, void *dst) { + unsigned short w[8], *a = w, *z, *i; + unsigned v, o; + const char *sc; + unsigned char *d = dst; + if (*c != ':') z = (unsigned short*)0; + else if (*++c != ':') return 0; + else ++c, z = a; + i = 0; + for(;;) { + v = 0; + sc = c; + for(;;) { + if (*c >= '0' && *c <= '9') o = *c - '0'; + else if (*c >= 'a' && *c <= 'f') o = *c - 'a' + 10; + else if (*c >= 'A' && *c <= 'F') o = *c - 'A' + 10; + else break; + v = (v << 4) | o; + if (v > 0xffff) return 0; + ++c; + } + if (sc == c) { + if (z == a && !*c) + break; + else + return 0; + } + if (*c == ':') { + if (a >= w + 8) + return 0; + *a++ = v; + if (*++c == ':') { + if (z) + return 0; + z = a; + if (!*++c) + break; + } + } + else if (!*c) { + if (a >= w + 8) + return 0; + *a++ = v; + break; + } + else if (*c == '.') { + if (a > w + 6) + return 0; + if (!mjt_pton4(sc, d)) + return 0; + *a++ = ((unsigned)(d[0]) << 8) | d[1]; + *a++ = ((unsigned)(d[2]) << 8) | d[3]; + break; + } + else + return 0; + } + v = w + 8 - a; + if ((v && !z) || (!v && z)) + return 0; + for(i = w; ; ++i) { + if (i == z) + while(v--) { *d++ = '\0'; *d++ = '\0'; } + if (i >= a) + break; + *d++ = (unsigned char)((*i >> 8) & 255); + *d++ = (unsigned char)(*i & 255); + } + return 1; +} + +inet_XtoX_decl int fn(pton)(int af, const char *src, void *dst) { + switch(af) { + /* don't use AF_*: don't mess with headers */ + case 2 /* AF_INET */: return mjt_pton4(src, dst); + case 10 /* AF_INET6 */: return mjt_pton6(src, dst); + default: errno = EAFNOSUPPORT; return -1; + } +} + +inet_XtoX_decl int fn(aton)(const char *src, struct in_addr *addr) { + return mjt_pton4(src, addr); +} + +#endif /* inet_XtoX_prototypes */ + +#endif /* inet_XtoX_no_pton */ + +#ifdef TEST + +int main(int argc, char **argv) { + int i; + char n0[16], n1[16]; + char p0[64], p1[64]; + int af = AF_INET; + int pl = sizeof(p0); + int r0, r1; + const char *s0, *s1; + + while((i = getopt(argc, argv, "46a:p:")) != EOF) switch(i) { + case '4': af = AF_INET; break; + case '6': af = AF_INET6; break; + case 'a': case 'p': pl = atoi(optarg); break; + default: return 1; + } + for(i = optind; i < argc; ++i) { + char *a = argv[i]; + + printf("%s:\n", a); + r0 = inet_pton(af, a, n0); + printf(" p2n stock: %s\n", + (r0 < 0 ? "(notsupp)" : !r0 ? "(inval)" : fn(ntop)(af,n0,p0,sizeof(p0)))); + r1 = fn(pton)(af, a, n1); + printf(" p2n this : %s\n", + (r1 < 0 ? "(notsupp)" : !r1 ? "(inval)" : fn(ntop)(af,n1,p1,sizeof(p1)))); + + if ((r0 > 0) != (r1 > 0) || + (r0 > 0 && r1 > 0 && memcmp(n0, n1, af == AF_INET ? 4 : 16) != 0)) + printf(" DIFFER!\n"); + + s0 = inet_ntop(af, n1, p0, pl); + printf(" n2p stock: %s\n", s0 ? s0 : "(inval)"); + s1 = fn(ntop)(af, n1, p1, pl); + printf(" n2p this : %s\n", s1 ? s1 : "(inval)"); + if ((s0 != 0) != (s1 != 0) || + (s0 && s1 && strcmp(s0, s1) != 0)) + printf(" DIFFER!\n"); + + } + return 0; +} + +#endif /* TEST */ diff --git a/contrib/udns/rblcheck.1 b/contrib/udns/rblcheck.1 new file mode 100644 index 00000000000..6c822c01c0b --- /dev/null +++ b/contrib/udns/rblcheck.1 @@ -0,0 +1,151 @@ +.\" rblcheck.1 +.\" rblckeck manpage +.\" +.\" Copyright (C) 2005 Michael Tokarev +.\" This file is part of UDNS library, an async DNS stub resolver. +.\" +.\" This library is free software; you can redistribute it and/or +.\" modify it under the terms of the GNU Lesser General Public +.\" License as published by the Free Software Foundation; either +.\" version 2.1 of the License, or (at your option) any later version. +.\" +.\" This library is distributed in the hope that it will be useful, +.\" but WITHOUT ANY WARRANTY; without even the implied warranty of +.\" MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU +.\" Lesser General Public License for more details. +.\" +.\" You should have received a copy of the GNU Lesser General Public +.\" License along with this library, in file named COPYING.LGPL; if not, +.\" write to the Free Software Foundation, Inc., 59 Temple Place, +.\" Suite 330, Boston, MA 02111-1307 USA + +.TH rblckeck 1 "Apr 2005" "User Utilities" + +.SH NAME +rblckeck \- DNSBL lookup utility + +.SH SYNOPSYS +.B rblcheck +.RB [\| \-s +.IR zone \|] +.RB [\| \-S +.IR zone\-file \|] +.RB [\| \-c \|] +.RB [\| \-tmvq \|] +.RB [\| \-n +.IR nsaddr \|] +.IR address \|.\|.\|. + +.SH DESCRIPTION +.B rblcheck +is a simple command-line to perform DNSBL (DNS-based blocklists) lookups. +For every IP address (or a name, in which case it will be resolved to an +address first), the utility verifies whenever it is listed in a (list of) +DNS blocklists specified with +.B \-s +or +.B \-S +options, optionally obtains text assotiated with the listing (usually it +is either some description about the reason of the listing or an URL +referring to such a description), and displays results on standard output. +.PP +The program is implemented on top of +.BR udns (3) +library. + +.SH OPTIONS + +The following options are recognized by +.BR rblcheck : + +.TP +.B \-s \fIzone\fR +add the given \fIzone\fR DNSBL name to the list of active zones. +.TP +.B \-S \fIzone-file\fR +add list of zones from the named \fIzone-file\fR to the list of +active zones (the file specifies one zone as the first word on a +line, empty lines and lines starting with `#' character are ignored). +.TP +.B \-c +reset active zone list. +.TP +.B \-v +be more verbose, produce more detailed output. +.TP +.B \-q +the opposite for \fB\-v\fR -- produce less detailed output. +.TP +.B \-t +obtain text for listed addresses. +.TP +.B \-n \fInsaddr\fR +Use the given nameserver (given as IPv4 or IPv6 address) instead of the +default. The same effect may be achieved by setting $NSCACHEIP environment +variable. +.TP +.B \-m +stop after first hit, ie after the first address which is found to be +listed. + +.TP +.B \-h +print short help and exit. + +.PP +If no +.BR \-s , +.BR \-S +and +.B \-c +options are given, +.B rblcheck +will try to obtain list of zones using $RBLCHECK_ZONES environment variable, +or ~/.rblcheckrc, or /etc/rblckechrc files, in that order. If no zones are +found, it will exit unsuccessefully. + +.SH "RETURN VALUE" +When no addresses given are listed and no errors occured, +.B rblcheck +exits with code 0. If at least one address is listed, +.B rblcheck +returns 100. In case of DNS errors, +.B rblcheck +returns 2. + +.SH ENVIRONMENT + +.TP +.B $RBLCHECK_ZONES +if no +.BR \-s , +.B \-S +or +.B \-c +option is given, +.B rblcheck +tries this variable to obtain list of DNSBL zones to check against. + +.SH FILES + +.TP +$HOME/.rblcheckrc and /etc/rblcheckrc +if no +.BR \-s , +.B \-S +or +.B \-c +option is given, and no $RBLCHECK_ZONES environment variable is set, +.B rblcheck +will try the two files (the first one that exists) to obtain list of +DNSBL zones to check against. +Each line specifies one zone (only first word in each line is used). +Empty lines and lines starting with `#' character are ignored. + +.SH "SEE ALSO" +.BR dnsget (1) +.BR resolv.conf (5) +.BR udns (3). + +.SH AUTHOR +This program and manual pages are written by Michael Tokarev. diff --git a/contrib/udns/rblcheck.c b/contrib/udns/rblcheck.c new file mode 100644 index 00000000000..82d29deeebf --- /dev/null +++ b/contrib/udns/rblcheck.c @@ -0,0 +1,378 @@ +/* rblcheck.c + dnsbl (rbl) checker application + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#ifdef HAVE_CONFIG_H +# include "config.h" +#endif +#include +#include +#include +#ifdef WINDOWS +# include +#else +# include +# include +# include +# include +#endif +#include +#include +#include +#include "udns.h" + +#ifndef HAVE_GETOPT +# include "getopt.c" +#endif + +static const char *version = "udns-rblcheck 0.4"; +static char *progname; + +static void error(int die, const char *fmt, ...) { + va_list ap; + fprintf(stderr, "%s: ", progname); + va_start(ap, fmt); vfprintf(stderr, fmt, ap); va_end(ap); + putc('\n', stderr); + fflush(stderr); + if (die) + exit(1); +} + +struct rblookup { + struct ipcheck *parent; + struct in_addr key; + const char *zone; + struct dns_rr_a4 *addr; + struct dns_rr_txt *txt; +}; + +struct ipcheck { + const char *name; + int naddr; + int listed; + struct rblookup *lookup; +}; + +#define notlisted ((void*)1) + +static int nzones, nzalloc; +static const char **zones; + +static int do_txt; +static int stopfirst; +static int verbose = 1; +/* verbosity level: + * <0 - only bare As/TXTs + * 0 - what RBL result + * 1(default) - what is listed by RBL: result + * 2 - what is[not ]listed by RBL: result, name lookups + */ + +static int listed; +static int failures; + +static void *ecalloc(int size, int cnt) { + void *t = calloc(size, cnt); + if (!t) + error(1, "out of memory"); + return t; +} + +static void addzone(const char *zone) { + if (nzones >= nzalloc) { + const char **zs = (const char**)ecalloc(sizeof(char*), (nzalloc += 16)); + if (zones) { + memcpy(zs, zones, nzones * sizeof(char*)); + free(zones); + } + zones = zs; + } + zones[nzones++] = zone; +} + +static int addzonefile(const char *fname) { + FILE *f = fopen(fname, "r"); + char linebuf[2048]; + if (!f) + return 0; + while(fgets(linebuf, sizeof(linebuf), f)) { + char *p = linebuf, *e; + while(*p == ' ' || *p == '\t') ++p; + if (*p == '#' || *p == '\n') continue; + e = p; + while(*e && *e != ' ' && *e != '\t' && *e != '\n') + ++e; + *e++ = '\0'; + p = memcpy(ecalloc(e - p, 1), p, e - p); // strdup + addzone(p); + } + fclose(f); + return 1; +} + +static void dnserror(struct rblookup *ipl, const char *what) { + char buf[4*4]; + error(0, "unable to %s for %s (%s): %s", + what, dns_ntop(AF_INET, &ipl->key, buf, sizeof(buf)), + ipl->zone, dns_strerror(dns_status(0))); + ++failures; +} + +static void display_result(struct ipcheck *ipc) { + int j; + struct rblookup *l, *le; + char buf[4*4]; + if (!ipc->naddr) return; + for (l = ipc->lookup, le = l + nzones * ipc->naddr; l < le; ++l) { + if (!l->addr) continue; + if (verbose < 2 && l->addr == notlisted) continue; + if (verbose >= 0) { + dns_ntop(AF_INET, &l->key, buf, sizeof(buf)); + if (ipc->name) printf("%s[%s]", ipc->name, buf); + else printf("%s", buf); + } + if (l->addr == notlisted) { + printf(" is NOT listed by %s\n", l->zone); + continue; + } + else if (verbose >= 1) + printf(" is listed by %s: ", l->zone); + else if (verbose >= 0) + printf(" %s ", l->zone); + if (verbose >= 1 || !do_txt) + for (j = 0; j < l->addr->dnsa4_nrr; ++j) + printf("%s%s", j ? " " : "", + dns_ntop(AF_INET, &l->addr->dnsa4_addr[j], buf, sizeof(buf))); + if (!do_txt) ; + else if (l->txt) { + for(j = 0; j < l->txt->dnstxt_nrr; ++j) { + unsigned char *t = l->txt->dnstxt_txt[j].txt; + unsigned char *e = t + l->txt->dnstxt_txt[j].len; + printf("%s\"", verbose > 0 ? "\n\t" : j ? " " : ""); + while(t < e) { + if (*t < ' ' || *t >= 127) printf("\\x%02x", *t); + else if (*t == '\\' || *t == '"') printf("\\%c", *t); + else putchar(*t); + ++t; + } + putchar('"'); + } + free(l->txt); + } + else + printf("%s", verbose > 0 ? "\n\t" : ""); + free(l->addr); + putchar('\n'); + } + free(ipc->lookup); +} + +static void txtcb(struct dns_ctx *ctx, struct dns_rr_txt *r, void *data) { + struct rblookup *ipl = data; + if (r) { + ipl->txt = r; + ++ipl->parent->listed; + } + else if (dns_status(ctx) != DNS_E_NXDOMAIN) + dnserror(ipl, "lookup DNSBL TXT record"); +} + +static void a4cb(struct dns_ctx *ctx, struct dns_rr_a4 *r, void *data) { + struct rblookup *ipl = data; + if (r) { + ipl->addr = r; + ++listed; + if (do_txt) { + if (dns_submit_a4dnsbl_txt(0, &ipl->key, ipl->zone, txtcb, ipl)) + return; + dnserror(ipl, "submit DNSBL TXT record"); + } + ++ipl->parent->listed; + } + else if (dns_status(ctx) != DNS_E_NXDOMAIN) + dnserror(ipl, "lookup DNSBL A record"); + else + ipl->addr = notlisted; +} + +static int +submit_a_queries(struct ipcheck *ipc, + int naddr, const struct in_addr *addr) { + int z, a; + struct rblookup *rl = ecalloc(sizeof(*rl), nzones * naddr); + ipc->lookup = rl; + ipc->naddr = naddr; + for(a = 0; a < naddr; ++a) { + for(z = 0; z < nzones; ++z) { + rl->key = addr[a]; + rl->zone = zones[z]; + rl->parent = ipc; + if (!dns_submit_a4dnsbl(0, &rl->key, rl->zone, a4cb, rl)) + dnserror(rl, "submit DNSBL A query"); + ++rl; + } + } + return 0; +} + +static void namecb(struct dns_ctx *ctx, struct dns_rr_a4 *rr, void *data) { + struct ipcheck *ipc = data; + if (rr) { + submit_a_queries(ipc, rr->dnsa4_nrr, rr->dnsa4_addr); + free(rr); + } + else { + error(0, "unable to lookup `%s': %s", + ipc->name, dns_strerror(dns_status(ctx))); + ++failures; + } +} + +static int submit(struct ipcheck *ipc) { + struct in_addr addr; + if (dns_pton(AF_INET, ipc->name, &addr) > 0) { + submit_a_queries(ipc, 1, &addr); + ipc->name = NULL; + } + else if (!dns_submit_a4(0, ipc->name, 0, namecb, ipc)) { + error(0, "unable to submit name query for %s: %s\n", + ipc->name, dns_strerror(dns_status(0))); + ++failures; + } + return 0; +} + +static void waitdns(struct ipcheck *ipc) { + struct timeval tv; + fd_set fds; + int c; + int fd = dns_sock(NULL); + time_t now = 0; + FD_ZERO(&fds); + while((c = dns_timeouts(NULL, -1, now)) > 0) { + FD_SET(fd, &fds); + tv.tv_sec = c; + tv.tv_usec = 0; + c = select(fd+1, &fds, NULL, NULL, &tv); + now = time(NULL); + if (c > 0) + dns_ioevent(NULL, now); + if (stopfirst && ipc->listed) + break; + } +} + +int main(int argc, char **argv) { + int c; + struct ipcheck ipc; + char *nameserver = NULL; + int zgiven = 0; + + if (!(progname = strrchr(argv[0], '/'))) progname = argv[0]; + else argv[0] = ++progname; + + while((c = getopt(argc, argv, "hqtvms:S:cn:")) != EOF) switch(c) { + case 's': ++zgiven; addzone(optarg); break; + case 'S': + ++zgiven; + if (addzonefile(optarg)) break; + error(1, "unable to read zonefile `%s'", optarg); + case 'c': ++zgiven; nzones = 0; break; + case 'q': --verbose; break; + case 'v': ++verbose; break; + case 't': do_txt = 1; break; + case 'n': nameserver = optarg; break; + case 'm': ++stopfirst; break; + case 'h': + printf("%s: %s (udns library version %s).\n", + progname, version, dns_version()); + printf("Usage is: %s [options] address..\n", progname); + printf( +"Where options are:\n" +" -h - print this help and exit\n" +" -s service - add the service (DNSBL zone) to the serice list\n" +" -S service-file - add the DNSBL zone(s) read from the given file\n" +" -c - clear service list\n" +" -v - increase verbosity level (more -vs => more verbose)\n" +" -q - decrease verbosity level (opposite of -v)\n" +" -t - obtain and print TXT records if any\n" +" -m - stop checking after first address match in any list\n" +" -n ipaddr - use the given nameserver instead of the default\n" +"(if no -s or -S option is given, use $RBLCHECK_ZONES, ~/.rblcheckrc\n" +"or /etc/rblcheckrc in that order)\n" + ); + return 0; + default: + error(1, "use `%s -h' for help", progname); + } + + if (!zgiven) { + char *s = getenv("RBLCHECK_ZONES"); + if (s) { + char *k; + s = strdup(s); + for(k = strtok(s, " \t"); k; k = strtok(NULL, " \t")) + addzone(k); + free(s); + } + else { /* probably worthless on windows? */ + char *path; + char *home = getenv("HOME"); + if (!home) home = "."; + path = malloc(strlen(home) + 1 + sizeof(".rblcheckrc")); + sprintf(path, "%s/.rblcheckrc", home); + if (!addzonefile(path)) + addzonefile("/etc/rblcheckrc"); + free(path); + } + } + if (!nzones) + error(1, "no service (zone) list specified (-s or -S option)"); + + argv += optind; + argc -= optind; + + if (!argc) + return 0; + + if (dns_init(NULL, 0) < 0) + error(1, "unable to initialize DNS library: %s", strerror(errno)); + if (nameserver) { + dns_add_serv(NULL, NULL); + if (dns_add_serv(NULL, nameserver) < 0) + error(1, "wrong IP address for a nameserver: `%s'", nameserver); + } + if (dns_open(NULL) < 0) + error(1, "unable to initialize DNS library: %s", strerror(errno)); + + for (c = 0; c < argc; ++c) { + if (c && (verbose > 1 || (verbose == 1 && do_txt))) putchar('\n'); + memset(&ipc, 0, sizeof(ipc)); + ipc.name = argv[c]; + submit(&ipc); + waitdns(&ipc); + display_result(&ipc); + if (stopfirst > 1 && listed) break; + } + + return listed ? 100 : failures ? 2 : 0; +} diff --git a/contrib/udns/udns.3 b/contrib/udns/udns.3 new file mode 100644 index 00000000000..23222aae9f7 --- /dev/null +++ b/contrib/udns/udns.3 @@ -0,0 +1,1352 @@ +.\" udns.3 +.\" udns library manpage +.\" +.\" Copyright (C) 2005-2014 Michael Tokarev +.\" This file is part of UDNS library, an async DNS stub resolver. +.\" +.\" This library is free software; you can redistribute it and/or +.\" modify it under the terms of the GNU Lesser General Public +.\" License as published by the Free Software Foundation; either +.\" version 2.1 of the License, or (at your option) any later version. +.\" +.\" This library is distributed in the hope that it will be useful, +.\" but WITHOUT ANY WARRANTY; without even the implied warranty of +.\" MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU +.\" Lesser General Public License for more details. +.\" +.\" You should have received a copy of the GNU Lesser General Public +.\" License along with this library, in file named COPYING.LGPL; if not, +.\" write to the Free Software Foundation, Inc., 59 Temple Place, +.\" Suite 330, Boston, MA 02111-1307 USA + +.TH udns 3 "Jan 2014" "Library Functions" + +.SH NAME +udns \- stub DNS resolver library + +.SH SYNOPSYS +.nf +#include +struct \fBdns_ctx\fR; +struct \fBdns_query\fR; +extern struct dns_ctx \fBdns_defctx\fR; +struct dns_ctx *\fIctx\fR; +typedef void \fBdns_query_fn\fR(\fIctx\fR, void *\fIresult\fR, void *\fIdata\fR); +typedef int +\fBdns_parse_fn\fR(const unsigned char *\fIqnd\fR, + const unsigned char *\fIpkt\fR, + const unsigned char *\fIcur\fR, + const unsigned char *\fIend\fR, + void **\fIresultp\fR); + +\fBcc\fR ... -l\fBudns\fR +.fi + +.SH DESCRIPTION + +.PP +The DNS library, \fBudns\fR, implements thread-safe stub DNS resolver +functionality, which may be used both traditional, syncronous way +and asyncronously, with application-supplied event loop. + +.PP +While DNS works with both TCP and UDP, performing UDP query first and +if the result does not fit in UDP buffer (512 bytes max for original +DNS protocol), retrying the query over TCP, the library uses UDP only, +but uses EDNS0 (RFC2671) extensions which allows larger UDP buffers. + +.PP +The library uses single UDP socket to perform all operations even when +asking multiple nameservers. This way, it is very simple to use the +library in asyncronous event-loop applications: an application should +add only single socket to the set of filedescriptors it monitors for I/O. + +.PP +The library uses two main objects, \fIresolver context\fR of type +\fBstruct\ dns_ctx\fR, and \fIquery structure\fR of type +\fBstruct\ dns_query\fR, both are opaque for an application. +Resolver context holds global information about the resolver, +such as list of nameservers to use, list of active requests and the like. +Query objects holds information about a single DNS query in progress and +are allocated/processed/freed by the library. Pointer to query structure +may be treated as an identifier of an in-progress query and may be used +to cancel the asyncronous query or to wait for it to complete. + +.PP +Asyncronous interface works as follows. An application initializes +resolver context, submits any number of queries for it using one of +supplied \fBdns_submit_\fIXXX\fR() routines (each return the query +identifier as pointer to query structure), waits for input on the +UDP socket used by the library, and gives some control to the library +by calling \fBdns_ioevent\fR() and \fBdns_timeouts\fR() routines when +appropriate. The library performs all necessary processing and executes +application supplied callback routine when a query completes (either +successefully or not), giving it the result if any, pointer to the +resolver context (from which completion status may be obtained), and +the data pointer supplied by an application when the query has been +submitted. When submitting a query, an application requests how to +handle the reply -- to either return raw DNS reply packet for its +own low-level processing, or it may provide an address of \fIparsing +routine\fR of type \fBdns_parse_fn\fR to perform conversion of on-wire +format into easy to use data structure (the library provides parsing +routines for several commonly used resource record types, as well as +type-safe higher-level inteface that requests parsing automatically). +The I/O monitoring and timeout handling may be either traditional +select() or poll() based, or any callback-driven technique may be +used. + +.PP +Additionally, the library provides traditional syncronous interface, +which may be intermixed with asyncronous calls (during syncronous +query processing, other asyncronous queries for the same resolver +context continued to be processed as usual). An application uses +one of numerous \fBdns_resolve_\fIXXX\fR() routines provided by the +library to perform a query. As with asyncronous interface, an +application may either request to return raw DNS packet or type-specific +data structure by providing the parsing routine to handle the reply. +Every routine from \fBdns_resolve_\fIXXX\fR() series return pointer +to result or NULL in case of any error. Query completion status +(or length of the raw DNS packet) is available from the resolver +context using \fBdns_status\fR() routine, the same way as for the +asyncronous interface. + +.PP +Internally, library uses on-wire format of domain names, referred +to as \fIDN format\fR in this manual page. This is a series of domain +\fIlabels\fR whith preceeding length byte, terminated by zero-length +label wich is integral part of the DN format. There are several routines +provided to convert from traditional asciiz string to DN and back. +Higher-level type-specific query interface hides the DN format from +an application. + +.SH "COMMON DEFINITIONS" + +.PP +Every DNS Resource Record (RR) has a \fItype\fR and a \fIclass\fR. +The library defines several integer constants, \fBDNS_C_\fIXXX\fR and +\fBDNS_T_\fIXXX\fR, to use as symbolic names for RR classes and types, +such as \fBDNS_C_IN\fR for Internet class, \fBDNS_T_A\fR for IPv4 +address record type and so on. See udns.h header file for complete list +of all such constants. + +.PP +The following constants are defined in udns.h header file: +.IP "\fBDNS_MAXDN\fR (255 bytes)" +Maximum length of the domain name in internal (on-wire) DN format. +.IP "\fBDNS_MAXLABEL\fR (63 bytes)" +Maximum length of a single label in DN format. +.IP "\fBDNS_MAXNAME\fR (1024 bytes)" +Maximum length of asciiz format of a domain name. +.IP "\fBDNS_HSIZE\fR (12 bytes)" +Size of header in DNS packet. +.IP "\fBDNS_PORT\fR (53)" +Default port to use when contacting a DNS server. +.IP "\fBDNS_MAXSERV\fR (6 servers)" +Maximum number of DNS servers to use. +.IP "\fBDNS_MAXPACKET\fR (512 bytes)" +Maximum length of DNS UDP packet as specified by original DNS protocol +.IP "\fBDNS_EDNS0PACKET\fR (4096 bytes)" +Default length of DNS UDP packet (with EDNS0 extensions) the library uses. +Note that recursive nameservers usually resides near the client asking them +to resolve names, e.g. on the same LAN segment or even on the same host, so +UDP packet fragmentation isn't a problem in most cases. Note also that +the size of actual packets will be as many bytes as actual reply size requires, +which is smaller than this value in almost all cases. + +.PP +Additionally, several constants are defined to simplify work with raw DNS +packets, such as DNS response codes (\fBDNS_R_\fIXXX\fR), DNS header layout +(\fBDNS_H_\fIXXX\fR) and others. Again, see udns.h for complete list. +Library error codes (\fBDNS_E_\fIXXX\fR) are described later in this +manual page. + +.SH "RESOLVER CONTEXT" + +.PP +Resolver context, of type \fBstruct\ dns_ctx\fR, is an object which is +opaque to an application. Several routines provided by the library +to initialize, copy and free resolver contexts. Most other high-level +routines in this library expects a pointer to resolver context, \fIctx\fR, +as the first argument. There is a default resolver context available, +named \fBdns_defctx\fR. When the context pointer \fIctx\fR passed to +a routine is NULL, \fBdns_defctx\fR is used. Several resolver contexts +may be active at the same time, for example, when an application is +multi-threaded and each thread uses resolver. +.PP +In order to use the library, an application should initialize and open +one or more resolver context objects. These are two separate actions, +performed by \fBdns_init\fR() (or \fBdns_reset\fR()), and \fBdns_open\fR(). +Between the two calls, an application is free to pefrorm additional +initialisation, such as setting custom nameservers, options or domain search +lists. Optionally, in case no additional custom initialisation is required, +\fBdns_init\fR() may open the context if \fIdo_open\fR argument (see below) +is non-zero. +.PP +When initializing resolver context, the library uses information from +system file /etc/resolv.conf (see \fBresolv.conf\fR(5)), consults +environment variables \fB$LOCALDOMAIN\fR, \fB$NSCACHEIP\fR, +\fB$NAMESERVERS\fR and \fB$RES_OPTIONS\fR, and local host name to obtain +list of local nameservers, domain name search list and various resolver +options. +.PP +The following routines to initialize resolver context are available: +.PP +.nf +void \fBdns_reset\fR(\fIctx\fR) +int \fBdns_init\fR(\fIctx\fR, int \fIdo_open\fR) +.fi +.RS +\fBdns_reset\fR() resets a given resolver context to default values, +preparing it to be opened by \fBdns_open\fR(). +It is ok to call this routine against opened and active context - all active +queries will be dropped, sockets will be closed and so on. This routine +does not initialize any parameters from system configuration files, use +\fBdns_init\fR() for this. There's no error return - operation always +succeeds. \fBdns_init\fR() does everything \fBdns_reset\fR() does, +plus initializes various parameters of the context according to system +configuration and process environment variables. If \fIdo_open\fR is +non-zero, \fBdns_init\fR() calls \fIdns_open\fR(), so that the whole +library initialisation is performed in a single step. +.RE +.PP +.nf +struct dns_ctx *\fBdns_new\fR(struct dns_ctx *\fIcopy\fR) +void \fBdns_free\fR(\fIctx\fR) +.fi +.RS +\fBdns_new\fR() allocates new resolver context and copies all parameters +for a given resolver context \fIcopy\fR, or default context if \fIcopy\fR +is NULL, and returns pointer to the newly allocated context. The context +being copied should be initialized. +\fBdns_new\fR() may fail if there's no memory available to make a copy +of \fIcopy\fR, in which case the routine will return NULL pointer. +\fBdns_free\fR() is used to close assotiated socket and free resolver +context resources and cancelling (abandoming) all active queries +assotiated with it. It's an error to free \fBdns_defctx\fR, only +dynamically allocated contexts returned by \fBdns_new\fR() are allowed +to be freed by \fBdns_free\fR(). +.RE +.PP +.nf +int \fBdns_add_serv\fR(\fIctx\fR, const char *\fIservaddr\fR) +int \fBdns_add_serv_s\fR(\fIctx\fR, const struct sockaddr *\fIsa\fR) +int \fBdns_add_srch\fR(\fIctx\fR, const char *\fIsrch\fR) +.fi +.RS +Add an element to list of nameservers (\fBdns_add_serv\fR(), as +asciiz-string \fIservaddr\fR with an IP address of the nameserver, +and \fBdns_add_serv_s\fR(), as initialized socket address \fIsa\fR), +or search list (\fBdns_add_srch\fR(), as a pointer to domain name) +for the given context \fIctx\fR. If the last argument is a NULL +pointer, the corresponding list (search or nameserver) is reset +instead. Upon successeful completion, each routine returns new +number of elements in the list in question. On error, negative +value is returned and global variable \fBerrno\fR is set appropriately. +It is an error to call any of this functions if the context is +opened (after \fBdns_open\fR() or \fBdns_init\fR() with non-zero argument). +.RE +.PP +.nf +int \fBdns_set_opts\fR(\fIctx\fR, const char *\fIopts\fR) +.fi +.RS +set resolver context options from \fIopts\fR string, in the same way as +processing \fBoptions\fR statement in resolv.conf and \fB$RES_OPTIONS\fR +environment variable. Return number of unrecognized/invalid options +found (all recognized and valid options gets processed). +.RE +.PP +.nf +void \fBdns_set_opt\fR(\fIctx\fR, int \fIopt\fR, \fIval\fR) +.fi +.RS +.B TODO +The \fIflags\fR argument is a bitmask with the following bits defined: +.IP \fBDNS_NOSRCH\fR +do not perform domain name search in search list. +.IP \fBDNS_NORD\fR +do not request recursion when performing queries +(i.e. don't set RD flag in querues). +.IP \fBDNS_AAONLY\fR +request authoritative answers only (i.e. set AA +flag in queries). +.RE + +.PP +.nf +int \fBdns_open\fR(\fIctx\fR) +int \fBdns_sock\fR(const \fIctx\fR) +void \fBdns_close\fR(\fIctx\fR) +.fi +.RS +\fBdns_open\fR() opens the UDP socket used for queries if not already +open, and return assotiated filedescriptor (or negative value in case +of error). Before any query can be submitted, the context should be +opened using this routine. And before opening, the context should be +initialized. +\fBdns_sock\fR() return the UDP socket if open, or -1 if not. +\fBdns_close\fR() closes the UDP socket if it was open, and drops all active +queries if any. +.RE + +.PP +.nf +int \fBdns_active\fR(const \fIctx\fR) +.fi +.RS +return number of active queries queued for the given context +\fIctx\fR, or zero if none. +.RE + +.PP +.nf +int \fBdns_status\fR(const \fIctx\fR) +.fi +.RS +return status code from last operation. When using syncronous +interface, this is the query completion status of the last query. +With asyncronous interface, from within the callback routine, +this is the query completion status of the query for which the +callback is being called. When query submission fails, this +is the error code indicating failure reason. All error codes +are negative and are represented by \fBDNS_E_\fIXXX\fR constants +described below. +.RE + +.PP +.nf +void \fBdns_ioevent\fR(\fIctx\fR, time_t \fInow\fR) +.fi +.RS +this routine may be called by an application to process I/O +events on the UDP socket used by the library, as returned +by \fBdns_sock\fR(). The routine tries to receive incoming +UDP datagram from the socket and process it. The socket is +set up to be non-blocking, so it is safe to call the routine +even if there's no data to read. The routine will process +as many datagrams as are queued for the socket, so it is +safe to use it with either level-triggered or edge-triggered +I/O monitoring model. The \fInow\fR argument is either a +current time as returned by \fBtime\fR(), or 0, in which +case the routine will obtain current time by it's own. +.RE + +.PP +.nf +int \fBdns_timeouts\fR(\fIctx\fR, int \fImaxwait\fR, time_t \fInow\fR) +.fi +.RS +process any pending timeouts and return number of secounds +from current time (\fInow\fR if it is not 0) to the time when +the library wants the application to pass it control to process +more queued requests. In case when there are no requests pending, +this time is -1. The routine will not request a time larger than +\fImaxwait\fR secounds if it is greather or equal to zero. If +\fInow\fR is 0, the routine will obtain current time by it's own; +when it is not 0, it should contain current time as returned by +\fBtime\fR(). +.RE + +.PP +.nf +typedef void \fBdns_utm_fn\fR(\fIctx\fR, int \fItimeout\fR, void *\fIdata\fR) +void \fBdns_set_tmcbck\fR(\fIctx\fR, dns_utm_fn *\fIutmfn\fR, void *\fIdata\fR) +.fi +.RS +An application may use custom callback-based I/O multiplexing mechanism. +Usually such a mechanism have concept of a \fItimer\fR, and an ability +to register a timer event in a form of a callback routine which will +be executed after certain amount of time. In order to use such an +event mechanism, udns provides an ability to register and de-register +timer events necessary for internal processing using whatever event +mechanism an application uses. For this to work, it is possible to +assotiate a pointer to a routine that will perform necessary work for +(de)registering timer events with a given resolver context, and +udns will call that routine at appropriate times. Prototype of +such a routine is shown by \fBdns_utm_fn\fR typedef above. Libudns +assotiates single timer with resolver context. User-supplied \fIutmfn\fR +routine will be called by the library with the following arguments: +.IP "\fIctx\fR == NULL" +delete user timer, at context free time or when an application changes +user timer request routine using \fBdns_set_tmcbck\fR(); +.IP "\fIctx\fR != NULL, \fItimeout\fR < 0" +don't fire timer anymore, when there are no active requests; +.IP "\fIctx\fR != NULL, \fItimeout\fR == 0" +fire timer at the next possibility, but not immediately; +.IP "\fIctx\fR != NULL, \fItimeout\fR > 0" +fire timer after \fItimeout\fR seconds after now. +.PP +The \fIdata\fR argument passed to the routine will be the same +as passed to \fBdns_set_tmcbck\fR(). +.PP +When a timer expires, an application should call \fBdns_timeouts\fR() +routine (see below). Non-callback timer usage is provided too. +.RE + +.PP +.B XXXX TODO: some more resolver context routines, like dns_set_dbgfn() etc. + +.SH "QUERY INTERFACE" + +.PP +There are two ways to perform DNS queries: traditional syncronous +way, when udns performs all the necessary processing and return +control to the application only when the query completes, and +asyncronous way, when an application submits one or more queries +to the library using given resolver context, and waits for completion +by monitoring filedescriptor used by library and calling library +routines to process input on that filedescriptor. Asyncronous mode +works with callback routines: an application supplies an address of +a routine to execute when the query completes, and a data pointer, +which is passed to the callback routine. + +.PP +Queries are submitted to the library in a form of \fBstruct\ dns_query\fR. +To perform asyncronous query, an application calls one of the +\fBdns_submit_\fIXXX\fR() rounines, and provides necessary information +for a callback, together with all the query parameters. +When the query completes, library will call application-supplied callback +routine, giving it the resolver context (wich holds query completion status), +dynamically allocated result (which will be either raw DNS packet or, if +applicatin requested parsing the result by specifying non-NULL parse routine, +ready-to-use type-specific structure), and a data pointer provided by an +application when it submitted the query. It is the application who's +responsible for freeing the result memory. +.PP +Generic query callback routine looks like this: +.nf +typedef void +\fBdns_query_fn\fR(\fIctx\fR, void *\fIresult\fR, void *\fIdata\fR) +.fi +Type-specific query interface expects similar form of callback +routine with the only difference in type of \fBresult\fR argument, +which will be pointer to specific data structure (decoded reply) +instead of this void pointer to raw DNS packet data. + +.PP +Result parsing routine looks like this: +.nf +typedef int +\fBdns_parse_fn\fR(const unsigned char *\fIqdn\fR, + const unsigned char *\fIpkt\fR, + const unsigned char *\fIcur\fR, + const unsigned char *\fIend\fR, + void **\fIresultp\fR); +.fi +When called by the library, the arguments are as follows: +\fIpkt\fR points to the start of the packet received; +\fIend\fR points past the end of the packet received; +\fIcur\fR points past the query DN in the query section of the +packet; +\fIqdn\fR points to the original query DN. +The routine should allocate a single buffer to hold the result, +parse the reply filling in the buffer, and return the buffer +using \fIresultp\fR argument. It returns 0 in case of error, +or udns error code (\fBDNS_E_\fIXXX\fR constants) in case of +error. +Note that by the time when the parse routine is called by the +library, packet is already verified to be a reply to the +original query, by matching query DN, query class and query type. + +.PP +Type-specific query inteface supplies necessary parsing routines +automatically. + +.PP +In case of error, query completion status as returned by +\fBdns_status\fR(\fIctx\fR), will contain one of the following values: +.IP "positive value" +length of raw DNS packet if parsing is not requested. +.IP 0 +the query was successeful and the \fIreply\fR points to type-specific +data structure. +.IP \fBDNS_E_TEMPFAIL\fR +temporary error, the resolver nameserver was not able to +process our query or timed out. +.IP \fBDNS_E_PROTOCOL\fR +protocol error, a nameserver returned malformed reply. +.IP \fBDNS_E_NXDOMAIN\fR +the domain name does not exist. +.IP \fBDNS_E_NODATA\fR +there is no data of requested type found. +.IP \fBDNS_E_NOMEM\fR +out of memory while processing request. +.IP \fBDNS_E_BADQUERY\fR +some aspect of the query (most common is the domain name in question) +is invalid, and the library can't even start a query. + +.PP +Library provides two series of routines which uses similar interface -- +one for asyncronous queries and another for syncronous queries. There +are two general low-level routines in each series to submit (asyncronous +interface) and resolve (syncronous interface) queries, as well as several +type-specific routines with more easy-to-use interfaces. To submit +an asyncronous query, use one of \fBdns_submit_\fIXXX\fR() routine, each +of which accepts query parameters, pointers to callback routine and to +callback data, and optional current time hint. Note type-specific +\fBdns_submit_\fIXXX\fR() routines expects specific type of the callback +routine as well, which accepts reply as a pointer to corresponding +structure, not a void pointer). Every \fBdns_submit_\fIXXX\fR() routine +return pointer to internal query structure of type struct\ dns_query, +used as an identifier for the given query. + +.PP +To resolve a query syncronously, use one of \fBdns_resolve_\fIXXX\fR() +routines, which accepts the same query parameters (but not the +callback pointers) as corresponding \fBdns_submit_\fIXXX\fR(), and +return the query result, which is the same as passed to the callback +routine in case of asyncronous interface. + +.PP +In either case, the result memory (if the query completed successefully) +is dynamically allocated and should be freed by an application. If +the query failed for any reason, the result will be NULL, and error +status will be available from \fBdns_status\fR(\fIctx\fR) routine +as shown above. + +.PP +.nf +struct dns_query * +\fBdns_submit_dn\fR(\fIctx\fR, + const unsigned char *\fIdn\fR, \fIqcls\fR, \fIqtyp\fR, \fIflags\fR, + \fIparse\fR, \fIcbck\fR, \fIdata\fR) +struct dns_query * +\fBdns_submit_p\fR(\fIctx\fR, + const char *\fIname\fR, \fIqcls\fR, \fIqtyp\fR, \fIflags\fR, + \fIparse\fR, \fIcbck\fR, \fIdata\fR) + enum dns_class \fIqcls\fR; + enum dns_type \fIqtyp\fR; + int \fIflags\fR; + dns_parse_fn *\fIparse\fR; + dns_query_fn *\fIcbck\fR; + void *\fIdata\fR; +.fi +.RS +submit a query for processing for the given resolver context \fIctx\fR. +Two routines differs only in 3rd argument, which is domain name in +DN format (\fIdn\fR) or asciiz string (\fIname\fR). The query will be +performed for the given domain name, with type \fIqtyp\fR in class \fIqcls\fR, +using option bits in \fIflags\fR, using RR parsing routine pointed by +\fIparse\fR if not-NULL, and upon completion, \fIcbck\fR function will +be called with the \fIdata\fR argument. +In case of successeful query submission, +the routine return pointer to internal query structure which may be treated +as an identifier of the query as used by the library, and may be used as an +argument for \fBdns_cancel\fR() routine. In case of error, NULL will be +returned, and context error status (available using \fIdns_status\fR() routine) +will be set to corresponding error code, which in this case may be +DNS_E_BADQUERY if the \fIname\fR of \fIdn\fR is invalid, DNS_E_NOMEM if +there's no memory available to allocate query structure, or DNS_E_TEMPFAIL +if an internal error occured. +.RE + +.PP +.nf +void *\fBdns_resolve_dn\fR(\fIctx\fR, + const unsigned char *\fIdn\fR, \fIqcls\fR, \fIqtyp\fR, \fIflags\fR, \fIparse\fR); +void *\fBdns_resolve_p\fR(\fIctx\fR, + const char *\fIname\fR, \fIqcls\fR, \fIqtyp\fR, \fIflags\fR, \fIparse\fR) + enum dns_class \fIqcls\fR; + enum dns_type \fIqtyp\fR; + int \fIflags\fR; + dns_parse_fn *\fIparse\fR; +.fi +.RS +syncronous interface. The routines perform all the steps necessary to resolve +the given query and return the result. If there's no positive result for any +reason, all the routines return NULL, and set context error status (available +using \fBdns_status\fR() routine) to indicate the error code. If the query +was successeful, context status code will contain either the length of the +raw DNS reply packet if \fIparse\fR argument was NULL (in which case the return +value is pointer to the reply DNS packet), or 0 (in which case the return value +is the result of \fIparse\fR routine). If the query successeful (return value +is not NULL), the memory returned was dynamically allocated by the library +and should be free()d by application after use. +.RE + +.PP +.nf +void *\fBdns_resolve\fR(\fIctx\fR, struct dns_query *\fIq\fR) +.fi +.RS +wait for the given query \fIq\fR, as returned by one of +\fBdns_submit_\fIXXX\fR() routines, for completion, and +return the result. The callback routine will not be called +for this query. After completion, the query identifier \fIq\fR +is not valid. Both \fBdns_resolve_dn\fR() and \fBdns_resolve_p\fR() +are just wrappers around corresponding submit routines and this +\fBdns_resolve\fR() routine. +.RE + +.PP +.nf +void \fBdns_cancel\fR(\fIctx\fR, struct dns_query *\fIq\fR) +.fi +.RS +cancel an active query \fIq\fR, without calling a callback routine. +After completion, the query identifier \fIq\fR is not valid. +.RE + +.SH "TYPE-SPECIFIC QUERIES" + +.PP +In addition to the generic low-level query interface, the library provides +a set of routines to perform specific queries in a type-safe manner, as +well as parsers for several well-known resource record types. The library +implements high-level interface for A, AAAA, PTR, MX and TXT records +and DNSBL and RHSBL functionality. These routines returns specific types +as result of a query, instead of raw DNS packets. The following types +and routines are available. + +.PP +.nf +struct \fBdns_rr_null\fR { + char *\fBdnsn_qname\fR; /* original query name */ + char *\fBdnsn_cname\fR; /* canonical name */ + unsigned \fBdnsn_ttl\fR; /* Time-To-Live (TTL) value */ + int \fBdnsn_nrr\fR; /* number of records in the set */ +}; +.fi +.PP +NULL RR set, used as a base for all other RR type structures. +Every RR structure as used by the library have four standard +fields as in struct\ \fBdns_rr_null\fR. + +.SS "IN A Queries" +.PP +.nf +struct \fBdns_rr_a4\fR { /* IN A RRset */ + char *\fBdnsa4_qname\fR; /* original query name */ + char *\fBdnsa4_cname\fR; /* canonical name */ + unsigned \fBdnsa4_ttl\fR; /* Time-To-Live (TTL) value */ + int \fBdnsa4_nrr\fR; /* number of addresses in the set */ + struct in_addr \fBdnsa4_addr\fR[]; /* array of addresses */ +}; +typedef void + \fBdns_query_a4_fn\fR(\fIctx\fR, struct dns_rr_a4 *\fIresult\fR, \fIdata\fR) +dns_parse_fn \fBdns_parse_a4\fB; +struct dns_query * +\fBdns_submit_a4\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR, + dns_query_a4_fn *\fIcbck\fR, \fIdata\fR); +struct dns_rr_a4 * +\fBdns_resolve_a4\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR); +.fi +.PP +The \fBdns_rr_a4\fR structure holds a result of an \fBIN A\fR query, +which is an array of IPv4 addresses. Callback routine for IN A queries +expected to be of type \fBdns_query_a4_fn\fR, which expects pointer to +\fBdns_rr_a4\fR structure as query result instead of raw DNS packet. +The \fBdns_parse_a4\fR() is used to convert raw DNS reply packet into +\fBdns_rr_a4\fR structure (it is used internally and may be used directly too +with generic query interface). Routines \fBdns_submit_a4\fR() and +\fBdns_resolve_a4\fR() are used to perform A IN queries in a type-safe +manner. The \fIname\fR parameter is the domain name in question, and +\fIflags\fR is query flags bitmask, with one bit, DNS_NOSRCH, of practical +interest (if the \fIname\fR is absolute, that is, it ends up with a dot, +DNS_NOSRCH flag will be set automatically). + +.SS "IN AAAA Queries" +.PP +.nf +struct \fBdns_rr_a6\fR { /* IN AAAA RRset */ + char *\fBdnsa6_qname\fR; /* original query name */ + char *\fBdnsa6_cname\fR; /* canonical name */ + unsigned \fBdnsa6_ttl\fR; /* Time-To-Live (TTL) value */ + int \fBdnsa6_nrr\fR; /* number of addresses in the set */ + struct in6_addr \fBdnsa6_addr\fR[]; /* array of addresses */ +}; +typedef void + \fBdns_query_a6_fn\fR(\fIctx\fR, struct dns_rr_a6 *\fIresult\fR, \fIdata\fR) +dns_parse_fn \fBdns_parse_a6\fB; +struct dns_query * +\fBdns_submit_a6\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR, + dns_query_a6_fn *\fIcbck\fR, \fIdata\fR); +struct dns_rr_a6 * +\fBdns_resolve_a6\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR); +.fi +.PP +The \fBdns_rr_a6\fR structure holds a result of an \fBIN AAAA\fR query, +which is an array of IPv6 addresses. Callback routine for IN AAAA queries +expected to be of type \fBdns_query_a6_fn\fR, which expects pointer to +\fBdns_rr_a6\fR structure as query result instead of raw DNS packet. +The \fBdns_parse_a6\fR() is used to convert raw DNS reply packet into +\fBdns_rr_a6\fR structure (it is used internally and may be used directly too +with generic query interface). Routines \fBdns_submit_a6\fR() and +\fBdns_resolve_a6\fR() are used to perform AAAA IN queries in a type-safe +manner. The \fIname\fR parameter is the domain name in question, and +\fIflags\fR is query flags bitmask, with one bit, DNS_NOSRCH, of practical +interest (if the \fIname\fR is absolute, that is, it ends up with a dot, +DNS_NOSRCH flag will be set automatically). + +.SS "IN PTR Queries" +.PP +.nf +struct \fBdns_rr_ptr\fR { /* IN PTR RRset */ + char *\fBdnsptr_qname\fR; /* original query name */ + char *\fBdnsptr_cname\fR; /* canonical name */ + unsigned \fBdnsptr_ttl\fR; /* Time-To-Live (TTL) value */ + int \fBdnsptr_nrr\fR; /* number of domain name pointers */ + char *\fBdnsptr_ptr\fR[]; /* array of domain name pointers */ +}; +typedef void + \fBdns_query_ptr_fn\fR(\fIctx\fR, struct dns_rr_ptr *\fIresult\fR, \fIdata\fR) +dns_parse_fn \fBdns_parse_ptr\fB; +struct dns_query * +\fBdns_submit_a4ptr\fB(\fIctx\fR, const struct in_addr *\fBaddr\fR, + dns_query_ptr_fn *\fIcbck\fR, \fIdata\fR); +struct dns_rr_ptr * +\fBdns_resolve_a4ptr\fB(\fIctx\fR, const struct in_addr *\fBaddr\fR); +struct dns_query * +\fBdns_submit_a6ptr\fB(\fIctx\fR, const struct in6_addr *\fBaddr\fR, + dns_query_ptr_fn *\fIcbck\fR, \fIdata\fR); +struct dns_rr_ptr * +\fBdns_resolve_a6ptr\fB(\fIctx\fR, const struct in6_addr *\fBaddr\fR); +.fi +.PP +The \fBdns_rr_ptr\fR structure holds a result of an IN PTR query, which +is an array of domain name pointers for a given IPv4 or IPv6 address. +Callback routine for IN PTR queries expected to be of type +\fBdns_query_ptr_fn\fR, which expects pointer to \fBdns_rr_ptr\fR +structure as query result instead of raw DNS packet. The \fBdns_parse_ptr\fR() +is used to convert raw DNS reply packet into \fBdns_rr_ptr\fR structure +(it is used internally and may be used directly too with generic query +interface). Routines \fBdns_submit_a4ptr\fR() and \fBdns_resolve_a4ptr\fR() +are used to perform IN PTR queries for IPv4 addresses in a type-safe +manner. Routines \fBdns_submit_a6ptr\fR() and \fBdns_resolve_a6ptr\fR() +are used to perform IN PTR queries for IPv6 addresses. + +.SS "IN MX Queries" +.PP +.nf +struct \fBdns_mx\fR { /* single MX record */ + int \fBpriority\fR; /* priority value of this MX */ + char *\fBname\fR; /* domain name of this MX */ +}; +struct \fBdns_rr_mx\fR { /* IN MX RRset */ + char *\fBdnsmx_qname\fR; /* original query name */ + char *\fBdnsmx_cname\fR; /* canonical name */ + unsigned \fBdnsmx_ttl\fR; /* Time-To-Live (TTL) value */ + int \fBdnsmx_nrr\fR; /* number of mail exchangers in the set */ + struct dns_mx \fBdnsmx_mx\fR[]; /* array of mail exchangers */ +}; +typedef void + \fBdns_query_mx_fn\fR(\fIctx\fR, struct dns_rr_mx *\fIresult\fR, \fIdata\fR) +dns_parse_fn \fBdns_parse_mx\fB; +struct dns_query * +\fBdns_submit_mx\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR, + dns_query_mx_fn *\fIcbck\fR, \fIdata\fR); +struct dns_rr_mx * +\fBdns_resolve_mx\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR); +.fi +.PP +The \fBdns_rr_mx\fR structure holds a result of an IN MX query, which +is an array of mail exchangers for a given domain. Callback routine for IN MX +queries expected to be of type \fBdns_query_mx_fn\fR, which expects pointer to +\fBdns_rr_mx\fR structure as query result instead of raw DNS packet. +The \fBdns_parse_mx\fR() is used to convert raw DNS reply packet into +\fBdns_rr_mx\fR structure (it is used internally and may be used directly too +with generic query interface). Routines \fBdns_submit_mx\fR() and +\fBdns_resolve_mx\fR() are used to perform IN MX queries in a type-safe +manner. The \fIname\fR parameter is the domain name in question, and +\fIflags\fR is query flags bitmask, with one bit, DNS_NOSRCH, of practical +interest (if the \fIname\fR is absolute, that is, it ends up with a dot, +DNS_NOSRCH flag will be set automatically). + +.SS "TXT Queries" +.PP +.nf +struct \fBdns_txt\fR { /* single TXT record */ + int \fBlen\fR; /* length of the text */ + unsigned char *\fBtxt\fR; /* pointer to the text */ +}; +struct \fBdns_rr_txt\fR { /* TXT RRset */ + char *\fBdnstxt_qname\fR; /* original query name */ + char *\fBdnstxt_cname\fR; /* canonical name */ + unsigned \fBdnstxt_ttl\fR; /* Time-To-Live (TTL) value */ + int \fBdnstxt_nrr\fR; /* number of text records in the set */ + struct dns_txt \fBdnstxt_txt\fR[]; /* array of TXT records */ +}; +typedef void + \fBdns_query_txt_fn\fR(\fIctx\fR, struct dns_rr_txt *\fIresult\fR, \fIdata\fR) +dns_parse_fn \fBdns_parse_txt\fB; +struct dns_query * +\fBdns_submit_txt\fB(\fIctx\fR, const char *\fIname\fR, enum dns_class \fIqcls\fR, + int \fIflags\fR, dns_query_txt_fn *\fIcbck\fR, \fIdata\fR); +struct dns_rr_txt * +\fBdns_resolve_txt\fB(\fIctx\fR, const char *\fIname\fR, + enum dns_class \fIqcls\fR, int \fIflags\fR); +.fi +.PP +The \fBdns_rr_txt\fR structure holds a result of a TXT query, which is an +array of text records for a given domain name. Callback routine for TXT +queries expected to be of type \fBdns_query_txt_fn\fR, which expects pointer +to \fBdns_rr_txt\fR structure as query result instead of raw DNS packet. +The \fBdns_parse_txt\fR() is used to convert raw DNS reply packet into +\fBdns_rr_txt\fR structure (it is used internally and may be used directly too +with generic query interface). Routines \fBdns_submit_txt\fR() and +\fBdns_resolve_txt\fR() are used to perform IN MX queries in a type-safe +manner. The \fIname\fR parameter is the domain name in question, and +\fIflags\fR is query flags bitmask, with one bit, DNS_NOSRCH, of practical +interest (if the \fIname\fR is absolute, that is, it ends up with a dot, +DNS_NOSRCH flag will be set automatically). Note that each TXT string +is represented by \fBstruct\ dns_txt\fR, while zero-terminated (and the +len field of the structure does not include the terminator), may contain +embedded null characters -- content of TXT records is not interpreted +by the library in any way. + +.SS "SRV Queries" +.PP +.nf +struct \fBdns_srv\fR { /* single SRV record */ + int \fBpriority\fR; /* priority of the record */ + int \fBweight\fR; /* weight of the record */ + int \fBport\fR; /* the port number to connect to */ + char *\fBname\fR; /* target host name */ +}; +struct \fBdns_rr_srv\fR { /* SRV RRset */ + char *\fBdnssrv_qname\fR; /* original query name */ + char *\fBdnssrv_cname\fR; /* canonical name */ + unsigned \fBdnssrv_ttl\fR; /* Time-To-Live (TTL) value */ + int \fBdnssrv_nrr\fR; /* number of text records in the set */ + struct dns_srv \fBdnssrv_srv\fR[]; /* array of SRV records */ +}; +typedef void + \fBdns_query_srv_fn\fR(\fIctx\fR, struct dns_rr_srv *\fIresult\fR, \fIdata\fR) +dns_parse_fn \fBdns_parse_srv\fB; +struct dns_query * +\fBdns_submit_srv\fB(\fIctx\fR, const char *\fIname\fR, const char *\fIservice\fR, const char *\fIprotocol\fR, + int \fIflags\fR, dns_query_txt_fn *\fIcbck\fR, \fIdata\fR); +struct dns_rr_srv * +\fBdns_resolve_srv\fB(\fIctx\fR, const char *\fIname\fR, const char *\fIservice\fR, const char *\fIprotocol\fR, + int \fIflags\fR); +.fi +.PP +The \fBdns_rr_srv\fR structure holds a result of an IN SRV (rfc2782) query, +which is an array of servers (together with port numbers) which are performing +operations for a given \fIservice\fR using given \fIprotocol\fR on a target +domain \fIname\fR. Callback routine for IN SRV queries expected to be of type +\fBdns_query_srv_fn\fR, which expects pointer to \fBdns_rr_srv\fR structure as +query result instead of raw DNS packet. The \fBdns_parse_srv\fR() is used to +convert raw DNS reply packet into \fBdns_rr_srv\fR structure (it is used +internally and may be used directly too with generic query interface). +Routines \fBdns_submit_srv\fR() and \fBdns_resolve_srv\fR() are used to +perform IN SRV queries in a type-safe manner. The \fIname\fR parameter +is the domain name in question, \fIservice\fR and \fRprotocl\fR specifies the +service and the protocol in question (the library will construct query DN +according to rfc2782 rules) and may be NULL (in this case the library +assumes \fIname\fR parameter holds the complete SRV query), and +\fIflags\fR is query flags bitmask, with one bit, DNS_NOSRCH, of practical +interest (if the \fIname\fR is absolute, that is, it ends up with a dot, +DNS_NOSRCH flag will be set automatically). + +.SS "NAPTR Queries" +.PP +.nf +struct \fBdns_naptr\fR { /* single NAPTR record */ + int \fBorder\fR; /* record order */ + int \fBpreference\fR; /* preference of this record */ + char *\fBflags\fR; /* application-specific flags */ + char *\fBservice\fR; /* service parameter */ + char *\fBregexp\fR; /* substitutional regular expression */ + char *\fBreplacement\fR; /* replacement string */ +}; +struct \fBdns_rr_naptr\fR { /* NAPTR RRset */ + char *\fBdnsnaptr_qname\fR; /* original query name */ + char *\fBdnsnaptr_cname\fR; /* canonical name */ + unsigned \fBdnsnaptr_ttl\fR; /* Time-To-Live (TTL) value */ + int \fBdnsnaptr_nrr\fR; /* number of text records in the set */ + struct dns_naptr \fBdnsnaptr_naptr\fR[]; /* array of NAPTR records */ +}; +typedef void + \fBdns_query_naptr_fn\fR(\fIctx\fR, struct dns_rr_naptr *\fIresult\fR, \fIdata\fR) +dns_parse_fn \fBdns_parse_naptr\fB; +struct dns_query * +\fBdns_submit_naptr\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR, + dns_query_txt_fn *\fIcbck\fR, \fIdata\fR); +struct dns_rr_naptr * +\fBdns_resolve_naptr\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR); +.fi +.PP +The \fBdns_rr_naptr\fR structure holds a result of an IN NAPTR (rfc3403) query. +Callback routine for IN NAPTR queries expected to be of type +\fBdns_query_naptr_fn\fR, expects pointer to \fBdns_rr_naptr\fR +structure as query result instead of raw DNS packet. +The \fBdns_parse_naptr\fR() is used to convert raw DNS reply packet into +\fBdns_rr_naptr\fR structure (it is used +internally and may be used directly too with generic query interface). +Routines \fBdns_submit_naptr\fR() and \fBdns_resolve_naptr\fR() are used to +perform IN NAPTR queries in a type-safe manner. The \fIname\fR parameter +is the domain name in question, and \fIflags\fR is query flags bitmask, +with one bit, DNS_NOSRCH, of practical interest (if the \fIname\fR is +absolute, that is, it ends up with a dot, DNS_NOSRCH flag will be set +automatically). + +.SS "DNSBL Interface" +.PP +A DNS-based blocklists, or a DNSBLs, are in wide use nowadays, especially +to protect mailservers from spammers. The library provides DNSBL interface, +a set of routines to perform queries against DNSBLs. Routines accepts an +IP address (IPv4 and IPv6 are both supported) and a base DNSBL zone as +query parameters, and returns either \fBdns_rr_a4\fR or \fBdns_rr_txt\fR +structure. Note that IPv6 interface return IPv4 RRset. +.PP +.nf +struct dns_query * +\fBdns_submit_a4dnsbl\fR(\fIctx\fR, + const struct in_addr *\fIaddr\fR, const char *\fIdnsbl\fR, + dns_query_a4_fn *\fIcbck\fR, void *\fIdata\fR); +struct dns_query * +\fBdns_submit_a4dnsbl_txt\fR(\fIctx\fR, + const struct in_addr *\fIaddr\fR, const char *\fIdnsbl\fR, + dns_query_txt_fn *\fIcbck\fR, void *\fIdata\fR); +struct dns_query * +\fBdns_submit_a6dnsbl\fR(\fIctx\fR, + const struct in6_addr *\fIaddr\fR, const char *\fIdnsbl\fR, + dns_query_a4_fn *\fIcbck\fR, void *\fIdata\fR); +struct dns_query * +\fBdns_submit_a6dnsbl_txt\fR(\fIctx\fR, + const struct in6_addr *\fIaddr\fR, const char *\fIdnsbl\fR, + dns_query_txt_fn *\fIcbck\fR, void *\fIdata\fR); +struct dns_rr_a4 *\fBdns_resolve_a4dnsbl\fR(\fIctx\fR, + const struct in_addr *\fIaddr\fR, const char *\fIdnsbl\fR) +struct dns_rr_txt *\fBdns_resolve_a4dnsbl_txt\fR(\fIctx\fR, + const struct in_addr *\fIaddr\fR, const char *\fIdnsbl\fR) +struct dns_rr_a4 *\fBdns_resolve_a6dnsbl\fR(\fIctx\fR, + const struct in6_addr *\fIaddr\fR, const char *\fIdnsbl\fR) +struct dns_rr_txt *\fBdns_resolve_a6dnsbl_txt\fR(\fIctx\fR, + const struct in6_addr *\fIaddr\fR, const char *\fIdnsbl\fR) +.fi +Perform (submit or resolve) a DNSBL query for the given \fIdnsbl\fR +domain and an IP \fIaddr\fR in question, requesting either A or TXT +records. + +.SS "RHSBL Interface" +.PP +RHSBL is similar to DNSBL, but instead of an IP address, the +parameter is a domain name. +.PP +.nf +struct dns_query * +\fBdns_submit_rhsbl\fR(\fIctx\fR, const char *\fIname\fR, const char *\fIrhsbl\fR, + dns_query_a4_fn *\fIcbck\fR, void *\fIdata\fR); +struct dns_query * +\fBdns_submit_rhsbl_txt\fR(\fIctx\fR, const char *\fIname\fR, const char *\fIrhsbl\fR, + dns_query_txt_fn *\fIcbck\fR, void *\fIdata\fR); +struct dns_rr_a4 * +\fBdns_resolve_rhsbl\fR(\fIctx\fR, const char *\fIname\fR, const char *\fIrhsbl\fR); +struct dns_rr_txt * +\fBdns_resolve_rhsbl_txt\fR(\fIctx\fR, const char *\fIname\fR, const char *\fIrhsbl\fR); +.fi +Perform (submit or resolve) a RHSBL query for the given \fIrhsbl\fR +domain and \fIname\fR in question, requesting either A or TXT records. + + +.SH "LOW-LEVEL INTERFACE" + +.SS "Domain Names (DNs)" + +.PP +A DN is a series of domain name labels each starts with length byte, +followed by empty label (label with zero length). The following +routines to work with DNs are provided. + +.PP +.nf +unsigned \fBdns_dnlen\fR(const unsigned char *\fIdn\fR) +.fi +.RS +return length of the domain name \fIdn\fR, including the terminating label. +.RE + +.PP +.nf +unsigned \fBdns_dnlabels\fR(const unsigned char *\fIdn\fR) +.fi +.RS +return number of non-zero labels in domain name \fIdn\fR. +.RE + +.PP +.nf +unsigned \fBdns_dnequal\fR(\fIdn1\fR, \fIdn2\fR) + const unsigned char *\fIdn1\fR, *\fIdn2\fR; +.fi +.RS +test whenever the two domain names, \fIdn1\fR and \fIdn2\fR, are +equal (case-insensitive). Return domain name length if equal +or 0 if not. +.RE + +.PP +.nf +unsigned \fBdns_dntodn\fR(\fIsdn\fR, \fIddn\fR, \fIdnsiz\fR) + const unsigned char *\fIsdn\fR; + unsigned char *\fIddn\fR; + unsigned \fIdnsiz\fR; +.fi +.RS +copies the source domain name \fIsdn\fR to destination buffer \fIddn\fR +of size \fIdnsiz\fR. Return domain name length or 0 if \fIddn\fR is +too small. +.RE + +.PP +.nf +int \fBdns_ptodn\fR(\fIname\fR, \fInamelen\fR, \fIdn\fR, \fIdnsiz\fR, \fIisabs\fR) +int \fBdns_sptodn\fR(\fIname\fR, \fIdn\fR, \fIdnsiz\fR) + const char *\fIname\fR; unsigned \fInamelen\fR; + unsigned char *\fIdn\fR; unsigned \fIdnsiz\fR; + int *\fIisabs\fR; +.fi +.RS +convert asciiz name \fIname\fR of length \fInamelen\fR to DN format, +placing result into buffer \fIdn\fR of size \fIdnsiz\fR. Return +length of the DN if successeful, 0 if the \fIdn\fR buffer supplied is +too small, or negative value if \fIname\fR is invalid. If \fIisabs\fR +is non-NULL and conversion was successeful, *\fIisabs\fR will be set to +either 1 or 0 depending whenever \fIname\fR was absolute (i.e. ending with +a dot) or not. Name length, \fInamelength\fR, may be zero, in which case +strlen(\fIname\fR) will be used. Second form, \fBdns_sptodn\fR(), is a +simplified form of \fBdns_ptodn\fR(), equivalent to +.br +.nf +\fBdns_ptodn\fR(\fIname\fR, 0, \fIdn\fR, \fIdnlen\fR, 0). +.fi +.RE + +.PP +.nf +extern const unsigned char \fBdns_inaddr_arpa_dn\fR[] +int \fBdns_a4todn\fR(const struct in_addr *\fIaddr\fR, const unsigned char *\fItdn\fR, + unsigned char *\fIdn\fR, unsigned \fIdnsiz\fR) +int \fBdns_a4ptodn\fR(const struct in_addr *\fIaddr\fR, const char *\fItname\fR, + unsigned char *\fIdn\fR, unsigned \fIdnsiz\fR) +extern const unsigned char \fBdns_ip6_arpa_dn\fR[] +int \fBdns_a6todn\fR(const struct in6_addr *\fIaddr\fR, const unsigned char *\fItdn\fR, + unsigned char *\fIdn\fR, unsigned \fIdnsiz\fR) +int \fBdns_a6ptodn\fR(const struct in6_addr *\fIaddr\fR, const char *\fItname\fR, + unsigned char *\fIdn\fR, unsigned \fIdnsiz\fR) +.fi +.RS +several variants of routines to convert IPv4 and IPv6 address \fIaddr\fR +into reverseDNS-like domain name in DN format, storing result in \fIdn\fR +of size \fIdnsiz\fR. \fItdn\fR (or \fItname\fR) is the base zone name, +like in-addr.arpa for IPv4 or in6.arpa for IPv6. If \fItdn\fR (or \fItname\fR) +is NULL, \fBdns_inaddr_arpa_dn\fR (or \fBdns_ip6_arpa_dn\fR) will be used. +The routines may be used to construct a DN for a DNSBL lookup for example. +All routines return length of the resulting DN on success, -1 if resulting +DN is invalid, or 0 if the \fIdn\fR buffer (\fIdnsiz\fR) is too small. +To hold standard rDNS DN, a buffer of size \fBDNS_A4RSIZE\fR (30 bytes) for +IPv4 address, or \fBDNS_A6RSIZE\fR (74 bytes) for IPv6 address, is sufficient. +.RE + +.PP +.nf +int \fBdns_dntop\fR(\fIdn\fR, \fIname\fR, \fInamesiz\fR) + const unsigned char *\fIdn\fR; + const char *\fIname\fR; unsigned \fInamesiz\fR; +.fi +.RS +convert domain name \fIdn\fR in DN format to asciiz string, placing result +into \fIname\fR buffer of size \fInamesiz\fR. Maximum length of asciiz +representation of domain name is \fBDNS_MAXNAME\fR (1024) bytes. Root +domain is represented as empty string. Return length of the resulting name +(including terminating character, i.e. strlen(name)+1) on success, 0 if the +\fIname\fR buffer is too small, or negative value if \fIdn\fR is invalid +(last case should never happen since all routines in this library which +produce domain names ensure the DNs generated are valid). +.RE + +.PP +.nf +const char *\fBdns_dntosp\fR(const unsigned char *\fIdn\fR) +.fi +.RS +convert domain name \fIdn\fR in DN format to asciiz string using static +buffer. Return the resulting asciiz string on success or NULL on failure. +Note since this routine uses static buffer, it is not thread-safe. +.RE + +.PP +.nf +unsigned \fBdns_dntop_size\fR(const unsigned char *\fIdn\fR) +.fi +.RS +return the buffer size needed to convert the \fIdn\fR domain name +in DN format to asciiz string, for \fBdns_dntop\fR(). The routine +return either the size of buffer required, including the trailing +zero byte, or 0 if \fIdn\fR is invalid. +.RE + +.SS "Working with DNS Packets" + +.PP +The following routines are provided to encode and decode DNS on-wire +packets. This is low-level interface. + +.PP +DNS response codes (returned by \fBdns_rcode\fR() routine) are +defined as constants prefixed with \fBDNS_R_\fR. See udns.h +header file for the complete list. In particular, constants +\fBDNS_R_NOERROR\fR (0), \fBDNS_R_SERVFAIL\fR, \fBDNS_R_NXDOMAIN\fR +may be of interest to an application. + +.PP +.nf +unsigned \fBdns_get16\fR(const unsigned char *\fIp\fR) +unsigned \fBdns_get32\fR(const unsigned char *\fIp\fR) +.fi +.RS +helper routines, convert 16-bit or 32-bit integer in on-wire +format pointed to by \fIp\fR to unsigned. +.RE + +.PP +.nf +unsigned char *\fBdns_put16\fR(unsigned char *\fId\fR, unsigned \fIn\fR) +unsigned char *\fBdns_put32\fR(unsigned char *\fId\fR, unsigned \fIn\fR) +.fi +.RS +helper routine, convert unsigned 16-bit or 32-bit integer \fIn\fR to +on-wire format to buffer pointed to by \fId\fR, return \fId\fR+2 or +\fId\fR+4. +.RE + +.PP +.nf +\fBDNS_HSIZE\fR (12) +.fi +.RS +defines size of DNS header. Data section +in the DNS packet immediately follows the header. In the header, +there are query identifier (id), various flags and codes, +and number of resource records in various data sections. +See udns.h header file for complete list of DNS header definitions. +.RE + +.PP +.nf +unsigned \fBdns_qid\fR(const unsigned char *\fIpkt\fR) +int \fBdns_rd\fR(const unsigned char *\fIpkt\fR) +int \fBdns_tc\fR(const unsigned char *\fIpkt\fR) +int \fBdns_aa\fR(const unsigned char *\fIpkt\fR) +int \fBdns_qr\fR(const unsigned char *\fIpkt\fR) +int \fBdns_ra\fR(const unsigned char *\fIpkt\fR) +unsigned \fBdns_opcode\fR(const unsigned char *\fIpkt\fR) +unsigned \fBdns_rcode\fR(const unsigned char *\fIpkt\fR) +unsigned \fBdns_numqd\fR(const unsigned char *\fIpkt\fR) +unsigned \fBdns_numan\fR(const unsigned char *\fIpkt\fR) +unsigned \fBdns_numns\fR(const unsigned char *\fIpkt\fR) +unsigned \fBdns_numar\fR(const unsigned char *\fIpkt\fR) +const unsigned char *\fBdns_payload\fR(const unsigned char *\fIpkt\fR) +.fi +.RS +return various parts from the DNS packet header \fIpkt\fR: +query identifier (qid), +recursion desired (rd) flag, +truncation occured (tc) flag, +authoritative answer (aa) flag, +query response (qr) flag, +recursion available (ra) flag, +operation code (opcode), +result code (rcode), +number of entries in question section (numqd), +number of answers (numan), +number of authority records (numns), +number of additional records (numar), +and the pointer to the packet data (payload). +.RE + +.PP +.nf +int \fBdns_getdn\fR(\fIpkt\fR, \fIcurp\fR, \fIpkte\fR, \fIdn\fR, \fIdnsiz\fR) +const unsigned char *\fBdns_skipdn\fR(\fIcur\fR, \fIpkte\fR) + const unsigned char *\fIpkt\fR, *\fIpkte\fR, **\fIcurp\fR, *\fIcur\fR; + unsigned char *\fIdn\fR; unsigned \fIdnsiz\fR; +.fi +.RS +\fBdns_getdn\fR() extract DN from DNS packet \fIpkt\fR which ends before +\fIpkte\fR starting at position *\fIcurp\fR into buffer pointed to by +\fIdn\fR of size \fIdnsiz\fR. Upon successeful completion, *\fIcurp\fR +will point to the next byte in the packet after the extracted domain name. +It return positive number (length of the DN if \fIdn\fR) upon successeful +completion, negative value on error (when the packet contains invalid data), +or zero if the \fIdnsiz\fR is too small (maximum length of a domain name is +\fBDNS_MAXDN\fR). \fBdns_skipdn\fR() return pointer to the next byte in +DNS packet which ends up before \fIpkte\fR after a domain name which starts +at the \fIcur\fP byte, or NULL if the packet is invalid. \fBdns_skipdn\fR() +is more or less equivalent to what \fBdns_getdn\fR() does, except it does not +actually extract the domain name in question, and uses simpler interface. +.RE + +.PP +.nf +struct \fBdns_rr\fR { + unsigned char \fBdnsrr_dn\fR[DNS_MAXDN]; /* the RR DN name */ + enum dns_class \fBdnsrr_cls\fR; /* class of the RR */ + enum dns_type \fBdnsrr_typ\fR; /* type of the RR */ + unsigned \fBdnsrr_ttl\fR; /* TTL value */ + unsigned \fBdnsrr_dsz\fR; /* size of data in bytes */ + const unsigned char *\fBdnsrr_dptr\fR; /* pointer to the first data byte */ + const unsigned char *\fBdnsrr_dend\fR; /* next byte after RR */ +}; +.fi +.RS +The \fBdns_rr\fR structure is used to hold information about +single DNS Resource Record (RR) in an easy to use form. +.RE + +.PP +.nf +struct \fBdns_parse\fR { + const unsigned char *\fBdnsp_pkt\fR; /* pointer to the packet being parsed */ + const unsigned char *\fBdnsp_end\fR; /* end of the packet pointer */ + const unsigned char *\fBdnsp_cur\fR; /* current packet positionn */ + const unsigned char *\fBdnsp_ans\fR; /* pointer to the answer section */ + int \fBdnsp_rrl\fR; /* number of RRs left */ + int \fBdnsp_nrr\fR; /* number of relevant RRs seen so far */ + unsigned \fBdnsp_ttl\fR; /* TTL value so far */ + const unsigned char *\fBdnsp_qdn\fR; /* the domain of interest or NULL */ + enum dns_class \fBdnsp_qcls\fR; /* class of interest or 0 for any */ + enum dns_type \fBdnsp_qtyp\fR; /* type of interest or 0 for any */ + unsigned char \fBdnsp_dnbuf\fR[DNS_MAXDN]; /* domain name buffer */ +}; +.fi +.RS +The \fBdns_parse\fR structure is used to parse DNS reply packet. +It holds information about the packet being parsed (dnsp_pkt, dnsp_end and +dnsp_cur fields), number of RRs in the current section left to do, and +the information about specific RR which we're looking for (dnsp_qdn, +dnsp_qcls and dnsp_qtyp fields). +.RE + +.PP +.nf +int \fBdns_initparse\fR(struct dns_parse *\fIp\fR, + const unsigned char *\fIqdn\fR, + const unsigned char *\fIpkt\fR, + const unsigned char *\fIcur\fR, + const unsigned char *\fIend\fR) +.fi +.RS +initializes the RR parsing structure \fIp\fR. Arguments \fIpkt\fR, \fIcur\fR +and \fIend\fR should describe the received packet: \fIpkt\fR is the start of +the packet, \fIend\fR points to the next byte after the end of the packet, +and \fIcur\fR points past the query DN in query section (to query class+type +information). And \fIqdn\fR points to the query DN. This is the arguments +passed to \fBdns_parse_fn\fR() routine. \fBdns_initparse\fR() initializes +\fBdnsp_pkt\fR, \fBdnsp_end\fR and \fBdnsp_qdn\fR fields to the corresponding +arguments, extracts and initializes \fBdnsp_qcls\fR and \fBdnsp_qtyp\fR +fields to the values found at \fIcur\fR pointer, initializes +\fBdnsp_cur\fR and \fBdnsp_ans\fR fields to be \fIcur\fR+4 (to the start of +answer section), and initializes \fBdnsp_rrl\fR field to be number of entries +in answer section. \fBdnsp_ttl\fR will be set to max TTL value, 0xffffffff, +and \fBdnsp_nrr\fR to 0. +.RE + +.PP +.nf +int \fBdns_nextrr\fR(struct dns_parse *\fIp\fR, struct dns_rr *\fIrr\fR); +.fi +.RS +searches for next RR in the packet based on the criteria provided in +the \fIp\fR structure, filling in the \fIrr\fR structure and +advancing \fIp\fR->\fBdnsp_cur\fR to the next RR in the packet. +RR selection is based on dnsp_qdn, dnsp_qcls and dnsp_qtyp fields in +the dns_parse structure. Any (or all) of the 3 fields may be 0, +which means any actual value from the packet is acceptable. In case +the field isn't 0 (or NULL for dnsp_qdn), only RRs with corresponding +characteristics are acceptable. Additionally, when dnsp_qdn is non-NULL, +\fBdns_nextrr\fR() performs automatic CNAME expansion. +Routine will return positive value on success, 0 in case it reached the end +of current section in the packet (\fIp\fR->\fBdnsp_rrl\fR is zero), or +negative value if next RR can not be decoded (packet format is invalid). +The routine updates \fIp\fR->\fBdnsp_qdn\fR automatically when this +field is non-NULL and it encounters appropriate CNAME RRs (saving CNAME +target in \fIp\fR->\fBdnsp_dnbuf\fR), so after end of the process, +\fIp\fR->\fBdnsp_qdn\fR will point to canonical name of the domain +in question. The routine updates \fIp\fR->\fBdnsp_ttl\fR value to +be the minimum TTL of all RRs found. +.RE + +.PP +.nf +void \fBdns_rewind\fR(struct dns_parse *\fIp\fR, const unsigned char *\fIqdn\fR) +.fi +.RS +this routine "rewinds" the packet parse state structure to be at the +same state as after a call to \fBdns_initparse\fR(), i.e. reposition +the parse structure \fIp\fR to the start of answer section and +initialize \fIp\fR->\fBdnsp_rrl\fR to the number of entries in +answer section. +.RE + +.PP +.nf +int \fBdns_stdrr_size\fR(const struct dns_parse *\fIp\fR); +.fi +.RS +return size to hold standard RRset structure information, as shown +in \fBdns_rr_null\fR structure (for the query and canonical +names). Used to calculate amount of memory to allocate for common +part of type-specific RR structures in parsing routines. +.RE + +.PP +.nf +void *\fBdns_stdrr_finish\fR(struct dns_rr_null *\fIret\fR, char *\fIcp\fR, + const struct dns_parse *\fIp\fR); +.fi +.RS +initializes standard RRset fields in \fIret\fR structure using buffer +pointed to by \fIcp\fR, which should have at least as many bytes +as \fBdns_stdrr_size\fR(\fIp\fR) returned. Used to finalize common +part of type-specific RR structures in parsing routines. +.RE + +.PP +See library source for usage examples of all the above low-level routines, +especially source of the parsing routines. + +.SS "Auxilary Routines" + +.PP +.nf +int \fBdns_pton\fR(int \fIaf\fR, const char *\fIsrc\fR, void *\fIdst\fR); +.fi +.RS +privides functionality similar to standard \fBinet_pton\fR() routine, +to convert printable representation of an IP address of family \fIaf\fR +(either \fBAF_INET\fR or \fBAF_INET6\fR) pointed to by \fIsrc\fR into +binary form suitable for socket addresses and transmission over network, +in buffer pointed to by \fIdst\fR. The destination buffer should be +of size 4 for \fBAF_INET\fR family or 16 for \fBAF_INET6\fR. +The return value is positive on success, 0 if \fIsrc\fR is not a valid text +representation of an address of family \fIaf\fR, or negative if the +given address family is not supported. +.RE + +.PP +.nf +const char *\fBdns_ntop\fR(int \fIaf\fR, const void *\fIsrc\fR, + char *\fIdst\fR, int \fIdstsize\fR) +.fi +.RS +privides functionality similar to standard \fBinet_ntop\fR() routine, +to convert binary representation of an IP address of family \fIaf\fR +(either \fBAF_INET\fR or \fBAF_INET6\fR) pointed to by \fIsrc\fR +(either 4 or 16 bytes) into printable form in buffer in buffer pointed +to by \fIdst\fR of size \fIdstsize\fR. The destination buffer should be +at least of size 16 bytes for \fBAF_INET\fR family or 46 bytes for +\fBAF_INET6\fR. The return value is either \fIdst\fR, or NULL pointer +if \fIdstsize\fR is too small to hold this address or if the given +address family is not supported. +.RE + +.SH AUTHOR +.PP +The \fBudns\fR library has been written by Michael Tokarev, mjt+udns@tls.msk.ru. + +.SH VERSION +.PP +This manual page corresponds to udns version 0.4, released Jan-2014. diff --git a/contrib/udns/udns.h b/contrib/udns/udns.h new file mode 100644 index 00000000000..371e6976404 --- /dev/null +++ b/contrib/udns/udns.h @@ -0,0 +1,778 @@ +/* udns.h + header file for the UDNS library. + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#ifndef UDNS_VERSION /* include guard */ + +#define UDNS_VERSION "0.4" + +#ifdef WINDOWS +# ifdef UDNS_DYNAMIC_LIBRARY +# ifdef DNS_LIBRARY_BUILD +# define UDNS_API __declspec(dllexport) +# define UDNS_DATA_API __declspec(dllexport) +# else +# define UDNS_API __declspec(dllimport) +# define UDNS_DATA_API __declspec(dllimport) +# endif +# endif +#endif + +#ifndef UDNS_API +# define UDNS_API +#endif +#ifndef UDNS_DATA_API +# define UDNS_DATA_API +#endif + +#include /* for time_t */ + +#ifdef __cplusplus +extern "C" { +#endif + +/* forward declarations if sockets stuff isn't #include'd */ +struct in_addr; +struct in6_addr; +struct sockaddr; + +/**************************************************************************/ +/**************** Common definitions **************************************/ + +UDNS_API const char * +dns_version(void); + +struct dns_ctx; +struct dns_query; + +/* shorthand for [const] unsigned char */ +typedef unsigned char dnsc_t; +typedef const unsigned char dnscc_t; + +#define DNS_MAXDN 255 /* max DN length */ +#define DNS_DNPAD 1 /* padding for DN buffers */ +#define DNS_MAXLABEL 63 /* max DN label length */ +#define DNS_MAXNAME 1024 /* max asciiz domain name length */ +#define DNS_HSIZE 12 /* DNS packet header size */ +#define DNS_PORT 53 /* default domain port */ +#define DNS_MAXSERV 6 /* max servers to consult */ +#define DNS_MAXPACKET 512 /* max traditional-DNS UDP packet size */ +#define DNS_EDNS0PACKET 4096 /* EDNS0 packet size to use */ + +enum dns_class { /* DNS RR Classes */ + DNS_C_INVALID = 0, /* invalid class */ + DNS_C_IN = 1, /* Internet */ + DNS_C_CH = 3, /* CHAOS */ + DNS_C_HS = 4, /* HESIOD */ + DNS_C_ANY = 255 /* wildcard */ +}; + +enum dns_type { /* DNS RR Types */ + DNS_T_INVALID = 0, /* Cookie. */ + DNS_T_A = 1, /* Host address. */ + DNS_T_NS = 2, /* Authoritative server. */ + DNS_T_MD = 3, /* Mail destination. */ + DNS_T_MF = 4, /* Mail forwarder. */ + DNS_T_CNAME = 5, /* Canonical name. */ + DNS_T_SOA = 6, /* Start of authority zone. */ + DNS_T_MB = 7, /* Mailbox domain name. */ + DNS_T_MG = 8, /* Mail group member. */ + DNS_T_MR = 9, /* Mail rename name. */ + DNS_T_NULL = 10, /* Null resource record. */ + DNS_T_WKS = 11, /* Well known service. */ + DNS_T_PTR = 12, /* Domain name pointer. */ + DNS_T_HINFO = 13, /* Host information. */ + DNS_T_MINFO = 14, /* Mailbox information. */ + DNS_T_MX = 15, /* Mail routing information. */ + DNS_T_TXT = 16, /* Text strings. */ + DNS_T_RP = 17, /* Responsible person. */ + DNS_T_AFSDB = 18, /* AFS cell database. */ + DNS_T_X25 = 19, /* X_25 calling address. */ + DNS_T_ISDN = 20, /* ISDN calling address. */ + DNS_T_RT = 21, /* Router. */ + DNS_T_NSAP = 22, /* NSAP address. */ + DNS_T_NSAP_PTR = 23, /* Reverse NSAP lookup (deprecated). */ + DNS_T_SIG = 24, /* Security signature. */ + DNS_T_KEY = 25, /* Security key. */ + DNS_T_PX = 26, /* X.400 mail mapping. */ + DNS_T_GPOS = 27, /* Geographical position (withdrawn). */ + DNS_T_AAAA = 28, /* Ip6 Address. */ + DNS_T_LOC = 29, /* Location Information. */ + DNS_T_NXT = 30, /* Next domain (security). */ + DNS_T_EID = 31, /* Endpoint identifier. */ + DNS_T_NIMLOC = 32, /* Nimrod Locator. */ + DNS_T_SRV = 33, /* Server Selection. */ + DNS_T_ATMA = 34, /* ATM Address */ + DNS_T_NAPTR = 35, /* Naming Authority PoinTeR */ + DNS_T_KX = 36, /* Key Exchange */ + DNS_T_CERT = 37, /* Certification record */ + DNS_T_A6 = 38, /* IPv6 address (deprecates AAAA) */ + DNS_T_DNAME = 39, /* Non-terminal DNAME (for IPv6) */ + DNS_T_SINK = 40, /* Kitchen sink (experimentatl) */ + DNS_T_OPT = 41, /* EDNS0 option (meta-RR) */ + DNS_T_DS = 43, /* DNSSEC */ + DNS_T_SSHFP = 44, + DNS_T_IPSECKEY = 45, + DNS_T_RRSIG = 46, /* DNSSEC */ + DNS_T_NSEC = 47, /* DNSSEC */ + DNS_T_DNSKEY = 48, + DNS_T_DHCID = 49, + DNS_T_NSEC3 = 50, + DNS_T_NSEC3PARAMS = 51, + DNS_T_TALINK = 58, /* draft-ietf-dnsop-trust-history */ + DNS_T_SPF = 99, + DNS_T_UINFO = 100, + DNS_T_UID = 101, + DNS_T_GID = 102, + DNS_T_UNSPEC = 103, + DNS_T_TSIG = 250, /* Transaction signature. */ + DNS_T_IXFR = 251, /* Incremental zone transfer. */ + DNS_T_AXFR = 252, /* Transfer zone of authority. */ + DNS_T_MAILB = 253, /* Transfer mailbox records. */ + DNS_T_MAILA = 254, /* Transfer mail agent records. */ + DNS_T_ANY = 255, /* Wildcard match. */ + DNS_T_ZXFR = 256, /* BIND-specific, nonstandard. */ + DNS_T_DLV = 32769, /* RFC 4431, 5074, DNSSEC Lookaside Validation */ + DNS_T_MAX = 65536 +}; + +/**************************************************************************/ +/**************** Domain Names (DNs) **************************************/ + +/* return length of the DN */ +UDNS_API unsigned +dns_dnlen(dnscc_t *dn); + +/* return #of labels in a DN */ +UDNS_API unsigned +dns_dnlabels(dnscc_t *dn); + +/* lower- and uppercase single DN char */ +#define DNS_DNLC(c) ((c) >= 'A' && (c) <= 'Z' ? (c) - 'A' + 'a' : (c)) +#define DNS_DNUC(c) ((c) >= 'a' && (c) <= 'z' ? (c) - 'a' + 'A' : (c)) + +/* compare the DNs, return dnlen of equal or 0 if not */ +UDNS_API unsigned +dns_dnequal(dnscc_t *dn1, dnscc_t *dn2); + +/* copy one DN to another, size checking */ +UDNS_API unsigned +dns_dntodn(dnscc_t *sdn, dnsc_t *ddn, unsigned ddnsiz); + +/* convert asciiz string of length namelen (0 to use strlen) to DN */ +UDNS_API int +dns_ptodn(const char *name, unsigned namelen, + dnsc_t *dn, unsigned dnsiz, int *isabs); + +/* simpler form of dns_ptodn() */ +#define dns_sptodn(name,dn,dnsiz) dns_ptodn((name),0,(dn),(dnsiz),0) + +UDNS_DATA_API extern dnscc_t dns_inaddr_arpa_dn[14]; +#define DNS_A4RSIZE 30 +UDNS_API int +dns_a4todn(const struct in_addr *addr, dnscc_t *tdn, + dnsc_t *dn, unsigned dnsiz); +UDNS_API int +dns_a4ptodn(const struct in_addr *addr, const char *tname, + dnsc_t *dn, unsigned dnsiz); +UDNS_API dnsc_t * +dns_a4todn_(const struct in_addr *addr, dnsc_t *dn, dnsc_t *dne); + +UDNS_DATA_API extern dnscc_t dns_ip6_arpa_dn[10]; +#define DNS_A6RSIZE 74 +UDNS_API int +dns_a6todn(const struct in6_addr *addr, dnscc_t *tdn, + dnsc_t *dn, unsigned dnsiz); +UDNS_API int +dns_a6ptodn(const struct in6_addr *addr, const char *tname, + dnsc_t *dn, unsigned dnsiz); +UDNS_API dnsc_t * +dns_a6todn_(const struct in6_addr *addr, dnsc_t *dn, dnsc_t *dne); + +/* convert DN into asciiz string */ +UDNS_API int +dns_dntop(dnscc_t *dn, char *name, unsigned namesiz); + +/* convert DN into asciiz string, using static buffer (NOT thread-safe!) */ +UDNS_API const char * +dns_dntosp(dnscc_t *dn); + +/* return buffer size (incl. null byte) required for asciiz form of a DN */ +UDNS_API unsigned +dns_dntop_size(dnscc_t *dn); + +/* either wrappers or reimplementations for inet_ntop() and inet_pton() */ +UDNS_API const char *dns_ntop(int af, const void *src, char *dst, int size); +UDNS_API int dns_pton(int af, const char *src, void *dst); + +/**************************************************************************/ +/**************** DNS raw packet layout ***********************************/ + +enum dns_rcode { /* reply codes */ + DNS_R_NOERROR = 0, /* ok, no error */ + DNS_R_FORMERR = 1, /* format error */ + DNS_R_SERVFAIL = 2, /* server failed */ + DNS_R_NXDOMAIN = 3, /* domain does not exists */ + DNS_R_NOTIMPL = 4, /* not implemented */ + DNS_R_REFUSED = 5, /* query refused */ + /* these are for BIND_UPDATE */ + DNS_R_YXDOMAIN = 6, /* Name exists */ + DNS_R_YXRRSET = 7, /* RRset exists */ + DNS_R_NXRRSET = 8, /* RRset does not exist */ + DNS_R_NOTAUTH = 9, /* Not authoritative for zone */ + DNS_R_NOTZONE = 10, /* Zone of record different from zone section */ + /*ns_r_max = 11,*/ + /* The following are TSIG extended errors */ + DNS_R_BADSIG = 16, + DNS_R_BADKEY = 17, + DNS_R_BADTIME = 18 +}; + +static __inline unsigned dns_get16(dnscc_t *s) { + return ((unsigned)s[0]<<8) | s[1]; +} +static __inline unsigned dns_get32(dnscc_t *s) { + return ((unsigned)s[0]<<24) | ((unsigned)s[1]<<16) + | ((unsigned)s[2]<<8) | s[3]; +} +static __inline dnsc_t *dns_put16(dnsc_t *d, unsigned n) { + *d++ = (dnsc_t)((n >> 8) & 255); *d++ = (dnsc_t)(n & 255); return d; +} +static __inline dnsc_t *dns_put32(dnsc_t *d, unsigned n) { + *d++ = (dnsc_t)((n >> 24) & 255); *d++ = (dnsc_t)((n >> 16) & 255); + *d++ = (dnsc_t)((n >> 8) & 255); *d++ = (dnsc_t)(n & 255); + return d; +} + +/* DNS Header layout */ +enum { + /* bytes 0:1 - query ID */ + DNS_H_QID1 = 0, + DNS_H_QID2 = 1, + DNS_H_QID = DNS_H_QID1, +#define dns_qid(pkt) dns_get16((pkt)+DNS_H_QID) + /* byte 2: flags1 */ + DNS_H_F1 = 2, + DNS_HF1_QR = 0x80, /* query response flag */ +#define dns_qr(pkt) ((pkt)[DNS_H_F1]&DNS_HF1_QR) + DNS_HF1_OPCODE = 0x78, /* opcode, 0 = query */ +#define dns_opcode(pkt) (((pkt)[DNS_H_F1]&DNS_HF1_OPCODE)>>3) + DNS_HF1_AA = 0x04, /* auth answer */ +#define dns_aa(pkt) ((pkt)[DNS_H_F1]&DNS_HF1_AA) + DNS_HF1_TC = 0x02, /* truncation flag */ +#define dns_tc(pkt) ((pkt)[DNS_H_F1]&DNS_HF1_TC) + DNS_HF1_RD = 0x01, /* recursion desired (may be set in query) */ +#define dns_rd(pkt) ((pkt)[DNS_H_F1]&DNS_HF1_RD) + /* byte 3: flags2 */ + DNS_H_F2 = 3, + DNS_HF2_RA = 0x80, /* recursion available */ +#define dns_ra(pkt) ((pkt)[DNS_H_F2]&DNS_HF2_RA) + DNS_HF2_Z = 0x40, /* reserved */ + DNS_HF2_AD = 0x20, /* DNSSEC: authentic data */ +#define dns_ad(pkt) ((pkt)[DNS_H_F2]&DNS_HF2_AD) + DNS_HF2_CD = 0x10, /* DNSSEC: checking disabled */ +#define dns_cd(pkt) ((pkt)[DNS_H_F2]&DNS_HF2_CD) + DNS_HF2_RCODE = 0x0f, /* response code, DNS_R_XXX above */ +#define dns_rcode(pkt) ((pkt)[DNS_H_F2]&DNS_HF2_RCODE) + /* bytes 4:5: qdcount, numqueries */ + DNS_H_QDCNT1 = 4, + DNS_H_QDCNT2 = 5, + DNS_H_QDCNT = DNS_H_QDCNT1, +#define dns_numqd(pkt) dns_get16((pkt)+4) + /* bytes 6:7: ancount, numanswers */ + DNS_H_ANCNT1 = 6, + DNS_H_ANCNT2 = 7, + DNS_H_ANCNT = DNS_H_ANCNT1, +#define dns_numan(pkt) dns_get16((pkt)+6) + /* bytes 8:9: nscount, numauthority */ + DNS_H_NSCNT1 = 8, + DNS_H_NSCNT2 = 9, + DNS_H_NSCNT = DNS_H_NSCNT1, +#define dns_numns(pkt) dns_get16((pkt)+8) + /* bytes 10:11: arcount, numadditional */ + DNS_H_ARCNT1 = 10, + DNS_H_ARCNT2 = 11, + DNS_H_ARCNT = DNS_H_ARCNT1, +#define dns_numar(pkt) dns_get16((pkt)+10) +#define dns_payload(pkt) ((pkt)+DNS_HSIZE) + /* EDNS0 (OPT RR) flags (Ext. Flags) */ + DNS_EF1_DO = 0x80, /* DNSSEC OK */ +}; + +/* packet buffer: start at pkt, end before pkte, current pos *curp. + * extract a DN and set *curp to the next byte after DN in packet. + * return -1 on error, 0 if dnsiz is too small, or dnlen on ok. + */ +UDNS_API int +dns_getdn(dnscc_t *pkt, dnscc_t **curp, dnscc_t *end, + dnsc_t *dn, unsigned dnsiz); + +/* skip the DN at position cur in packet ending before pkte, + * return pointer to the next byte after the DN or NULL on error */ +UDNS_API dnscc_t * +dns_skipdn(dnscc_t *end, dnscc_t *cur); + +struct dns_rr { /* DNS Resource Record */ + dnsc_t dnsrr_dn[DNS_MAXDN]; /* the DN of the RR */ + enum dns_class dnsrr_cls; /* Class */ + enum dns_type dnsrr_typ; /* Type */ + unsigned dnsrr_ttl; /* Time-To-Live (TTL) */ + unsigned dnsrr_dsz; /* data size */ + dnscc_t *dnsrr_dptr; /* pointer to start of data */ + dnscc_t *dnsrr_dend; /* past end of data */ +}; + +struct dns_parse { /* RR/packet parsing state */ + dnscc_t *dnsp_pkt; /* start of the packet */ + dnscc_t *dnsp_end; /* end of the packet */ + dnscc_t *dnsp_cur; /* current packet position */ + dnscc_t *dnsp_ans; /* start of answer section */ + int dnsp_rrl; /* number of RRs left to go */ + int dnsp_nrr; /* RR count so far */ + unsigned dnsp_ttl; /* TTL value so far */ + dnscc_t *dnsp_qdn; /* the RR DN we're looking for */ + enum dns_class dnsp_qcls; /* RR class we're looking for or 0 */ + enum dns_type dnsp_qtyp; /* RR type we're looking for or 0 */ + dnsc_t dnsp_dnbuf[DNS_MAXDN]; /* domain buffer */ +}; + +/* initialize the parse structure */ +UDNS_API void +dns_initparse(struct dns_parse *p, dnscc_t *qdn, + dnscc_t *pkt, dnscc_t *cur, dnscc_t *end); + +/* search next RR, <0=error, 0=no more RRs, >0 = found. */ +UDNS_API int +dns_nextrr(struct dns_parse *p, struct dns_rr *rr); + +UDNS_API void +dns_rewind(struct dns_parse *p, dnscc_t *qdn); + + +/**************************************************************************/ +/**************** Resolver Context ****************************************/ + +/* default resolver context */ +UDNS_DATA_API extern struct dns_ctx dns_defctx; + +/* reset resolver context to default state, close it if open, drop queries */ +UDNS_API void +dns_reset(struct dns_ctx *ctx); + +/* reset resolver context and read in system configuration */ +UDNS_API int +dns_init(struct dns_ctx *ctx, int do_open); + +/* return new resolver context with the same settings as copy */ +UDNS_API struct dns_ctx * +dns_new(const struct dns_ctx *copy); + +/* free resolver context returned by dns_new(); all queries are dropped */ +UDNS_API void +dns_free(struct dns_ctx *ctx); + +/* add nameserver for a resolver context (or reset nslist if serv==NULL) */ +UDNS_API int +dns_add_serv(struct dns_ctx *ctx, const char *serv); + +/* add nameserver using struct sockaddr structure (with ports) */ +UDNS_API int +dns_add_serv_s(struct dns_ctx *ctx, const struct sockaddr *sa); + +/* add search list element for a resolver context (or reset it if srch==NULL) */ +UDNS_API int +dns_add_srch(struct dns_ctx *ctx, const char *srch); + +/* set options for a resolver context */ +UDNS_API int +dns_set_opts(struct dns_ctx *ctx, const char *opts); + +enum dns_opt { /* options */ + DNS_OPT_FLAGS, /* flags, DNS_F_XXX */ + DNS_OPT_TIMEOUT, /* timeout in secounds */ + DNS_OPT_NTRIES, /* number of retries */ + DNS_OPT_NDOTS, /* ndots */ + DNS_OPT_UDPSIZE, /* EDNS0 UDP size */ + DNS_OPT_PORT, /* port to use */ +}; + +/* set or get (if val<0) an option */ +UDNS_API int +dns_set_opt(struct dns_ctx *ctx, enum dns_opt opt, int val); + +enum dns_flags { + DNS_NOSRCH = 0x00010000, /* do not perform search */ + DNS_NORD = 0x00020000, /* request no recursion */ + DNS_AAONLY = 0x00040000, /* set AA flag in queries */ + DNS_SET_DO = 0x00080000, /* set EDNS0 "DO" bit (DNSSEC OK) */ + DNS_SET_CD = 0x00100000, /* set CD bit (DNSSEC: checking disabled) */ +}; + +/* set the debug function pointer */ +typedef void +(dns_dbgfn)(int code, const struct sockaddr *sa, unsigned salen, + dnscc_t *pkt, int plen, + const struct dns_query *q, void *data); +UDNS_API void +dns_set_dbgfn(struct dns_ctx *ctx, dns_dbgfn *dbgfn); + +/* open and return UDP socket */ +UDNS_API int +dns_open(struct dns_ctx *ctx); + +/* return UDP socket or -1 if not open */ +UDNS_API int +dns_sock(const struct dns_ctx *ctx); + +/* close the UDP socket */ +UDNS_API void +dns_close(struct dns_ctx *ctx); + +/* return number of requests queued */ +UDNS_API int +dns_active(const struct dns_ctx *ctx); + +/* return status of the last operation */ +UDNS_API int +dns_status(const struct dns_ctx *ctx); +UDNS_API void +dns_setstatus(struct dns_ctx *ctx, int status); + +/* handle I/O event on UDP socket */ +UDNS_API void +dns_ioevent(struct dns_ctx *ctx, time_t now); + +/* process any timeouts, return time in secounds to the + * next timeout (or -1 if none) but not greather than maxwait */ +UDNS_API int +dns_timeouts(struct dns_ctx *ctx, int maxwait, time_t now); + +/* define timer requesting routine to use */ +typedef void dns_utm_fn(struct dns_ctx *ctx, int timeout, void *data); +UDNS_API void +dns_set_tmcbck(struct dns_ctx *ctx, dns_utm_fn *fn, void *data); + +/**************************************************************************/ +/**************** Making Queries ******************************************/ + +/* query callback routine */ +typedef void dns_query_fn(struct dns_ctx *ctx, void *result, void *data); + +/* query parse routine: raw DNS => application structure */ +typedef int +dns_parse_fn(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, + void **res); + +enum dns_status { + DNS_E_NOERROR = 0, /* ok, not an error */ + DNS_E_TEMPFAIL = -1, /* timeout, SERVFAIL or similar */ + DNS_E_PROTOCOL = -2, /* got garbled reply */ + DNS_E_NXDOMAIN = -3, /* domain does not exists */ + DNS_E_NODATA = -4, /* domain exists but no data of reqd type */ + DNS_E_NOMEM = -5, /* out of memory while processing */ + DNS_E_BADQUERY = -6 /* the query is malformed */ +}; + +/* submit generic DN query */ +UDNS_API struct dns_query * +dns_submit_dn(struct dns_ctx *ctx, + dnscc_t *dn, int qcls, int qtyp, int flags, + dns_parse_fn *parse, dns_query_fn *cbck, void *data); +/* submit generic name query */ +UDNS_API struct dns_query * +dns_submit_p(struct dns_ctx *ctx, + const char *name, int qcls, int qtyp, int flags, + dns_parse_fn *parse, dns_query_fn *cbck, void *data); + +/* cancel the given async query in progress */ +UDNS_API int +dns_cancel(struct dns_ctx *ctx, struct dns_query *q); + +/* resolve a generic query, return the answer */ +UDNS_API void * +dns_resolve_dn(struct dns_ctx *ctx, + dnscc_t *qdn, int qcls, int qtyp, int flags, + dns_parse_fn *parse); +UDNS_API void * +dns_resolve_p(struct dns_ctx *ctx, + const char *qname, int qcls, int qtyp, int flags, + dns_parse_fn *parse); +UDNS_API void * +dns_resolve(struct dns_ctx *ctx, struct dns_query *q); + + +/* Specific RR handlers */ + +#define dns_rr_common(prefix) \ + char *prefix##_cname; /* canonical name */ \ + char *prefix##_qname; /* original query name */ \ + unsigned prefix##_ttl; /* TTL value */ \ + int prefix##_nrr /* number of records */ + +struct dns_rr_null { /* NULL RRset, aka RRset template */ + dns_rr_common(dnsn); +}; + +UDNS_API int +dns_stdrr_size(const struct dns_parse *p); +UDNS_API void * +dns_stdrr_finish(struct dns_rr_null *ret, char *cp, const struct dns_parse *p); + +struct dns_rr_a4 { /* the A RRset */ + dns_rr_common(dnsa4); + struct in_addr *dnsa4_addr; /* array of addresses, naddr elements */ +}; + +UDNS_API dns_parse_fn dns_parse_a4; /* A RR parsing routine */ +typedef void /* A query callback routine */ +dns_query_a4_fn(struct dns_ctx *ctx, struct dns_rr_a4 *result, void *data); + +/* submit A IN query */ +UDNS_API struct dns_query * +dns_submit_a4(struct dns_ctx *ctx, const char *name, int flags, + dns_query_a4_fn *cbck, void *data); + +/* resolve A IN query */ +UDNS_API struct dns_rr_a4 * +dns_resolve_a4(struct dns_ctx *ctx, const char *name, int flags); + + +struct dns_rr_a6 { /* the AAAA RRset */ + dns_rr_common(dnsa6); + struct in6_addr *dnsa6_addr; /* array of addresses, naddr elements */ +}; + +UDNS_API dns_parse_fn dns_parse_a6; /* A RR parsing routine */ +typedef void /* A query callback routine */ +dns_query_a6_fn(struct dns_ctx *ctx, struct dns_rr_a6 *result, void *data); + +/* submit AAAA IN query */ +UDNS_API struct dns_query * +dns_submit_a6(struct dns_ctx *ctx, const char *name, int flags, + dns_query_a6_fn *cbck, void *data); + +/* resolve AAAA IN query */ +UDNS_API struct dns_rr_a6 * +dns_resolve_a6(struct dns_ctx *ctx, const char *name, int flags); + + +struct dns_rr_ptr { /* the PTR RRset */ + dns_rr_common(dnsptr); + char **dnsptr_ptr; /* array of PTRs */ +}; + +UDNS_API dns_parse_fn dns_parse_ptr; /* PTR RR parsing routine */ +typedef void /* PTR query callback */ +dns_query_ptr_fn(struct dns_ctx *ctx, struct dns_rr_ptr *result, void *data); +/* submit PTR IN in-addr.arpa query */ +UDNS_API struct dns_query * +dns_submit_a4ptr(struct dns_ctx *ctx, const struct in_addr *addr, + dns_query_ptr_fn *cbck, void *data); +/* resolve PTR IN in-addr.arpa query */ +UDNS_API struct dns_rr_ptr * +dns_resolve_a4ptr(struct dns_ctx *ctx, const struct in_addr *addr); + +/* the same as above, but for ip6.arpa */ +UDNS_API struct dns_query * +dns_submit_a6ptr(struct dns_ctx *ctx, const struct in6_addr *addr, + dns_query_ptr_fn *cbck, void *data); +UDNS_API struct dns_rr_ptr * +dns_resolve_a6ptr(struct dns_ctx *ctx, const struct in6_addr *addr); + + +struct dns_mx { /* single MX RR */ + int priority; /* MX priority */ + char *name; /* MX name */ +}; +struct dns_rr_mx { /* the MX RRset */ + dns_rr_common(dnsmx); + struct dns_mx *dnsmx_mx; /* array of MXes */ +}; +UDNS_API dns_parse_fn dns_parse_mx; /* MX RR parsing routine */ +typedef void /* MX RR callback */ +dns_query_mx_fn(struct dns_ctx *ctx, struct dns_rr_mx *result, void *data); +/* submit MX IN query */ +UDNS_API struct dns_query * +dns_submit_mx(struct dns_ctx *ctx, const char *name, int flags, + dns_query_mx_fn *cbck, void *data); +/* resolve MX IN query */ +UDNS_API struct dns_rr_mx * +dns_resolve_mx(struct dns_ctx *ctx, const char *name, int flags); + + +struct dns_txt { /* single TXT record */ + int len; /* length of the text */ + dnsc_t *txt; /* pointer to text buffer. May contain nulls. */ +}; +struct dns_rr_txt { /* the TXT RRset */ + dns_rr_common(dnstxt); + struct dns_txt *dnstxt_txt; /* array of TXT records */ +}; +UDNS_API dns_parse_fn dns_parse_txt; /* TXT RR parsing routine */ +typedef void /* TXT RR callback */ +dns_query_txt_fn(struct dns_ctx *ctx, struct dns_rr_txt *result, void *data); +/* submit TXT query */ +UDNS_API struct dns_query * +dns_submit_txt(struct dns_ctx *ctx, const char *name, int qcls, int flags, + dns_query_txt_fn *cbck, void *data); +/* resolve TXT query */ +UDNS_API struct dns_rr_txt * +dns_resolve_txt(struct dns_ctx *ctx, const char *name, int qcls, int flags); + + +struct dns_srv { /* single SRV RR */ + int priority; /* SRV priority */ + int weight; /* SRV weight */ + int port; /* SRV port */ + char *name; /* SRV name */ +}; +struct dns_rr_srv { /* the SRV RRset */ + dns_rr_common(dnssrv); + struct dns_srv *dnssrv_srv; /* array of SRVes */ +}; +UDNS_API dns_parse_fn dns_parse_srv; /* SRV RR parsing routine */ +typedef void /* SRV RR callback */ +dns_query_srv_fn(struct dns_ctx *ctx, struct dns_rr_srv *result, void *data); +/* submit SRV IN query */ +UDNS_API struct dns_query * +dns_submit_srv(struct dns_ctx *ctx, + const char *name, const char *srv, const char *proto, + int flags, dns_query_srv_fn *cbck, void *data); +/* resolve SRV IN query */ +UDNS_API struct dns_rr_srv * +dns_resolve_srv(struct dns_ctx *ctx, + const char *name, const char *srv, const char *proto, + int flags); + +/* NAPTR (RFC3403) RR type */ +struct dns_naptr { /* single NAPTR RR */ + int order; /* NAPTR order */ + int preference; /* NAPTR preference */ + char *flags; /* NAPTR flags */ + char *service; /* NAPTR service */ + char *regexp; /* NAPTR regexp */ + char *replacement; /* NAPTR replacement */ +}; + +struct dns_rr_naptr { /* the NAPTR RRset */ + dns_rr_common(dnsnaptr); + struct dns_naptr *dnsnaptr_naptr; /* array of NAPTRes */ +}; +UDNS_API dns_parse_fn dns_parse_naptr; /* NAPTR RR parsing routine */ +typedef void /* NAPTR RR callback */ +dns_query_naptr_fn(struct dns_ctx *ctx, + struct dns_rr_naptr *result, void *data); +/* submit NAPTR IN query */ +UDNS_API struct dns_query * +dns_submit_naptr(struct dns_ctx *ctx, const char *name, int flags, + dns_query_naptr_fn *cbck, void *data); +/* resolve NAPTR IN query */ +UDNS_API struct dns_rr_naptr * +dns_resolve_naptr(struct dns_ctx *ctx, const char *name, int flags); + + +UDNS_API struct dns_query * +dns_submit_a4dnsbl(struct dns_ctx *ctx, + const struct in_addr *addr, const char *dnsbl, + dns_query_a4_fn *cbck, void *data); +UDNS_API struct dns_query * +dns_submit_a4dnsbl_txt(struct dns_ctx *ctx, + const struct in_addr *addr, const char *dnsbl, + dns_query_txt_fn *cbck, void *data); +UDNS_API struct dns_rr_a4 * +dns_resolve_a4dnsbl(struct dns_ctx *ctx, + const struct in_addr *addr, const char *dnsbl); +UDNS_API struct dns_rr_txt * +dns_resolve_a4dnsbl_txt(struct dns_ctx *ctx, + const struct in_addr *addr, const char *dnsbl); + +UDNS_API struct dns_query * +dns_submit_a6dnsbl(struct dns_ctx *ctx, + const struct in6_addr *addr, const char *dnsbl, + dns_query_a4_fn *cbck, void *data); +UDNS_API struct dns_query * +dns_submit_a6dnsbl_txt(struct dns_ctx *ctx, + const struct in6_addr *addr, const char *dnsbl, + dns_query_txt_fn *cbck, void *data); +UDNS_API struct dns_rr_a4 * +dns_resolve_a6dnsbl(struct dns_ctx *ctx, + const struct in6_addr *addr, const char *dnsbl); +UDNS_API struct dns_rr_txt * +dns_resolve_a6dnsbl_txt(struct dns_ctx *ctx, + const struct in6_addr *addr, const char *dnsbl); + +UDNS_API struct dns_query * +dns_submit_rhsbl(struct dns_ctx *ctx, + const char *name, const char *rhsbl, + dns_query_a4_fn *cbck, void *data); +UDNS_API struct dns_query * +dns_submit_rhsbl_txt(struct dns_ctx *ctx, + const char *name, const char *rhsbl, + dns_query_txt_fn *cbck, void *data); +UDNS_API struct dns_rr_a4 * +dns_resolve_rhsbl(struct dns_ctx *ctx, const char *name, const char *rhsbl); +UDNS_API struct dns_rr_txt * +dns_resolve_rhsbl_txt(struct dns_ctx *ctx, const char *name, const char *rhsbl); + +/**************************************************************************/ +/**************** Names, Names ********************************************/ + +struct dns_nameval { + int val; + const char *name; +}; + +UDNS_DATA_API extern const struct dns_nameval dns_classtab[]; +UDNS_DATA_API extern const struct dns_nameval dns_typetab[]; +UDNS_DATA_API extern const struct dns_nameval dns_rcodetab[]; +UDNS_API int +dns_findname(const struct dns_nameval *nv, const char *name); +#define dns_findclassname(cls) dns_findname(dns_classtab, (cls)) +#define dns_findtypename(type) dns_findname(dns_typetab, (type)) +#define dns_findrcodename(rcode) dns_findname(dns_rcodetab, (rcode)) + +UDNS_API const char *dns_classname(enum dns_class cls); +UDNS_API const char *dns_typename(enum dns_type type); +UDNS_API const char *dns_rcodename(enum dns_rcode rcode); +const char *_dns_format_code(char *buf, const char *prefix, int code); + +UDNS_API const char *dns_strerror(int errnum); + +/* simple pseudo-random number generator, code by Bob Jenkins */ + +struct udns_jranctx { /* the context */ + unsigned a, b, c, d; +}; + +/* initialize the RNG with a given seed */ +UDNS_API void +udns_jraninit(struct udns_jranctx *x, unsigned seed); + +/* return next random number. 32bits on most platforms so far. */ +UDNS_API unsigned +udns_jranval(struct udns_jranctx *x); + +#ifdef __cplusplus +} /* extern "C" */ +#endif + +#endif /* include guard */ diff --git a/contrib/udns/udns_XtoX.c b/contrib/udns/udns_XtoX.c new file mode 100644 index 00000000000..60e3fdfa320 --- /dev/null +++ b/contrib/udns/udns_XtoX.c @@ -0,0 +1,50 @@ +/* udns_XtoX.c + udns_ntop() and udns_pton() routines, which are either + - wrappers for inet_ntop() and inet_pton() or + - reimplementations of those routines. + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#ifdef HAVE_CONFIG_H +# include "config.h" +#endif +#include "udns.h" + +#ifdef HAVE_INET_PTON_NTOP + +#include +#include +#include + +const char *dns_ntop(int af, const void *src, char *dst, int size) { + return inet_ntop(af, src, dst, size); +} + +int dns_pton(int af, const char *src, void *dst) { + return inet_pton(af, src, dst); +} + +#else + +#define inet_XtoX_prefix udns_ +#include "inet_XtoX.c" + +#endif diff --git a/contrib/udns/udns_bl.c b/contrib/udns/udns_bl.c new file mode 100644 index 00000000000..f6be39335f1 --- /dev/null +++ b/contrib/udns/udns_bl.c @@ -0,0 +1,160 @@ +/* udns_bl.c + DNSBL stuff + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#include "udns.h" +#ifndef NULL +# define NULL 0 +#endif + +struct dns_query * +dns_submit_a4dnsbl(struct dns_ctx *ctx, + const struct in_addr *addr, const char *dnsbl, + dns_query_a4_fn *cbck, void *data) { + dnsc_t dn[DNS_MAXDN]; + if (dns_a4ptodn(addr, dnsbl, dn, sizeof(dn)) <= 0) { + dns_setstatus(ctx, DNS_E_BADQUERY); + return NULL; + } + return + dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_A, DNS_NOSRCH, + dns_parse_a4, (dns_query_fn*)cbck, data); +} + +struct dns_query * +dns_submit_a4dnsbl_txt(struct dns_ctx *ctx, + const struct in_addr *addr, const char *dnsbl, + dns_query_txt_fn *cbck, void *data) { + dnsc_t dn[DNS_MAXDN]; + if (dns_a4ptodn(addr, dnsbl, dn, sizeof(dn)) <= 0) { + dns_setstatus(ctx, DNS_E_BADQUERY); + return NULL; + } + return + dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_TXT, DNS_NOSRCH, + dns_parse_txt, (dns_query_fn*)cbck, data); +} + +struct dns_rr_a4 * +dns_resolve_a4dnsbl(struct dns_ctx *ctx, + const struct in_addr *addr, const char *dnsbl) { + return (struct dns_rr_a4 *) + dns_resolve(ctx, dns_submit_a4dnsbl(ctx, addr, dnsbl, 0, 0)); +} + +struct dns_rr_txt * +dns_resolve_a4dnsbl_txt(struct dns_ctx *ctx, + const struct in_addr *addr, const char *dnsbl) { + return (struct dns_rr_txt *) + dns_resolve(ctx, dns_submit_a4dnsbl_txt(ctx, addr, dnsbl, 0, 0)); +} + + +struct dns_query * +dns_submit_a6dnsbl(struct dns_ctx *ctx, + const struct in6_addr *addr, const char *dnsbl, + dns_query_a4_fn *cbck, void *data) { + dnsc_t dn[DNS_MAXDN]; + if (dns_a6ptodn(addr, dnsbl, dn, sizeof(dn)) <= 0) { + dns_setstatus(ctx, DNS_E_BADQUERY); + return NULL; + } + return + dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_A, DNS_NOSRCH, + dns_parse_a4, (dns_query_fn*)cbck, data); +} + +struct dns_query * +dns_submit_a6dnsbl_txt(struct dns_ctx *ctx, + const struct in6_addr *addr, const char *dnsbl, + dns_query_txt_fn *cbck, void *data) { + dnsc_t dn[DNS_MAXDN]; + if (dns_a6ptodn(addr, dnsbl, dn, sizeof(dn)) <= 0) { + dns_setstatus(ctx, DNS_E_BADQUERY); + return NULL; + } + return + dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_TXT, DNS_NOSRCH, + dns_parse_txt, (dns_query_fn*)cbck, data); +} + +struct dns_rr_a4 * +dns_resolve_a6dnsbl(struct dns_ctx *ctx, + const struct in6_addr *addr, const char *dnsbl) { + return (struct dns_rr_a4 *) + dns_resolve(ctx, dns_submit_a6dnsbl(ctx, addr, dnsbl, 0, 0)); +} + +struct dns_rr_txt * +dns_resolve_a6dnsbl_txt(struct dns_ctx *ctx, + const struct in6_addr *addr, const char *dnsbl) { + return (struct dns_rr_txt *) + dns_resolve(ctx, dns_submit_a6dnsbl_txt(ctx, addr, dnsbl, 0, 0)); +} + +static int +dns_rhsbltodn(const char *name, const char *rhsbl, dnsc_t dn[DNS_MAXDN]) +{ + int l = dns_sptodn(name, dn, DNS_MAXDN); + if (l <= 0) return 0; + l = dns_sptodn(rhsbl, dn+l-1, DNS_MAXDN-l+1); + if (l <= 0) return 0; + return 1; +} + +struct dns_query * +dns_submit_rhsbl(struct dns_ctx *ctx, const char *name, const char *rhsbl, + dns_query_a4_fn *cbck, void *data) { + dnsc_t dn[DNS_MAXDN]; + if (!dns_rhsbltodn(name, rhsbl, dn)) { + dns_setstatus(ctx, DNS_E_BADQUERY); + return NULL; + } + return + dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_A, DNS_NOSRCH, + dns_parse_a4, (dns_query_fn*)cbck, data); +} +struct dns_query * +dns_submit_rhsbl_txt(struct dns_ctx *ctx, const char *name, const char *rhsbl, + dns_query_txt_fn *cbck, void *data) { + dnsc_t dn[DNS_MAXDN]; + if (!dns_rhsbltodn(name, rhsbl, dn)) { + dns_setstatus(ctx, DNS_E_BADQUERY); + return NULL; + } + return + dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_TXT, DNS_NOSRCH, + dns_parse_txt, (dns_query_fn*)cbck, data); +} + +struct dns_rr_a4 * +dns_resolve_rhsbl(struct dns_ctx *ctx, const char *name, const char *rhsbl) { + return (struct dns_rr_a4*) + dns_resolve(ctx, dns_submit_rhsbl(ctx, name, rhsbl, 0, 0)); +} + +struct dns_rr_txt * +dns_resolve_rhsbl_txt(struct dns_ctx *ctx, const char *name, const char *rhsbl) +{ + return (struct dns_rr_txt*) + dns_resolve(ctx, dns_submit_rhsbl_txt(ctx, name, rhsbl, 0, 0)); +} diff --git a/contrib/udns/udns_codes.c b/contrib/udns/udns_codes.c new file mode 100644 index 00000000000..c637e98c581 --- /dev/null +++ b/contrib/udns/udns_codes.c @@ -0,0 +1,199 @@ +/* Automatically generated. */ +#include "udns.h" + +const struct dns_nameval dns_typetab[] = { + {DNS_T_INVALID,"INVALID"}, + {DNS_T_A,"A"}, + {DNS_T_NS,"NS"}, + {DNS_T_MD,"MD"}, + {DNS_T_MF,"MF"}, + {DNS_T_CNAME,"CNAME"}, + {DNS_T_SOA,"SOA"}, + {DNS_T_MB,"MB"}, + {DNS_T_MG,"MG"}, + {DNS_T_MR,"MR"}, + {DNS_T_NULL,"NULL"}, + {DNS_T_WKS,"WKS"}, + {DNS_T_PTR,"PTR"}, + {DNS_T_HINFO,"HINFO"}, + {DNS_T_MINFO,"MINFO"}, + {DNS_T_MX,"MX"}, + {DNS_T_TXT,"TXT"}, + {DNS_T_RP,"RP"}, + {DNS_T_AFSDB,"AFSDB"}, + {DNS_T_X25,"X25"}, + {DNS_T_ISDN,"ISDN"}, + {DNS_T_RT,"RT"}, + {DNS_T_NSAP,"NSAP"}, + {DNS_T_NSAP_PTR,"NSAP_PTR"}, + {DNS_T_SIG,"SIG"}, + {DNS_T_KEY,"KEY"}, + {DNS_T_PX,"PX"}, + {DNS_T_GPOS,"GPOS"}, + {DNS_T_AAAA,"AAAA"}, + {DNS_T_LOC,"LOC"}, + {DNS_T_NXT,"NXT"}, + {DNS_T_EID,"EID"}, + {DNS_T_NIMLOC,"NIMLOC"}, + {DNS_T_SRV,"SRV"}, + {DNS_T_ATMA,"ATMA"}, + {DNS_T_NAPTR,"NAPTR"}, + {DNS_T_KX,"KX"}, + {DNS_T_CERT,"CERT"}, + {DNS_T_A6,"A6"}, + {DNS_T_DNAME,"DNAME"}, + {DNS_T_SINK,"SINK"}, + {DNS_T_OPT,"OPT"}, + {DNS_T_DS,"DS"}, + {DNS_T_SSHFP,"SSHFP"}, + {DNS_T_IPSECKEY,"IPSECKEY"}, + {DNS_T_RRSIG,"RRSIG"}, + {DNS_T_NSEC,"NSEC"}, + {DNS_T_DNSKEY,"DNSKEY"}, + {DNS_T_DHCID,"DHCID"}, + {DNS_T_NSEC3,"NSEC3"}, + {DNS_T_NSEC3PARAMS,"NSEC3PARAMS"}, + {DNS_T_TALINK,"TALINK"}, + {DNS_T_SPF,"SPF"}, + {DNS_T_UINFO,"UINFO"}, + {DNS_T_UID,"UID"}, + {DNS_T_GID,"GID"}, + {DNS_T_UNSPEC,"UNSPEC"}, + {DNS_T_TSIG,"TSIG"}, + {DNS_T_IXFR,"IXFR"}, + {DNS_T_AXFR,"AXFR"}, + {DNS_T_MAILB,"MAILB"}, + {DNS_T_MAILA,"MAILA"}, + {DNS_T_ANY,"ANY"}, + {DNS_T_ZXFR,"ZXFR"}, + {DNS_T_DLV,"DLV"}, + {DNS_T_MAX,"MAX"}, + {0,0}}; +const char *dns_typename(enum dns_type code) { + static char nm[20]; + switch(code) { + case DNS_T_INVALID: return dns_typetab[0].name; + case DNS_T_A: return dns_typetab[1].name; + case DNS_T_NS: return dns_typetab[2].name; + case DNS_T_MD: return dns_typetab[3].name; + case DNS_T_MF: return dns_typetab[4].name; + case DNS_T_CNAME: return dns_typetab[5].name; + case DNS_T_SOA: return dns_typetab[6].name; + case DNS_T_MB: return dns_typetab[7].name; + case DNS_T_MG: return dns_typetab[8].name; + case DNS_T_MR: return dns_typetab[9].name; + case DNS_T_NULL: return dns_typetab[10].name; + case DNS_T_WKS: return dns_typetab[11].name; + case DNS_T_PTR: return dns_typetab[12].name; + case DNS_T_HINFO: return dns_typetab[13].name; + case DNS_T_MINFO: return dns_typetab[14].name; + case DNS_T_MX: return dns_typetab[15].name; + case DNS_T_TXT: return dns_typetab[16].name; + case DNS_T_RP: return dns_typetab[17].name; + case DNS_T_AFSDB: return dns_typetab[18].name; + case DNS_T_X25: return dns_typetab[19].name; + case DNS_T_ISDN: return dns_typetab[20].name; + case DNS_T_RT: return dns_typetab[21].name; + case DNS_T_NSAP: return dns_typetab[22].name; + case DNS_T_NSAP_PTR: return dns_typetab[23].name; + case DNS_T_SIG: return dns_typetab[24].name; + case DNS_T_KEY: return dns_typetab[25].name; + case DNS_T_PX: return dns_typetab[26].name; + case DNS_T_GPOS: return dns_typetab[27].name; + case DNS_T_AAAA: return dns_typetab[28].name; + case DNS_T_LOC: return dns_typetab[29].name; + case DNS_T_NXT: return dns_typetab[30].name; + case DNS_T_EID: return dns_typetab[31].name; + case DNS_T_NIMLOC: return dns_typetab[32].name; + case DNS_T_SRV: return dns_typetab[33].name; + case DNS_T_ATMA: return dns_typetab[34].name; + case DNS_T_NAPTR: return dns_typetab[35].name; + case DNS_T_KX: return dns_typetab[36].name; + case DNS_T_CERT: return dns_typetab[37].name; + case DNS_T_A6: return dns_typetab[38].name; + case DNS_T_DNAME: return dns_typetab[39].name; + case DNS_T_SINK: return dns_typetab[40].name; + case DNS_T_OPT: return dns_typetab[41].name; + case DNS_T_DS: return dns_typetab[42].name; + case DNS_T_SSHFP: return dns_typetab[43].name; + case DNS_T_IPSECKEY: return dns_typetab[44].name; + case DNS_T_RRSIG: return dns_typetab[45].name; + case DNS_T_NSEC: return dns_typetab[46].name; + case DNS_T_DNSKEY: return dns_typetab[47].name; + case DNS_T_DHCID: return dns_typetab[48].name; + case DNS_T_NSEC3: return dns_typetab[49].name; + case DNS_T_NSEC3PARAMS: return dns_typetab[50].name; + case DNS_T_TALINK: return dns_typetab[51].name; + case DNS_T_SPF: return dns_typetab[52].name; + case DNS_T_UINFO: return dns_typetab[53].name; + case DNS_T_UID: return dns_typetab[54].name; + case DNS_T_GID: return dns_typetab[55].name; + case DNS_T_UNSPEC: return dns_typetab[56].name; + case DNS_T_TSIG: return dns_typetab[57].name; + case DNS_T_IXFR: return dns_typetab[58].name; + case DNS_T_AXFR: return dns_typetab[59].name; + case DNS_T_MAILB: return dns_typetab[60].name; + case DNS_T_MAILA: return dns_typetab[61].name; + case DNS_T_ANY: return dns_typetab[62].name; + case DNS_T_ZXFR: return dns_typetab[63].name; + case DNS_T_DLV: return dns_typetab[64].name; + case DNS_T_MAX: return dns_typetab[65].name; + } + return _dns_format_code(nm,"type",code); +} + +const struct dns_nameval dns_classtab[] = { + {DNS_C_INVALID,"INVALID"}, + {DNS_C_IN,"IN"}, + {DNS_C_CH,"CH"}, + {DNS_C_HS,"HS"}, + {DNS_C_ANY,"ANY"}, + {0,0}}; +const char *dns_classname(enum dns_class code) { + static char nm[20]; + switch(code) { + case DNS_C_INVALID: return dns_classtab[0].name; + case DNS_C_IN: return dns_classtab[1].name; + case DNS_C_CH: return dns_classtab[2].name; + case DNS_C_HS: return dns_classtab[3].name; + case DNS_C_ANY: return dns_classtab[4].name; + } + return _dns_format_code(nm,"class",code); +} + +const struct dns_nameval dns_rcodetab[] = { + {DNS_R_NOERROR,"NOERROR"}, + {DNS_R_FORMERR,"FORMERR"}, + {DNS_R_SERVFAIL,"SERVFAIL"}, + {DNS_R_NXDOMAIN,"NXDOMAIN"}, + {DNS_R_NOTIMPL,"NOTIMPL"}, + {DNS_R_REFUSED,"REFUSED"}, + {DNS_R_YXDOMAIN,"YXDOMAIN"}, + {DNS_R_YXRRSET,"YXRRSET"}, + {DNS_R_NXRRSET,"NXRRSET"}, + {DNS_R_NOTAUTH,"NOTAUTH"}, + {DNS_R_NOTZONE,"NOTZONE"}, + {DNS_R_BADSIG,"BADSIG"}, + {DNS_R_BADKEY,"BADKEY"}, + {DNS_R_BADTIME,"BADTIME"}, + {0,0}}; +const char *dns_rcodename(enum dns_rcode code) { + static char nm[20]; + switch(code) { + case DNS_R_NOERROR: return dns_rcodetab[0].name; + case DNS_R_FORMERR: return dns_rcodetab[1].name; + case DNS_R_SERVFAIL: return dns_rcodetab[2].name; + case DNS_R_NXDOMAIN: return dns_rcodetab[3].name; + case DNS_R_NOTIMPL: return dns_rcodetab[4].name; + case DNS_R_REFUSED: return dns_rcodetab[5].name; + case DNS_R_YXDOMAIN: return dns_rcodetab[6].name; + case DNS_R_YXRRSET: return dns_rcodetab[7].name; + case DNS_R_NXRRSET: return dns_rcodetab[8].name; + case DNS_R_NOTAUTH: return dns_rcodetab[9].name; + case DNS_R_NOTZONE: return dns_rcodetab[10].name; + case DNS_R_BADSIG: return dns_rcodetab[11].name; + case DNS_R_BADKEY: return dns_rcodetab[12].name; + case DNS_R_BADTIME: return dns_rcodetab[13].name; + } + return _dns_format_code(nm,"rcode",code); +} diff --git a/contrib/udns/udns_dn.c b/contrib/udns/udns_dn.c new file mode 100644 index 00000000000..ae3fd177882 --- /dev/null +++ b/contrib/udns/udns_dn.c @@ -0,0 +1,379 @@ +/* udns_dn.c + domain names manipulation routines + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#include +#include "udns.h" + +unsigned dns_dnlen(dnscc_t *dn) { + register dnscc_t *d = dn; + while(*d) + d += 1 + *d; + return (unsigned)(d - dn) + 1; +} + +unsigned dns_dnlabels(register dnscc_t *dn) { + register unsigned l = 0; + while(*dn) + ++l, dn += 1 + *dn; + return l; +} + +unsigned dns_dnequal(register dnscc_t *dn1, register dnscc_t *dn2) { + register unsigned c; + dnscc_t *dn = dn1; + for(;;) { + if ((c = *dn1++) != *dn2++) + return 0; + if (!c) + return (unsigned)(dn1 - dn); + while(c--) { + if (DNS_DNLC(*dn1) != DNS_DNLC(*dn2)) + return 0; + ++dn1; ++dn2; + } + } +} + +unsigned +dns_dntodn(dnscc_t *sdn, dnsc_t *ddn, unsigned ddnsiz) { + unsigned sdnlen = dns_dnlen(sdn); + if (ddnsiz < sdnlen) + return 0; + memcpy(ddn, sdn, sdnlen); + return sdnlen; +} + +int +dns_ptodn(const char *name, unsigned namelen, + dnsc_t *dn, unsigned dnsiz, int *isabs) +{ + dnsc_t *dp; /* current position in dn (len byte first) */ + dnsc_t *const de /* end of dn: last byte that can be filled up */ + = dn + (dnsiz >= DNS_MAXDN ? DNS_MAXDN : dnsiz) - 1; + dnscc_t *np = (dnscc_t *)name; + dnscc_t *ne = np + (namelen ? namelen : strlen((char*)np)); + dnsc_t *llab; /* start of last label (llab[-1] will be length) */ + unsigned c; /* next input character, or length of last label */ + + if (!dnsiz) + return 0; + dp = llab = dn + 1; + + while(np < ne) { + + if (*np == '.') { /* label delimiter */ + c = dp - llab; /* length of the label */ + if (!c) { /* empty label */ + if (np == (dnscc_t *)name && np + 1 == ne) { + /* special case for root dn, aka `.' */ + ++np; + break; + } + return -1; /* zero label */ + } + if (c > DNS_MAXLABEL) + return -1; /* label too long */ + llab[-1] = (dnsc_t)c; /* update len of last label */ + llab = ++dp; /* start new label, llab[-1] will be len of it */ + ++np; + continue; + } + + /* check whenever we may put out one more byte */ + if (dp >= de) /* too long? */ + return dnsiz >= DNS_MAXDN ? -1 : 0; + if (*np != '\\') { /* non-escape, simple case */ + *dp++ = *np++; + continue; + } + /* handle \-style escape */ + /* note that traditionally, domain names (gethostbyname etc) + * used decimal \dd notation, not octal \ooo (RFC1035), so + * we're following this tradition here. + */ + if (++np == ne) + return -1; /* bad escape */ + else if (*np >= '0' && *np <= '9') { /* decimal number */ + /* we allow not only exactly 3 digits as per RFC1035, + * but also 2 or 1, for better usability. */ + c = *np++ - '0'; + if (np < ne && *np >= '0' && *np <= '9') { /* 2digits */ + c = c * 10 + *np++ - '0'; + if (np < ne && *np >= '0' && *np <= '9') { + c = c * 10 + *np++ - '0'; + if (c > 255) + return -1; /* bad escape */ + } + } + } + else + c = *np++; + *dp++ = (dnsc_t)c; /* place next out byte */ + } + + if ((c = dp - llab) > DNS_MAXLABEL) + return -1; /* label too long */ + if ((llab[-1] = (dnsc_t)c) != 0) { + *dp++ = 0; + if (isabs) + *isabs = 0; + } + else if (isabs) + *isabs = 1; + + return dp - dn; +} + +dnscc_t dns_inaddr_arpa_dn[14] = "\07in-addr\04arpa"; + +dnsc_t * +dns_a4todn_(const struct in_addr *addr, dnsc_t *dn, dnsc_t *dne) { + const unsigned char *s = ((const unsigned char *)addr) + 4; + while(s > (const unsigned char *)addr) { + unsigned n = *--s; + dnsc_t *p = dn + 1; + if (n > 99) { + if (p + 2 > dne) return 0; + *p++ = n / 100 + '0'; + *p++ = (n % 100 / 10) + '0'; + *p = n % 10 + '0'; + } + else if (n > 9) { + if (p + 1 > dne) return 0; + *p++ = n / 10 + '0'; + *p = n % 10 + '0'; + } + else { + if (p > dne) return 0; + *p = n + '0'; + } + *dn = p - dn; + dn = p + 1; + } + return dn; +} + +int dns_a4todn(const struct in_addr *addr, dnscc_t *tdn, + dnsc_t *dn, unsigned dnsiz) { + dnsc_t *dne = dn + (dnsiz > DNS_MAXDN ? DNS_MAXDN : dnsiz); + dnsc_t *p; + unsigned l; + p = dns_a4todn_(addr, dn, dne); + if (!p) return 0; + if (!tdn) + tdn = dns_inaddr_arpa_dn; + l = dns_dnlen(tdn); + if (p + l > dne) return dnsiz >= DNS_MAXDN ? -1 : 0; + memcpy(p, tdn, l); + return (p + l) - dn; +} + +int dns_a4ptodn(const struct in_addr *addr, const char *tname, + dnsc_t *dn, unsigned dnsiz) { + dnsc_t *p; + int r; + if (!tname) + return dns_a4todn(addr, NULL, dn, dnsiz); + p = dns_a4todn_(addr, dn, dn + dnsiz); + if (!p) return 0; + r = dns_sptodn(tname, p, dnsiz - (p - dn)); + return r != 0 ? r : dnsiz >= DNS_MAXDN ? -1 : 0; +} + +dnscc_t dns_ip6_arpa_dn[10] = "\03ip6\04arpa"; + +dnsc_t * +dns_a6todn_(const struct in6_addr *addr, dnsc_t *dn, dnsc_t *dne) { + const unsigned char *s = ((const unsigned char *)addr) + 16; + if (dn + 64 > dne) return 0; + while(s > (const unsigned char *)addr) { + unsigned n = *--s & 0x0f; + *dn++ = 1; + *dn++ = n > 9 ? n + 'a' - 10 : n + '0'; + *dn++ = 1; + n = *s >> 4; + *dn++ = n > 9 ? n + 'a' - 10 : n + '0'; + } + return dn; +} + +int dns_a6todn(const struct in6_addr *addr, dnscc_t *tdn, + dnsc_t *dn, unsigned dnsiz) { + dnsc_t *dne = dn + (dnsiz > DNS_MAXDN ? DNS_MAXDN : dnsiz); + dnsc_t *p; + unsigned l; + p = dns_a6todn_(addr, dn, dne); + if (!p) return 0; + if (!tdn) + tdn = dns_ip6_arpa_dn; + l = dns_dnlen(tdn); + if (p + l > dne) return dnsiz >= DNS_MAXDN ? -1 : 0; + memcpy(p, tdn, l); + return (p + l) - dn; +} + +int dns_a6ptodn(const struct in6_addr *addr, const char *tname, + dnsc_t *dn, unsigned dnsiz) { + dnsc_t *p; + int r; + if (!tname) + return dns_a6todn(addr, NULL, dn, dnsiz); + p = dns_a6todn_(addr, dn, dn + dnsiz); + if (!p) return 0; + r = dns_sptodn(tname, p, dnsiz - (p - dn)); + return r != 0 ? r : dnsiz >= DNS_MAXDN ? -1 : 0; +} + +/* return size of buffer required to convert the dn into asciiz string. + * Keep in sync with dns_dntop() below. + */ +unsigned dns_dntop_size(dnscc_t *dn) { + unsigned size = 0; /* the size reqd */ + dnscc_t *le; /* label end */ + + while(*dn) { + /* *dn is the length of the next label, non-zero */ + if (size) + ++size; /* for the dot */ + le = dn + *dn + 1; + ++dn; + do { + switch(*dn) { + case '.': + case '\\': + /* Special modifiers in zone files. */ + case '"': + case ';': + case '@': + case '$': + size += 2; + break; + default: + if (*dn <= 0x20 || *dn >= 0x7f) + /* \ddd decimal notation */ + size += 4; + else + size += 1; + } + } while(++dn < le); + } + size += 1; /* zero byte at the end - string terminator */ + return size > DNS_MAXNAME ? 0 : size; +} + +/* Convert the dn into asciiz string. + * Keep in sync with dns_dntop_size() above. + */ +int dns_dntop(dnscc_t *dn, char *name, unsigned namesiz) { + char *np = name; /* current name ptr */ + char *const ne = name + namesiz; /* end of name */ + dnscc_t *le; /* label end */ + + while(*dn) { + /* *dn is the length of the next label, non-zero */ + if (np != name) { + if (np >= ne) goto toolong; + *np++ = '.'; + } + le = dn + *dn + 1; + ++dn; + do { + switch(*dn) { + case '.': + case '\\': + /* Special modifiers in zone files. */ + case '"': + case ';': + case '@': + case '$': + if (np + 2 > ne) goto toolong; + *np++ = '\\'; + *np++ = *dn; + break; + default: + if (*dn <= 0x20 || *dn >= 0x7f) { + /* \ddd decimal notation */ + if (np + 4 >= ne) goto toolong; + *np++ = '\\'; + *np++ = '0' + (*dn / 100); + *np++ = '0' + ((*dn % 100) / 10); + *np++ = '0' + (*dn % 10); + } + else { + if (np >= ne) goto toolong; + *np++ = *dn; + } + } + } while(++dn < le); + } + if (np >= ne) goto toolong; + *np++ = '\0'; + return np - name; +toolong: + return namesiz >= DNS_MAXNAME ? -1 : 0; +} + +#ifdef TEST +#include +#include + +int main(int argc, char **argv) { + int i; + int sz; + dnsc_t dn[DNS_MAXDN+10]; + dnsc_t *dl, *dp; + int isabs; + + sz = (argc > 1) ? atoi(argv[1]) : 0; + + for(i = 2; i < argc; ++i) { + int r = dns_ptodn(argv[i], 0, dn, sz, &isabs); + printf("%s: ", argv[i]); + if (r < 0) printf("error\n"); + else if (!r) printf("buffer too small\n"); + else { + printf("len=%d dnlen=%d size=%d name:", + r, dns_dnlen(dn), dns_dntop_size(dn)); + dl = dn; + while(*dl) { + printf(" %d=", *dl); + dp = dl + 1; + dl = dp + *dl; + while(dp < dl) { + if (*dp <= ' ' || *dp >= 0x7f) + printf("\\%03d", *dp); + else if (*dp == '.' || *dp == '\\') + printf("\\%c", *dp); + else + putchar(*dp); + ++dp; + } + } + if (isabs) putchar('.'); + putchar('\n'); + } + } + return 0; +} + +#endif /* TEST */ diff --git a/contrib/udns/udns_dntosp.c b/contrib/udns/udns_dntosp.c new file mode 100644 index 00000000000..823fde211e5 --- /dev/null +++ b/contrib/udns/udns_dntosp.c @@ -0,0 +1,30 @@ +/* udns_dntosp.c + dns_dntosp() = convert DN to asciiz string using static buffer + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#include "udns.h" + +static char name[DNS_MAXNAME]; + +const char *dns_dntosp(dnscc_t *dn) { + return dns_dntop(dn, name, sizeof(name)) > 0 ? name : 0; +} diff --git a/contrib/udns/udns_init.c b/contrib/udns/udns_init.c new file mode 100644 index 00000000000..493af58917f --- /dev/null +++ b/contrib/udns/udns_init.c @@ -0,0 +1,231 @@ +/* udns_init.c + resolver initialisation stuff + + Copyright (C) 2006 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#ifdef HAVE_CONFIG_H +# include "config.h" +#endif +#ifdef WINDOWS +# include /* includes */ +# include /* for dns server addresses etc */ +#else +# include +# include +# include +#endif /* !WINDOWS */ + +#include +#include +#include "udns.h" + +#define ISSPACE(x) (x == ' ' || x == '\t' || x == '\r' || x == '\n') + +static const char space[] = " \t\r\n"; + +static void dns_set_serv_internal(struct dns_ctx *ctx, char *serv) { + dns_add_serv(ctx, NULL); + for(serv = strtok(serv, space); serv; serv = strtok(NULL, space)) + dns_add_serv(ctx, serv); +} + +static void dns_set_srch_internal(struct dns_ctx *ctx, char *srch) { + dns_add_srch(ctx, NULL); + for(srch = strtok(srch, space); srch; srch = strtok(NULL, space)) + dns_add_srch(ctx, srch); +} + +#ifdef WINDOWS + +#ifndef NO_IPHLPAPI +/* Apparently, some systems does not have proper headers for IPHLPAIP to work. + * The best is to upgrade headers, but here's another, ugly workaround for + * this: compile with -DNO_IPHLPAPI. + */ + +typedef DWORD (WINAPI *GetAdaptersAddressesFunc)( + ULONG Family, DWORD Flags, PVOID Reserved, + PIP_ADAPTER_ADDRESSES pAdapterAddresses, + PULONG pOutBufLen); + +static int dns_initns_iphlpapi(struct dns_ctx *ctx) { + HANDLE h_iphlpapi; + GetAdaptersAddressesFunc pfnGetAdAddrs; + PIP_ADAPTER_ADDRESSES pAddr, pAddrBuf; + PIP_ADAPTER_DNS_SERVER_ADDRESS pDnsAddr; + ULONG ulOutBufLen; + DWORD dwRetVal; + int ret = -1; + + h_iphlpapi = LoadLibrary("iphlpapi.dll"); + if (!h_iphlpapi) + return -1; + pfnGetAdAddrs = (GetAdaptersAddressesFunc) + GetProcAddress(h_iphlpapi, "GetAdaptersAddresses"); + if (!pfnGetAdAddrs) goto freelib; + ulOutBufLen = 0; + dwRetVal = pfnGetAdAddrs(AF_UNSPEC, 0, NULL, NULL, &ulOutBufLen); + if (dwRetVal != ERROR_BUFFER_OVERFLOW) goto freelib; + pAddrBuf = malloc(ulOutBufLen); + if (!pAddrBuf) goto freelib; + dwRetVal = pfnGetAdAddrs(AF_UNSPEC, 0, NULL, pAddrBuf, &ulOutBufLen); + if (dwRetVal != ERROR_SUCCESS) goto freemem; + for (pAddr = pAddrBuf; pAddr; pAddr = pAddr->Next) + for (pDnsAddr = pAddr->FirstDnsServerAddress; + pDnsAddr; + pDnsAddr = pDnsAddr->Next) + dns_add_serv_s(ctx, pDnsAddr->Address.lpSockaddr); + ret = 0; +freemem: + free(pAddrBuf); +freelib: + FreeLibrary(h_iphlpapi); + return ret; +} + +#else /* NO_IPHLPAPI */ + +#define dns_initns_iphlpapi(ctx) (-1) + +#endif /* NO_IPHLPAPI */ + +static int dns_initns_registry(struct dns_ctx *ctx) { + LONG res; + HKEY hk; + DWORD type = REG_EXPAND_SZ | REG_SZ; + DWORD len; + char valBuf[1024]; + +#define REGKEY_WINNT "SYSTEM\\CurrentControlSet\\Services\\Tcpip\\Parameters" +#define REGKEY_WIN9x "SYSTEM\\CurrentControlSet\\Services\\VxD\\MSTCP" + res = RegOpenKeyEx(HKEY_LOCAL_MACHINE, REGKEY_WINNT, 0, KEY_QUERY_VALUE, &hk); + if (res != ERROR_SUCCESS) + res = RegOpenKeyEx(HKEY_LOCAL_MACHINE, REGKEY_WIN9x, + 0, KEY_QUERY_VALUE, &hk); + if (res != ERROR_SUCCESS) + return -1; + len = sizeof(valBuf) - 1; + res = RegQueryValueEx(hk, "NameServer", NULL, &type, (BYTE*)valBuf, &len); + if (res != ERROR_SUCCESS || !len || !valBuf[0]) { + len = sizeof(valBuf) - 1; + res = RegQueryValueEx(hk, "DhcpNameServer", NULL, &type, + (BYTE*)valBuf, &len); + } + RegCloseKey(hk); + if (res != ERROR_SUCCESS || !len || !valBuf[0]) + return -1; + valBuf[len] = '\0'; + /* nameservers are stored as a whitespace-seperate list: + * "192.168.1.1 123.21.32.12" */ + dns_set_serv_internal(ctx, valBuf); + return 0; +} + +#else /* !WINDOWS */ + +static int dns_init_resolvconf(struct dns_ctx *ctx) { + char *v; + char buf[2049]; /* this buffer is used to hold /etc/resolv.conf */ + int has_srch = 0; + + /* read resolv.conf... */ + { int fd = open("/etc/resolv.conf", O_RDONLY); + if (fd >= 0) { + int l = read(fd, buf, sizeof(buf) - 1); + close(fd); + buf[l < 0 ? 0 : l] = '\0'; + } + else + buf[0] = '\0'; + } + if (buf[0]) { /* ...and parse it */ + char *line, *nextline; + line = buf; + do { + nextline = strchr(line, '\n'); + if (nextline) *nextline++ = '\0'; + v = line; + while(*v && !ISSPACE(*v)) ++v; + if (!*v) continue; + *v++ = '\0'; + while(ISSPACE(*v)) ++v; + if (!*v) continue; + if (strcmp(line, "domain") == 0) { + dns_set_srch_internal(ctx, strtok(v, space)); + has_srch = 1; + } + else if (strcmp(line, "search") == 0) { + dns_set_srch_internal(ctx, v); + has_srch = 1; + } + else if (strcmp(line, "nameserver") == 0) + dns_add_serv(ctx, strtok(v, space)); + else if (strcmp(line, "options") == 0) + dns_set_opts(ctx, v); + } while((line = nextline) != NULL); + } + + buf[sizeof(buf)-1] = '\0'; + + /* get list of nameservers from env. vars. */ + if ((v = getenv("NSCACHEIP")) != NULL || + (v = getenv("NAMESERVERS")) != NULL) { + strncpy(buf, v, sizeof(buf) - 1); + dns_set_serv_internal(ctx, buf); + } + /* if $LOCALDOMAIN is set, use it for search list */ + if ((v = getenv("LOCALDOMAIN")) != NULL) { + strncpy(buf, v, sizeof(buf) - 1); + dns_set_srch_internal(ctx, buf); + has_srch = 1; + } + if ((v = getenv("RES_OPTIONS")) != NULL) + dns_set_opts(ctx, v); + + /* if still no search list, use local domain name */ + if (has_srch && + gethostname(buf, sizeof(buf) - 1) == 0 && + (v = strchr(buf, '.')) != NULL && + *++v != '\0') + dns_add_srch(ctx, v); + + return 0; +} + +#endif /* !WINDOWS */ + +int dns_init(struct dns_ctx *ctx, int do_open) { + if (!ctx) + ctx = &dns_defctx; + dns_reset(ctx); + +#ifdef WINDOWS + if (dns_initns_iphlpapi(ctx) != 0) + dns_initns_registry(ctx); + /*XXX WINDOWS: probably good to get default domain and search list too... + * And options. Something is in registry. */ + /*XXX WINDOWS: maybe environment variables are also useful? */ +#else + dns_init_resolvconf(ctx); +#endif + + return do_open ? dns_open(ctx) : 0; +} diff --git a/contrib/udns/udns_jran.c b/contrib/udns/udns_jran.c new file mode 100644 index 00000000000..19f9d02bb14 --- /dev/null +++ b/contrib/udns/udns_jran.c @@ -0,0 +1,52 @@ +/* udns_jran.c: small non-cryptographic random number generator + * taken from http://burtleburtle.net/bob/rand/smallprng.html + * by Bob Jenkins, Public domain. + */ + +#include "udns.h" + +#define rot32(x,k) (((x) << (k)) | ((x) >> (32-(k)))) +#define rot64(x,k) (((x) << (k)) | ((x) >> (64-(k)))) +#define tr32(x) ((x)&0xffffffffu) + +unsigned udns_jranval(struct udns_jranctx *x) { + /* This routine can be made to work with either 32 or 64bit words - + * if JRAN_32_64 is defined when compiling the file. + * We use if() instead of #if since there's no good + * portable way to check sizeof() in preprocessor without + * introducing some ugly configure-time checks. + * Most compilers will optimize the wrong branches away anyway. + * By default it assumes 32bit integers + */ +#ifdef JRAN_32_64 + if (sizeof(unsigned) == 4) { +#endif + unsigned e = tr32(x->a - rot32(x->b, 27)); + x->a = tr32(x->b ^ rot32(x->c, 17)); + x->b = tr32(x->c + x->d); + x->c = tr32(x->d + e); + x->d = tr32(e + x->a); +#ifdef JRAN_32_64 + } + else if (sizeof(unsigned) == 8) { /* assuming it's 64bits */ + unsigned e = x->a - rot64(x->b, 7); + x->a = x->b ^ rot64(x->c, 13); + x->b = x->c + rot64(x->d, 37); + x->c = x->d + e; + x->d = e + x->a; + } + else { + unsigned e = 0; + x->d = 1/e; /* bail */ + } +#endif + return x->d; +} + +void udns_jraninit(struct udns_jranctx *x, unsigned seed) { + unsigned i; + x->a = 0xf1ea5eed; + x->b = x->c = x->d = seed; + for (i = 0; i < 20; ++i) + (void)udns_jranval(x); +} diff --git a/contrib/udns/udns_misc.c b/contrib/udns/udns_misc.c new file mode 100644 index 00000000000..c162e70e77b --- /dev/null +++ b/contrib/udns/udns_misc.c @@ -0,0 +1,67 @@ +/* udns_misc.c + miscellaneous routines + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#include "udns.h" + +int dns_findname(const struct dns_nameval *nv, const char *name) { + register const char *a, *b; + for(; nv->name; ++nv) + for(a = name, b = nv->name; ; ++a, ++b) + if (DNS_DNUC(*a) != *b) break; + else if (!*a) return nv->val; + return -1; +} + +const char *_dns_format_code(char *buf, const char *prefix, int code) { + char *bp = buf; + unsigned c, n; + do *bp++ = DNS_DNUC(*prefix); + while(*++prefix); + *bp++ = '#'; + if (code < 0) code = -code, *bp++ = '-'; + n = 0; c = code; + do ++n; + while((c /= 10)); + c = code; + bp[n--] = '\0'; + do bp[n--] = c % 10 + '0'; + while((c /= 10)); + return buf; +} + +const char *dns_strerror(int err) { + if (err >= 0) return "successeful completion"; + switch(err) { + case DNS_E_TEMPFAIL: return "temporary failure in name resolution"; + case DNS_E_PROTOCOL: return "protocol error"; + case DNS_E_NXDOMAIN: return "domain name does not exist"; + case DNS_E_NODATA: return "valid domain but no data of requested type"; + case DNS_E_NOMEM: return "out of memory"; + case DNS_E_BADQUERY: return "malformed query"; + default: return "unknown error"; + } +} + +const char *dns_version(void) { + return UDNS_VERSION; +} diff --git a/contrib/udns/udns_parse.c b/contrib/udns/udns_parse.c new file mode 100644 index 00000000000..8924b1562d2 --- /dev/null +++ b/contrib/udns/udns_parse.c @@ -0,0 +1,169 @@ +/* udns_parse.c + raw DNS packet parsing routines + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#include +#include +#include "udns.h" + +dnscc_t *dns_skipdn(dnscc_t *cur, dnscc_t *end) { + unsigned c; + for(;;) { + if (cur >= end) + return NULL; + c = *cur++; + if (!c) + return cur; + if (c & 192) /* jump */ + return cur + 1 >= end ? NULL : cur + 1; + cur += c; + } +} + +int +dns_getdn(dnscc_t *pkt, dnscc_t **cur, dnscc_t *end, + register dnsc_t *dn, unsigned dnsiz) { + unsigned c; + dnscc_t *pp = *cur; /* current packet pointer */ + dnsc_t *dp = dn; /* current dn pointer */ + dnsc_t *const de /* end of the DN dest */ + = dn + (dnsiz < DNS_MAXDN ? dnsiz : DNS_MAXDN); + dnscc_t *jump = NULL; /* ptr after first jump if any */ + unsigned loop = 100; /* jump loop counter */ + + for(;;) { /* loop by labels */ + if (pp >= end) /* reached end of packet? */ + return -1; + c = *pp++; /* length of the label */ + if (!c) { /* empty label: terminate */ + if (dn >= de) /* can't fit terminator */ + goto noroom; + *dp++ = 0; + /* return next pos: either after the first jump or current */ + *cur = jump ? jump : pp; + return dp - dn; + } + if (c & 192) { /* jump */ + if (pp >= end) /* eop instead of jump pos */ + return -1; + if (!jump) jump = pp + 1; /* remember first jump */ + else if (!--loop) return -1; /* too many jumps */ + c = ((c & ~192) << 8) | *pp; /* new pos */ + if (c < DNS_HSIZE) /* don't allow jump into the header */ + return -1; + pp = pkt + c; + continue; + } + if (c > DNS_MAXLABEL) /* too long label? */ + return -1; + if (pp + c > end) /* label does not fit in packet? */ + return -1; + if (dp + c + 1 > de) /* if enouth room for the label */ + goto noroom; + *dp++ = c; /* label length */ + memcpy(dp, pp, c); /* and the label itself */ + dp += c; + pp += c; /* advance to the next label */ + } +noroom: + return dnsiz < DNS_MAXDN ? 0 : -1; +} + +void dns_rewind(struct dns_parse *p, dnscc_t *qdn) { + p->dnsp_qdn = qdn; + p->dnsp_cur = p->dnsp_ans; + p->dnsp_rrl = dns_numan(p->dnsp_pkt); + p->dnsp_ttl = 0xffffffffu; + p->dnsp_nrr = 0; +} + +void +dns_initparse(struct dns_parse *p, dnscc_t *qdn, + dnscc_t *pkt, dnscc_t *cur, dnscc_t *end) { + p->dnsp_pkt = pkt; + p->dnsp_end = end; + p->dnsp_rrl = dns_numan(pkt); + p->dnsp_qdn = qdn; + assert(cur + 4 <= end); + if ((p->dnsp_qtyp = dns_get16(cur+0)) == DNS_T_ANY) p->dnsp_qtyp = 0; + if ((p->dnsp_qcls = dns_get16(cur+2)) == DNS_C_ANY) p->dnsp_qcls = 0; + p->dnsp_cur = p->dnsp_ans = cur + 4; + p->dnsp_ttl = 0xffffffffu; + p->dnsp_nrr = 0; +} + +int dns_nextrr(struct dns_parse *p, struct dns_rr *rr) { + dnscc_t *cur = p->dnsp_cur; + while(p->dnsp_rrl > 0) { + --p->dnsp_rrl; + if (dns_getdn(p->dnsp_pkt, &cur, p->dnsp_end, + rr->dnsrr_dn, sizeof(rr->dnsrr_dn)) <= 0) + return -1; + if (cur + 10 > p->dnsp_end) + return -1; + rr->dnsrr_typ = dns_get16(cur); + rr->dnsrr_cls = dns_get16(cur+2); + rr->dnsrr_ttl = dns_get32(cur+4); + rr->dnsrr_dsz = dns_get16(cur+8); + rr->dnsrr_dptr = cur = cur + 10; + rr->dnsrr_dend = cur = cur + rr->dnsrr_dsz; + if (cur > p->dnsp_end) + return -1; + if (p->dnsp_qdn && !dns_dnequal(p->dnsp_qdn, rr->dnsrr_dn)) + continue; + if ((!p->dnsp_qcls || p->dnsp_qcls == rr->dnsrr_cls) && + (!p->dnsp_qtyp || p->dnsp_qtyp == rr->dnsrr_typ)) { + p->dnsp_cur = cur; + ++p->dnsp_nrr; + if (p->dnsp_ttl > rr->dnsrr_ttl) p->dnsp_ttl = rr->dnsrr_ttl; + return 1; + } + if (p->dnsp_qdn && rr->dnsrr_typ == DNS_T_CNAME && !p->dnsp_nrr) { + if (dns_getdn(p->dnsp_pkt, &rr->dnsrr_dptr, p->dnsp_end, + p->dnsp_dnbuf, sizeof(p->dnsp_dnbuf)) <= 0 || + rr->dnsrr_dptr != rr->dnsrr_dend) + return -1; + p->dnsp_qdn = p->dnsp_dnbuf; + if (p->dnsp_ttl > rr->dnsrr_ttl) p->dnsp_ttl = rr->dnsrr_ttl; + } + } + p->dnsp_cur = cur; + return 0; +} + +int dns_stdrr_size(const struct dns_parse *p) { + return + dns_dntop_size(p->dnsp_qdn) + + (p->dnsp_qdn == dns_payload(p->dnsp_pkt) ? 0 : + dns_dntop_size(dns_payload(p->dnsp_pkt))); +} + +void *dns_stdrr_finish(struct dns_rr_null *ret, char *cp, + const struct dns_parse *p) { + cp += dns_dntop(p->dnsp_qdn, (ret->dnsn_cname = cp), DNS_MAXNAME); + if (p->dnsp_qdn == dns_payload(p->dnsp_pkt)) + ret->dnsn_qname = ret->dnsn_cname; + else + dns_dntop(dns_payload(p->dnsp_pkt), (ret->dnsn_qname = cp), DNS_MAXNAME); + ret->dnsn_ttl = p->dnsp_ttl; + return ret; +} diff --git a/contrib/udns/udns_resolver.c b/contrib/udns/udns_resolver.c new file mode 100644 index 00000000000..b8f899a2460 --- /dev/null +++ b/contrib/udns/udns_resolver.c @@ -0,0 +1,1323 @@ +/* udns_resolver.c + resolver stuff (main module) + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#ifdef HAVE_CONFIG_H +# include "config.h" +#endif +#ifdef WINDOWS +# include /* includes */ +# include /* needed for struct in6_addr */ +#else +# include +# include +# include +# include +# include +# include +# ifdef HAVE_POLL +# include +# else +# ifdef HAVE_SYS_SELECT_H +# include +# endif +# endif +# ifdef HAVE_TIMES +# include +# endif +# define closesocket(sock) close(sock) +#endif /* !WINDOWS */ + +#include +#include +#include +#include +#include +#include +#include "udns.h" + +#ifndef EAFNOSUPPORT +# define EAFNOSUPPORT EINVAL +#endif +#ifndef MSG_DONTWAIT +# define MSG_DONTWAIT 0 +#endif + +struct dns_qlist { + struct dns_query *head, *tail; +}; + +struct dns_query { + struct dns_query *dnsq_next; /* double-linked list */ + struct dns_query *dnsq_prev; + unsigned dnsq_origdnl0; /* original query DN len w/o last 0 */ + unsigned dnsq_flags; /* control flags for this query */ + unsigned dnsq_servi; /* index of next server to try */ + unsigned dnsq_servwait; /* bitmask: servers left to wait */ + unsigned dnsq_servskip; /* bitmask: servers to skip */ + unsigned dnsq_servnEDNS0; /* bitmask: servers refusing EDNS0 */ + unsigned dnsq_try; /* number of tries made so far */ + dnscc_t *dnsq_nxtsrch; /* next search pointer @dnsc_srchbuf */ + time_t dnsq_deadline; /* when current try will expire */ + dns_parse_fn *dnsq_parse; /* parse: raw => application */ + dns_query_fn *dnsq_cbck; /* the callback to call when done */ + void *dnsq_cbdata; /* user data for the callback */ +#ifndef NDEBUG + struct dns_ctx *dnsq_ctx; /* the resolver context */ +#endif + /* char fields at the end to avoid padding */ + dnsc_t dnsq_id[2]; /* query ID */ + dnsc_t dnsq_typcls[4]; /* requested RR type+class */ + dnsc_t dnsq_dn[DNS_MAXDN+DNS_DNPAD]; /* the query DN +alignment */ +}; + +/* working with dns_query lists */ + +static __inline void qlist_init(struct dns_qlist *list) { + list->head = list->tail = NULL; +} + +static __inline void qlist_remove(struct dns_qlist *list, struct dns_query *q) { + if (q->dnsq_prev) q->dnsq_prev->dnsq_next = q->dnsq_next; + else list->head = q->dnsq_next; + if (q->dnsq_next) q->dnsq_next->dnsq_prev = q->dnsq_prev; + else list->tail = q->dnsq_prev; +} + +static __inline void +qlist_add_head(struct dns_qlist *list, struct dns_query *q) { + q->dnsq_next = list->head; + if (list->head) list->head->dnsq_prev = q; + else list->tail = q; + list->head = q; + q->dnsq_prev = NULL; +} + +static __inline void +qlist_insert_after(struct dns_qlist *list, + struct dns_query *q, struct dns_query *prev) { + if ((q->dnsq_prev = prev) != NULL) { + if ((q->dnsq_next = prev->dnsq_next) != NULL) + q->dnsq_next->dnsq_prev = q; + else + list->tail = q; + prev->dnsq_next = q; + } + else + qlist_add_head(list, q); +} + +union sockaddr_ns { + struct sockaddr sa; + struct sockaddr_in sin; +#ifdef HAVE_IPv6 + struct sockaddr_in6 sin6; +#endif +}; + +#define sin_eq(a,b) \ + ((a).sin_port == (b).sin_port && \ + (a).sin_addr.s_addr == (b).sin_addr.s_addr) +#define sin6_eq(a,b) \ + ((a).sin6_port == (b).sin6_port && \ + memcmp(&(a).sin6_addr, &(b).sin6_addr, sizeof(struct in6_addr)) == 0) + +struct dns_ctx { /* resolver context */ + /* settings */ + unsigned dnsc_flags; /* various flags */ + unsigned dnsc_timeout; /* timeout (base value) for queries */ + unsigned dnsc_ntries; /* number of retries */ + unsigned dnsc_ndots; /* ndots to assume absolute name */ + unsigned dnsc_port; /* default port (DNS_PORT) */ + unsigned dnsc_udpbuf; /* size of UDP buffer */ + /* array of nameserver addresses */ + union sockaddr_ns dnsc_serv[DNS_MAXSERV]; + unsigned dnsc_nserv; /* number of nameservers */ + unsigned dnsc_salen; /* length of socket addresses */ + dnsc_t dnsc_srchbuf[1024]; /* buffer for searchlist */ + dnsc_t *dnsc_srchend; /* current end of srchbuf */ + + dns_utm_fn *dnsc_utmfn; /* register/cancel timer events */ + void *dnsc_utmctx; /* user timer context for utmfn() */ + time_t dnsc_utmexp; /* when user timer expires */ + + dns_dbgfn *dnsc_udbgfn; /* debugging function */ + + /* dynamic data */ + struct udns_jranctx dnsc_jran; /* random number generator state */ + unsigned dnsc_nextid; /* next queue ID to use if !0 */ + int dnsc_udpsock; /* UDP socket */ + struct dns_qlist dnsc_qactive; /* active list sorted by deadline */ + int dnsc_nactive; /* number entries in dnsc_qactive */ + dnsc_t *dnsc_pbuf; /* packet buffer (udpbuf size) */ + int dnsc_qstatus; /* last query status value */ +}; + +static const struct { + const char *name; + enum dns_opt opt; + unsigned offset; + unsigned min, max; +} dns_opts[] = { +#define opt(name,opt,field,min,max) \ + {name,opt,offsetof(struct dns_ctx,field),min,max} + opt("retrans", DNS_OPT_TIMEOUT, dnsc_timeout, 1,300), + opt("timeout", DNS_OPT_TIMEOUT, dnsc_timeout, 1,300), + opt("retry", DNS_OPT_NTRIES, dnsc_ntries, 1,50), + opt("attempts", DNS_OPT_NTRIES, dnsc_ntries, 1,50), + opt("ndots", DNS_OPT_NDOTS, dnsc_ndots, 0,1000), + opt("port", DNS_OPT_PORT, dnsc_port, 1,0xffff), + opt("udpbuf", DNS_OPT_UDPSIZE, dnsc_udpbuf, DNS_MAXPACKET,65536), +#undef opt +}; +#define dns_ctxopt(ctx,idx) (*((unsigned*)(((char*)ctx)+dns_opts[idx].offset))) + +#define ISSPACE(x) (x == ' ' || x == '\t' || x == '\r' || x == '\n') + +struct dns_ctx dns_defctx; + +#define SETCTX(ctx) if (!ctx) ctx = &dns_defctx +#define SETCTXINITED(ctx) SETCTX(ctx); assert(CTXINITED(ctx)) +#define CTXINITED(ctx) (ctx->dnsc_flags & DNS_INITED) +#define SETCTXFRESH(ctx) SETCTXINITED(ctx); assert(!CTXOPEN(ctx)) +#define SETCTXINACTIVE(ctx) \ + SETCTXINITED(ctx); assert(!ctx->dnsc_nactive) +#define SETCTXOPEN(ctx) SETCTXINITED(ctx); assert(CTXOPEN(ctx)) +#define CTXOPEN(ctx) (ctx->dnsc_udpsock >= 0) + +#if defined(NDEBUG) || !defined(DEBUG) +#define dns_assert_ctx(ctx) +#else +static void dns_assert_ctx(const struct dns_ctx *ctx) { + int nactive = 0; + const struct dns_query *q; + for(q = ctx->dnsc_qactive.head; q; q = q->dnsq_next) { + assert(q->dnsq_ctx == ctx); + assert(q == (q->dnsq_next ? + q->dnsq_next->dnsq_prev : ctx->dnsc_qactive.tail)); + assert(q == (q->dnsq_prev ? + q->dnsq_prev->dnsq_next : ctx->dnsc_qactive.head)); + ++nactive; + } + assert(nactive == ctx->dnsc_nactive); +} +#endif + +enum { + DNS_INTERNAL = 0xffff, /* internal flags mask */ + DNS_INITED = 0x0001, /* the context is initialized */ + DNS_ASIS_DONE = 0x0002, /* search: skip the last as-is query */ + DNS_SEEN_NODATA = 0x0004, /* search: NODATA has been received */ +}; + +int dns_add_serv(struct dns_ctx *ctx, const char *serv) { + union sockaddr_ns *sns; + SETCTXFRESH(ctx); + if (!serv) + return (ctx->dnsc_nserv = 0); + if (ctx->dnsc_nserv >= DNS_MAXSERV) + return errno = ENFILE, -1; + sns = &ctx->dnsc_serv[ctx->dnsc_nserv]; + memset(sns, 0, sizeof(*sns)); + if (dns_pton(AF_INET, serv, &sns->sin.sin_addr) > 0) { + sns->sin.sin_family = AF_INET; + return ++ctx->dnsc_nserv; + } +#ifdef HAVE_IPv6 + if (dns_pton(AF_INET6, serv, &sns->sin6.sin6_addr) > 0) { + sns->sin6.sin6_family = AF_INET6; + return ++ctx->dnsc_nserv; + } +#endif + errno = EINVAL; + return -1; +} + +int dns_add_serv_s(struct dns_ctx *ctx, const struct sockaddr *sa) { + SETCTXFRESH(ctx); + if (!sa) + return (ctx->dnsc_nserv = 0); + if (ctx->dnsc_nserv >= DNS_MAXSERV) + return errno = ENFILE, -1; +#ifdef HAVE_IPv6 + else if (sa->sa_family == AF_INET6) + ctx->dnsc_serv[ctx->dnsc_nserv].sin6 = *(struct sockaddr_in6*)sa; +#endif + else if (sa->sa_family == AF_INET) + ctx->dnsc_serv[ctx->dnsc_nserv].sin = *(struct sockaddr_in*)sa; + else + return errno = EAFNOSUPPORT, -1; + return ++ctx->dnsc_nserv; +} + +int dns_set_opts(struct dns_ctx *ctx, const char *opts) { + unsigned i, v; + int err = 0; + SETCTXINACTIVE(ctx); + for(;;) { + while(ISSPACE(*opts)) ++opts; + if (!*opts) break; + for(i = 0; ; ++i) { + if (i >= sizeof(dns_opts)/sizeof(dns_opts[0])) { ++err; break; } + v = strlen(dns_opts[i].name); + if (strncmp(dns_opts[i].name, opts, v) != 0 || + (opts[v] != ':' && opts[v] != '=')) + continue; + opts += v + 1; + v = 0; + if (*opts < '0' || *opts > '9') { ++err; break; } + do v = v * 10 + (*opts++ - '0'); + while (*opts >= '0' && *opts <= '9'); + if (v < dns_opts[i].min) v = dns_opts[i].min; + if (v > dns_opts[i].max) v = dns_opts[i].max; + dns_ctxopt(ctx, i) = v; + break; + } + while(*opts && !ISSPACE(*opts)) ++opts; + } + return err; +} + +int dns_set_opt(struct dns_ctx *ctx, enum dns_opt opt, int val) { + int prev; + unsigned i; + SETCTXINACTIVE(ctx); + for(i = 0; i < sizeof(dns_opts)/sizeof(dns_opts[0]); ++i) { + if (dns_opts[i].opt != opt) continue; + prev = dns_ctxopt(ctx, i); + if (val >= 0) { + unsigned v = val; + if (v < dns_opts[i].min || v > dns_opts[i].max) { + errno = EINVAL; + return -1; + } + dns_ctxopt(ctx, i) = v; + } + return prev; + } + if (opt == DNS_OPT_FLAGS) { + prev = ctx->dnsc_flags & ~DNS_INTERNAL; + if (val >= 0) + ctx->dnsc_flags = + (ctx->dnsc_flags & DNS_INTERNAL) | (val & ~DNS_INTERNAL); + return prev; + } + errno = ENOSYS; + return -1; +} + +int dns_add_srch(struct dns_ctx *ctx, const char *srch) { + int dnl; + SETCTXINACTIVE(ctx); + if (!srch) { + memset(ctx->dnsc_srchbuf, 0, sizeof(ctx->dnsc_srchbuf)); + ctx->dnsc_srchend = ctx->dnsc_srchbuf; + return 0; + } + dnl = + sizeof(ctx->dnsc_srchbuf) - (ctx->dnsc_srchend - ctx->dnsc_srchbuf) - 1; + dnl = dns_sptodn(srch, ctx->dnsc_srchend, dnl); + if (dnl > 0) + ctx->dnsc_srchend += dnl; + ctx->dnsc_srchend[0] = '\0'; /* we ensure the list is always ends at . */ + if (dnl > 0) + return 0; + errno = EINVAL; + return -1; +} + +static void dns_drop_utm(struct dns_ctx *ctx) { + if (ctx->dnsc_utmfn) + ctx->dnsc_utmfn(NULL, -1, ctx->dnsc_utmctx); + ctx->dnsc_utmctx = NULL; + ctx->dnsc_utmexp = -1; +} + +static void +_dns_request_utm(struct dns_ctx *ctx, time_t now) { + struct dns_query *q; + time_t deadline; + int timeout; + q = ctx->dnsc_qactive.head; + if (!q) + deadline = -1, timeout = -1; + else if (!now || q->dnsq_deadline <= now) + deadline = 0, timeout = 0; + else + deadline = q->dnsq_deadline, timeout = (int)(deadline - now); + if (ctx->dnsc_utmexp == deadline) + return; + ctx->dnsc_utmfn(ctx, timeout, ctx->dnsc_utmctx); + ctx->dnsc_utmexp = deadline; +} + +static __inline void +dns_request_utm(struct dns_ctx *ctx, time_t now) { + if (ctx->dnsc_utmfn) + _dns_request_utm(ctx, now); +} + +void dns_set_dbgfn(struct dns_ctx *ctx, dns_dbgfn *dbgfn) { + SETCTXINITED(ctx); + ctx->dnsc_udbgfn = dbgfn; +} + +void +dns_set_tmcbck(struct dns_ctx *ctx, dns_utm_fn *fn, void *data) { + SETCTXINITED(ctx); + dns_drop_utm(ctx); + ctx->dnsc_utmfn = fn; + ctx->dnsc_utmctx = data; + if (CTXOPEN(ctx)) + dns_request_utm(ctx, 0); +} + +static unsigned dns_nonrandom_32(void) { +#ifdef WINDOWS + FILETIME ft; + GetSystemTimeAsFileTime(&ft); + return ft.dwLowDateTime; +#else + struct timeval tv; + gettimeofday(&tv, NULL); + return tv.tv_usec; +#endif +} + +/* This is historic deprecated API */ +UDNS_API unsigned dns_random16(void); +unsigned dns_random16(void) { + unsigned x = dns_nonrandom_32(); + return (x ^ (x >> 16)) & 0xffff; +} + +static void dns_init_rng(struct dns_ctx *ctx) { + udns_jraninit(&ctx->dnsc_jran, dns_nonrandom_32()); + ctx->dnsc_nextid = 0; +} + +void dns_close(struct dns_ctx *ctx) { + struct dns_query *q, *p; + SETCTX(ctx); + if (CTXINITED(ctx)) { + if (ctx->dnsc_udpsock >= 0) + closesocket(ctx->dnsc_udpsock); + ctx->dnsc_udpsock = -1; + if (ctx->dnsc_pbuf) + free(ctx->dnsc_pbuf); + ctx->dnsc_pbuf = NULL; + q = ctx->dnsc_qactive.head; + while((p = q) != NULL) { + q = q->dnsq_next; + free(p); + } + qlist_init(&ctx->dnsc_qactive); + ctx->dnsc_nactive = 0; + dns_drop_utm(ctx); + } +} + +void dns_reset(struct dns_ctx *ctx) { + SETCTX(ctx); + dns_close(ctx); + memset(ctx, 0, sizeof(*ctx)); + ctx->dnsc_timeout = 4; + ctx->dnsc_ntries = 3; + ctx->dnsc_ndots = 1; + ctx->dnsc_udpbuf = DNS_EDNS0PACKET; + ctx->dnsc_port = DNS_PORT; + ctx->dnsc_udpsock = -1; + ctx->dnsc_srchend = ctx->dnsc_srchbuf; + qlist_init(&ctx->dnsc_qactive); + dns_init_rng(ctx); + ctx->dnsc_flags = DNS_INITED; +} + +struct dns_ctx *dns_new(const struct dns_ctx *copy) { + struct dns_ctx *ctx; + SETCTXINITED(copy); + dns_assert_ctx(copy); + ctx = malloc(sizeof(*ctx)); + if (!ctx) + return NULL; + *ctx = *copy; + ctx->dnsc_udpsock = -1; + qlist_init(&ctx->dnsc_qactive); + ctx->dnsc_nactive = 0; + ctx->dnsc_pbuf = NULL; + ctx->dnsc_qstatus = 0; + ctx->dnsc_srchend = ctx->dnsc_srchbuf + + (copy->dnsc_srchend - copy->dnsc_srchbuf); + ctx->dnsc_utmfn = NULL; + ctx->dnsc_utmctx = NULL; + dns_init_rng(ctx); + return ctx; +} + +void dns_free(struct dns_ctx *ctx) { + assert(ctx != NULL && ctx != &dns_defctx); + dns_reset(ctx); + free(ctx); +} + +int dns_open(struct dns_ctx *ctx) { + int sock; + unsigned i; + int port; + union sockaddr_ns *sns; +#ifdef HAVE_IPv6 + unsigned have_inet6 = 0; +#endif + + SETCTXINITED(ctx); + assert(!CTXOPEN(ctx)); + + port = htons((unsigned short)ctx->dnsc_port); + /* ensure we have at least one server */ + if (!ctx->dnsc_nserv) { + sns = ctx->dnsc_serv; + sns->sin.sin_family = AF_INET; + sns->sin.sin_addr.s_addr = htonl(INADDR_LOOPBACK); + ctx->dnsc_nserv = 1; + } + + for (i = 0; i < ctx->dnsc_nserv; ++i) { + sns = &ctx->dnsc_serv[i]; + /* set port for each sockaddr */ +#ifdef HAVE_IPv6 + if (sns->sa.sa_family == AF_INET6) { + if (!sns->sin6.sin6_port) sns->sin6.sin6_port = (unsigned short)port; + ++have_inet6; + } + else +#endif + { + assert(sns->sa.sa_family == AF_INET); + if (!sns->sin.sin_port) sns->sin.sin_port = (unsigned short)port; + } + } + +#ifdef HAVE_IPv6 + if (have_inet6 && have_inet6 < ctx->dnsc_nserv) { + /* convert all IPv4 addresses to IPv6 V4MAPPED */ + struct sockaddr_in6 sin6; + memset(&sin6, 0, sizeof(sin6)); + sin6.sin6_family = AF_INET6; + /* V4MAPPED: ::ffff:1.2.3.4 */ + sin6.sin6_addr.s6_addr[10] = 0xff; + sin6.sin6_addr.s6_addr[11] = 0xff; + for(i = 0; i < ctx->dnsc_nserv; ++i) { + sns = &ctx->dnsc_serv[i]; + if (sns->sa.sa_family == AF_INET) { + sin6.sin6_port = sns->sin.sin_port; + memcpy(sin6.sin6_addr.s6_addr + 4*3, &sns->sin.sin_addr, 4); + sns->sin6 = sin6; + } + } + } + + ctx->dnsc_salen = have_inet6 ? + sizeof(struct sockaddr_in6) : sizeof(struct sockaddr_in); + + if (have_inet6) + sock = socket(PF_INET6, SOCK_DGRAM, IPPROTO_UDP); + else + sock = socket(PF_INET, SOCK_DGRAM, IPPROTO_UDP); +#else /* !HAVE_IPv6 */ + sock = socket(PF_INET, SOCK_DGRAM, IPPROTO_UDP); + ctx->dnsc_salen = sizeof(struct sockaddr_in); +#endif /* HAVE_IPv6 */ + + if (sock < 0) { + ctx->dnsc_qstatus = DNS_E_TEMPFAIL; + return -1; + } +#ifdef WINDOWS + { unsigned long on = 1; + if (ioctlsocket(sock, FIONBIO, &on) == SOCKET_ERROR) { + closesocket(sock); + ctx->dnsc_qstatus = DNS_E_TEMPFAIL; + return -1; + } + } +#else /* !WINDOWS */ + if (fcntl(sock, F_SETFL, fcntl(sock, F_GETFL) | O_NONBLOCK) < 0 || + fcntl(sock, F_SETFD, FD_CLOEXEC) < 0) { + closesocket(sock); + ctx->dnsc_qstatus = DNS_E_TEMPFAIL; + return -1; + } +#endif /* WINDOWS */ + /* allocate the packet buffer */ + if ((ctx->dnsc_pbuf = malloc(ctx->dnsc_udpbuf)) == NULL) { + closesocket(sock); + ctx->dnsc_qstatus = DNS_E_NOMEM; + errno = ENOMEM; + return -1; + } + + ctx->dnsc_udpsock = sock; + dns_request_utm(ctx, 0); + return sock; +} + +int dns_sock(const struct dns_ctx *ctx) { + SETCTXINITED(ctx); + return ctx->dnsc_udpsock; +} + +int dns_active(const struct dns_ctx *ctx) { + SETCTXINITED(ctx); + dns_assert_ctx(ctx); + return ctx->dnsc_nactive; +} + +int dns_status(const struct dns_ctx *ctx) { + SETCTX(ctx); + return ctx->dnsc_qstatus; +} +void dns_setstatus(struct dns_ctx *ctx, int status) { + SETCTX(ctx); + ctx->dnsc_qstatus = status; +} + +/* End the query: disconnect it from the active list, free it, + * and return the result to the caller. + */ +static void +dns_end_query(struct dns_ctx *ctx, struct dns_query *q, + int status, void *result) { + dns_query_fn *cbck = q->dnsq_cbck; + void *cbdata = q->dnsq_cbdata; + ctx->dnsc_qstatus = status; + assert((status < 0 && result == 0) || (status >= 0 && result != 0)); + assert(cbck != 0); /*XXX callback may be NULL */ + assert(ctx->dnsc_nactive > 0); + --ctx->dnsc_nactive; + qlist_remove(&ctx->dnsc_qactive, q); + /* force the query to be unconnected */ + /*memset(q, 0, sizeof(*q));*/ +#ifndef NDEBUG + q->dnsq_ctx = NULL; +#endif + free(q); + cbck(ctx, result, cbdata); +} + +#define DNS_DBG(ctx, code, sa, slen, pkt, plen) \ + do { \ + if (ctx->dnsc_udbgfn) \ + ctx->dnsc_udbgfn(code, (sa), slen, pkt, plen, 0, 0); \ + } while(0) +#define DNS_DBGQ(ctx, q, code, sa, slen, pkt, plen) \ + do { \ + if (ctx->dnsc_udbgfn) \ + ctx->dnsc_udbgfn(code, (sa), slen, pkt, plen, q, q->dnsq_cbdata); \ + } while(0) + +static void dns_newid(struct dns_ctx *ctx, struct dns_query *q) { + /* this is how we choose an identifier for a new query (qID). + * For now, it's just sequential number, incremented for every query, and + * thus obviously trivial to guess. + * There are two choices: + * a) use sequential numbers. It is plain insecure. In DNS, there are two + * places where random numbers are (or can) be used to increase security: + * random qID and random source port number. Without this randomness + * (udns uses fixed port for all queries), or when the randomness is weak, + * it's trivial to spoof query replies. With randomness however, it + * becomes a bit more difficult task. Too bad we only have 16 bits for + * our security, as qID is only two bytes. It isn't a security per se, + * to rely on those 16 bits - an attacker can just flood us with fake + * replies with all possible qIDs (only 65536 of them), and in this case, + * even if we'll use true random qIDs, we'll be in trouble (not protected + * against spoofing). Yes, this is only possible on a high-speed network + * (probably on the LAN only, since usually a border router for a LAN + * protects internal machines from packets with spoofed local addresses + * from outside, and usually a nameserver resides on LAN), but it's + * still very well possible to send us fake replies. + * In other words: there's nothing a DNS (stub) resolver can do against + * spoofing attacks, unless DNSSEC is in use, which helps here alot. + * Too bad that DNSSEC isn't widespread, so relying on it isn't an + * option in almost all cases... + * b) use random qID, based on some random-number generation mechanism. + * This way, we increase our protection a bit (see above - it's very weak + * still), but we also increase risk of qID reuse and matching late replies + * that comes to queries we've sent before against new queries. There are + * some more corner cases around that, as well - for example, normally, + * udns tries to find the query for a given reply by qID, *and* by + * verifying that the query DN and other parameters are also the same + * (so if the new query is against another domain name, old reply will + * be ignored automatically). But certain types of replies which we now + * handle - for example, FORMERR reply from servers which refuses to + * process EDNS0-enabled packets - comes without all the query parameters + * but the qID - so we're forced to use qID only when determining which + * query the given reply corresponds to. This makes us even more + * vulnerable to spoofing attacks, because an attacker don't even need to + * know which queries we perform to spoof the replies - he only needs to + * flood us with fake FORMERR "replies". + * + * That all to say: using sequential (or any other trivially guessable) + * numbers for qIDs is insecure, but the whole thing is inherently insecure + * as well, and this "extra weakness" that comes from weak qID choosing + * algorithm adds almost nothing to the underlying problem. + * + * It CAN NOT be made secure. Period. That's it. + * Unless we choose to implement DNSSEC, which is a whole different story. + * Forcing TCP mode makes it better, but who uses TCP for DNS anyway? + * (and it's hardly possible because of huge impact on the recursive + * nameservers). + * + * Note that ALL stub resolvers (again, unless they implement and enforce + * DNSSEC) suffers from this same problem. + * + * Here, I use a pseudo-random number generator for qIDs, instead of a + * simpler sequential IDs. This is _not_ more secure than sequential + * ID, but some found random IDs more enjoyeable for some reason. So + * here it goes. + */ + + /* Use random number and check if it's unique. + * If it's not, try again up to 5 times. + */ + unsigned loop; + dnsc_t c0, c1; + for(loop = 0; loop < 5; ++loop) { + const struct dns_query *c; + if (!ctx->dnsc_nextid) + ctx->dnsc_nextid = udns_jranval(&ctx->dnsc_jran); + c0 = ctx->dnsc_nextid & 0xff; + c1 = (ctx->dnsc_nextid >> 8) & 0xff; + ctx->dnsc_nextid >>= 16; + for(c = ctx->dnsc_qactive.head; c; c = c->dnsq_next) + if (c->dnsq_id[0] == c0 && c->dnsq_id[1] == c1) + break; /* found such entry, try again */ + if (!c) + break; + } + q->dnsq_id[0] = c0; q->dnsq_id[1] = c1; + + /* reset all parameters relevant for previous query lifetime */ + q->dnsq_try = 0; + q->dnsq_servi = 0; + /*XXX probably should keep dnsq_servnEDNS0 bits? + * See also comments in dns_ioevent() about FORMERR case */ + q->dnsq_servwait = q->dnsq_servskip = q->dnsq_servnEDNS0 = 0; +} + +/* Find next search suffix and fills in q->dnsq_dn. + * Return 0 if no more to try. */ +static int dns_next_srch(struct dns_ctx *ctx, struct dns_query *q) { + unsigned dnl; + + for(;;) { + if (q->dnsq_nxtsrch > ctx->dnsc_srchend) + return 0; + dnl = dns_dnlen(q->dnsq_nxtsrch); + if (dnl + q->dnsq_origdnl0 <= DNS_MAXDN && + (*q->dnsq_nxtsrch || !(q->dnsq_flags & DNS_ASIS_DONE))) + break; + q->dnsq_nxtsrch += dnl; + } + memcpy(q->dnsq_dn + q->dnsq_origdnl0, q->dnsq_nxtsrch, dnl); + if (!*q->dnsq_nxtsrch) + q->dnsq_flags |= DNS_ASIS_DONE; + q->dnsq_nxtsrch += dnl; + dns_newid(ctx, q); /* new ID for new qDN */ + return 1; +} + +/* find the server to try for current iteration. + * Note that current dnsq_servi may point to a server we should skip -- + * in that case advance to the next server. + * Return true if found, false if all tried. + */ +static int dns_find_serv(const struct dns_ctx *ctx, struct dns_query *q) { + while(q->dnsq_servi < ctx->dnsc_nserv) { + if (!(q->dnsq_servskip & (1 << q->dnsq_servi))) + return 1; + ++q->dnsq_servi; + } + return 0; +} + +/* format and send the query to a given server. + * In case of network problem (sendto() fails), return -1, + * else return 0. + */ +static int +dns_send_this(struct dns_ctx *ctx, struct dns_query *q, + unsigned servi, time_t now) { + unsigned qlen; + unsigned tries; + + { /* format the query buffer */ + dnsc_t *p = ctx->dnsc_pbuf; + memset(p, 0, DNS_HSIZE); + if (!(q->dnsq_flags & DNS_NORD)) p[DNS_H_F1] |= DNS_HF1_RD; + if (q->dnsq_flags & DNS_AAONLY) p[DNS_H_F1] |= DNS_HF1_AA; + if (q->dnsq_flags & DNS_SET_CD) p[DNS_H_F2] |= DNS_HF2_CD; + p[DNS_H_QDCNT2] = 1; + memcpy(p + DNS_H_QID, q->dnsq_id, 2); + p = dns_payload(p); + /* copy query dn */ + p += dns_dntodn(q->dnsq_dn, p, DNS_MAXDN); + /* query type and class */ + memcpy(p, q->dnsq_typcls, 4); p += 4; + /* add EDNS0 record. DO flag requires it */ + if (q->dnsq_flags & DNS_SET_DO || + (ctx->dnsc_udpbuf > DNS_MAXPACKET && + !(q->dnsq_servnEDNS0 & (1 << servi)))) { + *p++ = 0; /* empty (root) DN */ + p = dns_put16(p, DNS_T_OPT); + p = dns_put16(p, ctx->dnsc_udpbuf); + /* EDNS0 RCODE & VERSION; rest of the TTL field; RDLEN */ + memset(p, 0, 2+2+2); + if (q->dnsq_flags & DNS_SET_DO) p[2] |= DNS_EF1_DO; + p += 2+2+2; + ctx->dnsc_pbuf[DNS_H_ARCNT2] = 1; + } + qlen = p - ctx->dnsc_pbuf; + assert(qlen <= ctx->dnsc_udpbuf); + } + + /* send the query */ + tries = 10; + while (sendto(ctx->dnsc_udpsock, (void*)ctx->dnsc_pbuf, qlen, 0, + &ctx->dnsc_serv[servi].sa, ctx->dnsc_salen) < 0) { + /*XXX just ignore the sendto() error for now and try again. + * In the future, it may be possible to retrieve the error code + * and find which operation/query failed. + *XXX try the next server too? (if ENETUNREACH is returned immediately) + */ + if (--tries) continue; + /* if we can't send the query, fail it. */ + dns_end_query(ctx, q, DNS_E_TEMPFAIL, 0); + return -1; + } + DNS_DBGQ(ctx, q, 1, + &ctx->dnsc_serv[servi].sa, sizeof(union sockaddr_ns), + ctx->dnsc_pbuf, qlen); + q->dnsq_servwait |= 1 << servi; /* expect reply from this ns */ + + q->dnsq_deadline = now + + (dns_find_serv(ctx, q) ? 1 : ctx->dnsc_timeout << q->dnsq_try); + + /* move the query to the proper place, according to the new deadline */ + qlist_remove(&ctx->dnsc_qactive, q); + { /* insert from the tail */ + struct dns_query *p; + for(p = ctx->dnsc_qactive.tail; p; p = p->dnsq_prev) + if (p->dnsq_deadline <= q->dnsq_deadline) + break; + qlist_insert_after(&ctx->dnsc_qactive, q, p); + } + + return 0; +} + +/* send the query out using next available server + * and add it to the active list, or, if no servers available, + * end it. + */ +static void +dns_send(struct dns_ctx *ctx, struct dns_query *q, time_t now) { + + /* if we can't send the query, return TEMPFAIL even when searching: + * we can't be sure whenever the name we tried to search exists or not, + * so don't continue searching, or we may find the wrong name. */ + + if (!dns_find_serv(ctx, q)) { + /* no more servers in this iteration. Try the next cycle */ + q->dnsq_servi = 0; /* reset */ + q->dnsq_try++; /* next try */ + if (q->dnsq_try >= ctx->dnsc_ntries || + !dns_find_serv(ctx, q)) { + /* no more servers and tries, fail the query */ + /* return TEMPFAIL even when searching: no more tries for this + * searchlist, and no single definitive reply (handled in dns_ioevent() + * in NOERROR or NXDOMAIN cases) => all nameservers failed to process + * current search list element, so we don't know whenever the name exists. + */ + dns_end_query(ctx, q, DNS_E_TEMPFAIL, 0); + return; + } + } + + dns_send_this(ctx, q, q->dnsq_servi++, now); +} + +static void dns_dummy_cb(struct dns_ctx *ctx, void *result, void *data) { + if (result) free(result); + data = ctx = 0; /* used */ +} + +/* The (only, main, real) query submission routine. + * Allocate new query structure, initialize it, check validity of + * parameters, and add it to the head of the active list, without + * trying to send it (to be picked up on next event). + * Error return (without calling the callback routine) - + * no memory or wrong parameters. + *XXX The `no memory' case probably should go to the callback anyway... + */ +struct dns_query * +dns_submit_dn(struct dns_ctx *ctx, + dnscc_t *dn, int qcls, int qtyp, int flags, + dns_parse_fn *parse, dns_query_fn *cbck, void *data) { + struct dns_query *q; + SETCTXOPEN(ctx); + dns_assert_ctx(ctx); + + q = calloc(sizeof(*q), 1); + if (!q) { + ctx->dnsc_qstatus = DNS_E_NOMEM; + return NULL; + } + +#ifndef NDEBUG + q->dnsq_ctx = ctx; +#endif + q->dnsq_parse = parse; + q->dnsq_cbck = cbck ? cbck : dns_dummy_cb; + q->dnsq_cbdata = data; + + q->dnsq_origdnl0 = dns_dntodn(dn, q->dnsq_dn, sizeof(q->dnsq_dn)); + assert(q->dnsq_origdnl0 > 0); + --q->dnsq_origdnl0; /* w/o the trailing 0 */ + dns_put16(q->dnsq_typcls+0, qtyp); + dns_put16(q->dnsq_typcls+2, qcls); + q->dnsq_flags = (flags | ctx->dnsc_flags) & ~DNS_INTERNAL; + + if (flags & DNS_NOSRCH || + dns_dnlabels(q->dnsq_dn) > ctx->dnsc_ndots) { + q->dnsq_nxtsrch = flags & DNS_NOSRCH ? + ctx->dnsc_srchend /* end of the search list if no search requested */ : + ctx->dnsc_srchbuf /* beginning of the list, but try as-is first */; + q->dnsq_flags |= DNS_ASIS_DONE; + dns_newid(ctx, q); + } + else { + q->dnsq_nxtsrch = ctx->dnsc_srchbuf; + dns_next_srch(ctx, q); + } + + /* q->dnsq_deadline is set to 0 (calloc above): the new query is + * "already expired" when first inserted into queue, so it's safe + * to insert it into the head of the list. Next call to dns_timeouts() + * will actually send it. + */ + qlist_add_head(&ctx->dnsc_qactive, q); + ++ctx->dnsc_nactive; + dns_request_utm(ctx, 0); + + return q; +} + +struct dns_query * +dns_submit_p(struct dns_ctx *ctx, + const char *name, int qcls, int qtyp, int flags, + dns_parse_fn *parse, dns_query_fn *cbck, void *data) { + int isabs; + SETCTXOPEN(ctx); + if (dns_ptodn(name, 0, ctx->dnsc_pbuf, DNS_MAXDN, &isabs) <= 0) { + ctx->dnsc_qstatus = DNS_E_BADQUERY; + return NULL; + } + if (isabs) + flags |= DNS_NOSRCH; + return + dns_submit_dn(ctx, ctx->dnsc_pbuf, qcls, qtyp, flags, parse, cbck, data); +} + +/* process readable fd condition. + * To be usable in edge-triggered environment, the routine + * should consume all input so it should loop over. + * Note it isn't really necessary to loop here, because + * an application may perform the loop just fine by it's own, + * but in this case we should return some sensitive result, + * to indicate when to stop calling and error conditions. + * Note also we may encounter all sorts of recvfrom() + * errors which aren't fatal, and at the same time we may + * loop forever if an error IS fatal. + */ +void dns_ioevent(struct dns_ctx *ctx, time_t now) { + int r; + unsigned servi; + struct dns_query *q; + dnsc_t *pbuf; + dnscc_t *pend, *pcur; + void *result; + union sockaddr_ns sns; + socklen_t slen; + + SETCTX(ctx); + if (!CTXOPEN(ctx)) + return; + dns_assert_ctx(ctx); + pbuf = ctx->dnsc_pbuf; + + if (!now) now = time(NULL); + +again: /* receive the reply */ + + slen = sizeof(sns); + r = recvfrom(ctx->dnsc_udpsock, (void*)pbuf, ctx->dnsc_udpbuf, + MSG_DONTWAIT, &sns.sa, &slen); + if (r < 0) { + /*XXX just ignore recvfrom() errors for now. + * in the future it may be possible to determine which + * query failed and requeue it. + * Note there may be various error conditions, triggered + * by both local problems and remote problems. It isn't + * quite trivial to determine whenever an error is local + * or remote. On local errors, we should stop, while + * remote errors should be ignored (for now anyway). + */ +#ifdef WINDOWS + if (WSAGetLastError() == WSAEWOULDBLOCK) +#else + if (errno == EAGAIN) +#endif + { + dns_request_utm(ctx, now); + return; + } + goto again; + } + + pend = pbuf + r; + pcur = dns_payload(pbuf); + + /* check reply header */ + if (pcur > pend || dns_numqd(pbuf) > 1 || dns_opcode(pbuf) != 0) { + DNS_DBG(ctx, -1/*bad reply*/, &sns.sa, slen, pbuf, r); + goto again; + } + + /* find the matching query, by qID */ + for (q = ctx->dnsc_qactive.head; ; q = q->dnsq_next) { + if (!q) { + /* no more requests: old reply? */ + DNS_DBG(ctx, -5/*no matching query*/, &sns.sa, slen, pbuf, r); + goto again; + } + if (pbuf[DNS_H_QID1] == q->dnsq_id[0] && + pbuf[DNS_H_QID2] == q->dnsq_id[1]) + break; + } + + /* if we have numqd, compare with our query qDN */ + if (dns_numqd(pbuf)) { + /* decode the qDN */ + dnsc_t dn[DNS_MAXDN]; + if (dns_getdn(pbuf, &pcur, pend, dn, sizeof(dn)) < 0 || + pcur + 4 > pend) { + DNS_DBG(ctx, -1/*bad reply*/, &sns.sa, slen, pbuf, r); + goto again; + } + if (!dns_dnequal(dn, q->dnsq_dn) || + memcmp(pcur, q->dnsq_typcls, 4) != 0) { + /* not this query */ + DNS_DBG(ctx, -5/*no matching query*/, &sns.sa, slen, pbuf, r); + goto again; + } + /* here, query match, and pcur points past qDN in query section in pbuf */ + } + /* if no numqd, we only allow FORMERR rcode */ + else if (dns_rcode(pbuf) != DNS_R_FORMERR) { + /* treat it as bad reply if !FORMERR */ + DNS_DBG(ctx, -1/*bad reply*/, &sns.sa, slen, pbuf, r); + goto again; + } + else { + /* else it's FORMERR, handled below */ + } + + /* find server */ +#ifdef HAVE_IPv6 + if (sns.sa.sa_family == AF_INET6 && slen >= sizeof(sns.sin6)) { + for(servi = 0; servi < ctx->dnsc_nserv; ++servi) + if (sin6_eq(ctx->dnsc_serv[servi].sin6, sns.sin6)) + break; + } + else +#endif + if (sns.sa.sa_family == AF_INET && slen >= sizeof(sns.sin)) { + for(servi = 0; servi < ctx->dnsc_nserv; ++servi) + if (sin_eq(ctx->dnsc_serv[servi].sin, sns.sin)) + break; + } + else + servi = ctx->dnsc_nserv; + + /* check if we expect reply from this server. + * Note we can receive reply from first try if we're already at next */ + if (!(q->dnsq_servwait & (1 << servi))) { /* if ever asked this NS */ + DNS_DBG(ctx, -2/*wrong server*/, &sns.sa, slen, pbuf, r); + goto again; + } + + /* we got (some) reply for our query */ + + DNS_DBGQ(ctx, q, 0, &sns.sa, slen, pbuf, r); + q->dnsq_servwait &= ~(1 << servi); /* don't expect reply from this serv */ + + /* process the RCODE */ + switch(dns_rcode(pbuf)) { + + case DNS_R_NOERROR: + if (dns_tc(pbuf)) { + /* possible truncation. We can't deal with it. */ + /*XXX for now, treat TC bit the same as SERVFAIL. + * It is possible to: + * a) try to decode the reply - may be ANSWER section is ok; + * b) check if server understands EDNS0, and if it is, and + * answer still don't fit, end query. + */ + break; + } + if (!dns_numan(pbuf)) { /* no data of requested type */ + if (dns_next_srch(ctx, q)) { + /* if we're searching, try next searchlist element, + * but remember NODATA reply. */ + q->dnsq_flags |= DNS_SEEN_NODATA; + dns_send(ctx, q, now); + } + else + /* else - nothing to search any more - finish the query. + * It will be NODATA since we've seen a NODATA reply. */ + dns_end_query(ctx, q, DNS_E_NODATA, 0); + } + /* we've got a positive reply here */ + else if (q->dnsq_parse) { + /* if we have parsing routine, call it and return whatever it returned */ + /* don't try to re-search if NODATA here. For example, + * if we asked for A but only received CNAME. Unless we'll + * someday do recursive queries. And that's problematic too, since + * we may be dealing with specific AA-only nameservers for a given + * domain, but CNAME points elsewhere... + */ + r = q->dnsq_parse(q->dnsq_dn, pbuf, pcur, pend, &result); + dns_end_query(ctx, q, r, r < 0 ? NULL : result); + } + /* else just malloc+copy the raw DNS reply */ + else if ((result = malloc(r)) == NULL) + dns_end_query(ctx, q, DNS_E_NOMEM, NULL); + else { + memcpy(result, pbuf, r); + dns_end_query(ctx, q, r, result); + } + goto again; + + case DNS_R_NXDOMAIN: /* Non-existing domain. */ + if (dns_next_srch(ctx, q)) + /* more search entries exists, try them. */ + dns_send(ctx, q, now); + else + /* nothing to search anymore. End the query, returning either NODATA + * if we've seen it before, or NXDOMAIN if not. */ + dns_end_query(ctx, q, + q->dnsq_flags & DNS_SEEN_NODATA ? DNS_E_NODATA : DNS_E_NXDOMAIN, 0); + goto again; + + case DNS_R_FORMERR: + case DNS_R_NOTIMPL: + /* for FORMERR and NOTIMPL rcodes, if we tried EDNS0-enabled query, + * try w/o EDNS0. */ + if (ctx->dnsc_udpbuf > DNS_MAXPACKET && + !(q->dnsq_servnEDNS0 & (1 << servi))) { + /* we always trying EDNS0 first if enabled, and retry a given query + * if not available. Maybe it's better to remember inavailability of + * EDNS0 in ctx as a per-NS flag, and never try again for this NS. + * For long-running applications.. maybe they will change the nameserver + * while we're running? :) Also, since FORMERR is the only rcode we + * allow to be header-only, and in this case the only check we do to + * find a query it belongs to is qID (not qDN+qCLS+qTYP), it's much + * easier to spoof and to force us to perform non-EDNS0 queries only... + */ + q->dnsq_servnEDNS0 |= 1 << servi; + dns_send_this(ctx, q, servi, now); + goto again; + } + /* else we handle it the same as SERVFAIL etc */ + + case DNS_R_SERVFAIL: + case DNS_R_REFUSED: + /* for these rcodes, advance this request + * to the next server and reschedule */ + default: /* unknown rcode? hmmm... */ + break; + } + + /* here, we received unexpected reply */ + q->dnsq_servskip |= (1 << servi); /* don't retry this server */ + + /* we don't expect replies from this server anymore. + * But there may be other servers. Some may be still processing our + * query, and some may be left to try. + * We just ignore this reply and wait a bit more if some NSes haven't + * replied yet (dnsq_servwait != 0), and let the situation to be handled + * on next event processing. Timeout for this query is set correctly, + * if not taking into account the one-second difference - we can try + * next server in the same iteration sooner. + */ + + /* try next server */ + if (!q->dnsq_servwait) { + /* next retry: maybe some other servers will reply next time. + * dns_send() will end the query for us if no more servers to try. + * Note we can't continue with the next searchlist element here: + * we don't know if the current qdn exists or not, there's no definitive + * answer yet (which is seen in cases above). + *XXX standard resolver also tries as-is query in case all nameservers + * failed to process our query and if not tried before. We don't do it. + */ + dns_send(ctx, q, now); + } + else { + /* else don't do anything - not all servers replied yet */ + } + goto again; + +} + +/* handle all timeouts */ +int dns_timeouts(struct dns_ctx *ctx, int maxwait, time_t now) { + /* this is a hot routine */ + struct dns_query *q; + + SETCTX(ctx); + dns_assert_ctx(ctx); + + /* Pick up first entry from query list. + * If its deadline has passed, (re)send it + * (dns_send() will move it next in the list). + * If not, this is the query which determines the closest deadline. + */ + + q = ctx->dnsc_qactive.head; + if (!q) + return maxwait; + if (!now) + now = time(NULL); + do { + if (q->dnsq_deadline > now) { /* first non-expired query */ + int w = (int)(q->dnsq_deadline - now); + if (maxwait < 0 || maxwait > w) + maxwait = w; + break; + } + else { + /* process expired deadline */ + dns_send(ctx, q, now); + } + } while((q = ctx->dnsc_qactive.head) != NULL); + + dns_request_utm(ctx, now); /* update timer with new deadline */ + return maxwait; +} + +struct dns_resolve_data { + int dnsrd_done; + void *dnsrd_result; +}; + +static void dns_resolve_cb(struct dns_ctx *ctx, void *result, void *data) { + struct dns_resolve_data *d = data; + d->dnsrd_result = result; + d->dnsrd_done = 1; + ctx = ctx; +} + +void *dns_resolve(struct dns_ctx *ctx, struct dns_query *q) { + time_t now; + struct dns_resolve_data d; + int n; + SETCTXOPEN(ctx); + + if (!q) + return NULL; + + assert(ctx == q->dnsq_ctx); + dns_assert_ctx(ctx); + /* do not allow re-resolving syncronous queries */ + assert(q->dnsq_cbck != dns_resolve_cb && "can't resolve syncronous query"); + if (q->dnsq_cbck == dns_resolve_cb) { + ctx->dnsc_qstatus = DNS_E_BADQUERY; + return NULL; + } + q->dnsq_cbck = dns_resolve_cb; + q->dnsq_cbdata = &d; + d.dnsrd_done = 0; + + now = time(NULL); + while(!d.dnsrd_done && (n = dns_timeouts(ctx, -1, now)) >= 0) { +#ifdef HAVE_POLL + struct pollfd pfd; + pfd.fd = ctx->dnsc_udpsock; + pfd.events = POLLIN; + n = poll(&pfd, 1, n * 1000); +#else + fd_set rfd; + struct timeval tv; + FD_ZERO(&rfd); + FD_SET(ctx->dnsc_udpsock, &rfd); + tv.tv_sec = n; tv.tv_usec = 0; + n = select(ctx->dnsc_udpsock + 1, &rfd, NULL, NULL, &tv); +#endif + now = time(NULL); + if (n > 0) + dns_ioevent(ctx, now); + } + + return d.dnsrd_result; +} + +void *dns_resolve_dn(struct dns_ctx *ctx, + dnscc_t *dn, int qcls, int qtyp, int flags, + dns_parse_fn *parse) { + return + dns_resolve(ctx, + dns_submit_dn(ctx, dn, qcls, qtyp, flags, parse, NULL, NULL)); +} + +void *dns_resolve_p(struct dns_ctx *ctx, + const char *name, int qcls, int qtyp, int flags, + dns_parse_fn *parse) { + return + dns_resolve(ctx, + dns_submit_p(ctx, name, qcls, qtyp, flags, parse, NULL, NULL)); +} + +int dns_cancel(struct dns_ctx *ctx, struct dns_query *q) { + SETCTX(ctx); + dns_assert_ctx(ctx); + assert(q->dnsq_ctx == ctx); + /* do not allow cancelling syncronous queries */ + assert(q->dnsq_cbck != dns_resolve_cb && "can't cancel syncronous query"); + if (q->dnsq_cbck == dns_resolve_cb) + return (ctx->dnsc_qstatus = DNS_E_BADQUERY); + qlist_remove(&ctx->dnsc_qactive, q); + --ctx->dnsc_nactive; + dns_request_utm(ctx, 0); + return 0; +} + diff --git a/contrib/udns/udns_rr_a.c b/contrib/udns/udns_rr_a.c new file mode 100644 index 00000000000..4fdcbf9bc1c --- /dev/null +++ b/contrib/udns/udns_rr_a.c @@ -0,0 +1,123 @@ +/* udns_rr_a.c + parse/query A/AAAA IN records + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#include +#include +#include +#ifndef WINDOWS +# include +# include +#endif +#include "udns.h" + +/* here, we use common routine to parse both IPv4 and IPv6 addresses. + */ + +/* this structure should match dns_rr_a[46] */ +struct dns_rr_a { + dns_rr_common(dnsa); + unsigned char *dnsa_addr; +}; + +static int +dns_parse_a(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, + void **result, unsigned dsize) { + struct dns_rr_a *ret; + struct dns_parse p; + struct dns_rr rr; + int r; + + /* first, validate and count number of addresses */ + dns_initparse(&p, qdn, pkt, cur, end); + while((r = dns_nextrr(&p, &rr)) > 0) + if (rr.dnsrr_dsz != dsize) + return DNS_E_PROTOCOL; + if (r < 0) + return DNS_E_PROTOCOL; + else if (!p.dnsp_nrr) + return DNS_E_NODATA; + + ret = malloc(sizeof(*ret) + dsize * p.dnsp_nrr + dns_stdrr_size(&p)); + if (!ret) + return DNS_E_NOMEM; + + ret->dnsa_nrr = p.dnsp_nrr; + ret->dnsa_addr = (unsigned char*)(ret+1); + + /* copy the RRs */ + for (dns_rewind(&p, qdn), r = 0; dns_nextrr(&p, &rr); ++r) + memcpy(ret->dnsa_addr + dsize * r, rr.dnsrr_dptr, dsize); + + dns_stdrr_finish((struct dns_rr_null *)ret, + (char *)(ret->dnsa_addr + dsize * p.dnsp_nrr), &p); + *result = ret; + return 0; +} + +int +dns_parse_a4(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, + void **result) { +#ifdef AF_INET + assert(sizeof(struct in_addr) == 4); +#endif + assert(dns_get16(cur+2) == DNS_C_IN && dns_get16(cur+0) == DNS_T_A); + return dns_parse_a(qdn, pkt, cur, end, result, 4); +} + +struct dns_query * +dns_submit_a4(struct dns_ctx *ctx, const char *name, int flags, + dns_query_a4_fn *cbck, void *data) { + return + dns_submit_p(ctx, name, DNS_C_IN, DNS_T_A, flags, + dns_parse_a4, (dns_query_fn*)cbck, data); +} + +struct dns_rr_a4 * +dns_resolve_a4(struct dns_ctx *ctx, const char *name, int flags) { + return (struct dns_rr_a4 *) + dns_resolve_p(ctx, name, DNS_C_IN, DNS_T_A, flags, dns_parse_a4); +} + +int +dns_parse_a6(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, + void **result) { +#ifdef AF_INET6 + assert(sizeof(struct in6_addr) == 16); +#endif + assert(dns_get16(cur+2) == DNS_C_IN && dns_get16(cur+0) == DNS_T_AAAA); + return dns_parse_a(qdn, pkt, cur, end, result, 16); +} + +struct dns_query * +dns_submit_a6(struct dns_ctx *ctx, const char *name, int flags, + dns_query_a6_fn *cbck, void *data) { + return + dns_submit_p(ctx, name, DNS_C_IN, DNS_T_AAAA, flags, + dns_parse_a6, (dns_query_fn*)cbck, data); +} + +struct dns_rr_a6 * +dns_resolve_a6(struct dns_ctx *ctx, const char *name, int flags) { + return (struct dns_rr_a6 *) + dns_resolve_p(ctx, name, DNS_C_IN, DNS_T_AAAA, flags, dns_parse_a6); +} diff --git a/contrib/udns/udns_rr_mx.c b/contrib/udns/udns_rr_mx.c new file mode 100644 index 00000000000..0904e017177 --- /dev/null +++ b/contrib/udns/udns_rr_mx.c @@ -0,0 +1,91 @@ +/* udns_rr_mx.c + parse/query MX IN records + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#include +#include +#include +#include "udns.h" + +int +dns_parse_mx(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, + void **result) { + struct dns_rr_mx *ret; + struct dns_parse p; + struct dns_rr rr; + int r, l; + char *sp; + dnsc_t mx[DNS_MAXDN]; + + assert(dns_get16(cur+2) == DNS_C_IN && dns_get16(cur+0) == DNS_T_MX); + + /* first, validate the answer and count size of the result */ + l = 0; + dns_initparse(&p, qdn, pkt, cur, end); + while((r = dns_nextrr(&p, &rr)) > 0) { + cur = rr.dnsrr_dptr + 2; + r = dns_getdn(pkt, &cur, end, mx, sizeof(mx)); + if (r <= 0 || cur != rr.dnsrr_dend) + return DNS_E_PROTOCOL; + l += dns_dntop_size(mx); + } + if (r < 0) + return DNS_E_PROTOCOL; + if (!p.dnsp_nrr) + return DNS_E_NODATA; + + /* next, allocate and set up result */ + l += dns_stdrr_size(&p); + ret = malloc(sizeof(*ret) + sizeof(struct dns_mx) * p.dnsp_nrr + l); + if (!ret) + return DNS_E_NOMEM; + ret->dnsmx_nrr = p.dnsp_nrr; + ret->dnsmx_mx = (struct dns_mx *)(ret+1); + + /* and 3rd, fill in result, finally */ + sp = (char*)(ret->dnsmx_mx + p.dnsp_nrr); + for (dns_rewind(&p, qdn), r = 0; dns_nextrr(&p, &rr); ++r) { + ret->dnsmx_mx[r].name = sp; + cur = rr.dnsrr_dptr; + ret->dnsmx_mx[r].priority = dns_get16(cur); + cur += 2; + dns_getdn(pkt, &cur, end, mx, sizeof(mx)); + sp += dns_dntop(mx, sp, DNS_MAXNAME); + } + dns_stdrr_finish((struct dns_rr_null *)ret, sp, &p); + *result = ret; + return 0; +} + +struct dns_query * +dns_submit_mx(struct dns_ctx *ctx, const char *name, int flags, + dns_query_mx_fn *cbck, void *data) { + return + dns_submit_p(ctx, name, DNS_C_IN, DNS_T_MX, flags, + dns_parse_mx, (dns_query_fn *)cbck, data); +} + +struct dns_rr_mx * +dns_resolve_mx(struct dns_ctx *ctx, const char *name, int flags) { + return (struct dns_rr_mx *) + dns_resolve_p(ctx, name, DNS_C_IN, DNS_T_MX, flags, dns_parse_mx); +} diff --git a/contrib/udns/udns_rr_naptr.c b/contrib/udns/udns_rr_naptr.c new file mode 100644 index 00000000000..da30069db62 --- /dev/null +++ b/contrib/udns/udns_rr_naptr.c @@ -0,0 +1,128 @@ +/* udns_rr_naptr.c + parse/query NAPTR IN records + + Copyright (C) 2005 Michael Tokarev + Copyright (C) 2006 Mikael Magnusson + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#include +#include +#include +#include "udns.h" + +/* Get a single string for NAPTR record, pretty much like a DN label. + * String length is in first byte in *cur, so it can't be >255. + */ +static int dns_getstr(dnscc_t **cur, dnscc_t *ep, char *buf) +{ + unsigned l; + dnscc_t *cp = *cur; + + l = *cp++; + if (cp + l > ep) + return DNS_E_PROTOCOL; + if (buf) { + memcpy(buf, cp, l); + buf[l] = '\0'; + } + cp += l; + + *cur = cp; + return l + 1; +} + +int +dns_parse_naptr(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, + void **result) { + struct dns_rr_naptr *ret; + struct dns_parse p; + struct dns_rr rr; + int r, l; + char *sp; + dnsc_t dn[DNS_MAXDN]; + + assert(dns_get16(cur+2) == DNS_C_IN && dns_get16(cur+0) == DNS_T_NAPTR); + + /* first, validate the answer and count size of the result */ + l = 0; + dns_initparse(&p, qdn, pkt, cur, end); + while((r = dns_nextrr(&p, &rr)) > 0) { + int i; + dnscc_t *ep = rr.dnsrr_dend; + + /* first 4 bytes: order & preference */ + cur = rr.dnsrr_dptr + 4; + + /* flags, services and regexp */ + for (i = 0; i < 3; i++) { + r = dns_getstr(&cur, ep, NULL); + if (r < 0) + return r; + l += r; + } + /* replacement */ + r = dns_getdn(pkt, &cur, end, dn, sizeof(dn)); + if (r <= 0 || cur != rr.dnsrr_dend) + return DNS_E_PROTOCOL; + l += dns_dntop_size(dn); + } + if (r < 0) + return DNS_E_PROTOCOL; + if (!p.dnsp_nrr) + return DNS_E_NODATA; + + /* next, allocate and set up result */ + l += dns_stdrr_size(&p); + ret = malloc(sizeof(*ret) + sizeof(struct dns_naptr) * p.dnsp_nrr + l); + if (!ret) + return DNS_E_NOMEM; + ret->dnsnaptr_nrr = p.dnsp_nrr; + ret->dnsnaptr_naptr = (struct dns_naptr *)(ret+1); + + /* and 3rd, fill in result, finally */ + sp = (char*)(&ret->dnsnaptr_naptr[p.dnsp_nrr]); + for (dns_rewind(&p, qdn), r = 0; dns_nextrr(&p, &rr); ++r) { + cur = rr.dnsrr_dptr; + ret->dnsnaptr_naptr[r].order = dns_get16(cur); cur += 2; + ret->dnsnaptr_naptr[r].preference = dns_get16(cur); cur += 2; + sp += dns_getstr(&cur, end, (ret->dnsnaptr_naptr[r].flags = sp)); + sp += dns_getstr(&cur, end, (ret->dnsnaptr_naptr[r].service = sp)); + sp += dns_getstr(&cur, end, (ret->dnsnaptr_naptr[r].regexp = sp)); + dns_getdn(pkt, &cur, end, dn, sizeof(dn)); + sp += dns_dntop(dn, (ret->dnsnaptr_naptr[r].replacement = sp), DNS_MAXNAME); + } + dns_stdrr_finish((struct dns_rr_null *)ret, sp, &p); + *result = ret; + return 0; +} + +struct dns_query * +dns_submit_naptr(struct dns_ctx *ctx, const char *name, int flags, + dns_query_naptr_fn *cbck, void *data) { + return + dns_submit_p(ctx, name, DNS_C_IN, DNS_T_NAPTR, flags, + dns_parse_naptr, (dns_query_fn *)cbck, data); +} + +struct dns_rr_naptr * +dns_resolve_naptr(struct dns_ctx *ctx, const char *name, int flags) { + return (struct dns_rr_naptr *) + dns_resolve_p(ctx, name, DNS_C_IN, DNS_T_NAPTR, flags, dns_parse_naptr); +} diff --git a/contrib/udns/udns_rr_ptr.c b/contrib/udns/udns_rr_ptr.c new file mode 100644 index 00000000000..1f682aebc0f --- /dev/null +++ b/contrib/udns/udns_rr_ptr.c @@ -0,0 +1,109 @@ +/* udns_rr_ptr.c + parse/query PTR records + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#include +#include +#include "udns.h" + +int +dns_parse_ptr(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, + void **result) { + struct dns_rr_ptr *ret; + struct dns_parse p; + struct dns_rr rr; + int r, l, c; + char *sp; + dnsc_t ptr[DNS_MAXDN]; + + assert(dns_get16(cur+2) == DNS_C_IN && dns_get16(cur+0) == DNS_T_PTR); + + /* first, validate the answer and count size of the result */ + l = c = 0; + dns_initparse(&p, qdn, pkt, cur, end); + while((r = dns_nextrr(&p, &rr)) > 0) { + cur = rr.dnsrr_dptr; + r = dns_getdn(pkt, &cur, end, ptr, sizeof(ptr)); + if (r <= 0 || cur != rr.dnsrr_dend) + return DNS_E_PROTOCOL; + l += dns_dntop_size(ptr); + ++c; + } + if (r < 0) + return DNS_E_PROTOCOL; + if (!c) + return DNS_E_NODATA; + + /* next, allocate and set up result */ + ret = malloc(sizeof(*ret) + sizeof(char **) * c + l + dns_stdrr_size(&p)); + if (!ret) + return DNS_E_NOMEM; + ret->dnsptr_nrr = c; + ret->dnsptr_ptr = (char **)(ret+1); + + /* and 3rd, fill in result, finally */ + sp = (char*)(ret->dnsptr_ptr + c); + c = 0; + dns_rewind(&p, qdn); + while((r = dns_nextrr(&p, &rr)) > 0) { + ret->dnsptr_ptr[c] = sp; + cur = rr.dnsrr_dptr; + dns_getdn(pkt, &cur, end, ptr, sizeof(ptr)); + sp += dns_dntop(ptr, sp, DNS_MAXNAME); + ++c; + } + dns_stdrr_finish((struct dns_rr_null *)ret, sp, &p); + *result = ret; + return 0; +} + +struct dns_query * +dns_submit_a4ptr(struct dns_ctx *ctx, const struct in_addr *addr, + dns_query_ptr_fn *cbck, void *data) { + dnsc_t dn[DNS_A4RSIZE]; + dns_a4todn(addr, 0, dn, sizeof(dn)); + return + dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_PTR, DNS_NOSRCH, + dns_parse_ptr, (dns_query_fn *)cbck, data); +} + +struct dns_rr_ptr * +dns_resolve_a4ptr(struct dns_ctx *ctx, const struct in_addr *addr) { + return (struct dns_rr_ptr *) + dns_resolve(ctx, dns_submit_a4ptr(ctx, addr, NULL, NULL)); +} + +struct dns_query * +dns_submit_a6ptr(struct dns_ctx *ctx, const struct in6_addr *addr, + dns_query_ptr_fn *cbck, void *data) { + dnsc_t dn[DNS_A6RSIZE]; + dns_a6todn(addr, 0, dn, sizeof(dn)); + return + dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_PTR, DNS_NOSRCH, + dns_parse_ptr, (dns_query_fn *)cbck, data); +} + +struct dns_rr_ptr * +dns_resolve_a6ptr(struct dns_ctx *ctx, const struct in6_addr *addr) { + return (struct dns_rr_ptr *) + dns_resolve(ctx, dns_submit_a6ptr(ctx, addr, NULL, NULL)); +} diff --git a/contrib/udns/udns_rr_srv.c b/contrib/udns/udns_rr_srv.c new file mode 100644 index 00000000000..dfba4653a3a --- /dev/null +++ b/contrib/udns/udns_rr_srv.c @@ -0,0 +1,155 @@ +/* udns_rr_srv.c + parse/query SRV IN (rfc2782) records + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + Copyright 2005 Thadeu Lima de Souza Cascardo + + 2005-09-11: + Changed MX parser file into a SRV parser file + + */ + +#include +#include +#include +#include "udns.h" + +int +dns_parse_srv(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, + void **result) { + struct dns_rr_srv *ret; + struct dns_parse p; + struct dns_rr rr; + int r, l; + char *sp; + dnsc_t srv[DNS_MAXDN]; + + assert(dns_get16(cur+2) == DNS_C_IN && dns_get16(cur+0) == DNS_T_SRV); + + /* first, validate the answer and count size of the result */ + l = 0; + dns_initparse(&p, qdn, pkt, cur, end); + while((r = dns_nextrr(&p, &rr)) > 0) { + cur = rr.dnsrr_dptr + 6; + r = dns_getdn(pkt, &cur, end, srv, sizeof(srv)); + if (r <= 0 || cur != rr.dnsrr_dend) + return DNS_E_PROTOCOL; + l += dns_dntop_size(srv); + } + if (r < 0) + return DNS_E_PROTOCOL; + if (!p.dnsp_nrr) + return DNS_E_NODATA; + + /* next, allocate and set up result */ + l += dns_stdrr_size(&p); + ret = malloc(sizeof(*ret) + sizeof(struct dns_srv) * p.dnsp_nrr + l); + if (!ret) + return DNS_E_NOMEM; + ret->dnssrv_nrr = p.dnsp_nrr; + ret->dnssrv_srv = (struct dns_srv *)(ret+1); + + /* and 3rd, fill in result, finally */ + sp = (char*)(ret->dnssrv_srv + p.dnsp_nrr); + for (dns_rewind(&p, qdn), r = 0; dns_nextrr(&p, &rr); ++r) { + ret->dnssrv_srv[r].name = sp; + cur = rr.dnsrr_dptr; + ret->dnssrv_srv[r].priority = dns_get16(cur); + ret->dnssrv_srv[r].weight = dns_get16(cur+2); + ret->dnssrv_srv[r].port = dns_get16(cur+4); + cur += 6; + dns_getdn(pkt, &cur, end, srv, sizeof(srv)); + sp += dns_dntop(srv, sp, DNS_MAXNAME); + } + dns_stdrr_finish((struct dns_rr_null *)ret, sp, &p); + *result = ret; + return 0; +} + +/* Add a single service or proto name prepending an undescore (_), + * according to rfc2782 rules. + * Return 0 or the label length. + * Routing assumes dn holds enouth space for a single DN label. */ +static int add_sname(dnsc_t *dn, const char *sn) { + int l = dns_ptodn(sn, 0, dn + 1, DNS_MAXLABEL-1, NULL); + if (l <= 1 || l - 2 != dn[1]) + /* Should we really check if sn is exactly one label? Do we care? */ + return 0; + dn[0] = l - 1; + dn[1] = '_'; + return l; +} + +/* Construct a domain name for SRV query from the given name, service and proto. + * The code allows any combinations of srv and proto (both are non-NULL, + * both NULL, or either one is non-NULL). Whenever it makes any sense or not + * is left as an exercise to programmer. + * Return negative value on error (malformed query) or addition query flag(s). + */ +static int +build_srv_dn(dnsc_t *dn, const char *name, const char *srv, const char *proto) +{ + int p = 0, l, isabs; + if (srv) { + l = add_sname(dn + p, srv); + if (!l) + return -1; + p += l; + } + if (proto) { + l = add_sname(dn + p, proto); + if (!l) + return -1; + p += l; + } + l = dns_ptodn(name, 0, dn + p, DNS_MAXDN - p, &isabs); + if (l < 0) + return -1; + return isabs ? DNS_NOSRCH : 0; +} + +struct dns_query * +dns_submit_srv(struct dns_ctx *ctx, + const char *name, const char *srv, const char *proto, + int flags, dns_query_srv_fn *cbck, void *data) { + dnsc_t dn[DNS_MAXDN]; + int r = build_srv_dn(dn, name, srv, proto); + if (r < 0) { + dns_setstatus (ctx, DNS_E_BADQUERY); + return NULL; + } + return + dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_SRV, flags | r, + dns_parse_srv, (dns_query_fn *)cbck, data); +} + +struct dns_rr_srv * +dns_resolve_srv(struct dns_ctx *ctx, + const char *name, const char *srv, const char *proto, int flags) +{ + dnsc_t dn[DNS_MAXDN]; + int r = build_srv_dn(dn, name, srv, proto); + if (r < 0) { + dns_setstatus(ctx, DNS_E_BADQUERY); + return NULL; + } + return (struct dns_rr_srv *) + dns_resolve_dn(ctx, dn, DNS_C_IN, DNS_T_SRV, flags | r, dns_parse_srv); +} diff --git a/contrib/udns/udns_rr_txt.c b/contrib/udns/udns_rr_txt.c new file mode 100644 index 00000000000..97f1dfb18c3 --- /dev/null +++ b/contrib/udns/udns_rr_txt.c @@ -0,0 +1,98 @@ +/* udns_rr_txt.c + parse/query TXT records + + Copyright (C) 2005 Michael Tokarev + This file is part of UDNS library, an async DNS stub resolver. + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library, in file named COPYING.LGPL; if not, + write to the Free Software Foundation, Inc., 59 Temple Place, + Suite 330, Boston, MA 02111-1307 USA + + */ + +#include +#include +#include +#include "udns.h" + +int +dns_parse_txt(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, + void **result) { + struct dns_rr_txt *ret; + struct dns_parse p; + struct dns_rr rr; + int r, l; + dnsc_t *sp; + dnscc_t *cp, *ep; + + assert(dns_get16(cur+0) == DNS_T_TXT); + + /* first, validate the answer and count size of the result */ + l = 0; + dns_initparse(&p, qdn, pkt, cur, end); + while((r = dns_nextrr(&p, &rr)) > 0) { + cp = rr.dnsrr_dptr; ep = rr.dnsrr_dend; + while(cp < ep) { + r = *cp++; + if (cp + r > ep) + return DNS_E_PROTOCOL; + l += r; + cp += r; + } + } + if (r < 0) + return DNS_E_PROTOCOL; + if (!p.dnsp_nrr) + return DNS_E_NODATA; + + /* next, allocate and set up result */ + l += (sizeof(struct dns_txt) + 1) * p.dnsp_nrr + dns_stdrr_size(&p); + ret = malloc(sizeof(*ret) + l); + if (!ret) + return DNS_E_NOMEM; + ret->dnstxt_nrr = p.dnsp_nrr; + ret->dnstxt_txt = (struct dns_txt *)(ret+1); + + /* and 3rd, fill in result, finally */ + sp = (dnsc_t*)(ret->dnstxt_txt + p.dnsp_nrr); + for(dns_rewind(&p, qdn), r = 0; dns_nextrr(&p, &rr) > 0; ++r) { + ret->dnstxt_txt[r].txt = sp; + cp = rr.dnsrr_dptr; ep = rr.dnsrr_dend; + while(cp < ep) { + l = *cp++; + memcpy(sp, cp, l); + sp += l; + cp += l; + } + ret->dnstxt_txt[r].len = sp - ret->dnstxt_txt[r].txt; + *sp++ = '\0'; + } + dns_stdrr_finish((struct dns_rr_null *)ret, (char*)sp, &p); + *result = ret; + return 0; +} + +struct dns_query * +dns_submit_txt(struct dns_ctx *ctx, const char *name, int qcls, int flags, + dns_query_txt_fn *cbck, void *data) { + return + dns_submit_p(ctx, name, qcls, DNS_T_TXT, flags, + dns_parse_txt, (dns_query_fn *)cbck, data); +} + +struct dns_rr_txt * +dns_resolve_txt(struct dns_ctx *ctx, const char *name, int qcls, int flags) { + return (struct dns_rr_txt *) + dns_resolve_p(ctx, name, qcls, DNS_T_TXT, flags, dns_parse_txt); +} diff --git a/src/Access/Common/AllowedClientHosts.cpp b/src/Access/Common/AllowedClientHosts.cpp index 85d7065d823..09058a3c045 100644 --- a/src/Access/Common/AllowedClientHosts.cpp +++ b/src/Access/Common/AllowedClientHosts.cpp @@ -110,18 +110,23 @@ namespace } /// Returns the host name by its address. - String getHostByAddress(const IPAddress & address) + std::vector getHostsByAddress(const IPAddress & address) { - String host = DNSResolver::instance().reverseResolve(address); + std::vector hosts = DNSResolver::instance().reverseResolve(address); - /// Check that PTR record is resolved back to client address - if (!isAddressOfHost(address, host)) - throw Exception("Host " + String(host) + " isn't resolved back to " + address.toString(), ErrorCodes::DNS_ERROR); + if (hosts.empty()) + throw Exception(address.toString() + " could not be resolved", ErrorCodes::DNS_ERROR); - return host; + + for (auto & host : hosts) { + /// Check that PTR record is resolved back to client address + if (!isAddressOfHost(address, host)) + throw Exception("Host " + String(host) + " isn't resolved back to " + address.toString(), ErrorCodes::DNS_ERROR); + } + + return hosts; } - void parseLikePatternIfIPSubnet(const String & pattern, IPSubnet & subnet, IPAddress::Family address_family) { size_t slash = pattern.find('/'); @@ -520,20 +525,26 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const return true; /// Check `name_regexps`. - std::optional resolved_host; + std::optional> resolved_hosts; auto check_name_regexp = [&](const String & name_regexp_) { try { if (boost::iequals(name_regexp_, "localhost")) return is_client_local(); - if (!resolved_host) - resolved_host = getHostByAddress(client_v6); - if (resolved_host->empty()) - return false; - Poco::RegularExpression re(name_regexp_); - Poco::RegularExpression::Match match; - return re.match(*resolved_host, match) != 0; + if (!resolved_hosts) { + resolved_hosts = getHostsByAddress(client_address); + } + + for (const auto & host : resolved_hosts.value()) { + Poco::RegularExpression re(name_regexp_); + Poco::RegularExpression::Match match; + if (re.match(host, match) != 0) { + return true; + } + } + + return false; } catch (const Exception & e) { diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 10bdc464ac6..6d4e7813bfa 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -355,6 +355,8 @@ target_link_libraries(clickhouse_common_io ch_contrib::zlib pcg_random Poco::Foundation + ch_contrib::udns + ch_contrib::cpp-dns ) if (TARGET ch_contrib::cpuid) @@ -439,6 +441,14 @@ if (TARGET ch_contrib::avrocpp) dbms_target_link_libraries(PRIVATE ch_contrib::avrocpp) endif () +if (TARGET ch_contrib::udns) + dbms_target_link_libraries(PRIVATE ch_contrib::udns) +endif () + +if (TARGET ch_contrib::cpp-dns) + dbms_target_link_libraries(PRIVATE ch_contrib::cpp-dns) +endif () + if (TARGET OpenSSL::Crypto) dbms_target_link_libraries (PRIVATE OpenSSL::Crypto) target_link_libraries (clickhouse_common_io PRIVATE OpenSSL::Crypto) diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 0616e324b73..d5eebe579b4 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -12,6 +12,8 @@ #include #include #include +#include +#include namespace ProfileEvents { @@ -138,16 +140,27 @@ static DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host) return addresses; } -static String reverseResolveImpl(const Poco::Net::IPAddress & address) +static std::vector reverseResolveImpl(const Poco::Net::IPAddress & address) { - Poco::Net::SocketAddress sock_addr(address, 0); + boost::asio::io_service ioService; + YukiWorkshop::DNSResolver resolver(ioService); - /// Resolve by hand, because Poco::Net::DNS::hostByAddress(...) does getaddrinfo(...) after getnameinfo(...) - char host[1024]; - int err = getnameinfo(sock_addr.addr(), sock_addr.length(), host, sizeof(host), nullptr, 0, NI_NAMEREQD); - if (err) - throw Exception("Cannot getnameinfo(" + address.toString() + "): " + gai_strerror(err), ErrorCodes::DNS_ERROR); - return host; + std::vector ptrRecords; + + resolver.resolve_a4ptr(boost::asio::ip::address_v4::from_string(address.toString()), [&](int err, auto& hosts, auto&, auto&, uint) { + if (err) { + throw Exception("Cannot resolve: " + address.toString() + gai_strerror(err), ErrorCodes::DNS_ERROR); + } + + for (auto &it : hosts) { + ptrRecords.emplace_back(it); + } + + }); + + ioService.run(); + + return ptrRecords; } struct DNSResolver::Impl @@ -235,7 +248,7 @@ std::vector DNSResolver::resolveAddressList(const std: return addresses; } -String DNSResolver::reverseResolve(const Poco::Net::IPAddress & address) +std::vector DNSResolver::reverseResolve(const Poco::Net::IPAddress & address) { if (impl->disable_cache) return reverseResolveImpl(address); diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index fdd9799f96f..d1a8843d234 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -36,8 +36,8 @@ public: std::vector resolveAddressList(const std::string & host, UInt16 port); - /// Accepts host IP and resolves its host name - String reverseResolve(const Poco::Net::IPAddress & address); + /// Accepts host IP and resolves its host names + std::vector reverseResolve(const Poco::Net::IPAddress & address); /// Get this server host name String getHostName(); From e030d49d97af8dce4d9c3470b4bef1936547f454 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 3 Jun 2022 08:27:16 -0300 Subject: [PATCH 052/659] Remove unnecessary CMake comments --- contrib/cpp-dns-cmake/CMakeLists.txt | 18 ------------------ contrib/udns-cmake/CMakeLists.txt | 28 ---------------------------- 2 files changed, 46 deletions(-) diff --git a/contrib/cpp-dns-cmake/CMakeLists.txt b/contrib/cpp-dns-cmake/CMakeLists.txt index 342e189ff36..96669df9e26 100644 --- a/contrib/cpp-dns-cmake/CMakeLists.txt +++ b/contrib/cpp-dns-cmake/CMakeLists.txt @@ -1,29 +1,11 @@ set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/cpp-dns") -# If not, ASIO standalone mode. -#find_path(LIBUDNS_INCLUDE_DIR -# NAME udns.h -# HINTS /opt/local/include) -# -#find_library(lib_udns udns -# PATHS /opt/local/lib) - -#if (lib_udns) -# message("Found libudns!") -#else() -# message(FATAL_ERROR "udns not found") -#endif() - -#find_package(Boost 1.78 COMPONENTS system REQUIRED)#[[]] - set(SRCS "${LIBRARY_DIR}/DNSResolver.cpp") set(HDRS "${LIBRARY_DIR}/DNSResolver.hpp") add_library(_cpp-dns ${SRCS} ${HDRS}) -#message("Libraries: lb: ${LIBRARY_DIR} bid: ${Boost_INCLUDE_DIRS} bl: ${Boost_LIBRARIES} libudns: ${lib_udns}") - target_link_libraries(_cpp-dns boost::system boost::headers_only ch_contrib::udns) target_include_directories(_cpp-dns SYSTEM BEFORE PUBLIC ${LIBRARY_DIR}) diff --git a/contrib/udns-cmake/CMakeLists.txt b/contrib/udns-cmake/CMakeLists.txt index d56be90e11f..8a3e9bc4f8f 100644 --- a/contrib/udns-cmake/CMakeLists.txt +++ b/contrib/udns-cmake/CMakeLists.txt @@ -1,33 +1,5 @@ set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/udns") -#add_library(udns -# udns_dn.c -# udns_dntosp.c -# udns_parse.c -# udns_resolver.c -# udns_init.c -# udns_misc.c -# udns_XtoX.c -# udns_rr_a.c -# udns_rr_ptr.c -# udns_rr_mx.c -# udns_rr_txt.c -# udns_bl.c -# udns_rr_srv.c -# udns_rr_naptr.c -# udns_codes.c -# udns_jran.c -# ) - -#add_library(_udns -# "${LIBRARY_DIR}/udns_dn.c" -# "${LIBRARY_DIR}/dnsget.c" "${LIBRARY_DIR}/ex-rdns.c" "${LIBRARY_DIR}/getopt.c" "${LIBRARY_DIR}/inet_XtoX.c" -# "${LIBRARY_DIR}/rblcheck.c" "${LIBRARY_DIR}/udns_bl.c" "${LIBRARY_DIR}/udns_dntosp.c" -# "${LIBRARY_DIR}/udns_init.c" "${LIBRARY_DIR}/udns_jran.c" "${LIBRARY_DIR}/udns_misc.c" "${LIBRARY_DIR}/udns_parse.c" -# "${LIBRARY_DIR}/udns_resolver.c" "${LIBRARY_DIR}/udns_rr_a.c" "${LIBRARY_DIR}/udns_rr_mx.c" -# "${LIBRARY_DIR}/udns_rr_naptr.c" "${LIBRARY_DIR}/udns_rr_ptr.c" "${LIBRARY_DIR}/udns_rr_srv.c" "${LIBRARY_DIR}/udns_rr_txt.c" -# "${LIBRARY_DIR}/udns_XtoX.c" ) - add_library(_udns "${LIBRARY_DIR}/udns_dn.c" "${LIBRARY_DIR}/udns_dntosp.c" From 78519847a67c3fcf81005976512293df13c8ee21 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 3 Jun 2022 09:00:09 -0300 Subject: [PATCH 053/659] Fix wrong udns submodule --- .gitmodules | 8 +- contrib/udns | 1 + contrib/udns/COPYING.LGPL | 502 ------------- contrib/udns/Makefile.in | 197 ----- contrib/udns/NEWS | 136 ---- contrib/udns/NOTES | 226 ------ contrib/udns/README.md | 1 - contrib/udns/TODO | 59 -- contrib/udns/config.h | 7 - contrib/udns/configure | 166 ----- contrib/udns/configure.lib | 268 ------- contrib/udns/conftest.c | 3 - contrib/udns/conftest.out | 7 - contrib/udns/dnsget.1 | 195 ----- contrib/udns/dnsget.c | 759 ------------------- contrib/udns/ex-rdns.c | 114 --- contrib/udns/getopt.c | 165 ----- contrib/udns/inet_XtoX.c | 327 -------- contrib/udns/rblcheck.1 | 151 ---- contrib/udns/rblcheck.c | 378 ---------- contrib/udns/udns.3 | 1352 ---------------------------------- contrib/udns/udns.h | 778 ------------------- contrib/udns/udns_XtoX.c | 50 -- contrib/udns/udns_bl.c | 160 ---- contrib/udns/udns_codes.c | 199 ----- contrib/udns/udns_dn.c | 379 ---------- contrib/udns/udns_dntosp.c | 30 - contrib/udns/udns_init.c | 231 ------ contrib/udns/udns_jran.c | 52 -- contrib/udns/udns_misc.c | 67 -- contrib/udns/udns_parse.c | 169 ----- contrib/udns/udns_resolver.c | 1323 --------------------------------- contrib/udns/udns_rr_a.c | 123 ---- contrib/udns/udns_rr_mx.c | 91 --- contrib/udns/udns_rr_naptr.c | 128 ---- contrib/udns/udns_rr_ptr.c | 109 --- contrib/udns/udns_rr_srv.c | 155 ---- contrib/udns/udns_rr_txt.c | 98 --- 38 files changed, 5 insertions(+), 9159 deletions(-) create mode 160000 contrib/udns delete mode 100644 contrib/udns/COPYING.LGPL delete mode 100644 contrib/udns/Makefile.in delete mode 100644 contrib/udns/NEWS delete mode 100644 contrib/udns/NOTES delete mode 100644 contrib/udns/README.md delete mode 100644 contrib/udns/TODO delete mode 100644 contrib/udns/config.h delete mode 100755 contrib/udns/configure delete mode 100644 contrib/udns/configure.lib delete mode 100644 contrib/udns/conftest.c delete mode 100644 contrib/udns/conftest.out delete mode 100644 contrib/udns/dnsget.1 delete mode 100644 contrib/udns/dnsget.c delete mode 100644 contrib/udns/ex-rdns.c delete mode 100644 contrib/udns/getopt.c delete mode 100644 contrib/udns/inet_XtoX.c delete mode 100644 contrib/udns/rblcheck.1 delete mode 100644 contrib/udns/rblcheck.c delete mode 100644 contrib/udns/udns.3 delete mode 100644 contrib/udns/udns.h delete mode 100644 contrib/udns/udns_XtoX.c delete mode 100644 contrib/udns/udns_bl.c delete mode 100644 contrib/udns/udns_codes.c delete mode 100644 contrib/udns/udns_dn.c delete mode 100644 contrib/udns/udns_dntosp.c delete mode 100644 contrib/udns/udns_init.c delete mode 100644 contrib/udns/udns_jran.c delete mode 100644 contrib/udns/udns_misc.c delete mode 100644 contrib/udns/udns_parse.c delete mode 100644 contrib/udns/udns_resolver.c delete mode 100644 contrib/udns/udns_rr_a.c delete mode 100644 contrib/udns/udns_rr_mx.c delete mode 100644 contrib/udns/udns_rr_naptr.c delete mode 100644 contrib/udns/udns_rr_ptr.c delete mode 100644 contrib/udns/udns_rr_srv.c delete mode 100644 contrib/udns/udns_rr_txt.c diff --git a/.gitmodules b/.gitmodules index 7d0ca61ebe2..2bdc01c728d 100644 --- a/.gitmodules +++ b/.gitmodules @@ -274,9 +274,9 @@ [submodule "contrib/base-x"] path = contrib/base-x url = https://github.com/ClickHouse/base-x.git -[submodule "contrib/cpp-dns"] - path = contrib/cpp-dns - url = git@github.com:YukiWorkshop/cpp-dns.git [submodule "contrib/udns"] path = contrib/udns - url = https://github.com/arthurpassos/udns.git \ No newline at end of file + url = https://github.com/arthurpassos/udns.git +[submodule "contrib/cpp-dns"] + path = contrib/cpp-dns + url = git@github.com:YukiWorkshop/cpp-dns.git \ No newline at end of file diff --git a/contrib/udns b/contrib/udns new file mode 160000 index 00000000000..f535fbe9b39 --- /dev/null +++ b/contrib/udns @@ -0,0 +1 @@ +Subproject commit f535fbe9b39f6bbbf91475834cfefef9c7edab75 diff --git a/contrib/udns/COPYING.LGPL b/contrib/udns/COPYING.LGPL deleted file mode 100644 index 4362b49151d..00000000000 --- a/contrib/udns/COPYING.LGPL +++ /dev/null @@ -1,502 +0,0 @@ - GNU LESSER GENERAL PUBLIC LICENSE - Version 2.1, February 1999 - - Copyright (C) 1991, 1999 Free Software Foundation, Inc. - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA - Everyone is permitted to copy and distribute verbatim copies - of this license document, but changing it is not allowed. - -[This is the first released version of the Lesser GPL. It also counts - as the successor of the GNU Library Public License, version 2, hence - the version number 2.1.] - - Preamble - - The licenses for most software are designed to take away your -freedom to share and change it. By contrast, the GNU General Public -Licenses are intended to guarantee your freedom to share and change -free software--to make sure the software is free for all its users. - - This license, the Lesser General Public License, applies to some -specially designated software packages--typically libraries--of the -Free Software Foundation and other authors who decide to use it. You -can use it too, but we suggest you first think carefully about whether -this license or the ordinary General Public License is the better -strategy to use in any particular case, based on the explanations below. - - When we speak of free software, we are referring to freedom of use, -not price. Our General Public Licenses are designed to make sure that -you have the freedom to distribute copies of free software (and charge -for this service if you wish); that you receive source code or can get -it if you want it; that you can change the software and use pieces of -it in new free programs; and that you are informed that you can do -these things. - - To protect your rights, we need to make restrictions that forbid -distributors to deny you these rights or to ask you to surrender these -rights. These restrictions translate to certain responsibilities for -you if you distribute copies of the library or if you modify it. - - For example, if you distribute copies of the library, whether gratis -or for a fee, you must give the recipients all the rights that we gave -you. You must make sure that they, too, receive or can get the source -code. If you link other code with the library, you must provide -complete object files to the recipients, so that they can relink them -with the library after making changes to the library and recompiling -it. And you must show them these terms so they know their rights. - - We protect your rights with a two-step method: (1) we copyright the -library, and (2) we offer you this license, which gives you legal -permission to copy, distribute and/or modify the library. - - To protect each distributor, we want to make it very clear that -there is no warranty for the free library. Also, if the library is -modified by someone else and passed on, the recipients should know -that what they have is not the original version, so that the original -author's reputation will not be affected by problems that might be -introduced by others. - - Finally, software patents pose a constant threat to the existence of -any free program. We wish to make sure that a company cannot -effectively restrict the users of a free program by obtaining a -restrictive license from a patent holder. Therefore, we insist that -any patent license obtained for a version of the library must be -consistent with the full freedom of use specified in this license. - - Most GNU software, including some libraries, is covered by the -ordinary GNU General Public License. This license, the GNU Lesser -General Public License, applies to certain designated libraries, and -is quite different from the ordinary General Public License. We use -this license for certain libraries in order to permit linking those -libraries into non-free programs. - - When a program is linked with a library, whether statically or using -a shared library, the combination of the two is legally speaking a -combined work, a derivative of the original library. The ordinary -General Public License therefore permits such linking only if the -entire combination fits its criteria of freedom. The Lesser General -Public License permits more lax criteria for linking other code with -the library. - - We call this license the "Lesser" General Public License because it -does Less to protect the user's freedom than the ordinary General -Public License. It also provides other free software developers Less -of an advantage over competing non-free programs. These disadvantages -are the reason we use the ordinary General Public License for many -libraries. However, the Lesser license provides advantages in certain -special circumstances. - - For example, on rare occasions, there may be a special need to -encourage the widest possible use of a certain library, so that it becomes -a de-facto standard. To achieve this, non-free programs must be -allowed to use the library. A more frequent case is that a free -library does the same job as widely used non-free libraries. In this -case, there is little to gain by limiting the free library to free -software only, so we use the Lesser General Public License. - - In other cases, permission to use a particular library in non-free -programs enables a greater number of people to use a large body of -free software. For example, permission to use the GNU C Library in -non-free programs enables many more people to use the whole GNU -operating system, as well as its variant, the GNU/Linux operating -system. - - Although the Lesser General Public License is Less protective of the -users' freedom, it does ensure that the user of a program that is -linked with the Library has the freedom and the wherewithal to run -that program using a modified version of the Library. - - The precise terms and conditions for copying, distribution and -modification follow. Pay close attention to the difference between a -"work based on the library" and a "work that uses the library". The -former contains code derived from the library, whereas the latter must -be combined with the library in order to run. - - GNU LESSER GENERAL PUBLIC LICENSE - TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION - - 0. This License Agreement applies to any software library or other -program which contains a notice placed by the copyright holder or -other authorized party saying it may be distributed under the terms of -this Lesser General Public License (also called "this License"). -Each licensee is addressed as "you". - - A "library" means a collection of software functions and/or data -prepared so as to be conveniently linked with application programs -(which use some of those functions and data) to form executables. - - The "Library", below, refers to any such software library or work -which has been distributed under these terms. A "work based on the -Library" means either the Library or any derivative work under -copyright law: that is to say, a work containing the Library or a -portion of it, either verbatim or with modifications and/or translated -straightforwardly into another language. (Hereinafter, translation is -included without limitation in the term "modification".) - - "Source code" for a work means the preferred form of the work for -making modifications to it. For a library, complete source code means -all the source code for all modules it contains, plus any associated -interface definition files, plus the scripts used to control compilation -and installation of the library. - - Activities other than copying, distribution and modification are not -covered by this License; they are outside its scope. The act of -running a program using the Library is not restricted, and output from -such a program is covered only if its contents constitute a work based -on the Library (independent of the use of the Library in a tool for -writing it). Whether that is true depends on what the Library does -and what the program that uses the Library does. - - 1. You may copy and distribute verbatim copies of the Library's -complete source code as you receive it, in any medium, provided that -you conspicuously and appropriately publish on each copy an -appropriate copyright notice and disclaimer of warranty; keep intact -all the notices that refer to this License and to the absence of any -warranty; and distribute a copy of this License along with the -Library. - - You may charge a fee for the physical act of transferring a copy, -and you may at your option offer warranty protection in exchange for a -fee. - - 2. You may modify your copy or copies of the Library or any portion -of it, thus forming a work based on the Library, and copy and -distribute such modifications or work under the terms of Section 1 -above, provided that you also meet all of these conditions: - - a) The modified work must itself be a software library. - - b) You must cause the files modified to carry prominent notices - stating that you changed the files and the date of any change. - - c) You must cause the whole of the work to be licensed at no - charge to all third parties under the terms of this License. - - d) If a facility in the modified Library refers to a function or a - table of data to be supplied by an application program that uses - the facility, other than as an argument passed when the facility - is invoked, then you must make a good faith effort to ensure that, - in the event an application does not supply such function or - table, the facility still operates, and performs whatever part of - its purpose remains meaningful. - - (For example, a function in a library to compute square roots has - a purpose that is entirely well-defined independent of the - application. Therefore, Subsection 2d requires that any - application-supplied function or table used by this function must - be optional: if the application does not supply it, the square - root function must still compute square roots.) - -These requirements apply to the modified work as a whole. If -identifiable sections of that work are not derived from the Library, -and can be reasonably considered independent and separate works in -themselves, then this License, and its terms, do not apply to those -sections when you distribute them as separate works. But when you -distribute the same sections as part of a whole which is a work based -on the Library, the distribution of the whole must be on the terms of -this License, whose permissions for other licensees extend to the -entire whole, and thus to each and every part regardless of who wrote -it. - -Thus, it is not the intent of this section to claim rights or contest -your rights to work written entirely by you; rather, the intent is to -exercise the right to control the distribution of derivative or -collective works based on the Library. - -In addition, mere aggregation of another work not based on the Library -with the Library (or with a work based on the Library) on a volume of -a storage or distribution medium does not bring the other work under -the scope of this License. - - 3. You may opt to apply the terms of the ordinary GNU General Public -License instead of this License to a given copy of the Library. To do -this, you must alter all the notices that refer to this License, so -that they refer to the ordinary GNU General Public License, version 2, -instead of to this License. (If a newer version than version 2 of the -ordinary GNU General Public License has appeared, then you can specify -that version instead if you wish.) Do not make any other change in -these notices. - - Once this change is made in a given copy, it is irreversible for -that copy, so the ordinary GNU General Public License applies to all -subsequent copies and derivative works made from that copy. - - This option is useful when you wish to copy part of the code of -the Library into a program that is not a library. - - 4. You may copy and distribute the Library (or a portion or -derivative of it, under Section 2) in object code or executable form -under the terms of Sections 1 and 2 above provided that you accompany -it with the complete corresponding machine-readable source code, which -must be distributed under the terms of Sections 1 and 2 above on a -medium customarily used for software interchange. - - If distribution of object code is made by offering access to copy -from a designated place, then offering equivalent access to copy the -source code from the same place satisfies the requirement to -distribute the source code, even though third parties are not -compelled to copy the source along with the object code. - - 5. A program that contains no derivative of any portion of the -Library, but is designed to work with the Library by being compiled or -linked with it, is called a "work that uses the Library". Such a -work, in isolation, is not a derivative work of the Library, and -therefore falls outside the scope of this License. - - However, linking a "work that uses the Library" with the Library -creates an executable that is a derivative of the Library (because it -contains portions of the Library), rather than a "work that uses the -library". The executable is therefore covered by this License. -Section 6 states terms for distribution of such executables. - - When a "work that uses the Library" uses material from a header file -that is part of the Library, the object code for the work may be a -derivative work of the Library even though the source code is not. -Whether this is true is especially significant if the work can be -linked without the Library, or if the work is itself a library. The -threshold for this to be true is not precisely defined by law. - - If such an object file uses only numerical parameters, data -structure layouts and accessors, and small macros and small inline -functions (ten lines or less in length), then the use of the object -file is unrestricted, regardless of whether it is legally a derivative -work. (Executables containing this object code plus portions of the -Library will still fall under Section 6.) - - Otherwise, if the work is a derivative of the Library, you may -distribute the object code for the work under the terms of Section 6. -Any executables containing that work also fall under Section 6, -whether or not they are linked directly with the Library itself. - - 6. As an exception to the Sections above, you may also combine or -link a "work that uses the Library" with the Library to produce a -work containing portions of the Library, and distribute that work -under terms of your choice, provided that the terms permit -modification of the work for the customer's own use and reverse -engineering for debugging such modifications. - - You must give prominent notice with each copy of the work that the -Library is used in it and that the Library and its use are covered by -this License. You must supply a copy of this License. If the work -during execution displays copyright notices, you must include the -copyright notice for the Library among them, as well as a reference -directing the user to the copy of this License. Also, you must do one -of these things: - - a) Accompany the work with the complete corresponding - machine-readable source code for the Library including whatever - changes were used in the work (which must be distributed under - Sections 1 and 2 above); and, if the work is an executable linked - with the Library, with the complete machine-readable "work that - uses the Library", as object code and/or source code, so that the - user can modify the Library and then relink to produce a modified - executable containing the modified Library. (It is understood - that the user who changes the contents of definitions files in the - Library will not necessarily be able to recompile the application - to use the modified definitions.) - - b) Use a suitable shared library mechanism for linking with the - Library. A suitable mechanism is one that (1) uses at run time a - copy of the library already present on the user's computer system, - rather than copying library functions into the executable, and (2) - will operate properly with a modified version of the library, if - the user installs one, as long as the modified version is - interface-compatible with the version that the work was made with. - - c) Accompany the work with a written offer, valid for at - least three years, to give the same user the materials - specified in Subsection 6a, above, for a charge no more - than the cost of performing this distribution. - - d) If distribution of the work is made by offering access to copy - from a designated place, offer equivalent access to copy the above - specified materials from the same place. - - e) Verify that the user has already received a copy of these - materials or that you have already sent this user a copy. - - For an executable, the required form of the "work that uses the -Library" must include any data and utility programs needed for -reproducing the executable from it. However, as a special exception, -the materials to be distributed need not include anything that is -normally distributed (in either source or binary form) with the major -components (compiler, kernel, and so on) of the operating system on -which the executable runs, unless that component itself accompanies -the executable. - - It may happen that this requirement contradicts the license -restrictions of other proprietary libraries that do not normally -accompany the operating system. Such a contradiction means you cannot -use both them and the Library together in an executable that you -distribute. - - 7. You may place library facilities that are a work based on the -Library side-by-side in a single library together with other library -facilities not covered by this License, and distribute such a combined -library, provided that the separate distribution of the work based on -the Library and of the other library facilities is otherwise -permitted, and provided that you do these two things: - - a) Accompany the combined library with a copy of the same work - based on the Library, uncombined with any other library - facilities. This must be distributed under the terms of the - Sections above. - - b) Give prominent notice with the combined library of the fact - that part of it is a work based on the Library, and explaining - where to find the accompanying uncombined form of the same work. - - 8. You may not copy, modify, sublicense, link with, or distribute -the Library except as expressly provided under this License. Any -attempt otherwise to copy, modify, sublicense, link with, or -distribute the Library is void, and will automatically terminate your -rights under this License. However, parties who have received copies, -or rights, from you under this License will not have their licenses -terminated so long as such parties remain in full compliance. - - 9. You are not required to accept this License, since you have not -signed it. However, nothing else grants you permission to modify or -distribute the Library or its derivative works. These actions are -prohibited by law if you do not accept this License. Therefore, by -modifying or distributing the Library (or any work based on the -Library), you indicate your acceptance of this License to do so, and -all its terms and conditions for copying, distributing or modifying -the Library or works based on it. - - 10. Each time you redistribute the Library (or any work based on the -Library), the recipient automatically receives a license from the -original licensor to copy, distribute, link with or modify the Library -subject to these terms and conditions. You may not impose any further -restrictions on the recipients' exercise of the rights granted herein. -You are not responsible for enforcing compliance by third parties with -this License. - - 11. If, as a consequence of a court judgment or allegation of patent -infringement or for any other reason (not limited to patent issues), -conditions are imposed on you (whether by court order, agreement or -otherwise) that contradict the conditions of this License, they do not -excuse you from the conditions of this License. If you cannot -distribute so as to satisfy simultaneously your obligations under this -License and any other pertinent obligations, then as a consequence you -may not distribute the Library at all. For example, if a patent -license would not permit royalty-free redistribution of the Library by -all those who receive copies directly or indirectly through you, then -the only way you could satisfy both it and this License would be to -refrain entirely from distribution of the Library. - -If any portion of this section is held invalid or unenforceable under any -particular circumstance, the balance of the section is intended to apply, -and the section as a whole is intended to apply in other circumstances. - -It is not the purpose of this section to induce you to infringe any -patents or other property right claims or to contest validity of any -such claims; this section has the sole purpose of protecting the -integrity of the free software distribution system which is -implemented by public license practices. Many people have made -generous contributions to the wide range of software distributed -through that system in reliance on consistent application of that -system; it is up to the author/donor to decide if he or she is willing -to distribute software through any other system and a licensee cannot -impose that choice. - -This section is intended to make thoroughly clear what is believed to -be a consequence of the rest of this License. - - 12. If the distribution and/or use of the Library is restricted in -certain countries either by patents or by copyrighted interfaces, the -original copyright holder who places the Library under this License may add -an explicit geographical distribution limitation excluding those countries, -so that distribution is permitted only in or among countries not thus -excluded. In such case, this License incorporates the limitation as if -written in the body of this License. - - 13. The Free Software Foundation may publish revised and/or new -versions of the Lesser General Public License from time to time. -Such new versions will be similar in spirit to the present version, -but may differ in detail to address new problems or concerns. - -Each version is given a distinguishing version number. If the Library -specifies a version number of this License which applies to it and -"any later version", you have the option of following the terms and -conditions either of that version or of any later version published by -the Free Software Foundation. If the Library does not specify a -license version number, you may choose any version ever published by -the Free Software Foundation. - - 14. If you wish to incorporate parts of the Library into other free -programs whose distribution conditions are incompatible with these, -write to the author to ask for permission. For software which is -copyrighted by the Free Software Foundation, write to the Free -Software Foundation; we sometimes make exceptions for this. Our -decision will be guided by the two goals of preserving the free status -of all derivatives of our free software and of promoting the sharing -and reuse of software generally. - - NO WARRANTY - - 15. BECAUSE THE LIBRARY IS LICENSED FREE OF CHARGE, THERE IS NO -WARRANTY FOR THE LIBRARY, TO THE EXTENT PERMITTED BY APPLICABLE LAW. -EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR -OTHER PARTIES PROVIDE THE LIBRARY "AS IS" WITHOUT WARRANTY OF ANY -KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE -IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR -PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE -LIBRARY IS WITH YOU. SHOULD THE LIBRARY PROVE DEFECTIVE, YOU ASSUME -THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION. - - 16. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN -WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY -AND/OR REDISTRIBUTE THE LIBRARY AS PERMITTED ABOVE, BE LIABLE TO YOU -FOR DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR -CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE -LIBRARY (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING -RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A -FAILURE OF THE LIBRARY TO OPERATE WITH ANY OTHER SOFTWARE), EVEN IF -SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH -DAMAGES. - - END OF TERMS AND CONDITIONS - - How to Apply These Terms to Your New Libraries - - If you develop a new library, and you want it to be of the greatest -possible use to the public, we recommend making it free software that -everyone can redistribute and change. You can do so by permitting -redistribution under these terms (or, alternatively, under the terms of the -ordinary General Public License). - - To apply these terms, attach the following notices to the library. It is -safest to attach them to the start of each source file to most effectively -convey the exclusion of warranty; and each file should have at least the -"copyright" line and a pointer to where the full notice is found. - - - Copyright (C) - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library; if not, write to the Free Software - Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA - -Also add information on how to contact you by electronic and paper mail. - -You should also get your employer (if you work as a programmer) or your -school, if any, to sign a "copyright disclaimer" for the library, if -necessary. Here is a sample; alter the names: - - Yoyodyne, Inc., hereby disclaims all copyright interest in the - library `Frob' (a library for tweaking knobs) written by James Random Hacker. - - , 1 April 1990 - Ty Coon, President of Vice - -That's all there is to it! diff --git a/contrib/udns/Makefile.in b/contrib/udns/Makefile.in deleted file mode 100644 index ec085206655..00000000000 --- a/contrib/udns/Makefile.in +++ /dev/null @@ -1,197 +0,0 @@ -#! /usr/bin/make -rf -# Makefile.in -# libudns Makefile -# -# Copyright (C) 2005 Michael Tokarev -# This file is part of UDNS library, an async DNS stub resolver. -# -# This library is free software; you can redistribute it and/or -# modify it under the terms of the GNU Lesser General Public -# License as published by the Free Software Foundation; either -# version 2.1 of the License, or (at your option) any later version. -# -# This library is distributed in the hope that it will be useful, -# but WITHOUT ANY WARRANTY; without even the implied warranty of -# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU -# Lesser General Public License for more details. -# -# You should have received a copy of the GNU Lesser General Public -# License along with this library, in file named COPYING.LGPL; if not, -# write to the Free Software Foundation, Inc., 59 Temple Place, -# Suite 330, Boston, MA 02111-1307 USA - -NAME = udns -VERS = 0.4 -SOVER = 0 - -SRCS = udns_dn.c udns_dntosp.c udns_parse.c udns_resolver.c udns_init.c \ - udns_misc.c udns_XtoX.c \ - udns_rr_a.c udns_rr_ptr.c udns_rr_mx.c udns_rr_txt.c udns_bl.c \ - udns_rr_srv.c udns_rr_naptr.c udns_codes.c udns_jran.c -USRCS = dnsget.c rblcheck.c ex-rdns.c -DIST = COPYING.LGPL udns.h udns.3 dnsget.1 rblcheck.1 $(SRCS) $(USRCS) \ - NEWS TODO NOTES Makefile.in configure configure.lib \ - inet_XtoX.c getopt.c - -OBJS = $(SRCS:.c=.o) $(GEN:.c=.o) -LIB = lib$(NAME).a -LIBFL = -L. -l$(NAME) - -SOBJS = $(OBJS:.o=.lo) -SOLIB = lib$(NAME)_s.so -SOLIBV = lib$(NAME).so.$(SOVER) -SOLIBFL= -L. -l$(NAME)_s - -UTILS = $(USRCS:.c=) -UOBJS = $(USRCS:.c=.o) -SOUTILS = $(USRCS:.c=_s) - -NAMEPFX = $(NAME)-$(VERS) - -CC = @CC@ -CFLAGS = @CFLAGS@ -CDEFS = @CDEFS@ -LD = @LD@ -LDFLAGS = @LDFLAGS@ -LIBS = @LIBS@ -LDSHARED = $(LD) -shared -PICFLAGS = -fPIC -AWK = awk -TAR = tar - -all: static - -.SUFFIXES: .c .o .lo - -static: $(LIB) $(UTILS) -staticlib: $(LIB) -$(LIB): $(OBJS) - -rm -f $@ - $(AR) rv $@ $(OBJS) -.c.o: - $(CC) $(CFLAGS) $(CDEFS) -c $< - -shared: $(SOLIBV) $(SOUTILS) -sharedlib: $(SOLIBV) - -$(SOLIBV): $(SOBJS) - $(LDSHARED) -Wl,--soname,$(SOLIBV) -o $@ $(SOBJS) $(LDFLAGS) $(LIBS) -$(SOLIB): $(SOLIBV) - rm -f $@ - ln -s $(SOLIBV) $@ -.c.lo: - $(CC) $(CFLAGS) $(PICFLAGS) $(CDEFS) -o $@ -c $< - -# udns_codes.c is generated from udns.h -udns_codes.c: udns.h - @echo Generating $@ - @set -e; exec >$@.tmp; \ - set T type C class R rcode; \ - echo "/* Automatically generated. */"; \ - echo "#include \"udns.h\""; \ - while [ "$$1" ]; do \ - echo; \ - echo "const struct dns_nameval dns_$${2}tab[] = {"; \ - $(AWK) "/^ DNS_$${1}_[A-Z0-9_]+[ ]*=/ \ - { printf \" {%s,\\\"%s\\\"},\\n\", \$$1, substr(\$$1,7) }" \ - udns.h ; \ - echo " {0,0}};"; \ - echo "const char *dns_$${2}name(enum dns_$${2} code) {"; \ - echo " static char nm[20];"; \ - echo " switch(code) {"; \ - $(AWK) "BEGIN{i=0} \ - /^ DNS_$${1}_[A-Z0-9_]+[ ]*=/ \ - {printf \" case %s: return dns_$${2}tab[%d].name;\\n\",\$$1,i++}\ - " udns.h ; \ - echo " }"; \ - echo " return _dns_format_code(nm,\"$$2\",code);"; \ - echo "}"; \ - shift 2; \ - done - @mv $@.tmp $@ - -udns.3.html: udns.3 - groff -man -Thtml udns.3 > $@.tmp - mv $@.tmp $@ - -dist: $(NAMEPFX).tar.gz -$(NAMEPFX).tar.gz: $(DIST) - $(TAR) -cv -f $@ -z --transform 's|^|$(NAMEPFX)/|' $(DIST) - -subdist: - cp -p $(DIST) $(TARGET)/ - -clean: - rm -f $(OBJS) - rm -f $(SOBJS) - rm -f $(UOBJS) - rm -f config.log -distclean: clean - rm -f $(LIB) $(SOLIB) $(SOLIBV) udns.3.html - rm -f $(UTILS) $(SOUTILS) - rm -f config.status config.h Makefile - - -Makefile: configure configure.lib Makefile.in - ./configure - @echo - @echo Please rerun make >&2 - @exit 1 - -.PHONY: all static staticlib shared sharedlib dist clean distclean subdist \ - depend dep deps - -depend dep deps: $(SRCS) $(USRC) - @echo Generating deps for: - @echo \ $(SRCS) - @echo \ $(USRCS) - @sed '/^# depend/q' Makefile.in > Makefile.tmp - @set -e; \ - for f in $(SRCS) $(USRCS); do \ - echo $${f%.c}.o $${f%.c}.lo: $$f \ - `sed -n 's/^#[ ]*include[ ]*"\(.*\)".*/\1/p' $$f`; \ - done >> Makefile.tmp; \ - for f in $(USRCS:.c=.o); do \ - echo "$${f%.?}: $$f \$$(LIB)"; \ - echo " \$$(LD) \$$(LDLAGS) -o \$$@ $$f \$$(LIBFL) \$$(LIBS)"; \ - echo "$${f%.?}_s: $$f \$$(SOLIB)"; \ - echo " \$$(LD) \$$(LDFLAGS) -o \$$@ $$f \$$(SOLIBFL)"; \ - done >> Makefile.tmp ; \ - if cmp Makefile.tmp Makefile.in >/dev/null 2>&1 ; then \ - echo Makefile.in unchanged; rm -f Makefile.tmp; \ - else \ - echo Updating Makfile.in; mv -f Makefile.tmp Makefile.in ; \ - fi - -# depend -udns_dn.o udns_dn.lo: udns_dn.c udns.h -udns_dntosp.o udns_dntosp.lo: udns_dntosp.c udns.h -udns_parse.o udns_parse.lo: udns_parse.c udns.h -udns_resolver.o udns_resolver.lo: udns_resolver.c config.h udns.h -udns_init.o udns_init.lo: udns_init.c config.h udns.h -udns_misc.o udns_misc.lo: udns_misc.c udns.h -udns_XtoX.o udns_XtoX.lo: udns_XtoX.c config.h udns.h inet_XtoX.c -udns_rr_a.o udns_rr_a.lo: udns_rr_a.c udns.h -udns_rr_ptr.o udns_rr_ptr.lo: udns_rr_ptr.c udns.h -udns_rr_mx.o udns_rr_mx.lo: udns_rr_mx.c udns.h -udns_rr_txt.o udns_rr_txt.lo: udns_rr_txt.c udns.h -udns_bl.o udns_bl.lo: udns_bl.c udns.h -udns_rr_srv.o udns_rr_srv.lo: udns_rr_srv.c udns.h -udns_rr_naptr.o udns_rr_naptr.lo: udns_rr_naptr.c udns.h -udns_codes.o udns_codes.lo: udns_codes.c udns.h -udns_jran.o udns_jran.lo: udns_jran.c udns.h -dnsget.o dnsget.lo: dnsget.c config.h udns.h getopt.c -rblcheck.o rblcheck.lo: rblcheck.c config.h udns.h getopt.c -ex-rdns.o ex-rdns.lo: ex-rdns.c udns.h -dnsget: dnsget.o $(LIB) - $(LD) $(LDLAGS) -o $@ dnsget.o $(LIBFL) $(LIBS) -dnsget_s: dnsget.o $(SOLIB) - $(LD) $(LDFLAGS) -o $@ dnsget.o $(SOLIBFL) -rblcheck: rblcheck.o $(LIB) - $(LD) $(LDLAGS) -o $@ rblcheck.o $(LIBFL) $(LIBS) -rblcheck_s: rblcheck.o $(SOLIB) - $(LD) $(LDFLAGS) -o $@ rblcheck.o $(SOLIBFL) -ex-rdns: ex-rdns.o $(LIB) - $(LD) $(LDLAGS) -o $@ ex-rdns.o $(LIBFL) $(LIBS) -ex-rdns_s: ex-rdns.o $(SOLIB) - $(LD) $(LDFLAGS) -o $@ ex-rdns.o $(SOLIBFL) diff --git a/contrib/udns/NEWS b/contrib/udns/NEWS deleted file mode 100644 index 88aff6fa420..00000000000 --- a/contrib/udns/NEWS +++ /dev/null @@ -1,136 +0,0 @@ -NEWS -User-visible changes in udns library. Recent changes on top. - -0.4 (Jan 2014) - - - bugfix: fix a bug in new list code introduced in 0.3 - - portability: use $(LD)/$(LDFLAGS)/$(LIBS) - -0.3 (Jan 2014) - - - bugfix: refactor double-linked list implementation in udns_resolver.c - (internal to the library) to be more strict-aliasing-friendly, because - old code were miscompiled by gcc. - - - bugfix: forgotten strdup() in rblcheck - -0.2 (Dec 2011) - - - bugfix: SRV RR handling: fix domain name parsing and crash in case - if no port is specified on input for SRV record query - - - (trivial api) dns_set_opts() now returns number of unrecognized - options instead of always returning 0 - - - dnsget: combine -f and -o options in dnsget (and stop documenting -f), - and report unknown/invalid -o options (and error out) - - - dnsget: pretty-print SSHFP RRs - - 0.1 (Dec 2010) - - - bugfix: udns_new(old) - when actually cloning another context - - makes the new context referencing memory from old, which leads - to crashes when old is modified later - - - use random queue IDs (the 16bit qID) in queries instead of sequentional - ones, based on simple pseudo-random RNG by Bob Jenkins (udns_jran.[ch]). - Some people believe that this improves security (CVE-2008-1447). I'm - still not convinced (see comments in udns_resolver.c), but it isn't - difficult to add after all. - - - deprecate dns_random16() function which was declared in udns.h - (not anymore) but never documented. In order to keep ABI compatible - it is still exported. - - - library has a way now to set query flags (DNS_SET_DO; DNS_SET_CD). - - - dnsget now prints non-printable chars in all strings in DNS RRs using - decimal escape sequences (\%03u) instead of hexadecimal (\%02x) when - before - other DNS software does it like this. - - - recognize a few more record types in dnsget, notable some DNSSEC RRs; - add -f option for dnsget to set query flags. - - - udns is not a Debian native package anymore (was a wrong idea) - -0.0.9 (16 Jan 2007) - - - incompat: minor API changes in dns_init() &friends. dns_init() - now requires extra `struct dns_ctx *' argument. Not bumped - soversion yet - I only expect one "release" with this change. - - - many small bugfixes, here and there - - - more robust FORMERR replies handling - not only such replies are now - recognized, but udns retries queries without EDNS0 extensions if tried - with, but server reported FORMERR - - - portability changes, udns now includes getopt() implementation fo - the systems lacking it (mostly windows), and dns_ntop()&dns_pton(), - which are either just wrappers for system functions or reimplementations. - - - build is now based on autoconf-like configuration - - - NAPTR (RFC3403) RR decoding support - - - new file NOTES which complements TODO somewhat, and includes some - important shortcomings - - - many internal cleanups, including some preparations for better error - recovery, security and robustness (and thus API changes) - - - removed some #defines which are now unused (like DNS_MAXSRCH) - - - changed WIN32 to WINDOWS everywhere in preprocessor tests, - to be able to build it on win64 as well - -0.0.8 (12 Sep 2005) - - - added SRV records (rfc2782) parsing, - thanks to Thadeu Lima de Souza Cascardo for implementation. - - - bugfixes: - o use uninitialized value when no reply, library died with assertion: - assert((status < 0 && result == 0) || (status >= 0 && result != 0)). - o on some OSes, struct sockaddr_in has additional fields, so - memcmp'ing two sockaddresses does not work. - - - rblcheck(.1) - -0.0.7 (20 Apr 2005) - - - dnsget.1 manpage and several enhancements to dnsget. - - - allow nameserver names for -n option of dnsget. - - - API change: all dns_submit*() routines now does not expect - last `now' argument, since requests aren't sent immediately - anymore. - - - API change: different application timer callback mechanism. - Udns now uses single per-context timer instead of per-query. - - - don't assume DNS replies only contain backward DN pointers, - allow forward pointers too. Change parsing API. - - - debianize - -0.0.6 (08 Apr 2005) - - - use double sorted list for requests (sorted by deadline). - This should significantly speed up timeout processing for - large number of requests. - - - changed debugging interface, so it is finally useable - (still not documented). - - - dnsget routine is now Officially Useable, and sometimes - even more useable than `host' from BIND distribution - (and sometimes not - dnsget does not have -C option - and TCP mode) - - - Debian packaging in debian/ -- udns is now maintained as a - native Debian package. - - - alot (and I really mean alot) of code cleanups all over. diff --git a/contrib/udns/NOTES b/contrib/udns/NOTES deleted file mode 100644 index b99a077bb22..00000000000 --- a/contrib/udns/NOTES +++ /dev/null @@ -1,226 +0,0 @@ -Assorted notes about udns (library). - -UDP-only mode -~~~~~~~~~~~~~ - -First of all, since udns is (currently) UDP-only, there are some -shortcomings. - -It assumes that a reply will fit into a UDP buffer. With adoption of EDNS0, -and general robustness of IP stacks, in most cases it's not an issue. But -in some cases there may be problems: - - - if an RRset is "very large" so it does not fit even in buffer of size - requested by the library (current default is 4096; some servers limits - it further), we will not see the reply, or will only see "damaged" - reply (depending on the server). - - - many DNS servers ignores EDNS0 option requests. In this case, no matter - which buffer size udns library will request, such servers reply is limited - to 512 bytes (standard pre-EDNS0 DNS packet size). (Udns falls back to - non-EDNO0 query if EDNS0-enabled one received FORMERR or NOTIMPL error). - -The problem is that with this, udns currently will not consider replies with -TC (truncation) bit set, and will treat such replies the same way as it -treats SERVFAIL replies, thus trying next server, or temp-failing the query -if no more servers to try. In other words, if the reply is really large, or -if the servers you're using don't support EDNS0, your application will be -unable to resolve a given name. - -Yet it's not common situation - in practice, it's very rare. - -Implementing TCP mode isn't difficult, but it complicates API significantly. -Currently udns uses only single UDP socket (or - maybe in the future - two, -see below), but in case of TCP, it will need to open and close sockets for -TCP connections left and right, and that have to be integrated into an -application's event loop in an easy and efficient way. Plus all the -timeouts - different for connect(), write, and several stages of read. - -IPv6 vs IPv4 usage -~~~~~~~~~~~~~~~~~~ - -This is only relevant for nameservers reachable over IPv6, NOT for IPv6 -queries. I.e., if you've IPv6 addresses in 'nameservers' line in your -/etc/resolv.conf file. Even more: if you have BOTH IPv6 AND IPv4 addresses -there. Or pass them to udns initialization routines. - -Since udns uses a single UDP socket to communicate with all nameservers, -it should support both v4 and v6 communications. Most current platforms -supports this mode - using PF_INET6 socket and V4MAPPED addresses, i.e, -"tunnelling" IPv4 inside IPv6. But not all systems supports this. And -more, it has been said that such mode is deprecated. - -So, list only IPv4 or only IPv6 addresses, but don't mix them, in your -/etc/resolv.conf. - -An alternative is to use two sockets instead of 1 - one for IPv6 and one -for IPv4. For now I'm not sure if it's worth the complexity - again, of -the API, not the library itself (but this will not simplify library either). - -Single socket for all queries -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -Using single UDP socket for sending queries to all nameservers has obvious -advantages. First it's, again, trivial, simple to use API. And simple -library too. Also, after sending queries to all nameservers (in case first -didn't reply in time), we will be able to receive late reply from first -nameserver and accept it. - -But this mode has disadvantages too. Most important is that it's much easier -to send fake reply to us, as the UDP port where we expects the reply to come -to is constant during the whole lifetime of an application. More secure -implementations uses random port for every single query. While port number -(16 bits integer) can not hold much randomness, it's still of some help. -Ok, udns is a stub resolver, so it expects sorta friendly environment, but -on LAN it's usually much easier to fire an attack, due to the speed of local -network, where a bad guy can generate alot of packets in a short time. - -Spoofing of replies (Kaminsky attack, CVE-2008-1447) -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -While udns uses random numbers for query IDs, it uses single UDP port for -all queries (see previous item). And even if it used random UDP port for -each query, the attack described in CVE-2008-1447 is still quite trivial. -This is not specific to udns library unfortunately - it is inherent property -of the protocol. Udns is designed to work in a LAN, it needs full recursive -resolver nearby, and modern LAN usually uses high-bandwidth equipment which -makes the Kaminsky attack trivial. The problem is that even with qID (16 -bits) and random UDP port (about 20 bits available to a regular process) -combined still can not hold enough randomness, so on a fast network it is -still easy to flood the target with fake replies and hit the "right" reply -before real reply comes. So random qIDs don't add much protection anyway, -even if this feature is implemented in udns, and using all available -techniques wont solve it either. - -See also long comment in udns_resolver.c, udns_newid(). - -Assumptions about RRs returned -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -Currently udns processes records in the reply it received sequentially. -This means that order of the records is significant. For example, if -we asked for foo.bar A, but the server returned that foo.bar is a CNAME -(alias) for bar.baz, and bar.baz, in turn, has address 1.2.3.4, when -the CNAME should come first in reply, followed by A. While DNS specs -does not say anything about order of records - it's an rrSET - unordered, - -I think an implementation which returns the records in "wrong" order is -somewhat insane... - -CNAME recursion -~~~~~~~~~~~~~~~ - -Another interesting point is the handling of CNAMEs returned as replies -to non-CNAME queries. If we asked for foo.bar A, but it's a CNAME, udns -expects BOTH the CNAME itself and the target DN to be present in the reply. -In other words, udns DOES NOT RECURSE CNAMES. If we asked for foo.bar A, -but only record in reply was that foo.bar is a CNAME for bar.baz, udns will -return no records to an application (NXDOMAIN). Strictly speaking, udns -should repeat the query asking for bar.baz A, and recurse. But since it's -stub resolver, recursive resolver should recurse for us instead. - -It's not very difficult to implement, however. Probably with some (global?) -flag to en/dis-able the feature. Provided there's some demand for it. - -To clarify: udns handles CNAME recursion in a single reply packet just fine. - -Note also that standard gethostbyname() routine does not recurse in this -situation, too. - -Error reporting -~~~~~~~~~~~~~~~ - -Too many places in the code (various failure paths) sets generic "TEMPFAIL" -error condition. For example, if no nameserver replied to our query, an -application will get generic TEMPFAIL, instead of something like TIMEDOUT. -This probably should be fixed, but most applications don't care about the -exact reasons of failure - 4 common cases are already too much: - - query returned some valid data - - NXDOMAIN - - valid domain but no data of requested type - =NXDOMAIN in most cases - - temporary error - this one sometimes (incorrectly!) treated as NXDOMAIN - by (naive) applications. -DNS isn't yes/no, it's at least 3 variants, temp err being the 3rd important -case! And adding more variations for the temp error case is complicating things -even more - again, from an application writer standpoint. For diagnostics, -such more specific error cases are of good help. - -Planned API changes -~~~~~~~~~~~~~~~~~~~ - -At least one thing I want to change for some future version is a way how -queries are submitted and how replies are handled. - -I want to made dns_query object to be owned by an application. So that instead -of udns library allocating it for the lifetime of query, it will be pre- -allocated by an application. This simplifies and enhances query submitting -interface, and complicates it a bit too, in simplest cases. - -Currently, we have: - -dns_submit_dn(dn, cls, typ, flags, parse, cbck, data) -dns_submit_p(name, cls, typ, flags, parse, cbck, data) -dns_submit_a4(ctx, name, flags, cbck, data) - -and so on -- with many parameters missed for type-specific cases, but generic -cases being too complex for most common usage. - -Instead, with dns_query being owned by an app, we will be able to separately -set up various parts of the query - domain name (various forms), type&class, -parser, flags, callback... and even change them at runtime. And we will also -be able to reuse query structures, instead of allocating/freeing them every -time. So the whole thing will look something like: - - q = dns_alloc_query(); - dns_submit(dns_q_flags(dns_q_a4(q, name, cbck), DNS_F_NOSRCH), data); - -The idea is to have a set of functions accepting struct dns_query* and -returning it (so the calls can be "nested" like the above), to set up -relevant parts of the query - specific type of callback, conversion from -(type-specific) query parameters into a domain name (this is for type- -specific query initializers), and setting various flags and options and -type&class things. - -One example where this is almost essential - if we want to support -per-query set of nameservers (which isn't at all useless: imagine a -high-volume mail server, were we want to direct DNSBL queries to a separate -set of nameservers, and rDNS queries to their own set and so on). Adding -another argument (set of nameservers to use) to EVERY query submitting -routine is.. insane. Especially since in 99% cases it will be set to -default NULL. But with such "nesting" of query initializers, it becomes -trivial. - -This change (the way how queries gets submitted) will NOT break API/ABI -compatibility with old versions, since the new submitting API works in -parallel with current (and current will use the new one as building -blocks, instead of doing all work at once). - -Another way to do the same is to manipulate query object right after a -query has been submitted, but before any events processing (during this -time, query object is allocated and initialized, but no actual network -packets were sent - it will happen on the next event processing). But -this way it become impossible to perform syncronous resolver calls, since -those calls hide query objects they use internally. - -Speaking of replies handling - the planned change is to stop using dynamic -memory (malloc) inside the library. That is, instead of allocating a buffer -for a reply dynamically in a parsing routine (or memdup'ing the raw reply -packet if no parsing routine is specified), I want udns to return the packet -buffer it uses internally, and change parsing routines to expect a buffer -for result. When parsing, a routine will return true amount of memory it -will need to place the result, regardless of whenever it has enough room -or not, so that an application can (re)allocate properly sized buffer and -call a parsing routine again. - -This, in theory, also can be done without breaking current API/ABI, but in -that case we'll again need a parallel set of routines (parsing included), -which makes the library more complicated with too many ways of doing the -same thing. Still, code reuse is at good level. - -Another modification I plan to include is to have an ability to work in -terms of domain names (DNs) as used with on-wire DNS packets, not only -with asciiz representations of them. For this to work, the above two -changes (query submission and result passing) have to be completed first -(esp. the query submission part), so that it will be possible to specify -some additional query flags (for example) to request domain names instead -of the text strings, and to allow easy query submissions with either DNs -or text strings. diff --git a/contrib/udns/README.md b/contrib/udns/README.md deleted file mode 100644 index f5fcc68aa79..00000000000 --- a/contrib/udns/README.md +++ /dev/null @@ -1 +0,0 @@ -# udns diff --git a/contrib/udns/TODO b/contrib/udns/TODO deleted file mode 100644 index 0dc9b967c60..00000000000 --- a/contrib/udns/TODO +++ /dev/null @@ -1,59 +0,0 @@ -TODO - -The following is mostly an internal, not user-visible stuff. - -* rearrange an API to make dns_query object owned by application, - so that it'll look like this: - struct dns_query *q; - q = dns_query_alloc(ctx); - dns_query_set(q, options, domain_name, flags, ...); - dns_query_submit(ctx, q); - For more information see NOTES file, section "Planned API changes". - -* allow NULL callbacks? Or provide separate resolver - context list of queries which are done but wich did not - have callback, and dns_pick() routine to retrieve results - from this query, i.e. allow non-callback usage? The - non-callback usage may be handy sometimes (any *good* - example?), but it will be difficult to provide type-safe - non-callback interface due to various RR-specific types - in use. - -* DNS_OPT_FLAGS should be DNS_OPT_ADDFLAGS and DNS_OPT_SETFLAGS. - Currently one can't add a single flag bit but preserve - existing bits... at least not without retrieving all current - flags before, which isn't that bad anyway. - -* dns_set_opts() may process flags too (such as aaonly etc) - -* a way to disable $NSCACHEIP et al processing? - (with now separate dns_init() and dns_reset(), it has finer - control, but still no way to init from system files but ignore - environment variables and the like) - -* initialize/open the context automatically, and be more - liberal about initialization in general? - -* dns_init(ctx, do_open) - make the parameter opposite, aka - dns_init(ctx, skip_open) ? - -* allow TCP queue? - -* more accurate error reporting. Currently, udns always returns TEMPFAIL, - but don't specify why it happened (ENOMEM, timeout, etc). - -* check the error value returned by recvfrom() and - sendto() and determine which errors to ignore. - -* maybe merge dns_timeouts() and dns_ioevent(), to have - only one entry point for everything? For traditional - select-loop-based eventloop it may be easier, but for - callback-driven event loops the two should be separate. - Provide an option, or a single dns_events() entry point - for select-loop approach, or just call dns_ioevent() - from within dns_timeouts() (probably after renaming - it to be dns_events()) ? - -* implement /etc/hosts lookup too, ala [c-]ares?? - -* sortlist support? diff --git a/contrib/udns/config.h b/contrib/udns/config.h deleted file mode 100644 index 5d79367247f..00000000000 --- a/contrib/udns/config.h +++ /dev/null @@ -1,7 +0,0 @@ -#pragma once -/* automatically generated by configure. */ - -#define HAVE_GETOPT 1 -#define HAVE_INET_PTON_NTOP 1 -#define HAVE_IPv6 1 -#define HAVE_POLL 1 \ No newline at end of file diff --git a/contrib/udns/configure b/contrib/udns/configure deleted file mode 100755 index a3c0a0dd3d0..00000000000 --- a/contrib/udns/configure +++ /dev/null @@ -1,166 +0,0 @@ -#! /bin/sh -# autoconf-style configuration script -# - -set -e - -name=udns - -if [ -f udns.h -a -f udns_resolver.c ] ; then : -else - echo "configure: error: sources not found at `pwd`" >&2 - exit 1 -fi - -options="ipv6" - -for opt in $options; do - eval enable_$opt= -done - -if [ -f config.status ]; then - . ./config.status -fi - -enable() { - opt=`echo "$1" | sed 's/^--[^-]*-//'` - case "$opt" in - ipv6) ;; - *) echo "configure: unrecognized option \`$1'" >&2; exit 1;; - esac - eval enable_$opt=$2 -} - -while [ $# -gt 0 ]; do - case "$1" in - --disable-*|--without-*|--no-*) enable "$1" n;; - --enable-*|--with-*) enable "$1" y;; - --help | --hel | --he | --h | -help | -hel | -he | -h ) - cat <&2; exit 1 ;; - esac - shift -done - -. ./configure.lib - -ac_msg "configure" -ac_result "$name package" - -ac_prog_c_compiler_v -ac_prog_ranlib_v - -ac_ign ac_yesno "for getopt()" ac_have GETOPT ac_link < -extern int optind; -extern char *optarg; -extern int getopt(int, char **, char *); -int main(int argc, char **argv) { - getopt(argc, argv, "abc"); - return optarg ? optind : 0; -} -EOF - -if ac_library_find_v 'socket and connect' "" "-lsocket -lnsl" < -int main() { int socket_fd = socket(); connect(socket_fd); return 0; } -EOF -then : -else - ac_fatal "cannot find libraries needed for sockets" -fi - -ac_ign \ - ac_yesno "for inet_pton() && inet_ntop()" \ - ac_have INET_PTON_NTOP \ - ac_link < -#include -#include -int main() { - char buf[64]; - long x = 0; - inet_pton(AF_INET, &x, buf); - return inet_ntop(AF_INET, &x, buf, sizeof(buf)); -} -EOF - -if ac_yesno "for socklen_t" ac_compile < -#include -int foo() { socklen_t len; len = 0; return len; } -EOF -then : -else - ac_define socklen_t int -fi - -if [ n != "$enable_ipv6" ]; then -if ac_yesno "for IPv6" ac_have IPv6 ac_compile < -#include -#include -int main() { - struct sockaddr_in6 sa; - sa.sin6_family = AF_INET6; - return 0; -} -EOF -then : -elif [ "$enable_ipv6" ]; then - ac_fatal "IPv6 is requested but not available" -fi -fi # !disable_ipv6? - -if ac_yesno "for poll()" ac_have POLL ac_link < -#include -int main() { - struct pollfd pfd[2]; - return poll(pfd, 2, 10); -} -EOF -then : -else - ac_ign ac_yesno "for sys/select.h" ac_have SYS_SELECT_H ac_cpp < -#include -EOF -fi - -ac_config_h -ac_output Makefile -ac_msg "creating config.status" -rm -f config.status -{ -echo "# automatically generated by configure to hold command-line options" -echo -found= -for opt in $options; do - eval val=\$enable_$opt - if [ -n "$val" ]; then - echo enable_$opt=$val - found=y - fi -done -if [ ! "$found" ]; then - echo "# (no options encountered)" -fi -} > config.status -ac_result ok - -ac_result "all done." -exit 0 diff --git a/contrib/udns/configure.lib b/contrib/udns/configure.lib deleted file mode 100644 index 541177a095b..00000000000 --- a/contrib/udns/configure.lib +++ /dev/null @@ -1,268 +0,0 @@ -# configure.lib -# a library of shell routines for simple autoconf system -# - -set -e -ac_substitutes= -rm -f conftest* config.log -exec 5>config.log -cat <&5 -This file contains any messages produced by compilers etc while -running configure, to aid debugging if configure script makes a mistake. - -EOF - -case `echo "a\c"` in - *c*) ac_en=-n ac_ec= ;; - *) ac_en= ac_ec='\c' ;; -esac - -##### Messages -ac_msg() { - echo $ac_en "$*... $ac_ec" - echo ">>> $*" >&5 -} -ac_checking() { - echo $ac_en "checking $*... $ac_ec" - echo ">>> checking $*" >&5 -} -ac_result() { - echo "$1" - echo "=== $1" >&5 -} -ac_fatal() { - echo "configure: fatal: $*" >&2 - echo "=== FATAL: $*" >&5 - exit 1 -} -ac_warning() { - echo "configure: warning: $*" >&2 - echo "=== WARNING: $*" >&5 -} -ac_ign() { - "$@" || : -} - -# ac_run command... -# captures output in conftest.out -ac_run() { - # apparently UnixWare (for one) /bin/sh optimizes the following "if" - # "away", by checking if there's such a command BEFORE redirecting - # output. So error message (like "gcc: command not found") goes - # to stderr instead of to conftest.out, and `cat conftest.out' below - # fails. - if "$@" >conftest.out 2>&1; then - return 0 - else - echo "==== Command invocation failed. Command line was:" >&5 - echo "$*" >&5 - echo "==== compiler input was:" >&5 - cat conftest.c >&5 - echo "==== output was:" >&5 - cat conftest.out >&5 - echo "====" >&5 - return 1 - fi -} - -# common case for ac_verbose: yes/no result -ac_yesno() { - ac_checking "$1" - shift - if "$@"; then - ac_result yes - return 0 - else - ac_result no - return 1 - fi -} - -ac_subst() { - ac_substitutes="$ac_substitutes $*" -} - -ac_define() { - CDEFS="$CDEFS -D$1=${2:-1}" -} - -ac_have() { - ac_what=$1; shift - if "$@"; then - ac_define HAVE_$ac_what - eval ac_have_$ac_what=yes - return 0 - else - eval ac_have_$ac_what=no - return 1 - fi -} - -##### Compiling, linking - -# run a compiler -ac_run_compiler() { - rm -f conftest*; cat >conftest.c - ac_run $CC $CFLAGS $CDEFS "$@" conftest.c -} - -ac_compile() { - ac_run_compiler -c -} - -ac_link() { - ac_run_compiler -o conftest $LIBS "$@" -} - -ac_cpp() { - ac_run_compiler -E "$@" -} - -### check for C compiler. Set $CC, $CFLAGS etc -ac_prog_c_compiler_v() { - ac_checking "for C compiler" - rm -f conftest* - echo 'int main(int argc, char **argv) { return 0; }' >conftest.c - - if [ -n "$CC" ]; then - if ac_run $CC -o conftest conftest.c && ac_run ./conftest; then - ac_result "\$CC ($CC)" - else - ac_result no - ac_fatal "\$CC ($CC) is not a working compiler" - fi - else - for cc in gcc cc ; do - if ac_run $cc -o conftest conftest.c && ac_run ./conftest; then - ac_result "$cc" - CC=$cc - break - fi - done - if [ -z "$CC" ]; then - ac_result no - ac_fatal "no working C compiler found in \$PATH. please set \$CC variable" - fi - fi - if [ -z "$CFLAGS" ]; then - if ac_yesno "whenever C compiler ($CC) is GNU CC" \ - ac_grep_cpp yEs_mAsTeR <conftest.c - for lib in "$@"; do - if ac_run $CC $CFLAGS $LDFLAGS conftest.c -o conftest $LIBS $lib; then - found=y - break - fi - done - if [ ! "$found" ]; then - ac_result "not found" - return 1 - fi - if [ -z "$lib" ]; then - ac_result "ok (none needed)" - else - ac_result "ok ($lib)" - LIBS="$LIBS $lib" - fi -} - -ac_compile_run() { - ac_link "$@" && ac_run ./conftest -} - -ac_grep_cpp() { - pattern="$1"; shift - ac_cpp "$@" && grep "$pattern" conftest.out >/dev/null -} - -ac_output() { - for var in $ac_substitutes; do - eval echo "\"s|@$var@|\$$var|\"" - done >conftest.sed - for file in "$@"; do - ac_msg "creating $file" - if [ -f $file.in ]; then - sed -f conftest.sed $file.in > $file.tmp - mv -f $file.tmp $file - ac_result ok - else - ac_result failed - ac_fatal "$file.in not found" - fi - done - rm -f conftest* -} - -ac_config_h() { - h=${1:-config.h} - ac_msg "creating $h" - rm -f $1.tmp - echo "/* automatically generated by configure. */" > $h.tmp - echo "$CDEFS" | tr ' ' ' -' | sed -e 's/^-D/#define /' -e 's/=/ /' >> $h.tmp - if [ -f $h ] && cmp -s $h.tmp $h ; then - rm -f $h.tmp - ac_result unchanged - else - mv -f $h.tmp $h - ac_result ok - fi - CDEFS=-DHAVE_CONFIG_H -} diff --git a/contrib/udns/conftest.c b/contrib/udns/conftest.c deleted file mode 100644 index 5bc7720cbaa..00000000000 --- a/contrib/udns/conftest.c +++ /dev/null @@ -1,3 +0,0 @@ -#include - -int main() { socket(); connect(); return 0; } diff --git a/contrib/udns/conftest.out b/contrib/udns/conftest.out deleted file mode 100644 index cfff425825a..00000000000 --- a/contrib/udns/conftest.out +++ /dev/null @@ -1,7 +0,0 @@ -conftest.c:1:14: error: implicit declaration of function 'socket' is invalid in C99 [-Werror,-Wimplicit-function-declaration] -int main() { socket(); connect(); return 0; } - ^ -conftest.c:1:24: error: implicit declaration of function 'connect' is invalid in C99 [-Werror,-Wimplicit-function-declaration] -int main() { socket(); connect(); return 0; } - ^ -2 errors generated. diff --git a/contrib/udns/dnsget.1 b/contrib/udns/dnsget.1 deleted file mode 100644 index 200557fe947..00000000000 --- a/contrib/udns/dnsget.1 +++ /dev/null @@ -1,195 +0,0 @@ -.\" dnsget.1: dnsget manpage -.\" -.\" Copyright (C) 2005-2014 Michael Tokarev -.\" This file is part of UDNS library, an async DNS stub resolver. -.\" -.\" This library is free software; you can redistribute it and/or -.\" modify it under the terms of the GNU Lesser General Public -.\" License as published by the Free Software Foundation; either -.\" version 2.1 of the License, or (at your option) any later version. -.\" -.\" This library is distributed in the hope that it will be useful, -.\" but WITHOUT ANY WARRANTY; without even the implied warranty of -.\" MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU -.\" Lesser General Public License for more details. -.\" -.\" You should have received a copy of the GNU Lesser General Public -.\" License along with this library, in file named COPYING.LGPL; if not, -.\" write to the Free Software Foundation, Inc., 59 Temple Place, -.\" Suite 330, Boston, MA 02111-1307 USA - -.TH dnsget 1 "Jan 2014" "User Utilities" - -.SH NAME -dnsget \- DNS lookup utility - -.SH SYNOPSYS -.B dnsget -.RB [\| \-v \||\| \-q \|] -.RB [\| \-c -.IR class \|] -.RB [\| \-t -.IR type \|] -.RB [\| \-o -.IR opt , opt ,...] -.IR name \|.\|.\|. - -.SH DESCRIPTION -.B dnsget -is a simple command-line to perform DNS lookups, similar to -.BR host (1) -and -.BR dig (1). -It is useable for both interactive/debugging scenarious and -in scripts. -The program is implemented using -.BR udns (3) -library. - -.PP -By default, -.B dnsget -produces a human-readable output, similar to -.RS -.nf -alias.example.com. CNAME www.example.com. -www.example.com. A 192.168.1.1 -www.example.com. MX 10 mx.example.com. -.fi -.RE -which is just sufficient to see how a given name resolves. -Output format is controllable with -.B \-v -and -.B \-q -options -- the former increases verbosity level up to printing -the whole DNS contents of all packets sent and received, which -is suitable for debugging DNS problems, while the latter reduces -the level, making output more quiet, up to bare result with no -error messages, which is good for scripts. - -.SH OPTIONS - -The following options are recognized by -.BR dnsget : - -.TP -.B \-v -produce more detailed output. More -.BR \-v 's -means more details will be produced. With single -.BR \-v , dnsget -will print contents of all received DNS packets (in a readable format), -while with -.BR \-vv , -it will output all outgoing DNS packets too. - -.TP -.B \-q -the opposite for \fB\-v\fR -- produce less detailed output. -With single -.BR \-q , dnsget -will only show (decoded) data from final DNS resource records (RR), -while -.B \-qq -also suppresses error messages. - -.TP -\fB\-t \fItype\fR -request record(s) of the given type \fItype\fR. By default, -.B dnsget -will ask for IPv4 address (A) record, or for PTR record if the -argument in question is an IPv4 or IPv6 address. Recognized -types include A, AAAA, MX, TXT, CNAME, PTR, NS, SOA, ANY and -others. - -.TP -\fB\-c \fIclass\fR -request DNS record(s) of the given class \fIclass\fR. By -default -.B dnsget -uses IN class. Valid classes include IN, CH, HS, ANY. - -.TP -.B \-a -(compatibility option). Equivalent to setting query type to -.B ANY -and increasing verbosity level -.RB ( \-v ). - -.TP -.B \-C -(planned) - -.TP -.B \-x -(planned) - -.TP -\fB\-o \fIopt\fR,\fIopt\fR,... -(may be specified several times). -Set resolver options (in a form \fIoption\fR:\fIvalue\fR) as if they -were set in -.RB $ RES_OPTIONS -environment variable, or set query flags: -.RS -.TP -\fBtimeout\fR:\fIsec\fR -Set initial query timeout to \fIsec\fR. -.TP -\fBattempts\fR:\fInum\fR -(re)try every query \fInum\fR times before failing. -.TP -\fBudpbuf\fR:\fIbytes\fR -set DNS UDP buffer size to \fIbytes\fR bytes. Valid values -are from 512 to 65535. If \fIbytes\fR is greather than 512, -EDNS0 (RFC 2671) extensions will be used. -.TP -\fBport\fR:\fInum\fR -Use given UDP port number \fInum\fR instead of the default port 53 (domain). -.TP -\fBaa\fR -set AA (auth only) query bit. -.TP -\fBnord\fR -do not set RD (recursion desired) query bit (set by default). -.TP -\fBdnssec\fR or \fBdo\fR -set DNSSEC OK (DO) query flag (\fBdnsget\fR does not verify DNSSEC signatures, -only displays them; this is set in EDNS RR). -.TP -\fBcd\fR -set CD (checking disabled) query bit. -.RE - -.TP -\fB\-n \fInameserver\fR -Use the given nameserver(s) (may be specified more than once) -instead of the default. Using this option has the same same effect as -.RB $ NSCACHEIP -or -.RB $ NAMESERVERS -environment variables, with the only difference that only IPv4 addresses -are recognized for now, and it is possible to specify names (which will -be resolved using default settings) instead of IP addresses. - -.TP -.B \-h -print short help and exit. - -.SH "RETURN VALUE" -When all names where resovled successefully, -.B dnsget -exits with zero exit status. If at least one name was not found, -.B dnsget -will exit with return code 100. If some other error occured during -name resolution, it will exit with code 99. In case of usage or -initialization error, -.B dnsget -will return 1. - -.SH "SEE ALSO" -.BR host (1) -.BR dig (1) -.BR resolv.conf (5) -.BR udns (3). diff --git a/contrib/udns/dnsget.c b/contrib/udns/dnsget.c deleted file mode 100644 index 417e8d9743c..00000000000 --- a/contrib/udns/dnsget.c +++ /dev/null @@ -1,759 +0,0 @@ -/* dnsget.c - simple host/dig-like application using UDNS library - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#ifdef HAVE_CONFIG_H -# include "config.h" -#endif -#ifdef WINDOWS -#include -#include -#else -#include -#include -#include -#include -#include -#endif -#include -#include -#include -#include -#include -#include -#include "udns.h" - -#ifndef HAVE_GETOPT -# include "getopt.c" -#endif - -#ifndef AF_INET6 -# define AF_INET6 10 -#endif - -static char *progname; -static int verbose = 1; -static int errors; -static int notfound; - -/* verbosity level: - * <0 - bare result - * 0 - bare result and error messages - * 1 - readable result - * 2 - received packet contents and `trying ...' stuff - * 3 - sent and received packet contents - */ - -static void die(int errnum, const char *fmt, ...) { - va_list ap; - fprintf(stderr, "%s: ", progname); - va_start(ap, fmt); vfprintf(stderr, fmt, ap); va_end(ap); - if (errnum) fprintf(stderr, ": %s\n", strerror(errnum)); - else putc('\n', stderr); - fflush(stderr); - exit(1); -} - -static const char *dns_xntop(int af, const void *src) { - static char buf[6*5+4*4]; - return dns_ntop(af, src, buf, sizeof(buf)); -} - -struct query { - const char *name; /* original query string */ - unsigned char *dn; /* the DN being looked up */ - enum dns_type qtyp; /* type of the query */ -}; - -static void query_free(struct query *q) { - free(q->dn); - free(q); -} - -static struct query * -query_new(const char *name, const unsigned char *dn, enum dns_type qtyp) { - struct query *q = malloc(sizeof(*q)); - unsigned l = dns_dnlen(dn); - unsigned char *cdn = malloc(l); - if (!q || !cdn) die(0, "out of memory"); - memcpy(cdn, dn, l); - q->name = name; - q->dn = cdn; - q->qtyp = qtyp; - return q; -} - -static enum dns_class qcls = DNS_C_IN; - -static void -dnserror(struct query *q, int errnum) { - if (verbose >= 0) - fprintf(stderr, "%s: unable to lookup %s record for %s: %s\n", progname, - dns_typename(q->qtyp), dns_dntosp(q->dn), dns_strerror(errnum)); - if (errnum == DNS_E_NXDOMAIN || errnum == DNS_E_NODATA) - ++notfound; - else - ++errors; - query_free(q); -} - -static const unsigned char * -printtxt(const unsigned char *c) { - unsigned n = *c++; - const unsigned char *e = c + n; - if (verbose > 0) while(c < e) { - if (*c < ' ' || *c >= 127) printf("\\%03u", *c); - else if (*c == '\\' || *c == '"') printf("\\%c", *c); - else putchar(*c); - ++c; - } - else - fwrite(c, n, 1, stdout); - return e; -} - -static void -printhex(const unsigned char *c, const unsigned char *e) { - while(c < e) - printf("%02x", *c++); -} - -static unsigned char to_b64[] = -"ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/"; - -static void -printb64(const unsigned char *c, const unsigned char *e) { - while(c < e) { - putchar(to_b64[c[0] >> 2]); - if (c+1 < e) { - putchar(to_b64[(c[0] & 0x3) << 4 | c[1] >> 4]); - if (c+2 < e) { - putchar(to_b64[(c[1] & 0xf) << 2 | c[2] >> 6]); - putchar(to_b64[c[2] & 0x3f]); - } - else { - putchar(to_b64[(c[1] & 0xf) << 2]); - putchar('='); - break; - } - } - else { - putchar(to_b64[(c[0] & 0x3) << 4]); - putchar('='); - putchar('='); - break; - } - c += 3; - } -} - -static void -printdate(time_t time) { - struct tm *tm = gmtime(&time); - printf("%04d%02d%02d%02d%02d%02d", - tm->tm_year + 1900, tm->tm_mon + 1, tm->tm_mday, - tm->tm_hour, tm->tm_min, tm->tm_sec); -} - -static void -printrr(const struct dns_parse *p, struct dns_rr *rr) { - const unsigned char *pkt = p->dnsp_pkt; - const unsigned char *end = p->dnsp_end; - const unsigned char *dptr = rr->dnsrr_dptr; - const unsigned char *dend = rr->dnsrr_dend; - unsigned char *dn = rr->dnsrr_dn; - const unsigned char *c; - unsigned n; - - if (verbose > 0) { - if (verbose > 1) { - if (!p->dnsp_rrl && !rr->dnsrr_dn[0] && rr->dnsrr_typ == DNS_T_OPT) { - printf(";EDNS%d OPT record (UDPsize: %d, ERcode: %d, Flags: 0x%02x): %d bytes\n", - (rr->dnsrr_ttl>>16) & 0xff, /* version */ - rr->dnsrr_cls, /* udp size */ - (rr->dnsrr_ttl>>24) & 0xff, /* extended rcode */ - rr->dnsrr_ttl & 0xffff, /* flags */ - rr->dnsrr_dsz); - return; - } - n = printf("%s.", dns_dntosp(rr->dnsrr_dn)); - printf("%s%u\t%s\t%s\t", - n > 15 ? "\t" : n > 7 ? "\t\t" : "\t\t\t", - rr->dnsrr_ttl, - dns_classname(rr->dnsrr_cls), - dns_typename(rr->dnsrr_typ)); - } - else - printf("%s. %s ", dns_dntosp(rr->dnsrr_dn), dns_typename(rr->dnsrr_typ)); - } - - switch(rr->dnsrr_typ) { - - case DNS_T_CNAME: - case DNS_T_PTR: - case DNS_T_NS: - case DNS_T_MB: - case DNS_T_MD: - case DNS_T_MF: - case DNS_T_MG: - case DNS_T_MR: - if (dns_getdn(pkt, &dptr, end, dn, DNS_MAXDN) <= 0) goto xperr; - printf("%s.", dns_dntosp(dn)); - break; - - case DNS_T_A: - if (rr->dnsrr_dsz != 4) goto xperr; - printf("%d.%d.%d.%d", dptr[0], dptr[1], dptr[2], dptr[3]); - break; - - case DNS_T_AAAA: - if (rr->dnsrr_dsz != 16) goto xperr; - printf("%s", dns_xntop(AF_INET6, dptr)); - break; - - case DNS_T_MX: - c = dptr + 2; - if (dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0 || c != dend) goto xperr; - printf("%d %s.", dns_get16(dptr), dns_dntosp(dn)); - break; - - case DNS_T_TXT: - /* first verify it */ - for(c = dptr; c < dend; c += n) { - n = *c++; - if (c + n > dend) goto xperr; - } - c = dptr; n = 0; - while (c < dend) { - if (verbose > 0) printf(n++ ? "\" \"":"\""); - c = printtxt(c); - } - if (verbose > 0) putchar('"'); - break; - - case DNS_T_HINFO: /* CPU, OS */ - c = dptr; - n = *c++; if ((c += n) >= dend) goto xperr; - n = *c++; if ((c += n) != dend) goto xperr; - c = dptr; - if (verbose > 0) putchar('"'); - c = printtxt(c); - if (verbose > 0) printf("\" \""); else putchar(' '); - printtxt(c); - if (verbose > 0) putchar('"'); - break; - - case DNS_T_WKS: - c = dptr; - if (dptr + 4 + 2 >= end) goto xperr; - printf("%s %d", dns_xntop(AF_INET, dptr), dptr[4]); - c = dptr + 5; - for (n = 0; c < dend; ++c, n += 8) { - if (*c) { - unsigned b; - for (b = 0; b < 8; ++b) - if (*c & (1 << (7-b))) printf(" %d", n + b); - } - } - break; - - case DNS_T_SRV: /* prio weight port targetDN */ - c = dptr; - c += 2 + 2 + 2; - if (dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0 || c != dend) goto xperr; - c = dptr; - printf("%d %d %d %s.", - dns_get16(c+0), dns_get16(c+2), dns_get16(c+4), - dns_dntosp(dn)); - break; - - case DNS_T_NAPTR: /* order pref flags serv regexp repl */ - c = dptr; - c += 4; /* order, pref */ - for (n = 0; n < 3; ++n) - if (c >= dend) goto xperr; - else c += *c + 1; - if (dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0 || c != dend) goto xperr; - c = dptr; - printf("%u %u", dns_get16(c+0), dns_get16(c+2)); - c += 4; - for(n = 0; n < 3; ++n) { - putchar(' '); - if (verbose > 0) putchar('"'); - c = printtxt(c); - if (verbose > 0) putchar('"'); - } - printf(" %s.", dns_dntosp(dn)); - break; - - case DNS_T_KEY: - case DNS_T_DNSKEY: - /* flags(2) proto(1) algo(1) pubkey */ - case DNS_T_DS: - case DNS_T_DLV: - /* ktag(2) proto(1) algo(1) pubkey */ - c = dptr; - if (c + 2 + 1 + 1 > dend) goto xperr; - printf("%d %d %d", dns_get16(c), c[2], c[3]); - c += 2 + 1 + 1; - if (c < dend) { - putchar(' '); - printb64(c, dend); - } - break; - - case DNS_T_SIG: - case DNS_T_RRSIG: - /* type(2) algo(1) labels(1) ottl(4) sexp(4) sinc(4) tag(2) sdn sig */ - c = dptr; - c += 2 + 1 + 1 + 4 + 4 + 4 + 2; - if (dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0) goto xperr; - printf("%s %u %u %u ", - dns_typename(dns_get16(dptr)), dptr[2], dptr[3], dns_get32(dptr+4)); - printdate(dns_get32(dptr+8)); - putchar(' '); - printdate(dns_get32(dptr+12)); - printf(" %d %s. ", dns_get16(dptr+10), dns_dntosp(dn)); - printb64(c, dend); - break; - - case DNS_T_SSHFP: /* algo(1), fp type(1), fp... */ - if (dend < dptr + 3) goto xperr; - printf("%u %u ", dptr[0], dptr[1]); /* algo, fp type */ - printhex(dptr + 2, dend); - break; - -#if 0 /* unused RR types? */ - case DNS_T_NSEC: /* nextDN bitmaps */ - c = dptr; - if (dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0) goto xperr; - printf("%s.", dns_dntosp(dn)); - unfinished. - break; -#endif - - - case DNS_T_SOA: - c = dptr; - if (dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0 || - dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0 || - c + 4*5 != dend) - goto xperr; - dns_getdn(pkt, &dptr, end, dn, DNS_MAXDN); - printf("%s. ", dns_dntosp(dn)); - dns_getdn(pkt, &dptr, end, dn, DNS_MAXDN); - printf("%s. ", dns_dntosp(dn)); - printf("%u %u %u %u %u", - dns_get32(dptr), dns_get32(dptr+4), dns_get32(dptr+8), - dns_get32(dptr+12), dns_get32(dptr+16)); - break; - - case DNS_T_MINFO: - c = dptr; - if (dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0 || - dns_getdn(pkt, &c, end, dn, DNS_MAXDN) <= 0 || - c != dend) - goto xperr; - dns_getdn(pkt, &dptr, end, dn, DNS_MAXDN); - printf("%s. ", dns_dntosp(dn)); - dns_getdn(pkt, &dptr, end, dn, DNS_MAXDN); - printf("%s.", dns_dntosp(dn)); - break; - - case DNS_T_NULL: - default: - printhex(dptr, dend); - break; - } - putchar('\n'); - return; - -xperr: - printf("\n"); - ++errors; -} - -static int -printsection(struct dns_parse *p, int nrr, const char *sname) { - struct dns_rr rr; - int r; - if (!nrr) return 0; - if (verbose > 1) printf("\n;; %s section (%d):\n", sname, nrr); - - p->dnsp_rrl = nrr; - while((r = dns_nextrr(p, &rr)) > 0) - printrr(p, &rr); - if (r < 0) printf("<>\n"); - return r; -} - -/* dbgcb will only be called if verbose > 1 */ -static void -dbgcb(int code, const struct sockaddr *sa, unsigned slen, - const unsigned char *pkt, int r, - const struct dns_query *unused_q, void *unused_data) { - struct dns_parse p; - const unsigned char *cur, *end; - int numqd; - - if (code > 0) { - printf(";; trying %s.\n", dns_dntosp(dns_payload(pkt))); - printf(";; sending %d bytes query to ", r); - } - else - printf(";; received %d bytes response from ", r); - if (sa->sa_family == AF_INET && slen >= sizeof(struct sockaddr_in)) - printf("%s port %d\n", - dns_xntop(AF_INET, &((struct sockaddr_in*)sa)->sin_addr), - htons(((struct sockaddr_in*)sa)->sin_port)); -#ifdef HAVE_IPv6 - else if (sa->sa_family == AF_INET6 && slen >= sizeof(struct sockaddr_in6)) - printf("%s port %d\n", - dns_xntop(AF_INET6, &((struct sockaddr_in6*)sa)->sin6_addr), - htons(((struct sockaddr_in6*)sa)->sin6_port)); -#endif - else - printf("<>\n", sa->sa_family); - if (code > 0 && verbose < 3) { - putchar('\n'); - return; - } - - if (code == -2) printf(";; reply from unexpected source\n"); - if (code == -5) printf(";; reply to a query we didn't sent (or old)\n"); - if (r < DNS_HSIZE) { - printf(";; short packet (%d bytes)\n", r); - return; - } - if (dns_opcode(pkt) != 0) - printf(";; unexpected opcode %d\n", dns_opcode(pkt)); - if (dns_tc(pkt) != 0) - printf(";; warning: TC bit set, probably incomplete reply\n"); - - printf(";; ->>HEADER<<- opcode: "); - switch(dns_opcode(pkt)) { - case 0: printf("QUERY"); break; - case 1: printf("IQUERY"); break; - case 2: printf("STATUS"); break; - default: printf("UNKNOWN(%u)", dns_opcode(pkt)); break; - } - printf(", status: %s, id: %d, size: %d\n;; flags:", - dns_rcodename(dns_rcode(pkt)), dns_qid(pkt), r); - if (dns_qr(pkt)) printf(" qr"); - if (dns_aa(pkt)) printf(" aa"); - if (dns_tc(pkt)) printf(" tc"); - if (dns_rd(pkt)) printf(" rd"); - if (dns_ra(pkt)) printf(" ra"); - /* if (dns_z(pkt)) printf(" z"); only one reserved bit left */ - if (dns_ad(pkt)) printf(" ad"); - if (dns_cd(pkt)) printf(" cd"); - numqd = dns_numqd(pkt); - printf("; QUERY: %d, ANSWER: %d, AUTHORITY: %d, ADDITIONAL: %d\n", - numqd, dns_numan(pkt), dns_numns(pkt), dns_numar(pkt)); - if (numqd != 1) - printf(";; unexpected number of entries in QUERY section: %d\n", - numqd); - printf("\n;; QUERY SECTION (%d):\n", numqd); - cur = dns_payload(pkt); - end = pkt + r; - while(numqd--) { - if (dns_getdn(pkt, &cur, end, p.dnsp_dnbuf, DNS_MAXDN) <= 0 || - cur + 4 > end) { - printf("; invalid query section\n"); - return; - } - r = printf(";%s.", dns_dntosp(p.dnsp_dnbuf)); - printf("%s%s\t%s\n", - r > 23 ? "\t" : r > 15 ? "\t\t" : r > 7 ? "\t\t\t" : "\t\t\t\t", - dns_classname(dns_get16(cur+2)), dns_typename(dns_get16(cur))); - cur += 4; - } - - p.dnsp_pkt = pkt; - p.dnsp_cur = p.dnsp_ans = cur; - p.dnsp_end = end; - p.dnsp_qdn = NULL; - p.dnsp_qcls = p.dnsp_qtyp = 0; - p.dnsp_ttl = 0xffffffffu; - p.dnsp_nrr = 0; - - r = printsection(&p, dns_numan(pkt), "ANSWER"); - if (r == 0) - r = printsection(&p, dns_numns(pkt), "AUTHORITY"); - if (r == 0) - r = printsection(&p, dns_numar(pkt), "ADDITIONAL"); - putchar('\n'); -} - -static void dnscb(struct dns_ctx *ctx, void *result, void *data) { - int r = dns_status(ctx); - struct query *q = data; - struct dns_parse p; - struct dns_rr rr; - unsigned nrr; - unsigned char dn[DNS_MAXDN]; - const unsigned char *pkt, *cur, *end; - if (!result) { - dnserror(q, r); - return; - } - pkt = result; end = pkt + r; cur = dns_payload(pkt); - dns_getdn(pkt, &cur, end, dn, sizeof(dn)); - dns_initparse(&p, NULL, pkt, cur, end); - p.dnsp_qcls = p.dnsp_qtyp = 0; - nrr = 0; - while((r = dns_nextrr(&p, &rr)) > 0) { - if (!dns_dnequal(dn, rr.dnsrr_dn)) continue; - if ((qcls == DNS_C_ANY || qcls == rr.dnsrr_cls) && - (q->qtyp == DNS_T_ANY || q->qtyp == rr.dnsrr_typ)) - ++nrr; - else if (rr.dnsrr_typ == DNS_T_CNAME && !nrr) { - if (dns_getdn(pkt, &rr.dnsrr_dptr, end, - p.dnsp_dnbuf, sizeof(p.dnsp_dnbuf)) <= 0 || - rr.dnsrr_dptr != rr.dnsrr_dend) { - r = DNS_E_PROTOCOL; - break; - } - else { - if (verbose == 1) { - printf("%s.", dns_dntosp(dn)); - printf(" CNAME %s.\n", dns_dntosp(p.dnsp_dnbuf)); - } - dns_dntodn(p.dnsp_dnbuf, dn, sizeof(dn)); - } - } - } - if (!r && !nrr) - r = DNS_E_NODATA; - if (r < 0) { - dnserror(q, r); - free(result); - return; - } - if (verbose < 2) { /* else it is already printed by dbgfn */ - dns_rewind(&p, NULL); - p.dnsp_qtyp = q->qtyp == DNS_T_ANY ? 0 : q->qtyp; - p.dnsp_qcls = qcls == DNS_C_ANY ? 0 : qcls; - while(dns_nextrr(&p, &rr)) - printrr(&p, &rr); - } - free(result); - query_free(q); -} - -int main(int argc, char **argv) { - int i; - int fd; - fd_set fds; - struct timeval tv; - time_t now; - char *ns[DNS_MAXSERV]; - int nns = 0; - struct query *q; - enum dns_type qtyp = 0; - struct dns_ctx *nctx = NULL; - int flags = 0; - - if (!(progname = strrchr(argv[0], '/'))) progname = argv[0]; - else argv[0] = ++progname; - - if (argc <= 1) - die(0, "try `%s -h' for help", progname); - - if (dns_init(NULL, 0) < 0 || !(nctx = dns_new(NULL))) - die(errno, "unable to initialize dns library"); - /* we keep two dns contexts: one may be needed to resolve - * nameservers if given as names, using default options. - */ - - while((i = getopt(argc, argv, "vqt:c:an:o:f:h")) != EOF) switch(i) { - case 'v': ++verbose; break; - case 'q': --verbose; break; - case 't': - if (optarg[0] == '*' && !optarg[1]) - i = DNS_T_ANY; - else if ((i = dns_findtypename(optarg)) <= 0) - die(0, "unrecognized query type `%s'", optarg); - qtyp = i; - break; - case 'c': - if (optarg[0] == '*' && !optarg[1]) - i = DNS_C_ANY; - else if ((i = dns_findclassname(optarg)) < 0) - die(0, "unrecognized query class `%s'", optarg); - qcls = i; - break; - case 'a': - qtyp = DNS_T_ANY; - ++verbose; - break; - case 'n': - if (nns >= DNS_MAXSERV) - die(0, "too many nameservers, %d max", DNS_MAXSERV); - ns[nns++] = optarg; - break; - case 'o': - case 'f': { - char *opt; - const char *const delim = " \t,;"; - for(opt = strtok(optarg, delim); opt != NULL; opt = strtok(NULL, delim)) { - if (dns_set_opts(NULL, optarg) == 0) - ; - else if (strcmp(opt, "aa") == 0) flags |= DNS_AAONLY; - else if (strcmp(optarg, "nord") == 0) flags |= DNS_NORD; - else if (strcmp(optarg, "dnssec") == 0) flags |= DNS_SET_DO; - else if (strcmp(optarg, "do") == 0) flags |= DNS_SET_DO; - else if (strcmp(optarg, "cd") == 0) flags |= DNS_SET_CD; - else - die(0, "invalid option: `%s'", opt); - } - break; - } - case 'h': - printf( -"%s: simple DNS query tool (using udns version %s)\n" -"Usage: %s [options] domain-name...\n" -"where options are:\n" -" -h - print this help and exit\n" -" -v - be more verbose\n" -" -q - be less verbose\n" -" -t type - set query type (A, AAA, PTR etc)\n" -" -c class - set query class (IN (default), CH, HS, *)\n" -" -a - equivalent to -t ANY -v\n" -" -n ns - use given nameserver(s) instead of default\n" -" (may be specified multiple times)\n" -" -o opt,opt,... (comma- or space-separated list,\n" -" may be specified more than once):\n" -" set resovler options (the same as setting $RES_OPTIONS):\n" -" timeout:sec - initial query timeout\n" -" attempts:num - number of attempt to resovle a query\n" -" ndots:num - if name has more than num dots, lookup it before search\n" -" port:num - port number for queries instead of default 53\n" -" udpbuf:num - size of UDP buffer (use EDNS0 if >512)\n" -" or query flags:\n" -" aa,nord,dnssec,do,cd - set query flag (auth-only, no recursion,\n" -" enable DNSSEC (DNSSEC Ok), check disabled)\n" - , progname, dns_version(), progname); - return 0; - default: - die(0, "try `%s -h' for help", progname); - } - - argc -= optind; argv += optind; - if (!argc) - die(0, "no name(s) to query specified"); - - if (nns) { - /* if nameservers given as names, resolve them. - * We only allow IPv4 nameservers as names for now. - * Ok, it is easy enouth to try both AAAA and A, - * but the question is what to do by default. - */ - struct sockaddr_in sin; - int j, r = 0, opened = 0; - memset(&sin, 0, sizeof(sin)); - sin.sin_family = AF_INET; - sin.sin_port = htons(dns_set_opt(NULL, DNS_OPT_PORT, -1)); - dns_add_serv(NULL, NULL); - for(i = 0; i < nns; ++i) { - if (dns_pton(AF_INET, ns[i], &sin.sin_addr) <= 0) { - struct dns_rr_a4 *rr; - if (!opened) { - if (dns_open(nctx) < 0) - die(errno, "unable to initialize dns context"); - opened = 1; - } - rr = dns_resolve_a4(nctx, ns[i], 0); - if (!rr) - die(0, "unable to resolve nameserver %s: %s", - ns[i], dns_strerror(dns_status(nctx))); - for(j = 0; j < rr->dnsa4_nrr; ++j) { - sin.sin_addr = rr->dnsa4_addr[j]; - if ((r = dns_add_serv_s(NULL, (struct sockaddr *)&sin)) < 0) - break; - } - free(rr); - } - else - r = dns_add_serv_s(NULL, (struct sockaddr *)&sin); - if (r < 0) - die(errno, "unable to add nameserver %s", - dns_xntop(AF_INET, &sin.sin_addr)); - } - } - dns_free(nctx); - - fd = dns_open(NULL); - if (fd < 0) - die(errno, "unable to initialize dns context"); - - if (verbose > 1) - dns_set_dbgfn(NULL, dbgcb); - - if (flags) - dns_set_opt(NULL, DNS_OPT_FLAGS, flags); - - for (i = 0; i < argc; ++i) { - char *name = argv[i]; - union { - struct in_addr addr; - struct in6_addr addr6; - } a; - unsigned char dn[DNS_MAXDN]; - enum dns_type l_qtyp = 0; - int abs; - if (dns_pton(AF_INET, name, &a.addr) > 0) { - dns_a4todn(&a.addr, 0, dn, sizeof(dn)); - l_qtyp = DNS_T_PTR; - abs = 1; - } -#ifdef HAVE_IPv6 - else if (dns_pton(AF_INET6, name, &a.addr6) > 0) { - dns_a6todn(&a.addr6, 0, dn, sizeof(dn)); - l_qtyp = DNS_T_PTR; - abs = 1; - } -#endif - else if (!dns_ptodn(name, strlen(name), dn, sizeof(dn), &abs)) - die(0, "invalid name `%s'\n", name); - else - l_qtyp = DNS_T_A; - if (qtyp) l_qtyp = qtyp; - q = query_new(name, dn, l_qtyp); - if (abs) abs = DNS_NOSRCH; - if (!dns_submit_dn(NULL, dn, qcls, l_qtyp, abs, 0, dnscb, q)) - dnserror(q, dns_status(NULL)); - } - - FD_ZERO(&fds); - now = 0; - while((i = dns_timeouts(NULL, -1, now)) > 0) { - FD_SET(fd, &fds); - tv.tv_sec = i; - tv.tv_usec = 0; - i = select(fd+1, &fds, 0, 0, &tv); - now = time(NULL); - if (i > 0) dns_ioevent(NULL, now); - } - - return errors ? 1 : notfound ? 100 : 0; -} diff --git a/contrib/udns/ex-rdns.c b/contrib/udns/ex-rdns.c deleted file mode 100644 index 1e1e90d4b1d..00000000000 --- a/contrib/udns/ex-rdns.c +++ /dev/null @@ -1,114 +0,0 @@ -/* ex-rdns.c - parallel rDNS resolver example - read IP addresses from stdin, - write domain names to stdout - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "udns.h" - -static int curq; - -static const char *n2ip(const unsigned char *c) { - static char b[sizeof("255.255.255.255")]; - sprintf(b, "%u.%u.%u.%u", c[0], c[1], c[2], c[3]); - return b; -} -static void dnscb(struct dns_ctx *ctx, struct dns_rr_ptr *rr, void *data) { - const char *ip = n2ip((unsigned char *)&data); - int i; - --curq; - if (rr) { - printf("%s", ip); - for(i = 0; i < rr->dnsptr_nrr; ++i) - printf(" %s", rr->dnsptr_ptr[i]); - putchar('\n'); - free(rr); - } - else - fprintf(stderr, "%s: %s\n", ip, dns_strerror(dns_status(ctx))); -} - -int main(int argc, char **argv) { - int c; - time_t now; - int maxq = 10; - struct pollfd pfd; - char linebuf[1024]; - char *eol; - int eof; - - if (dns_init(NULL, 1) < 0) { - fprintf(stderr, "unable to initialize dns library\n"); - return 1; - } - while((c = getopt(argc, argv, "m:r")) != EOF) switch(c) { - case 'm': maxq = atoi(optarg); break; - case 'r': - dns_set_opt(0, DNS_OPT_FLAGS, - dns_set_opt(0, DNS_OPT_FLAGS, -1) | DNS_NORD); - break; - default: return 1; - } - if (argc != optind) return 1; - - pfd.fd = dns_sock(0); - pfd.events = POLLIN; - now = time(NULL); - c = optind; - eof = 0; - while(curq || !eof) { - if (!eof && curq < maxq) { - union { struct in_addr a; void *p; } pa; - if (!fgets(linebuf, sizeof(linebuf), stdin)) { - eof = 1; - continue; - } - eol = strchr(linebuf, '\n'); - if (eol) *eol = '\0'; - if (!linebuf[0]) continue; - if (dns_pton(AF_INET, linebuf, &pa.a) <= 0) - fprintf(stderr, "%s: invalid address\n", linebuf); - else if (dns_submit_a4ptr(0, &pa.a, dnscb, pa.p) == 0) - fprintf(stderr, "%s: unable to submit query: %s\n", - linebuf, dns_strerror(dns_status(0))); - else - ++curq; - continue; - } - if (curq) { - c = dns_timeouts(0, -1, now); - c = poll(&pfd, 1, c < 0 ? -1 : c * 1000); - now = time(NULL); - if (c) - dns_ioevent(0, now); - } - } - return 0; -} diff --git a/contrib/udns/getopt.c b/contrib/udns/getopt.c deleted file mode 100644 index e15a7a0579e..00000000000 --- a/contrib/udns/getopt.c +++ /dev/null @@ -1,165 +0,0 @@ -/* getopt.c - * Simple getopt() implementation. - * - * Standard interface: - * extern int getopt(int argc, char *const *argv, const char *opts); - * extern int optind; current index in argv[] - * extern char *optarg; argument for the current option - * extern int optopt; the current option - * extern int opterr; to control error printing - * - * Some minor extensions: - * ignores leading `+' sign in opts[] (unemplemented GNU extension) - * handles optional arguments, in form "x::" in opts[] - * if opts[] starts with `:', will return `:' in case of missing required - * argument, instead of '?'. - * - * Compile with -DGETOPT_NO_OPTERR to never print errors internally. - * Compile with -DGETOPT_NO_STDIO to use write() calls instead of fprintf() for - * error reporting (ignored with -DGETOPT_NO_OPTERR). - * Compile with -DGETOPT_CLASS=static to get static linkage. - * Compile with -DGETOPT_MY to redefine all visible symbols to be prefixed - * with "my_", like my_getopt instead of getopt. - * Compile with -DTEST to get a test executable. - * - * Written by Michael Tokarev. Public domain. - */ - -#include - -#ifndef GETOPT_CLASS -# define GETOPT_CLASS -#endif -#ifdef GETOPT_MY -# define optarg my_optarg -# define optind my_optind -# define opterr my_opterr -# define optopt my_optopt -# define getopt my_getopt -#endif - -GETOPT_CLASS char *optarg /* = NULL */; -GETOPT_CLASS int optind = 1; -GETOPT_CLASS int opterr = 1; -GETOPT_CLASS int optopt; - -static char *nextc /* = NULL */; - -#if defined(GETOPT_NO_OPTERR) - -#define printerr(argv, msg) - -#elif defined(GETOPT_NO_STDIO) - -extern int write(int, void *, int); - -static void printerr(char *const *argv, const char *msg) { - if (opterr) { - char buf[64]; - unsigned pl = strlen(argv[0]); - unsigned ml = strlen(msg); - char *p; - if (pl + /*": "*/2 + ml + /*" -- c\n"*/6 > sizeof(buf)) { - write(2, argv[0], pl); - p = buf; - } - else { - memcpy(buf, argv[0], ml); - p = buf + pl; - } - *p++ = ':'; *p++ = ' '; - memcpy(p, msg, ml); p += ml; - *p++ = ' '; *p++ = '-'; *p++ = '-'; *p++ = ' '; - *p++ = optopt; - *p++ = '\n'; - write(2, buf, p - buf); - } -} - -#else - -#include -static void printerr(char *const *argv, const char *msg) { - if (opterr) - fprintf(stderr, "%s: %s -- %c\n", argv[0], msg, optopt); -} - -#endif - -GETOPT_CLASS int getopt(int argc, char *const *argv, const char *opts) { - char *p; - - optarg = 0; - if (*opts == '+') /* GNU extension (permutation) - isn't supported */ - ++opts; - - if (!optind) { /* a way to reset things */ - nextc = 0; - optind = 1; - } - - if (!nextc || !*nextc) { /* advance to the next argv element */ - /* done scanning? */ - if (optind >= argc) - return -1; - /* not an optional argument */ - if (argv[optind][0] != '-') - return -1; - /* bare `-' */ - if (argv[optind][1] == '\0') - return -1; - /* special case `--' argument */ - if (argv[optind][1] == '-' && argv[optind][2] == '\0') { - ++optind; - return -1; - } - nextc = argv[optind] + 1; - } - - optopt = *nextc++; - if (!*nextc) - ++optind; - p = strchr(opts, optopt); - if (!p || optopt == ':') { - printerr(argv, "illegal option"); - return '?'; - } - if (p[1] == ':') { - if (*nextc) { - optarg = nextc; - nextc = NULL; - ++optind; - } - else if (p[2] != ':') { /* required argument */ - if (optind >= argc) { - printerr(argv, "option requires an argument"); - return *opts == ':' ? ':' : '?'; - } - else - optarg = argv[optind++]; - } - } - return optopt; -} - -#ifdef TEST - -#include - -int main(int argc, char **argv) { - int c; - while((c = getopt(argc, argv, "ab:c::")) != -1) switch(c) { - case 'a': - case 'b': - case 'c': - printf("option %c %s\n", c, optarg ? optarg : "(none)"); - break; - default: - return -1; - } - for(c = optind; c < argc; ++c) - printf("non-opt: %s\n", argv[c]); - return 0; -} - -#endif diff --git a/contrib/udns/inet_XtoX.c b/contrib/udns/inet_XtoX.c deleted file mode 100644 index 50b5f8e81f3..00000000000 --- a/contrib/udns/inet_XtoX.c +++ /dev/null @@ -1,327 +0,0 @@ -/* inet_XtoX.c - * Simple implementation of the following functions: - * inet_ntop(), inet_ntoa(), inet_pton(), inet_aton(). - * - * Differences from traditional implementaitons: - * o modifies destination buffers even on error return. - * o no fancy (hex, or 1.2) input support in inet_aton() - * o inet_aton() does not accept junk after an IP address. - * o inet_ntop(AF_INET) requires at least 16 bytes in dest, - * and inet_ntop(AF_INET6) at least 40 bytes - * (traditional inet_ntop() will try to fit anyway) - * - * Compile with -Dinet_XtoX_prefix=pfx_ to have pfx_*() instead of inet_*() - * Compile with -Dinet_XtoX_no_ntop or -Dinet_XtoX_no_pton - * to disable net2str or str2net conversions. - * - * #define inet_XtoX_prototypes and #include "this_file.c" - * to get function prototypes only (but not for inet_ntoa()). - * #define inet_XtoX_decl to be `static' for static visibility, - * or use __declspec(dllexport) or somesuch... - * - * Compile with -DTEST to test against stock implementation. - * - * Written by Michael Tokarev. Public domain. - */ - -#ifdef inet_XtoX_prototypes - -struct in_addr; - -#else - -#include - -#ifdef TEST - -# include -# include -# include -# include -# include -# include -# include -# undef inet_XtoX_prefix -# define inet_XtoX_prefix mjt_inet_ -# undef inet_XtoX_no_ntop -# undef inet_XtoX_no_pton - -#else /* !TEST */ - -struct in_addr { /* declare it here to avoid messing with headers */ - unsigned char x[4]; -}; - -#endif /* TEST */ - -#endif /* inet_XtoX_prototypes */ - -#ifndef inet_XtoX_prefix -# define inet_XtoX_prefix inet_ -#endif -#ifndef inet_XtoX_decl -# define inet_XtoX_decl /*empty*/ -#endif - -#define cc2_(x,y) cc2__(x,y) -#define cc2__(x,y) x##y -#define fn(x) cc2_(inet_XtoX_prefix,x) - -#ifndef inet_XtoX_no_ntop - -inet_XtoX_decl const char * -fn(ntop)(int af, const void *src, char *dst, unsigned size); - -#ifndef inet_XtoX_prototypes - -static int mjt_ntop4(const void *_src, char *dst, int size) { - unsigned i, x, r; - char *p; - const unsigned char *s = _src; - if (size < 4*4) /* for simplicity, disallow non-max-size buffer */ - return 0; - for (i = 0, p = dst; i < 4; ++i) { - if (i) *p++ = '.'; - x = r = s[i]; - if (x > 99) { *p++ = (char)(r / 100 + '0'); r %= 100; } - if (x > 9) { *p++ = (char)(r / 10 + '0'); r %= 10; } - *p++ = (char)(r + '0'); - } - *p = '\0'; - return 1; -} - -static char *hexc(char *p, unsigned x) { - static char hex[16] = "0123456789abcdef"; - if (x > 0x0fff) *p++ = hex[(x >>12) & 15]; - if (x > 0x00ff) *p++ = hex[(x >> 8) & 15]; - if (x > 0x000f) *p++ = hex[(x >> 4) & 15]; - *p++ = hex[x & 15]; - return p; -} - -static int mjt_ntop6(const void *_src, char *dst, int size) { - unsigned i; - unsigned short w[8]; - unsigned bs = 0, cs = 0; - unsigned bl = 0, cl = 0; - char *p; - const unsigned char *s = _src; - - if (size < 40) /* for simplicity, disallow non-max-size buffer */ - return 0; - - for(i = 0; i < 8; ++i, s += 2) { - w[i] = (((unsigned short)(s[0])) << 8) | s[1]; - if (!w[i]) { - if (!cl++) cs = i; - } - else { - if (cl > bl) bl = cl, bs = cs; - } - } - if (cl > bl) bl = cl, bs = cs; - p = dst; - if (bl == 1) - bl = 0; - if (bl) { - for(i = 0; i < bs; ++i) { - if (i) *p++ = ':'; - p = hexc(p, w[i]); - } - *p++ = ':'; - i += bl; - if (i == 8) - *p++ = ':'; - } - else - i = 0; - for(; i < 8; ++i) { - if (i) *p++ = ':'; - if (i == 6 && !bs && (bl == 6 || (bl == 5 && w[5] == 0xffff))) - return mjt_ntop4(s - 4, p, size - (p - dst)); - p = hexc(p, w[i]); - } - *p = '\0'; - return 1; -} - -inet_XtoX_decl const char * -fn(ntop)(int af, const void *src, char *dst, unsigned size) { - switch(af) { - /* don't use AF_*: don't mess with headers */ - case 2: /* AF_INET */ if (mjt_ntop4(src, dst, size)) return dst; break; - case 10: /* AF_INET6 */ if (mjt_ntop6(src, dst, size)) return dst; break; - default: errno = EAFNOSUPPORT; return (char*)0; - } - errno = ENOSPC; - return (char*)0; -} - -inet_XtoX_decl const char * -fn(ntoa)(struct in_addr addr) { - static char buf[4*4]; - mjt_ntop4(&addr, buf, sizeof(buf)); - return buf; -} - -#endif /* inet_XtoX_prototypes */ -#endif /* inet_XtoX_no_ntop */ - -#ifndef inet_XtoX_no_pton - -inet_XtoX_decl int fn(pton)(int af, const char *src, void *dst); -inet_XtoX_decl int fn(aton)(const char *src, struct in_addr *addr); - -#ifndef inet_XtoX_prototypes - -static int mjt_pton4(const char *c, void *dst) { - unsigned char *a = dst; - unsigned n, o; - for (n = 0; n < 4; ++n) { - if (*c < '0' || *c > '9') - return 0; - o = *c++ - '0'; - while(*c >= '0' && *c <= '9') - if ((o = o * 10 + (*c++ - '0')) > 255) - return 0; - if (*c++ != (n == 3 ? '\0' : '.')) - return 0; - *a++ = (unsigned char)o; - } - return 1; -} - -static int mjt_pton6(const char *c, void *dst) { - unsigned short w[8], *a = w, *z, *i; - unsigned v, o; - const char *sc; - unsigned char *d = dst; - if (*c != ':') z = (unsigned short*)0; - else if (*++c != ':') return 0; - else ++c, z = a; - i = 0; - for(;;) { - v = 0; - sc = c; - for(;;) { - if (*c >= '0' && *c <= '9') o = *c - '0'; - else if (*c >= 'a' && *c <= 'f') o = *c - 'a' + 10; - else if (*c >= 'A' && *c <= 'F') o = *c - 'A' + 10; - else break; - v = (v << 4) | o; - if (v > 0xffff) return 0; - ++c; - } - if (sc == c) { - if (z == a && !*c) - break; - else - return 0; - } - if (*c == ':') { - if (a >= w + 8) - return 0; - *a++ = v; - if (*++c == ':') { - if (z) - return 0; - z = a; - if (!*++c) - break; - } - } - else if (!*c) { - if (a >= w + 8) - return 0; - *a++ = v; - break; - } - else if (*c == '.') { - if (a > w + 6) - return 0; - if (!mjt_pton4(sc, d)) - return 0; - *a++ = ((unsigned)(d[0]) << 8) | d[1]; - *a++ = ((unsigned)(d[2]) << 8) | d[3]; - break; - } - else - return 0; - } - v = w + 8 - a; - if ((v && !z) || (!v && z)) - return 0; - for(i = w; ; ++i) { - if (i == z) - while(v--) { *d++ = '\0'; *d++ = '\0'; } - if (i >= a) - break; - *d++ = (unsigned char)((*i >> 8) & 255); - *d++ = (unsigned char)(*i & 255); - } - return 1; -} - -inet_XtoX_decl int fn(pton)(int af, const char *src, void *dst) { - switch(af) { - /* don't use AF_*: don't mess with headers */ - case 2 /* AF_INET */: return mjt_pton4(src, dst); - case 10 /* AF_INET6 */: return mjt_pton6(src, dst); - default: errno = EAFNOSUPPORT; return -1; - } -} - -inet_XtoX_decl int fn(aton)(const char *src, struct in_addr *addr) { - return mjt_pton4(src, addr); -} - -#endif /* inet_XtoX_prototypes */ - -#endif /* inet_XtoX_no_pton */ - -#ifdef TEST - -int main(int argc, char **argv) { - int i; - char n0[16], n1[16]; - char p0[64], p1[64]; - int af = AF_INET; - int pl = sizeof(p0); - int r0, r1; - const char *s0, *s1; - - while((i = getopt(argc, argv, "46a:p:")) != EOF) switch(i) { - case '4': af = AF_INET; break; - case '6': af = AF_INET6; break; - case 'a': case 'p': pl = atoi(optarg); break; - default: return 1; - } - for(i = optind; i < argc; ++i) { - char *a = argv[i]; - - printf("%s:\n", a); - r0 = inet_pton(af, a, n0); - printf(" p2n stock: %s\n", - (r0 < 0 ? "(notsupp)" : !r0 ? "(inval)" : fn(ntop)(af,n0,p0,sizeof(p0)))); - r1 = fn(pton)(af, a, n1); - printf(" p2n this : %s\n", - (r1 < 0 ? "(notsupp)" : !r1 ? "(inval)" : fn(ntop)(af,n1,p1,sizeof(p1)))); - - if ((r0 > 0) != (r1 > 0) || - (r0 > 0 && r1 > 0 && memcmp(n0, n1, af == AF_INET ? 4 : 16) != 0)) - printf(" DIFFER!\n"); - - s0 = inet_ntop(af, n1, p0, pl); - printf(" n2p stock: %s\n", s0 ? s0 : "(inval)"); - s1 = fn(ntop)(af, n1, p1, pl); - printf(" n2p this : %s\n", s1 ? s1 : "(inval)"); - if ((s0 != 0) != (s1 != 0) || - (s0 && s1 && strcmp(s0, s1) != 0)) - printf(" DIFFER!\n"); - - } - return 0; -} - -#endif /* TEST */ diff --git a/contrib/udns/rblcheck.1 b/contrib/udns/rblcheck.1 deleted file mode 100644 index 6c822c01c0b..00000000000 --- a/contrib/udns/rblcheck.1 +++ /dev/null @@ -1,151 +0,0 @@ -.\" rblcheck.1 -.\" rblckeck manpage -.\" -.\" Copyright (C) 2005 Michael Tokarev -.\" This file is part of UDNS library, an async DNS stub resolver. -.\" -.\" This library is free software; you can redistribute it and/or -.\" modify it under the terms of the GNU Lesser General Public -.\" License as published by the Free Software Foundation; either -.\" version 2.1 of the License, or (at your option) any later version. -.\" -.\" This library is distributed in the hope that it will be useful, -.\" but WITHOUT ANY WARRANTY; without even the implied warranty of -.\" MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU -.\" Lesser General Public License for more details. -.\" -.\" You should have received a copy of the GNU Lesser General Public -.\" License along with this library, in file named COPYING.LGPL; if not, -.\" write to the Free Software Foundation, Inc., 59 Temple Place, -.\" Suite 330, Boston, MA 02111-1307 USA - -.TH rblckeck 1 "Apr 2005" "User Utilities" - -.SH NAME -rblckeck \- DNSBL lookup utility - -.SH SYNOPSYS -.B rblcheck -.RB [\| \-s -.IR zone \|] -.RB [\| \-S -.IR zone\-file \|] -.RB [\| \-c \|] -.RB [\| \-tmvq \|] -.RB [\| \-n -.IR nsaddr \|] -.IR address \|.\|.\|. - -.SH DESCRIPTION -.B rblcheck -is a simple command-line to perform DNSBL (DNS-based blocklists) lookups. -For every IP address (or a name, in which case it will be resolved to an -address first), the utility verifies whenever it is listed in a (list of) -DNS blocklists specified with -.B \-s -or -.B \-S -options, optionally obtains text assotiated with the listing (usually it -is either some description about the reason of the listing or an URL -referring to such a description), and displays results on standard output. -.PP -The program is implemented on top of -.BR udns (3) -library. - -.SH OPTIONS - -The following options are recognized by -.BR rblcheck : - -.TP -.B \-s \fIzone\fR -add the given \fIzone\fR DNSBL name to the list of active zones. -.TP -.B \-S \fIzone-file\fR -add list of zones from the named \fIzone-file\fR to the list of -active zones (the file specifies one zone as the first word on a -line, empty lines and lines starting with `#' character are ignored). -.TP -.B \-c -reset active zone list. -.TP -.B \-v -be more verbose, produce more detailed output. -.TP -.B \-q -the opposite for \fB\-v\fR -- produce less detailed output. -.TP -.B \-t -obtain text for listed addresses. -.TP -.B \-n \fInsaddr\fR -Use the given nameserver (given as IPv4 or IPv6 address) instead of the -default. The same effect may be achieved by setting $NSCACHEIP environment -variable. -.TP -.B \-m -stop after first hit, ie after the first address which is found to be -listed. - -.TP -.B \-h -print short help and exit. - -.PP -If no -.BR \-s , -.BR \-S -and -.B \-c -options are given, -.B rblcheck -will try to obtain list of zones using $RBLCHECK_ZONES environment variable, -or ~/.rblcheckrc, or /etc/rblckechrc files, in that order. If no zones are -found, it will exit unsuccessefully. - -.SH "RETURN VALUE" -When no addresses given are listed and no errors occured, -.B rblcheck -exits with code 0. If at least one address is listed, -.B rblcheck -returns 100. In case of DNS errors, -.B rblcheck -returns 2. - -.SH ENVIRONMENT - -.TP -.B $RBLCHECK_ZONES -if no -.BR \-s , -.B \-S -or -.B \-c -option is given, -.B rblcheck -tries this variable to obtain list of DNSBL zones to check against. - -.SH FILES - -.TP -$HOME/.rblcheckrc and /etc/rblcheckrc -if no -.BR \-s , -.B \-S -or -.B \-c -option is given, and no $RBLCHECK_ZONES environment variable is set, -.B rblcheck -will try the two files (the first one that exists) to obtain list of -DNSBL zones to check against. -Each line specifies one zone (only first word in each line is used). -Empty lines and lines starting with `#' character are ignored. - -.SH "SEE ALSO" -.BR dnsget (1) -.BR resolv.conf (5) -.BR udns (3). - -.SH AUTHOR -This program and manual pages are written by Michael Tokarev. diff --git a/contrib/udns/rblcheck.c b/contrib/udns/rblcheck.c deleted file mode 100644 index 82d29deeebf..00000000000 --- a/contrib/udns/rblcheck.c +++ /dev/null @@ -1,378 +0,0 @@ -/* rblcheck.c - dnsbl (rbl) checker application - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#ifdef HAVE_CONFIG_H -# include "config.h" -#endif -#include -#include -#include -#ifdef WINDOWS -# include -#else -# include -# include -# include -# include -#endif -#include -#include -#include -#include "udns.h" - -#ifndef HAVE_GETOPT -# include "getopt.c" -#endif - -static const char *version = "udns-rblcheck 0.4"; -static char *progname; - -static void error(int die, const char *fmt, ...) { - va_list ap; - fprintf(stderr, "%s: ", progname); - va_start(ap, fmt); vfprintf(stderr, fmt, ap); va_end(ap); - putc('\n', stderr); - fflush(stderr); - if (die) - exit(1); -} - -struct rblookup { - struct ipcheck *parent; - struct in_addr key; - const char *zone; - struct dns_rr_a4 *addr; - struct dns_rr_txt *txt; -}; - -struct ipcheck { - const char *name; - int naddr; - int listed; - struct rblookup *lookup; -}; - -#define notlisted ((void*)1) - -static int nzones, nzalloc; -static const char **zones; - -static int do_txt; -static int stopfirst; -static int verbose = 1; -/* verbosity level: - * <0 - only bare As/TXTs - * 0 - what RBL result - * 1(default) - what is listed by RBL: result - * 2 - what is[not ]listed by RBL: result, name lookups - */ - -static int listed; -static int failures; - -static void *ecalloc(int size, int cnt) { - void *t = calloc(size, cnt); - if (!t) - error(1, "out of memory"); - return t; -} - -static void addzone(const char *zone) { - if (nzones >= nzalloc) { - const char **zs = (const char**)ecalloc(sizeof(char*), (nzalloc += 16)); - if (zones) { - memcpy(zs, zones, nzones * sizeof(char*)); - free(zones); - } - zones = zs; - } - zones[nzones++] = zone; -} - -static int addzonefile(const char *fname) { - FILE *f = fopen(fname, "r"); - char linebuf[2048]; - if (!f) - return 0; - while(fgets(linebuf, sizeof(linebuf), f)) { - char *p = linebuf, *e; - while(*p == ' ' || *p == '\t') ++p; - if (*p == '#' || *p == '\n') continue; - e = p; - while(*e && *e != ' ' && *e != '\t' && *e != '\n') - ++e; - *e++ = '\0'; - p = memcpy(ecalloc(e - p, 1), p, e - p); // strdup - addzone(p); - } - fclose(f); - return 1; -} - -static void dnserror(struct rblookup *ipl, const char *what) { - char buf[4*4]; - error(0, "unable to %s for %s (%s): %s", - what, dns_ntop(AF_INET, &ipl->key, buf, sizeof(buf)), - ipl->zone, dns_strerror(dns_status(0))); - ++failures; -} - -static void display_result(struct ipcheck *ipc) { - int j; - struct rblookup *l, *le; - char buf[4*4]; - if (!ipc->naddr) return; - for (l = ipc->lookup, le = l + nzones * ipc->naddr; l < le; ++l) { - if (!l->addr) continue; - if (verbose < 2 && l->addr == notlisted) continue; - if (verbose >= 0) { - dns_ntop(AF_INET, &l->key, buf, sizeof(buf)); - if (ipc->name) printf("%s[%s]", ipc->name, buf); - else printf("%s", buf); - } - if (l->addr == notlisted) { - printf(" is NOT listed by %s\n", l->zone); - continue; - } - else if (verbose >= 1) - printf(" is listed by %s: ", l->zone); - else if (verbose >= 0) - printf(" %s ", l->zone); - if (verbose >= 1 || !do_txt) - for (j = 0; j < l->addr->dnsa4_nrr; ++j) - printf("%s%s", j ? " " : "", - dns_ntop(AF_INET, &l->addr->dnsa4_addr[j], buf, sizeof(buf))); - if (!do_txt) ; - else if (l->txt) { - for(j = 0; j < l->txt->dnstxt_nrr; ++j) { - unsigned char *t = l->txt->dnstxt_txt[j].txt; - unsigned char *e = t + l->txt->dnstxt_txt[j].len; - printf("%s\"", verbose > 0 ? "\n\t" : j ? " " : ""); - while(t < e) { - if (*t < ' ' || *t >= 127) printf("\\x%02x", *t); - else if (*t == '\\' || *t == '"') printf("\\%c", *t); - else putchar(*t); - ++t; - } - putchar('"'); - } - free(l->txt); - } - else - printf("%s", verbose > 0 ? "\n\t" : ""); - free(l->addr); - putchar('\n'); - } - free(ipc->lookup); -} - -static void txtcb(struct dns_ctx *ctx, struct dns_rr_txt *r, void *data) { - struct rblookup *ipl = data; - if (r) { - ipl->txt = r; - ++ipl->parent->listed; - } - else if (dns_status(ctx) != DNS_E_NXDOMAIN) - dnserror(ipl, "lookup DNSBL TXT record"); -} - -static void a4cb(struct dns_ctx *ctx, struct dns_rr_a4 *r, void *data) { - struct rblookup *ipl = data; - if (r) { - ipl->addr = r; - ++listed; - if (do_txt) { - if (dns_submit_a4dnsbl_txt(0, &ipl->key, ipl->zone, txtcb, ipl)) - return; - dnserror(ipl, "submit DNSBL TXT record"); - } - ++ipl->parent->listed; - } - else if (dns_status(ctx) != DNS_E_NXDOMAIN) - dnserror(ipl, "lookup DNSBL A record"); - else - ipl->addr = notlisted; -} - -static int -submit_a_queries(struct ipcheck *ipc, - int naddr, const struct in_addr *addr) { - int z, a; - struct rblookup *rl = ecalloc(sizeof(*rl), nzones * naddr); - ipc->lookup = rl; - ipc->naddr = naddr; - for(a = 0; a < naddr; ++a) { - for(z = 0; z < nzones; ++z) { - rl->key = addr[a]; - rl->zone = zones[z]; - rl->parent = ipc; - if (!dns_submit_a4dnsbl(0, &rl->key, rl->zone, a4cb, rl)) - dnserror(rl, "submit DNSBL A query"); - ++rl; - } - } - return 0; -} - -static void namecb(struct dns_ctx *ctx, struct dns_rr_a4 *rr, void *data) { - struct ipcheck *ipc = data; - if (rr) { - submit_a_queries(ipc, rr->dnsa4_nrr, rr->dnsa4_addr); - free(rr); - } - else { - error(0, "unable to lookup `%s': %s", - ipc->name, dns_strerror(dns_status(ctx))); - ++failures; - } -} - -static int submit(struct ipcheck *ipc) { - struct in_addr addr; - if (dns_pton(AF_INET, ipc->name, &addr) > 0) { - submit_a_queries(ipc, 1, &addr); - ipc->name = NULL; - } - else if (!dns_submit_a4(0, ipc->name, 0, namecb, ipc)) { - error(0, "unable to submit name query for %s: %s\n", - ipc->name, dns_strerror(dns_status(0))); - ++failures; - } - return 0; -} - -static void waitdns(struct ipcheck *ipc) { - struct timeval tv; - fd_set fds; - int c; - int fd = dns_sock(NULL); - time_t now = 0; - FD_ZERO(&fds); - while((c = dns_timeouts(NULL, -1, now)) > 0) { - FD_SET(fd, &fds); - tv.tv_sec = c; - tv.tv_usec = 0; - c = select(fd+1, &fds, NULL, NULL, &tv); - now = time(NULL); - if (c > 0) - dns_ioevent(NULL, now); - if (stopfirst && ipc->listed) - break; - } -} - -int main(int argc, char **argv) { - int c; - struct ipcheck ipc; - char *nameserver = NULL; - int zgiven = 0; - - if (!(progname = strrchr(argv[0], '/'))) progname = argv[0]; - else argv[0] = ++progname; - - while((c = getopt(argc, argv, "hqtvms:S:cn:")) != EOF) switch(c) { - case 's': ++zgiven; addzone(optarg); break; - case 'S': - ++zgiven; - if (addzonefile(optarg)) break; - error(1, "unable to read zonefile `%s'", optarg); - case 'c': ++zgiven; nzones = 0; break; - case 'q': --verbose; break; - case 'v': ++verbose; break; - case 't': do_txt = 1; break; - case 'n': nameserver = optarg; break; - case 'm': ++stopfirst; break; - case 'h': - printf("%s: %s (udns library version %s).\n", - progname, version, dns_version()); - printf("Usage is: %s [options] address..\n", progname); - printf( -"Where options are:\n" -" -h - print this help and exit\n" -" -s service - add the service (DNSBL zone) to the serice list\n" -" -S service-file - add the DNSBL zone(s) read from the given file\n" -" -c - clear service list\n" -" -v - increase verbosity level (more -vs => more verbose)\n" -" -q - decrease verbosity level (opposite of -v)\n" -" -t - obtain and print TXT records if any\n" -" -m - stop checking after first address match in any list\n" -" -n ipaddr - use the given nameserver instead of the default\n" -"(if no -s or -S option is given, use $RBLCHECK_ZONES, ~/.rblcheckrc\n" -"or /etc/rblcheckrc in that order)\n" - ); - return 0; - default: - error(1, "use `%s -h' for help", progname); - } - - if (!zgiven) { - char *s = getenv("RBLCHECK_ZONES"); - if (s) { - char *k; - s = strdup(s); - for(k = strtok(s, " \t"); k; k = strtok(NULL, " \t")) - addzone(k); - free(s); - } - else { /* probably worthless on windows? */ - char *path; - char *home = getenv("HOME"); - if (!home) home = "."; - path = malloc(strlen(home) + 1 + sizeof(".rblcheckrc")); - sprintf(path, "%s/.rblcheckrc", home); - if (!addzonefile(path)) - addzonefile("/etc/rblcheckrc"); - free(path); - } - } - if (!nzones) - error(1, "no service (zone) list specified (-s or -S option)"); - - argv += optind; - argc -= optind; - - if (!argc) - return 0; - - if (dns_init(NULL, 0) < 0) - error(1, "unable to initialize DNS library: %s", strerror(errno)); - if (nameserver) { - dns_add_serv(NULL, NULL); - if (dns_add_serv(NULL, nameserver) < 0) - error(1, "wrong IP address for a nameserver: `%s'", nameserver); - } - if (dns_open(NULL) < 0) - error(1, "unable to initialize DNS library: %s", strerror(errno)); - - for (c = 0; c < argc; ++c) { - if (c && (verbose > 1 || (verbose == 1 && do_txt))) putchar('\n'); - memset(&ipc, 0, sizeof(ipc)); - ipc.name = argv[c]; - submit(&ipc); - waitdns(&ipc); - display_result(&ipc); - if (stopfirst > 1 && listed) break; - } - - return listed ? 100 : failures ? 2 : 0; -} diff --git a/contrib/udns/udns.3 b/contrib/udns/udns.3 deleted file mode 100644 index 23222aae9f7..00000000000 --- a/contrib/udns/udns.3 +++ /dev/null @@ -1,1352 +0,0 @@ -.\" udns.3 -.\" udns library manpage -.\" -.\" Copyright (C) 2005-2014 Michael Tokarev -.\" This file is part of UDNS library, an async DNS stub resolver. -.\" -.\" This library is free software; you can redistribute it and/or -.\" modify it under the terms of the GNU Lesser General Public -.\" License as published by the Free Software Foundation; either -.\" version 2.1 of the License, or (at your option) any later version. -.\" -.\" This library is distributed in the hope that it will be useful, -.\" but WITHOUT ANY WARRANTY; without even the implied warranty of -.\" MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU -.\" Lesser General Public License for more details. -.\" -.\" You should have received a copy of the GNU Lesser General Public -.\" License along with this library, in file named COPYING.LGPL; if not, -.\" write to the Free Software Foundation, Inc., 59 Temple Place, -.\" Suite 330, Boston, MA 02111-1307 USA - -.TH udns 3 "Jan 2014" "Library Functions" - -.SH NAME -udns \- stub DNS resolver library - -.SH SYNOPSYS -.nf -#include -struct \fBdns_ctx\fR; -struct \fBdns_query\fR; -extern struct dns_ctx \fBdns_defctx\fR; -struct dns_ctx *\fIctx\fR; -typedef void \fBdns_query_fn\fR(\fIctx\fR, void *\fIresult\fR, void *\fIdata\fR); -typedef int -\fBdns_parse_fn\fR(const unsigned char *\fIqnd\fR, - const unsigned char *\fIpkt\fR, - const unsigned char *\fIcur\fR, - const unsigned char *\fIend\fR, - void **\fIresultp\fR); - -\fBcc\fR ... -l\fBudns\fR -.fi - -.SH DESCRIPTION - -.PP -The DNS library, \fBudns\fR, implements thread-safe stub DNS resolver -functionality, which may be used both traditional, syncronous way -and asyncronously, with application-supplied event loop. - -.PP -While DNS works with both TCP and UDP, performing UDP query first and -if the result does not fit in UDP buffer (512 bytes max for original -DNS protocol), retrying the query over TCP, the library uses UDP only, -but uses EDNS0 (RFC2671) extensions which allows larger UDP buffers. - -.PP -The library uses single UDP socket to perform all operations even when -asking multiple nameservers. This way, it is very simple to use the -library in asyncronous event-loop applications: an application should -add only single socket to the set of filedescriptors it monitors for I/O. - -.PP -The library uses two main objects, \fIresolver context\fR of type -\fBstruct\ dns_ctx\fR, and \fIquery structure\fR of type -\fBstruct\ dns_query\fR, both are opaque for an application. -Resolver context holds global information about the resolver, -such as list of nameservers to use, list of active requests and the like. -Query objects holds information about a single DNS query in progress and -are allocated/processed/freed by the library. Pointer to query structure -may be treated as an identifier of an in-progress query and may be used -to cancel the asyncronous query or to wait for it to complete. - -.PP -Asyncronous interface works as follows. An application initializes -resolver context, submits any number of queries for it using one of -supplied \fBdns_submit_\fIXXX\fR() routines (each return the query -identifier as pointer to query structure), waits for input on the -UDP socket used by the library, and gives some control to the library -by calling \fBdns_ioevent\fR() and \fBdns_timeouts\fR() routines when -appropriate. The library performs all necessary processing and executes -application supplied callback routine when a query completes (either -successefully or not), giving it the result if any, pointer to the -resolver context (from which completion status may be obtained), and -the data pointer supplied by an application when the query has been -submitted. When submitting a query, an application requests how to -handle the reply -- to either return raw DNS reply packet for its -own low-level processing, or it may provide an address of \fIparsing -routine\fR of type \fBdns_parse_fn\fR to perform conversion of on-wire -format into easy to use data structure (the library provides parsing -routines for several commonly used resource record types, as well as -type-safe higher-level inteface that requests parsing automatically). -The I/O monitoring and timeout handling may be either traditional -select() or poll() based, or any callback-driven technique may be -used. - -.PP -Additionally, the library provides traditional syncronous interface, -which may be intermixed with asyncronous calls (during syncronous -query processing, other asyncronous queries for the same resolver -context continued to be processed as usual). An application uses -one of numerous \fBdns_resolve_\fIXXX\fR() routines provided by the -library to perform a query. As with asyncronous interface, an -application may either request to return raw DNS packet or type-specific -data structure by providing the parsing routine to handle the reply. -Every routine from \fBdns_resolve_\fIXXX\fR() series return pointer -to result or NULL in case of any error. Query completion status -(or length of the raw DNS packet) is available from the resolver -context using \fBdns_status\fR() routine, the same way as for the -asyncronous interface. - -.PP -Internally, library uses on-wire format of domain names, referred -to as \fIDN format\fR in this manual page. This is a series of domain -\fIlabels\fR whith preceeding length byte, terminated by zero-length -label wich is integral part of the DN format. There are several routines -provided to convert from traditional asciiz string to DN and back. -Higher-level type-specific query interface hides the DN format from -an application. - -.SH "COMMON DEFINITIONS" - -.PP -Every DNS Resource Record (RR) has a \fItype\fR and a \fIclass\fR. -The library defines several integer constants, \fBDNS_C_\fIXXX\fR and -\fBDNS_T_\fIXXX\fR, to use as symbolic names for RR classes and types, -such as \fBDNS_C_IN\fR for Internet class, \fBDNS_T_A\fR for IPv4 -address record type and so on. See udns.h header file for complete list -of all such constants. - -.PP -The following constants are defined in udns.h header file: -.IP "\fBDNS_MAXDN\fR (255 bytes)" -Maximum length of the domain name in internal (on-wire) DN format. -.IP "\fBDNS_MAXLABEL\fR (63 bytes)" -Maximum length of a single label in DN format. -.IP "\fBDNS_MAXNAME\fR (1024 bytes)" -Maximum length of asciiz format of a domain name. -.IP "\fBDNS_HSIZE\fR (12 bytes)" -Size of header in DNS packet. -.IP "\fBDNS_PORT\fR (53)" -Default port to use when contacting a DNS server. -.IP "\fBDNS_MAXSERV\fR (6 servers)" -Maximum number of DNS servers to use. -.IP "\fBDNS_MAXPACKET\fR (512 bytes)" -Maximum length of DNS UDP packet as specified by original DNS protocol -.IP "\fBDNS_EDNS0PACKET\fR (4096 bytes)" -Default length of DNS UDP packet (with EDNS0 extensions) the library uses. -Note that recursive nameservers usually resides near the client asking them -to resolve names, e.g. on the same LAN segment or even on the same host, so -UDP packet fragmentation isn't a problem in most cases. Note also that -the size of actual packets will be as many bytes as actual reply size requires, -which is smaller than this value in almost all cases. - -.PP -Additionally, several constants are defined to simplify work with raw DNS -packets, such as DNS response codes (\fBDNS_R_\fIXXX\fR), DNS header layout -(\fBDNS_H_\fIXXX\fR) and others. Again, see udns.h for complete list. -Library error codes (\fBDNS_E_\fIXXX\fR) are described later in this -manual page. - -.SH "RESOLVER CONTEXT" - -.PP -Resolver context, of type \fBstruct\ dns_ctx\fR, is an object which is -opaque to an application. Several routines provided by the library -to initialize, copy and free resolver contexts. Most other high-level -routines in this library expects a pointer to resolver context, \fIctx\fR, -as the first argument. There is a default resolver context available, -named \fBdns_defctx\fR. When the context pointer \fIctx\fR passed to -a routine is NULL, \fBdns_defctx\fR is used. Several resolver contexts -may be active at the same time, for example, when an application is -multi-threaded and each thread uses resolver. -.PP -In order to use the library, an application should initialize and open -one or more resolver context objects. These are two separate actions, -performed by \fBdns_init\fR() (or \fBdns_reset\fR()), and \fBdns_open\fR(). -Between the two calls, an application is free to pefrorm additional -initialisation, such as setting custom nameservers, options or domain search -lists. Optionally, in case no additional custom initialisation is required, -\fBdns_init\fR() may open the context if \fIdo_open\fR argument (see below) -is non-zero. -.PP -When initializing resolver context, the library uses information from -system file /etc/resolv.conf (see \fBresolv.conf\fR(5)), consults -environment variables \fB$LOCALDOMAIN\fR, \fB$NSCACHEIP\fR, -\fB$NAMESERVERS\fR and \fB$RES_OPTIONS\fR, and local host name to obtain -list of local nameservers, domain name search list and various resolver -options. -.PP -The following routines to initialize resolver context are available: -.PP -.nf -void \fBdns_reset\fR(\fIctx\fR) -int \fBdns_init\fR(\fIctx\fR, int \fIdo_open\fR) -.fi -.RS -\fBdns_reset\fR() resets a given resolver context to default values, -preparing it to be opened by \fBdns_open\fR(). -It is ok to call this routine against opened and active context - all active -queries will be dropped, sockets will be closed and so on. This routine -does not initialize any parameters from system configuration files, use -\fBdns_init\fR() for this. There's no error return - operation always -succeeds. \fBdns_init\fR() does everything \fBdns_reset\fR() does, -plus initializes various parameters of the context according to system -configuration and process environment variables. If \fIdo_open\fR is -non-zero, \fBdns_init\fR() calls \fIdns_open\fR(), so that the whole -library initialisation is performed in a single step. -.RE -.PP -.nf -struct dns_ctx *\fBdns_new\fR(struct dns_ctx *\fIcopy\fR) -void \fBdns_free\fR(\fIctx\fR) -.fi -.RS -\fBdns_new\fR() allocates new resolver context and copies all parameters -for a given resolver context \fIcopy\fR, or default context if \fIcopy\fR -is NULL, and returns pointer to the newly allocated context. The context -being copied should be initialized. -\fBdns_new\fR() may fail if there's no memory available to make a copy -of \fIcopy\fR, in which case the routine will return NULL pointer. -\fBdns_free\fR() is used to close assotiated socket and free resolver -context resources and cancelling (abandoming) all active queries -assotiated with it. It's an error to free \fBdns_defctx\fR, only -dynamically allocated contexts returned by \fBdns_new\fR() are allowed -to be freed by \fBdns_free\fR(). -.RE -.PP -.nf -int \fBdns_add_serv\fR(\fIctx\fR, const char *\fIservaddr\fR) -int \fBdns_add_serv_s\fR(\fIctx\fR, const struct sockaddr *\fIsa\fR) -int \fBdns_add_srch\fR(\fIctx\fR, const char *\fIsrch\fR) -.fi -.RS -Add an element to list of nameservers (\fBdns_add_serv\fR(), as -asciiz-string \fIservaddr\fR with an IP address of the nameserver, -and \fBdns_add_serv_s\fR(), as initialized socket address \fIsa\fR), -or search list (\fBdns_add_srch\fR(), as a pointer to domain name) -for the given context \fIctx\fR. If the last argument is a NULL -pointer, the corresponding list (search or nameserver) is reset -instead. Upon successeful completion, each routine returns new -number of elements in the list in question. On error, negative -value is returned and global variable \fBerrno\fR is set appropriately. -It is an error to call any of this functions if the context is -opened (after \fBdns_open\fR() or \fBdns_init\fR() with non-zero argument). -.RE -.PP -.nf -int \fBdns_set_opts\fR(\fIctx\fR, const char *\fIopts\fR) -.fi -.RS -set resolver context options from \fIopts\fR string, in the same way as -processing \fBoptions\fR statement in resolv.conf and \fB$RES_OPTIONS\fR -environment variable. Return number of unrecognized/invalid options -found (all recognized and valid options gets processed). -.RE -.PP -.nf -void \fBdns_set_opt\fR(\fIctx\fR, int \fIopt\fR, \fIval\fR) -.fi -.RS -.B TODO -The \fIflags\fR argument is a bitmask with the following bits defined: -.IP \fBDNS_NOSRCH\fR -do not perform domain name search in search list. -.IP \fBDNS_NORD\fR -do not request recursion when performing queries -(i.e. don't set RD flag in querues). -.IP \fBDNS_AAONLY\fR -request authoritative answers only (i.e. set AA -flag in queries). -.RE - -.PP -.nf -int \fBdns_open\fR(\fIctx\fR) -int \fBdns_sock\fR(const \fIctx\fR) -void \fBdns_close\fR(\fIctx\fR) -.fi -.RS -\fBdns_open\fR() opens the UDP socket used for queries if not already -open, and return assotiated filedescriptor (or negative value in case -of error). Before any query can be submitted, the context should be -opened using this routine. And before opening, the context should be -initialized. -\fBdns_sock\fR() return the UDP socket if open, or -1 if not. -\fBdns_close\fR() closes the UDP socket if it was open, and drops all active -queries if any. -.RE - -.PP -.nf -int \fBdns_active\fR(const \fIctx\fR) -.fi -.RS -return number of active queries queued for the given context -\fIctx\fR, or zero if none. -.RE - -.PP -.nf -int \fBdns_status\fR(const \fIctx\fR) -.fi -.RS -return status code from last operation. When using syncronous -interface, this is the query completion status of the last query. -With asyncronous interface, from within the callback routine, -this is the query completion status of the query for which the -callback is being called. When query submission fails, this -is the error code indicating failure reason. All error codes -are negative and are represented by \fBDNS_E_\fIXXX\fR constants -described below. -.RE - -.PP -.nf -void \fBdns_ioevent\fR(\fIctx\fR, time_t \fInow\fR) -.fi -.RS -this routine may be called by an application to process I/O -events on the UDP socket used by the library, as returned -by \fBdns_sock\fR(). The routine tries to receive incoming -UDP datagram from the socket and process it. The socket is -set up to be non-blocking, so it is safe to call the routine -even if there's no data to read. The routine will process -as many datagrams as are queued for the socket, so it is -safe to use it with either level-triggered or edge-triggered -I/O monitoring model. The \fInow\fR argument is either a -current time as returned by \fBtime\fR(), or 0, in which -case the routine will obtain current time by it's own. -.RE - -.PP -.nf -int \fBdns_timeouts\fR(\fIctx\fR, int \fImaxwait\fR, time_t \fInow\fR) -.fi -.RS -process any pending timeouts and return number of secounds -from current time (\fInow\fR if it is not 0) to the time when -the library wants the application to pass it control to process -more queued requests. In case when there are no requests pending, -this time is -1. The routine will not request a time larger than -\fImaxwait\fR secounds if it is greather or equal to zero. If -\fInow\fR is 0, the routine will obtain current time by it's own; -when it is not 0, it should contain current time as returned by -\fBtime\fR(). -.RE - -.PP -.nf -typedef void \fBdns_utm_fn\fR(\fIctx\fR, int \fItimeout\fR, void *\fIdata\fR) -void \fBdns_set_tmcbck\fR(\fIctx\fR, dns_utm_fn *\fIutmfn\fR, void *\fIdata\fR) -.fi -.RS -An application may use custom callback-based I/O multiplexing mechanism. -Usually such a mechanism have concept of a \fItimer\fR, and an ability -to register a timer event in a form of a callback routine which will -be executed after certain amount of time. In order to use such an -event mechanism, udns provides an ability to register and de-register -timer events necessary for internal processing using whatever event -mechanism an application uses. For this to work, it is possible to -assotiate a pointer to a routine that will perform necessary work for -(de)registering timer events with a given resolver context, and -udns will call that routine at appropriate times. Prototype of -such a routine is shown by \fBdns_utm_fn\fR typedef above. Libudns -assotiates single timer with resolver context. User-supplied \fIutmfn\fR -routine will be called by the library with the following arguments: -.IP "\fIctx\fR == NULL" -delete user timer, at context free time or when an application changes -user timer request routine using \fBdns_set_tmcbck\fR(); -.IP "\fIctx\fR != NULL, \fItimeout\fR < 0" -don't fire timer anymore, when there are no active requests; -.IP "\fIctx\fR != NULL, \fItimeout\fR == 0" -fire timer at the next possibility, but not immediately; -.IP "\fIctx\fR != NULL, \fItimeout\fR > 0" -fire timer after \fItimeout\fR seconds after now. -.PP -The \fIdata\fR argument passed to the routine will be the same -as passed to \fBdns_set_tmcbck\fR(). -.PP -When a timer expires, an application should call \fBdns_timeouts\fR() -routine (see below). Non-callback timer usage is provided too. -.RE - -.PP -.B XXXX TODO: some more resolver context routines, like dns_set_dbgfn() etc. - -.SH "QUERY INTERFACE" - -.PP -There are two ways to perform DNS queries: traditional syncronous -way, when udns performs all the necessary processing and return -control to the application only when the query completes, and -asyncronous way, when an application submits one or more queries -to the library using given resolver context, and waits for completion -by monitoring filedescriptor used by library and calling library -routines to process input on that filedescriptor. Asyncronous mode -works with callback routines: an application supplies an address of -a routine to execute when the query completes, and a data pointer, -which is passed to the callback routine. - -.PP -Queries are submitted to the library in a form of \fBstruct\ dns_query\fR. -To perform asyncronous query, an application calls one of the -\fBdns_submit_\fIXXX\fR() rounines, and provides necessary information -for a callback, together with all the query parameters. -When the query completes, library will call application-supplied callback -routine, giving it the resolver context (wich holds query completion status), -dynamically allocated result (which will be either raw DNS packet or, if -applicatin requested parsing the result by specifying non-NULL parse routine, -ready-to-use type-specific structure), and a data pointer provided by an -application when it submitted the query. It is the application who's -responsible for freeing the result memory. -.PP -Generic query callback routine looks like this: -.nf -typedef void -\fBdns_query_fn\fR(\fIctx\fR, void *\fIresult\fR, void *\fIdata\fR) -.fi -Type-specific query interface expects similar form of callback -routine with the only difference in type of \fBresult\fR argument, -which will be pointer to specific data structure (decoded reply) -instead of this void pointer to raw DNS packet data. - -.PP -Result parsing routine looks like this: -.nf -typedef int -\fBdns_parse_fn\fR(const unsigned char *\fIqdn\fR, - const unsigned char *\fIpkt\fR, - const unsigned char *\fIcur\fR, - const unsigned char *\fIend\fR, - void **\fIresultp\fR); -.fi -When called by the library, the arguments are as follows: -\fIpkt\fR points to the start of the packet received; -\fIend\fR points past the end of the packet received; -\fIcur\fR points past the query DN in the query section of the -packet; -\fIqdn\fR points to the original query DN. -The routine should allocate a single buffer to hold the result, -parse the reply filling in the buffer, and return the buffer -using \fIresultp\fR argument. It returns 0 in case of error, -or udns error code (\fBDNS_E_\fIXXX\fR constants) in case of -error. -Note that by the time when the parse routine is called by the -library, packet is already verified to be a reply to the -original query, by matching query DN, query class and query type. - -.PP -Type-specific query inteface supplies necessary parsing routines -automatically. - -.PP -In case of error, query completion status as returned by -\fBdns_status\fR(\fIctx\fR), will contain one of the following values: -.IP "positive value" -length of raw DNS packet if parsing is not requested. -.IP 0 -the query was successeful and the \fIreply\fR points to type-specific -data structure. -.IP \fBDNS_E_TEMPFAIL\fR -temporary error, the resolver nameserver was not able to -process our query or timed out. -.IP \fBDNS_E_PROTOCOL\fR -protocol error, a nameserver returned malformed reply. -.IP \fBDNS_E_NXDOMAIN\fR -the domain name does not exist. -.IP \fBDNS_E_NODATA\fR -there is no data of requested type found. -.IP \fBDNS_E_NOMEM\fR -out of memory while processing request. -.IP \fBDNS_E_BADQUERY\fR -some aspect of the query (most common is the domain name in question) -is invalid, and the library can't even start a query. - -.PP -Library provides two series of routines which uses similar interface -- -one for asyncronous queries and another for syncronous queries. There -are two general low-level routines in each series to submit (asyncronous -interface) and resolve (syncronous interface) queries, as well as several -type-specific routines with more easy-to-use interfaces. To submit -an asyncronous query, use one of \fBdns_submit_\fIXXX\fR() routine, each -of which accepts query parameters, pointers to callback routine and to -callback data, and optional current time hint. Note type-specific -\fBdns_submit_\fIXXX\fR() routines expects specific type of the callback -routine as well, which accepts reply as a pointer to corresponding -structure, not a void pointer). Every \fBdns_submit_\fIXXX\fR() routine -return pointer to internal query structure of type struct\ dns_query, -used as an identifier for the given query. - -.PP -To resolve a query syncronously, use one of \fBdns_resolve_\fIXXX\fR() -routines, which accepts the same query parameters (but not the -callback pointers) as corresponding \fBdns_submit_\fIXXX\fR(), and -return the query result, which is the same as passed to the callback -routine in case of asyncronous interface. - -.PP -In either case, the result memory (if the query completed successefully) -is dynamically allocated and should be freed by an application. If -the query failed for any reason, the result will be NULL, and error -status will be available from \fBdns_status\fR(\fIctx\fR) routine -as shown above. - -.PP -.nf -struct dns_query * -\fBdns_submit_dn\fR(\fIctx\fR, - const unsigned char *\fIdn\fR, \fIqcls\fR, \fIqtyp\fR, \fIflags\fR, - \fIparse\fR, \fIcbck\fR, \fIdata\fR) -struct dns_query * -\fBdns_submit_p\fR(\fIctx\fR, - const char *\fIname\fR, \fIqcls\fR, \fIqtyp\fR, \fIflags\fR, - \fIparse\fR, \fIcbck\fR, \fIdata\fR) - enum dns_class \fIqcls\fR; - enum dns_type \fIqtyp\fR; - int \fIflags\fR; - dns_parse_fn *\fIparse\fR; - dns_query_fn *\fIcbck\fR; - void *\fIdata\fR; -.fi -.RS -submit a query for processing for the given resolver context \fIctx\fR. -Two routines differs only in 3rd argument, which is domain name in -DN format (\fIdn\fR) or asciiz string (\fIname\fR). The query will be -performed for the given domain name, with type \fIqtyp\fR in class \fIqcls\fR, -using option bits in \fIflags\fR, using RR parsing routine pointed by -\fIparse\fR if not-NULL, and upon completion, \fIcbck\fR function will -be called with the \fIdata\fR argument. -In case of successeful query submission, -the routine return pointer to internal query structure which may be treated -as an identifier of the query as used by the library, and may be used as an -argument for \fBdns_cancel\fR() routine. In case of error, NULL will be -returned, and context error status (available using \fIdns_status\fR() routine) -will be set to corresponding error code, which in this case may be -DNS_E_BADQUERY if the \fIname\fR of \fIdn\fR is invalid, DNS_E_NOMEM if -there's no memory available to allocate query structure, or DNS_E_TEMPFAIL -if an internal error occured. -.RE - -.PP -.nf -void *\fBdns_resolve_dn\fR(\fIctx\fR, - const unsigned char *\fIdn\fR, \fIqcls\fR, \fIqtyp\fR, \fIflags\fR, \fIparse\fR); -void *\fBdns_resolve_p\fR(\fIctx\fR, - const char *\fIname\fR, \fIqcls\fR, \fIqtyp\fR, \fIflags\fR, \fIparse\fR) - enum dns_class \fIqcls\fR; - enum dns_type \fIqtyp\fR; - int \fIflags\fR; - dns_parse_fn *\fIparse\fR; -.fi -.RS -syncronous interface. The routines perform all the steps necessary to resolve -the given query and return the result. If there's no positive result for any -reason, all the routines return NULL, and set context error status (available -using \fBdns_status\fR() routine) to indicate the error code. If the query -was successeful, context status code will contain either the length of the -raw DNS reply packet if \fIparse\fR argument was NULL (in which case the return -value is pointer to the reply DNS packet), or 0 (in which case the return value -is the result of \fIparse\fR routine). If the query successeful (return value -is not NULL), the memory returned was dynamically allocated by the library -and should be free()d by application after use. -.RE - -.PP -.nf -void *\fBdns_resolve\fR(\fIctx\fR, struct dns_query *\fIq\fR) -.fi -.RS -wait for the given query \fIq\fR, as returned by one of -\fBdns_submit_\fIXXX\fR() routines, for completion, and -return the result. The callback routine will not be called -for this query. After completion, the query identifier \fIq\fR -is not valid. Both \fBdns_resolve_dn\fR() and \fBdns_resolve_p\fR() -are just wrappers around corresponding submit routines and this -\fBdns_resolve\fR() routine. -.RE - -.PP -.nf -void \fBdns_cancel\fR(\fIctx\fR, struct dns_query *\fIq\fR) -.fi -.RS -cancel an active query \fIq\fR, without calling a callback routine. -After completion, the query identifier \fIq\fR is not valid. -.RE - -.SH "TYPE-SPECIFIC QUERIES" - -.PP -In addition to the generic low-level query interface, the library provides -a set of routines to perform specific queries in a type-safe manner, as -well as parsers for several well-known resource record types. The library -implements high-level interface for A, AAAA, PTR, MX and TXT records -and DNSBL and RHSBL functionality. These routines returns specific types -as result of a query, instead of raw DNS packets. The following types -and routines are available. - -.PP -.nf -struct \fBdns_rr_null\fR { - char *\fBdnsn_qname\fR; /* original query name */ - char *\fBdnsn_cname\fR; /* canonical name */ - unsigned \fBdnsn_ttl\fR; /* Time-To-Live (TTL) value */ - int \fBdnsn_nrr\fR; /* number of records in the set */ -}; -.fi -.PP -NULL RR set, used as a base for all other RR type structures. -Every RR structure as used by the library have four standard -fields as in struct\ \fBdns_rr_null\fR. - -.SS "IN A Queries" -.PP -.nf -struct \fBdns_rr_a4\fR { /* IN A RRset */ - char *\fBdnsa4_qname\fR; /* original query name */ - char *\fBdnsa4_cname\fR; /* canonical name */ - unsigned \fBdnsa4_ttl\fR; /* Time-To-Live (TTL) value */ - int \fBdnsa4_nrr\fR; /* number of addresses in the set */ - struct in_addr \fBdnsa4_addr\fR[]; /* array of addresses */ -}; -typedef void - \fBdns_query_a4_fn\fR(\fIctx\fR, struct dns_rr_a4 *\fIresult\fR, \fIdata\fR) -dns_parse_fn \fBdns_parse_a4\fB; -struct dns_query * -\fBdns_submit_a4\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR, - dns_query_a4_fn *\fIcbck\fR, \fIdata\fR); -struct dns_rr_a4 * -\fBdns_resolve_a4\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR); -.fi -.PP -The \fBdns_rr_a4\fR structure holds a result of an \fBIN A\fR query, -which is an array of IPv4 addresses. Callback routine for IN A queries -expected to be of type \fBdns_query_a4_fn\fR, which expects pointer to -\fBdns_rr_a4\fR structure as query result instead of raw DNS packet. -The \fBdns_parse_a4\fR() is used to convert raw DNS reply packet into -\fBdns_rr_a4\fR structure (it is used internally and may be used directly too -with generic query interface). Routines \fBdns_submit_a4\fR() and -\fBdns_resolve_a4\fR() are used to perform A IN queries in a type-safe -manner. The \fIname\fR parameter is the domain name in question, and -\fIflags\fR is query flags bitmask, with one bit, DNS_NOSRCH, of practical -interest (if the \fIname\fR is absolute, that is, it ends up with a dot, -DNS_NOSRCH flag will be set automatically). - -.SS "IN AAAA Queries" -.PP -.nf -struct \fBdns_rr_a6\fR { /* IN AAAA RRset */ - char *\fBdnsa6_qname\fR; /* original query name */ - char *\fBdnsa6_cname\fR; /* canonical name */ - unsigned \fBdnsa6_ttl\fR; /* Time-To-Live (TTL) value */ - int \fBdnsa6_nrr\fR; /* number of addresses in the set */ - struct in6_addr \fBdnsa6_addr\fR[]; /* array of addresses */ -}; -typedef void - \fBdns_query_a6_fn\fR(\fIctx\fR, struct dns_rr_a6 *\fIresult\fR, \fIdata\fR) -dns_parse_fn \fBdns_parse_a6\fB; -struct dns_query * -\fBdns_submit_a6\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR, - dns_query_a6_fn *\fIcbck\fR, \fIdata\fR); -struct dns_rr_a6 * -\fBdns_resolve_a6\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR); -.fi -.PP -The \fBdns_rr_a6\fR structure holds a result of an \fBIN AAAA\fR query, -which is an array of IPv6 addresses. Callback routine for IN AAAA queries -expected to be of type \fBdns_query_a6_fn\fR, which expects pointer to -\fBdns_rr_a6\fR structure as query result instead of raw DNS packet. -The \fBdns_parse_a6\fR() is used to convert raw DNS reply packet into -\fBdns_rr_a6\fR structure (it is used internally and may be used directly too -with generic query interface). Routines \fBdns_submit_a6\fR() and -\fBdns_resolve_a6\fR() are used to perform AAAA IN queries in a type-safe -manner. The \fIname\fR parameter is the domain name in question, and -\fIflags\fR is query flags bitmask, with one bit, DNS_NOSRCH, of practical -interest (if the \fIname\fR is absolute, that is, it ends up with a dot, -DNS_NOSRCH flag will be set automatically). - -.SS "IN PTR Queries" -.PP -.nf -struct \fBdns_rr_ptr\fR { /* IN PTR RRset */ - char *\fBdnsptr_qname\fR; /* original query name */ - char *\fBdnsptr_cname\fR; /* canonical name */ - unsigned \fBdnsptr_ttl\fR; /* Time-To-Live (TTL) value */ - int \fBdnsptr_nrr\fR; /* number of domain name pointers */ - char *\fBdnsptr_ptr\fR[]; /* array of domain name pointers */ -}; -typedef void - \fBdns_query_ptr_fn\fR(\fIctx\fR, struct dns_rr_ptr *\fIresult\fR, \fIdata\fR) -dns_parse_fn \fBdns_parse_ptr\fB; -struct dns_query * -\fBdns_submit_a4ptr\fB(\fIctx\fR, const struct in_addr *\fBaddr\fR, - dns_query_ptr_fn *\fIcbck\fR, \fIdata\fR); -struct dns_rr_ptr * -\fBdns_resolve_a4ptr\fB(\fIctx\fR, const struct in_addr *\fBaddr\fR); -struct dns_query * -\fBdns_submit_a6ptr\fB(\fIctx\fR, const struct in6_addr *\fBaddr\fR, - dns_query_ptr_fn *\fIcbck\fR, \fIdata\fR); -struct dns_rr_ptr * -\fBdns_resolve_a6ptr\fB(\fIctx\fR, const struct in6_addr *\fBaddr\fR); -.fi -.PP -The \fBdns_rr_ptr\fR structure holds a result of an IN PTR query, which -is an array of domain name pointers for a given IPv4 or IPv6 address. -Callback routine for IN PTR queries expected to be of type -\fBdns_query_ptr_fn\fR, which expects pointer to \fBdns_rr_ptr\fR -structure as query result instead of raw DNS packet. The \fBdns_parse_ptr\fR() -is used to convert raw DNS reply packet into \fBdns_rr_ptr\fR structure -(it is used internally and may be used directly too with generic query -interface). Routines \fBdns_submit_a4ptr\fR() and \fBdns_resolve_a4ptr\fR() -are used to perform IN PTR queries for IPv4 addresses in a type-safe -manner. Routines \fBdns_submit_a6ptr\fR() and \fBdns_resolve_a6ptr\fR() -are used to perform IN PTR queries for IPv6 addresses. - -.SS "IN MX Queries" -.PP -.nf -struct \fBdns_mx\fR { /* single MX record */ - int \fBpriority\fR; /* priority value of this MX */ - char *\fBname\fR; /* domain name of this MX */ -}; -struct \fBdns_rr_mx\fR { /* IN MX RRset */ - char *\fBdnsmx_qname\fR; /* original query name */ - char *\fBdnsmx_cname\fR; /* canonical name */ - unsigned \fBdnsmx_ttl\fR; /* Time-To-Live (TTL) value */ - int \fBdnsmx_nrr\fR; /* number of mail exchangers in the set */ - struct dns_mx \fBdnsmx_mx\fR[]; /* array of mail exchangers */ -}; -typedef void - \fBdns_query_mx_fn\fR(\fIctx\fR, struct dns_rr_mx *\fIresult\fR, \fIdata\fR) -dns_parse_fn \fBdns_parse_mx\fB; -struct dns_query * -\fBdns_submit_mx\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR, - dns_query_mx_fn *\fIcbck\fR, \fIdata\fR); -struct dns_rr_mx * -\fBdns_resolve_mx\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR); -.fi -.PP -The \fBdns_rr_mx\fR structure holds a result of an IN MX query, which -is an array of mail exchangers for a given domain. Callback routine for IN MX -queries expected to be of type \fBdns_query_mx_fn\fR, which expects pointer to -\fBdns_rr_mx\fR structure as query result instead of raw DNS packet. -The \fBdns_parse_mx\fR() is used to convert raw DNS reply packet into -\fBdns_rr_mx\fR structure (it is used internally and may be used directly too -with generic query interface). Routines \fBdns_submit_mx\fR() and -\fBdns_resolve_mx\fR() are used to perform IN MX queries in a type-safe -manner. The \fIname\fR parameter is the domain name in question, and -\fIflags\fR is query flags bitmask, with one bit, DNS_NOSRCH, of practical -interest (if the \fIname\fR is absolute, that is, it ends up with a dot, -DNS_NOSRCH flag will be set automatically). - -.SS "TXT Queries" -.PP -.nf -struct \fBdns_txt\fR { /* single TXT record */ - int \fBlen\fR; /* length of the text */ - unsigned char *\fBtxt\fR; /* pointer to the text */ -}; -struct \fBdns_rr_txt\fR { /* TXT RRset */ - char *\fBdnstxt_qname\fR; /* original query name */ - char *\fBdnstxt_cname\fR; /* canonical name */ - unsigned \fBdnstxt_ttl\fR; /* Time-To-Live (TTL) value */ - int \fBdnstxt_nrr\fR; /* number of text records in the set */ - struct dns_txt \fBdnstxt_txt\fR[]; /* array of TXT records */ -}; -typedef void - \fBdns_query_txt_fn\fR(\fIctx\fR, struct dns_rr_txt *\fIresult\fR, \fIdata\fR) -dns_parse_fn \fBdns_parse_txt\fB; -struct dns_query * -\fBdns_submit_txt\fB(\fIctx\fR, const char *\fIname\fR, enum dns_class \fIqcls\fR, - int \fIflags\fR, dns_query_txt_fn *\fIcbck\fR, \fIdata\fR); -struct dns_rr_txt * -\fBdns_resolve_txt\fB(\fIctx\fR, const char *\fIname\fR, - enum dns_class \fIqcls\fR, int \fIflags\fR); -.fi -.PP -The \fBdns_rr_txt\fR structure holds a result of a TXT query, which is an -array of text records for a given domain name. Callback routine for TXT -queries expected to be of type \fBdns_query_txt_fn\fR, which expects pointer -to \fBdns_rr_txt\fR structure as query result instead of raw DNS packet. -The \fBdns_parse_txt\fR() is used to convert raw DNS reply packet into -\fBdns_rr_txt\fR structure (it is used internally and may be used directly too -with generic query interface). Routines \fBdns_submit_txt\fR() and -\fBdns_resolve_txt\fR() are used to perform IN MX queries in a type-safe -manner. The \fIname\fR parameter is the domain name in question, and -\fIflags\fR is query flags bitmask, with one bit, DNS_NOSRCH, of practical -interest (if the \fIname\fR is absolute, that is, it ends up with a dot, -DNS_NOSRCH flag will be set automatically). Note that each TXT string -is represented by \fBstruct\ dns_txt\fR, while zero-terminated (and the -len field of the structure does not include the terminator), may contain -embedded null characters -- content of TXT records is not interpreted -by the library in any way. - -.SS "SRV Queries" -.PP -.nf -struct \fBdns_srv\fR { /* single SRV record */ - int \fBpriority\fR; /* priority of the record */ - int \fBweight\fR; /* weight of the record */ - int \fBport\fR; /* the port number to connect to */ - char *\fBname\fR; /* target host name */ -}; -struct \fBdns_rr_srv\fR { /* SRV RRset */ - char *\fBdnssrv_qname\fR; /* original query name */ - char *\fBdnssrv_cname\fR; /* canonical name */ - unsigned \fBdnssrv_ttl\fR; /* Time-To-Live (TTL) value */ - int \fBdnssrv_nrr\fR; /* number of text records in the set */ - struct dns_srv \fBdnssrv_srv\fR[]; /* array of SRV records */ -}; -typedef void - \fBdns_query_srv_fn\fR(\fIctx\fR, struct dns_rr_srv *\fIresult\fR, \fIdata\fR) -dns_parse_fn \fBdns_parse_srv\fB; -struct dns_query * -\fBdns_submit_srv\fB(\fIctx\fR, const char *\fIname\fR, const char *\fIservice\fR, const char *\fIprotocol\fR, - int \fIflags\fR, dns_query_txt_fn *\fIcbck\fR, \fIdata\fR); -struct dns_rr_srv * -\fBdns_resolve_srv\fB(\fIctx\fR, const char *\fIname\fR, const char *\fIservice\fR, const char *\fIprotocol\fR, - int \fIflags\fR); -.fi -.PP -The \fBdns_rr_srv\fR structure holds a result of an IN SRV (rfc2782) query, -which is an array of servers (together with port numbers) which are performing -operations for a given \fIservice\fR using given \fIprotocol\fR on a target -domain \fIname\fR. Callback routine for IN SRV queries expected to be of type -\fBdns_query_srv_fn\fR, which expects pointer to \fBdns_rr_srv\fR structure as -query result instead of raw DNS packet. The \fBdns_parse_srv\fR() is used to -convert raw DNS reply packet into \fBdns_rr_srv\fR structure (it is used -internally and may be used directly too with generic query interface). -Routines \fBdns_submit_srv\fR() and \fBdns_resolve_srv\fR() are used to -perform IN SRV queries in a type-safe manner. The \fIname\fR parameter -is the domain name in question, \fIservice\fR and \fRprotocl\fR specifies the -service and the protocol in question (the library will construct query DN -according to rfc2782 rules) and may be NULL (in this case the library -assumes \fIname\fR parameter holds the complete SRV query), and -\fIflags\fR is query flags bitmask, with one bit, DNS_NOSRCH, of practical -interest (if the \fIname\fR is absolute, that is, it ends up with a dot, -DNS_NOSRCH flag will be set automatically). - -.SS "NAPTR Queries" -.PP -.nf -struct \fBdns_naptr\fR { /* single NAPTR record */ - int \fBorder\fR; /* record order */ - int \fBpreference\fR; /* preference of this record */ - char *\fBflags\fR; /* application-specific flags */ - char *\fBservice\fR; /* service parameter */ - char *\fBregexp\fR; /* substitutional regular expression */ - char *\fBreplacement\fR; /* replacement string */ -}; -struct \fBdns_rr_naptr\fR { /* NAPTR RRset */ - char *\fBdnsnaptr_qname\fR; /* original query name */ - char *\fBdnsnaptr_cname\fR; /* canonical name */ - unsigned \fBdnsnaptr_ttl\fR; /* Time-To-Live (TTL) value */ - int \fBdnsnaptr_nrr\fR; /* number of text records in the set */ - struct dns_naptr \fBdnsnaptr_naptr\fR[]; /* array of NAPTR records */ -}; -typedef void - \fBdns_query_naptr_fn\fR(\fIctx\fR, struct dns_rr_naptr *\fIresult\fR, \fIdata\fR) -dns_parse_fn \fBdns_parse_naptr\fB; -struct dns_query * -\fBdns_submit_naptr\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR, - dns_query_txt_fn *\fIcbck\fR, \fIdata\fR); -struct dns_rr_naptr * -\fBdns_resolve_naptr\fB(\fIctx\fR, const char *\fIname\fR, int \fIflags\fR); -.fi -.PP -The \fBdns_rr_naptr\fR structure holds a result of an IN NAPTR (rfc3403) query. -Callback routine for IN NAPTR queries expected to be of type -\fBdns_query_naptr_fn\fR, expects pointer to \fBdns_rr_naptr\fR -structure as query result instead of raw DNS packet. -The \fBdns_parse_naptr\fR() is used to convert raw DNS reply packet into -\fBdns_rr_naptr\fR structure (it is used -internally and may be used directly too with generic query interface). -Routines \fBdns_submit_naptr\fR() and \fBdns_resolve_naptr\fR() are used to -perform IN NAPTR queries in a type-safe manner. The \fIname\fR parameter -is the domain name in question, and \fIflags\fR is query flags bitmask, -with one bit, DNS_NOSRCH, of practical interest (if the \fIname\fR is -absolute, that is, it ends up with a dot, DNS_NOSRCH flag will be set -automatically). - -.SS "DNSBL Interface" -.PP -A DNS-based blocklists, or a DNSBLs, are in wide use nowadays, especially -to protect mailservers from spammers. The library provides DNSBL interface, -a set of routines to perform queries against DNSBLs. Routines accepts an -IP address (IPv4 and IPv6 are both supported) and a base DNSBL zone as -query parameters, and returns either \fBdns_rr_a4\fR or \fBdns_rr_txt\fR -structure. Note that IPv6 interface return IPv4 RRset. -.PP -.nf -struct dns_query * -\fBdns_submit_a4dnsbl\fR(\fIctx\fR, - const struct in_addr *\fIaddr\fR, const char *\fIdnsbl\fR, - dns_query_a4_fn *\fIcbck\fR, void *\fIdata\fR); -struct dns_query * -\fBdns_submit_a4dnsbl_txt\fR(\fIctx\fR, - const struct in_addr *\fIaddr\fR, const char *\fIdnsbl\fR, - dns_query_txt_fn *\fIcbck\fR, void *\fIdata\fR); -struct dns_query * -\fBdns_submit_a6dnsbl\fR(\fIctx\fR, - const struct in6_addr *\fIaddr\fR, const char *\fIdnsbl\fR, - dns_query_a4_fn *\fIcbck\fR, void *\fIdata\fR); -struct dns_query * -\fBdns_submit_a6dnsbl_txt\fR(\fIctx\fR, - const struct in6_addr *\fIaddr\fR, const char *\fIdnsbl\fR, - dns_query_txt_fn *\fIcbck\fR, void *\fIdata\fR); -struct dns_rr_a4 *\fBdns_resolve_a4dnsbl\fR(\fIctx\fR, - const struct in_addr *\fIaddr\fR, const char *\fIdnsbl\fR) -struct dns_rr_txt *\fBdns_resolve_a4dnsbl_txt\fR(\fIctx\fR, - const struct in_addr *\fIaddr\fR, const char *\fIdnsbl\fR) -struct dns_rr_a4 *\fBdns_resolve_a6dnsbl\fR(\fIctx\fR, - const struct in6_addr *\fIaddr\fR, const char *\fIdnsbl\fR) -struct dns_rr_txt *\fBdns_resolve_a6dnsbl_txt\fR(\fIctx\fR, - const struct in6_addr *\fIaddr\fR, const char *\fIdnsbl\fR) -.fi -Perform (submit or resolve) a DNSBL query for the given \fIdnsbl\fR -domain and an IP \fIaddr\fR in question, requesting either A or TXT -records. - -.SS "RHSBL Interface" -.PP -RHSBL is similar to DNSBL, but instead of an IP address, the -parameter is a domain name. -.PP -.nf -struct dns_query * -\fBdns_submit_rhsbl\fR(\fIctx\fR, const char *\fIname\fR, const char *\fIrhsbl\fR, - dns_query_a4_fn *\fIcbck\fR, void *\fIdata\fR); -struct dns_query * -\fBdns_submit_rhsbl_txt\fR(\fIctx\fR, const char *\fIname\fR, const char *\fIrhsbl\fR, - dns_query_txt_fn *\fIcbck\fR, void *\fIdata\fR); -struct dns_rr_a4 * -\fBdns_resolve_rhsbl\fR(\fIctx\fR, const char *\fIname\fR, const char *\fIrhsbl\fR); -struct dns_rr_txt * -\fBdns_resolve_rhsbl_txt\fR(\fIctx\fR, const char *\fIname\fR, const char *\fIrhsbl\fR); -.fi -Perform (submit or resolve) a RHSBL query for the given \fIrhsbl\fR -domain and \fIname\fR in question, requesting either A or TXT records. - - -.SH "LOW-LEVEL INTERFACE" - -.SS "Domain Names (DNs)" - -.PP -A DN is a series of domain name labels each starts with length byte, -followed by empty label (label with zero length). The following -routines to work with DNs are provided. - -.PP -.nf -unsigned \fBdns_dnlen\fR(const unsigned char *\fIdn\fR) -.fi -.RS -return length of the domain name \fIdn\fR, including the terminating label. -.RE - -.PP -.nf -unsigned \fBdns_dnlabels\fR(const unsigned char *\fIdn\fR) -.fi -.RS -return number of non-zero labels in domain name \fIdn\fR. -.RE - -.PP -.nf -unsigned \fBdns_dnequal\fR(\fIdn1\fR, \fIdn2\fR) - const unsigned char *\fIdn1\fR, *\fIdn2\fR; -.fi -.RS -test whenever the two domain names, \fIdn1\fR and \fIdn2\fR, are -equal (case-insensitive). Return domain name length if equal -or 0 if not. -.RE - -.PP -.nf -unsigned \fBdns_dntodn\fR(\fIsdn\fR, \fIddn\fR, \fIdnsiz\fR) - const unsigned char *\fIsdn\fR; - unsigned char *\fIddn\fR; - unsigned \fIdnsiz\fR; -.fi -.RS -copies the source domain name \fIsdn\fR to destination buffer \fIddn\fR -of size \fIdnsiz\fR. Return domain name length or 0 if \fIddn\fR is -too small. -.RE - -.PP -.nf -int \fBdns_ptodn\fR(\fIname\fR, \fInamelen\fR, \fIdn\fR, \fIdnsiz\fR, \fIisabs\fR) -int \fBdns_sptodn\fR(\fIname\fR, \fIdn\fR, \fIdnsiz\fR) - const char *\fIname\fR; unsigned \fInamelen\fR; - unsigned char *\fIdn\fR; unsigned \fIdnsiz\fR; - int *\fIisabs\fR; -.fi -.RS -convert asciiz name \fIname\fR of length \fInamelen\fR to DN format, -placing result into buffer \fIdn\fR of size \fIdnsiz\fR. Return -length of the DN if successeful, 0 if the \fIdn\fR buffer supplied is -too small, or negative value if \fIname\fR is invalid. If \fIisabs\fR -is non-NULL and conversion was successeful, *\fIisabs\fR will be set to -either 1 or 0 depending whenever \fIname\fR was absolute (i.e. ending with -a dot) or not. Name length, \fInamelength\fR, may be zero, in which case -strlen(\fIname\fR) will be used. Second form, \fBdns_sptodn\fR(), is a -simplified form of \fBdns_ptodn\fR(), equivalent to -.br -.nf -\fBdns_ptodn\fR(\fIname\fR, 0, \fIdn\fR, \fIdnlen\fR, 0). -.fi -.RE - -.PP -.nf -extern const unsigned char \fBdns_inaddr_arpa_dn\fR[] -int \fBdns_a4todn\fR(const struct in_addr *\fIaddr\fR, const unsigned char *\fItdn\fR, - unsigned char *\fIdn\fR, unsigned \fIdnsiz\fR) -int \fBdns_a4ptodn\fR(const struct in_addr *\fIaddr\fR, const char *\fItname\fR, - unsigned char *\fIdn\fR, unsigned \fIdnsiz\fR) -extern const unsigned char \fBdns_ip6_arpa_dn\fR[] -int \fBdns_a6todn\fR(const struct in6_addr *\fIaddr\fR, const unsigned char *\fItdn\fR, - unsigned char *\fIdn\fR, unsigned \fIdnsiz\fR) -int \fBdns_a6ptodn\fR(const struct in6_addr *\fIaddr\fR, const char *\fItname\fR, - unsigned char *\fIdn\fR, unsigned \fIdnsiz\fR) -.fi -.RS -several variants of routines to convert IPv4 and IPv6 address \fIaddr\fR -into reverseDNS-like domain name in DN format, storing result in \fIdn\fR -of size \fIdnsiz\fR. \fItdn\fR (or \fItname\fR) is the base zone name, -like in-addr.arpa for IPv4 or in6.arpa for IPv6. If \fItdn\fR (or \fItname\fR) -is NULL, \fBdns_inaddr_arpa_dn\fR (or \fBdns_ip6_arpa_dn\fR) will be used. -The routines may be used to construct a DN for a DNSBL lookup for example. -All routines return length of the resulting DN on success, -1 if resulting -DN is invalid, or 0 if the \fIdn\fR buffer (\fIdnsiz\fR) is too small. -To hold standard rDNS DN, a buffer of size \fBDNS_A4RSIZE\fR (30 bytes) for -IPv4 address, or \fBDNS_A6RSIZE\fR (74 bytes) for IPv6 address, is sufficient. -.RE - -.PP -.nf -int \fBdns_dntop\fR(\fIdn\fR, \fIname\fR, \fInamesiz\fR) - const unsigned char *\fIdn\fR; - const char *\fIname\fR; unsigned \fInamesiz\fR; -.fi -.RS -convert domain name \fIdn\fR in DN format to asciiz string, placing result -into \fIname\fR buffer of size \fInamesiz\fR. Maximum length of asciiz -representation of domain name is \fBDNS_MAXNAME\fR (1024) bytes. Root -domain is represented as empty string. Return length of the resulting name -(including terminating character, i.e. strlen(name)+1) on success, 0 if the -\fIname\fR buffer is too small, or negative value if \fIdn\fR is invalid -(last case should never happen since all routines in this library which -produce domain names ensure the DNs generated are valid). -.RE - -.PP -.nf -const char *\fBdns_dntosp\fR(const unsigned char *\fIdn\fR) -.fi -.RS -convert domain name \fIdn\fR in DN format to asciiz string using static -buffer. Return the resulting asciiz string on success or NULL on failure. -Note since this routine uses static buffer, it is not thread-safe. -.RE - -.PP -.nf -unsigned \fBdns_dntop_size\fR(const unsigned char *\fIdn\fR) -.fi -.RS -return the buffer size needed to convert the \fIdn\fR domain name -in DN format to asciiz string, for \fBdns_dntop\fR(). The routine -return either the size of buffer required, including the trailing -zero byte, or 0 if \fIdn\fR is invalid. -.RE - -.SS "Working with DNS Packets" - -.PP -The following routines are provided to encode and decode DNS on-wire -packets. This is low-level interface. - -.PP -DNS response codes (returned by \fBdns_rcode\fR() routine) are -defined as constants prefixed with \fBDNS_R_\fR. See udns.h -header file for the complete list. In particular, constants -\fBDNS_R_NOERROR\fR (0), \fBDNS_R_SERVFAIL\fR, \fBDNS_R_NXDOMAIN\fR -may be of interest to an application. - -.PP -.nf -unsigned \fBdns_get16\fR(const unsigned char *\fIp\fR) -unsigned \fBdns_get32\fR(const unsigned char *\fIp\fR) -.fi -.RS -helper routines, convert 16-bit or 32-bit integer in on-wire -format pointed to by \fIp\fR to unsigned. -.RE - -.PP -.nf -unsigned char *\fBdns_put16\fR(unsigned char *\fId\fR, unsigned \fIn\fR) -unsigned char *\fBdns_put32\fR(unsigned char *\fId\fR, unsigned \fIn\fR) -.fi -.RS -helper routine, convert unsigned 16-bit or 32-bit integer \fIn\fR to -on-wire format to buffer pointed to by \fId\fR, return \fId\fR+2 or -\fId\fR+4. -.RE - -.PP -.nf -\fBDNS_HSIZE\fR (12) -.fi -.RS -defines size of DNS header. Data section -in the DNS packet immediately follows the header. In the header, -there are query identifier (id), various flags and codes, -and number of resource records in various data sections. -See udns.h header file for complete list of DNS header definitions. -.RE - -.PP -.nf -unsigned \fBdns_qid\fR(const unsigned char *\fIpkt\fR) -int \fBdns_rd\fR(const unsigned char *\fIpkt\fR) -int \fBdns_tc\fR(const unsigned char *\fIpkt\fR) -int \fBdns_aa\fR(const unsigned char *\fIpkt\fR) -int \fBdns_qr\fR(const unsigned char *\fIpkt\fR) -int \fBdns_ra\fR(const unsigned char *\fIpkt\fR) -unsigned \fBdns_opcode\fR(const unsigned char *\fIpkt\fR) -unsigned \fBdns_rcode\fR(const unsigned char *\fIpkt\fR) -unsigned \fBdns_numqd\fR(const unsigned char *\fIpkt\fR) -unsigned \fBdns_numan\fR(const unsigned char *\fIpkt\fR) -unsigned \fBdns_numns\fR(const unsigned char *\fIpkt\fR) -unsigned \fBdns_numar\fR(const unsigned char *\fIpkt\fR) -const unsigned char *\fBdns_payload\fR(const unsigned char *\fIpkt\fR) -.fi -.RS -return various parts from the DNS packet header \fIpkt\fR: -query identifier (qid), -recursion desired (rd) flag, -truncation occured (tc) flag, -authoritative answer (aa) flag, -query response (qr) flag, -recursion available (ra) flag, -operation code (opcode), -result code (rcode), -number of entries in question section (numqd), -number of answers (numan), -number of authority records (numns), -number of additional records (numar), -and the pointer to the packet data (payload). -.RE - -.PP -.nf -int \fBdns_getdn\fR(\fIpkt\fR, \fIcurp\fR, \fIpkte\fR, \fIdn\fR, \fIdnsiz\fR) -const unsigned char *\fBdns_skipdn\fR(\fIcur\fR, \fIpkte\fR) - const unsigned char *\fIpkt\fR, *\fIpkte\fR, **\fIcurp\fR, *\fIcur\fR; - unsigned char *\fIdn\fR; unsigned \fIdnsiz\fR; -.fi -.RS -\fBdns_getdn\fR() extract DN from DNS packet \fIpkt\fR which ends before -\fIpkte\fR starting at position *\fIcurp\fR into buffer pointed to by -\fIdn\fR of size \fIdnsiz\fR. Upon successeful completion, *\fIcurp\fR -will point to the next byte in the packet after the extracted domain name. -It return positive number (length of the DN if \fIdn\fR) upon successeful -completion, negative value on error (when the packet contains invalid data), -or zero if the \fIdnsiz\fR is too small (maximum length of a domain name is -\fBDNS_MAXDN\fR). \fBdns_skipdn\fR() return pointer to the next byte in -DNS packet which ends up before \fIpkte\fR after a domain name which starts -at the \fIcur\fP byte, or NULL if the packet is invalid. \fBdns_skipdn\fR() -is more or less equivalent to what \fBdns_getdn\fR() does, except it does not -actually extract the domain name in question, and uses simpler interface. -.RE - -.PP -.nf -struct \fBdns_rr\fR { - unsigned char \fBdnsrr_dn\fR[DNS_MAXDN]; /* the RR DN name */ - enum dns_class \fBdnsrr_cls\fR; /* class of the RR */ - enum dns_type \fBdnsrr_typ\fR; /* type of the RR */ - unsigned \fBdnsrr_ttl\fR; /* TTL value */ - unsigned \fBdnsrr_dsz\fR; /* size of data in bytes */ - const unsigned char *\fBdnsrr_dptr\fR; /* pointer to the first data byte */ - const unsigned char *\fBdnsrr_dend\fR; /* next byte after RR */ -}; -.fi -.RS -The \fBdns_rr\fR structure is used to hold information about -single DNS Resource Record (RR) in an easy to use form. -.RE - -.PP -.nf -struct \fBdns_parse\fR { - const unsigned char *\fBdnsp_pkt\fR; /* pointer to the packet being parsed */ - const unsigned char *\fBdnsp_end\fR; /* end of the packet pointer */ - const unsigned char *\fBdnsp_cur\fR; /* current packet positionn */ - const unsigned char *\fBdnsp_ans\fR; /* pointer to the answer section */ - int \fBdnsp_rrl\fR; /* number of RRs left */ - int \fBdnsp_nrr\fR; /* number of relevant RRs seen so far */ - unsigned \fBdnsp_ttl\fR; /* TTL value so far */ - const unsigned char *\fBdnsp_qdn\fR; /* the domain of interest or NULL */ - enum dns_class \fBdnsp_qcls\fR; /* class of interest or 0 for any */ - enum dns_type \fBdnsp_qtyp\fR; /* type of interest or 0 for any */ - unsigned char \fBdnsp_dnbuf\fR[DNS_MAXDN]; /* domain name buffer */ -}; -.fi -.RS -The \fBdns_parse\fR structure is used to parse DNS reply packet. -It holds information about the packet being parsed (dnsp_pkt, dnsp_end and -dnsp_cur fields), number of RRs in the current section left to do, and -the information about specific RR which we're looking for (dnsp_qdn, -dnsp_qcls and dnsp_qtyp fields). -.RE - -.PP -.nf -int \fBdns_initparse\fR(struct dns_parse *\fIp\fR, - const unsigned char *\fIqdn\fR, - const unsigned char *\fIpkt\fR, - const unsigned char *\fIcur\fR, - const unsigned char *\fIend\fR) -.fi -.RS -initializes the RR parsing structure \fIp\fR. Arguments \fIpkt\fR, \fIcur\fR -and \fIend\fR should describe the received packet: \fIpkt\fR is the start of -the packet, \fIend\fR points to the next byte after the end of the packet, -and \fIcur\fR points past the query DN in query section (to query class+type -information). And \fIqdn\fR points to the query DN. This is the arguments -passed to \fBdns_parse_fn\fR() routine. \fBdns_initparse\fR() initializes -\fBdnsp_pkt\fR, \fBdnsp_end\fR and \fBdnsp_qdn\fR fields to the corresponding -arguments, extracts and initializes \fBdnsp_qcls\fR and \fBdnsp_qtyp\fR -fields to the values found at \fIcur\fR pointer, initializes -\fBdnsp_cur\fR and \fBdnsp_ans\fR fields to be \fIcur\fR+4 (to the start of -answer section), and initializes \fBdnsp_rrl\fR field to be number of entries -in answer section. \fBdnsp_ttl\fR will be set to max TTL value, 0xffffffff, -and \fBdnsp_nrr\fR to 0. -.RE - -.PP -.nf -int \fBdns_nextrr\fR(struct dns_parse *\fIp\fR, struct dns_rr *\fIrr\fR); -.fi -.RS -searches for next RR in the packet based on the criteria provided in -the \fIp\fR structure, filling in the \fIrr\fR structure and -advancing \fIp\fR->\fBdnsp_cur\fR to the next RR in the packet. -RR selection is based on dnsp_qdn, dnsp_qcls and dnsp_qtyp fields in -the dns_parse structure. Any (or all) of the 3 fields may be 0, -which means any actual value from the packet is acceptable. In case -the field isn't 0 (or NULL for dnsp_qdn), only RRs with corresponding -characteristics are acceptable. Additionally, when dnsp_qdn is non-NULL, -\fBdns_nextrr\fR() performs automatic CNAME expansion. -Routine will return positive value on success, 0 in case it reached the end -of current section in the packet (\fIp\fR->\fBdnsp_rrl\fR is zero), or -negative value if next RR can not be decoded (packet format is invalid). -The routine updates \fIp\fR->\fBdnsp_qdn\fR automatically when this -field is non-NULL and it encounters appropriate CNAME RRs (saving CNAME -target in \fIp\fR->\fBdnsp_dnbuf\fR), so after end of the process, -\fIp\fR->\fBdnsp_qdn\fR will point to canonical name of the domain -in question. The routine updates \fIp\fR->\fBdnsp_ttl\fR value to -be the minimum TTL of all RRs found. -.RE - -.PP -.nf -void \fBdns_rewind\fR(struct dns_parse *\fIp\fR, const unsigned char *\fIqdn\fR) -.fi -.RS -this routine "rewinds" the packet parse state structure to be at the -same state as after a call to \fBdns_initparse\fR(), i.e. reposition -the parse structure \fIp\fR to the start of answer section and -initialize \fIp\fR->\fBdnsp_rrl\fR to the number of entries in -answer section. -.RE - -.PP -.nf -int \fBdns_stdrr_size\fR(const struct dns_parse *\fIp\fR); -.fi -.RS -return size to hold standard RRset structure information, as shown -in \fBdns_rr_null\fR structure (for the query and canonical -names). Used to calculate amount of memory to allocate for common -part of type-specific RR structures in parsing routines. -.RE - -.PP -.nf -void *\fBdns_stdrr_finish\fR(struct dns_rr_null *\fIret\fR, char *\fIcp\fR, - const struct dns_parse *\fIp\fR); -.fi -.RS -initializes standard RRset fields in \fIret\fR structure using buffer -pointed to by \fIcp\fR, which should have at least as many bytes -as \fBdns_stdrr_size\fR(\fIp\fR) returned. Used to finalize common -part of type-specific RR structures in parsing routines. -.RE - -.PP -See library source for usage examples of all the above low-level routines, -especially source of the parsing routines. - -.SS "Auxilary Routines" - -.PP -.nf -int \fBdns_pton\fR(int \fIaf\fR, const char *\fIsrc\fR, void *\fIdst\fR); -.fi -.RS -privides functionality similar to standard \fBinet_pton\fR() routine, -to convert printable representation of an IP address of family \fIaf\fR -(either \fBAF_INET\fR or \fBAF_INET6\fR) pointed to by \fIsrc\fR into -binary form suitable for socket addresses and transmission over network, -in buffer pointed to by \fIdst\fR. The destination buffer should be -of size 4 for \fBAF_INET\fR family or 16 for \fBAF_INET6\fR. -The return value is positive on success, 0 if \fIsrc\fR is not a valid text -representation of an address of family \fIaf\fR, or negative if the -given address family is not supported. -.RE - -.PP -.nf -const char *\fBdns_ntop\fR(int \fIaf\fR, const void *\fIsrc\fR, - char *\fIdst\fR, int \fIdstsize\fR) -.fi -.RS -privides functionality similar to standard \fBinet_ntop\fR() routine, -to convert binary representation of an IP address of family \fIaf\fR -(either \fBAF_INET\fR or \fBAF_INET6\fR) pointed to by \fIsrc\fR -(either 4 or 16 bytes) into printable form in buffer in buffer pointed -to by \fIdst\fR of size \fIdstsize\fR. The destination buffer should be -at least of size 16 bytes for \fBAF_INET\fR family or 46 bytes for -\fBAF_INET6\fR. The return value is either \fIdst\fR, or NULL pointer -if \fIdstsize\fR is too small to hold this address or if the given -address family is not supported. -.RE - -.SH AUTHOR -.PP -The \fBudns\fR library has been written by Michael Tokarev, mjt+udns@tls.msk.ru. - -.SH VERSION -.PP -This manual page corresponds to udns version 0.4, released Jan-2014. diff --git a/contrib/udns/udns.h b/contrib/udns/udns.h deleted file mode 100644 index 371e6976404..00000000000 --- a/contrib/udns/udns.h +++ /dev/null @@ -1,778 +0,0 @@ -/* udns.h - header file for the UDNS library. - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#ifndef UDNS_VERSION /* include guard */ - -#define UDNS_VERSION "0.4" - -#ifdef WINDOWS -# ifdef UDNS_DYNAMIC_LIBRARY -# ifdef DNS_LIBRARY_BUILD -# define UDNS_API __declspec(dllexport) -# define UDNS_DATA_API __declspec(dllexport) -# else -# define UDNS_API __declspec(dllimport) -# define UDNS_DATA_API __declspec(dllimport) -# endif -# endif -#endif - -#ifndef UDNS_API -# define UDNS_API -#endif -#ifndef UDNS_DATA_API -# define UDNS_DATA_API -#endif - -#include /* for time_t */ - -#ifdef __cplusplus -extern "C" { -#endif - -/* forward declarations if sockets stuff isn't #include'd */ -struct in_addr; -struct in6_addr; -struct sockaddr; - -/**************************************************************************/ -/**************** Common definitions **************************************/ - -UDNS_API const char * -dns_version(void); - -struct dns_ctx; -struct dns_query; - -/* shorthand for [const] unsigned char */ -typedef unsigned char dnsc_t; -typedef const unsigned char dnscc_t; - -#define DNS_MAXDN 255 /* max DN length */ -#define DNS_DNPAD 1 /* padding for DN buffers */ -#define DNS_MAXLABEL 63 /* max DN label length */ -#define DNS_MAXNAME 1024 /* max asciiz domain name length */ -#define DNS_HSIZE 12 /* DNS packet header size */ -#define DNS_PORT 53 /* default domain port */ -#define DNS_MAXSERV 6 /* max servers to consult */ -#define DNS_MAXPACKET 512 /* max traditional-DNS UDP packet size */ -#define DNS_EDNS0PACKET 4096 /* EDNS0 packet size to use */ - -enum dns_class { /* DNS RR Classes */ - DNS_C_INVALID = 0, /* invalid class */ - DNS_C_IN = 1, /* Internet */ - DNS_C_CH = 3, /* CHAOS */ - DNS_C_HS = 4, /* HESIOD */ - DNS_C_ANY = 255 /* wildcard */ -}; - -enum dns_type { /* DNS RR Types */ - DNS_T_INVALID = 0, /* Cookie. */ - DNS_T_A = 1, /* Host address. */ - DNS_T_NS = 2, /* Authoritative server. */ - DNS_T_MD = 3, /* Mail destination. */ - DNS_T_MF = 4, /* Mail forwarder. */ - DNS_T_CNAME = 5, /* Canonical name. */ - DNS_T_SOA = 6, /* Start of authority zone. */ - DNS_T_MB = 7, /* Mailbox domain name. */ - DNS_T_MG = 8, /* Mail group member. */ - DNS_T_MR = 9, /* Mail rename name. */ - DNS_T_NULL = 10, /* Null resource record. */ - DNS_T_WKS = 11, /* Well known service. */ - DNS_T_PTR = 12, /* Domain name pointer. */ - DNS_T_HINFO = 13, /* Host information. */ - DNS_T_MINFO = 14, /* Mailbox information. */ - DNS_T_MX = 15, /* Mail routing information. */ - DNS_T_TXT = 16, /* Text strings. */ - DNS_T_RP = 17, /* Responsible person. */ - DNS_T_AFSDB = 18, /* AFS cell database. */ - DNS_T_X25 = 19, /* X_25 calling address. */ - DNS_T_ISDN = 20, /* ISDN calling address. */ - DNS_T_RT = 21, /* Router. */ - DNS_T_NSAP = 22, /* NSAP address. */ - DNS_T_NSAP_PTR = 23, /* Reverse NSAP lookup (deprecated). */ - DNS_T_SIG = 24, /* Security signature. */ - DNS_T_KEY = 25, /* Security key. */ - DNS_T_PX = 26, /* X.400 mail mapping. */ - DNS_T_GPOS = 27, /* Geographical position (withdrawn). */ - DNS_T_AAAA = 28, /* Ip6 Address. */ - DNS_T_LOC = 29, /* Location Information. */ - DNS_T_NXT = 30, /* Next domain (security). */ - DNS_T_EID = 31, /* Endpoint identifier. */ - DNS_T_NIMLOC = 32, /* Nimrod Locator. */ - DNS_T_SRV = 33, /* Server Selection. */ - DNS_T_ATMA = 34, /* ATM Address */ - DNS_T_NAPTR = 35, /* Naming Authority PoinTeR */ - DNS_T_KX = 36, /* Key Exchange */ - DNS_T_CERT = 37, /* Certification record */ - DNS_T_A6 = 38, /* IPv6 address (deprecates AAAA) */ - DNS_T_DNAME = 39, /* Non-terminal DNAME (for IPv6) */ - DNS_T_SINK = 40, /* Kitchen sink (experimentatl) */ - DNS_T_OPT = 41, /* EDNS0 option (meta-RR) */ - DNS_T_DS = 43, /* DNSSEC */ - DNS_T_SSHFP = 44, - DNS_T_IPSECKEY = 45, - DNS_T_RRSIG = 46, /* DNSSEC */ - DNS_T_NSEC = 47, /* DNSSEC */ - DNS_T_DNSKEY = 48, - DNS_T_DHCID = 49, - DNS_T_NSEC3 = 50, - DNS_T_NSEC3PARAMS = 51, - DNS_T_TALINK = 58, /* draft-ietf-dnsop-trust-history */ - DNS_T_SPF = 99, - DNS_T_UINFO = 100, - DNS_T_UID = 101, - DNS_T_GID = 102, - DNS_T_UNSPEC = 103, - DNS_T_TSIG = 250, /* Transaction signature. */ - DNS_T_IXFR = 251, /* Incremental zone transfer. */ - DNS_T_AXFR = 252, /* Transfer zone of authority. */ - DNS_T_MAILB = 253, /* Transfer mailbox records. */ - DNS_T_MAILA = 254, /* Transfer mail agent records. */ - DNS_T_ANY = 255, /* Wildcard match. */ - DNS_T_ZXFR = 256, /* BIND-specific, nonstandard. */ - DNS_T_DLV = 32769, /* RFC 4431, 5074, DNSSEC Lookaside Validation */ - DNS_T_MAX = 65536 -}; - -/**************************************************************************/ -/**************** Domain Names (DNs) **************************************/ - -/* return length of the DN */ -UDNS_API unsigned -dns_dnlen(dnscc_t *dn); - -/* return #of labels in a DN */ -UDNS_API unsigned -dns_dnlabels(dnscc_t *dn); - -/* lower- and uppercase single DN char */ -#define DNS_DNLC(c) ((c) >= 'A' && (c) <= 'Z' ? (c) - 'A' + 'a' : (c)) -#define DNS_DNUC(c) ((c) >= 'a' && (c) <= 'z' ? (c) - 'a' + 'A' : (c)) - -/* compare the DNs, return dnlen of equal or 0 if not */ -UDNS_API unsigned -dns_dnequal(dnscc_t *dn1, dnscc_t *dn2); - -/* copy one DN to another, size checking */ -UDNS_API unsigned -dns_dntodn(dnscc_t *sdn, dnsc_t *ddn, unsigned ddnsiz); - -/* convert asciiz string of length namelen (0 to use strlen) to DN */ -UDNS_API int -dns_ptodn(const char *name, unsigned namelen, - dnsc_t *dn, unsigned dnsiz, int *isabs); - -/* simpler form of dns_ptodn() */ -#define dns_sptodn(name,dn,dnsiz) dns_ptodn((name),0,(dn),(dnsiz),0) - -UDNS_DATA_API extern dnscc_t dns_inaddr_arpa_dn[14]; -#define DNS_A4RSIZE 30 -UDNS_API int -dns_a4todn(const struct in_addr *addr, dnscc_t *tdn, - dnsc_t *dn, unsigned dnsiz); -UDNS_API int -dns_a4ptodn(const struct in_addr *addr, const char *tname, - dnsc_t *dn, unsigned dnsiz); -UDNS_API dnsc_t * -dns_a4todn_(const struct in_addr *addr, dnsc_t *dn, dnsc_t *dne); - -UDNS_DATA_API extern dnscc_t dns_ip6_arpa_dn[10]; -#define DNS_A6RSIZE 74 -UDNS_API int -dns_a6todn(const struct in6_addr *addr, dnscc_t *tdn, - dnsc_t *dn, unsigned dnsiz); -UDNS_API int -dns_a6ptodn(const struct in6_addr *addr, const char *tname, - dnsc_t *dn, unsigned dnsiz); -UDNS_API dnsc_t * -dns_a6todn_(const struct in6_addr *addr, dnsc_t *dn, dnsc_t *dne); - -/* convert DN into asciiz string */ -UDNS_API int -dns_dntop(dnscc_t *dn, char *name, unsigned namesiz); - -/* convert DN into asciiz string, using static buffer (NOT thread-safe!) */ -UDNS_API const char * -dns_dntosp(dnscc_t *dn); - -/* return buffer size (incl. null byte) required for asciiz form of a DN */ -UDNS_API unsigned -dns_dntop_size(dnscc_t *dn); - -/* either wrappers or reimplementations for inet_ntop() and inet_pton() */ -UDNS_API const char *dns_ntop(int af, const void *src, char *dst, int size); -UDNS_API int dns_pton(int af, const char *src, void *dst); - -/**************************************************************************/ -/**************** DNS raw packet layout ***********************************/ - -enum dns_rcode { /* reply codes */ - DNS_R_NOERROR = 0, /* ok, no error */ - DNS_R_FORMERR = 1, /* format error */ - DNS_R_SERVFAIL = 2, /* server failed */ - DNS_R_NXDOMAIN = 3, /* domain does not exists */ - DNS_R_NOTIMPL = 4, /* not implemented */ - DNS_R_REFUSED = 5, /* query refused */ - /* these are for BIND_UPDATE */ - DNS_R_YXDOMAIN = 6, /* Name exists */ - DNS_R_YXRRSET = 7, /* RRset exists */ - DNS_R_NXRRSET = 8, /* RRset does not exist */ - DNS_R_NOTAUTH = 9, /* Not authoritative for zone */ - DNS_R_NOTZONE = 10, /* Zone of record different from zone section */ - /*ns_r_max = 11,*/ - /* The following are TSIG extended errors */ - DNS_R_BADSIG = 16, - DNS_R_BADKEY = 17, - DNS_R_BADTIME = 18 -}; - -static __inline unsigned dns_get16(dnscc_t *s) { - return ((unsigned)s[0]<<8) | s[1]; -} -static __inline unsigned dns_get32(dnscc_t *s) { - return ((unsigned)s[0]<<24) | ((unsigned)s[1]<<16) - | ((unsigned)s[2]<<8) | s[3]; -} -static __inline dnsc_t *dns_put16(dnsc_t *d, unsigned n) { - *d++ = (dnsc_t)((n >> 8) & 255); *d++ = (dnsc_t)(n & 255); return d; -} -static __inline dnsc_t *dns_put32(dnsc_t *d, unsigned n) { - *d++ = (dnsc_t)((n >> 24) & 255); *d++ = (dnsc_t)((n >> 16) & 255); - *d++ = (dnsc_t)((n >> 8) & 255); *d++ = (dnsc_t)(n & 255); - return d; -} - -/* DNS Header layout */ -enum { - /* bytes 0:1 - query ID */ - DNS_H_QID1 = 0, - DNS_H_QID2 = 1, - DNS_H_QID = DNS_H_QID1, -#define dns_qid(pkt) dns_get16((pkt)+DNS_H_QID) - /* byte 2: flags1 */ - DNS_H_F1 = 2, - DNS_HF1_QR = 0x80, /* query response flag */ -#define dns_qr(pkt) ((pkt)[DNS_H_F1]&DNS_HF1_QR) - DNS_HF1_OPCODE = 0x78, /* opcode, 0 = query */ -#define dns_opcode(pkt) (((pkt)[DNS_H_F1]&DNS_HF1_OPCODE)>>3) - DNS_HF1_AA = 0x04, /* auth answer */ -#define dns_aa(pkt) ((pkt)[DNS_H_F1]&DNS_HF1_AA) - DNS_HF1_TC = 0x02, /* truncation flag */ -#define dns_tc(pkt) ((pkt)[DNS_H_F1]&DNS_HF1_TC) - DNS_HF1_RD = 0x01, /* recursion desired (may be set in query) */ -#define dns_rd(pkt) ((pkt)[DNS_H_F1]&DNS_HF1_RD) - /* byte 3: flags2 */ - DNS_H_F2 = 3, - DNS_HF2_RA = 0x80, /* recursion available */ -#define dns_ra(pkt) ((pkt)[DNS_H_F2]&DNS_HF2_RA) - DNS_HF2_Z = 0x40, /* reserved */ - DNS_HF2_AD = 0x20, /* DNSSEC: authentic data */ -#define dns_ad(pkt) ((pkt)[DNS_H_F2]&DNS_HF2_AD) - DNS_HF2_CD = 0x10, /* DNSSEC: checking disabled */ -#define dns_cd(pkt) ((pkt)[DNS_H_F2]&DNS_HF2_CD) - DNS_HF2_RCODE = 0x0f, /* response code, DNS_R_XXX above */ -#define dns_rcode(pkt) ((pkt)[DNS_H_F2]&DNS_HF2_RCODE) - /* bytes 4:5: qdcount, numqueries */ - DNS_H_QDCNT1 = 4, - DNS_H_QDCNT2 = 5, - DNS_H_QDCNT = DNS_H_QDCNT1, -#define dns_numqd(pkt) dns_get16((pkt)+4) - /* bytes 6:7: ancount, numanswers */ - DNS_H_ANCNT1 = 6, - DNS_H_ANCNT2 = 7, - DNS_H_ANCNT = DNS_H_ANCNT1, -#define dns_numan(pkt) dns_get16((pkt)+6) - /* bytes 8:9: nscount, numauthority */ - DNS_H_NSCNT1 = 8, - DNS_H_NSCNT2 = 9, - DNS_H_NSCNT = DNS_H_NSCNT1, -#define dns_numns(pkt) dns_get16((pkt)+8) - /* bytes 10:11: arcount, numadditional */ - DNS_H_ARCNT1 = 10, - DNS_H_ARCNT2 = 11, - DNS_H_ARCNT = DNS_H_ARCNT1, -#define dns_numar(pkt) dns_get16((pkt)+10) -#define dns_payload(pkt) ((pkt)+DNS_HSIZE) - /* EDNS0 (OPT RR) flags (Ext. Flags) */ - DNS_EF1_DO = 0x80, /* DNSSEC OK */ -}; - -/* packet buffer: start at pkt, end before pkte, current pos *curp. - * extract a DN and set *curp to the next byte after DN in packet. - * return -1 on error, 0 if dnsiz is too small, or dnlen on ok. - */ -UDNS_API int -dns_getdn(dnscc_t *pkt, dnscc_t **curp, dnscc_t *end, - dnsc_t *dn, unsigned dnsiz); - -/* skip the DN at position cur in packet ending before pkte, - * return pointer to the next byte after the DN or NULL on error */ -UDNS_API dnscc_t * -dns_skipdn(dnscc_t *end, dnscc_t *cur); - -struct dns_rr { /* DNS Resource Record */ - dnsc_t dnsrr_dn[DNS_MAXDN]; /* the DN of the RR */ - enum dns_class dnsrr_cls; /* Class */ - enum dns_type dnsrr_typ; /* Type */ - unsigned dnsrr_ttl; /* Time-To-Live (TTL) */ - unsigned dnsrr_dsz; /* data size */ - dnscc_t *dnsrr_dptr; /* pointer to start of data */ - dnscc_t *dnsrr_dend; /* past end of data */ -}; - -struct dns_parse { /* RR/packet parsing state */ - dnscc_t *dnsp_pkt; /* start of the packet */ - dnscc_t *dnsp_end; /* end of the packet */ - dnscc_t *dnsp_cur; /* current packet position */ - dnscc_t *dnsp_ans; /* start of answer section */ - int dnsp_rrl; /* number of RRs left to go */ - int dnsp_nrr; /* RR count so far */ - unsigned dnsp_ttl; /* TTL value so far */ - dnscc_t *dnsp_qdn; /* the RR DN we're looking for */ - enum dns_class dnsp_qcls; /* RR class we're looking for or 0 */ - enum dns_type dnsp_qtyp; /* RR type we're looking for or 0 */ - dnsc_t dnsp_dnbuf[DNS_MAXDN]; /* domain buffer */ -}; - -/* initialize the parse structure */ -UDNS_API void -dns_initparse(struct dns_parse *p, dnscc_t *qdn, - dnscc_t *pkt, dnscc_t *cur, dnscc_t *end); - -/* search next RR, <0=error, 0=no more RRs, >0 = found. */ -UDNS_API int -dns_nextrr(struct dns_parse *p, struct dns_rr *rr); - -UDNS_API void -dns_rewind(struct dns_parse *p, dnscc_t *qdn); - - -/**************************************************************************/ -/**************** Resolver Context ****************************************/ - -/* default resolver context */ -UDNS_DATA_API extern struct dns_ctx dns_defctx; - -/* reset resolver context to default state, close it if open, drop queries */ -UDNS_API void -dns_reset(struct dns_ctx *ctx); - -/* reset resolver context and read in system configuration */ -UDNS_API int -dns_init(struct dns_ctx *ctx, int do_open); - -/* return new resolver context with the same settings as copy */ -UDNS_API struct dns_ctx * -dns_new(const struct dns_ctx *copy); - -/* free resolver context returned by dns_new(); all queries are dropped */ -UDNS_API void -dns_free(struct dns_ctx *ctx); - -/* add nameserver for a resolver context (or reset nslist if serv==NULL) */ -UDNS_API int -dns_add_serv(struct dns_ctx *ctx, const char *serv); - -/* add nameserver using struct sockaddr structure (with ports) */ -UDNS_API int -dns_add_serv_s(struct dns_ctx *ctx, const struct sockaddr *sa); - -/* add search list element for a resolver context (or reset it if srch==NULL) */ -UDNS_API int -dns_add_srch(struct dns_ctx *ctx, const char *srch); - -/* set options for a resolver context */ -UDNS_API int -dns_set_opts(struct dns_ctx *ctx, const char *opts); - -enum dns_opt { /* options */ - DNS_OPT_FLAGS, /* flags, DNS_F_XXX */ - DNS_OPT_TIMEOUT, /* timeout in secounds */ - DNS_OPT_NTRIES, /* number of retries */ - DNS_OPT_NDOTS, /* ndots */ - DNS_OPT_UDPSIZE, /* EDNS0 UDP size */ - DNS_OPT_PORT, /* port to use */ -}; - -/* set or get (if val<0) an option */ -UDNS_API int -dns_set_opt(struct dns_ctx *ctx, enum dns_opt opt, int val); - -enum dns_flags { - DNS_NOSRCH = 0x00010000, /* do not perform search */ - DNS_NORD = 0x00020000, /* request no recursion */ - DNS_AAONLY = 0x00040000, /* set AA flag in queries */ - DNS_SET_DO = 0x00080000, /* set EDNS0 "DO" bit (DNSSEC OK) */ - DNS_SET_CD = 0x00100000, /* set CD bit (DNSSEC: checking disabled) */ -}; - -/* set the debug function pointer */ -typedef void -(dns_dbgfn)(int code, const struct sockaddr *sa, unsigned salen, - dnscc_t *pkt, int plen, - const struct dns_query *q, void *data); -UDNS_API void -dns_set_dbgfn(struct dns_ctx *ctx, dns_dbgfn *dbgfn); - -/* open and return UDP socket */ -UDNS_API int -dns_open(struct dns_ctx *ctx); - -/* return UDP socket or -1 if not open */ -UDNS_API int -dns_sock(const struct dns_ctx *ctx); - -/* close the UDP socket */ -UDNS_API void -dns_close(struct dns_ctx *ctx); - -/* return number of requests queued */ -UDNS_API int -dns_active(const struct dns_ctx *ctx); - -/* return status of the last operation */ -UDNS_API int -dns_status(const struct dns_ctx *ctx); -UDNS_API void -dns_setstatus(struct dns_ctx *ctx, int status); - -/* handle I/O event on UDP socket */ -UDNS_API void -dns_ioevent(struct dns_ctx *ctx, time_t now); - -/* process any timeouts, return time in secounds to the - * next timeout (or -1 if none) but not greather than maxwait */ -UDNS_API int -dns_timeouts(struct dns_ctx *ctx, int maxwait, time_t now); - -/* define timer requesting routine to use */ -typedef void dns_utm_fn(struct dns_ctx *ctx, int timeout, void *data); -UDNS_API void -dns_set_tmcbck(struct dns_ctx *ctx, dns_utm_fn *fn, void *data); - -/**************************************************************************/ -/**************** Making Queries ******************************************/ - -/* query callback routine */ -typedef void dns_query_fn(struct dns_ctx *ctx, void *result, void *data); - -/* query parse routine: raw DNS => application structure */ -typedef int -dns_parse_fn(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, - void **res); - -enum dns_status { - DNS_E_NOERROR = 0, /* ok, not an error */ - DNS_E_TEMPFAIL = -1, /* timeout, SERVFAIL or similar */ - DNS_E_PROTOCOL = -2, /* got garbled reply */ - DNS_E_NXDOMAIN = -3, /* domain does not exists */ - DNS_E_NODATA = -4, /* domain exists but no data of reqd type */ - DNS_E_NOMEM = -5, /* out of memory while processing */ - DNS_E_BADQUERY = -6 /* the query is malformed */ -}; - -/* submit generic DN query */ -UDNS_API struct dns_query * -dns_submit_dn(struct dns_ctx *ctx, - dnscc_t *dn, int qcls, int qtyp, int flags, - dns_parse_fn *parse, dns_query_fn *cbck, void *data); -/* submit generic name query */ -UDNS_API struct dns_query * -dns_submit_p(struct dns_ctx *ctx, - const char *name, int qcls, int qtyp, int flags, - dns_parse_fn *parse, dns_query_fn *cbck, void *data); - -/* cancel the given async query in progress */ -UDNS_API int -dns_cancel(struct dns_ctx *ctx, struct dns_query *q); - -/* resolve a generic query, return the answer */ -UDNS_API void * -dns_resolve_dn(struct dns_ctx *ctx, - dnscc_t *qdn, int qcls, int qtyp, int flags, - dns_parse_fn *parse); -UDNS_API void * -dns_resolve_p(struct dns_ctx *ctx, - const char *qname, int qcls, int qtyp, int flags, - dns_parse_fn *parse); -UDNS_API void * -dns_resolve(struct dns_ctx *ctx, struct dns_query *q); - - -/* Specific RR handlers */ - -#define dns_rr_common(prefix) \ - char *prefix##_cname; /* canonical name */ \ - char *prefix##_qname; /* original query name */ \ - unsigned prefix##_ttl; /* TTL value */ \ - int prefix##_nrr /* number of records */ - -struct dns_rr_null { /* NULL RRset, aka RRset template */ - dns_rr_common(dnsn); -}; - -UDNS_API int -dns_stdrr_size(const struct dns_parse *p); -UDNS_API void * -dns_stdrr_finish(struct dns_rr_null *ret, char *cp, const struct dns_parse *p); - -struct dns_rr_a4 { /* the A RRset */ - dns_rr_common(dnsa4); - struct in_addr *dnsa4_addr; /* array of addresses, naddr elements */ -}; - -UDNS_API dns_parse_fn dns_parse_a4; /* A RR parsing routine */ -typedef void /* A query callback routine */ -dns_query_a4_fn(struct dns_ctx *ctx, struct dns_rr_a4 *result, void *data); - -/* submit A IN query */ -UDNS_API struct dns_query * -dns_submit_a4(struct dns_ctx *ctx, const char *name, int flags, - dns_query_a4_fn *cbck, void *data); - -/* resolve A IN query */ -UDNS_API struct dns_rr_a4 * -dns_resolve_a4(struct dns_ctx *ctx, const char *name, int flags); - - -struct dns_rr_a6 { /* the AAAA RRset */ - dns_rr_common(dnsa6); - struct in6_addr *dnsa6_addr; /* array of addresses, naddr elements */ -}; - -UDNS_API dns_parse_fn dns_parse_a6; /* A RR parsing routine */ -typedef void /* A query callback routine */ -dns_query_a6_fn(struct dns_ctx *ctx, struct dns_rr_a6 *result, void *data); - -/* submit AAAA IN query */ -UDNS_API struct dns_query * -dns_submit_a6(struct dns_ctx *ctx, const char *name, int flags, - dns_query_a6_fn *cbck, void *data); - -/* resolve AAAA IN query */ -UDNS_API struct dns_rr_a6 * -dns_resolve_a6(struct dns_ctx *ctx, const char *name, int flags); - - -struct dns_rr_ptr { /* the PTR RRset */ - dns_rr_common(dnsptr); - char **dnsptr_ptr; /* array of PTRs */ -}; - -UDNS_API dns_parse_fn dns_parse_ptr; /* PTR RR parsing routine */ -typedef void /* PTR query callback */ -dns_query_ptr_fn(struct dns_ctx *ctx, struct dns_rr_ptr *result, void *data); -/* submit PTR IN in-addr.arpa query */ -UDNS_API struct dns_query * -dns_submit_a4ptr(struct dns_ctx *ctx, const struct in_addr *addr, - dns_query_ptr_fn *cbck, void *data); -/* resolve PTR IN in-addr.arpa query */ -UDNS_API struct dns_rr_ptr * -dns_resolve_a4ptr(struct dns_ctx *ctx, const struct in_addr *addr); - -/* the same as above, but for ip6.arpa */ -UDNS_API struct dns_query * -dns_submit_a6ptr(struct dns_ctx *ctx, const struct in6_addr *addr, - dns_query_ptr_fn *cbck, void *data); -UDNS_API struct dns_rr_ptr * -dns_resolve_a6ptr(struct dns_ctx *ctx, const struct in6_addr *addr); - - -struct dns_mx { /* single MX RR */ - int priority; /* MX priority */ - char *name; /* MX name */ -}; -struct dns_rr_mx { /* the MX RRset */ - dns_rr_common(dnsmx); - struct dns_mx *dnsmx_mx; /* array of MXes */ -}; -UDNS_API dns_parse_fn dns_parse_mx; /* MX RR parsing routine */ -typedef void /* MX RR callback */ -dns_query_mx_fn(struct dns_ctx *ctx, struct dns_rr_mx *result, void *data); -/* submit MX IN query */ -UDNS_API struct dns_query * -dns_submit_mx(struct dns_ctx *ctx, const char *name, int flags, - dns_query_mx_fn *cbck, void *data); -/* resolve MX IN query */ -UDNS_API struct dns_rr_mx * -dns_resolve_mx(struct dns_ctx *ctx, const char *name, int flags); - - -struct dns_txt { /* single TXT record */ - int len; /* length of the text */ - dnsc_t *txt; /* pointer to text buffer. May contain nulls. */ -}; -struct dns_rr_txt { /* the TXT RRset */ - dns_rr_common(dnstxt); - struct dns_txt *dnstxt_txt; /* array of TXT records */ -}; -UDNS_API dns_parse_fn dns_parse_txt; /* TXT RR parsing routine */ -typedef void /* TXT RR callback */ -dns_query_txt_fn(struct dns_ctx *ctx, struct dns_rr_txt *result, void *data); -/* submit TXT query */ -UDNS_API struct dns_query * -dns_submit_txt(struct dns_ctx *ctx, const char *name, int qcls, int flags, - dns_query_txt_fn *cbck, void *data); -/* resolve TXT query */ -UDNS_API struct dns_rr_txt * -dns_resolve_txt(struct dns_ctx *ctx, const char *name, int qcls, int flags); - - -struct dns_srv { /* single SRV RR */ - int priority; /* SRV priority */ - int weight; /* SRV weight */ - int port; /* SRV port */ - char *name; /* SRV name */ -}; -struct dns_rr_srv { /* the SRV RRset */ - dns_rr_common(dnssrv); - struct dns_srv *dnssrv_srv; /* array of SRVes */ -}; -UDNS_API dns_parse_fn dns_parse_srv; /* SRV RR parsing routine */ -typedef void /* SRV RR callback */ -dns_query_srv_fn(struct dns_ctx *ctx, struct dns_rr_srv *result, void *data); -/* submit SRV IN query */ -UDNS_API struct dns_query * -dns_submit_srv(struct dns_ctx *ctx, - const char *name, const char *srv, const char *proto, - int flags, dns_query_srv_fn *cbck, void *data); -/* resolve SRV IN query */ -UDNS_API struct dns_rr_srv * -dns_resolve_srv(struct dns_ctx *ctx, - const char *name, const char *srv, const char *proto, - int flags); - -/* NAPTR (RFC3403) RR type */ -struct dns_naptr { /* single NAPTR RR */ - int order; /* NAPTR order */ - int preference; /* NAPTR preference */ - char *flags; /* NAPTR flags */ - char *service; /* NAPTR service */ - char *regexp; /* NAPTR regexp */ - char *replacement; /* NAPTR replacement */ -}; - -struct dns_rr_naptr { /* the NAPTR RRset */ - dns_rr_common(dnsnaptr); - struct dns_naptr *dnsnaptr_naptr; /* array of NAPTRes */ -}; -UDNS_API dns_parse_fn dns_parse_naptr; /* NAPTR RR parsing routine */ -typedef void /* NAPTR RR callback */ -dns_query_naptr_fn(struct dns_ctx *ctx, - struct dns_rr_naptr *result, void *data); -/* submit NAPTR IN query */ -UDNS_API struct dns_query * -dns_submit_naptr(struct dns_ctx *ctx, const char *name, int flags, - dns_query_naptr_fn *cbck, void *data); -/* resolve NAPTR IN query */ -UDNS_API struct dns_rr_naptr * -dns_resolve_naptr(struct dns_ctx *ctx, const char *name, int flags); - - -UDNS_API struct dns_query * -dns_submit_a4dnsbl(struct dns_ctx *ctx, - const struct in_addr *addr, const char *dnsbl, - dns_query_a4_fn *cbck, void *data); -UDNS_API struct dns_query * -dns_submit_a4dnsbl_txt(struct dns_ctx *ctx, - const struct in_addr *addr, const char *dnsbl, - dns_query_txt_fn *cbck, void *data); -UDNS_API struct dns_rr_a4 * -dns_resolve_a4dnsbl(struct dns_ctx *ctx, - const struct in_addr *addr, const char *dnsbl); -UDNS_API struct dns_rr_txt * -dns_resolve_a4dnsbl_txt(struct dns_ctx *ctx, - const struct in_addr *addr, const char *dnsbl); - -UDNS_API struct dns_query * -dns_submit_a6dnsbl(struct dns_ctx *ctx, - const struct in6_addr *addr, const char *dnsbl, - dns_query_a4_fn *cbck, void *data); -UDNS_API struct dns_query * -dns_submit_a6dnsbl_txt(struct dns_ctx *ctx, - const struct in6_addr *addr, const char *dnsbl, - dns_query_txt_fn *cbck, void *data); -UDNS_API struct dns_rr_a4 * -dns_resolve_a6dnsbl(struct dns_ctx *ctx, - const struct in6_addr *addr, const char *dnsbl); -UDNS_API struct dns_rr_txt * -dns_resolve_a6dnsbl_txt(struct dns_ctx *ctx, - const struct in6_addr *addr, const char *dnsbl); - -UDNS_API struct dns_query * -dns_submit_rhsbl(struct dns_ctx *ctx, - const char *name, const char *rhsbl, - dns_query_a4_fn *cbck, void *data); -UDNS_API struct dns_query * -dns_submit_rhsbl_txt(struct dns_ctx *ctx, - const char *name, const char *rhsbl, - dns_query_txt_fn *cbck, void *data); -UDNS_API struct dns_rr_a4 * -dns_resolve_rhsbl(struct dns_ctx *ctx, const char *name, const char *rhsbl); -UDNS_API struct dns_rr_txt * -dns_resolve_rhsbl_txt(struct dns_ctx *ctx, const char *name, const char *rhsbl); - -/**************************************************************************/ -/**************** Names, Names ********************************************/ - -struct dns_nameval { - int val; - const char *name; -}; - -UDNS_DATA_API extern const struct dns_nameval dns_classtab[]; -UDNS_DATA_API extern const struct dns_nameval dns_typetab[]; -UDNS_DATA_API extern const struct dns_nameval dns_rcodetab[]; -UDNS_API int -dns_findname(const struct dns_nameval *nv, const char *name); -#define dns_findclassname(cls) dns_findname(dns_classtab, (cls)) -#define dns_findtypename(type) dns_findname(dns_typetab, (type)) -#define dns_findrcodename(rcode) dns_findname(dns_rcodetab, (rcode)) - -UDNS_API const char *dns_classname(enum dns_class cls); -UDNS_API const char *dns_typename(enum dns_type type); -UDNS_API const char *dns_rcodename(enum dns_rcode rcode); -const char *_dns_format_code(char *buf, const char *prefix, int code); - -UDNS_API const char *dns_strerror(int errnum); - -/* simple pseudo-random number generator, code by Bob Jenkins */ - -struct udns_jranctx { /* the context */ - unsigned a, b, c, d; -}; - -/* initialize the RNG with a given seed */ -UDNS_API void -udns_jraninit(struct udns_jranctx *x, unsigned seed); - -/* return next random number. 32bits on most platforms so far. */ -UDNS_API unsigned -udns_jranval(struct udns_jranctx *x); - -#ifdef __cplusplus -} /* extern "C" */ -#endif - -#endif /* include guard */ diff --git a/contrib/udns/udns_XtoX.c b/contrib/udns/udns_XtoX.c deleted file mode 100644 index 60e3fdfa320..00000000000 --- a/contrib/udns/udns_XtoX.c +++ /dev/null @@ -1,50 +0,0 @@ -/* udns_XtoX.c - udns_ntop() and udns_pton() routines, which are either - - wrappers for inet_ntop() and inet_pton() or - - reimplementations of those routines. - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#ifdef HAVE_CONFIG_H -# include "config.h" -#endif -#include "udns.h" - -#ifdef HAVE_INET_PTON_NTOP - -#include -#include -#include - -const char *dns_ntop(int af, const void *src, char *dst, int size) { - return inet_ntop(af, src, dst, size); -} - -int dns_pton(int af, const char *src, void *dst) { - return inet_pton(af, src, dst); -} - -#else - -#define inet_XtoX_prefix udns_ -#include "inet_XtoX.c" - -#endif diff --git a/contrib/udns/udns_bl.c b/contrib/udns/udns_bl.c deleted file mode 100644 index f6be39335f1..00000000000 --- a/contrib/udns/udns_bl.c +++ /dev/null @@ -1,160 +0,0 @@ -/* udns_bl.c - DNSBL stuff - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#include "udns.h" -#ifndef NULL -# define NULL 0 -#endif - -struct dns_query * -dns_submit_a4dnsbl(struct dns_ctx *ctx, - const struct in_addr *addr, const char *dnsbl, - dns_query_a4_fn *cbck, void *data) { - dnsc_t dn[DNS_MAXDN]; - if (dns_a4ptodn(addr, dnsbl, dn, sizeof(dn)) <= 0) { - dns_setstatus(ctx, DNS_E_BADQUERY); - return NULL; - } - return - dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_A, DNS_NOSRCH, - dns_parse_a4, (dns_query_fn*)cbck, data); -} - -struct dns_query * -dns_submit_a4dnsbl_txt(struct dns_ctx *ctx, - const struct in_addr *addr, const char *dnsbl, - dns_query_txt_fn *cbck, void *data) { - dnsc_t dn[DNS_MAXDN]; - if (dns_a4ptodn(addr, dnsbl, dn, sizeof(dn)) <= 0) { - dns_setstatus(ctx, DNS_E_BADQUERY); - return NULL; - } - return - dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_TXT, DNS_NOSRCH, - dns_parse_txt, (dns_query_fn*)cbck, data); -} - -struct dns_rr_a4 * -dns_resolve_a4dnsbl(struct dns_ctx *ctx, - const struct in_addr *addr, const char *dnsbl) { - return (struct dns_rr_a4 *) - dns_resolve(ctx, dns_submit_a4dnsbl(ctx, addr, dnsbl, 0, 0)); -} - -struct dns_rr_txt * -dns_resolve_a4dnsbl_txt(struct dns_ctx *ctx, - const struct in_addr *addr, const char *dnsbl) { - return (struct dns_rr_txt *) - dns_resolve(ctx, dns_submit_a4dnsbl_txt(ctx, addr, dnsbl, 0, 0)); -} - - -struct dns_query * -dns_submit_a6dnsbl(struct dns_ctx *ctx, - const struct in6_addr *addr, const char *dnsbl, - dns_query_a4_fn *cbck, void *data) { - dnsc_t dn[DNS_MAXDN]; - if (dns_a6ptodn(addr, dnsbl, dn, sizeof(dn)) <= 0) { - dns_setstatus(ctx, DNS_E_BADQUERY); - return NULL; - } - return - dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_A, DNS_NOSRCH, - dns_parse_a4, (dns_query_fn*)cbck, data); -} - -struct dns_query * -dns_submit_a6dnsbl_txt(struct dns_ctx *ctx, - const struct in6_addr *addr, const char *dnsbl, - dns_query_txt_fn *cbck, void *data) { - dnsc_t dn[DNS_MAXDN]; - if (dns_a6ptodn(addr, dnsbl, dn, sizeof(dn)) <= 0) { - dns_setstatus(ctx, DNS_E_BADQUERY); - return NULL; - } - return - dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_TXT, DNS_NOSRCH, - dns_parse_txt, (dns_query_fn*)cbck, data); -} - -struct dns_rr_a4 * -dns_resolve_a6dnsbl(struct dns_ctx *ctx, - const struct in6_addr *addr, const char *dnsbl) { - return (struct dns_rr_a4 *) - dns_resolve(ctx, dns_submit_a6dnsbl(ctx, addr, dnsbl, 0, 0)); -} - -struct dns_rr_txt * -dns_resolve_a6dnsbl_txt(struct dns_ctx *ctx, - const struct in6_addr *addr, const char *dnsbl) { - return (struct dns_rr_txt *) - dns_resolve(ctx, dns_submit_a6dnsbl_txt(ctx, addr, dnsbl, 0, 0)); -} - -static int -dns_rhsbltodn(const char *name, const char *rhsbl, dnsc_t dn[DNS_MAXDN]) -{ - int l = dns_sptodn(name, dn, DNS_MAXDN); - if (l <= 0) return 0; - l = dns_sptodn(rhsbl, dn+l-1, DNS_MAXDN-l+1); - if (l <= 0) return 0; - return 1; -} - -struct dns_query * -dns_submit_rhsbl(struct dns_ctx *ctx, const char *name, const char *rhsbl, - dns_query_a4_fn *cbck, void *data) { - dnsc_t dn[DNS_MAXDN]; - if (!dns_rhsbltodn(name, rhsbl, dn)) { - dns_setstatus(ctx, DNS_E_BADQUERY); - return NULL; - } - return - dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_A, DNS_NOSRCH, - dns_parse_a4, (dns_query_fn*)cbck, data); -} -struct dns_query * -dns_submit_rhsbl_txt(struct dns_ctx *ctx, const char *name, const char *rhsbl, - dns_query_txt_fn *cbck, void *data) { - dnsc_t dn[DNS_MAXDN]; - if (!dns_rhsbltodn(name, rhsbl, dn)) { - dns_setstatus(ctx, DNS_E_BADQUERY); - return NULL; - } - return - dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_TXT, DNS_NOSRCH, - dns_parse_txt, (dns_query_fn*)cbck, data); -} - -struct dns_rr_a4 * -dns_resolve_rhsbl(struct dns_ctx *ctx, const char *name, const char *rhsbl) { - return (struct dns_rr_a4*) - dns_resolve(ctx, dns_submit_rhsbl(ctx, name, rhsbl, 0, 0)); -} - -struct dns_rr_txt * -dns_resolve_rhsbl_txt(struct dns_ctx *ctx, const char *name, const char *rhsbl) -{ - return (struct dns_rr_txt*) - dns_resolve(ctx, dns_submit_rhsbl_txt(ctx, name, rhsbl, 0, 0)); -} diff --git a/contrib/udns/udns_codes.c b/contrib/udns/udns_codes.c deleted file mode 100644 index c637e98c581..00000000000 --- a/contrib/udns/udns_codes.c +++ /dev/null @@ -1,199 +0,0 @@ -/* Automatically generated. */ -#include "udns.h" - -const struct dns_nameval dns_typetab[] = { - {DNS_T_INVALID,"INVALID"}, - {DNS_T_A,"A"}, - {DNS_T_NS,"NS"}, - {DNS_T_MD,"MD"}, - {DNS_T_MF,"MF"}, - {DNS_T_CNAME,"CNAME"}, - {DNS_T_SOA,"SOA"}, - {DNS_T_MB,"MB"}, - {DNS_T_MG,"MG"}, - {DNS_T_MR,"MR"}, - {DNS_T_NULL,"NULL"}, - {DNS_T_WKS,"WKS"}, - {DNS_T_PTR,"PTR"}, - {DNS_T_HINFO,"HINFO"}, - {DNS_T_MINFO,"MINFO"}, - {DNS_T_MX,"MX"}, - {DNS_T_TXT,"TXT"}, - {DNS_T_RP,"RP"}, - {DNS_T_AFSDB,"AFSDB"}, - {DNS_T_X25,"X25"}, - {DNS_T_ISDN,"ISDN"}, - {DNS_T_RT,"RT"}, - {DNS_T_NSAP,"NSAP"}, - {DNS_T_NSAP_PTR,"NSAP_PTR"}, - {DNS_T_SIG,"SIG"}, - {DNS_T_KEY,"KEY"}, - {DNS_T_PX,"PX"}, - {DNS_T_GPOS,"GPOS"}, - {DNS_T_AAAA,"AAAA"}, - {DNS_T_LOC,"LOC"}, - {DNS_T_NXT,"NXT"}, - {DNS_T_EID,"EID"}, - {DNS_T_NIMLOC,"NIMLOC"}, - {DNS_T_SRV,"SRV"}, - {DNS_T_ATMA,"ATMA"}, - {DNS_T_NAPTR,"NAPTR"}, - {DNS_T_KX,"KX"}, - {DNS_T_CERT,"CERT"}, - {DNS_T_A6,"A6"}, - {DNS_T_DNAME,"DNAME"}, - {DNS_T_SINK,"SINK"}, - {DNS_T_OPT,"OPT"}, - {DNS_T_DS,"DS"}, - {DNS_T_SSHFP,"SSHFP"}, - {DNS_T_IPSECKEY,"IPSECKEY"}, - {DNS_T_RRSIG,"RRSIG"}, - {DNS_T_NSEC,"NSEC"}, - {DNS_T_DNSKEY,"DNSKEY"}, - {DNS_T_DHCID,"DHCID"}, - {DNS_T_NSEC3,"NSEC3"}, - {DNS_T_NSEC3PARAMS,"NSEC3PARAMS"}, - {DNS_T_TALINK,"TALINK"}, - {DNS_T_SPF,"SPF"}, - {DNS_T_UINFO,"UINFO"}, - {DNS_T_UID,"UID"}, - {DNS_T_GID,"GID"}, - {DNS_T_UNSPEC,"UNSPEC"}, - {DNS_T_TSIG,"TSIG"}, - {DNS_T_IXFR,"IXFR"}, - {DNS_T_AXFR,"AXFR"}, - {DNS_T_MAILB,"MAILB"}, - {DNS_T_MAILA,"MAILA"}, - {DNS_T_ANY,"ANY"}, - {DNS_T_ZXFR,"ZXFR"}, - {DNS_T_DLV,"DLV"}, - {DNS_T_MAX,"MAX"}, - {0,0}}; -const char *dns_typename(enum dns_type code) { - static char nm[20]; - switch(code) { - case DNS_T_INVALID: return dns_typetab[0].name; - case DNS_T_A: return dns_typetab[1].name; - case DNS_T_NS: return dns_typetab[2].name; - case DNS_T_MD: return dns_typetab[3].name; - case DNS_T_MF: return dns_typetab[4].name; - case DNS_T_CNAME: return dns_typetab[5].name; - case DNS_T_SOA: return dns_typetab[6].name; - case DNS_T_MB: return dns_typetab[7].name; - case DNS_T_MG: return dns_typetab[8].name; - case DNS_T_MR: return dns_typetab[9].name; - case DNS_T_NULL: return dns_typetab[10].name; - case DNS_T_WKS: return dns_typetab[11].name; - case DNS_T_PTR: return dns_typetab[12].name; - case DNS_T_HINFO: return dns_typetab[13].name; - case DNS_T_MINFO: return dns_typetab[14].name; - case DNS_T_MX: return dns_typetab[15].name; - case DNS_T_TXT: return dns_typetab[16].name; - case DNS_T_RP: return dns_typetab[17].name; - case DNS_T_AFSDB: return dns_typetab[18].name; - case DNS_T_X25: return dns_typetab[19].name; - case DNS_T_ISDN: return dns_typetab[20].name; - case DNS_T_RT: return dns_typetab[21].name; - case DNS_T_NSAP: return dns_typetab[22].name; - case DNS_T_NSAP_PTR: return dns_typetab[23].name; - case DNS_T_SIG: return dns_typetab[24].name; - case DNS_T_KEY: return dns_typetab[25].name; - case DNS_T_PX: return dns_typetab[26].name; - case DNS_T_GPOS: return dns_typetab[27].name; - case DNS_T_AAAA: return dns_typetab[28].name; - case DNS_T_LOC: return dns_typetab[29].name; - case DNS_T_NXT: return dns_typetab[30].name; - case DNS_T_EID: return dns_typetab[31].name; - case DNS_T_NIMLOC: return dns_typetab[32].name; - case DNS_T_SRV: return dns_typetab[33].name; - case DNS_T_ATMA: return dns_typetab[34].name; - case DNS_T_NAPTR: return dns_typetab[35].name; - case DNS_T_KX: return dns_typetab[36].name; - case DNS_T_CERT: return dns_typetab[37].name; - case DNS_T_A6: return dns_typetab[38].name; - case DNS_T_DNAME: return dns_typetab[39].name; - case DNS_T_SINK: return dns_typetab[40].name; - case DNS_T_OPT: return dns_typetab[41].name; - case DNS_T_DS: return dns_typetab[42].name; - case DNS_T_SSHFP: return dns_typetab[43].name; - case DNS_T_IPSECKEY: return dns_typetab[44].name; - case DNS_T_RRSIG: return dns_typetab[45].name; - case DNS_T_NSEC: return dns_typetab[46].name; - case DNS_T_DNSKEY: return dns_typetab[47].name; - case DNS_T_DHCID: return dns_typetab[48].name; - case DNS_T_NSEC3: return dns_typetab[49].name; - case DNS_T_NSEC3PARAMS: return dns_typetab[50].name; - case DNS_T_TALINK: return dns_typetab[51].name; - case DNS_T_SPF: return dns_typetab[52].name; - case DNS_T_UINFO: return dns_typetab[53].name; - case DNS_T_UID: return dns_typetab[54].name; - case DNS_T_GID: return dns_typetab[55].name; - case DNS_T_UNSPEC: return dns_typetab[56].name; - case DNS_T_TSIG: return dns_typetab[57].name; - case DNS_T_IXFR: return dns_typetab[58].name; - case DNS_T_AXFR: return dns_typetab[59].name; - case DNS_T_MAILB: return dns_typetab[60].name; - case DNS_T_MAILA: return dns_typetab[61].name; - case DNS_T_ANY: return dns_typetab[62].name; - case DNS_T_ZXFR: return dns_typetab[63].name; - case DNS_T_DLV: return dns_typetab[64].name; - case DNS_T_MAX: return dns_typetab[65].name; - } - return _dns_format_code(nm,"type",code); -} - -const struct dns_nameval dns_classtab[] = { - {DNS_C_INVALID,"INVALID"}, - {DNS_C_IN,"IN"}, - {DNS_C_CH,"CH"}, - {DNS_C_HS,"HS"}, - {DNS_C_ANY,"ANY"}, - {0,0}}; -const char *dns_classname(enum dns_class code) { - static char nm[20]; - switch(code) { - case DNS_C_INVALID: return dns_classtab[0].name; - case DNS_C_IN: return dns_classtab[1].name; - case DNS_C_CH: return dns_classtab[2].name; - case DNS_C_HS: return dns_classtab[3].name; - case DNS_C_ANY: return dns_classtab[4].name; - } - return _dns_format_code(nm,"class",code); -} - -const struct dns_nameval dns_rcodetab[] = { - {DNS_R_NOERROR,"NOERROR"}, - {DNS_R_FORMERR,"FORMERR"}, - {DNS_R_SERVFAIL,"SERVFAIL"}, - {DNS_R_NXDOMAIN,"NXDOMAIN"}, - {DNS_R_NOTIMPL,"NOTIMPL"}, - {DNS_R_REFUSED,"REFUSED"}, - {DNS_R_YXDOMAIN,"YXDOMAIN"}, - {DNS_R_YXRRSET,"YXRRSET"}, - {DNS_R_NXRRSET,"NXRRSET"}, - {DNS_R_NOTAUTH,"NOTAUTH"}, - {DNS_R_NOTZONE,"NOTZONE"}, - {DNS_R_BADSIG,"BADSIG"}, - {DNS_R_BADKEY,"BADKEY"}, - {DNS_R_BADTIME,"BADTIME"}, - {0,0}}; -const char *dns_rcodename(enum dns_rcode code) { - static char nm[20]; - switch(code) { - case DNS_R_NOERROR: return dns_rcodetab[0].name; - case DNS_R_FORMERR: return dns_rcodetab[1].name; - case DNS_R_SERVFAIL: return dns_rcodetab[2].name; - case DNS_R_NXDOMAIN: return dns_rcodetab[3].name; - case DNS_R_NOTIMPL: return dns_rcodetab[4].name; - case DNS_R_REFUSED: return dns_rcodetab[5].name; - case DNS_R_YXDOMAIN: return dns_rcodetab[6].name; - case DNS_R_YXRRSET: return dns_rcodetab[7].name; - case DNS_R_NXRRSET: return dns_rcodetab[8].name; - case DNS_R_NOTAUTH: return dns_rcodetab[9].name; - case DNS_R_NOTZONE: return dns_rcodetab[10].name; - case DNS_R_BADSIG: return dns_rcodetab[11].name; - case DNS_R_BADKEY: return dns_rcodetab[12].name; - case DNS_R_BADTIME: return dns_rcodetab[13].name; - } - return _dns_format_code(nm,"rcode",code); -} diff --git a/contrib/udns/udns_dn.c b/contrib/udns/udns_dn.c deleted file mode 100644 index ae3fd177882..00000000000 --- a/contrib/udns/udns_dn.c +++ /dev/null @@ -1,379 +0,0 @@ -/* udns_dn.c - domain names manipulation routines - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#include -#include "udns.h" - -unsigned dns_dnlen(dnscc_t *dn) { - register dnscc_t *d = dn; - while(*d) - d += 1 + *d; - return (unsigned)(d - dn) + 1; -} - -unsigned dns_dnlabels(register dnscc_t *dn) { - register unsigned l = 0; - while(*dn) - ++l, dn += 1 + *dn; - return l; -} - -unsigned dns_dnequal(register dnscc_t *dn1, register dnscc_t *dn2) { - register unsigned c; - dnscc_t *dn = dn1; - for(;;) { - if ((c = *dn1++) != *dn2++) - return 0; - if (!c) - return (unsigned)(dn1 - dn); - while(c--) { - if (DNS_DNLC(*dn1) != DNS_DNLC(*dn2)) - return 0; - ++dn1; ++dn2; - } - } -} - -unsigned -dns_dntodn(dnscc_t *sdn, dnsc_t *ddn, unsigned ddnsiz) { - unsigned sdnlen = dns_dnlen(sdn); - if (ddnsiz < sdnlen) - return 0; - memcpy(ddn, sdn, sdnlen); - return sdnlen; -} - -int -dns_ptodn(const char *name, unsigned namelen, - dnsc_t *dn, unsigned dnsiz, int *isabs) -{ - dnsc_t *dp; /* current position in dn (len byte first) */ - dnsc_t *const de /* end of dn: last byte that can be filled up */ - = dn + (dnsiz >= DNS_MAXDN ? DNS_MAXDN : dnsiz) - 1; - dnscc_t *np = (dnscc_t *)name; - dnscc_t *ne = np + (namelen ? namelen : strlen((char*)np)); - dnsc_t *llab; /* start of last label (llab[-1] will be length) */ - unsigned c; /* next input character, or length of last label */ - - if (!dnsiz) - return 0; - dp = llab = dn + 1; - - while(np < ne) { - - if (*np == '.') { /* label delimiter */ - c = dp - llab; /* length of the label */ - if (!c) { /* empty label */ - if (np == (dnscc_t *)name && np + 1 == ne) { - /* special case for root dn, aka `.' */ - ++np; - break; - } - return -1; /* zero label */ - } - if (c > DNS_MAXLABEL) - return -1; /* label too long */ - llab[-1] = (dnsc_t)c; /* update len of last label */ - llab = ++dp; /* start new label, llab[-1] will be len of it */ - ++np; - continue; - } - - /* check whenever we may put out one more byte */ - if (dp >= de) /* too long? */ - return dnsiz >= DNS_MAXDN ? -1 : 0; - if (*np != '\\') { /* non-escape, simple case */ - *dp++ = *np++; - continue; - } - /* handle \-style escape */ - /* note that traditionally, domain names (gethostbyname etc) - * used decimal \dd notation, not octal \ooo (RFC1035), so - * we're following this tradition here. - */ - if (++np == ne) - return -1; /* bad escape */ - else if (*np >= '0' && *np <= '9') { /* decimal number */ - /* we allow not only exactly 3 digits as per RFC1035, - * but also 2 or 1, for better usability. */ - c = *np++ - '0'; - if (np < ne && *np >= '0' && *np <= '9') { /* 2digits */ - c = c * 10 + *np++ - '0'; - if (np < ne && *np >= '0' && *np <= '9') { - c = c * 10 + *np++ - '0'; - if (c > 255) - return -1; /* bad escape */ - } - } - } - else - c = *np++; - *dp++ = (dnsc_t)c; /* place next out byte */ - } - - if ((c = dp - llab) > DNS_MAXLABEL) - return -1; /* label too long */ - if ((llab[-1] = (dnsc_t)c) != 0) { - *dp++ = 0; - if (isabs) - *isabs = 0; - } - else if (isabs) - *isabs = 1; - - return dp - dn; -} - -dnscc_t dns_inaddr_arpa_dn[14] = "\07in-addr\04arpa"; - -dnsc_t * -dns_a4todn_(const struct in_addr *addr, dnsc_t *dn, dnsc_t *dne) { - const unsigned char *s = ((const unsigned char *)addr) + 4; - while(s > (const unsigned char *)addr) { - unsigned n = *--s; - dnsc_t *p = dn + 1; - if (n > 99) { - if (p + 2 > dne) return 0; - *p++ = n / 100 + '0'; - *p++ = (n % 100 / 10) + '0'; - *p = n % 10 + '0'; - } - else if (n > 9) { - if (p + 1 > dne) return 0; - *p++ = n / 10 + '0'; - *p = n % 10 + '0'; - } - else { - if (p > dne) return 0; - *p = n + '0'; - } - *dn = p - dn; - dn = p + 1; - } - return dn; -} - -int dns_a4todn(const struct in_addr *addr, dnscc_t *tdn, - dnsc_t *dn, unsigned dnsiz) { - dnsc_t *dne = dn + (dnsiz > DNS_MAXDN ? DNS_MAXDN : dnsiz); - dnsc_t *p; - unsigned l; - p = dns_a4todn_(addr, dn, dne); - if (!p) return 0; - if (!tdn) - tdn = dns_inaddr_arpa_dn; - l = dns_dnlen(tdn); - if (p + l > dne) return dnsiz >= DNS_MAXDN ? -1 : 0; - memcpy(p, tdn, l); - return (p + l) - dn; -} - -int dns_a4ptodn(const struct in_addr *addr, const char *tname, - dnsc_t *dn, unsigned dnsiz) { - dnsc_t *p; - int r; - if (!tname) - return dns_a4todn(addr, NULL, dn, dnsiz); - p = dns_a4todn_(addr, dn, dn + dnsiz); - if (!p) return 0; - r = dns_sptodn(tname, p, dnsiz - (p - dn)); - return r != 0 ? r : dnsiz >= DNS_MAXDN ? -1 : 0; -} - -dnscc_t dns_ip6_arpa_dn[10] = "\03ip6\04arpa"; - -dnsc_t * -dns_a6todn_(const struct in6_addr *addr, dnsc_t *dn, dnsc_t *dne) { - const unsigned char *s = ((const unsigned char *)addr) + 16; - if (dn + 64 > dne) return 0; - while(s > (const unsigned char *)addr) { - unsigned n = *--s & 0x0f; - *dn++ = 1; - *dn++ = n > 9 ? n + 'a' - 10 : n + '0'; - *dn++ = 1; - n = *s >> 4; - *dn++ = n > 9 ? n + 'a' - 10 : n + '0'; - } - return dn; -} - -int dns_a6todn(const struct in6_addr *addr, dnscc_t *tdn, - dnsc_t *dn, unsigned dnsiz) { - dnsc_t *dne = dn + (dnsiz > DNS_MAXDN ? DNS_MAXDN : dnsiz); - dnsc_t *p; - unsigned l; - p = dns_a6todn_(addr, dn, dne); - if (!p) return 0; - if (!tdn) - tdn = dns_ip6_arpa_dn; - l = dns_dnlen(tdn); - if (p + l > dne) return dnsiz >= DNS_MAXDN ? -1 : 0; - memcpy(p, tdn, l); - return (p + l) - dn; -} - -int dns_a6ptodn(const struct in6_addr *addr, const char *tname, - dnsc_t *dn, unsigned dnsiz) { - dnsc_t *p; - int r; - if (!tname) - return dns_a6todn(addr, NULL, dn, dnsiz); - p = dns_a6todn_(addr, dn, dn + dnsiz); - if (!p) return 0; - r = dns_sptodn(tname, p, dnsiz - (p - dn)); - return r != 0 ? r : dnsiz >= DNS_MAXDN ? -1 : 0; -} - -/* return size of buffer required to convert the dn into asciiz string. - * Keep in sync with dns_dntop() below. - */ -unsigned dns_dntop_size(dnscc_t *dn) { - unsigned size = 0; /* the size reqd */ - dnscc_t *le; /* label end */ - - while(*dn) { - /* *dn is the length of the next label, non-zero */ - if (size) - ++size; /* for the dot */ - le = dn + *dn + 1; - ++dn; - do { - switch(*dn) { - case '.': - case '\\': - /* Special modifiers in zone files. */ - case '"': - case ';': - case '@': - case '$': - size += 2; - break; - default: - if (*dn <= 0x20 || *dn >= 0x7f) - /* \ddd decimal notation */ - size += 4; - else - size += 1; - } - } while(++dn < le); - } - size += 1; /* zero byte at the end - string terminator */ - return size > DNS_MAXNAME ? 0 : size; -} - -/* Convert the dn into asciiz string. - * Keep in sync with dns_dntop_size() above. - */ -int dns_dntop(dnscc_t *dn, char *name, unsigned namesiz) { - char *np = name; /* current name ptr */ - char *const ne = name + namesiz; /* end of name */ - dnscc_t *le; /* label end */ - - while(*dn) { - /* *dn is the length of the next label, non-zero */ - if (np != name) { - if (np >= ne) goto toolong; - *np++ = '.'; - } - le = dn + *dn + 1; - ++dn; - do { - switch(*dn) { - case '.': - case '\\': - /* Special modifiers in zone files. */ - case '"': - case ';': - case '@': - case '$': - if (np + 2 > ne) goto toolong; - *np++ = '\\'; - *np++ = *dn; - break; - default: - if (*dn <= 0x20 || *dn >= 0x7f) { - /* \ddd decimal notation */ - if (np + 4 >= ne) goto toolong; - *np++ = '\\'; - *np++ = '0' + (*dn / 100); - *np++ = '0' + ((*dn % 100) / 10); - *np++ = '0' + (*dn % 10); - } - else { - if (np >= ne) goto toolong; - *np++ = *dn; - } - } - } while(++dn < le); - } - if (np >= ne) goto toolong; - *np++ = '\0'; - return np - name; -toolong: - return namesiz >= DNS_MAXNAME ? -1 : 0; -} - -#ifdef TEST -#include -#include - -int main(int argc, char **argv) { - int i; - int sz; - dnsc_t dn[DNS_MAXDN+10]; - dnsc_t *dl, *dp; - int isabs; - - sz = (argc > 1) ? atoi(argv[1]) : 0; - - for(i = 2; i < argc; ++i) { - int r = dns_ptodn(argv[i], 0, dn, sz, &isabs); - printf("%s: ", argv[i]); - if (r < 0) printf("error\n"); - else if (!r) printf("buffer too small\n"); - else { - printf("len=%d dnlen=%d size=%d name:", - r, dns_dnlen(dn), dns_dntop_size(dn)); - dl = dn; - while(*dl) { - printf(" %d=", *dl); - dp = dl + 1; - dl = dp + *dl; - while(dp < dl) { - if (*dp <= ' ' || *dp >= 0x7f) - printf("\\%03d", *dp); - else if (*dp == '.' || *dp == '\\') - printf("\\%c", *dp); - else - putchar(*dp); - ++dp; - } - } - if (isabs) putchar('.'); - putchar('\n'); - } - } - return 0; -} - -#endif /* TEST */ diff --git a/contrib/udns/udns_dntosp.c b/contrib/udns/udns_dntosp.c deleted file mode 100644 index 823fde211e5..00000000000 --- a/contrib/udns/udns_dntosp.c +++ /dev/null @@ -1,30 +0,0 @@ -/* udns_dntosp.c - dns_dntosp() = convert DN to asciiz string using static buffer - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#include "udns.h" - -static char name[DNS_MAXNAME]; - -const char *dns_dntosp(dnscc_t *dn) { - return dns_dntop(dn, name, sizeof(name)) > 0 ? name : 0; -} diff --git a/contrib/udns/udns_init.c b/contrib/udns/udns_init.c deleted file mode 100644 index 493af58917f..00000000000 --- a/contrib/udns/udns_init.c +++ /dev/null @@ -1,231 +0,0 @@ -/* udns_init.c - resolver initialisation stuff - - Copyright (C) 2006 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#ifdef HAVE_CONFIG_H -# include "config.h" -#endif -#ifdef WINDOWS -# include /* includes */ -# include /* for dns server addresses etc */ -#else -# include -# include -# include -#endif /* !WINDOWS */ - -#include -#include -#include "udns.h" - -#define ISSPACE(x) (x == ' ' || x == '\t' || x == '\r' || x == '\n') - -static const char space[] = " \t\r\n"; - -static void dns_set_serv_internal(struct dns_ctx *ctx, char *serv) { - dns_add_serv(ctx, NULL); - for(serv = strtok(serv, space); serv; serv = strtok(NULL, space)) - dns_add_serv(ctx, serv); -} - -static void dns_set_srch_internal(struct dns_ctx *ctx, char *srch) { - dns_add_srch(ctx, NULL); - for(srch = strtok(srch, space); srch; srch = strtok(NULL, space)) - dns_add_srch(ctx, srch); -} - -#ifdef WINDOWS - -#ifndef NO_IPHLPAPI -/* Apparently, some systems does not have proper headers for IPHLPAIP to work. - * The best is to upgrade headers, but here's another, ugly workaround for - * this: compile with -DNO_IPHLPAPI. - */ - -typedef DWORD (WINAPI *GetAdaptersAddressesFunc)( - ULONG Family, DWORD Flags, PVOID Reserved, - PIP_ADAPTER_ADDRESSES pAdapterAddresses, - PULONG pOutBufLen); - -static int dns_initns_iphlpapi(struct dns_ctx *ctx) { - HANDLE h_iphlpapi; - GetAdaptersAddressesFunc pfnGetAdAddrs; - PIP_ADAPTER_ADDRESSES pAddr, pAddrBuf; - PIP_ADAPTER_DNS_SERVER_ADDRESS pDnsAddr; - ULONG ulOutBufLen; - DWORD dwRetVal; - int ret = -1; - - h_iphlpapi = LoadLibrary("iphlpapi.dll"); - if (!h_iphlpapi) - return -1; - pfnGetAdAddrs = (GetAdaptersAddressesFunc) - GetProcAddress(h_iphlpapi, "GetAdaptersAddresses"); - if (!pfnGetAdAddrs) goto freelib; - ulOutBufLen = 0; - dwRetVal = pfnGetAdAddrs(AF_UNSPEC, 0, NULL, NULL, &ulOutBufLen); - if (dwRetVal != ERROR_BUFFER_OVERFLOW) goto freelib; - pAddrBuf = malloc(ulOutBufLen); - if (!pAddrBuf) goto freelib; - dwRetVal = pfnGetAdAddrs(AF_UNSPEC, 0, NULL, pAddrBuf, &ulOutBufLen); - if (dwRetVal != ERROR_SUCCESS) goto freemem; - for (pAddr = pAddrBuf; pAddr; pAddr = pAddr->Next) - for (pDnsAddr = pAddr->FirstDnsServerAddress; - pDnsAddr; - pDnsAddr = pDnsAddr->Next) - dns_add_serv_s(ctx, pDnsAddr->Address.lpSockaddr); - ret = 0; -freemem: - free(pAddrBuf); -freelib: - FreeLibrary(h_iphlpapi); - return ret; -} - -#else /* NO_IPHLPAPI */ - -#define dns_initns_iphlpapi(ctx) (-1) - -#endif /* NO_IPHLPAPI */ - -static int dns_initns_registry(struct dns_ctx *ctx) { - LONG res; - HKEY hk; - DWORD type = REG_EXPAND_SZ | REG_SZ; - DWORD len; - char valBuf[1024]; - -#define REGKEY_WINNT "SYSTEM\\CurrentControlSet\\Services\\Tcpip\\Parameters" -#define REGKEY_WIN9x "SYSTEM\\CurrentControlSet\\Services\\VxD\\MSTCP" - res = RegOpenKeyEx(HKEY_LOCAL_MACHINE, REGKEY_WINNT, 0, KEY_QUERY_VALUE, &hk); - if (res != ERROR_SUCCESS) - res = RegOpenKeyEx(HKEY_LOCAL_MACHINE, REGKEY_WIN9x, - 0, KEY_QUERY_VALUE, &hk); - if (res != ERROR_SUCCESS) - return -1; - len = sizeof(valBuf) - 1; - res = RegQueryValueEx(hk, "NameServer", NULL, &type, (BYTE*)valBuf, &len); - if (res != ERROR_SUCCESS || !len || !valBuf[0]) { - len = sizeof(valBuf) - 1; - res = RegQueryValueEx(hk, "DhcpNameServer", NULL, &type, - (BYTE*)valBuf, &len); - } - RegCloseKey(hk); - if (res != ERROR_SUCCESS || !len || !valBuf[0]) - return -1; - valBuf[len] = '\0'; - /* nameservers are stored as a whitespace-seperate list: - * "192.168.1.1 123.21.32.12" */ - dns_set_serv_internal(ctx, valBuf); - return 0; -} - -#else /* !WINDOWS */ - -static int dns_init_resolvconf(struct dns_ctx *ctx) { - char *v; - char buf[2049]; /* this buffer is used to hold /etc/resolv.conf */ - int has_srch = 0; - - /* read resolv.conf... */ - { int fd = open("/etc/resolv.conf", O_RDONLY); - if (fd >= 0) { - int l = read(fd, buf, sizeof(buf) - 1); - close(fd); - buf[l < 0 ? 0 : l] = '\0'; - } - else - buf[0] = '\0'; - } - if (buf[0]) { /* ...and parse it */ - char *line, *nextline; - line = buf; - do { - nextline = strchr(line, '\n'); - if (nextline) *nextline++ = '\0'; - v = line; - while(*v && !ISSPACE(*v)) ++v; - if (!*v) continue; - *v++ = '\0'; - while(ISSPACE(*v)) ++v; - if (!*v) continue; - if (strcmp(line, "domain") == 0) { - dns_set_srch_internal(ctx, strtok(v, space)); - has_srch = 1; - } - else if (strcmp(line, "search") == 0) { - dns_set_srch_internal(ctx, v); - has_srch = 1; - } - else if (strcmp(line, "nameserver") == 0) - dns_add_serv(ctx, strtok(v, space)); - else if (strcmp(line, "options") == 0) - dns_set_opts(ctx, v); - } while((line = nextline) != NULL); - } - - buf[sizeof(buf)-1] = '\0'; - - /* get list of nameservers from env. vars. */ - if ((v = getenv("NSCACHEIP")) != NULL || - (v = getenv("NAMESERVERS")) != NULL) { - strncpy(buf, v, sizeof(buf) - 1); - dns_set_serv_internal(ctx, buf); - } - /* if $LOCALDOMAIN is set, use it for search list */ - if ((v = getenv("LOCALDOMAIN")) != NULL) { - strncpy(buf, v, sizeof(buf) - 1); - dns_set_srch_internal(ctx, buf); - has_srch = 1; - } - if ((v = getenv("RES_OPTIONS")) != NULL) - dns_set_opts(ctx, v); - - /* if still no search list, use local domain name */ - if (has_srch && - gethostname(buf, sizeof(buf) - 1) == 0 && - (v = strchr(buf, '.')) != NULL && - *++v != '\0') - dns_add_srch(ctx, v); - - return 0; -} - -#endif /* !WINDOWS */ - -int dns_init(struct dns_ctx *ctx, int do_open) { - if (!ctx) - ctx = &dns_defctx; - dns_reset(ctx); - -#ifdef WINDOWS - if (dns_initns_iphlpapi(ctx) != 0) - dns_initns_registry(ctx); - /*XXX WINDOWS: probably good to get default domain and search list too... - * And options. Something is in registry. */ - /*XXX WINDOWS: maybe environment variables are also useful? */ -#else - dns_init_resolvconf(ctx); -#endif - - return do_open ? dns_open(ctx) : 0; -} diff --git a/contrib/udns/udns_jran.c b/contrib/udns/udns_jran.c deleted file mode 100644 index 19f9d02bb14..00000000000 --- a/contrib/udns/udns_jran.c +++ /dev/null @@ -1,52 +0,0 @@ -/* udns_jran.c: small non-cryptographic random number generator - * taken from http://burtleburtle.net/bob/rand/smallprng.html - * by Bob Jenkins, Public domain. - */ - -#include "udns.h" - -#define rot32(x,k) (((x) << (k)) | ((x) >> (32-(k)))) -#define rot64(x,k) (((x) << (k)) | ((x) >> (64-(k)))) -#define tr32(x) ((x)&0xffffffffu) - -unsigned udns_jranval(struct udns_jranctx *x) { - /* This routine can be made to work with either 32 or 64bit words - - * if JRAN_32_64 is defined when compiling the file. - * We use if() instead of #if since there's no good - * portable way to check sizeof() in preprocessor without - * introducing some ugly configure-time checks. - * Most compilers will optimize the wrong branches away anyway. - * By default it assumes 32bit integers - */ -#ifdef JRAN_32_64 - if (sizeof(unsigned) == 4) { -#endif - unsigned e = tr32(x->a - rot32(x->b, 27)); - x->a = tr32(x->b ^ rot32(x->c, 17)); - x->b = tr32(x->c + x->d); - x->c = tr32(x->d + e); - x->d = tr32(e + x->a); -#ifdef JRAN_32_64 - } - else if (sizeof(unsigned) == 8) { /* assuming it's 64bits */ - unsigned e = x->a - rot64(x->b, 7); - x->a = x->b ^ rot64(x->c, 13); - x->b = x->c + rot64(x->d, 37); - x->c = x->d + e; - x->d = e + x->a; - } - else { - unsigned e = 0; - x->d = 1/e; /* bail */ - } -#endif - return x->d; -} - -void udns_jraninit(struct udns_jranctx *x, unsigned seed) { - unsigned i; - x->a = 0xf1ea5eed; - x->b = x->c = x->d = seed; - for (i = 0; i < 20; ++i) - (void)udns_jranval(x); -} diff --git a/contrib/udns/udns_misc.c b/contrib/udns/udns_misc.c deleted file mode 100644 index c162e70e77b..00000000000 --- a/contrib/udns/udns_misc.c +++ /dev/null @@ -1,67 +0,0 @@ -/* udns_misc.c - miscellaneous routines - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#include "udns.h" - -int dns_findname(const struct dns_nameval *nv, const char *name) { - register const char *a, *b; - for(; nv->name; ++nv) - for(a = name, b = nv->name; ; ++a, ++b) - if (DNS_DNUC(*a) != *b) break; - else if (!*a) return nv->val; - return -1; -} - -const char *_dns_format_code(char *buf, const char *prefix, int code) { - char *bp = buf; - unsigned c, n; - do *bp++ = DNS_DNUC(*prefix); - while(*++prefix); - *bp++ = '#'; - if (code < 0) code = -code, *bp++ = '-'; - n = 0; c = code; - do ++n; - while((c /= 10)); - c = code; - bp[n--] = '\0'; - do bp[n--] = c % 10 + '0'; - while((c /= 10)); - return buf; -} - -const char *dns_strerror(int err) { - if (err >= 0) return "successeful completion"; - switch(err) { - case DNS_E_TEMPFAIL: return "temporary failure in name resolution"; - case DNS_E_PROTOCOL: return "protocol error"; - case DNS_E_NXDOMAIN: return "domain name does not exist"; - case DNS_E_NODATA: return "valid domain but no data of requested type"; - case DNS_E_NOMEM: return "out of memory"; - case DNS_E_BADQUERY: return "malformed query"; - default: return "unknown error"; - } -} - -const char *dns_version(void) { - return UDNS_VERSION; -} diff --git a/contrib/udns/udns_parse.c b/contrib/udns/udns_parse.c deleted file mode 100644 index 8924b1562d2..00000000000 --- a/contrib/udns/udns_parse.c +++ /dev/null @@ -1,169 +0,0 @@ -/* udns_parse.c - raw DNS packet parsing routines - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#include -#include -#include "udns.h" - -dnscc_t *dns_skipdn(dnscc_t *cur, dnscc_t *end) { - unsigned c; - for(;;) { - if (cur >= end) - return NULL; - c = *cur++; - if (!c) - return cur; - if (c & 192) /* jump */ - return cur + 1 >= end ? NULL : cur + 1; - cur += c; - } -} - -int -dns_getdn(dnscc_t *pkt, dnscc_t **cur, dnscc_t *end, - register dnsc_t *dn, unsigned dnsiz) { - unsigned c; - dnscc_t *pp = *cur; /* current packet pointer */ - dnsc_t *dp = dn; /* current dn pointer */ - dnsc_t *const de /* end of the DN dest */ - = dn + (dnsiz < DNS_MAXDN ? dnsiz : DNS_MAXDN); - dnscc_t *jump = NULL; /* ptr after first jump if any */ - unsigned loop = 100; /* jump loop counter */ - - for(;;) { /* loop by labels */ - if (pp >= end) /* reached end of packet? */ - return -1; - c = *pp++; /* length of the label */ - if (!c) { /* empty label: terminate */ - if (dn >= de) /* can't fit terminator */ - goto noroom; - *dp++ = 0; - /* return next pos: either after the first jump or current */ - *cur = jump ? jump : pp; - return dp - dn; - } - if (c & 192) { /* jump */ - if (pp >= end) /* eop instead of jump pos */ - return -1; - if (!jump) jump = pp + 1; /* remember first jump */ - else if (!--loop) return -1; /* too many jumps */ - c = ((c & ~192) << 8) | *pp; /* new pos */ - if (c < DNS_HSIZE) /* don't allow jump into the header */ - return -1; - pp = pkt + c; - continue; - } - if (c > DNS_MAXLABEL) /* too long label? */ - return -1; - if (pp + c > end) /* label does not fit in packet? */ - return -1; - if (dp + c + 1 > de) /* if enouth room for the label */ - goto noroom; - *dp++ = c; /* label length */ - memcpy(dp, pp, c); /* and the label itself */ - dp += c; - pp += c; /* advance to the next label */ - } -noroom: - return dnsiz < DNS_MAXDN ? 0 : -1; -} - -void dns_rewind(struct dns_parse *p, dnscc_t *qdn) { - p->dnsp_qdn = qdn; - p->dnsp_cur = p->dnsp_ans; - p->dnsp_rrl = dns_numan(p->dnsp_pkt); - p->dnsp_ttl = 0xffffffffu; - p->dnsp_nrr = 0; -} - -void -dns_initparse(struct dns_parse *p, dnscc_t *qdn, - dnscc_t *pkt, dnscc_t *cur, dnscc_t *end) { - p->dnsp_pkt = pkt; - p->dnsp_end = end; - p->dnsp_rrl = dns_numan(pkt); - p->dnsp_qdn = qdn; - assert(cur + 4 <= end); - if ((p->dnsp_qtyp = dns_get16(cur+0)) == DNS_T_ANY) p->dnsp_qtyp = 0; - if ((p->dnsp_qcls = dns_get16(cur+2)) == DNS_C_ANY) p->dnsp_qcls = 0; - p->dnsp_cur = p->dnsp_ans = cur + 4; - p->dnsp_ttl = 0xffffffffu; - p->dnsp_nrr = 0; -} - -int dns_nextrr(struct dns_parse *p, struct dns_rr *rr) { - dnscc_t *cur = p->dnsp_cur; - while(p->dnsp_rrl > 0) { - --p->dnsp_rrl; - if (dns_getdn(p->dnsp_pkt, &cur, p->dnsp_end, - rr->dnsrr_dn, sizeof(rr->dnsrr_dn)) <= 0) - return -1; - if (cur + 10 > p->dnsp_end) - return -1; - rr->dnsrr_typ = dns_get16(cur); - rr->dnsrr_cls = dns_get16(cur+2); - rr->dnsrr_ttl = dns_get32(cur+4); - rr->dnsrr_dsz = dns_get16(cur+8); - rr->dnsrr_dptr = cur = cur + 10; - rr->dnsrr_dend = cur = cur + rr->dnsrr_dsz; - if (cur > p->dnsp_end) - return -1; - if (p->dnsp_qdn && !dns_dnequal(p->dnsp_qdn, rr->dnsrr_dn)) - continue; - if ((!p->dnsp_qcls || p->dnsp_qcls == rr->dnsrr_cls) && - (!p->dnsp_qtyp || p->dnsp_qtyp == rr->dnsrr_typ)) { - p->dnsp_cur = cur; - ++p->dnsp_nrr; - if (p->dnsp_ttl > rr->dnsrr_ttl) p->dnsp_ttl = rr->dnsrr_ttl; - return 1; - } - if (p->dnsp_qdn && rr->dnsrr_typ == DNS_T_CNAME && !p->dnsp_nrr) { - if (dns_getdn(p->dnsp_pkt, &rr->dnsrr_dptr, p->dnsp_end, - p->dnsp_dnbuf, sizeof(p->dnsp_dnbuf)) <= 0 || - rr->dnsrr_dptr != rr->dnsrr_dend) - return -1; - p->dnsp_qdn = p->dnsp_dnbuf; - if (p->dnsp_ttl > rr->dnsrr_ttl) p->dnsp_ttl = rr->dnsrr_ttl; - } - } - p->dnsp_cur = cur; - return 0; -} - -int dns_stdrr_size(const struct dns_parse *p) { - return - dns_dntop_size(p->dnsp_qdn) + - (p->dnsp_qdn == dns_payload(p->dnsp_pkt) ? 0 : - dns_dntop_size(dns_payload(p->dnsp_pkt))); -} - -void *dns_stdrr_finish(struct dns_rr_null *ret, char *cp, - const struct dns_parse *p) { - cp += dns_dntop(p->dnsp_qdn, (ret->dnsn_cname = cp), DNS_MAXNAME); - if (p->dnsp_qdn == dns_payload(p->dnsp_pkt)) - ret->dnsn_qname = ret->dnsn_cname; - else - dns_dntop(dns_payload(p->dnsp_pkt), (ret->dnsn_qname = cp), DNS_MAXNAME); - ret->dnsn_ttl = p->dnsp_ttl; - return ret; -} diff --git a/contrib/udns/udns_resolver.c b/contrib/udns/udns_resolver.c deleted file mode 100644 index b8f899a2460..00000000000 --- a/contrib/udns/udns_resolver.c +++ /dev/null @@ -1,1323 +0,0 @@ -/* udns_resolver.c - resolver stuff (main module) - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#ifdef HAVE_CONFIG_H -# include "config.h" -#endif -#ifdef WINDOWS -# include /* includes */ -# include /* needed for struct in6_addr */ -#else -# include -# include -# include -# include -# include -# include -# ifdef HAVE_POLL -# include -# else -# ifdef HAVE_SYS_SELECT_H -# include -# endif -# endif -# ifdef HAVE_TIMES -# include -# endif -# define closesocket(sock) close(sock) -#endif /* !WINDOWS */ - -#include -#include -#include -#include -#include -#include -#include "udns.h" - -#ifndef EAFNOSUPPORT -# define EAFNOSUPPORT EINVAL -#endif -#ifndef MSG_DONTWAIT -# define MSG_DONTWAIT 0 -#endif - -struct dns_qlist { - struct dns_query *head, *tail; -}; - -struct dns_query { - struct dns_query *dnsq_next; /* double-linked list */ - struct dns_query *dnsq_prev; - unsigned dnsq_origdnl0; /* original query DN len w/o last 0 */ - unsigned dnsq_flags; /* control flags for this query */ - unsigned dnsq_servi; /* index of next server to try */ - unsigned dnsq_servwait; /* bitmask: servers left to wait */ - unsigned dnsq_servskip; /* bitmask: servers to skip */ - unsigned dnsq_servnEDNS0; /* bitmask: servers refusing EDNS0 */ - unsigned dnsq_try; /* number of tries made so far */ - dnscc_t *dnsq_nxtsrch; /* next search pointer @dnsc_srchbuf */ - time_t dnsq_deadline; /* when current try will expire */ - dns_parse_fn *dnsq_parse; /* parse: raw => application */ - dns_query_fn *dnsq_cbck; /* the callback to call when done */ - void *dnsq_cbdata; /* user data for the callback */ -#ifndef NDEBUG - struct dns_ctx *dnsq_ctx; /* the resolver context */ -#endif - /* char fields at the end to avoid padding */ - dnsc_t dnsq_id[2]; /* query ID */ - dnsc_t dnsq_typcls[4]; /* requested RR type+class */ - dnsc_t dnsq_dn[DNS_MAXDN+DNS_DNPAD]; /* the query DN +alignment */ -}; - -/* working with dns_query lists */ - -static __inline void qlist_init(struct dns_qlist *list) { - list->head = list->tail = NULL; -} - -static __inline void qlist_remove(struct dns_qlist *list, struct dns_query *q) { - if (q->dnsq_prev) q->dnsq_prev->dnsq_next = q->dnsq_next; - else list->head = q->dnsq_next; - if (q->dnsq_next) q->dnsq_next->dnsq_prev = q->dnsq_prev; - else list->tail = q->dnsq_prev; -} - -static __inline void -qlist_add_head(struct dns_qlist *list, struct dns_query *q) { - q->dnsq_next = list->head; - if (list->head) list->head->dnsq_prev = q; - else list->tail = q; - list->head = q; - q->dnsq_prev = NULL; -} - -static __inline void -qlist_insert_after(struct dns_qlist *list, - struct dns_query *q, struct dns_query *prev) { - if ((q->dnsq_prev = prev) != NULL) { - if ((q->dnsq_next = prev->dnsq_next) != NULL) - q->dnsq_next->dnsq_prev = q; - else - list->tail = q; - prev->dnsq_next = q; - } - else - qlist_add_head(list, q); -} - -union sockaddr_ns { - struct sockaddr sa; - struct sockaddr_in sin; -#ifdef HAVE_IPv6 - struct sockaddr_in6 sin6; -#endif -}; - -#define sin_eq(a,b) \ - ((a).sin_port == (b).sin_port && \ - (a).sin_addr.s_addr == (b).sin_addr.s_addr) -#define sin6_eq(a,b) \ - ((a).sin6_port == (b).sin6_port && \ - memcmp(&(a).sin6_addr, &(b).sin6_addr, sizeof(struct in6_addr)) == 0) - -struct dns_ctx { /* resolver context */ - /* settings */ - unsigned dnsc_flags; /* various flags */ - unsigned dnsc_timeout; /* timeout (base value) for queries */ - unsigned dnsc_ntries; /* number of retries */ - unsigned dnsc_ndots; /* ndots to assume absolute name */ - unsigned dnsc_port; /* default port (DNS_PORT) */ - unsigned dnsc_udpbuf; /* size of UDP buffer */ - /* array of nameserver addresses */ - union sockaddr_ns dnsc_serv[DNS_MAXSERV]; - unsigned dnsc_nserv; /* number of nameservers */ - unsigned dnsc_salen; /* length of socket addresses */ - dnsc_t dnsc_srchbuf[1024]; /* buffer for searchlist */ - dnsc_t *dnsc_srchend; /* current end of srchbuf */ - - dns_utm_fn *dnsc_utmfn; /* register/cancel timer events */ - void *dnsc_utmctx; /* user timer context for utmfn() */ - time_t dnsc_utmexp; /* when user timer expires */ - - dns_dbgfn *dnsc_udbgfn; /* debugging function */ - - /* dynamic data */ - struct udns_jranctx dnsc_jran; /* random number generator state */ - unsigned dnsc_nextid; /* next queue ID to use if !0 */ - int dnsc_udpsock; /* UDP socket */ - struct dns_qlist dnsc_qactive; /* active list sorted by deadline */ - int dnsc_nactive; /* number entries in dnsc_qactive */ - dnsc_t *dnsc_pbuf; /* packet buffer (udpbuf size) */ - int dnsc_qstatus; /* last query status value */ -}; - -static const struct { - const char *name; - enum dns_opt opt; - unsigned offset; - unsigned min, max; -} dns_opts[] = { -#define opt(name,opt,field,min,max) \ - {name,opt,offsetof(struct dns_ctx,field),min,max} - opt("retrans", DNS_OPT_TIMEOUT, dnsc_timeout, 1,300), - opt("timeout", DNS_OPT_TIMEOUT, dnsc_timeout, 1,300), - opt("retry", DNS_OPT_NTRIES, dnsc_ntries, 1,50), - opt("attempts", DNS_OPT_NTRIES, dnsc_ntries, 1,50), - opt("ndots", DNS_OPT_NDOTS, dnsc_ndots, 0,1000), - opt("port", DNS_OPT_PORT, dnsc_port, 1,0xffff), - opt("udpbuf", DNS_OPT_UDPSIZE, dnsc_udpbuf, DNS_MAXPACKET,65536), -#undef opt -}; -#define dns_ctxopt(ctx,idx) (*((unsigned*)(((char*)ctx)+dns_opts[idx].offset))) - -#define ISSPACE(x) (x == ' ' || x == '\t' || x == '\r' || x == '\n') - -struct dns_ctx dns_defctx; - -#define SETCTX(ctx) if (!ctx) ctx = &dns_defctx -#define SETCTXINITED(ctx) SETCTX(ctx); assert(CTXINITED(ctx)) -#define CTXINITED(ctx) (ctx->dnsc_flags & DNS_INITED) -#define SETCTXFRESH(ctx) SETCTXINITED(ctx); assert(!CTXOPEN(ctx)) -#define SETCTXINACTIVE(ctx) \ - SETCTXINITED(ctx); assert(!ctx->dnsc_nactive) -#define SETCTXOPEN(ctx) SETCTXINITED(ctx); assert(CTXOPEN(ctx)) -#define CTXOPEN(ctx) (ctx->dnsc_udpsock >= 0) - -#if defined(NDEBUG) || !defined(DEBUG) -#define dns_assert_ctx(ctx) -#else -static void dns_assert_ctx(const struct dns_ctx *ctx) { - int nactive = 0; - const struct dns_query *q; - for(q = ctx->dnsc_qactive.head; q; q = q->dnsq_next) { - assert(q->dnsq_ctx == ctx); - assert(q == (q->dnsq_next ? - q->dnsq_next->dnsq_prev : ctx->dnsc_qactive.tail)); - assert(q == (q->dnsq_prev ? - q->dnsq_prev->dnsq_next : ctx->dnsc_qactive.head)); - ++nactive; - } - assert(nactive == ctx->dnsc_nactive); -} -#endif - -enum { - DNS_INTERNAL = 0xffff, /* internal flags mask */ - DNS_INITED = 0x0001, /* the context is initialized */ - DNS_ASIS_DONE = 0x0002, /* search: skip the last as-is query */ - DNS_SEEN_NODATA = 0x0004, /* search: NODATA has been received */ -}; - -int dns_add_serv(struct dns_ctx *ctx, const char *serv) { - union sockaddr_ns *sns; - SETCTXFRESH(ctx); - if (!serv) - return (ctx->dnsc_nserv = 0); - if (ctx->dnsc_nserv >= DNS_MAXSERV) - return errno = ENFILE, -1; - sns = &ctx->dnsc_serv[ctx->dnsc_nserv]; - memset(sns, 0, sizeof(*sns)); - if (dns_pton(AF_INET, serv, &sns->sin.sin_addr) > 0) { - sns->sin.sin_family = AF_INET; - return ++ctx->dnsc_nserv; - } -#ifdef HAVE_IPv6 - if (dns_pton(AF_INET6, serv, &sns->sin6.sin6_addr) > 0) { - sns->sin6.sin6_family = AF_INET6; - return ++ctx->dnsc_nserv; - } -#endif - errno = EINVAL; - return -1; -} - -int dns_add_serv_s(struct dns_ctx *ctx, const struct sockaddr *sa) { - SETCTXFRESH(ctx); - if (!sa) - return (ctx->dnsc_nserv = 0); - if (ctx->dnsc_nserv >= DNS_MAXSERV) - return errno = ENFILE, -1; -#ifdef HAVE_IPv6 - else if (sa->sa_family == AF_INET6) - ctx->dnsc_serv[ctx->dnsc_nserv].sin6 = *(struct sockaddr_in6*)sa; -#endif - else if (sa->sa_family == AF_INET) - ctx->dnsc_serv[ctx->dnsc_nserv].sin = *(struct sockaddr_in*)sa; - else - return errno = EAFNOSUPPORT, -1; - return ++ctx->dnsc_nserv; -} - -int dns_set_opts(struct dns_ctx *ctx, const char *opts) { - unsigned i, v; - int err = 0; - SETCTXINACTIVE(ctx); - for(;;) { - while(ISSPACE(*opts)) ++opts; - if (!*opts) break; - for(i = 0; ; ++i) { - if (i >= sizeof(dns_opts)/sizeof(dns_opts[0])) { ++err; break; } - v = strlen(dns_opts[i].name); - if (strncmp(dns_opts[i].name, opts, v) != 0 || - (opts[v] != ':' && opts[v] != '=')) - continue; - opts += v + 1; - v = 0; - if (*opts < '0' || *opts > '9') { ++err; break; } - do v = v * 10 + (*opts++ - '0'); - while (*opts >= '0' && *opts <= '9'); - if (v < dns_opts[i].min) v = dns_opts[i].min; - if (v > dns_opts[i].max) v = dns_opts[i].max; - dns_ctxopt(ctx, i) = v; - break; - } - while(*opts && !ISSPACE(*opts)) ++opts; - } - return err; -} - -int dns_set_opt(struct dns_ctx *ctx, enum dns_opt opt, int val) { - int prev; - unsigned i; - SETCTXINACTIVE(ctx); - for(i = 0; i < sizeof(dns_opts)/sizeof(dns_opts[0]); ++i) { - if (dns_opts[i].opt != opt) continue; - prev = dns_ctxopt(ctx, i); - if (val >= 0) { - unsigned v = val; - if (v < dns_opts[i].min || v > dns_opts[i].max) { - errno = EINVAL; - return -1; - } - dns_ctxopt(ctx, i) = v; - } - return prev; - } - if (opt == DNS_OPT_FLAGS) { - prev = ctx->dnsc_flags & ~DNS_INTERNAL; - if (val >= 0) - ctx->dnsc_flags = - (ctx->dnsc_flags & DNS_INTERNAL) | (val & ~DNS_INTERNAL); - return prev; - } - errno = ENOSYS; - return -1; -} - -int dns_add_srch(struct dns_ctx *ctx, const char *srch) { - int dnl; - SETCTXINACTIVE(ctx); - if (!srch) { - memset(ctx->dnsc_srchbuf, 0, sizeof(ctx->dnsc_srchbuf)); - ctx->dnsc_srchend = ctx->dnsc_srchbuf; - return 0; - } - dnl = - sizeof(ctx->dnsc_srchbuf) - (ctx->dnsc_srchend - ctx->dnsc_srchbuf) - 1; - dnl = dns_sptodn(srch, ctx->dnsc_srchend, dnl); - if (dnl > 0) - ctx->dnsc_srchend += dnl; - ctx->dnsc_srchend[0] = '\0'; /* we ensure the list is always ends at . */ - if (dnl > 0) - return 0; - errno = EINVAL; - return -1; -} - -static void dns_drop_utm(struct dns_ctx *ctx) { - if (ctx->dnsc_utmfn) - ctx->dnsc_utmfn(NULL, -1, ctx->dnsc_utmctx); - ctx->dnsc_utmctx = NULL; - ctx->dnsc_utmexp = -1; -} - -static void -_dns_request_utm(struct dns_ctx *ctx, time_t now) { - struct dns_query *q; - time_t deadline; - int timeout; - q = ctx->dnsc_qactive.head; - if (!q) - deadline = -1, timeout = -1; - else if (!now || q->dnsq_deadline <= now) - deadline = 0, timeout = 0; - else - deadline = q->dnsq_deadline, timeout = (int)(deadline - now); - if (ctx->dnsc_utmexp == deadline) - return; - ctx->dnsc_utmfn(ctx, timeout, ctx->dnsc_utmctx); - ctx->dnsc_utmexp = deadline; -} - -static __inline void -dns_request_utm(struct dns_ctx *ctx, time_t now) { - if (ctx->dnsc_utmfn) - _dns_request_utm(ctx, now); -} - -void dns_set_dbgfn(struct dns_ctx *ctx, dns_dbgfn *dbgfn) { - SETCTXINITED(ctx); - ctx->dnsc_udbgfn = dbgfn; -} - -void -dns_set_tmcbck(struct dns_ctx *ctx, dns_utm_fn *fn, void *data) { - SETCTXINITED(ctx); - dns_drop_utm(ctx); - ctx->dnsc_utmfn = fn; - ctx->dnsc_utmctx = data; - if (CTXOPEN(ctx)) - dns_request_utm(ctx, 0); -} - -static unsigned dns_nonrandom_32(void) { -#ifdef WINDOWS - FILETIME ft; - GetSystemTimeAsFileTime(&ft); - return ft.dwLowDateTime; -#else - struct timeval tv; - gettimeofday(&tv, NULL); - return tv.tv_usec; -#endif -} - -/* This is historic deprecated API */ -UDNS_API unsigned dns_random16(void); -unsigned dns_random16(void) { - unsigned x = dns_nonrandom_32(); - return (x ^ (x >> 16)) & 0xffff; -} - -static void dns_init_rng(struct dns_ctx *ctx) { - udns_jraninit(&ctx->dnsc_jran, dns_nonrandom_32()); - ctx->dnsc_nextid = 0; -} - -void dns_close(struct dns_ctx *ctx) { - struct dns_query *q, *p; - SETCTX(ctx); - if (CTXINITED(ctx)) { - if (ctx->dnsc_udpsock >= 0) - closesocket(ctx->dnsc_udpsock); - ctx->dnsc_udpsock = -1; - if (ctx->dnsc_pbuf) - free(ctx->dnsc_pbuf); - ctx->dnsc_pbuf = NULL; - q = ctx->dnsc_qactive.head; - while((p = q) != NULL) { - q = q->dnsq_next; - free(p); - } - qlist_init(&ctx->dnsc_qactive); - ctx->dnsc_nactive = 0; - dns_drop_utm(ctx); - } -} - -void dns_reset(struct dns_ctx *ctx) { - SETCTX(ctx); - dns_close(ctx); - memset(ctx, 0, sizeof(*ctx)); - ctx->dnsc_timeout = 4; - ctx->dnsc_ntries = 3; - ctx->dnsc_ndots = 1; - ctx->dnsc_udpbuf = DNS_EDNS0PACKET; - ctx->dnsc_port = DNS_PORT; - ctx->dnsc_udpsock = -1; - ctx->dnsc_srchend = ctx->dnsc_srchbuf; - qlist_init(&ctx->dnsc_qactive); - dns_init_rng(ctx); - ctx->dnsc_flags = DNS_INITED; -} - -struct dns_ctx *dns_new(const struct dns_ctx *copy) { - struct dns_ctx *ctx; - SETCTXINITED(copy); - dns_assert_ctx(copy); - ctx = malloc(sizeof(*ctx)); - if (!ctx) - return NULL; - *ctx = *copy; - ctx->dnsc_udpsock = -1; - qlist_init(&ctx->dnsc_qactive); - ctx->dnsc_nactive = 0; - ctx->dnsc_pbuf = NULL; - ctx->dnsc_qstatus = 0; - ctx->dnsc_srchend = ctx->dnsc_srchbuf + - (copy->dnsc_srchend - copy->dnsc_srchbuf); - ctx->dnsc_utmfn = NULL; - ctx->dnsc_utmctx = NULL; - dns_init_rng(ctx); - return ctx; -} - -void dns_free(struct dns_ctx *ctx) { - assert(ctx != NULL && ctx != &dns_defctx); - dns_reset(ctx); - free(ctx); -} - -int dns_open(struct dns_ctx *ctx) { - int sock; - unsigned i; - int port; - union sockaddr_ns *sns; -#ifdef HAVE_IPv6 - unsigned have_inet6 = 0; -#endif - - SETCTXINITED(ctx); - assert(!CTXOPEN(ctx)); - - port = htons((unsigned short)ctx->dnsc_port); - /* ensure we have at least one server */ - if (!ctx->dnsc_nserv) { - sns = ctx->dnsc_serv; - sns->sin.sin_family = AF_INET; - sns->sin.sin_addr.s_addr = htonl(INADDR_LOOPBACK); - ctx->dnsc_nserv = 1; - } - - for (i = 0; i < ctx->dnsc_nserv; ++i) { - sns = &ctx->dnsc_serv[i]; - /* set port for each sockaddr */ -#ifdef HAVE_IPv6 - if (sns->sa.sa_family == AF_INET6) { - if (!sns->sin6.sin6_port) sns->sin6.sin6_port = (unsigned short)port; - ++have_inet6; - } - else -#endif - { - assert(sns->sa.sa_family == AF_INET); - if (!sns->sin.sin_port) sns->sin.sin_port = (unsigned short)port; - } - } - -#ifdef HAVE_IPv6 - if (have_inet6 && have_inet6 < ctx->dnsc_nserv) { - /* convert all IPv4 addresses to IPv6 V4MAPPED */ - struct sockaddr_in6 sin6; - memset(&sin6, 0, sizeof(sin6)); - sin6.sin6_family = AF_INET6; - /* V4MAPPED: ::ffff:1.2.3.4 */ - sin6.sin6_addr.s6_addr[10] = 0xff; - sin6.sin6_addr.s6_addr[11] = 0xff; - for(i = 0; i < ctx->dnsc_nserv; ++i) { - sns = &ctx->dnsc_serv[i]; - if (sns->sa.sa_family == AF_INET) { - sin6.sin6_port = sns->sin.sin_port; - memcpy(sin6.sin6_addr.s6_addr + 4*3, &sns->sin.sin_addr, 4); - sns->sin6 = sin6; - } - } - } - - ctx->dnsc_salen = have_inet6 ? - sizeof(struct sockaddr_in6) : sizeof(struct sockaddr_in); - - if (have_inet6) - sock = socket(PF_INET6, SOCK_DGRAM, IPPROTO_UDP); - else - sock = socket(PF_INET, SOCK_DGRAM, IPPROTO_UDP); -#else /* !HAVE_IPv6 */ - sock = socket(PF_INET, SOCK_DGRAM, IPPROTO_UDP); - ctx->dnsc_salen = sizeof(struct sockaddr_in); -#endif /* HAVE_IPv6 */ - - if (sock < 0) { - ctx->dnsc_qstatus = DNS_E_TEMPFAIL; - return -1; - } -#ifdef WINDOWS - { unsigned long on = 1; - if (ioctlsocket(sock, FIONBIO, &on) == SOCKET_ERROR) { - closesocket(sock); - ctx->dnsc_qstatus = DNS_E_TEMPFAIL; - return -1; - } - } -#else /* !WINDOWS */ - if (fcntl(sock, F_SETFL, fcntl(sock, F_GETFL) | O_NONBLOCK) < 0 || - fcntl(sock, F_SETFD, FD_CLOEXEC) < 0) { - closesocket(sock); - ctx->dnsc_qstatus = DNS_E_TEMPFAIL; - return -1; - } -#endif /* WINDOWS */ - /* allocate the packet buffer */ - if ((ctx->dnsc_pbuf = malloc(ctx->dnsc_udpbuf)) == NULL) { - closesocket(sock); - ctx->dnsc_qstatus = DNS_E_NOMEM; - errno = ENOMEM; - return -1; - } - - ctx->dnsc_udpsock = sock; - dns_request_utm(ctx, 0); - return sock; -} - -int dns_sock(const struct dns_ctx *ctx) { - SETCTXINITED(ctx); - return ctx->dnsc_udpsock; -} - -int dns_active(const struct dns_ctx *ctx) { - SETCTXINITED(ctx); - dns_assert_ctx(ctx); - return ctx->dnsc_nactive; -} - -int dns_status(const struct dns_ctx *ctx) { - SETCTX(ctx); - return ctx->dnsc_qstatus; -} -void dns_setstatus(struct dns_ctx *ctx, int status) { - SETCTX(ctx); - ctx->dnsc_qstatus = status; -} - -/* End the query: disconnect it from the active list, free it, - * and return the result to the caller. - */ -static void -dns_end_query(struct dns_ctx *ctx, struct dns_query *q, - int status, void *result) { - dns_query_fn *cbck = q->dnsq_cbck; - void *cbdata = q->dnsq_cbdata; - ctx->dnsc_qstatus = status; - assert((status < 0 && result == 0) || (status >= 0 && result != 0)); - assert(cbck != 0); /*XXX callback may be NULL */ - assert(ctx->dnsc_nactive > 0); - --ctx->dnsc_nactive; - qlist_remove(&ctx->dnsc_qactive, q); - /* force the query to be unconnected */ - /*memset(q, 0, sizeof(*q));*/ -#ifndef NDEBUG - q->dnsq_ctx = NULL; -#endif - free(q); - cbck(ctx, result, cbdata); -} - -#define DNS_DBG(ctx, code, sa, slen, pkt, plen) \ - do { \ - if (ctx->dnsc_udbgfn) \ - ctx->dnsc_udbgfn(code, (sa), slen, pkt, plen, 0, 0); \ - } while(0) -#define DNS_DBGQ(ctx, q, code, sa, slen, pkt, plen) \ - do { \ - if (ctx->dnsc_udbgfn) \ - ctx->dnsc_udbgfn(code, (sa), slen, pkt, plen, q, q->dnsq_cbdata); \ - } while(0) - -static void dns_newid(struct dns_ctx *ctx, struct dns_query *q) { - /* this is how we choose an identifier for a new query (qID). - * For now, it's just sequential number, incremented for every query, and - * thus obviously trivial to guess. - * There are two choices: - * a) use sequential numbers. It is plain insecure. In DNS, there are two - * places where random numbers are (or can) be used to increase security: - * random qID and random source port number. Without this randomness - * (udns uses fixed port for all queries), or when the randomness is weak, - * it's trivial to spoof query replies. With randomness however, it - * becomes a bit more difficult task. Too bad we only have 16 bits for - * our security, as qID is only two bytes. It isn't a security per se, - * to rely on those 16 bits - an attacker can just flood us with fake - * replies with all possible qIDs (only 65536 of them), and in this case, - * even if we'll use true random qIDs, we'll be in trouble (not protected - * against spoofing). Yes, this is only possible on a high-speed network - * (probably on the LAN only, since usually a border router for a LAN - * protects internal machines from packets with spoofed local addresses - * from outside, and usually a nameserver resides on LAN), but it's - * still very well possible to send us fake replies. - * In other words: there's nothing a DNS (stub) resolver can do against - * spoofing attacks, unless DNSSEC is in use, which helps here alot. - * Too bad that DNSSEC isn't widespread, so relying on it isn't an - * option in almost all cases... - * b) use random qID, based on some random-number generation mechanism. - * This way, we increase our protection a bit (see above - it's very weak - * still), but we also increase risk of qID reuse and matching late replies - * that comes to queries we've sent before against new queries. There are - * some more corner cases around that, as well - for example, normally, - * udns tries to find the query for a given reply by qID, *and* by - * verifying that the query DN and other parameters are also the same - * (so if the new query is against another domain name, old reply will - * be ignored automatically). But certain types of replies which we now - * handle - for example, FORMERR reply from servers which refuses to - * process EDNS0-enabled packets - comes without all the query parameters - * but the qID - so we're forced to use qID only when determining which - * query the given reply corresponds to. This makes us even more - * vulnerable to spoofing attacks, because an attacker don't even need to - * know which queries we perform to spoof the replies - he only needs to - * flood us with fake FORMERR "replies". - * - * That all to say: using sequential (or any other trivially guessable) - * numbers for qIDs is insecure, but the whole thing is inherently insecure - * as well, and this "extra weakness" that comes from weak qID choosing - * algorithm adds almost nothing to the underlying problem. - * - * It CAN NOT be made secure. Period. That's it. - * Unless we choose to implement DNSSEC, which is a whole different story. - * Forcing TCP mode makes it better, but who uses TCP for DNS anyway? - * (and it's hardly possible because of huge impact on the recursive - * nameservers). - * - * Note that ALL stub resolvers (again, unless they implement and enforce - * DNSSEC) suffers from this same problem. - * - * Here, I use a pseudo-random number generator for qIDs, instead of a - * simpler sequential IDs. This is _not_ more secure than sequential - * ID, but some found random IDs more enjoyeable for some reason. So - * here it goes. - */ - - /* Use random number and check if it's unique. - * If it's not, try again up to 5 times. - */ - unsigned loop; - dnsc_t c0, c1; - for(loop = 0; loop < 5; ++loop) { - const struct dns_query *c; - if (!ctx->dnsc_nextid) - ctx->dnsc_nextid = udns_jranval(&ctx->dnsc_jran); - c0 = ctx->dnsc_nextid & 0xff; - c1 = (ctx->dnsc_nextid >> 8) & 0xff; - ctx->dnsc_nextid >>= 16; - for(c = ctx->dnsc_qactive.head; c; c = c->dnsq_next) - if (c->dnsq_id[0] == c0 && c->dnsq_id[1] == c1) - break; /* found such entry, try again */ - if (!c) - break; - } - q->dnsq_id[0] = c0; q->dnsq_id[1] = c1; - - /* reset all parameters relevant for previous query lifetime */ - q->dnsq_try = 0; - q->dnsq_servi = 0; - /*XXX probably should keep dnsq_servnEDNS0 bits? - * See also comments in dns_ioevent() about FORMERR case */ - q->dnsq_servwait = q->dnsq_servskip = q->dnsq_servnEDNS0 = 0; -} - -/* Find next search suffix and fills in q->dnsq_dn. - * Return 0 if no more to try. */ -static int dns_next_srch(struct dns_ctx *ctx, struct dns_query *q) { - unsigned dnl; - - for(;;) { - if (q->dnsq_nxtsrch > ctx->dnsc_srchend) - return 0; - dnl = dns_dnlen(q->dnsq_nxtsrch); - if (dnl + q->dnsq_origdnl0 <= DNS_MAXDN && - (*q->dnsq_nxtsrch || !(q->dnsq_flags & DNS_ASIS_DONE))) - break; - q->dnsq_nxtsrch += dnl; - } - memcpy(q->dnsq_dn + q->dnsq_origdnl0, q->dnsq_nxtsrch, dnl); - if (!*q->dnsq_nxtsrch) - q->dnsq_flags |= DNS_ASIS_DONE; - q->dnsq_nxtsrch += dnl; - dns_newid(ctx, q); /* new ID for new qDN */ - return 1; -} - -/* find the server to try for current iteration. - * Note that current dnsq_servi may point to a server we should skip -- - * in that case advance to the next server. - * Return true if found, false if all tried. - */ -static int dns_find_serv(const struct dns_ctx *ctx, struct dns_query *q) { - while(q->dnsq_servi < ctx->dnsc_nserv) { - if (!(q->dnsq_servskip & (1 << q->dnsq_servi))) - return 1; - ++q->dnsq_servi; - } - return 0; -} - -/* format and send the query to a given server. - * In case of network problem (sendto() fails), return -1, - * else return 0. - */ -static int -dns_send_this(struct dns_ctx *ctx, struct dns_query *q, - unsigned servi, time_t now) { - unsigned qlen; - unsigned tries; - - { /* format the query buffer */ - dnsc_t *p = ctx->dnsc_pbuf; - memset(p, 0, DNS_HSIZE); - if (!(q->dnsq_flags & DNS_NORD)) p[DNS_H_F1] |= DNS_HF1_RD; - if (q->dnsq_flags & DNS_AAONLY) p[DNS_H_F1] |= DNS_HF1_AA; - if (q->dnsq_flags & DNS_SET_CD) p[DNS_H_F2] |= DNS_HF2_CD; - p[DNS_H_QDCNT2] = 1; - memcpy(p + DNS_H_QID, q->dnsq_id, 2); - p = dns_payload(p); - /* copy query dn */ - p += dns_dntodn(q->dnsq_dn, p, DNS_MAXDN); - /* query type and class */ - memcpy(p, q->dnsq_typcls, 4); p += 4; - /* add EDNS0 record. DO flag requires it */ - if (q->dnsq_flags & DNS_SET_DO || - (ctx->dnsc_udpbuf > DNS_MAXPACKET && - !(q->dnsq_servnEDNS0 & (1 << servi)))) { - *p++ = 0; /* empty (root) DN */ - p = dns_put16(p, DNS_T_OPT); - p = dns_put16(p, ctx->dnsc_udpbuf); - /* EDNS0 RCODE & VERSION; rest of the TTL field; RDLEN */ - memset(p, 0, 2+2+2); - if (q->dnsq_flags & DNS_SET_DO) p[2] |= DNS_EF1_DO; - p += 2+2+2; - ctx->dnsc_pbuf[DNS_H_ARCNT2] = 1; - } - qlen = p - ctx->dnsc_pbuf; - assert(qlen <= ctx->dnsc_udpbuf); - } - - /* send the query */ - tries = 10; - while (sendto(ctx->dnsc_udpsock, (void*)ctx->dnsc_pbuf, qlen, 0, - &ctx->dnsc_serv[servi].sa, ctx->dnsc_salen) < 0) { - /*XXX just ignore the sendto() error for now and try again. - * In the future, it may be possible to retrieve the error code - * and find which operation/query failed. - *XXX try the next server too? (if ENETUNREACH is returned immediately) - */ - if (--tries) continue; - /* if we can't send the query, fail it. */ - dns_end_query(ctx, q, DNS_E_TEMPFAIL, 0); - return -1; - } - DNS_DBGQ(ctx, q, 1, - &ctx->dnsc_serv[servi].sa, sizeof(union sockaddr_ns), - ctx->dnsc_pbuf, qlen); - q->dnsq_servwait |= 1 << servi; /* expect reply from this ns */ - - q->dnsq_deadline = now + - (dns_find_serv(ctx, q) ? 1 : ctx->dnsc_timeout << q->dnsq_try); - - /* move the query to the proper place, according to the new deadline */ - qlist_remove(&ctx->dnsc_qactive, q); - { /* insert from the tail */ - struct dns_query *p; - for(p = ctx->dnsc_qactive.tail; p; p = p->dnsq_prev) - if (p->dnsq_deadline <= q->dnsq_deadline) - break; - qlist_insert_after(&ctx->dnsc_qactive, q, p); - } - - return 0; -} - -/* send the query out using next available server - * and add it to the active list, or, if no servers available, - * end it. - */ -static void -dns_send(struct dns_ctx *ctx, struct dns_query *q, time_t now) { - - /* if we can't send the query, return TEMPFAIL even when searching: - * we can't be sure whenever the name we tried to search exists or not, - * so don't continue searching, or we may find the wrong name. */ - - if (!dns_find_serv(ctx, q)) { - /* no more servers in this iteration. Try the next cycle */ - q->dnsq_servi = 0; /* reset */ - q->dnsq_try++; /* next try */ - if (q->dnsq_try >= ctx->dnsc_ntries || - !dns_find_serv(ctx, q)) { - /* no more servers and tries, fail the query */ - /* return TEMPFAIL even when searching: no more tries for this - * searchlist, and no single definitive reply (handled in dns_ioevent() - * in NOERROR or NXDOMAIN cases) => all nameservers failed to process - * current search list element, so we don't know whenever the name exists. - */ - dns_end_query(ctx, q, DNS_E_TEMPFAIL, 0); - return; - } - } - - dns_send_this(ctx, q, q->dnsq_servi++, now); -} - -static void dns_dummy_cb(struct dns_ctx *ctx, void *result, void *data) { - if (result) free(result); - data = ctx = 0; /* used */ -} - -/* The (only, main, real) query submission routine. - * Allocate new query structure, initialize it, check validity of - * parameters, and add it to the head of the active list, without - * trying to send it (to be picked up on next event). - * Error return (without calling the callback routine) - - * no memory or wrong parameters. - *XXX The `no memory' case probably should go to the callback anyway... - */ -struct dns_query * -dns_submit_dn(struct dns_ctx *ctx, - dnscc_t *dn, int qcls, int qtyp, int flags, - dns_parse_fn *parse, dns_query_fn *cbck, void *data) { - struct dns_query *q; - SETCTXOPEN(ctx); - dns_assert_ctx(ctx); - - q = calloc(sizeof(*q), 1); - if (!q) { - ctx->dnsc_qstatus = DNS_E_NOMEM; - return NULL; - } - -#ifndef NDEBUG - q->dnsq_ctx = ctx; -#endif - q->dnsq_parse = parse; - q->dnsq_cbck = cbck ? cbck : dns_dummy_cb; - q->dnsq_cbdata = data; - - q->dnsq_origdnl0 = dns_dntodn(dn, q->dnsq_dn, sizeof(q->dnsq_dn)); - assert(q->dnsq_origdnl0 > 0); - --q->dnsq_origdnl0; /* w/o the trailing 0 */ - dns_put16(q->dnsq_typcls+0, qtyp); - dns_put16(q->dnsq_typcls+2, qcls); - q->dnsq_flags = (flags | ctx->dnsc_flags) & ~DNS_INTERNAL; - - if (flags & DNS_NOSRCH || - dns_dnlabels(q->dnsq_dn) > ctx->dnsc_ndots) { - q->dnsq_nxtsrch = flags & DNS_NOSRCH ? - ctx->dnsc_srchend /* end of the search list if no search requested */ : - ctx->dnsc_srchbuf /* beginning of the list, but try as-is first */; - q->dnsq_flags |= DNS_ASIS_DONE; - dns_newid(ctx, q); - } - else { - q->dnsq_nxtsrch = ctx->dnsc_srchbuf; - dns_next_srch(ctx, q); - } - - /* q->dnsq_deadline is set to 0 (calloc above): the new query is - * "already expired" when first inserted into queue, so it's safe - * to insert it into the head of the list. Next call to dns_timeouts() - * will actually send it. - */ - qlist_add_head(&ctx->dnsc_qactive, q); - ++ctx->dnsc_nactive; - dns_request_utm(ctx, 0); - - return q; -} - -struct dns_query * -dns_submit_p(struct dns_ctx *ctx, - const char *name, int qcls, int qtyp, int flags, - dns_parse_fn *parse, dns_query_fn *cbck, void *data) { - int isabs; - SETCTXOPEN(ctx); - if (dns_ptodn(name, 0, ctx->dnsc_pbuf, DNS_MAXDN, &isabs) <= 0) { - ctx->dnsc_qstatus = DNS_E_BADQUERY; - return NULL; - } - if (isabs) - flags |= DNS_NOSRCH; - return - dns_submit_dn(ctx, ctx->dnsc_pbuf, qcls, qtyp, flags, parse, cbck, data); -} - -/* process readable fd condition. - * To be usable in edge-triggered environment, the routine - * should consume all input so it should loop over. - * Note it isn't really necessary to loop here, because - * an application may perform the loop just fine by it's own, - * but in this case we should return some sensitive result, - * to indicate when to stop calling and error conditions. - * Note also we may encounter all sorts of recvfrom() - * errors which aren't fatal, and at the same time we may - * loop forever if an error IS fatal. - */ -void dns_ioevent(struct dns_ctx *ctx, time_t now) { - int r; - unsigned servi; - struct dns_query *q; - dnsc_t *pbuf; - dnscc_t *pend, *pcur; - void *result; - union sockaddr_ns sns; - socklen_t slen; - - SETCTX(ctx); - if (!CTXOPEN(ctx)) - return; - dns_assert_ctx(ctx); - pbuf = ctx->dnsc_pbuf; - - if (!now) now = time(NULL); - -again: /* receive the reply */ - - slen = sizeof(sns); - r = recvfrom(ctx->dnsc_udpsock, (void*)pbuf, ctx->dnsc_udpbuf, - MSG_DONTWAIT, &sns.sa, &slen); - if (r < 0) { - /*XXX just ignore recvfrom() errors for now. - * in the future it may be possible to determine which - * query failed and requeue it. - * Note there may be various error conditions, triggered - * by both local problems and remote problems. It isn't - * quite trivial to determine whenever an error is local - * or remote. On local errors, we should stop, while - * remote errors should be ignored (for now anyway). - */ -#ifdef WINDOWS - if (WSAGetLastError() == WSAEWOULDBLOCK) -#else - if (errno == EAGAIN) -#endif - { - dns_request_utm(ctx, now); - return; - } - goto again; - } - - pend = pbuf + r; - pcur = dns_payload(pbuf); - - /* check reply header */ - if (pcur > pend || dns_numqd(pbuf) > 1 || dns_opcode(pbuf) != 0) { - DNS_DBG(ctx, -1/*bad reply*/, &sns.sa, slen, pbuf, r); - goto again; - } - - /* find the matching query, by qID */ - for (q = ctx->dnsc_qactive.head; ; q = q->dnsq_next) { - if (!q) { - /* no more requests: old reply? */ - DNS_DBG(ctx, -5/*no matching query*/, &sns.sa, slen, pbuf, r); - goto again; - } - if (pbuf[DNS_H_QID1] == q->dnsq_id[0] && - pbuf[DNS_H_QID2] == q->dnsq_id[1]) - break; - } - - /* if we have numqd, compare with our query qDN */ - if (dns_numqd(pbuf)) { - /* decode the qDN */ - dnsc_t dn[DNS_MAXDN]; - if (dns_getdn(pbuf, &pcur, pend, dn, sizeof(dn)) < 0 || - pcur + 4 > pend) { - DNS_DBG(ctx, -1/*bad reply*/, &sns.sa, slen, pbuf, r); - goto again; - } - if (!dns_dnequal(dn, q->dnsq_dn) || - memcmp(pcur, q->dnsq_typcls, 4) != 0) { - /* not this query */ - DNS_DBG(ctx, -5/*no matching query*/, &sns.sa, slen, pbuf, r); - goto again; - } - /* here, query match, and pcur points past qDN in query section in pbuf */ - } - /* if no numqd, we only allow FORMERR rcode */ - else if (dns_rcode(pbuf) != DNS_R_FORMERR) { - /* treat it as bad reply if !FORMERR */ - DNS_DBG(ctx, -1/*bad reply*/, &sns.sa, slen, pbuf, r); - goto again; - } - else { - /* else it's FORMERR, handled below */ - } - - /* find server */ -#ifdef HAVE_IPv6 - if (sns.sa.sa_family == AF_INET6 && slen >= sizeof(sns.sin6)) { - for(servi = 0; servi < ctx->dnsc_nserv; ++servi) - if (sin6_eq(ctx->dnsc_serv[servi].sin6, sns.sin6)) - break; - } - else -#endif - if (sns.sa.sa_family == AF_INET && slen >= sizeof(sns.sin)) { - for(servi = 0; servi < ctx->dnsc_nserv; ++servi) - if (sin_eq(ctx->dnsc_serv[servi].sin, sns.sin)) - break; - } - else - servi = ctx->dnsc_nserv; - - /* check if we expect reply from this server. - * Note we can receive reply from first try if we're already at next */ - if (!(q->dnsq_servwait & (1 << servi))) { /* if ever asked this NS */ - DNS_DBG(ctx, -2/*wrong server*/, &sns.sa, slen, pbuf, r); - goto again; - } - - /* we got (some) reply for our query */ - - DNS_DBGQ(ctx, q, 0, &sns.sa, slen, pbuf, r); - q->dnsq_servwait &= ~(1 << servi); /* don't expect reply from this serv */ - - /* process the RCODE */ - switch(dns_rcode(pbuf)) { - - case DNS_R_NOERROR: - if (dns_tc(pbuf)) { - /* possible truncation. We can't deal with it. */ - /*XXX for now, treat TC bit the same as SERVFAIL. - * It is possible to: - * a) try to decode the reply - may be ANSWER section is ok; - * b) check if server understands EDNS0, and if it is, and - * answer still don't fit, end query. - */ - break; - } - if (!dns_numan(pbuf)) { /* no data of requested type */ - if (dns_next_srch(ctx, q)) { - /* if we're searching, try next searchlist element, - * but remember NODATA reply. */ - q->dnsq_flags |= DNS_SEEN_NODATA; - dns_send(ctx, q, now); - } - else - /* else - nothing to search any more - finish the query. - * It will be NODATA since we've seen a NODATA reply. */ - dns_end_query(ctx, q, DNS_E_NODATA, 0); - } - /* we've got a positive reply here */ - else if (q->dnsq_parse) { - /* if we have parsing routine, call it and return whatever it returned */ - /* don't try to re-search if NODATA here. For example, - * if we asked for A but only received CNAME. Unless we'll - * someday do recursive queries. And that's problematic too, since - * we may be dealing with specific AA-only nameservers for a given - * domain, but CNAME points elsewhere... - */ - r = q->dnsq_parse(q->dnsq_dn, pbuf, pcur, pend, &result); - dns_end_query(ctx, q, r, r < 0 ? NULL : result); - } - /* else just malloc+copy the raw DNS reply */ - else if ((result = malloc(r)) == NULL) - dns_end_query(ctx, q, DNS_E_NOMEM, NULL); - else { - memcpy(result, pbuf, r); - dns_end_query(ctx, q, r, result); - } - goto again; - - case DNS_R_NXDOMAIN: /* Non-existing domain. */ - if (dns_next_srch(ctx, q)) - /* more search entries exists, try them. */ - dns_send(ctx, q, now); - else - /* nothing to search anymore. End the query, returning either NODATA - * if we've seen it before, or NXDOMAIN if not. */ - dns_end_query(ctx, q, - q->dnsq_flags & DNS_SEEN_NODATA ? DNS_E_NODATA : DNS_E_NXDOMAIN, 0); - goto again; - - case DNS_R_FORMERR: - case DNS_R_NOTIMPL: - /* for FORMERR and NOTIMPL rcodes, if we tried EDNS0-enabled query, - * try w/o EDNS0. */ - if (ctx->dnsc_udpbuf > DNS_MAXPACKET && - !(q->dnsq_servnEDNS0 & (1 << servi))) { - /* we always trying EDNS0 first if enabled, and retry a given query - * if not available. Maybe it's better to remember inavailability of - * EDNS0 in ctx as a per-NS flag, and never try again for this NS. - * For long-running applications.. maybe they will change the nameserver - * while we're running? :) Also, since FORMERR is the only rcode we - * allow to be header-only, and in this case the only check we do to - * find a query it belongs to is qID (not qDN+qCLS+qTYP), it's much - * easier to spoof and to force us to perform non-EDNS0 queries only... - */ - q->dnsq_servnEDNS0 |= 1 << servi; - dns_send_this(ctx, q, servi, now); - goto again; - } - /* else we handle it the same as SERVFAIL etc */ - - case DNS_R_SERVFAIL: - case DNS_R_REFUSED: - /* for these rcodes, advance this request - * to the next server and reschedule */ - default: /* unknown rcode? hmmm... */ - break; - } - - /* here, we received unexpected reply */ - q->dnsq_servskip |= (1 << servi); /* don't retry this server */ - - /* we don't expect replies from this server anymore. - * But there may be other servers. Some may be still processing our - * query, and some may be left to try. - * We just ignore this reply and wait a bit more if some NSes haven't - * replied yet (dnsq_servwait != 0), and let the situation to be handled - * on next event processing. Timeout for this query is set correctly, - * if not taking into account the one-second difference - we can try - * next server in the same iteration sooner. - */ - - /* try next server */ - if (!q->dnsq_servwait) { - /* next retry: maybe some other servers will reply next time. - * dns_send() will end the query for us if no more servers to try. - * Note we can't continue with the next searchlist element here: - * we don't know if the current qdn exists or not, there's no definitive - * answer yet (which is seen in cases above). - *XXX standard resolver also tries as-is query in case all nameservers - * failed to process our query and if not tried before. We don't do it. - */ - dns_send(ctx, q, now); - } - else { - /* else don't do anything - not all servers replied yet */ - } - goto again; - -} - -/* handle all timeouts */ -int dns_timeouts(struct dns_ctx *ctx, int maxwait, time_t now) { - /* this is a hot routine */ - struct dns_query *q; - - SETCTX(ctx); - dns_assert_ctx(ctx); - - /* Pick up first entry from query list. - * If its deadline has passed, (re)send it - * (dns_send() will move it next in the list). - * If not, this is the query which determines the closest deadline. - */ - - q = ctx->dnsc_qactive.head; - if (!q) - return maxwait; - if (!now) - now = time(NULL); - do { - if (q->dnsq_deadline > now) { /* first non-expired query */ - int w = (int)(q->dnsq_deadline - now); - if (maxwait < 0 || maxwait > w) - maxwait = w; - break; - } - else { - /* process expired deadline */ - dns_send(ctx, q, now); - } - } while((q = ctx->dnsc_qactive.head) != NULL); - - dns_request_utm(ctx, now); /* update timer with new deadline */ - return maxwait; -} - -struct dns_resolve_data { - int dnsrd_done; - void *dnsrd_result; -}; - -static void dns_resolve_cb(struct dns_ctx *ctx, void *result, void *data) { - struct dns_resolve_data *d = data; - d->dnsrd_result = result; - d->dnsrd_done = 1; - ctx = ctx; -} - -void *dns_resolve(struct dns_ctx *ctx, struct dns_query *q) { - time_t now; - struct dns_resolve_data d; - int n; - SETCTXOPEN(ctx); - - if (!q) - return NULL; - - assert(ctx == q->dnsq_ctx); - dns_assert_ctx(ctx); - /* do not allow re-resolving syncronous queries */ - assert(q->dnsq_cbck != dns_resolve_cb && "can't resolve syncronous query"); - if (q->dnsq_cbck == dns_resolve_cb) { - ctx->dnsc_qstatus = DNS_E_BADQUERY; - return NULL; - } - q->dnsq_cbck = dns_resolve_cb; - q->dnsq_cbdata = &d; - d.dnsrd_done = 0; - - now = time(NULL); - while(!d.dnsrd_done && (n = dns_timeouts(ctx, -1, now)) >= 0) { -#ifdef HAVE_POLL - struct pollfd pfd; - pfd.fd = ctx->dnsc_udpsock; - pfd.events = POLLIN; - n = poll(&pfd, 1, n * 1000); -#else - fd_set rfd; - struct timeval tv; - FD_ZERO(&rfd); - FD_SET(ctx->dnsc_udpsock, &rfd); - tv.tv_sec = n; tv.tv_usec = 0; - n = select(ctx->dnsc_udpsock + 1, &rfd, NULL, NULL, &tv); -#endif - now = time(NULL); - if (n > 0) - dns_ioevent(ctx, now); - } - - return d.dnsrd_result; -} - -void *dns_resolve_dn(struct dns_ctx *ctx, - dnscc_t *dn, int qcls, int qtyp, int flags, - dns_parse_fn *parse) { - return - dns_resolve(ctx, - dns_submit_dn(ctx, dn, qcls, qtyp, flags, parse, NULL, NULL)); -} - -void *dns_resolve_p(struct dns_ctx *ctx, - const char *name, int qcls, int qtyp, int flags, - dns_parse_fn *parse) { - return - dns_resolve(ctx, - dns_submit_p(ctx, name, qcls, qtyp, flags, parse, NULL, NULL)); -} - -int dns_cancel(struct dns_ctx *ctx, struct dns_query *q) { - SETCTX(ctx); - dns_assert_ctx(ctx); - assert(q->dnsq_ctx == ctx); - /* do not allow cancelling syncronous queries */ - assert(q->dnsq_cbck != dns_resolve_cb && "can't cancel syncronous query"); - if (q->dnsq_cbck == dns_resolve_cb) - return (ctx->dnsc_qstatus = DNS_E_BADQUERY); - qlist_remove(&ctx->dnsc_qactive, q); - --ctx->dnsc_nactive; - dns_request_utm(ctx, 0); - return 0; -} - diff --git a/contrib/udns/udns_rr_a.c b/contrib/udns/udns_rr_a.c deleted file mode 100644 index 4fdcbf9bc1c..00000000000 --- a/contrib/udns/udns_rr_a.c +++ /dev/null @@ -1,123 +0,0 @@ -/* udns_rr_a.c - parse/query A/AAAA IN records - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#include -#include -#include -#ifndef WINDOWS -# include -# include -#endif -#include "udns.h" - -/* here, we use common routine to parse both IPv4 and IPv6 addresses. - */ - -/* this structure should match dns_rr_a[46] */ -struct dns_rr_a { - dns_rr_common(dnsa); - unsigned char *dnsa_addr; -}; - -static int -dns_parse_a(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, - void **result, unsigned dsize) { - struct dns_rr_a *ret; - struct dns_parse p; - struct dns_rr rr; - int r; - - /* first, validate and count number of addresses */ - dns_initparse(&p, qdn, pkt, cur, end); - while((r = dns_nextrr(&p, &rr)) > 0) - if (rr.dnsrr_dsz != dsize) - return DNS_E_PROTOCOL; - if (r < 0) - return DNS_E_PROTOCOL; - else if (!p.dnsp_nrr) - return DNS_E_NODATA; - - ret = malloc(sizeof(*ret) + dsize * p.dnsp_nrr + dns_stdrr_size(&p)); - if (!ret) - return DNS_E_NOMEM; - - ret->dnsa_nrr = p.dnsp_nrr; - ret->dnsa_addr = (unsigned char*)(ret+1); - - /* copy the RRs */ - for (dns_rewind(&p, qdn), r = 0; dns_nextrr(&p, &rr); ++r) - memcpy(ret->dnsa_addr + dsize * r, rr.dnsrr_dptr, dsize); - - dns_stdrr_finish((struct dns_rr_null *)ret, - (char *)(ret->dnsa_addr + dsize * p.dnsp_nrr), &p); - *result = ret; - return 0; -} - -int -dns_parse_a4(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, - void **result) { -#ifdef AF_INET - assert(sizeof(struct in_addr) == 4); -#endif - assert(dns_get16(cur+2) == DNS_C_IN && dns_get16(cur+0) == DNS_T_A); - return dns_parse_a(qdn, pkt, cur, end, result, 4); -} - -struct dns_query * -dns_submit_a4(struct dns_ctx *ctx, const char *name, int flags, - dns_query_a4_fn *cbck, void *data) { - return - dns_submit_p(ctx, name, DNS_C_IN, DNS_T_A, flags, - dns_parse_a4, (dns_query_fn*)cbck, data); -} - -struct dns_rr_a4 * -dns_resolve_a4(struct dns_ctx *ctx, const char *name, int flags) { - return (struct dns_rr_a4 *) - dns_resolve_p(ctx, name, DNS_C_IN, DNS_T_A, flags, dns_parse_a4); -} - -int -dns_parse_a6(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, - void **result) { -#ifdef AF_INET6 - assert(sizeof(struct in6_addr) == 16); -#endif - assert(dns_get16(cur+2) == DNS_C_IN && dns_get16(cur+0) == DNS_T_AAAA); - return dns_parse_a(qdn, pkt, cur, end, result, 16); -} - -struct dns_query * -dns_submit_a6(struct dns_ctx *ctx, const char *name, int flags, - dns_query_a6_fn *cbck, void *data) { - return - dns_submit_p(ctx, name, DNS_C_IN, DNS_T_AAAA, flags, - dns_parse_a6, (dns_query_fn*)cbck, data); -} - -struct dns_rr_a6 * -dns_resolve_a6(struct dns_ctx *ctx, const char *name, int flags) { - return (struct dns_rr_a6 *) - dns_resolve_p(ctx, name, DNS_C_IN, DNS_T_AAAA, flags, dns_parse_a6); -} diff --git a/contrib/udns/udns_rr_mx.c b/contrib/udns/udns_rr_mx.c deleted file mode 100644 index 0904e017177..00000000000 --- a/contrib/udns/udns_rr_mx.c +++ /dev/null @@ -1,91 +0,0 @@ -/* udns_rr_mx.c - parse/query MX IN records - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#include -#include -#include -#include "udns.h" - -int -dns_parse_mx(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, - void **result) { - struct dns_rr_mx *ret; - struct dns_parse p; - struct dns_rr rr; - int r, l; - char *sp; - dnsc_t mx[DNS_MAXDN]; - - assert(dns_get16(cur+2) == DNS_C_IN && dns_get16(cur+0) == DNS_T_MX); - - /* first, validate the answer and count size of the result */ - l = 0; - dns_initparse(&p, qdn, pkt, cur, end); - while((r = dns_nextrr(&p, &rr)) > 0) { - cur = rr.dnsrr_dptr + 2; - r = dns_getdn(pkt, &cur, end, mx, sizeof(mx)); - if (r <= 0 || cur != rr.dnsrr_dend) - return DNS_E_PROTOCOL; - l += dns_dntop_size(mx); - } - if (r < 0) - return DNS_E_PROTOCOL; - if (!p.dnsp_nrr) - return DNS_E_NODATA; - - /* next, allocate and set up result */ - l += dns_stdrr_size(&p); - ret = malloc(sizeof(*ret) + sizeof(struct dns_mx) * p.dnsp_nrr + l); - if (!ret) - return DNS_E_NOMEM; - ret->dnsmx_nrr = p.dnsp_nrr; - ret->dnsmx_mx = (struct dns_mx *)(ret+1); - - /* and 3rd, fill in result, finally */ - sp = (char*)(ret->dnsmx_mx + p.dnsp_nrr); - for (dns_rewind(&p, qdn), r = 0; dns_nextrr(&p, &rr); ++r) { - ret->dnsmx_mx[r].name = sp; - cur = rr.dnsrr_dptr; - ret->dnsmx_mx[r].priority = dns_get16(cur); - cur += 2; - dns_getdn(pkt, &cur, end, mx, sizeof(mx)); - sp += dns_dntop(mx, sp, DNS_MAXNAME); - } - dns_stdrr_finish((struct dns_rr_null *)ret, sp, &p); - *result = ret; - return 0; -} - -struct dns_query * -dns_submit_mx(struct dns_ctx *ctx, const char *name, int flags, - dns_query_mx_fn *cbck, void *data) { - return - dns_submit_p(ctx, name, DNS_C_IN, DNS_T_MX, flags, - dns_parse_mx, (dns_query_fn *)cbck, data); -} - -struct dns_rr_mx * -dns_resolve_mx(struct dns_ctx *ctx, const char *name, int flags) { - return (struct dns_rr_mx *) - dns_resolve_p(ctx, name, DNS_C_IN, DNS_T_MX, flags, dns_parse_mx); -} diff --git a/contrib/udns/udns_rr_naptr.c b/contrib/udns/udns_rr_naptr.c deleted file mode 100644 index da30069db62..00000000000 --- a/contrib/udns/udns_rr_naptr.c +++ /dev/null @@ -1,128 +0,0 @@ -/* udns_rr_naptr.c - parse/query NAPTR IN records - - Copyright (C) 2005 Michael Tokarev - Copyright (C) 2006 Mikael Magnusson - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#include -#include -#include -#include "udns.h" - -/* Get a single string for NAPTR record, pretty much like a DN label. - * String length is in first byte in *cur, so it can't be >255. - */ -static int dns_getstr(dnscc_t **cur, dnscc_t *ep, char *buf) -{ - unsigned l; - dnscc_t *cp = *cur; - - l = *cp++; - if (cp + l > ep) - return DNS_E_PROTOCOL; - if (buf) { - memcpy(buf, cp, l); - buf[l] = '\0'; - } - cp += l; - - *cur = cp; - return l + 1; -} - -int -dns_parse_naptr(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, - void **result) { - struct dns_rr_naptr *ret; - struct dns_parse p; - struct dns_rr rr; - int r, l; - char *sp; - dnsc_t dn[DNS_MAXDN]; - - assert(dns_get16(cur+2) == DNS_C_IN && dns_get16(cur+0) == DNS_T_NAPTR); - - /* first, validate the answer and count size of the result */ - l = 0; - dns_initparse(&p, qdn, pkt, cur, end); - while((r = dns_nextrr(&p, &rr)) > 0) { - int i; - dnscc_t *ep = rr.dnsrr_dend; - - /* first 4 bytes: order & preference */ - cur = rr.dnsrr_dptr + 4; - - /* flags, services and regexp */ - for (i = 0; i < 3; i++) { - r = dns_getstr(&cur, ep, NULL); - if (r < 0) - return r; - l += r; - } - /* replacement */ - r = dns_getdn(pkt, &cur, end, dn, sizeof(dn)); - if (r <= 0 || cur != rr.dnsrr_dend) - return DNS_E_PROTOCOL; - l += dns_dntop_size(dn); - } - if (r < 0) - return DNS_E_PROTOCOL; - if (!p.dnsp_nrr) - return DNS_E_NODATA; - - /* next, allocate and set up result */ - l += dns_stdrr_size(&p); - ret = malloc(sizeof(*ret) + sizeof(struct dns_naptr) * p.dnsp_nrr + l); - if (!ret) - return DNS_E_NOMEM; - ret->dnsnaptr_nrr = p.dnsp_nrr; - ret->dnsnaptr_naptr = (struct dns_naptr *)(ret+1); - - /* and 3rd, fill in result, finally */ - sp = (char*)(&ret->dnsnaptr_naptr[p.dnsp_nrr]); - for (dns_rewind(&p, qdn), r = 0; dns_nextrr(&p, &rr); ++r) { - cur = rr.dnsrr_dptr; - ret->dnsnaptr_naptr[r].order = dns_get16(cur); cur += 2; - ret->dnsnaptr_naptr[r].preference = dns_get16(cur); cur += 2; - sp += dns_getstr(&cur, end, (ret->dnsnaptr_naptr[r].flags = sp)); - sp += dns_getstr(&cur, end, (ret->dnsnaptr_naptr[r].service = sp)); - sp += dns_getstr(&cur, end, (ret->dnsnaptr_naptr[r].regexp = sp)); - dns_getdn(pkt, &cur, end, dn, sizeof(dn)); - sp += dns_dntop(dn, (ret->dnsnaptr_naptr[r].replacement = sp), DNS_MAXNAME); - } - dns_stdrr_finish((struct dns_rr_null *)ret, sp, &p); - *result = ret; - return 0; -} - -struct dns_query * -dns_submit_naptr(struct dns_ctx *ctx, const char *name, int flags, - dns_query_naptr_fn *cbck, void *data) { - return - dns_submit_p(ctx, name, DNS_C_IN, DNS_T_NAPTR, flags, - dns_parse_naptr, (dns_query_fn *)cbck, data); -} - -struct dns_rr_naptr * -dns_resolve_naptr(struct dns_ctx *ctx, const char *name, int flags) { - return (struct dns_rr_naptr *) - dns_resolve_p(ctx, name, DNS_C_IN, DNS_T_NAPTR, flags, dns_parse_naptr); -} diff --git a/contrib/udns/udns_rr_ptr.c b/contrib/udns/udns_rr_ptr.c deleted file mode 100644 index 1f682aebc0f..00000000000 --- a/contrib/udns/udns_rr_ptr.c +++ /dev/null @@ -1,109 +0,0 @@ -/* udns_rr_ptr.c - parse/query PTR records - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#include -#include -#include "udns.h" - -int -dns_parse_ptr(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, - void **result) { - struct dns_rr_ptr *ret; - struct dns_parse p; - struct dns_rr rr; - int r, l, c; - char *sp; - dnsc_t ptr[DNS_MAXDN]; - - assert(dns_get16(cur+2) == DNS_C_IN && dns_get16(cur+0) == DNS_T_PTR); - - /* first, validate the answer and count size of the result */ - l = c = 0; - dns_initparse(&p, qdn, pkt, cur, end); - while((r = dns_nextrr(&p, &rr)) > 0) { - cur = rr.dnsrr_dptr; - r = dns_getdn(pkt, &cur, end, ptr, sizeof(ptr)); - if (r <= 0 || cur != rr.dnsrr_dend) - return DNS_E_PROTOCOL; - l += dns_dntop_size(ptr); - ++c; - } - if (r < 0) - return DNS_E_PROTOCOL; - if (!c) - return DNS_E_NODATA; - - /* next, allocate and set up result */ - ret = malloc(sizeof(*ret) + sizeof(char **) * c + l + dns_stdrr_size(&p)); - if (!ret) - return DNS_E_NOMEM; - ret->dnsptr_nrr = c; - ret->dnsptr_ptr = (char **)(ret+1); - - /* and 3rd, fill in result, finally */ - sp = (char*)(ret->dnsptr_ptr + c); - c = 0; - dns_rewind(&p, qdn); - while((r = dns_nextrr(&p, &rr)) > 0) { - ret->dnsptr_ptr[c] = sp; - cur = rr.dnsrr_dptr; - dns_getdn(pkt, &cur, end, ptr, sizeof(ptr)); - sp += dns_dntop(ptr, sp, DNS_MAXNAME); - ++c; - } - dns_stdrr_finish((struct dns_rr_null *)ret, sp, &p); - *result = ret; - return 0; -} - -struct dns_query * -dns_submit_a4ptr(struct dns_ctx *ctx, const struct in_addr *addr, - dns_query_ptr_fn *cbck, void *data) { - dnsc_t dn[DNS_A4RSIZE]; - dns_a4todn(addr, 0, dn, sizeof(dn)); - return - dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_PTR, DNS_NOSRCH, - dns_parse_ptr, (dns_query_fn *)cbck, data); -} - -struct dns_rr_ptr * -dns_resolve_a4ptr(struct dns_ctx *ctx, const struct in_addr *addr) { - return (struct dns_rr_ptr *) - dns_resolve(ctx, dns_submit_a4ptr(ctx, addr, NULL, NULL)); -} - -struct dns_query * -dns_submit_a6ptr(struct dns_ctx *ctx, const struct in6_addr *addr, - dns_query_ptr_fn *cbck, void *data) { - dnsc_t dn[DNS_A6RSIZE]; - dns_a6todn(addr, 0, dn, sizeof(dn)); - return - dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_PTR, DNS_NOSRCH, - dns_parse_ptr, (dns_query_fn *)cbck, data); -} - -struct dns_rr_ptr * -dns_resolve_a6ptr(struct dns_ctx *ctx, const struct in6_addr *addr) { - return (struct dns_rr_ptr *) - dns_resolve(ctx, dns_submit_a6ptr(ctx, addr, NULL, NULL)); -} diff --git a/contrib/udns/udns_rr_srv.c b/contrib/udns/udns_rr_srv.c deleted file mode 100644 index dfba4653a3a..00000000000 --- a/contrib/udns/udns_rr_srv.c +++ /dev/null @@ -1,155 +0,0 @@ -/* udns_rr_srv.c - parse/query SRV IN (rfc2782) records - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - Copyright 2005 Thadeu Lima de Souza Cascardo - - 2005-09-11: - Changed MX parser file into a SRV parser file - - */ - -#include -#include -#include -#include "udns.h" - -int -dns_parse_srv(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, - void **result) { - struct dns_rr_srv *ret; - struct dns_parse p; - struct dns_rr rr; - int r, l; - char *sp; - dnsc_t srv[DNS_MAXDN]; - - assert(dns_get16(cur+2) == DNS_C_IN && dns_get16(cur+0) == DNS_T_SRV); - - /* first, validate the answer and count size of the result */ - l = 0; - dns_initparse(&p, qdn, pkt, cur, end); - while((r = dns_nextrr(&p, &rr)) > 0) { - cur = rr.dnsrr_dptr + 6; - r = dns_getdn(pkt, &cur, end, srv, sizeof(srv)); - if (r <= 0 || cur != rr.dnsrr_dend) - return DNS_E_PROTOCOL; - l += dns_dntop_size(srv); - } - if (r < 0) - return DNS_E_PROTOCOL; - if (!p.dnsp_nrr) - return DNS_E_NODATA; - - /* next, allocate and set up result */ - l += dns_stdrr_size(&p); - ret = malloc(sizeof(*ret) + sizeof(struct dns_srv) * p.dnsp_nrr + l); - if (!ret) - return DNS_E_NOMEM; - ret->dnssrv_nrr = p.dnsp_nrr; - ret->dnssrv_srv = (struct dns_srv *)(ret+1); - - /* and 3rd, fill in result, finally */ - sp = (char*)(ret->dnssrv_srv + p.dnsp_nrr); - for (dns_rewind(&p, qdn), r = 0; dns_nextrr(&p, &rr); ++r) { - ret->dnssrv_srv[r].name = sp; - cur = rr.dnsrr_dptr; - ret->dnssrv_srv[r].priority = dns_get16(cur); - ret->dnssrv_srv[r].weight = dns_get16(cur+2); - ret->dnssrv_srv[r].port = dns_get16(cur+4); - cur += 6; - dns_getdn(pkt, &cur, end, srv, sizeof(srv)); - sp += dns_dntop(srv, sp, DNS_MAXNAME); - } - dns_stdrr_finish((struct dns_rr_null *)ret, sp, &p); - *result = ret; - return 0; -} - -/* Add a single service or proto name prepending an undescore (_), - * according to rfc2782 rules. - * Return 0 or the label length. - * Routing assumes dn holds enouth space for a single DN label. */ -static int add_sname(dnsc_t *dn, const char *sn) { - int l = dns_ptodn(sn, 0, dn + 1, DNS_MAXLABEL-1, NULL); - if (l <= 1 || l - 2 != dn[1]) - /* Should we really check if sn is exactly one label? Do we care? */ - return 0; - dn[0] = l - 1; - dn[1] = '_'; - return l; -} - -/* Construct a domain name for SRV query from the given name, service and proto. - * The code allows any combinations of srv and proto (both are non-NULL, - * both NULL, or either one is non-NULL). Whenever it makes any sense or not - * is left as an exercise to programmer. - * Return negative value on error (malformed query) or addition query flag(s). - */ -static int -build_srv_dn(dnsc_t *dn, const char *name, const char *srv, const char *proto) -{ - int p = 0, l, isabs; - if (srv) { - l = add_sname(dn + p, srv); - if (!l) - return -1; - p += l; - } - if (proto) { - l = add_sname(dn + p, proto); - if (!l) - return -1; - p += l; - } - l = dns_ptodn(name, 0, dn + p, DNS_MAXDN - p, &isabs); - if (l < 0) - return -1; - return isabs ? DNS_NOSRCH : 0; -} - -struct dns_query * -dns_submit_srv(struct dns_ctx *ctx, - const char *name, const char *srv, const char *proto, - int flags, dns_query_srv_fn *cbck, void *data) { - dnsc_t dn[DNS_MAXDN]; - int r = build_srv_dn(dn, name, srv, proto); - if (r < 0) { - dns_setstatus (ctx, DNS_E_BADQUERY); - return NULL; - } - return - dns_submit_dn(ctx, dn, DNS_C_IN, DNS_T_SRV, flags | r, - dns_parse_srv, (dns_query_fn *)cbck, data); -} - -struct dns_rr_srv * -dns_resolve_srv(struct dns_ctx *ctx, - const char *name, const char *srv, const char *proto, int flags) -{ - dnsc_t dn[DNS_MAXDN]; - int r = build_srv_dn(dn, name, srv, proto); - if (r < 0) { - dns_setstatus(ctx, DNS_E_BADQUERY); - return NULL; - } - return (struct dns_rr_srv *) - dns_resolve_dn(ctx, dn, DNS_C_IN, DNS_T_SRV, flags | r, dns_parse_srv); -} diff --git a/contrib/udns/udns_rr_txt.c b/contrib/udns/udns_rr_txt.c deleted file mode 100644 index 97f1dfb18c3..00000000000 --- a/contrib/udns/udns_rr_txt.c +++ /dev/null @@ -1,98 +0,0 @@ -/* udns_rr_txt.c - parse/query TXT records - - Copyright (C) 2005 Michael Tokarev - This file is part of UDNS library, an async DNS stub resolver. - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Lesser General Public - License as published by the Free Software Foundation; either - version 2.1 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Lesser General Public License for more details. - - You should have received a copy of the GNU Lesser General Public - License along with this library, in file named COPYING.LGPL; if not, - write to the Free Software Foundation, Inc., 59 Temple Place, - Suite 330, Boston, MA 02111-1307 USA - - */ - -#include -#include -#include -#include "udns.h" - -int -dns_parse_txt(dnscc_t *qdn, dnscc_t *pkt, dnscc_t *cur, dnscc_t *end, - void **result) { - struct dns_rr_txt *ret; - struct dns_parse p; - struct dns_rr rr; - int r, l; - dnsc_t *sp; - dnscc_t *cp, *ep; - - assert(dns_get16(cur+0) == DNS_T_TXT); - - /* first, validate the answer and count size of the result */ - l = 0; - dns_initparse(&p, qdn, pkt, cur, end); - while((r = dns_nextrr(&p, &rr)) > 0) { - cp = rr.dnsrr_dptr; ep = rr.dnsrr_dend; - while(cp < ep) { - r = *cp++; - if (cp + r > ep) - return DNS_E_PROTOCOL; - l += r; - cp += r; - } - } - if (r < 0) - return DNS_E_PROTOCOL; - if (!p.dnsp_nrr) - return DNS_E_NODATA; - - /* next, allocate and set up result */ - l += (sizeof(struct dns_txt) + 1) * p.dnsp_nrr + dns_stdrr_size(&p); - ret = malloc(sizeof(*ret) + l); - if (!ret) - return DNS_E_NOMEM; - ret->dnstxt_nrr = p.dnsp_nrr; - ret->dnstxt_txt = (struct dns_txt *)(ret+1); - - /* and 3rd, fill in result, finally */ - sp = (dnsc_t*)(ret->dnstxt_txt + p.dnsp_nrr); - for(dns_rewind(&p, qdn), r = 0; dns_nextrr(&p, &rr) > 0; ++r) { - ret->dnstxt_txt[r].txt = sp; - cp = rr.dnsrr_dptr; ep = rr.dnsrr_dend; - while(cp < ep) { - l = *cp++; - memcpy(sp, cp, l); - sp += l; - cp += l; - } - ret->dnstxt_txt[r].len = sp - ret->dnstxt_txt[r].txt; - *sp++ = '\0'; - } - dns_stdrr_finish((struct dns_rr_null *)ret, (char*)sp, &p); - *result = ret; - return 0; -} - -struct dns_query * -dns_submit_txt(struct dns_ctx *ctx, const char *name, int qcls, int flags, - dns_query_txt_fn *cbck, void *data) { - return - dns_submit_p(ctx, name, qcls, DNS_T_TXT, flags, - dns_parse_txt, (dns_query_fn *)cbck, data); -} - -struct dns_rr_txt * -dns_resolve_txt(struct dns_ctx *ctx, const char *name, int qcls, int flags) { - return (struct dns_rr_txt *) - dns_resolve_p(ctx, name, qcls, DNS_T_TXT, flags, dns_parse_txt); -} From 3d4d71c7393c0d35343eebeaaee01e0e7d973fea Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 8 Jun 2022 17:26:52 +0200 Subject: [PATCH 054/659] Remove dbms ctarget ppp-dns and udns link --- src/CMakeLists.txt | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 6d4e7813bfa..f66ffe4a5ee 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -355,8 +355,6 @@ target_link_libraries(clickhouse_common_io ch_contrib::zlib pcg_random Poco::Foundation - ch_contrib::udns - ch_contrib::cpp-dns ) if (TARGET ch_contrib::cpuid) @@ -442,11 +440,12 @@ if (TARGET ch_contrib::avrocpp) endif () if (TARGET ch_contrib::udns) - dbms_target_link_libraries(PRIVATE ch_contrib::udns) + target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::udns) endif () + if (TARGET ch_contrib::cpp-dns) - dbms_target_link_libraries(PRIVATE ch_contrib::cpp-dns) + target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::cpp-dns) endif () if (TARGET OpenSSL::Crypto) From 52c5d7e0f9777260bba610937d1324044a0f32c7 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 8 Jun 2022 18:56:29 +0200 Subject: [PATCH 055/659] Add cpp-dns and udns to docker/test/fasttest --- docker/test/fasttest/run.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 2bbdd978e5e..bbbf2d49a7a 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -135,6 +135,9 @@ function clone_submodules contrib/replxx contrib/wyhash contrib/hashidsxx + contrib/liburing + contrib/udns + contrib/cpp-dns ) git submodule sync From 9d842ee357713e4b9fabd9c0363a8f8b8b6ef5d9 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 9 Jun 2022 08:23:53 -0300 Subject: [PATCH 056/659] Use https submodule url instead of ssh for cpp-dns --- .gitmodules | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitmodules b/.gitmodules index 2bdc01c728d..f212a901b8a 100644 --- a/.gitmodules +++ b/.gitmodules @@ -279,4 +279,4 @@ url = https://github.com/arthurpassos/udns.git [submodule "contrib/cpp-dns"] path = contrib/cpp-dns - url = git@github.com:YukiWorkshop/cpp-dns.git \ No newline at end of file + url = https://github.com/YukiWorkshop/cpp-dns.git \ No newline at end of file From dcda1c5eb963c283c2f2f22e61fb8a4433b311ee Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 15 Jun 2022 09:50:05 -0300 Subject: [PATCH 057/659] Fix style case and DNS error message --- src/Common/DNSResolver.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index d5eebe579b4..3f328768148 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -142,25 +142,25 @@ static DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host) static std::vector reverseResolveImpl(const Poco::Net::IPAddress & address) { - boost::asio::io_service ioService; - YukiWorkshop::DNSResolver resolver(ioService); + boost::asio::io_service io_service; + YukiWorkshop::DNSResolver resolver(io_service); - std::vector ptrRecords; + std::vector ptr_records; resolver.resolve_a4ptr(boost::asio::ip::address_v4::from_string(address.toString()), [&](int err, auto& hosts, auto&, auto&, uint) { if (err) { - throw Exception("Cannot resolve: " + address.toString() + gai_strerror(err), ErrorCodes::DNS_ERROR); + throw Exception("Cannot resolve: " + address.toString() + YukiWorkshop::DNSResolver::error_string(err), ErrorCodes::DNS_ERROR); } for (auto &it : hosts) { - ptrRecords.emplace_back(it); + ptr_records.emplace_back(it); } }); - ioService.run(); + io_service.run(); - return ptrRecords; + return ptr_records; } struct DNSResolver::Impl From 5c00dcd8849c4912dd5f1220b7b3fb0456aefe01 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 4 Jul 2022 09:08:45 -0300 Subject: [PATCH 058/659] Link cpp-dns/udns unconditionally and update submodules to clickhouse fork --- .gitmodules | 4 ++-- contrib/cpp-dns-cmake/CMakeLists.txt | 4 ++-- contrib/udns-cmake/CMakeLists.txt | 20 ++++++++++---------- src/CMakeLists.txt | 10 ++-------- src/Common/DNSResolver.cpp | 4 ++-- 5 files changed, 18 insertions(+), 24 deletions(-) diff --git a/.gitmodules b/.gitmodules index f212a901b8a..0b813722c60 100644 --- a/.gitmodules +++ b/.gitmodules @@ -276,7 +276,7 @@ url = https://github.com/ClickHouse/base-x.git [submodule "contrib/udns"] path = contrib/udns - url = https://github.com/arthurpassos/udns.git + url = https://github.com/ClickHouse/udns [submodule "contrib/cpp-dns"] path = contrib/cpp-dns - url = https://github.com/YukiWorkshop/cpp-dns.git \ No newline at end of file + url = https://github.com/ClickHouse/cpp-dns \ No newline at end of file diff --git a/contrib/cpp-dns-cmake/CMakeLists.txt b/contrib/cpp-dns-cmake/CMakeLists.txt index 96669df9e26..2ba803c28df 100644 --- a/contrib/cpp-dns-cmake/CMakeLists.txt +++ b/contrib/cpp-dns-cmake/CMakeLists.txt @@ -2,9 +2,9 @@ set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/cpp-dns") set(SRCS "${LIBRARY_DIR}/DNSResolver.cpp") -set(HDRS "${LIBRARY_DIR}/DNSResolver.hpp") +#set(HDRS "${LIBRARY_DIR}/DNSResolver.hpp") -add_library(_cpp-dns ${SRCS} ${HDRS}) +add_library(_cpp-dns ${SRCS}) target_link_libraries(_cpp-dns boost::system boost::headers_only ch_contrib::udns) diff --git a/contrib/udns-cmake/CMakeLists.txt b/contrib/udns-cmake/CMakeLists.txt index 8a3e9bc4f8f..1266ea2b847 100644 --- a/contrib/udns-cmake/CMakeLists.txt +++ b/contrib/udns-cmake/CMakeLists.txt @@ -1,22 +1,22 @@ set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/udns") add_library(_udns - "${LIBRARY_DIR}/udns_dn.c" + "${LIBRARY_DIR}/udns_bl.c" + "${LIBRARY_DIR}/udns_codes.c" "${LIBRARY_DIR}/udns_dntosp.c" + "${LIBRARY_DIR}/udns_init.c" + "${LIBRARY_DIR}/udns_jran.c" + "${LIBRARY_DIR}/udns_misc.c" "${LIBRARY_DIR}/udns_parse.c" "${LIBRARY_DIR}/udns_resolver.c" - "${LIBRARY_DIR}/udns_init.c" - "${LIBRARY_DIR}/udns_misc.c" - "${LIBRARY_DIR}/udns_XtoX.c" "${LIBRARY_DIR}/udns_rr_a.c" - "${LIBRARY_DIR}/udns_rr_ptr.c" "${LIBRARY_DIR}/udns_rr_mx.c" - "${LIBRARY_DIR}/udns_rr_txt.c" - "${LIBRARY_DIR}/udns_bl.c" - "${LIBRARY_DIR}/udns_rr_srv.c" "${LIBRARY_DIR}/udns_rr_naptr.c" - "${LIBRARY_DIR}/udns_codes.c" - "${LIBRARY_DIR}/udns_jran.c" + "${LIBRARY_DIR}/udns_rr_ptr.c" + "${LIBRARY_DIR}/udns_rr_srv.c" + "${LIBRARY_DIR}/udns_rr_txt.c" + "${LIBRARY_DIR}/udns_XtoX.c" + "${LIBRARY_DIR}/udns_dn.c" ) target_compile_definitions(_udns PRIVATE -DHAVE_CONFIG_H) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index f66ffe4a5ee..dfff3bcfa1d 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -439,14 +439,8 @@ if (TARGET ch_contrib::avrocpp) dbms_target_link_libraries(PRIVATE ch_contrib::avrocpp) endif () -if (TARGET ch_contrib::udns) - target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::udns) -endif () - - -if (TARGET ch_contrib::cpp-dns) - target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::cpp-dns) -endif () +target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::udns) +target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::cpp-dns) if (TARGET OpenSSL::Crypto) dbms_target_link_libraries (PRIVATE OpenSSL::Crypto) diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 3f328768148..dff1f8e7521 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -147,12 +147,12 @@ static std::vector reverseResolveImpl(const Poco::Net::IPAddress & addre std::vector ptr_records; - resolver.resolve_a4ptr(boost::asio::ip::address_v4::from_string(address.toString()), [&](int err, auto& hosts, auto&, auto&, uint) { + resolver.resolve_a4ptr(boost::asio::ip::address_v4::from_string(address.toString()), [&](int err, auto & hosts, auto &, auto &, uint) { if (err) { throw Exception("Cannot resolve: " + address.toString() + YukiWorkshop::DNSResolver::error_string(err), ErrorCodes::DNS_ERROR); } - for (auto &it : hosts) { + for (auto & it : hosts) { ptr_records.emplace_back(it); } From 6c987fb28e160e61b91866bb6dff93bff2f2f009 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 4 Jul 2022 10:13:50 -0300 Subject: [PATCH 059/659] Minor adjustments --- contrib/cpp-dns-cmake/CMakeLists.txt | 2 -- docker/test/fasttest/run.sh | 1 - 2 files changed, 3 deletions(-) diff --git a/contrib/cpp-dns-cmake/CMakeLists.txt b/contrib/cpp-dns-cmake/CMakeLists.txt index 2ba803c28df..20e6eb04088 100644 --- a/contrib/cpp-dns-cmake/CMakeLists.txt +++ b/contrib/cpp-dns-cmake/CMakeLists.txt @@ -2,8 +2,6 @@ set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/cpp-dns") set(SRCS "${LIBRARY_DIR}/DNSResolver.cpp") -#set(HDRS "${LIBRARY_DIR}/DNSResolver.hpp") - add_library(_cpp-dns ${SRCS}) target_link_libraries(_cpp-dns boost::system boost::headers_only ch_contrib::udns) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index bbbf2d49a7a..ce75e599eac 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -135,7 +135,6 @@ function clone_submodules contrib/replxx contrib/wyhash contrib/hashidsxx - contrib/liburing contrib/udns contrib/cpp-dns ) From e1794493812e8d6ed9da5f4c04ae0828b0ed12dc Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 4 Jul 2022 15:56:42 +0000 Subject: [PATCH 060/659] Fix some tests --- src/Interpreters/ExpressionAnalyzer.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index f982f066d3b..bab017f5a11 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -344,6 +344,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) group_by_kind = GroupByKind::GROUPING_SETS; else group_by_kind = GroupByKind::ORDINARY; + bool use_nulls = group_by_kind != GroupByKind::ORDINARY && getContext()->getSettingsRef().group_by_use_nulls; /// For GROUPING SETS with multiple groups we always add virtual __grouping_set column /// With set number, which is used as an additional key at the stage of merging aggregating data. @@ -398,7 +399,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) } } - NameAndTypePair key{column_name, makeNullable(node->result_type)}; + NameAndTypePair key{column_name, use_nulls ? makeNullable(node->result_type) : node->result_type }; grouping_set_list.push_back(key); @@ -452,7 +453,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) } } - NameAndTypePair key = select_query->group_by_with_rollup || select_query->group_by_with_cube ? NameAndTypePair{ column_name, makeNullable(node->result_type) } : NameAndTypePair{column_name, node->result_type}; + NameAndTypePair key = NameAndTypePair{ column_name, use_nulls ? makeNullable(node->result_type) : node->result_type }; /// Aggregation keys are uniqued. if (!unique_keys.contains(key.name)) @@ -1903,7 +1904,8 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( before_aggregation = chain.getLastActions(); before_aggregation_with_nullable = chain.getLastActions(); - query_analyzer.appendGroupByModifiers(before_aggregation, chain, only_types); + if (settings.group_by_use_nulls) + query_analyzer.appendGroupByModifiers(before_aggregation, chain, only_types); finalize_chain(chain); From 864ab205824640430da00bda75d82193731a02c7 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 4 Jul 2022 16:17:58 +0000 Subject: [PATCH 061/659] Use correct intermediate header for ROLLUP and CUBE --- src/Processors/Transforms/RollupTransform.cpp | 4 +++- ...02313_group_by_modifiers_with_non_default_types.reference} | 0 ...ql => 02313_group_by_modifiers_with_non_default_types.sql} | 0 3 files changed, 3 insertions(+), 1 deletion(-) rename tests/queries/0_stateless/{02313_group_by_modifiers_with_non-default_types.reference => 02313_group_by_modifiers_with_non_default_types.reference} (100%) rename tests/queries/0_stateless/{02313_group_by_modifiers_with_non-default_types.sql => 02313_group_by_modifiers_with_non_default_types.sql} (100%) diff --git a/src/Processors/Transforms/RollupTransform.cpp b/src/Processors/Transforms/RollupTransform.cpp index fc2159b0f44..0ab5900447f 100644 --- a/src/Processors/Transforms/RollupTransform.cpp +++ b/src/Processors/Transforms/RollupTransform.cpp @@ -18,12 +18,14 @@ GroupByModifierTransform::GroupByModifierTransform(Block header, AggregatingTran keys.emplace_back(input.getHeader().getPositionByName(key)); intermediate_header = getOutputPort().getHeader(); + intermediate_header.erase(0); + if (use_nulls) { auto output_aggregator_params = params->params; - intermediate_header.erase(0); output_aggregator = std::make_unique(intermediate_header, output_aggregator_params); } + LOG_DEBUG(&Poco::Logger::get("GroupByModifierTransform"), "Intermediate header: {}", intermediate_header.dumpStructure()); } void GroupByModifierTransform::consume(Chunk chunk) diff --git a/tests/queries/0_stateless/02313_group_by_modifiers_with_non-default_types.reference b/tests/queries/0_stateless/02313_group_by_modifiers_with_non_default_types.reference similarity index 100% rename from tests/queries/0_stateless/02313_group_by_modifiers_with_non-default_types.reference rename to tests/queries/0_stateless/02313_group_by_modifiers_with_non_default_types.reference diff --git a/tests/queries/0_stateless/02313_group_by_modifiers_with_non-default_types.sql b/tests/queries/0_stateless/02313_group_by_modifiers_with_non_default_types.sql similarity index 100% rename from tests/queries/0_stateless/02313_group_by_modifiers_with_non-default_types.sql rename to tests/queries/0_stateless/02313_group_by_modifiers_with_non_default_types.sql From 614ce26ee797eb612da1c5b63926b4509796db23 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 4 Jul 2022 18:09:48 +0000 Subject: [PATCH 062/659] Add group_be_use_nulls setting to stress tests --- docker/test/stress/stress | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index e195f81b551..9be3059795c 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -44,6 +44,9 @@ def get_options(i, backward_compatibility_check): if i == 13: client_options.append("memory_tracker_fault_probability=0.001") + if i % 17 == 1 and not backward_compatibility_check: + client_options.append("group_by_use_nulls=1") + if client_options: options.append(" --client-option " + " ".join(client_options)) From 659a6452f5c2460c46f8ef6781d6fc32bcb39c02 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 5 Jul 2022 13:10:09 +0000 Subject: [PATCH 063/659] Added SettingMap. Use Map for additional_table_filters. --- src/Columns/ColumnMap.cpp | 7 +- src/Core/Settings.h | 3 +- src/Core/SettingsFields.cpp | 67 ++++++++++++++++++ src/Core/SettingsFields.h | 22 ++++++ src/Interpreters/InterpreterSelectQuery.cpp | 69 ++++-------------- src/Parsers/ParserSetQuery.cpp | 53 +++++++++++++- .../02346_additional_filters.reference | 70 +++++++++---------- .../0_stateless/02346_additional_filters.sql | 70 +++++++++---------- 8 files changed, 234 insertions(+), 127 deletions(-) diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 06851f992ac..d5dbfb3e5af 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -99,7 +99,12 @@ void ColumnMap::insertData(const char *, size_t) void ColumnMap::insert(const Field & x) { const auto & map = DB::get(x); - nested->insert(Array(map.begin(), map.end())); + + Array array; + for (size_t i = 0; i < map.size(); i += 2) + array.push_back(Tuple({map[i], map[i + 1]})); + + nested->insert(array); } void ColumnMap::insertDefault() diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e24c69ef287..79d141ec180 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -587,7 +587,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) \ M(Bool, allow_unrestricted_reads_from_keeper, false, "Allow unrestricted (without condition on path) reads from system.zookeeper table, can be handy, but is not safe for zookeeper", 0) \ \ - M(String, additional_filters, "", "Additional filter expression which would be applied after reading from specified table. Syntax : 'table1:expression1;database.table2:expression2'", 0) \ + M(Map, additional_table_filters, "", "Additional filter expression which would be applied after reading from specified table. Syntax : {'table1' : 'expression', 'database.table2' : 'expression'}", 0) \ + M(Map, additional_result_filter, "", "Additional filter expression which would be applied to query result", 0) \ \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index d35865d35c6..bce8ede8134 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include #include @@ -51,6 +53,34 @@ namespace else return applyVisitor(FieldVisitorConvertToNumber(), f); } + + Map stringToMap(const String & str) + { + /// Allow empty string as an empty map + if (str.empty()) + return {}; + + auto type_string = std::make_shared(); + DataTypeMap type_map(type_string, type_string); + auto serialization = type_map.getSerialization(ISerialization::Kind::DEFAULT); + auto column = type_map.createColumn(); + + ReadBufferFromString buf(str); + serialization->deserializeTextEscaped(*column, buf, {}); + return (*column)[0].safeGet(); + } + + Map fieldToMap(const Field & f) + { + if (f.getType() == Field::Types::String) + { + /// Allow to parse Map from string field. For the convenience. + const auto & str = f.get(); + return stringToMap(str); + } + + return f.safeGet(); + } } template @@ -290,6 +320,43 @@ void SettingFieldString::readBinary(ReadBuffer & in) *this = std::move(str); } +SettingFieldMap::SettingFieldMap(const Field & f) : value(fieldToMap(f)) {} + +String SettingFieldMap::toString() const +{ + auto type_string = std::make_shared(); + DataTypeMap type_map(type_string, type_string); + auto serialization = type_map.getSerialization(ISerialization::Kind::DEFAULT); + auto column = type_map.createColumn(); + column->insert(value); + + WriteBufferFromOwnString out; + serialization->serializeTextEscaped(*column, 0, out, {}); + return out.str(); +} + + +SettingFieldMap & SettingFieldMap::operator =(const Field & f) +{ + value = fieldToMap(f); + return *this; +} + +void SettingFieldMap::parseFromString(const String & str) +{ + value = stringToMap(str); +} + +void SettingFieldMap::writeBinary(WriteBuffer & out) const +{ + DB::writeBinary(value, out); +} + +void SettingFieldMap::readBinary(ReadBuffer & in) +{ + DB::readBinary(value, in); +} + namespace { diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index dcc99f4a2c0..bd5bd6764c4 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -168,6 +168,28 @@ struct SettingFieldString void readBinary(ReadBuffer & in); }; +struct SettingFieldMap +{ +public: + Map value; + bool changed = false; + + explicit SettingFieldMap(const Map & map = {}) : value(map) {} + explicit SettingFieldMap(Map && map) : value(std::move(map)) {} + explicit SettingFieldMap(const Field & f); + + SettingFieldMap & operator =(const Map & map) { value = map; changed = true; return *this; } + SettingFieldMap & operator =(const Field & f); + + operator const Map &() const { return value; } /// NOLINT + explicit operator Field() const { return value; } + + String toString() const; + void parseFromString(const String & str); + + void writeBinary(WriteBuffer & out) const; + void readBinary(ReadBuffer & in); +}; struct SettingFieldChar { diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 90ad364256f..10915c5384e 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -284,68 +284,29 @@ static void checkAccessRightsForSelect( context->checkAccess(AccessType::SELECT, table_id, syntax_analyzer_result.requiredSourceColumnsForAccessCheck()); } -/// Parse additional filter for table in format 'table.name:expression' -/// Examples: 'default.hits:UserID=12345', 'visits:UserID = 0 ? 1 : 0' static ASTPtr parseAdditionalFilterConditionForTable( - const char * start, - const char * end, + const Map & setting, const DatabaseAndTableWithAlias & target, const Context & context) { - const char delimiter = ':'; - - const char * pos = start; - for (; pos < end; ++pos) - if (*pos == delimiter) - break; - - std::string_view table(start, pos - start); - - if (pos == end) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "No table is specified for additional filter {}. Expected syntax: 'table:condition'", - table); - - trim(table); - - if ((table == target.table && context.getCurrentDatabase() == target.database) || - (table == target.database + '.' + target.table)) + for (size_t i = 0; i < setting.size(); ++i) { - /// Try to parse expression - ParserExpression parser; - const auto & settings = context.getSettingsRef(); - return parseQuery(parser, pos + 1, end, "additional filter", settings.max_query_size, settings.max_parser_depth); - } + const auto & tuple = setting[i].safeGet(); + auto & table = tuple.at(0).safeGet(); + auto & filter = tuple.at(1).safeGet(); - return nullptr; -} - -static ASTPtr parseAdditionalFilterConditionForTable( - const std::string & setting, - const DatabaseAndTableWithAlias & target, - const Context & context) -{ - if (setting.empty()) - return nullptr; - - const char delimiter = ';'; - - const char * start = setting.data(); - const char * end = start + setting.size(); - for (const char * pos = start; pos < end; ++pos) - { - if (*pos == delimiter) + if ((table == target.table && context.getCurrentDatabase() == target.database) || + (table == target.database + '.' + target.table)) { - if (auto ast = parseAdditionalFilterConditionForTable(start, pos, target, context)) - return ast; - - start = pos + 1; + /// Try to parse expression + ParserExpression parser; + const auto & settings = context.getSettingsRef(); + return parseQuery( + parser, filter.data(), filter.data() + filter.size(), + "additional filter", settings.max_query_size, settings.max_parser_depth); } } - if (start < end) - return parseAdditionalFilterConditionForTable(start, end, target, context); - return nullptr; } @@ -528,9 +489,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (storage) view = dynamic_cast(storage.get()); - if (!settings.additional_filters.value.empty() && storage && !joined_tables.tablesWithColumns().empty()) + if (!settings.additional_table_filters.value.empty() && storage && !joined_tables.tablesWithColumns().empty()) query_info.additional_filter_ast = parseAdditionalFilterConditionForTable( - settings.additional_filters, joined_tables.tablesWithColumns().front().table, *context); + settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); auto analyze = [&] (bool try_move_to_prewhere) { diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index 5f69db633ac..0ff437bcfb1 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -12,12 +12,63 @@ namespace DB { +class ParserLiteralOrMap : public IParserBase +{ +public: +protected: + const char * getName() const override { return "literal or map"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + { + ParserLiteral literal; + if (literal.parse(pos, node, expected)) + return true; + } + + ParserToken l_br(TokenType::OpeningCurlyBrace); + ParserToken r_br(TokenType::ClosingCurlyBrace); + ParserToken comma(TokenType::Comma); + ParserToken colon(TokenType::Colon); + ParserStringLiteral literal; + + if (!l_br.ignore(pos, expected)) + return false; + + Map map; + + while (!r_br.ignore(pos, expected)) + { + if (!map.empty() && !comma.ignore(pos, expected)) + return false; + + ASTPtr key; + ASTPtr val; + + if (!literal.parse(pos, key, expected)) + return false; + + if (!colon.ignore(pos, expected)) + return false; + + if (!literal.parse(pos, val, expected)) + return false; + + Tuple tuple; + tuple.push_back(std::move(key->as()->value)); + tuple.push_back(std::move(val->as()->value)); + map.push_back(std::move(tuple)); + } + + node = std::make_shared(std::move(map)); + return true; + } +}; /// Parse `name = value`. bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & pos, Expected & expected) { ParserCompoundIdentifier name_p; - ParserLiteral value_p; + ParserLiteralOrMap value_p; ParserToken s_eq(TokenType::Equals); ASTPtr name; diff --git a/tests/queries/0_stateless/02346_additional_filters.reference b/tests/queries/0_stateless/02346_additional_filters.reference index 7b74a7a9488..2403af60094 100644 --- a/tests/queries/0_stateless/02346_additional_filters.reference +++ b/tests/queries/0_stateless/02346_additional_filters.reference @@ -5,54 +5,54 @@ select * from table_1; 2 bb 3 ccc 4 dddd -select * from table_1 settings additional_filters='table_1 : x != 2'; +select * from table_1 settings additional_table_filters={'table_1' : 'x != 2'}; 1 aa 3 ccc 4 dddd -select * from table_1 settings additional_filters='table_1 : x != 2 and x != 3'; +select * from table_1 settings additional_table_filters={'table_1' : 'x != 2 and x != 3'}; 1 aa 4 dddd -select x from table_1 settings additional_filters='table_1 : x != 2'; +select x from table_1 settings additional_table_filters={'table_1' : 'x != 2'}; 1 3 4 -select y from table_1 settings additional_filters='table_1 : x != 2'; +select y from table_1 settings additional_table_filters={'table_1' : 'x != 2'}; aa ccc dddd -select * from table_1 where x != 3 settings additional_filters='table_1 : x != 2'; +select * from table_1 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; 1 aa 4 dddd -select * from table_1 prewhere x != 4 settings additional_filters='table_1 : x != 2'; +select * from table_1 prewhere x != 4 settings additional_table_filters={'table_1' : 'x != 2'}; 1 aa 3 ccc -select * from table_1 prewhere x != 4 where x != 3 settings additional_filters='table_1 : x != 2'; +select * from table_1 prewhere x != 4 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; 1 aa -select x from table_1 where x != 3 settings additional_filters='table_1 : x != 2'; +select x from table_1 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; 1 4 -select x from table_1 prewhere x != 4 settings additional_filters='table_1 : x != 2'; +select x from table_1 prewhere x != 4 settings additional_table_filters={'table_1' : 'x != 2'}; 1 3 -select x from table_1 prewhere x != 4 where x != 3 settings additional_filters='table_1 : x != 2'; +select x from table_1 prewhere x != 4 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; 1 -select y from table_1 where x != 3 settings additional_filters='table_1 : x != 2'; +select y from table_1 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; aa dddd -select y from table_1 prewhere x != 4 settings additional_filters='table_1 : x != 2'; +select y from table_1 prewhere x != 4 settings additional_table_filters={'table_1' : 'x != 2'}; aa ccc -select y from table_1 prewhere x != 4 where x != 3 settings additional_filters='table_1 : x != 2'; +select y from table_1 prewhere x != 4 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; aa -select x from table_1 where x != 2 settings additional_filters='table_1 : x != 2'; +select x from table_1 where x != 2 settings additional_table_filters={'table_1' : 'x != 2'}; 1 3 4 -select x from table_1 prewhere x != 2 settings additional_filters='table_1 : x != 2'; +select x from table_1 prewhere x != 2 settings additional_table_filters={'table_1' : 'x != 2'}; 1 3 4 -select x from table_1 prewhere x != 2 where x != 2 settings additional_filters='table_1 : x != 2'; +select x from table_1 prewhere x != 2 where x != 2 settings additional_table_filters={'table_1' : 'x != 2'}; 1 3 4 @@ -62,19 +62,19 @@ select * from system.numbers limit 5; 2 3 4 -select * from system.numbers limit 5 settings additional_filters='system.numbers : number != 3'; +select * from system.numbers limit 5 settings additional_table_filters={'system.numbers' : 'number != 3'}; 0 1 2 4 5 -select * from system.numbers limit 5 settings additional_filters='system.numbers:number != 3;table_1:x!=2'; +select * from system.numbers limit 5 settings additional_table_filters={'system.numbers':'number != 3','table_1':'x!=2'}; 0 1 2 4 5 -select * from (select number from system.numbers limit 5 union all select x from table_1) order by number settings additional_filters='system.numbers:number != 3;table_1:x!=2'; +select * from (select number from system.numbers limit 5 union all select x from table_1) order by number settings additional_table_filters={'system.numbers':'number != 3','table_1':'x!=2'}; 0 1 1 @@ -83,13 +83,13 @@ select * from (select number from system.numbers limit 5 union all select x from 4 4 5 -select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x settings additional_filters='system.numbers : number != 3; table_1 : x != 2'; +select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x settings additional_table_filters={'system.numbers' : 'number != 3', 'table_1' : 'x != 2'}; 0 0 1 1 aa 2 0 4 4 dddd 5 0 -select b + 1 as c from (select a + 1 as b from (select x + 1 as a from table_1)) settings additional_filters='table_1 : x != 2 and x != 3'; +select b + 1 as c from (select a + 1 as b from (select x + 1 as a from table_1)) settings additional_table_filters={'table_1' : 'x != 2 and x != 3'}; 4 7 -- { echoOn } @@ -99,18 +99,18 @@ select * from v_numbers; 3 4 5 -select * from v_numbers settings additional_filters='system.numbers : number != 3'; +select * from v_numbers settings additional_table_filters={'system.numbers' : 'number != 3'}; 1 2 3 5 6 -select * from v_numbers settings additional_filters='v_numbers : x != 3'; +select * from v_numbers settings additional_table_filters={'v_numbers' : 'x != 3'}; 1 2 4 5 -select * from v_numbers settings additional_filters='system.numbers : number != 3; v_numbers : x != 3'; +select * from v_numbers settings additional_table_filters={'system.numbers' : 'number != 3', 'v_numbers' : 'x != 3'}; 1 2 5 @@ -123,16 +123,16 @@ select * from mv_table; 5 eeeee 6 ffffff 7 ggggggg -select * from mv_table settings additional_filters='mv_table : x != 5'; +select * from mv_table settings additional_table_filters={'mv_table' : 'x != 5'}; 4 dddd 6 ffffff 7 ggggggg -select * from mv_table settings additional_filters='table_1 : x != 5'; +select * from mv_table settings additional_table_filters={'table_1' : 'x != 5'}; 4 dddd 5 eeeee 6 ffffff 7 ggggggg -select * from mv_table settings additional_filters='table_2 : x != 5'; +select * from mv_table settings additional_table_filters={'table_2' : 'x != 5'}; 4 dddd 5 eeeee 6 ffffff @@ -149,7 +149,7 @@ select * from m_table order by x; 5 eeeee 6 ffffff 7 ggggggg -select * from m_table order by x settings additional_filters='table_1 : x != 2'; +select * from m_table order by x settings additional_table_filters={'table_1' : 'x != 2'}; 1 aa 2 bb 3 ccc @@ -158,7 +158,7 @@ select * from m_table order by x settings additional_filters='table_1 : x != 2'; 5 eeeee 6 ffffff 7 ggggggg -select * from m_table order by x settings additional_filters='table_2 : x != 5'; +select * from m_table order by x settings additional_table_filters={'table_2' : 'x != 5'}; 1 aa 2 bb 3 ccc @@ -167,7 +167,7 @@ select * from m_table order by x settings additional_filters='table_2 : x != 5' 5 eeeee 6 ffffff 7 ggggggg -select * from m_table order by x settings additional_filters='table_1 : x != 2; table_2 : x != 5'; +select * from m_table order by x settings additional_table_filters={'table_1' : 'x != 2', 'table_2' : 'x != 5'}; 1 aa 2 bb 3 ccc @@ -176,7 +176,7 @@ select * from m_table order by x settings additional_filters='table_1 : x != 2; 5 eeeee 6 ffffff 7 ggggggg -select * from m_table order by x settings additional_filters='table_1 : x != 4'; +select * from m_table order by x settings additional_table_filters={'table_1' : 'x != 4'}; 1 aa 2 bb 3 ccc @@ -185,7 +185,7 @@ select * from m_table order by x settings additional_filters='table_1 : x != 4' 5 eeeee 6 ffffff 7 ggggggg -select * from m_table order by x settings additional_filters='table_2 : x != 4'; +select * from m_table order by x settings additional_table_filters={'table_2' : 'x != 4'}; 1 aa 2 bb 3 ccc @@ -194,7 +194,7 @@ select * from m_table order by x settings additional_filters='table_2 : x != 4' 5 eeeee 6 ffffff 7 ggggggg -select * from m_table order by x settings additional_filters='table_1 : x != 4; table_2 : x != 4'; +select * from m_table order by x settings additional_table_filters={'table_1' : 'x != 4', 'table_2' : 'x != 4'}; 1 aa 2 bb 3 ccc @@ -203,14 +203,14 @@ select * from m_table order by x settings additional_filters='table_1 : x != 4; 5 eeeee 6 ffffff 7 ggggggg -select * from m_table order by x settings additional_filters='m_table : x != 4'; +select * from m_table order by x settings additional_table_filters={'m_table' : 'x != 4'}; 1 aa 2 bb 3 ccc 5 eeeee 6 ffffff 7 ggggggg -select * from m_table order by x settings additional_filters='m_table : x != 4; table_1 : x != 2; table_2 : x != 5'; +select * from m_table order by x settings additional_table_filters={'m_table' : 'x != 4', 'table_1' : 'x != 2', 'table_2' : 'x != 5'}; 1 aa 2 bb 3 ccc diff --git a/tests/queries/0_stateless/02346_additional_filters.sql b/tests/queries/0_stateless/02346_additional_filters.sql index 9c8185ff549..9383e9d7da2 100644 --- a/tests/queries/0_stateless/02346_additional_filters.sql +++ b/tests/queries/0_stateless/02346_additional_filters.sql @@ -9,29 +9,29 @@ insert into table_1 values (1, 'aa'), (2, 'bb'), (3, 'ccc'), (4, 'dddd'); -- { echoOn } select * from table_1; -select * from table_1 settings additional_filters='table_1 : x != 2'; -select * from table_1 settings additional_filters='table_1 : x != 2 and x != 3'; -select x from table_1 settings additional_filters='table_1 : x != 2'; -select y from table_1 settings additional_filters='table_1 : x != 2'; -select * from table_1 where x != 3 settings additional_filters='table_1 : x != 2'; -select * from table_1 prewhere x != 4 settings additional_filters='table_1 : x != 2'; -select * from table_1 prewhere x != 4 where x != 3 settings additional_filters='table_1 : x != 2'; -select x from table_1 where x != 3 settings additional_filters='table_1 : x != 2'; -select x from table_1 prewhere x != 4 settings additional_filters='table_1 : x != 2'; -select x from table_1 prewhere x != 4 where x != 3 settings additional_filters='table_1 : x != 2'; -select y from table_1 where x != 3 settings additional_filters='table_1 : x != 2'; -select y from table_1 prewhere x != 4 settings additional_filters='table_1 : x != 2'; -select y from table_1 prewhere x != 4 where x != 3 settings additional_filters='table_1 : x != 2'; -select x from table_1 where x != 2 settings additional_filters='table_1 : x != 2'; -select x from table_1 prewhere x != 2 settings additional_filters='table_1 : x != 2'; -select x from table_1 prewhere x != 2 where x != 2 settings additional_filters='table_1 : x != 2'; +select * from table_1 settings additional_table_filters={'table_1' : 'x != 2'}; +select * from table_1 settings additional_table_filters={'table_1' : 'x != 2 and x != 3'}; +select x from table_1 settings additional_table_filters={'table_1' : 'x != 2'}; +select y from table_1 settings additional_table_filters={'table_1' : 'x != 2'}; +select * from table_1 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; +select * from table_1 prewhere x != 4 settings additional_table_filters={'table_1' : 'x != 2'}; +select * from table_1 prewhere x != 4 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; +select x from table_1 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; +select x from table_1 prewhere x != 4 settings additional_table_filters={'table_1' : 'x != 2'}; +select x from table_1 prewhere x != 4 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; +select y from table_1 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; +select y from table_1 prewhere x != 4 settings additional_table_filters={'table_1' : 'x != 2'}; +select y from table_1 prewhere x != 4 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; +select x from table_1 where x != 2 settings additional_table_filters={'table_1' : 'x != 2'}; +select x from table_1 prewhere x != 2 settings additional_table_filters={'table_1' : 'x != 2'}; +select x from table_1 prewhere x != 2 where x != 2 settings additional_table_filters={'table_1' : 'x != 2'}; select * from system.numbers limit 5; -select * from system.numbers limit 5 settings additional_filters='system.numbers : number != 3'; -select * from system.numbers limit 5 settings additional_filters='system.numbers:number != 3;table_1:x!=2'; -select * from (select number from system.numbers limit 5 union all select x from table_1) order by number settings additional_filters='system.numbers:number != 3;table_1:x!=2'; -select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x settings additional_filters='system.numbers : number != 3; table_1 : x != 2'; -select b + 1 as c from (select a + 1 as b from (select x + 1 as a from table_1)) settings additional_filters='table_1 : x != 2 and x != 3'; +select * from system.numbers limit 5 settings additional_table_filters={'system.numbers' : 'number != 3'}; +select * from system.numbers limit 5 settings additional_table_filters={'system.numbers':'number != 3','table_1':'x!=2'}; +select * from (select number from system.numbers limit 5 union all select x from table_1) order by number settings additional_table_filters={'system.numbers':'number != 3','table_1':'x!=2'}; +select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x settings additional_table_filters={'system.numbers' : 'number != 3', 'table_1' : 'x != 2'}; +select b + 1 as c from (select a + 1 as b from (select x + 1 as a from table_1)) settings additional_table_filters={'table_1' : 'x != 2 and x != 3'}; -- { echoOff } @@ -39,9 +39,9 @@ create view v_numbers as select number + 1 as x from system.numbers limit 5; -- { echoOn } select * from v_numbers; -select * from v_numbers settings additional_filters='system.numbers : number != 3'; -select * from v_numbers settings additional_filters='v_numbers : x != 3'; -select * from v_numbers settings additional_filters='system.numbers : number != 3; v_numbers : x != 3'; +select * from v_numbers settings additional_table_filters={'system.numbers' : 'number != 3'}; +select * from v_numbers settings additional_table_filters={'v_numbers' : 'x != 3'}; +select * from v_numbers settings additional_table_filters={'system.numbers' : 'number != 3', 'v_numbers' : 'x != 3'}; -- { echoOff } @@ -54,9 +54,9 @@ create materialized view mv_table to table_2 (x UInt32, y String) as select * fr -- additional filter for inner tables for Materialized View does not work because it does not create internal interpreter -- probably it is expected select * from mv_table; -select * from mv_table settings additional_filters='mv_table : x != 5'; -select * from mv_table settings additional_filters='table_1 : x != 5'; -select * from mv_table settings additional_filters='table_2 : x != 5'; +select * from mv_table settings additional_table_filters={'mv_table' : 'x != 5'}; +select * from mv_table settings additional_table_filters={'table_1' : 'x != 5'}; +select * from mv_table settings additional_table_filters={'table_2' : 'x != 5'}; -- { echoOff } @@ -66,11 +66,11 @@ create table m_table (x UInt32, y String) engine = Merge(currentDatabase(), '^ta -- additional filter for inner tables for Merge does not work because it does not create internal interpreter -- probably it is expected select * from m_table order by x; -select * from m_table order by x settings additional_filters='table_1 : x != 2'; -select * from m_table order by x settings additional_filters='table_2 : x != 5'; -select * from m_table order by x settings additional_filters='table_1 : x != 2; table_2 : x != 5'; -select * from m_table order by x settings additional_filters='table_1 : x != 4'; -select * from m_table order by x settings additional_filters='table_2 : x != 4'; -select * from m_table order by x settings additional_filters='table_1 : x != 4; table_2 : x != 4'; -select * from m_table order by x settings additional_filters='m_table : x != 4'; -select * from m_table order by x settings additional_filters='m_table : x != 4; table_1 : x != 2; table_2 : x != 5'; +select * from m_table order by x settings additional_table_filters={'table_1' : 'x != 2'}; +select * from m_table order by x settings additional_table_filters={'table_2' : 'x != 5'}; +select * from m_table order by x settings additional_table_filters={'table_1' : 'x != 2', 'table_2' : 'x != 5'}; +select * from m_table order by x settings additional_table_filters={'table_1' : 'x != 4'}; +select * from m_table order by x settings additional_table_filters={'table_2' : 'x != 4'}; +select * from m_table order by x settings additional_table_filters={'table_1' : 'x != 4', 'table_2' : 'x != 4'}; +select * from m_table order by x settings additional_table_filters={'m_table' : 'x != 4'}; +select * from m_table order by x settings additional_table_filters={'m_table' : 'x != 4', 'table_1' : 'x != 2', 'table_2' : 'x != 5'}; From 8b008fb726512822318994e345b4454840728998 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 5 Jul 2022 09:42:44 -0400 Subject: [PATCH 064/659] remove nasm as a submodule --- .gitmodules | 3 --- contrib/nasm | 1 - 2 files changed, 4 deletions(-) delete mode 160000 contrib/nasm diff --git a/.gitmodules b/.gitmodules index ca10e25c056..502039c0da2 100644 --- a/.gitmodules +++ b/.gitmodules @@ -262,9 +262,6 @@ [submodule "contrib/qpl"] path = contrib/qpl url = https://github.com/intel/qpl.git -[submodule "contrib/nasm"] - path = contrib/nasm - url = https://github.com/netwide-assembler/nasm.git [submodule "contrib/wyhash"] path = contrib/wyhash url = https://github.com/wangyi-fudan/wyhash.git diff --git a/contrib/nasm b/contrib/nasm deleted file mode 160000 index 3f9fc2a3a71..00000000000 --- a/contrib/nasm +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 3f9fc2a3a7134936cbbae5780beb4319694f702a From 463c03b84e36dce86c6cad7745ec4e3cd0ee732a Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 5 Jul 2022 14:08:44 -0400 Subject: [PATCH 065/659] improve qpl section in CMakefile:remove windows build flag, modify qpl build option dependency on avx2/avx512 --- CMakeLists.txt | 8 --- contrib/CMakeLists.txt | 4 +- contrib/qpl-cmake/CMakeLists.txt | 92 ++++++++++---------------------- src/CMakeLists.txt | 5 +- 4 files changed, 31 insertions(+), 78 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 87aa9c521f9..c8bb1a2d1ca 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -122,14 +122,6 @@ if (ENABLE_FUZZING) set (ENABLE_PROTOBUF 1) endif() -if (OS_LINUX AND ARCH_AMD64 AND NOT BUILD_STANDALONE_KEEPER) - option (ENABLE_QPL "Enable Intel® Query Processing Library" ON) - if (ENABLE_QPL) - set (ENABLE_QPL_COMPRESSION 1) - set (ENABLE_QPL_ANALYTIC 1) - endif() -endif() - # Global libraries # See: # - default_libs.cmake diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 9c54ed26f87..16f345fe6bc 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -156,9 +156,7 @@ endif() add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) add_contrib (base-x-cmake base-x) -if (ENABLE_QPL) - add_contrib (qpl-cmake qpl) -endif() +add_contrib (qpl-cmake qpl) # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index e6714c671e1..a33532bcb52 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -1,8 +1,18 @@ ## The Intel® QPL provides high performance implementations of data processing functions for existing hardware accelerator, and/or software path in case if hardware accelerator is not available. +if (OS_LINUX AND ARCH_AMD64 AND (HAVE_AVX2 OR HAVE_AVX512) AND NOT BUILD_STANDALONE_KEEPER) + option (ENABLE_QPL "Enable Intel® Query Processing Library" ${ENABLE_LIBRARIES}) +elseif(ENABLE_QPL) + message (${RECONFIGURE_MESSAGE_LEVEL} "QPL library is only supported on x86_64") +endif() + +if (NOT ENABLE_QPL) + message(STATUS "Not using QPL") + return() +endif() set (QPL_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl") set (QPL_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl/sources") -set (QPL_BINARY_DIR "${ClickHouse_SOURCE_DIR}/build/contrib/qpl") +set (QPL_BINARY_DIR "${ClickHouse_BINARY_DIR}/build/contrib/qpl") set (UUID_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl-cmake") set (EFFICIENT_WAIT ON) @@ -13,7 +23,7 @@ set (SANITIZE_THREADS OFF) set (LIB_FUZZING_ENGINE OFF) function(GetLibraryVersion _content _outputVar) - string(REGEX MATCHALL ".*Qpl VERSION+.* (.+).*LANGUAGES" VERSION_REGEX "${_content}") + string(REGEX MATCHALL "Qpl VERSION (.+) LANGUAGES" VERSION_REGEX "${_content}") SET(${_outputVar} ${CMAKE_MATCH_1} PARENT_SCOPE) endfunction() @@ -81,9 +91,6 @@ set(ISAL_ASM_SRC ${QPL_SRC_DIR}/isal/igzip/igzip_body.asm ${QPL_SRC_DIR}/isal/crc/crc32_iscsi_01.asm ${QPL_SRC_DIR}/isal/crc/crc32_iscsi_by16_10.asm) -set(DISABLED_WARNINGS_FLAGS /wd4244 /wd4101 /wd4267 /wd4018 /wd4146 /wd4334 /wd4308) -set(FEATURE_FLAGS -DHAVE_AS_KNOWS_AVX512 -DAS_FEATURE_LEVEL=10) - # Adding ISA-L library target add_library(isal OBJECT ${ISAL_C_SRC}) add_library(isal_asm OBJECT ${ISAL_ASM_SRC}) @@ -100,7 +107,6 @@ set_target_properties(isal PROPERTIES target_compile_options(isal PRIVATE "$<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}>" - "$<$:/WX;${DISABLED_WARNINGS_FLAGS};${QPL_WINDOWS_TOOLCHAIN_REQUIRED_FLAGS}>" "$<$:>" "$<$:>") @@ -151,24 +157,16 @@ target_include_directories(qplcore_avx512 PRIVATE $) set_target_properties(qplcore_avx512 PROPERTIES - $<$:C_STANDARD 18> $<$:C_STANDARD 17>) target_link_libraries(qplcore_avx512 ${CMAKE_DL_LIBS} isal) -if (WIN32) - target_compile_options(qplcore_avx512 - PRIVATE ${QPL_WINDOWS_TOOLCHAIN_REQUIRED_FLAGS} - PRIVATE /arch:AVX512 - PRIVATE "$<$:>" - PRIVATE "$<$:-O2>") -else () - target_compile_options(qplcore_avx512 - PRIVATE ${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS} - PRIVATE -march=skylake-avx512 - PRIVATE "$<$:>" - PRIVATE "$<$:-O3;-D_FORTIFY_SOURCE=2>") -endif () +target_compile_options(qplcore_avx512 + PRIVATE ${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS} + PRIVATE -march=skylake-avx512 + PRIVATE "$<$:>" + PRIVATE "$<$:-O3;-D_FORTIFY_SOURCE=2>") + target_compile_definitions(qplcore_avx512 PUBLIC QPL_BADARG_CHECK) @@ -189,22 +187,14 @@ target_include_directories(qplcore_px PRIVATE $) set_target_properties(qplcore_px PROPERTIES - $<$:C_STANDARD 18> $<$:C_STANDARD 17>) target_link_libraries(qplcore_px isal ${CMAKE_DL_LIBS}) -if (WIN32) - target_compile_options(qplcore_px - PRIVATE "$<$:>" - PRIVATE ${QPL_WINDOWS_TOOLCHAIN_REQUIRED_FLAGS} - PRIVATE "$<$:-O2>") -else () - target_compile_options(qplcore_px - PRIVATE ${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS} - PRIVATE "$<$:>" - PRIVATE "$<$:-O3;-D_FORTIFY_SOURCE=2>") -endif () +target_compile_options(qplcore_px + PRIVATE ${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS} + PRIVATE "$<$:>" + PRIVATE "$<$:-O3;-D_FORTIFY_SOURCE=2>") target_compile_definitions(qplcore_px PUBLIC QPL_BADARG_CHECK) @@ -227,21 +217,12 @@ target_include_directories(core_iaa PRIVATE $) set_target_properties(core_iaa PROPERTIES - $<$:C_STANDARD 18> $<$:C_STANDARD 17> CXX_STANDARD 17) -if (WIN32) - modify_standard_language_flag(LANGUAGE_NAME "CXX" - FLAG_NAME "/GR" - NEW_FLAG_VALUE "-") -endif () - target_compile_options(core_iaa PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}; - $<$:-O3;-D_FORTIFY_SOURCE=2>> - PRIVATE $<$:${QPL_WINDOWS_TOOLCHAIN_REQUIRED_FLAGS}; - $<$:-O2>>) + $<$:-O3;-D_FORTIFY_SOURCE=2>>) target_compile_features(core_iaa PRIVATE c_std_11) @@ -276,25 +257,14 @@ add_library(middle_layer_lib OBJECT ${GENERATED_AVX512_TABLES_SRC} ${MIDDLE_LAYER_SRC}) -if (WIN32) - modify_standard_language_flag(LANGUAGE_NAME "CXX" - FLAG_NAME "/GR" - NEW_FLAG_VALUE "-") -endif () - target_compile_options(middle_layer_lib PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}; ${QPL_LINUX_TOOLCHAIN_DYNAMIC_LIBRARY_FLAGS}; $<$:-O3;-D_FORTIFY_SOURCE=2>> - PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}> - PRIVATE $<$:${QPL_WINDOWS_TOOLCHAIN_REQUIRED_FLAGS}; - ${QPL_WINDOWS_TOOLCHAIN_DYNAMIC_LIBRARY_FLAGS} - $<$:-O2>> - PRIVATE $<$:${QPL_WINDOWS_TOOLCHAIN_CPP_EMBEDDED_FLAGS}>) + PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}>) target_compile_definitions(middle_layer_lib PUBLIC QPL_VERSION="${QPL_VERSION}" - PUBLIC $<$:_ENABLE_EXTENDED_ALIGNED_STORAGE> PUBLIC $<$:LOG_HW_INIT> PUBLIC $<$:QPL_EFFICIENT_WAIT> PUBLIC QPL_BADARG_CHECK) @@ -335,7 +305,6 @@ target_include_directories(_qpl PRIVATE $) set_target_properties(_qpl PROPERTIES - $<$:C_STANDARD 18> $<$:C_STANDARD 17> CXX_STANDARD 17) @@ -343,17 +312,12 @@ target_compile_options(_qpl PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}; ${QPL_LINUX_TOOLCHAIN_DYNAMIC_LIBRARY_FLAGS}; $<$:-O3;-D_FORTIFY_SOURCE=2>> - PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}> - PRIVATE $<$:${QPL_WINDOWS_TOOLCHAIN_REQUIRED_FLAGS}; - ${QPL_WINDOWS_TOOLCHAIN_DYNAMIC_LIBRARY_FLAGS} - $<$:-O2>> - PRIVATE $<$:${QPL_WINDOWS_TOOLCHAIN_CPP_EMBEDDED_FLAGS}>) + PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}>) target_compile_definitions(_qpl - PUBLIC -DQPL_LIB - PUBLIC -DQPL_BADARG_CHECK - PUBLIC -DENABLE_QPL_COMPRESSION - PUBLIC $<$:_ENABLE_EXTENDED_ALIGNED_STORAGE>) + PRIVATE -DQPL_LIB + PRIVATE -DQPL_BADARG_CHECK + PUBLIC -DENABLE_QPL_COMPRESSION) target_link_libraries(_qpl PRIVATE ${CMAKE_DL_LIBS}) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 0489cc7e527..e3f17939086 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -235,13 +235,12 @@ macro(remove_item_object_library name common_file) list (REMOVE_ITEM ${${name}_sources} ${common_file}) endif () endmacro() - add_object_library(clickhouse_access Access) add_object_library(clickhouse_backups Backups) add_object_library(clickhouse_core Core) add_object_library(clickhouse_core_mysql Core/MySQL) add_object_library(clickhouse_compression Compression) -if (NOT ENABLE_QPL_COMPRESSION) +if (NOT ENABLE_QPL) remove_item_object_library(clickhouse_compression Compression/CompressionCodecDeflate.cpp) else () set_source_files_properties( @@ -534,7 +533,7 @@ endif () target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::lz4) -if (ENABLE_QPL_COMPRESSION) +if (TARGET ch_contrib::qpl) dbms_target_link_libraries(PUBLIC ch_contrib::qpl) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/qpl/include) endif () From ba38969e9c7979e529a57abfa99fb2b611d2d004 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 5 Jul 2022 15:28:41 -0400 Subject: [PATCH 066/659] enable qpl for clickhouse-keeper build --- contrib/qpl-cmake/CMakeLists.txt | 10 ++++++---- docker/test/fasttest/run.sh | 1 - programs/keeper/CMakeLists.txt | 11 +++++++++-- src/CMakeLists.txt | 21 ++++++--------------- src/Compression/CompressionCodecDeflate.cpp | 2 ++ src/Compression/CompressionFactory.cpp | 4 ++-- 6 files changed, 25 insertions(+), 24 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index a33532bcb52..0155914657c 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -1,8 +1,8 @@ ## The Intel® QPL provides high performance implementations of data processing functions for existing hardware accelerator, and/or software path in case if hardware accelerator is not available. -if (OS_LINUX AND ARCH_AMD64 AND (HAVE_AVX2 OR HAVE_AVX512) AND NOT BUILD_STANDALONE_KEEPER) +if (OS_LINUX AND ARCH_AMD64 AND (HAVE_AVX2 OR HAVE_AVX512)) option (ENABLE_QPL "Enable Intel® Query Processing Library" ${ENABLE_LIBRARIES}) elseif(ENABLE_QPL) - message (${RECONFIGURE_MESSAGE_LEVEL} "QPL library is only supported on x86_64") + message (${RECONFIGURE_MESSAGE_LEVEL} "QPL library is only supported on x86_64 arch with avx2/avx512 support") endif() if (NOT ENABLE_QPL) @@ -113,9 +113,11 @@ target_compile_options(isal PRIVATE target_compile_options(isal_asm PUBLIC "-I${QPL_SRC_DIR}/isal/include/" PUBLIC "-I${QPL_SRC_DIR}/isal/igzip/" PUBLIC "-I${QPL_SRC_DIR}/isal/crc/" - PUBLIC "-DHAVE_AS_KNOWS_AVX512" PUBLIC "-DAS_FEATURE_LEVEL=10" PUBLIC "-DQPL_LIB") +if (HAVE_AVX512) + target_compile_options(isal_asm PUBLIC "-DHAVE_AS_KNOWS_AVX512") +endif() if (SANITIZE STREQUAL "undefined") get_target_property(target_options isal_asm COMPILE_OPTIONS) @@ -323,4 +325,4 @@ target_link_libraries(_qpl PRIVATE ${CMAKE_DL_LIBS}) add_library (ch_contrib::qpl ALIAS _qpl) -target_include_directories(_qpl PUBLIC "${QPL_PROJECT_DIR}/include") +target_include_directories(_qpl SYSTEM BEFORE PUBLIC "${QPL_PROJECT_DIR}/include") diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 2725efc8ea8..4daf3680f4e 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -134,7 +134,6 @@ function clone_submodules contrib/jemalloc contrib/replxx contrib/qpl - contrib/nasm contrib/wyhash contrib/hashidsxx ) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index c77b335b615..bfe034db87a 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -75,6 +75,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecMultiple.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecNone.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecZSTD.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecDeflate.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionFactory.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/ICompressionCodec.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/LZ4_decompress_faster.cpp @@ -103,7 +104,11 @@ if (BUILD_STANDALONE_KEEPER) # Remove some redundant dependencies target_compile_definitions (clickhouse-keeper PRIVATE -DKEEPER_STANDALONE_BUILD) - + if (ENABLE_QPL) + set_source_files_properties( + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecDeflate.cpp + PROPERTIES COMPILE_FLAGS "-mwaitpkg") + endif () target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_SOURCE_DIR}/../../src") # uses includes from src directory target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/../../src/Core/include") # uses some includes from core target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/../../src") # uses some includes from common @@ -127,7 +132,9 @@ if (BUILD_STANDALONE_KEEPER) ${LINK_RESOURCE_LIB_STANDALONE_KEEPER} ) - + if (TARGET ch_contrib::qpl) + target_link_libraries(clickhouse-keeper PRIVATE ch_contrib::qpl ${LINK_RESOURCE_LIB_STANDALONE_KEEPER}) + endif () add_dependencies(clickhouse-keeper clickhouse_keeper_configs) set_target_properties(clickhouse-keeper PROPERTIES RUNTIME_OUTPUT_DIRECTORY ../) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index e3f17939086..e120433f037 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -228,25 +228,11 @@ macro(add_object_library name common_path) endif () endmacro() -macro(remove_item_object_library name common_file) - if (USE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) - list (REMOVE_ITEM dbms_sources ${common_file}) - else () - list (REMOVE_ITEM ${${name}_sources} ${common_file}) - endif () -endmacro() add_object_library(clickhouse_access Access) add_object_library(clickhouse_backups Backups) add_object_library(clickhouse_core Core) add_object_library(clickhouse_core_mysql Core/MySQL) add_object_library(clickhouse_compression Compression) -if (NOT ENABLE_QPL) -remove_item_object_library(clickhouse_compression Compression/CompressionCodecDeflate.cpp) -else () -set_source_files_properties( - Compression/CompressionCodecDeflate.cpp - PROPERTIES COMPILE_FLAGS "-mwaitpkg") -endif () add_object_library(clickhouse_querypipeline QueryPipeline) add_object_library(clickhouse_datatypes DataTypes) add_object_library(clickhouse_datatypes_serializations DataTypes/Serializations) @@ -356,6 +342,12 @@ set_source_files_properties( Columns/ColumnString.cpp PROPERTIES COMPILE_FLAGS "${X86_INTRINSICS_FLAGS}") +if (ENABLE_QPL) +set_source_files_properties( + Compression/CompressionCodecDeflate.cpp + PROPERTIES COMPILE_FLAGS "-mwaitpkg") +endif () + target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::re2_st) target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::re2) @@ -535,7 +527,6 @@ target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::lz4) if (TARGET ch_contrib::qpl) dbms_target_link_libraries(PUBLIC ch_contrib::qpl) -dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/qpl/include) endif () dbms_target_link_libraries(PRIVATE _boost_context) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 2eca32714e6..4e2ff0c06ef 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -1,3 +1,4 @@ +#ifdef ENABLE_QPL_COMPRESSION #include #include #include @@ -355,3 +356,4 @@ void registerCodecDeflate(CompressionCodecFactory & factory) "DEFLATE", static_cast(CompressionMethodByte::Deflate), [&]() { return std::make_shared(); }); } } +#endif diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 8667f8e515a..bc12d35319c 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -188,10 +188,10 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecZSTD(*this); registerCodecLZ4HC(*this); registerCodecMultiple(*this); -#ifndef KEEPER_STANDALONE_BUILD #ifdef ENABLE_QPL_COMPRESSION registerCodecDeflate(*this); -#endif +#endif +#ifndef KEEPER_STANDALONE_BUILD registerCodecDelta(*this); registerCodecT64(*this); registerCodecDoubleDelta(*this); From fbe3d84860f0fe85694e487e6df5adda46aee6ef Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 5 Jul 2022 15:29:53 -0400 Subject: [PATCH 067/659] revise typo issue --- src/Compression/CompressionFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index bc12d35319c..78351479c89 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -190,7 +190,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecMultiple(*this); #ifdef ENABLE_QPL_COMPRESSION registerCodecDeflate(*this); -#endif +#endif #ifndef KEEPER_STANDALONE_BUILD registerCodecDelta(*this); registerCodecT64(*this); From 08907f822088e8e45ad11c6d15d84f78c0d9d424 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 5 Jul 2022 16:54:17 -0400 Subject: [PATCH 068/659] change HAVE_AVX512 in qpl cmake to ENABLE_AVX512 --- contrib/qpl-cmake/CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 0155914657c..45a576cd3cf 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -1,5 +1,5 @@ ## The Intel® QPL provides high performance implementations of data processing functions for existing hardware accelerator, and/or software path in case if hardware accelerator is not available. -if (OS_LINUX AND ARCH_AMD64 AND (HAVE_AVX2 OR HAVE_AVX512)) +if (OS_LINUX AND ARCH_AMD64 AND (ENABLE_AVX2 OR ENABLE_AVX512)) option (ENABLE_QPL "Enable Intel® Query Processing Library" ${ENABLE_LIBRARIES}) elseif(ENABLE_QPL) message (${RECONFIGURE_MESSAGE_LEVEL} "QPL library is only supported on x86_64 arch with avx2/avx512 support") @@ -115,7 +115,7 @@ target_compile_options(isal_asm PUBLIC "-I${QPL_SRC_DIR}/isal/include/" PUBLIC "-I${QPL_SRC_DIR}/isal/crc/" PUBLIC "-DAS_FEATURE_LEVEL=10" PUBLIC "-DQPL_LIB") -if (HAVE_AVX512) +if (ENABLE_AVX512) target_compile_options(isal_asm PUBLIC "-DHAVE_AS_KNOWS_AVX512") endif() From 720c179963d9821bb4a93d25c5b49b16ca8d59a4 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 5 Jul 2022 17:40:54 -0400 Subject: [PATCH 069/659] remove redundant flags --- cmake/cpu_features.cmake | 2 +- contrib/qpl-cmake/CMakeLists.txt | 19 +++---------------- 2 files changed, 4 insertions(+), 17 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 7b966e1acac..ad8d55e6c60 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -18,7 +18,7 @@ option (ENABLE_PCLMULQDQ "Use pclmulqdq instructions on x86_64" 1) option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1) option (ENABLE_AVX "Use AVX instructions on x86_64" 0) option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0) -option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 0) +option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 1) option (ENABLE_BMI "Use BMI instructions on x86_64" 0) option (ENABLE_AVX2_FOR_SPEC_OP "Use avx2 instructions for specific operations on x86_64" 0) option (ENABLE_AVX512_FOR_SPEC_OP "Use avx512 instructions for specific operations on x86_64" 0) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 45a576cd3cf..e7affa7add0 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -101,10 +101,6 @@ target_include_directories(isal PRIVATE ${QPL_SRC_DIR}/isal/include PUBLIC ${QPL_SRC_DIR}/isal/igzip) -set_target_properties(isal PROPERTIES - CXX_STANDARD 11 - C_STANDARD 99) - target_compile_options(isal PRIVATE "$<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}>" "$<$:>" @@ -130,8 +126,6 @@ target_compile_definitions(isal PUBLIC NDEBUG) # [SUBDIR]core-sw -enable_language(C) - # # Create avx512 library # @@ -149,7 +143,9 @@ file(GLOB DATA_SOURCES # Create library add_library(qplcore_avx512 OBJECT ${SOURCES}) - +# Two libraries will be created which implemented by AVX512 and non-AVX512 respectively. +# The Program will check SIMD capabilities of the target system at runtime and use up to AVX512 if available. +# PLATFORM=2 means AVX512 implementation; PLATFORM=0 means non-AVX512 implementation. target_compile_definitions(qplcore_avx512 PRIVATE PLATFORM=2) target_include_directories(qplcore_avx512 @@ -218,10 +214,6 @@ target_include_directories(core_iaa PRIVATE $ PRIVATE $) -set_target_properties(core_iaa PROPERTIES - $<$:C_STANDARD 17> - CXX_STANDARD 17) - target_compile_options(core_iaa PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}; $<$:-O3;-D_FORTIFY_SOURCE=2>>) @@ -271,7 +263,6 @@ target_compile_definitions(middle_layer_lib PUBLIC $<$:QPL_EFFICIENT_WAIT> PUBLIC QPL_BADARG_CHECK) -set_target_properties(middle_layer_lib PROPERTIES CXX_STANDARD 17) set_source_files_properties(${GENERATED_PX_TABLES_SRC} PROPERTIES COMPILE_DEFINITIONS PLATFORM=0) set_source_files_properties(${GENERATED_AVX512_TABLES_SRC} PROPERTIES COMPILE_DEFINITIONS PLATFORM=2) @@ -306,10 +297,6 @@ target_include_directories(_qpl PRIVATE $ PRIVATE $) -set_target_properties(_qpl PROPERTIES - $<$:C_STANDARD 17> - CXX_STANDARD 17) - target_compile_options(_qpl PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}; ${QPL_LINUX_TOOLCHAIN_DYNAMIC_LIBRARY_FLAGS}; From aeb4f9c6be8523c83da22753d6e84e67b2839b50 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 6 Jul 2022 16:56:17 +0800 Subject: [PATCH 070/659] fixed: crash caused by IHiveFile be shared among threads --- src/Storages/Hive/HiveFile.cpp | 8 +++++++- src/Storages/Hive/HiveFile.h | 4 ++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 57acbdd577b..92eee681ea1 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -79,8 +79,12 @@ Range createRangeFromParquetStatistics(std::shared_ptr IHiveFile::getRows() { - if (!rows) + if (!has_init_rows) + { + std::lock_guard lock(mutex); rows = getRowsImpl(); + has_init_rows = true; + } return rows; } @@ -88,6 +92,7 @@ void IHiveFile::loadFileMinMaxIndex() { if (file_minmax_idx_loaded) return; + std::lock_guard lock(mutex); loadFileMinMaxIndexImpl(); file_minmax_idx_loaded = true; } @@ -96,6 +101,7 @@ void IHiveFile::loadSplitMinMaxIndexes() { if (split_minmax_idxes_loaded) return; + std::lock_guard lock(mutex); loadSplitMinMaxIndexesImpl(); split_minmax_idxes_loaded = true; } diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index cbdf17bd5b5..a4bd345aa48 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -149,6 +149,7 @@ protected: String path; UInt64 last_modify_time; size_t size; + std::atomic has_init_rows = false; std::optional rows; NamesAndTypesList index_names_and_types; @@ -162,6 +163,9 @@ protected: /// Skip splits for this file after applying minmax index (if any) std::unordered_set skip_splits; std::shared_ptr storage_settings; + + /// IHiveFile would be shared among multi threads, need lock's protection to update min/max indexes. + std::mutex mutex; }; using HiveFilePtr = std::shared_ptr; From f63b27498febae35a2ab42d15f7c4967d12d4694 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 6 Jul 2022 12:37:37 +0000 Subject: [PATCH 071/659] Add additional_result_filter setting. --- src/Columns/ColumnMap.cpp | 7 +- src/Core/Settings.h | 2 +- src/Core/SettingsFields.cpp | 8 +- .../IInterpreterUnionOrSelectQuery.cpp | 54 ++++++++++++ .../IInterpreterUnionOrSelectQuery.h | 2 + .../InterpreterSelectIntersectExceptQuery.cpp | 1 + .../InterpreterSelectWithUnionQuery.cpp | 1 + .../02346_additional_filters.reference | 82 ++++++++++++++----- .../0_stateless/02346_additional_filters.sql | 24 +++++- 9 files changed, 146 insertions(+), 35 deletions(-) diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index d5dbfb3e5af..06851f992ac 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -99,12 +99,7 @@ void ColumnMap::insertData(const char *, size_t) void ColumnMap::insert(const Field & x) { const auto & map = DB::get(x); - - Array array; - for (size_t i = 0; i < map.size(); i += 2) - array.push_back(Tuple({map[i], map[i + 1]})); - - nested->insert(array); + nested->insert(Array(map.begin(), map.end())); } void ColumnMap::insertDefault() diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 79d141ec180..792fff0275a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -588,7 +588,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, allow_unrestricted_reads_from_keeper, false, "Allow unrestricted (without condition on path) reads from system.zookeeper table, can be handy, but is not safe for zookeeper", 0) \ \ M(Map, additional_table_filters, "", "Additional filter expression which would be applied after reading from specified table. Syntax : {'table1' : 'expression', 'database.table2' : 'expression'}", 0) \ - M(Map, additional_result_filter, "", "Additional filter expression which would be applied to query result", 0) \ + M(String, additional_result_filter, "", "Additional filter expression which would be applied to query result", 0) \ \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index bce8ede8134..089644926b2 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -338,13 +338,13 @@ String SettingFieldMap::toString() const SettingFieldMap & SettingFieldMap::operator =(const Field & f) { - value = fieldToMap(f); + *this = fieldToMap(f); return *this; } void SettingFieldMap::parseFromString(const String & str) { - value = stringToMap(str); + *this = stringToMap(str); } void SettingFieldMap::writeBinary(WriteBuffer & out) const @@ -354,7 +354,9 @@ void SettingFieldMap::writeBinary(WriteBuffer & out) const void SettingFieldMap::readBinary(ReadBuffer & in) { - DB::readBinary(value, in); + Map map; + DB::readBinary(map, in); + *this = map; } diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index 4ac1d33468f..05486f65da5 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -4,6 +4,13 @@ #include #include #include +#include +#include +#include +#include +#include +#include +#include namespace DB { @@ -81,6 +88,53 @@ void IInterpreterUnionOrSelectQuery::setQuota(QueryPipeline & pipeline) const pipeline.setQuota(quota); } +static ASTPtr parseAdditionalPostFilter(const Context & context) +{ + const auto & settings = context.getSettingsRef(); + const String & filter = settings.additional_result_filter; + if (filter.empty()) + return nullptr; + + ParserExpression parser; + return parseQuery( + parser, filter.data(), filter.data() + filter.size(), + "additional filter", settings.max_query_size, settings.max_parser_depth); +} + +static ActionsDAGPtr makeAdditionalPostFilter(ASTPtr & ast, ContextPtr context, const Block & header) +{ + auto syntax_result = TreeRewriter(context).analyze(ast, header.getNamesAndTypesList()); + String result_column_name = ast->getColumnName(); + auto dag = ExpressionAnalyzer(ast, syntax_result, context).getActionsDAG(false, false); + const ActionsDAG::Node * result_node = &dag->findInIndex(result_column_name); + auto & index = dag->getIndex(); + index.clear(); + index.reserve(dag->getInputs().size() + 1); + for (const auto * node : dag->getInputs()) + index.push_back(node); + + index.push_back(result_node); + + return dag; +} + +void IInterpreterUnionOrSelectQuery::addAdditionalPostFilter(QueryPlan & plan) const +{ + if (options.subquery_depth != 0) + return; + + auto ast = parseAdditionalPostFilter(*context); + if (!ast) + return; + + auto dag = makeAdditionalPostFilter(ast, context, plan.getCurrentDataStream().header); + std::string filter_name = dag->getIndex().back()->result_name; + auto filter_step = std::make_unique( + plan.getCurrentDataStream(), std::move(dag), std::move(filter_name), true); + filter_step->setStepDescription("Additional result filter"); + plan.addStep(std::move(filter_step)); +} + void IInterpreterUnionOrSelectQuery::addStorageLimits(const StorageLimitsList & limits) { for (const auto & val : limits) diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index 1af9a25614e..b3d98612e37 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -72,6 +72,8 @@ protected: /// Set quotas to query pipeline. void setQuota(QueryPipeline & pipeline) const; + /// Add filter from additional_post_filter setting. + void addAdditionalPostFilter(QueryPlan & plan) const; static StorageLimits getStorageLimits(const Context & context, const SelectQueryOptions & options); }; diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 4d0c82d3345..d6add3f77a9 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -138,6 +138,7 @@ void InterpreterSelectIntersectExceptQuery::buildQueryPlan(QueryPlan & query_pla auto step = std::make_unique(std::move(data_streams), final_operator, max_threads); query_plan.unitePlans(std::move(step), std::move(plans)); + addAdditionalPostFilter(query_plan); query_plan.addInterpreterContext(context); } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index f0150fe663f..1db168cd5bb 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -341,6 +341,7 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) } } + addAdditionalPostFilter(query_plan); query_plan.addInterpreterContext(context); } diff --git a/tests/queries/0_stateless/02346_additional_filters.reference b/tests/queries/0_stateless/02346_additional_filters.reference index 2403af60094..0cd345f71cd 100644 --- a/tests/queries/0_stateless/02346_additional_filters.reference +++ b/tests/queries/0_stateless/02346_additional_filters.reference @@ -1,33 +1,33 @@ -- { echoOn } select * from table_1; -1 aa +1 a 2 bb 3 ccc 4 dddd select * from table_1 settings additional_table_filters={'table_1' : 'x != 2'}; -1 aa +1 a 3 ccc 4 dddd select * from table_1 settings additional_table_filters={'table_1' : 'x != 2 and x != 3'}; -1 aa +1 a 4 dddd select x from table_1 settings additional_table_filters={'table_1' : 'x != 2'}; 1 3 4 select y from table_1 settings additional_table_filters={'table_1' : 'x != 2'}; -aa +a ccc dddd select * from table_1 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; -1 aa +1 a 4 dddd select * from table_1 prewhere x != 4 settings additional_table_filters={'table_1' : 'x != 2'}; -1 aa +1 a 3 ccc select * from table_1 prewhere x != 4 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; -1 aa +1 a select x from table_1 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; 1 4 @@ -37,13 +37,13 @@ select x from table_1 prewhere x != 4 settings additional_table_filters={'table_ select x from table_1 prewhere x != 4 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; 1 select y from table_1 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; -aa +a dddd select y from table_1 prewhere x != 4 settings additional_table_filters={'table_1' : 'x != 2'}; -aa +a ccc select y from table_1 prewhere x != 4 where x != 3 settings additional_table_filters={'table_1' : 'x != 2'}; -aa +a select x from table_1 where x != 2 settings additional_table_filters={'table_1' : 'x != 2'}; 1 3 @@ -56,6 +56,10 @@ select x from table_1 prewhere x != 2 where x != 2 settings additional_table_fil 1 3 4 +select * from remote('127.0.0.{1,2}', system.one) settings additional_table_filters={'system.one' : 'dummy = 0'}; +0 +0 +select * from remote('127.0.0.{1,2}', system.one) settings additional_table_filters={'system.one' : 'dummy != 0'}; select * from system.numbers limit 5; 0 1 @@ -85,7 +89,7 @@ select * from (select number from system.numbers limit 5 union all select x from 5 select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x settings additional_table_filters={'system.numbers' : 'number != 3', 'table_1' : 'x != 2'}; 0 0 -1 1 aa +1 1 a 2 0 4 4 dddd 5 0 @@ -115,9 +119,9 @@ select * from v_numbers settings additional_table_filters={'system.numbers' : 'n 2 5 6 --- { echoOn } -- additional filter for inner tables for Materialized View does not work because it does not create internal interpreter -- probably it is expected +-- { echoOn } select * from mv_table; 4 dddd 5 eeeee @@ -137,11 +141,11 @@ select * from mv_table settings additional_table_filters={'table_2' : 'x != 5'}; 5 eeeee 6 ffffff 7 ggggggg --- { echoOn } -- additional filter for inner tables for Merge does not work because it does not create internal interpreter -- probably it is expected +-- { echoOn } select * from m_table order by x; -1 aa +1 a 2 bb 3 ccc 4 dddd @@ -150,7 +154,7 @@ select * from m_table order by x; 6 ffffff 7 ggggggg select * from m_table order by x settings additional_table_filters={'table_1' : 'x != 2'}; -1 aa +1 a 2 bb 3 ccc 4 dddd @@ -159,7 +163,7 @@ select * from m_table order by x settings additional_table_filters={'table_1' : 6 ffffff 7 ggggggg select * from m_table order by x settings additional_table_filters={'table_2' : 'x != 5'}; -1 aa +1 a 2 bb 3 ccc 4 dddd @@ -168,7 +172,7 @@ select * from m_table order by x settings additional_table_filters={'table_2' : 6 ffffff 7 ggggggg select * from m_table order by x settings additional_table_filters={'table_1' : 'x != 2', 'table_2' : 'x != 5'}; -1 aa +1 a 2 bb 3 ccc 4 dddd @@ -177,7 +181,7 @@ select * from m_table order by x settings additional_table_filters={'table_1' : 6 ffffff 7 ggggggg select * from m_table order by x settings additional_table_filters={'table_1' : 'x != 4'}; -1 aa +1 a 2 bb 3 ccc 4 dddd @@ -186,7 +190,7 @@ select * from m_table order by x settings additional_table_filters={'table_1' : 6 ffffff 7 ggggggg select * from m_table order by x settings additional_table_filters={'table_2' : 'x != 4'}; -1 aa +1 a 2 bb 3 ccc 4 dddd @@ -195,7 +199,7 @@ select * from m_table order by x settings additional_table_filters={'table_2' : 6 ffffff 7 ggggggg select * from m_table order by x settings additional_table_filters={'table_1' : 'x != 4', 'table_2' : 'x != 4'}; -1 aa +1 a 2 bb 3 ccc 4 dddd @@ -204,16 +208,50 @@ select * from m_table order by x settings additional_table_filters={'table_1' : 6 ffffff 7 ggggggg select * from m_table order by x settings additional_table_filters={'m_table' : 'x != 4'}; -1 aa +1 a 2 bb 3 ccc 5 eeeee 6 ffffff 7 ggggggg select * from m_table order by x settings additional_table_filters={'m_table' : 'x != 4', 'table_1' : 'x != 2', 'table_2' : 'x != 5'}; -1 aa +1 a 2 bb 3 ccc 5 eeeee 6 ffffff 7 ggggggg +-- additional_result_filter + +select * from table_1 settings additional_result_filter='x != 2'; +1 a +3 ccc +4 dddd +select *, x != 2 from table_1 settings additional_result_filter='x != 2'; +1 a 1 +3 ccc 1 +4 dddd 1 +select * from table_1 where x != 1 settings additional_result_filter='x != 2'; +3 ccc +4 dddd +select * from table_1 where x != 1 settings additional_result_filter='x != 2 and x != 3'; +4 dddd +select * from table_1 prewhere x != 3 where x != 1 settings additional_result_filter='x != 2'; +4 dddd +select * from table_1 limit 3 settings additional_result_filter='x != 2'; +1 a +3 ccc +select x + 1 from table_1 settings additional_result_filter='`plus(x, 1)` != 2'; +3 +4 +5 +select * from (select x + 1 as a, y from table_1 union all select x as a, y from table_1) order by a, y settings additional_result_filter='a = 3'; +3 bb +3 ccc +select * from (select x + 1 as a, y from table_1 union all select x as a, y from table_1) order by a, y settings additional_result_filter='a != 3'; +1 a +2 a +2 bb +4 ccc +4 dddd +5 dddd diff --git a/tests/queries/0_stateless/02346_additional_filters.sql b/tests/queries/0_stateless/02346_additional_filters.sql index 9383e9d7da2..24e04b9dc8b 100644 --- a/tests/queries/0_stateless/02346_additional_filters.sql +++ b/tests/queries/0_stateless/02346_additional_filters.sql @@ -4,7 +4,7 @@ drop table if exists v_numbers; drop table if exists mv_table; create table table_1 (x UInt32, y String) engine = MergeTree order by x; -insert into table_1 values (1, 'aa'), (2, 'bb'), (3, 'ccc'), (4, 'dddd'); +insert into table_1 values (1, 'a'), (2, 'bb'), (3, 'ccc'), (4, 'dddd'); -- { echoOn } @@ -26,6 +26,9 @@ select x from table_1 where x != 2 settings additional_table_filters={'table_1' select x from table_1 prewhere x != 2 settings additional_table_filters={'table_1' : 'x != 2'}; select x from table_1 prewhere x != 2 where x != 2 settings additional_table_filters={'table_1' : 'x != 2'}; +select * from remote('127.0.0.{1,2}', system.one) settings additional_table_filters={'system.one' : 'dummy = 0'}; +select * from remote('127.0.0.{1,2}', system.one) settings additional_table_filters={'system.one' : 'dummy != 0'}; + select * from system.numbers limit 5; select * from system.numbers limit 5 settings additional_table_filters={'system.numbers' : 'number != 3'}; select * from system.numbers limit 5 settings additional_table_filters={'system.numbers':'number != 3','table_1':'x!=2'}; @@ -50,9 +53,9 @@ insert into table_2 values (4, 'dddd'), (5, 'eeeee'), (6, 'ffffff'), (7, 'gggggg create materialized view mv_table to table_2 (x UInt32, y String) as select * from table_1; --- { echoOn } -- additional filter for inner tables for Materialized View does not work because it does not create internal interpreter -- probably it is expected +-- { echoOn } select * from mv_table; select * from mv_table settings additional_table_filters={'mv_table' : 'x != 5'}; select * from mv_table settings additional_table_filters={'table_1' : 'x != 5'}; @@ -62,9 +65,9 @@ select * from mv_table settings additional_table_filters={'table_2' : 'x != 5'}; create table m_table (x UInt32, y String) engine = Merge(currentDatabase(), '^table_'); --- { echoOn } -- additional filter for inner tables for Merge does not work because it does not create internal interpreter -- probably it is expected +-- { echoOn } select * from m_table order by x; select * from m_table order by x settings additional_table_filters={'table_1' : 'x != 2'}; select * from m_table order by x settings additional_table_filters={'table_2' : 'x != 5'}; @@ -74,3 +77,18 @@ select * from m_table order by x settings additional_table_filters={'table_2' : select * from m_table order by x settings additional_table_filters={'table_1' : 'x != 4', 'table_2' : 'x != 4'}; select * from m_table order by x settings additional_table_filters={'m_table' : 'x != 4'}; select * from m_table order by x settings additional_table_filters={'m_table' : 'x != 4', 'table_1' : 'x != 2', 'table_2' : 'x != 5'}; + +-- additional_result_filter + +select * from table_1 settings additional_result_filter='x != 2'; +select *, x != 2 from table_1 settings additional_result_filter='x != 2'; +select * from table_1 where x != 1 settings additional_result_filter='x != 2'; +select * from table_1 where x != 1 settings additional_result_filter='x != 2 and x != 3'; +select * from table_1 prewhere x != 3 where x != 1 settings additional_result_filter='x != 2'; + +select * from table_1 limit 3 settings additional_result_filter='x != 2'; + +select x + 1 from table_1 settings additional_result_filter='`plus(x, 1)` != 2'; + +select * from (select x + 1 as a, y from table_1 union all select x as a, y from table_1) order by a, y settings additional_result_filter='a = 3'; +select * from (select x + 1 as a, y from table_1 union all select x as a, y from table_1) order by a, y settings additional_result_filter='a != 3'; From 65cd0cc61c365dcb6aded25e12b94bf3bdf68e02 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 6 Jul 2022 13:44:06 +0000 Subject: [PATCH 072/659] Fixing style. --- src/Interpreters/InterpreterSelectQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 10915c5384e..b8b74213e1a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -105,7 +105,6 @@ namespace ErrorCodes extern const int INVALID_WITH_FILL_EXPRESSION; extern const int ACCESS_DENIED; extern const int UNKNOWN_IDENTIFIER; - extern const int BAD_ARGUMENTS; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. From b13b1cf2265b8f73c62ca224180e43526d471a3b Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 6 Jul 2022 10:34:22 -0400 Subject: [PATCH 073/659] add some comments in CMakeLists.txt --- contrib/qpl-cmake/CMakeLists.txt | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index e7affa7add0..a227e042776 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -32,6 +32,10 @@ GetLibraryVersion("${HEADER_CONTENT}" QPL_VERSION) message(STATUS "Intel QPL version: ${QPL_VERSION}") +# There are 5 source subdirectories under $QPL_SRC_DIR: isal, c_api, core-sw, middle-layer, c_api. +# Generate 7 library targets: middle_layer_lib, isal, isal_asm, qplcore_px, qplcore_avx512, core_iaa, middle_layer_lib. +# Output ch_contrib::qpl by linking with 7 library targets. + include("${QPL_PROJECT_DIR}/cmake/CompileOptions.cmake") # check nasm compiler @@ -106,6 +110,8 @@ target_compile_options(isal PRIVATE "$<$:>" "$<$:>") +# AS_FEATURE_LEVEL=10 means "Check SIMD capabilities of the target system at runtime and use up to AVX512 if available". +# HAVE_KNOWS_AVX512 means rely on AVX512 being available on the target system. target_compile_options(isal_asm PUBLIC "-I${QPL_SRC_DIR}/isal/include/" PUBLIC "-I${QPL_SRC_DIR}/isal/igzip/" PUBLIC "-I${QPL_SRC_DIR}/isal/crc/" @@ -115,6 +121,8 @@ if (ENABLE_AVX512) target_compile_options(isal_asm PUBLIC "-DHAVE_AS_KNOWS_AVX512") endif() +# Here must remove "-fno-sanitize=undefined" from COMPILE_OPTIONS. +# Otherwise nasm compiler would fail to proceed due to unrecognition of "-fno-sanitize=undefined" if (SANITIZE STREQUAL "undefined") get_target_property(target_options isal_asm COMPILE_OPTIONS) list(REMOVE_ITEM target_options "-fno-sanitize=undefined") @@ -225,8 +233,6 @@ target_compile_definitions(core_iaa PRIVATE QPL_BADARG_CHECK PRIVATE $<$:LOG_HW_INIT>) # [SUBDIR]middle-layer -enable_language(CXX) - generate_unpack_kernel_arrays(${QPL_BINARY_DIR}) file(GLOB MIDDLE_LAYER_SRC @@ -276,9 +282,8 @@ target_include_directories(middle_layer_lib PUBLIC $) target_compile_definitions(middle_layer_lib PUBLIC -DQPL_LIB) -# [SUBDIR]c_api -enable_language(C CXX) +# [SUBDIR]c_api file(GLOB_RECURSE QPL_C_API_SRC ${QPL_SRC_DIR}/c_api/*.c ${QPL_SRC_DIR}/c_api/*.cpp) From 3f8524f53d20d3a14a1c4b01ebd72916cda7f0a2 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 6 Jul 2022 11:17:27 -0400 Subject: [PATCH 074/659] disable avx512 by default --- cmake/cpu_features.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index ad8d55e6c60..7b966e1acac 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -18,7 +18,7 @@ option (ENABLE_PCLMULQDQ "Use pclmulqdq instructions on x86_64" 1) option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1) option (ENABLE_AVX "Use AVX instructions on x86_64" 0) option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0) -option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 1) +option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 0) option (ENABLE_BMI "Use BMI instructions on x86_64" 0) option (ENABLE_AVX2_FOR_SPEC_OP "Use avx2 instructions for specific operations on x86_64" 0) option (ENABLE_AVX512_FOR_SPEC_OP "Use avx512 instructions for specific operations on x86_64" 0) From 60549c0a77d6156e62e24df08e8dd41fcdc9e15b Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 6 Jul 2022 11:29:47 -0400 Subject: [PATCH 075/659] remove license link --- docs/en/development/contrib.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index 9fb77613458..8c1f6b5fc9e 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -86,7 +86,6 @@ The list of third-party libraries: | xz | [Public Domain](https://github.com/xz-mirror/xz/blob/869b9d1b4edd6df07f819d360d306251f8147353/COPYING) | | zlib-ng | [zLib](https://github.com/ClickHouse-Extras/zlib-ng/blob/6a5e93b9007782115f7f7e5235dedc81c4f1facb/LICENSE.md) | | zstd | [BSD](https://github.com/facebook/zstd/blob/a488ba114ec17ea1054b9057c26a046fc122b3b6/LICENSE) | -| qpl | [MIT](https://github.com/intel/qpl/blob/cdc8442f7a5e7a6ff6eea39c69665e0c5034d85d/LICENSE) | The list of third-party libraries can be obtained by the following query: From f91d73fa3769c22fe7e43277e6d85001c468a444 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 6 Jul 2022 12:19:37 -0400 Subject: [PATCH 076/659] adjust AS_FEATURE_LEVEL --- contrib/qpl-cmake/CMakeLists.txt | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index a227e042776..283d8877ff7 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -110,15 +110,17 @@ target_compile_options(isal PRIVATE "$<$:>" "$<$:>") -# AS_FEATURE_LEVEL=10 means "Check SIMD capabilities of the target system at runtime and use up to AVX512 if available". -# HAVE_KNOWS_AVX512 means rely on AVX512 being available on the target system. target_compile_options(isal_asm PUBLIC "-I${QPL_SRC_DIR}/isal/include/" PUBLIC "-I${QPL_SRC_DIR}/isal/igzip/" PUBLIC "-I${QPL_SRC_DIR}/isal/crc/" - PUBLIC "-DAS_FEATURE_LEVEL=10" PUBLIC "-DQPL_LIB") + +# AS_FEATURE_LEVEL=10 means "Check SIMD capabilities of the target system at runtime and use up to AVX512 if available". +# HAVE_KNOWS_AVX512 means rely on AVX512 being available on the target system. if (ENABLE_AVX512) - target_compile_options(isal_asm PUBLIC "-DHAVE_AS_KNOWS_AVX512") + target_compile_options(isal_asm PUBLIC "-DHAVE_AS_KNOWS_AVX512" "-DAS_FEATURE_LEVEL=10") +else() + target_compile_options(isal_asm PUBLIC "-DAS_FEATURE_LEVEL=5") endif() # Here must remove "-fno-sanitize=undefined" from COMPILE_OPTIONS. From 65a91864b3912525ae04a9181f1445378cbf3bd2 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 6 Jul 2022 13:26:07 -0400 Subject: [PATCH 077/659] revise comments --- contrib/qpl-cmake/CMakeLists.txt | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 283d8877ff7..7b504c235c8 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -136,10 +136,11 @@ target_compile_definitions(isal PUBLIC NDEBUG) # [SUBDIR]core-sw -# -# Create avx512 library -# -#set(CMAKE_INCLUDE_CURRENT_DIR ON) +# Two libraries:qplcore_avx512/qplcore_px for SW fallback will be created which are implemented by AVX512 and non-AVX512 instructions respectively. +# The upper level QPL API will check SIMD capabilities of the target system at runtime and decide to call AVX512 function or non-AVX512 function. +# Hence, here we don't need put qplcore_avx512 under an ENABLE_AVX512 CMake switch. +# Actually, if we do that, some undefined symbols errors would happen because both of AVX512 function and non-AVX512 function are referenced by QPL API. +# PLATFORM=2 means AVX512 implementation; PLATFORM=0 means non-AVX512 implementation. # Find Core Sources file(GLOB SOURCES @@ -151,11 +152,9 @@ file(GLOB SOURCES file(GLOB DATA_SOURCES ${QPL_SRC_DIR}/core-sw/src/data/*.c) -# Create library +# Create avx512 library add_library(qplcore_avx512 OBJECT ${SOURCES}) -# Two libraries will be created which implemented by AVX512 and non-AVX512 respectively. -# The Program will check SIMD capabilities of the target system at runtime and use up to AVX512 if available. -# PLATFORM=2 means AVX512 implementation; PLATFORM=0 means non-AVX512 implementation. + target_compile_definitions(qplcore_avx512 PRIVATE PLATFORM=2) target_include_directories(qplcore_avx512 From 567f90d8a4bef9130704ab2d481e7d0fff23f2a1 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 6 Jul 2022 18:18:55 -0400 Subject: [PATCH 078/659] add CodecMode for deflate codec --- src/Compression/CompressedReadBufferBase.cpp | 15 ++++-- src/Compression/CompressedReadBufferBase.h | 6 ++- .../CompressedReadBufferFromFile.cpp | 14 +++--- src/Compression/CompressionCodecDeflate.cpp | 48 +++++++++---------- src/Compression/CompressionCodecDeflate.h | 3 -- src/Compression/ICompressionCodec.cpp | 16 ++----- src/Compression/ICompressionCodec.h | 42 ++++++++-------- 7 files changed, 69 insertions(+), 75 deletions(-) diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index da2edccb776..7eff176a6d2 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -290,14 +290,14 @@ static void readHeaderAndGetCodec(const char * compressed_buffer, size_t size_de } -void CompressedReadBufferBase::decompressTo(char * to, size_t size_decompressed, size_t size_compressed_without_checksum, UInt8 req_type) +void CompressedReadBufferBase::decompressTo(char * to, size_t size_decompressed, size_t size_compressed_without_checksum) { readHeaderAndGetCodec(compressed_buffer, size_decompressed, codec, allow_different_codecs); - codec->decompress(compressed_buffer, size_compressed_without_checksum, to, req_type); + codec->decompress(compressed_buffer, size_compressed_without_checksum, to); } -void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum, UInt8 req_type) +void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum) { readHeaderAndGetCodec(compressed_buffer, size_decompressed, codec, allow_different_codecs); @@ -315,7 +315,7 @@ void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_d to = BufferBase::Buffer(compressed_buffer + header_size, compressed_buffer + size_compressed_without_checksum); } else - codec->decompress(compressed_buffer, size_compressed_without_checksum, to.begin(), req_type); + codec->decompress(compressed_buffer, size_compressed_without_checksum, to.begin()); } void CompressedReadBufferBase::decompressFlush() const @@ -326,6 +326,13 @@ void CompressedReadBufferBase::decompressFlush() const } } +void CompressedReadBufferBase::setDecompressMode(ICompressionCodec::CodecMode mode) +{ + if (codec) + { + codec->setDecompressMode(mode); + } +} /// 'compressed_in' could be initialized lazily, but before first call of 'readCompressedData'. CompressedReadBufferBase::CompressedReadBufferBase(ReadBuffer * in, bool allow_different_codecs_) : compressed_in(in), own_compressed_buffer(0), allow_different_codecs(allow_different_codecs_) diff --git a/src/Compression/CompressedReadBufferBase.h b/src/Compression/CompressedReadBufferBase.h index 3b822e540b9..074494afdbd 100644 --- a/src/Compression/CompressedReadBufferBase.h +++ b/src/Compression/CompressedReadBufferBase.h @@ -40,13 +40,15 @@ protected: size_t readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum, bool always_copy); size_t readCompressedDataBlockHold(size_t & size_decompressed, size_t & size_compressed_without_checksum); /// Decompress into memory pointed by `to` - void decompressTo(char * to, size_t size_decompressed, size_t size_compressed_without_checksum, UInt8 req_type=0); + void decompressTo(char * to, size_t size_decompressed, size_t size_compressed_without_checksum); /// This method can change location of `to` to avoid unnecessary copy if data is uncompressed. /// It is more efficient for compression codec NONE but not suitable if you want to decompress into specific location. - void decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum, UInt8 req_type=0); + void decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum); /// Flush all asynchronous decompress request void decompressFlush() const; + void setDecompressMode(ICompressionCodec::CodecMode mode); + public: /// 'compressed_in' could be initialized lazily, but before first call of 'readCompressedData'. explicit CompressedReadBufferBase(ReadBuffer * in = nullptr, bool allow_different_codecs_ = false); diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index 997ee7ec7e4..54519cd3a11 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -91,7 +91,7 @@ void CompressedReadBufferFromFile::seek(size_t offset_in_compressed_file, size_t size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) { size_t bytes_read = 0; - UInt8 req_type = 0; + ICompressionCodec::CodecMode decompress_mode = ICompressionCodec::CodecMode::Synchronous; bool read_tail = false; /// If there are unread bytes in the buffer, then we copy needed to `to`. @@ -108,13 +108,13 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) if (new_size_compressed) { - req_type = 1; + decompress_mode = ICompressionCodec::CodecMode::Asynchronous; } else { decompressFlush(); /// here switch to unhold block in compress_in, we must flush for previous blocks completely hold in compress_in new_size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum, false); - req_type = 0; + decompress_mode = ICompressionCodec::CodecMode::Synchronous; } size_compressed = 0; /// file_in no longer points to the end of the block in working_buffer. @@ -127,7 +127,8 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) /// need to skip some bytes in decompressed data (seek happened before readBig call). if (nextimpl_working_buffer_offset == 0 && size_decompressed + additional_size_at_the_end_of_buffer <= n - bytes_read) { - decompressTo(to + bytes_read, size_decompressed, size_compressed_without_checksum, req_type); //Async req + setDecompressMode(decompress_mode); + decompressTo(to + bytes_read, size_decompressed, size_compressed_without_checksum); bytes_read += size_decompressed; bytes += size_decompressed; } @@ -141,6 +142,7 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) assert(size_decompressed + additional_size_at_the_end_of_buffer > 0); memory.resize(size_decompressed + additional_size_at_the_end_of_buffer); working_buffer = Buffer(memory.data(), &memory[size_decompressed]); + setDecompressMode(ICompressionCodec::CodecMode::Synchronous); decompress(working_buffer, size_decompressed, size_compressed_without_checksum); /// Read partial data from first block. Won't run here at second block. @@ -160,8 +162,8 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) memory.resize(size_decompressed + additional_size_at_the_end_of_buffer); working_buffer = Buffer(memory.data(), &memory[size_decompressed]); - - decompress(working_buffer, size_decompressed, size_compressed_without_checksum, 1); + setDecompressMode(ICompressionCodec::CodecMode::Asynchronous); + decompress(working_buffer, size_decompressed, size_compressed_without_checksum); read_tail = true; break; } diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 4e2ff0c06ef..7fbcc0e67c2 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -291,11 +291,6 @@ uint8_t CompressionCodecDeflate::getMethodByte() const return static_cast(CompressionMethodByte::Deflate); } -bool CompressionCodecDeflate::isAsyncSupported() const -{ - return hwCodec->hwEnabled; -} - void CompressionCodecDeflate::updateHash(SipHash & hash) const { getCodecDesc()->updateTreeHash(hash); @@ -324,25 +319,30 @@ uint32_t CompressionCodecDeflate::doCompressData(const char * source, uint32_t s void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const { - uint32_t res = 0; - if (hwCodec->hwEnabled) - res = hwCodec->doDecompressData(source, source_size, dest, uncompressed_size); - if (0 == res) - swCodec->doDecompressData(source, source_size, dest, uncompressed_size); -} - -void CompressionCodecDeflate::doDecompressDataSW(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const -{ - return swCodec->doDecompressData(source, source_size, dest, uncompressed_size); -} - -void CompressionCodecDeflate::doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) -{ - uint32_t res = 0; - if (hwCodec->hwEnabled) - res = hwCodec->doDecompressDataReq(source, source_size, dest, uncompressed_size); - if (0 == res) - swCodec->doDecompressData(source, source_size, dest, uncompressed_size); + switch (getDecompressMode()) + { + case CodecMode::Synchronous: + { + uint32_t res = 0; + if (hwCodec->hwEnabled) + res = hwCodec->doDecompressData(source, source_size, dest, uncompressed_size); + if (0 == res) + swCodec->doDecompressData(source, source_size, dest, uncompressed_size); + break; + } + case CodecMode::Asynchronous: + { + uint32_t res = 0; + if (hwCodec->hwEnabled) + res = hwCodec->doDecompressDataReq(source, source_size, dest, uncompressed_size); + if (0 == res) + swCodec->doDecompressData(source, source_size, dest, uncompressed_size); + break; + } + case CodecMode::SoftwareFallback: + swCodec->doDecompressData(source, source_size, dest, uncompressed_size); + break; + } } void CompressionCodecDeflate::doDecompressDataFlush() diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index 70a4a9cb529..dc205234392 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -218,7 +218,6 @@ public: //~CompressionCodecDeflate() ; uint8_t getMethodByte() const override; void updateHash(SipHash & hash) const override; - bool isAsyncSupported() const override; protected: bool isCompression() const override @@ -232,8 +231,6 @@ protected: uint32_t doCompressData(const char * source, uint32_t source_size, char * dest) const override; uint32_t doCompressDataSW(const char * source, uint32_t source_size, char * dest) const; void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; - void doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) override; - void doDecompressDataSW(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; void doDecompressDataFlush() override; private: diff --git a/src/Compression/ICompressionCodec.cpp b/src/Compression/ICompressionCodec.cpp index 06bd7cc7558..cde60cc6d34 100644 --- a/src/Compression/ICompressionCodec.cpp +++ b/src/Compression/ICompressionCodec.cpp @@ -91,7 +91,7 @@ UInt32 ICompressionCodec::compress(const char * source, UInt32 source_size, char return header_size + compressed_bytes_written; } -UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, char * dest, UInt8 req_type) +UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, char * dest) { assert(source != nullptr && dest != nullptr); @@ -105,18 +105,7 @@ UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, ch throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Can't decompress data with codec byte {} using codec with byte {}", method, our_method); UInt32 decompressed_size = readDecompressedBlockSize(source); - switch (req_type) - { - case 0: - doDecompressData(&source[header_size], source_size - header_size, dest, decompressed_size); - break; - case 1: - doDecompressDataReq(&source[header_size], source_size - header_size, dest, decompressed_size); - break; - case 2: - doDecompressDataSW(&source[header_size], source_size - header_size, dest, decompressed_size); - break; - } + doDecompressData(&source[header_size], source_size - header_size, dest, decompressed_size); return decompressed_size; } @@ -124,6 +113,7 @@ UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, ch void ICompressionCodec::decompressFlush() { doDecompressDataFlush(); + decompressMode = CodecMode::Synchronous; } UInt32 ICompressionCodec::readCompressedBlockSize(const char * source) diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 03b5ca2b4e5..5c7eb5b235d 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -26,6 +26,13 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size); class ICompressionCodec : private boost::noncopyable { public: + enum class CodecMode + { + Synchronous, //synchronous request by default; + Asynchronous, //asynchronous request, must be used in pair with decompressFlush; + SoftwareFallback //Fallback to SW decompressor; + }; + virtual ~ICompressionCodec() = default; /// Byte which indicates codec in compressed file @@ -46,21 +53,21 @@ public: UInt32 compress(const char * source, UInt32 source_size, char * dest) const; /// Decompress bytes from compressed source to dest. Dest should preallocate memory; - // reqType is specific for HW decompressor: - //0 means synchronous request by default; - //1 means asynchronous request, must be used in pair with decompressFlush; - //2 means SW decompressor instead of HW - UInt32 decompress(const char * source, UInt32 source_size, char * dest, UInt8 req_type = 0); + UInt32 decompress(const char * source, UInt32 source_size, char * dest); + + CodecMode getDecompressMode() const + { + return decompressMode; + } + + void setDecompressMode(CodecMode mode) + { + decompressMode = mode; + } /// Flush all asynchronous request for decompression void decompressFlush(void); - /// Some codecs (QPL_deflate, for example) support asynchronous request - virtual bool isAsyncSupported() const - { - return false; - } - /// Number of bytes, that will be used to compress uncompressed_size bytes with current codec virtual UInt32 getCompressedReserveSize(UInt32 uncompressed_size) const { @@ -111,18 +118,6 @@ protected: /// Actually decompress data without header virtual void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const = 0; - /// Asynchronous decompression request to HW decompressor - virtual void doDecompressDataReq(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) - { - doDecompressData(source, source_size, dest, uncompressed_size); - } - - /// SW decompressor instead of HW - virtual void doDecompressDataSW(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const - { - doDecompressData(source, source_size, dest, uncompressed_size); - } - /// Flush asynchronous request for decompression virtual void doDecompressDataFlush() { @@ -132,6 +127,7 @@ protected: private: ASTPtr full_codec_desc; + CodecMode decompressMode{CodecMode::Synchronous}; }; } From 4b3020c9a539df08cf6571944a8f770833e4ab13 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jul 2022 00:33:13 +0200 Subject: [PATCH 079/659] Add a test for simdjson --- .../0_stateless/02353_simdjson_buffer_overflow.reference | 0 tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql | 4 ++++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02353_simdjson_buffer_overflow.reference create mode 100644 tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql diff --git a/tests/queries/0_stateless/02353_simdjson_buffer_overflow.reference b/tests/queries/0_stateless/02353_simdjson_buffer_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql b/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql new file mode 100644 index 00000000000..763c99a77c3 --- /dev/null +++ b/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql @@ -0,0 +1,4 @@ +SET max_execution_time = 3; +SET timeout_overflow_mode = 'break'; + +SELECT count() FROM system.numbers_mt WHERE NOT ignore(JSONExtract('{' || repeat('"a":"b",', rand() % 10) || '"c":"d"}', 'a', 'String')) FORMAT Null; From 7b262b62c24d090ee06af4adc6daf6f493320abe Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 6 Jul 2022 18:34:31 -0400 Subject: [PATCH 080/659] restore decompress to be const --- src/Compression/ICompressionCodec.cpp | 2 +- src/Compression/ICompressionCodec.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Compression/ICompressionCodec.cpp b/src/Compression/ICompressionCodec.cpp index cde60cc6d34..5fbbd6bef8b 100644 --- a/src/Compression/ICompressionCodec.cpp +++ b/src/Compression/ICompressionCodec.cpp @@ -91,7 +91,7 @@ UInt32 ICompressionCodec::compress(const char * source, UInt32 source_size, char return header_size + compressed_bytes_written; } -UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, char * dest) +UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, char * dest) const { assert(source != nullptr && dest != nullptr); diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 5c7eb5b235d..f536c643607 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -53,7 +53,7 @@ public: UInt32 compress(const char * source, UInt32 source_size, char * dest) const; /// Decompress bytes from compressed source to dest. Dest should preallocate memory; - UInt32 decompress(const char * source, UInt32 source_size, char * dest); + UInt32 decompress(const char * source, UInt32 source_size, char * dest) const; CodecMode getDecompressMode() const { From c6db15458a40a0e28bf2ac8bb39b9d5fcd24f8b8 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 6 Jul 2022 20:40:41 -0400 Subject: [PATCH 081/659] build utils/self-extracting-executable/compressor whenever we want to build compressed binary --- programs/CMakeLists.txt | 2 +- utils/CMakeLists.txt | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 2282daac90b..2c04ad98fa3 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -18,7 +18,7 @@ option (ENABLE_CLICKHOUSE_SERVER "Server mode (main mode)" ${ENABLE_CLICKHOUSE_A option (ENABLE_CLICKHOUSE_CLIENT "Client mode (interactive tui/shell that connects to the server)" ${ENABLE_CLICKHOUSE_ALL}) -if (CLICKHOUSE_SPLIT_BINARY OR NOT ENABLE_UTILS) +if (CLICKHOUSE_SPLIT_BINARY) option (ENABLE_CLICKHOUSE_SELF_EXTRACTING "Self-extracting executable" OFF) else () option (ENABLE_CLICKHOUSE_SELF_EXTRACTING "Self-extracting executable" ON) diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 3d1cc300417..3e09b3ecd64 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -9,6 +9,10 @@ else() endif() include(../cmake/limit_jobs.cmake) +if (ENABLE_CLICKHOUSE_SELF_EXTRACTING) + add_subdirectory (self-extracting-executable) +endif () + # Utils used in package add_subdirectory (config-processor) add_subdirectory (report) @@ -32,7 +36,6 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) add_subdirectory (check-mysql-binlog) add_subdirectory (keeper-bench) add_subdirectory (graphite-rollup) - add_subdirectory (self-extracting-executable) if (TARGET ch_contrib::nuraft) add_subdirectory (keeper-data-dumper) From 4cdc6604068a379d538bf70652fe5a4230e2d821 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 6 Jul 2022 21:37:11 -0400 Subject: [PATCH 082/659] change flush name to be more descriptive --- src/Compression/CompressedReadBufferBase.cpp | 4 ++-- src/Compression/CompressedReadBufferBase.h | 2 +- src/Compression/CompressedReadBufferFromFile.cpp | 4 ++-- src/Compression/CompressionCodecDeflate.cpp | 6 +++--- src/Compression/CompressionCodecDeflate.h | 4 ++-- src/Compression/ICompressionCodec.cpp | 4 ++-- src/Compression/ICompressionCodec.h | 6 +++--- 7 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 7eff176a6d2..5a51d8377d5 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -318,11 +318,11 @@ void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_d codec->decompress(compressed_buffer, size_compressed_without_checksum, to.begin()); } -void CompressedReadBufferBase::decompressFlush() const +void CompressedReadBufferBase::flushAsynchronousDecompressRequests() const { if (codec) { - codec->decompressFlush(); + codec->flushAsynchronousDecompressRequests(); } } diff --git a/src/Compression/CompressedReadBufferBase.h b/src/Compression/CompressedReadBufferBase.h index 074494afdbd..85f49bd3510 100644 --- a/src/Compression/CompressedReadBufferBase.h +++ b/src/Compression/CompressedReadBufferBase.h @@ -46,7 +46,7 @@ protected: /// It is more efficient for compression codec NONE but not suitable if you want to decompress into specific location. void decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum); /// Flush all asynchronous decompress request - void decompressFlush() const; + void flushAsynchronousDecompressRequests() const; void setDecompressMode(ICompressionCodec::CodecMode mode); public: diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index 54519cd3a11..0b9dfa7be9e 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -112,7 +112,7 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) } else { - decompressFlush(); /// here switch to unhold block in compress_in, we must flush for previous blocks completely hold in compress_in + flushAsynchronousDecompressRequests(); /// here switch to unhold block in compress_in, we must flush for previous blocks completely hold in compress_in new_size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum, false); decompress_mode = ICompressionCodec::CodecMode::Synchronous; } @@ -169,7 +169,7 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) } } - decompressFlush(); + flushAsynchronousDecompressRequests(); if (read_tail) { diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 7fbcc0e67c2..99f710b40b8 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -159,7 +159,7 @@ uint32_t HardwareCodecDeflate::doDecompressDataReq(const char * source, uint32_t } } -void HardwareCodecDeflate::doDecompressDataFlush() +void HardwareCodecDeflate::doFlushAsynchronousDecompressRequests() { uint32_t job_id = 0; qpl_job * job_ptr = nullptr; @@ -345,10 +345,10 @@ void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t sou } } -void CompressionCodecDeflate::doDecompressDataFlush() +void CompressionCodecDeflate::doFlushAsynchronousDecompressRequests() { if (hwCodec->hwEnabled) - hwCodec->doDecompressDataFlush(); + hwCodec->doFlushAsynchronousDecompressRequests(); } void registerCodecDeflate(CompressionCodecFactory & factory) { diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index dc205234392..b1c20f71496 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -205,7 +205,7 @@ public: uint32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const; uint32_t doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const; uint32_t doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); - void doDecompressDataFlush(); + void doFlushAsynchronousDecompressRequests(); private: std::map jobDecompAsyncMap; @@ -231,7 +231,7 @@ protected: uint32_t doCompressData(const char * source, uint32_t source_size, char * dest) const override; uint32_t doCompressDataSW(const char * source, uint32_t source_size, char * dest) const; void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; - void doDecompressDataFlush() override; + void doFlushAsynchronousDecompressRequests() override; private: uint32_t getMaxCompressedDataSize(uint32_t uncompressed_size) const override; diff --git a/src/Compression/ICompressionCodec.cpp b/src/Compression/ICompressionCodec.cpp index 5fbbd6bef8b..7aaa21776e9 100644 --- a/src/Compression/ICompressionCodec.cpp +++ b/src/Compression/ICompressionCodec.cpp @@ -110,9 +110,9 @@ UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, ch return decompressed_size; } -void ICompressionCodec::decompressFlush() +void ICompressionCodec::flushAsynchronousDecompressRequests() { - doDecompressDataFlush(); + doFlushAsynchronousDecompressRequests(); decompressMode = CodecMode::Synchronous; } diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index f536c643607..84162856e60 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -29,7 +29,7 @@ public: enum class CodecMode { Synchronous, //synchronous request by default; - Asynchronous, //asynchronous request, must be used in pair with decompressFlush; + Asynchronous, //asynchronous request, must be used in pair with flushAsynchronousDecompressRequests; SoftwareFallback //Fallback to SW decompressor; }; @@ -66,7 +66,7 @@ public: } /// Flush all asynchronous request for decompression - void decompressFlush(void); + void flushAsynchronousDecompressRequests(); /// Number of bytes, that will be used to compress uncompressed_size bytes with current codec virtual UInt32 getCompressedReserveSize(UInt32 uncompressed_size) const @@ -119,7 +119,7 @@ protected: virtual void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const = 0; /// Flush asynchronous request for decompression - virtual void doDecompressDataFlush() + virtual void doFlushAsynchronousDecompressRequests() { } /// Construct and set codec description from codec name and arguments. Must be called in codec constructor. From cda081dc57ad3656e228c0465ac52bcf99239a11 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 7 Jul 2022 09:41:52 +0800 Subject: [PATCH 083/659] update codes --- src/Storages/Hive/HiveFile.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 92eee681ea1..c4d4a28c88f 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -82,6 +82,8 @@ std::optional IHiveFile::getRows() if (!has_init_rows) { std::lock_guard lock(mutex); + if (has_init_rows) + return rows; rows = getRowsImpl(); has_init_rows = true; } @@ -93,6 +95,8 @@ void IHiveFile::loadFileMinMaxIndex() if (file_minmax_idx_loaded) return; std::lock_guard lock(mutex); + if (file_minmax_idx_loaded) + return; loadFileMinMaxIndexImpl(); file_minmax_idx_loaded = true; } @@ -102,6 +106,8 @@ void IHiveFile::loadSplitMinMaxIndexes() if (split_minmax_idxes_loaded) return; std::lock_guard lock(mutex); + if (split_minmax_idxes_loaded) + return; loadSplitMinMaxIndexesImpl(); split_minmax_idxes_loaded = true; } From 337f379bc3b7e677e004e6a06e3c42e67b91a1f1 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 7 Jul 2022 09:45:49 +0800 Subject: [PATCH 084/659] update codes --- src/Storages/Hive/HiveFile.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index c4d4a28c88f..112798fea4e 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -82,10 +82,11 @@ std::optional IHiveFile::getRows() if (!has_init_rows) { std::lock_guard lock(mutex); - if (has_init_rows) - return rows; - rows = getRowsImpl(); - has_init_rows = true; + if (!has_init_rows) + { + rows = getRowsImpl(); + has_init_rows = true; + } } return rows; } From 8a3f124982c004492039e4fa76f91f6d0a034123 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 6 Jul 2022 22:01:21 -0400 Subject: [PATCH 085/659] add self-extracting to clickhouse-bundle --- programs/CMakeLists.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 2c04ad98fa3..dfbcb3ede03 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -434,6 +434,9 @@ else () endif () set (CLICKHOUSE_BUNDLE) + if (ENABLE_CLICKHOUSE_SELF_EXTRACTING) + list(APPEND CLICKHOUSE_BUNDLE self-extracting) + endif () if (ENABLE_CLICKHOUSE_SERVER) add_custom_target (clickhouse-server ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-server DEPENDS clickhouse) install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-server" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) From 3422c0479da71b805ca244ea40362edb56d211d6 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 6 Jul 2022 23:31:22 -0400 Subject: [PATCH 086/659] add more comments for flush --- src/Compression/ICompressionCodec.h | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 84162856e60..423a2d58212 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -26,11 +26,18 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size); class ICompressionCodec : private boost::noncopyable { public: + /// Three kinds of codec mode: + /// Synchronous mode which is commonly used by default; + /// --- For the codec with HW decompressor, it means submit request to HW and busy wait till complete. + /// Asynchronous mode which required HW decompressor support; + /// --- For the codec with HW decompressor, it means submit request to HW and return immeditately. + /// --- Must be used in pair with flushAsynchronousDecompressRequests. + /// SoftwareFallback mode is exclusively defined for the codec with HW decompressor, enable its capability of "fallback to SW codec". enum class CodecMode { - Synchronous, //synchronous request by default; - Asynchronous, //asynchronous request, must be used in pair with flushAsynchronousDecompressRequests; - SoftwareFallback //Fallback to SW decompressor; + Synchronous, + Asynchronous, + SoftwareFallback }; virtual ~ICompressionCodec() = default; @@ -65,7 +72,12 @@ public: decompressMode = mode; } - /// Flush all asynchronous request for decompression + /// Flush result for previous asynchronous decompression requests. + /// This function must be called following several requests offload to HW. + /// To make sure asynchronous results have been flushed into target buffer completely. + /// Meanwhile, source and target buffer for decompression should not be overwritten until this function execute completely. + /// Otherwise it would conflict with HW offloading and cause exception. + /// For QPL deflate, it support the maximum number of requests equal to DeflateJobHWPool::jobPoolSize void flushAsynchronousDecompressRequests(); /// Number of bytes, that will be used to compress uncompressed_size bytes with current codec From 6fa036f6ee27201f8952d89f464534306eea4338 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 6 Jul 2022 23:48:09 -0400 Subject: [PATCH 087/659] remove qpl from clickhouse-keeper --- programs/keeper/CMakeLists.txt | 10 +--------- src/Compression/CompressionFactory.cpp | 2 +- 2 files changed, 2 insertions(+), 10 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index bfe034db87a..4da7031ec19 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -75,7 +75,6 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecMultiple.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecNone.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecZSTD.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecDeflate.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionFactory.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/ICompressionCodec.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/LZ4_decompress_faster.cpp @@ -104,11 +103,6 @@ if (BUILD_STANDALONE_KEEPER) # Remove some redundant dependencies target_compile_definitions (clickhouse-keeper PRIVATE -DKEEPER_STANDALONE_BUILD) - if (ENABLE_QPL) - set_source_files_properties( - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecDeflate.cpp - PROPERTIES COMPILE_FLAGS "-mwaitpkg") - endif () target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_SOURCE_DIR}/../../src") # uses includes from src directory target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/../../src/Core/include") # uses some includes from core target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/../../src") # uses some includes from common @@ -132,9 +126,7 @@ if (BUILD_STANDALONE_KEEPER) ${LINK_RESOURCE_LIB_STANDALONE_KEEPER} ) - if (TARGET ch_contrib::qpl) - target_link_libraries(clickhouse-keeper PRIVATE ch_contrib::qpl ${LINK_RESOURCE_LIB_STANDALONE_KEEPER}) - endif () + add_dependencies(clickhouse-keeper clickhouse_keeper_configs) set_target_properties(clickhouse-keeper PROPERTIES RUNTIME_OUTPUT_DIRECTORY ../) diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 78351479c89..8667f8e515a 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -188,10 +188,10 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecZSTD(*this); registerCodecLZ4HC(*this); registerCodecMultiple(*this); +#ifndef KEEPER_STANDALONE_BUILD #ifdef ENABLE_QPL_COMPRESSION registerCodecDeflate(*this); #endif -#ifndef KEEPER_STANDALONE_BUILD registerCodecDelta(*this); registerCodecT64(*this); registerCodecDoubleDelta(*this); From ee838224fdceeb4149a5c6d8fea42aeee03deaf5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jul 2022 08:17:24 +0300 Subject: [PATCH 088/659] Update 02353_simdjson_buffer_overflow.sql --- tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql b/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql index 763c99a77c3..0861ae48f37 100644 --- a/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql +++ b/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + SET max_execution_time = 3; SET timeout_overflow_mode = 'break'; From 19bd3d8d9ffd415f198cd575f30eb0d0e2e9ffe9 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 7 Jul 2022 16:02:38 +0800 Subject: [PATCH 089/659] Nullable partition pruning fix --- src/Storages/MergeTree/PartitionPruner.cpp | 12 ++++++++++++ .../02353_partition_prune_nullable_key.reference | 1 + .../02353_partition_prune_nullable_key.sql | 9 +++++++++ 3 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02353_partition_prune_nullable_key.reference create mode 100644 tests/queries/0_stateless/02353_partition_prune_nullable_key.sql diff --git a/src/Storages/MergeTree/PartitionPruner.cpp b/src/Storages/MergeTree/PartitionPruner.cpp index e85e8e833f8..dce52ef64e9 100644 --- a/src/Storages/MergeTree/PartitionPruner.cpp +++ b/src/Storages/MergeTree/PartitionPruner.cpp @@ -15,9 +15,21 @@ bool PartitionPruner::canBePruned(const DataPart & part) { const auto & partition_value = part.partition.value; std::vector index_value(partition_value.begin(), partition_value.end()); + for (auto & field : index_value) + { + // NULL_LAST + if (field.isNull()) + field = POSITIVE_INFINITY; + } is_valid = partition_condition.mayBeTrueInRange( partition_value.size(), index_value.data(), index_value.data(), partition_key.data_types); partition_filter_map.emplace(partition_id, is_valid); + if (!is_valid) + { + WriteBufferFromOwnString buf; + part.partition.serializeText(part.storage, buf, FormatSettings{}); + LOG_TRACE(&Poco::Logger::get("PartitionPruner"), "Partition {} gets pruned", buf.str()); + } } return !is_valid; } diff --git a/tests/queries/0_stateless/02353_partition_prune_nullable_key.reference b/tests/queries/0_stateless/02353_partition_prune_nullable_key.reference new file mode 100644 index 00000000000..dec7d2fabd2 --- /dev/null +++ b/tests/queries/0_stateless/02353_partition_prune_nullable_key.reference @@ -0,0 +1 @@ +\N diff --git a/tests/queries/0_stateless/02353_partition_prune_nullable_key.sql b/tests/queries/0_stateless/02353_partition_prune_nullable_key.sql new file mode 100644 index 00000000000..5a5109c3140 --- /dev/null +++ b/tests/queries/0_stateless/02353_partition_prune_nullable_key.sql @@ -0,0 +1,9 @@ +drop table if exists n; + +create table n(nc Nullable(int)) engine = MergeTree order by (tuple()) partition by (nc) settings allow_nullable_key = 1; + +insert into n values (null); + +select * from n where nc is null; + +drop table n; From a8b9ad651f2b866b2a514bdbd9ade44443a04174 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 7 Jul 2022 12:29:23 +0000 Subject: [PATCH 090/659] Add compatibility setting --- src/Core/BaseSettings.h | 17 ++++- src/Core/Settings.cpp | 39 +++++++++++ src/Core/Settings.h | 9 +++ src/Core/SettingsChangesHistory.h | 105 ++++++++++++++++++++++++++++++ 4 files changed, 169 insertions(+), 1 deletion(-) create mode 100644 src/Core/SettingsChangesHistory.h diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index a4ddc6571ed..d8b41109d5e 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -43,9 +43,14 @@ class BaseSettings : public TTraits::Data { using CustomSettingMap = std::unordered_map, SettingFieldCustom>>; public: + BaseSettings() = default; + BaseSettings(const BaseSettings &) = default; + BaseSettings & operator=(const BaseSettings &) = default; + virtual ~BaseSettings() = default; + using Traits = TTraits; - void set(const std::string_view & name, const Field & value); + virtual void set(const std::string_view & name, const Field & value); Field get(const std::string_view & name) const; void setString(const std::string_view & name, const String & value); @@ -62,6 +67,8 @@ public: /// Resets all the settings to their default values. void resetToDefault(); + /// Resets specified setting to its default value. + void resetToDefault(const std::string_view & name); bool has(const std::string_view & name) const { return hasBuiltin(name) || hasCustom(name); } static bool hasBuiltin(const std::string_view & name); @@ -315,6 +322,14 @@ void BaseSettings::resetToDefault() custom_settings_map.clear(); } +template +void BaseSettings::resetToDefault(const std::string_view & name) +{ + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + accessor.resetValueToDefault(*this, index); +} + template bool BaseSettings::hasBuiltin(const std::string_view & name) { diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 5251569505e..65c0adfc4b4 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -1,5 +1,6 @@ #include "Settings.h" +#include #include #include #include @@ -145,6 +146,44 @@ std::vector Settings::getAllRegisteredNames() const return all_settings; } +void Settings::set(const std::string_view & name, const Field & value) +{ + BaseSettings::set(name, value); + + if (name == "compatibility") + applyCompatibilitySetting(); + /// If we change setting that was changed by compatibility setting before + /// we should remove it from settings_changed_by_compatibility_setting, + /// otherwise the next time we will change compatibility setting + /// this setting will be changed too (and we don't want it). + else if (settings_changed_by_compatibility_setting.contains(name)) + settings_changed_by_compatibility_setting.erase(name); +} + +void Settings::applyCompatibilitySetting() +{ + /// First, revert all changes applied by previous compatibility setting + for (const auto & setting_name : settings_changed_by_compatibility_setting) + resetToDefault(setting_name); + + settings_changed_by_compatibility_setting.clear(); + String compatibility = getString("compatibility"); + /// If setting value is empty, we don't need to change settings + if (compatibility.empty()) + return; + + ClickHouseVersion version(compatibility); + for (const auto & [setting_name, history] : settings_changes_history) + { + /// If this setting was changed manually, we don't change it + if (isChanged(setting_name)) + continue; + + BaseSettings::set(setting_name, history.getValueForVersion(version)); + settings_changed_by_compatibility_setting.insert(setting_name); + } +} + IMPLEMENT_SETTINGS_TRAITS(FormatFactorySettingsTraits, FORMAT_FACTORY_SETTINGS) } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 91ff139c87c..c00f04356bf 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -593,6 +593,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, allow_deprecated_database_ordinary, false, "Allow to create databases with deprecated Ordinary engine", 0) \ M(Bool, allow_deprecated_syntax_for_merge_tree, false, "Allow to create *MergeTree tables with deprecated engine definition syntax", 0) \ \ + M(String, compatibility, "", "Azaz", 0) \ + \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ @@ -819,6 +821,13 @@ struct Settings : public BaseSettings, public IHints<2, Settings void addProgramOption(boost::program_options::options_description & options, const SettingFieldRef & field); void addProgramOptionAsMultitoken(boost::program_options::options_description & options, const SettingFieldRef & field); + + void set(const std::string_view & name, const Field & value) override; + +private: + void applyCompatibilitySetting(); + + std::unordered_set settings_changed_by_compatibility_setting; }; /* diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h new file mode 100644 index 00000000000..7dcd510d653 --- /dev/null +++ b/src/Core/SettingsChangesHistory.h @@ -0,0 +1,105 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +class ClickHouseVersion +{ +public: + ClickHouseVersion(const String & version) + { + Strings split; + boost::split(split, version, [](char c){ return c == '.'; }); + components.reserve(split.size()); + for (const auto & split_element : split) + { + size_t component; + if (!tryParse(component, split_element)) + throw Exception{ErrorCodes::BAD_ARGUMENTS, "Cannot parse ClickHouse version here: {}", version}; + components.push_back(component); + } + } + + ClickHouseVersion(const char * version) : ClickHouseVersion(String(version)) {} + + bool operator<(const ClickHouseVersion & other) const + { + return components < other.components; + } + +private: + std::vector components; +}; + +struct SettingChangesHistory +{ + struct Change + { + Change(const Field & value_, const ClickHouseVersion & version_) : value(value_), version(version_) {} + + Field value; + ClickHouseVersion version; + }; + + SettingChangesHistory(const Field & initial_value_, const std::vector & changes_) : initial_value(initial_value_), changes(changes_) {} + + Field getValueForVersion(const ClickHouseVersion & version) const + { + Field value = initial_value; + for (const auto & change : changes) + { + if (version < change.version) + return value; + value = change.value; + } + return value; + } + + Field initial_value; + std::vector changes; +}; + +const std::unordered_map settings_changes_history = +{ + {"enable_positional_arguments", {false, {{true, "22.7"}}}}, + {"output_format_json_named_tuples_as_objects", {false, {{true, "22.6"}}}}, + {"memory_overcommit_ratio_denominator", {0, {{1073741824, "22.5"}}}}, + {"memory_overcommit_ratio_denominator_for_user", {0, {{1073741824, "22.5"}}}}, + {"allow_settings_after_format_in_insert", {true, {{false, "22.4"}}}}, + {"cast_ipv4_ipv6_default_on_conversion_error", {true, {{false, "22.3"}}}}, + {"stream_like_engine_allow_direct_select", {true, {{false, "21.12"}}}}, + {"output_format_decimal_trailing_zeros", {true, {{false, "21.9"}}}}, + {"use_hedged_requests", {false, {{true, "21.9"}}}}, + {"legacy_column_name_of_tuple_literal", {true, {{false, "21.7"}}}}, + {"async_socket_for_remote", {true, {{false, "21.3"}, {true, "21.5"}}}}, + {"optimize_normalize_count_variants", {false, {{true, "21.3"}}}}, + {"normalize_function_names", {false, {{true, "21.3"}}}}, + {"enable_global_with_statement", {false, {{true, "21.2"}}}}, + {"insert_quorum_parallel", {false, {{true, "21.1"}}}}, + {"input_format_null_as_default", {false, {{true, "21.1"}}}}, + {"optimize_on_insert", {false, {{true, "21.1"}}}}, + {"use_compact_format_in_distributed_parts_names", {false, {{true, "21.1"}}}}, + {"format_regexp_escaping_rule", {"Escaped", {{"Raw", "20.10"}}}}, + {"show_table_uuid_in_table_create_query_if_not_nil", {true, {{false, "20.7"}}}}, + {"input_format_with_names_use_header", {false, {{true, "20.5"}}}}, + {"allow_suspicious_codecs", {true, {{false, "20.5"}}}}, + {"validate_polygons", {false, {{true, "20.4"}}}}, + {"enable_scalar_subquery_optimization", {false, {{true, "19.18"}}}}, + {"any_join_distinct_right_table_keys", {true, {{false, "19.14"}}}}, + {"input_format_defaults_for_omitted_fields", {false, {{true, "19.12"}}}}, + {"max_partitions_per_insert_block", {0, {{100, "19.5"}}}}, + {"enable_optimize_predicate_expression", {0, {{1, "18.12.17"}}}}, +}; + + +} From 43c5a68d14af6afa39df76abf8aae60e7cd94dde Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 7 Jul 2022 10:04:17 -0400 Subject: [PATCH 091/659] Cosmetic work --- src/Compression/CompressionCodecDeflate.h | 7 +------ src/Compression/ICompressionCodec.h | 7 ++++--- 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index b1c20f71496..b6cd292b0c4 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -1,13 +1,8 @@ #pragma once -#include -#include #include -#include -#include -#include #include -#include + namespace Poco { class Logger; diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 423a2d58212..16574cccd47 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -62,11 +62,13 @@ public: /// Decompress bytes from compressed source to dest. Dest should preallocate memory; UInt32 decompress(const char * source, UInt32 source_size, char * dest) const; + /// Get current decompression mode CodecMode getDecompressMode() const { return decompressMode; } + /// if set mode to CodecMode::Asynchronous, must be followed with flushAsynchronousDecompressRequests void setDecompressMode(CodecMode mode) { decompressMode = mode; @@ -131,9 +133,8 @@ protected: virtual void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const = 0; /// Flush asynchronous request for decompression - virtual void doFlushAsynchronousDecompressRequests() - { - } + virtual void doFlushAsynchronousDecompressRequests(){} + /// Construct and set codec description from codec name and arguments. Must be called in codec constructor. void setCodecDescription(const String & name, const ASTs & arguments = {}); From 85eae40765c25bde8d3f63acfa20bb796b5ec9d2 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 7 Jul 2022 10:26:57 -0400 Subject: [PATCH 092/659] remove inline due to negligible benefits --- src/Compression/CompressionCodecDeflate.h | 24 +++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index b6cd292b0c4..b924abb0dcc 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -23,11 +23,11 @@ public: static std::atomic_bool jobLock[jobPoolSize]; bool jobPoolEnabled; - bool ALWAYS_INLINE jobPoolReady() + bool jobPoolReady() { return jobPoolEnabled; } - qpl_job * ALWAYS_INLINE acquireJob(uint32_t * job_id) + qpl_job * acquireJob(uint32_t * job_id) { if (jobPoolEnabled) { @@ -50,7 +50,7 @@ public: return nullptr; } } - qpl_job * ALWAYS_INLINE releaseJob(uint32_t job_id) + qpl_job * releaseJob(uint32_t job_id) { if (jobPoolEnabled) { @@ -63,7 +63,7 @@ public: return nullptr; } } - qpl_job * ALWAYS_INLINE getJobPtr(uint32_t job_id) + qpl_job * getJobPtr(uint32_t job_id) { if (jobPoolEnabled) { @@ -77,7 +77,7 @@ public: } private: - size_t ALWAYS_INLINE random(uint32_t pool_size) + size_t random(uint32_t pool_size) { size_t tsc = 0; unsigned lo, hi; @@ -86,7 +86,7 @@ private: return (static_cast((tsc * 44485709377909ULL) >> 4)) % pool_size; } - int32_t ALWAYS_INLINE get_job_size_helper() + int32_t get_job_size_helper() { static uint32_t size = 0; if (size == 0) @@ -100,7 +100,7 @@ private: return size; } - int32_t ALWAYS_INLINE init_job_helper(qpl_job * qpl_job_ptr) + int32_t init_job_helper(qpl_job * qpl_job_ptr) { if (qpl_job_ptr == nullptr) { @@ -114,7 +114,7 @@ private: return 0; } - int32_t ALWAYS_INLINE initJobPool() + int32_t initJobPool() { static bool initialized = false; @@ -137,13 +137,13 @@ private: return 0; } - bool ALWAYS_INLINE tryLockJob(size_t index) + bool tryLockJob(size_t index) { bool expected = false; return jobLock[index].compare_exchange_strong(expected, true); } - void ALWAYS_INLINE destroyJobPool() + void destroyJobPool() { const uint32_t size = get_job_size_helper(); for (uint32_t i = 0; i < jobPoolSize && size > 0; ++i) @@ -167,10 +167,10 @@ private: ReleaseJobObjectGuard() = delete; public: - ALWAYS_INLINE ReleaseJobObjectGuard(const uint32_t i) : index(i) + ReleaseJobObjectGuard(const uint32_t i) : index(i) { } - ALWAYS_INLINE ~ReleaseJobObjectGuard() + ~ReleaseJobObjectGuard() { jobLock[index].store(false); } From 768f1cb172e1a114b93dbb53e61e49668bd6f6b9 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 7 Jul 2022 15:00:06 +0000 Subject: [PATCH 093/659] Add tests --- src/Core/SettingsChangesHistory.h | 2 ++ .../02324_compatibility_setting.reference | 19 +++++++++++++ .../02324_compatibility_setting.sh | 28 +++++++++++++++++++ .../02325_compatibility_setting_2.reference | 8 ++++++ .../02325_compatibility_setting_2.sql | 13 +++++++++ 5 files changed, 70 insertions(+) create mode 100644 tests/queries/0_stateless/02324_compatibility_setting.reference create mode 100755 tests/queries/0_stateless/02324_compatibility_setting.sh create mode 100644 tests/queries/0_stateless/02325_compatibility_setting_2.reference create mode 100644 tests/queries/0_stateless/02325_compatibility_setting_2.sql diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 7dcd510d653..8ccb6ebf86f 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -77,6 +77,8 @@ const std::unordered_map settings_changes_history {"memory_overcommit_ratio_denominator_for_user", {0, {{1073741824, "22.5"}}}}, {"allow_settings_after_format_in_insert", {true, {{false, "22.4"}}}}, {"cast_ipv4_ipv6_default_on_conversion_error", {true, {{false, "22.3"}}}}, + {"input_format_ipv4_default_on_conversion_error", {true, {{false, "22.3"}}}}, + {"input_format_ipv6_default_on_conversion_error", {true, {{false, "22.3"}}}}, {"stream_like_engine_allow_direct_select", {true, {{false, "21.12"}}}}, {"output_format_decimal_trailing_zeros", {true, {{false, "21.9"}}}}, {"use_hedged_requests", {false, {{true, "21.9"}}}}, diff --git a/tests/queries/0_stateless/02324_compatibility_setting.reference b/tests/queries/0_stateless/02324_compatibility_setting.reference new file mode 100644 index 00000000000..e3a9ed7a73e --- /dev/null +++ b/tests/queries/0_stateless/02324_compatibility_setting.reference @@ -0,0 +1,19 @@ +allow_settings_after_format_in_insert +22.3 +1 +1 +22.4 +0 +22.5 +0 +async_socket_for_remote +21.2 +1 +21.3 +0 +21.4 +0 +21.5 +1 +21.6 +1 diff --git a/tests/queries/0_stateless/02324_compatibility_setting.sh b/tests/queries/0_stateless/02324_compatibility_setting.sh new file mode 100755 index 00000000000..043f045d9be --- /dev/null +++ b/tests/queries/0_stateless/02324_compatibility_setting.sh @@ -0,0 +1,28 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo "allow_settings_after_format_in_insert" +echo "22.3" +$CLICKHOUSE_CLIENT --compatibility=22.3 -q "select value from system.settings where name='allow_settings_after_format_in_insert'" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&compatibility=22.3" -d "select value from system.settings where name='allow_settings_after_format_in_insert'" +echo "22.4" +$CLICKHOUSE_CLIENT --compatibility=22.4 -q "select value from system.settings where name='allow_settings_after_format_in_insert'" +echo "22.5" +$CLICKHOUSE_CLIENT --compatibility=22.5 -q "select value from system.settings where name='allow_settings_after_format_in_insert'" + + +echo "async_socket_for_remote" +echo "21.2" +$CLICKHOUSE_CLIENT --compatibility=21.2 -q "select value from system.settings where name='async_socket_for_remote'" +echo "21.3" +$CLICKHOUSE_CLIENT --compatibility=21.3 -q "select value from system.settings where name='async_socket_for_remote'" +echo "21.4" +$CLICKHOUSE_CLIENT --compatibility=21.4 -q "select value from system.settings where name='async_socket_for_remote'" +echo "21.5" +$CLICKHOUSE_CLIENT --compatibility=21.5 -q "select value from system.settings where name='async_socket_for_remote'" +echo "21.6" +$CLICKHOUSE_CLIENT --compatibility=21.6 -q "select value from system.settings where name='async_socket_for_remote'" + diff --git a/tests/queries/0_stateless/02325_compatibility_setting_2.reference b/tests/queries/0_stateless/02325_compatibility_setting_2.reference new file mode 100644 index 00000000000..9eed1825cc8 --- /dev/null +++ b/tests/queries/0_stateless/02325_compatibility_setting_2.reference @@ -0,0 +1,8 @@ +0 +1 +0 +1 +0 +1 +1 +1 diff --git a/tests/queries/0_stateless/02325_compatibility_setting_2.sql b/tests/queries/0_stateless/02325_compatibility_setting_2.sql new file mode 100644 index 00000000000..5ce0bf1ef8b --- /dev/null +++ b/tests/queries/0_stateless/02325_compatibility_setting_2.sql @@ -0,0 +1,13 @@ +select value from system.settings where name='allow_settings_after_format_in_insert'; +select value from system.settings where name='allow_settings_after_format_in_insert' settings compatibility='22.3'; +select value from system.settings where name='allow_settings_after_format_in_insert'; +set compatibility = '22.3'; +select value from system.settings where name='allow_settings_after_format_in_insert'; +set compatibility = '22.4'; +select value from system.settings where name='allow_settings_after_format_in_insert'; +set allow_settings_after_format_in_insert=1; +select value from system.settings where name='allow_settings_after_format_in_insert'; +set compatibility = '22.4'; +select value from system.settings where name='allow_settings_after_format_in_insert'; +set compatibility = '22.3'; +select value from system.settings where name='allow_settings_after_format_in_insert'; From 5f7002eafcf55edd91eb9cbd09af0a7ff6dfdab2 Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Thu, 7 Jul 2022 09:08:46 -0700 Subject: [PATCH 094/659] Include boringssl header files --- contrib/thrift-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index e4ed25b27b8..3454b1b87e3 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -92,5 +92,5 @@ include_directories("${CMAKE_CURRENT_BINARY_DIR}") add_library(_thrift ${thriftcpp_SOURCES} ${thriftcpp_threads_SOURCES}) add_library(ch_contrib::thrift ALIAS _thrift) -target_include_directories(_thrift SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src" ${CMAKE_CURRENT_BINARY_DIR}) +target_include_directories(_thrift SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src" ${CMAKE_CURRENT_BINARY_DIR} "${ClickHouse_SOURCE_DIR}/contrib/boringssl/include") target_link_libraries (_thrift PUBLIC boost::headers_only) From ad82bb80ed70a1b5dd6eab2727ca35c510d8f645 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 7 Jul 2022 12:10:06 -0400 Subject: [PATCH 095/659] Cosmetic: Constants to be UPPERCASE_AND_UNDERSCORE --- src/Compression/CompressionCodecDeflate.cpp | 4 +-- src/Compression/CompressionCodecDeflate.h | 30 ++++++++++----------- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 99f710b40b8..405df9d840b 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -16,8 +16,8 @@ namespace ErrorCodes extern const int CANNOT_DECOMPRESS; } //DeflateJobHWPool -qpl_job * DeflateJobHWPool::jobPool[jobPoolSize]; -std::atomic_bool DeflateJobHWPool::jobLock[jobPoolSize]; +qpl_job * DeflateJobHWPool::jobPool[JOB_POOL_SIZE]; +std::atomic_bool DeflateJobHWPool::jobLocks[JOB_POOL_SIZE]; DeflateJobHWPool & DeflateJobHWPool::instance() { diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index b924abb0dcc..702c0a82ac4 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -17,10 +17,10 @@ public: DeflateJobHWPool(); ~DeflateJobHWPool(); static DeflateJobHWPool & instance(); - static constexpr auto jobPoolSize = 1024; + static constexpr auto JOB_POOL_SIZE = 1024; static constexpr qpl_path_t PATH = qpl_path_hardware; - static qpl_job * jobPool[jobPoolSize]; - static std::atomic_bool jobLock[jobPoolSize]; + static qpl_job * jobPool[JOB_POOL_SIZE]; + static std::atomic_bool jobLocks[JOB_POOL_SIZE]; bool jobPoolEnabled; bool jobPoolReady() @@ -32,17 +32,17 @@ public: if (jobPoolEnabled) { uint32_t retry = 0; - auto index = random(jobPoolSize); + auto index = random(JOB_POOL_SIZE); while (tryLockJob(index) == false) { - index = random(jobPoolSize); + index = random(JOB_POOL_SIZE); retry++; - if (retry > jobPoolSize) + if (retry > JOB_POOL_SIZE) { return nullptr; } } - *job_id = jobPoolSize - index; + *job_id = JOB_POOL_SIZE - index; return jobPool[index]; } else @@ -54,7 +54,7 @@ public: { if (jobPoolEnabled) { - uint32_t index = jobPoolSize - job_id; + uint32_t index = JOB_POOL_SIZE - job_id; ReleaseJobObjectGuard _(index); return jobPool[index]; } @@ -67,7 +67,7 @@ public: { if (jobPoolEnabled) { - uint32_t index = jobPoolSize - job_id; + uint32_t index = JOB_POOL_SIZE - job_id; return jobPool[index]; } else @@ -123,14 +123,14 @@ private: const int32_t size = get_job_size_helper(); if (size < 0) return -1; - for (int i = 0; i < jobPoolSize; ++i) + for (int i = 0; i < JOB_POOL_SIZE; ++i) { jobPool[i] = nullptr; qpl_job * qpl_job_ptr = reinterpret_cast(new uint8_t[size]); if (init_job_helper(qpl_job_ptr) < 0) return -1; jobPool[i] = qpl_job_ptr; - jobLock[i].store(false); + jobLocks[i].store(false); } initialized = true; } @@ -140,13 +140,13 @@ private: bool tryLockJob(size_t index) { bool expected = false; - return jobLock[index].compare_exchange_strong(expected, true); + return jobLocks[index].compare_exchange_strong(expected, true); } void destroyJobPool() { const uint32_t size = get_job_size_helper(); - for (uint32_t i = 0; i < jobPoolSize && size > 0; ++i) + for (uint32_t i = 0; i < JOB_POOL_SIZE && size > 0; ++i) { while (tryLockJob(i) == false) { @@ -157,7 +157,7 @@ private: delete[] jobPool[i]; } jobPool[i] = nullptr; - jobLock[i].store(false); + jobLocks[i].store(false); } } @@ -172,7 +172,7 @@ private: } ~ReleaseJobObjectGuard() { - jobLock[index].store(false); + jobLocks[index].store(false); } }; Poco::Logger * log; From 24243c51bef04a45fe25dc30c859a8f555181444 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 7 Jul 2022 16:40:35 +0000 Subject: [PATCH 096/659] Fix distributed grouping sets with nulls --- src/Interpreters/InterpreterSelectQuery.cpp | 12 +- ...3_group_by_use_nulls_distributed.reference | 157 ++++++++++++++++++ .../02343_group_by_use_nulls_distributed.sql | 51 ++++++ 3 files changed, 218 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02343_group_by_use_nulls_distributed.reference create mode 100644 tests/queries/0_stateless/02343_group_by_use_nulls_distributed.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 551bd072586..a1dd0b7ad0e 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -786,8 +786,16 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (analysis_result.use_grouping_set_key) res.insert({ nullptr, std::make_shared(), "__grouping_set" }); - for (const auto & key : query_analyzer->aggregationKeys()) - res.insert({nullptr, header.getByName(key.name).type, key.name}); + if (context->getSettingsRef().group_by_use_nulls) + { + for (const auto & key : query_analyzer->aggregationKeys()) + res.insert({nullptr, makeNullable(header.getByName(key.name).type), key.name}); + } + else + { + for (const auto & key : query_analyzer->aggregationKeys()) + res.insert({nullptr, header.getByName(key.name).type, key.name}); + } for (const auto & aggregate : query_analyzer->aggregates()) { diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls_distributed.reference b/tests/queries/0_stateless/02343_group_by_use_nulls_distributed.reference new file mode 100644 index 00000000000..7a9263e883c --- /dev/null +++ b/tests/queries/0_stateless/02343_group_by_use_nulls_distributed.reference @@ -0,0 +1,157 @@ +-- { echoOn } +SELECT number, number % 2, sum(number) AS val +FROM remote('127.0.0.{2,3}', numbers(10)) +GROUP BY ROLLUP(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; +0 0 0 +0 \N 0 +1 1 2 +1 \N 2 +2 0 4 +2 \N 4 +3 1 6 +3 \N 6 +4 0 8 +4 \N 8 +5 1 10 +5 \N 10 +6 0 12 +6 \N 12 +7 1 14 +7 \N 14 +8 0 16 +8 \N 16 +9 1 18 +9 \N 18 +\N \N 90 +SELECT number, number % 2, sum(number) AS val +FROM remote('127.0.0.{2,3}', numbers(10)) +GROUP BY ROLLUP(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=0; +0 0 0 +0 0 0 +0 0 90 +1 0 2 +1 1 2 +2 0 4 +2 0 4 +3 0 6 +3 1 6 +4 0 8 +4 0 8 +5 0 10 +5 1 10 +6 0 12 +6 0 12 +7 0 14 +7 1 14 +8 0 16 +8 0 16 +9 0 18 +9 1 18 +SELECT number, number % 2, sum(number) AS val +FROM remote('127.0.0.{2,3}', numbers(10)) +GROUP BY CUBE(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; +0 0 0 +0 \N 0 +1 1 2 +1 \N 2 +2 0 4 +2 \N 4 +3 1 6 +3 \N 6 +4 0 8 +4 \N 8 +5 1 10 +5 \N 10 +6 0 12 +6 \N 12 +7 1 14 +7 \N 14 +8 0 16 +8 \N 16 +9 1 18 +9 \N 18 +\N 0 40 +\N 1 50 +\N \N 90 +SELECT number, number % 2, sum(number) AS val +FROM remote('127.0.0.{2,3}', numbers(10)) +GROUP BY CUBE(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=0; +0 0 0 +0 0 0 +0 0 40 +0 0 90 +0 1 50 +1 0 2 +1 1 2 +2 0 4 +2 0 4 +3 0 6 +3 1 6 +4 0 8 +4 0 8 +5 0 10 +5 1 10 +6 0 12 +6 0 12 +7 0 14 +7 1 14 +8 0 16 +8 0 16 +9 0 18 +9 1 18 +SELECT + number, + number % 2, + sum(number) AS val +FROM remote('127.0.0.{2,3}', numbers(10)) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls = 1; +0 \N 0 +1 \N 2 +2 \N 4 +3 \N 6 +4 \N 8 +5 \N 10 +6 \N 12 +7 \N 14 +8 \N 16 +9 \N 18 +\N 0 40 +\N 1 50 +SELECT + number, + number % 2, + sum(number) AS val +FROM remote('127.0.0.{2,3}', numbers(10)) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls = 0; +0 0 0 +0 0 40 +0 1 50 +1 0 2 +2 0 4 +3 0 6 +4 0 8 +5 0 10 +6 0 12 +7 0 14 +8 0 16 +9 0 18 diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls_distributed.sql b/tests/queries/0_stateless/02343_group_by_use_nulls_distributed.sql new file mode 100644 index 00000000000..15ac1127de7 --- /dev/null +++ b/tests/queries/0_stateless/02343_group_by_use_nulls_distributed.sql @@ -0,0 +1,51 @@ +-- { echoOn } +SELECT number, number % 2, sum(number) AS val +FROM remote('127.0.0.{2,3}', numbers(10)) +GROUP BY ROLLUP(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; + +SELECT number, number % 2, sum(number) AS val +FROM remote('127.0.0.{2,3}', numbers(10)) +GROUP BY ROLLUP(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=0; + +SELECT number, number % 2, sum(number) AS val +FROM remote('127.0.0.{2,3}', numbers(10)) +GROUP BY CUBE(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; + +SELECT number, number % 2, sum(number) AS val +FROM remote('127.0.0.{2,3}', numbers(10)) +GROUP BY CUBE(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=0; + +SELECT + number, + number % 2, + sum(number) AS val +FROM remote('127.0.0.{2,3}', numbers(10)) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls = 1; + +SELECT + number, + number % 2, + sum(number) AS val +FROM remote('127.0.0.{2,3}', numbers(10)) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls = 0; + From 4cfc0eb7449272a6b03ad57f3f89f7cd68d97125 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jul 2022 21:13:24 +0300 Subject: [PATCH 097/659] Update 02353_simdjson_buffer_overflow.sql --- tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql b/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql index 0861ae48f37..0572536e19e 100644 --- a/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql +++ b/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql @@ -1,4 +1,5 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-msan +-- Tag: no-msan: fuzzer can make this query very memory hungry, and under MSan, the MemoryTracker cannot account for the additional memory used by sanitizer, and OOM happens. SET max_execution_time = 3; SET timeout_overflow_mode = 'break'; From f80aea9eaca55d7e7ce8962a53aee14eb66a5139 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 7 Jul 2022 14:14:47 -0400 Subject: [PATCH 098/659] Cosmetic issue and improve flush function --- src/Compression/CompressionCodecDeflate.cpp | 81 ++++++++++++--------- src/Compression/CompressionCodecDeflate.h | 61 ++++++---------- src/Compression/ICompressionCodec.cpp | 6 -- src/Compression/ICompressionCodec.h | 33 ++++----- 4 files changed, 85 insertions(+), 96 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 405df9d840b..69d237b2325 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -19,27 +19,45 @@ namespace ErrorCodes qpl_job * DeflateJobHWPool::jobPool[JOB_POOL_SIZE]; std::atomic_bool DeflateJobHWPool::jobLocks[JOB_POOL_SIZE]; +bool DeflateJobHWPool::initJobPool() +{ + if (jobPoolEnabled == false) + { + const int32_t size = get_job_size_helper(); + if (size < 0) + return false; + for (int i = 0; i < JOB_POOL_SIZE; ++i) + { + jobPool[i] = nullptr; + qpl_job * qpl_job_ptr = reinterpret_cast(new uint8_t[size]); + if (init_job_helper(qpl_job_ptr) < 0) + return false; + jobPool[i] = qpl_job_ptr; + jobLocks[i].store(false); + } + jobPoolEnabled = true; + } + return jobPoolEnabled; +} + DeflateJobHWPool & DeflateJobHWPool::instance() { static DeflateJobHWPool ret; return ret; } -DeflateJobHWPool::DeflateJobHWPool() +DeflateJobHWPool::DeflateJobHWPool():jobPoolEnabled(false) { log = &Poco::Logger::get("DeflateJobHWPool"); - if (initJobPool() < 0) - { - jobPoolEnabled = false; + if (!initJobPool()) LOG_WARNING(log, "DeflateJobHWPool is not ready. Please check if IAA hardware support.Auto switch to deflate software codec here"); - } - else - jobPoolEnabled = true; } + DeflateJobHWPool::~DeflateJobHWPool() { destroyJobPool(); } + //HardwareCodecDeflate HardwareCodecDeflate::HardwareCodecDeflate() { @@ -159,7 +177,7 @@ uint32_t HardwareCodecDeflate::doDecompressDataReq(const char * source, uint32_t } } -void HardwareCodecDeflate::doFlushAsynchronousDecompressRequests() +void HardwareCodecDeflate::flushAsynchronousDecompressRequests() { uint32_t job_id = 0; qpl_job * job_ptr = nullptr; @@ -193,10 +211,9 @@ void HardwareCodecDeflate::doFlushAsynchronousDecompressRequests() } } } -//SoftwareCodecDeflate -SoftwareCodecDeflate::SoftwareCodecDeflate() + +SoftwareCodecDeflate::SoftwareCodecDeflate():jobSWPtr(nullptr) { - jobSWPtr = nullptr; } SoftwareCodecDeflate::~SoftwareCodecDeflate() @@ -279,11 +296,11 @@ void SoftwareCodecDeflate::doDecompressData(const char * source, uint32_t source } //CompressionCodecDeflate -CompressionCodecDeflate::CompressionCodecDeflate() +CompressionCodecDeflate::CompressionCodecDeflate(): + hw_codec(std::make_unique()), + sw_codec(std::make_unique()) { setCodecDescription("DEFLATE"); - hwCodec = std::make_unique(); - swCodec = std::make_unique(); } uint8_t CompressionCodecDeflate::getMethodByte() const @@ -296,24 +313,19 @@ void CompressionCodecDeflate::updateHash(SipHash & hash) const getCodecDesc()->updateTreeHash(hash); } -#define DEFLATE_COMPRESSBOUND(isize) ((isize) + ((isize) >> 12) + ((isize) >> 14) + ((isize) >> 25) + 13) //Aligned with ZLIB uint32_t CompressionCodecDeflate::getMaxCompressedDataSize(uint32_t uncompressed_size) const { - return DEFLATE_COMPRESSBOUND(uncompressed_size); -} - -uint32_t CompressionCodecDeflate::doCompressDataSW(const char * source, uint32_t source_size, char * dest) const -{ - return swCodec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); + /// Aligned with ZLIB + return ((uncompressed_size) + ((uncompressed_size) >> 12) + ((uncompressed_size) >> 14) + ((uncompressed_size) >> 25) + 13); } uint32_t CompressionCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest) const { uint32_t res = 0; - if (hwCodec->hwEnabled) - res = hwCodec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); + if (hw_codec->hwEnabled) + res = hw_codec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); if (0 == res) - res = swCodec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); + res = sw_codec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); return res; } @@ -324,31 +336,32 @@ void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t sou case CodecMode::Synchronous: { uint32_t res = 0; - if (hwCodec->hwEnabled) - res = hwCodec->doDecompressData(source, source_size, dest, uncompressed_size); + if (hw_codec->hwEnabled) + res = hw_codec->doDecompressData(source, source_size, dest, uncompressed_size); if (0 == res) - swCodec->doDecompressData(source, source_size, dest, uncompressed_size); + sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); break; } case CodecMode::Asynchronous: { uint32_t res = 0; - if (hwCodec->hwEnabled) - res = hwCodec->doDecompressDataReq(source, source_size, dest, uncompressed_size); + if (hw_codec->hwEnabled) + res = hw_codec->doDecompressDataReq(source, source_size, dest, uncompressed_size); if (0 == res) - swCodec->doDecompressData(source, source_size, dest, uncompressed_size); + sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); break; } case CodecMode::SoftwareFallback: - swCodec->doDecompressData(source, source_size, dest, uncompressed_size); + sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); break; } } -void CompressionCodecDeflate::doFlushAsynchronousDecompressRequests() +void CompressionCodecDeflate::flushAsynchronousDecompressRequests() { - if (hwCodec->hwEnabled) - hwCodec->doFlushAsynchronousDecompressRequests(); + if (hw_codec->hwEnabled) + hw_codec->flushAsynchronousDecompressRequests(); + setDecompressMode(CodecMode::Synchronous); } void registerCodecDeflate(CompressionCodecFactory & factory) { diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index 702c0a82ac4..1ab038077de 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -21,15 +21,16 @@ public: static constexpr qpl_path_t PATH = qpl_path_hardware; static qpl_job * jobPool[JOB_POOL_SIZE]; static std::atomic_bool jobLocks[JOB_POOL_SIZE]; - bool jobPoolEnabled; + bool jobPoolEnabled = false; - bool jobPoolReady() + bool jobPoolReady() const { return jobPoolEnabled; } + qpl_job * acquireJob(uint32_t * job_id) { - if (jobPoolEnabled) + if (jobPoolReady()) { uint32_t retry = 0; auto index = random(JOB_POOL_SIZE); @@ -50,9 +51,10 @@ public: return nullptr; } } + qpl_job * releaseJob(uint32_t job_id) { - if (jobPoolEnabled) + if (jobPoolReady()) { uint32_t index = JOB_POOL_SIZE - job_id; ReleaseJobObjectGuard _(index); @@ -63,9 +65,10 @@ public: return nullptr; } } - qpl_job * getJobPtr(uint32_t job_id) + + qpl_job * getJobPtr(uint32_t job_id) const { - if (jobPoolEnabled) + if (jobPoolReady()) { uint32_t index = JOB_POOL_SIZE - job_id; return jobPool[index]; @@ -77,7 +80,10 @@ public: } private: - size_t random(uint32_t pool_size) + /// Returns true if Job pool initialization succeeded, otherwise false + bool initJobPool(); + + size_t random(uint32_t pool_size) const { size_t tsc = 0; unsigned lo, hi; @@ -86,7 +92,7 @@ private: return (static_cast((tsc * 44485709377909ULL) >> 4)) % pool_size; } - int32_t get_job_size_helper() + int32_t get_job_size_helper() const { static uint32_t size = 0; if (size == 0) @@ -114,29 +120,6 @@ private: return 0; } - int32_t initJobPool() - { - static bool initialized = false; - - if (initialized == false) - { - const int32_t size = get_job_size_helper(); - if (size < 0) - return -1; - for (int i = 0; i < JOB_POOL_SIZE; ++i) - { - jobPool[i] = nullptr; - qpl_job * qpl_job_ptr = reinterpret_cast(new uint8_t[size]); - if (init_job_helper(qpl_job_ptr) < 0) - return -1; - jobPool[i] = qpl_job_ptr; - jobLocks[i].store(false); - } - initialized = true; - } - return 0; - } - bool tryLockJob(size_t index) { bool expected = false; @@ -170,6 +153,7 @@ private: ReleaseJobObjectGuard(const uint32_t i) : index(i) { } + ~ReleaseJobObjectGuard() { jobLocks[index].store(false); @@ -177,6 +161,7 @@ private: }; Poco::Logger * log; }; + class SoftwareCodecDeflate { public: @@ -186,7 +171,7 @@ public: void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); private: - qpl_job * jobSWPtr; //Software Job Codec Ptr + qpl_job * jobSWPtr; std::unique_ptr jobSWbuffer; qpl_job * getJobCodecPtr(); }; @@ -200,7 +185,7 @@ public: uint32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const; uint32_t doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const; uint32_t doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); - void doFlushAsynchronousDecompressRequests(); + void flushAsynchronousDecompressRequests(); private: std::map jobDecompAsyncMap; @@ -210,7 +195,6 @@ class CompressionCodecDeflate : public ICompressionCodec { public: CompressionCodecDeflate(); - //~CompressionCodecDeflate() ; uint8_t getMethodByte() const override; void updateHash(SipHash & hash) const override; @@ -219,19 +203,20 @@ protected: { return true; } + bool isGenericCompression() const override { return true; } + uint32_t doCompressData(const char * source, uint32_t source_size, char * dest) const override; - uint32_t doCompressDataSW(const char * source, uint32_t source_size, char * dest) const; void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; - void doFlushAsynchronousDecompressRequests() override; + void flushAsynchronousDecompressRequests() override; private: uint32_t getMaxCompressedDataSize(uint32_t uncompressed_size) const override; - std::unique_ptr hwCodec; - std::unique_ptr swCodec; + std::unique_ptr hw_codec; + std::unique_ptr sw_codec; }; } diff --git a/src/Compression/ICompressionCodec.cpp b/src/Compression/ICompressionCodec.cpp index 7aaa21776e9..c48ca99d452 100644 --- a/src/Compression/ICompressionCodec.cpp +++ b/src/Compression/ICompressionCodec.cpp @@ -110,12 +110,6 @@ UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, ch return decompressed_size; } -void ICompressionCodec::flushAsynchronousDecompressRequests() -{ - doFlushAsynchronousDecompressRequests(); - decompressMode = CodecMode::Synchronous; -} - UInt32 ICompressionCodec::readCompressedBlockSize(const char * source) { UInt32 compressed_block_size = unalignedLoad(&source[1]); diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 16574cccd47..b6ef47e6b45 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -26,20 +26,6 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size); class ICompressionCodec : private boost::noncopyable { public: - /// Three kinds of codec mode: - /// Synchronous mode which is commonly used by default; - /// --- For the codec with HW decompressor, it means submit request to HW and busy wait till complete. - /// Asynchronous mode which required HW decompressor support; - /// --- For the codec with HW decompressor, it means submit request to HW and return immeditately. - /// --- Must be used in pair with flushAsynchronousDecompressRequests. - /// SoftwareFallback mode is exclusively defined for the codec with HW decompressor, enable its capability of "fallback to SW codec". - enum class CodecMode - { - Synchronous, - Asynchronous, - SoftwareFallback - }; - virtual ~ICompressionCodec() = default; /// Byte which indicates codec in compressed file @@ -62,6 +48,20 @@ public: /// Decompress bytes from compressed source to dest. Dest should preallocate memory; UInt32 decompress(const char * source, UInt32 source_size, char * dest) const; + /// Three kinds of codec mode: + /// Synchronous mode which is commonly used by default; + /// --- For the codec with HW decompressor, it means submit request to HW and busy wait till complete. + /// Asynchronous mode which required HW decompressor support; + /// --- For the codec with HW decompressor, it means submit request to HW and return immeditately. + /// --- Must be used in pair with flushAsynchronousDecompressRequests. + /// SoftwareFallback mode is exclusively defined for the codec with HW decompressor, enable its capability of "fallback to SW codec". + enum class CodecMode + { + Synchronous, + Asynchronous, + SoftwareFallback + }; + /// Get current decompression mode CodecMode getDecompressMode() const { @@ -80,7 +80,7 @@ public: /// Meanwhile, source and target buffer for decompression should not be overwritten until this function execute completely. /// Otherwise it would conflict with HW offloading and cause exception. /// For QPL deflate, it support the maximum number of requests equal to DeflateJobHWPool::jobPoolSize - void flushAsynchronousDecompressRequests(); + virtual void flushAsynchronousDecompressRequests(){} /// Number of bytes, that will be used to compress uncompressed_size bytes with current codec virtual UInt32 getCompressedReserveSize(UInt32 uncompressed_size) const @@ -132,9 +132,6 @@ protected: /// Actually decompress data without header virtual void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const = 0; - /// Flush asynchronous request for decompression - virtual void doFlushAsynchronousDecompressRequests(){} - /// Construct and set codec description from codec name and arguments. Must be called in codec constructor. void setCodecDescription(const String & name, const ASTs & arguments = {}); From 2765e2a25952c4f496b40a91c34db9ee7b9e8066 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 7 Jul 2022 14:44:42 -0400 Subject: [PATCH 099/659] remove unused function in deflate job pool --- src/Compression/CompressionCodecDeflate.cpp | 18 +++---- src/Compression/CompressionCodecDeflate.h | 53 +++------------------ 2 files changed, 15 insertions(+), 56 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 69d237b2325..14b3de5dd9f 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -15,29 +15,29 @@ namespace ErrorCodes extern const int CANNOT_COMPRESS; extern const int CANNOT_DECOMPRESS; } -//DeflateJobHWPool +/// DeflateJobHWPool is resource pool for provide the job objects which is required to save context infomation during offload asynchronous compression to IAA. qpl_job * DeflateJobHWPool::jobPool[JOB_POOL_SIZE]; std::atomic_bool DeflateJobHWPool::jobLocks[JOB_POOL_SIZE]; bool DeflateJobHWPool::initJobPool() { - if (jobPoolEnabled == false) + if (job_pool_ready == false) { - const int32_t size = get_job_size_helper(); - if (size < 0) + uint32_t size = 0; + if (qpl_get_job_size(PATH, &size) != QPL_STS_OK) return false; + for (int i = 0; i < JOB_POOL_SIZE; ++i) { - jobPool[i] = nullptr; qpl_job * qpl_job_ptr = reinterpret_cast(new uint8_t[size]); - if (init_job_helper(qpl_job_ptr) < 0) + if ((nullptr == qpl_job_ptr) || (qpl_init_job(PATH, qpl_job_ptr) != QPL_STS_OK)) return false; jobPool[i] = qpl_job_ptr; jobLocks[i].store(false); } - jobPoolEnabled = true; + job_pool_ready = true; } - return jobPoolEnabled; + return job_pool_ready; } DeflateJobHWPool & DeflateJobHWPool::instance() @@ -46,7 +46,7 @@ DeflateJobHWPool & DeflateJobHWPool::instance() return ret; } -DeflateJobHWPool::DeflateJobHWPool():jobPoolEnabled(false) +DeflateJobHWPool::DeflateJobHWPool():job_pool_ready(false) { log = &Poco::Logger::get("DeflateJobHWPool"); if (!initJobPool()) diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index 1ab038077de..1ff78942796 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -11,6 +11,7 @@ class Logger; namespace DB { +/// DeflateJobHWPool is resource pool for provide the job objects which is required to save context infomation during offload asynchronous compression to IAA. class DeflateJobHWPool { public: @@ -21,11 +22,11 @@ public: static constexpr qpl_path_t PATH = qpl_path_hardware; static qpl_job * jobPool[JOB_POOL_SIZE]; static std::atomic_bool jobLocks[JOB_POOL_SIZE]; - bool jobPoolEnabled = false; + bool job_pool_ready = false; bool jobPoolReady() const { - return jobPoolEnabled; + return job_pool_ready; } qpl_job * acquireJob(uint32_t * job_id) @@ -66,19 +67,6 @@ public: } } - qpl_job * getJobPtr(uint32_t job_id) const - { - if (jobPoolReady()) - { - uint32_t index = JOB_POOL_SIZE - job_id; - return jobPool[index]; - } - else - { - return nullptr; - } - } - private: /// Returns true if Job pool initialization succeeded, otherwise false bool initJobPool(); @@ -92,34 +80,6 @@ private: return (static_cast((tsc * 44485709377909ULL) >> 4)) % pool_size; } - int32_t get_job_size_helper() const - { - static uint32_t size = 0; - if (size == 0) - { - const auto status = qpl_get_job_size(PATH, &size); - if (status != QPL_STS_OK) - { - return -1; - } - } - return size; - } - - int32_t init_job_helper(qpl_job * qpl_job_ptr) - { - if (qpl_job_ptr == nullptr) - { - return -1; - } - auto status = qpl_init_job(PATH, qpl_job_ptr); - if (status != QPL_STS_OK) - { - return -1; - } - return 0; - } - bool tryLockJob(size_t index) { bool expected = false; @@ -128,12 +88,11 @@ private: void destroyJobPool() { - const uint32_t size = get_job_size_helper(); + uint32_t size = 0; + qpl_get_job_size(PATH, &size); for (uint32_t i = 0; i < JOB_POOL_SIZE && size > 0; ++i) { - while (tryLockJob(i) == false) - { - } + while (tryLockJob(i) == false); if (jobPool[i]) { qpl_fini_job(jobPool[i]); From 1587385f7ad34f243cd8cd4e92490b71ce62d131 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 7 Jul 2022 18:53:20 +0000 Subject: [PATCH 100/659] Cleanup code --- src/Columns/ColumnNullable.cpp | 14 +++++++ src/Columns/ColumnNullable.h | 1 + src/DataTypes/DataTypeNullable.cpp | 7 ++++ src/DataTypes/DataTypeNullable.h | 1 + src/Interpreters/ExpressionAnalyzer.cpp | 21 +++-------- src/Interpreters/ExpressionAnalyzer.h | 1 - src/Interpreters/InterpreterSelectQuery.cpp | 3 +- src/Processors/QueryPlan/AggregatingStep.cpp | 37 ++++--------------- src/Processors/QueryPlan/AggregatingStep.h | 1 - src/Processors/Transforms/RollupTransform.cpp | 7 +--- 10 files changed, 39 insertions(+), 54 deletions(-) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 435e6bf1fbc..7677da991b9 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -785,4 +785,18 @@ ColumnPtr makeNullable(const ColumnPtr & column) return ColumnNullable::create(column, ColumnUInt8::create(column->size(), 0)); } +ColumnPtr makeNullableSafe(const ColumnPtr & column) +{ + if (isColumnNullable(*column)) + return column; + + if (isColumnConst(*column)) + return ColumnConst::create(makeNullableSafe(assert_cast(*column).getDataColumnPtr()), column->size()); + + if (column->canBeInsideNullable()) + return makeNullable(column); + + return column; +} + } diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 60951dfcc2e..fb765db454d 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -221,5 +221,6 @@ private: }; ColumnPtr makeNullable(const ColumnPtr & column); +ColumnPtr makeNullableSafe(const ColumnPtr & column); } diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index b354b1278be..a14fb785b96 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -85,6 +85,13 @@ DataTypePtr makeNullable(const DataTypePtr & type) return std::make_shared(type); } +DataTypePtr makeNullableSafe(const DataTypePtr & type) +{ + if (type->canBeInsideNullable()) + return makeNullable(type); + return type; +} + DataTypePtr removeNullable(const DataTypePtr & type) { if (type->isNullable()) diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index c87e4f77008..379119b364c 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -51,6 +51,7 @@ private: DataTypePtr makeNullable(const DataTypePtr & type); +DataTypePtr makeNullableSafe(const DataTypePtr & type); DataTypePtr removeNullable(const DataTypePtr & type); } diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index bab017f5a11..177a919c0de 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -41,12 +41,11 @@ #include -#include "Common/logger_useful.h" #include #include -#include "Columns/ColumnNullable.h" -#include "Core/ColumnsWithTypeAndName.h" -#include "DataTypes/IDataType.h" +#include +#include +#include #include #include #include @@ -68,7 +67,6 @@ #include #include #include -#include namespace DB { @@ -399,7 +397,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) } } - NameAndTypePair key{column_name, use_nulls ? makeNullable(node->result_type) : node->result_type }; + NameAndTypePair key{column_name, use_nulls ? makeNullableSafe(node->result_type) : node->result_type }; grouping_set_list.push_back(key); @@ -453,7 +451,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) } } - NameAndTypePair key = NameAndTypePair{ column_name, use_nulls ? makeNullable(node->result_type) : node->result_type }; + NameAndTypePair key = NameAndTypePair{ column_name, use_nulls ? makeNullableSafe(node->result_type) : node->result_type }; /// Aggregation keys are uniqued. if (!unique_keys.contains(key.name)) @@ -1446,7 +1444,7 @@ void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAGPtr & befor if (isAggregateFunction(source_column.type)) result_columns.push_back(source_column); else - result_columns.emplace_back(makeNullable(source_column.type), source_column.name); + result_columns.emplace_back(makeNullableSafe(source_column.type), source_column.name); } ExpressionActionsChain::Step & step = chain.lastStep(before_aggregation->getNamesAndTypesList()); @@ -1632,8 +1630,6 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActio ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); - LOG_DEBUG(&Poco::Logger::get("SelectQueryExpressionAnalyzer"), "Before output: {}", step.actions()->getNamesAndTypesList().toString()); - NamesWithAliases result_columns; ASTs asts = select_query->select()->children; @@ -1675,11 +1671,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActio } auto actions = chain.getLastActions(); - LOG_DEBUG(&Poco::Logger::get("SelectQueryExpressionAnalyzer"), "Before projection: {}", actions->getNamesAndTypesList().toString()); - actions->project(result_columns); - LOG_DEBUG(&Poco::Logger::get("SelectQueryExpressionAnalyzer"), "After projection: {}", actions->getNamesAndTypesList().toString()); - return actions; } @@ -1903,7 +1895,6 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( query_analyzer.appendAggregateFunctionsArguments(chain, only_types || !first_stage); before_aggregation = chain.getLastActions(); - before_aggregation_with_nullable = chain.getLastActions(); if (settings.group_by_use_nulls) query_analyzer.appendGroupByModifiers(before_aggregation, chain, only_types); diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 01135cfdc9e..f288162845c 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -246,7 +246,6 @@ struct ExpressionAnalysisResult JoinPtr join; ActionsDAGPtr before_where; ActionsDAGPtr before_aggregation; - ActionsDAGPtr before_aggregation_with_nullable; ActionsDAGPtr before_having; String having_column_name; bool remove_having_filter = false; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a1dd0b7ad0e..0c9a3033012 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -582,7 +582,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Calculate structure of the result. result_header = getSampleBlockImpl(); - LOG_DEBUG(&Poco::Logger::get("InterpreterSelectQuery"), "Result header: {}", result_header.dumpStructure()); }; analyze(shouldMoveToPrewhere()); @@ -789,7 +788,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (context->getSettingsRef().group_by_use_nulls) { for (const auto & key : query_analyzer->aggregationKeys()) - res.insert({nullptr, makeNullable(header.getByName(key.name).type), key.name}); + res.insert({nullptr, makeNullableSafe(header.getByName(key.name).type), key.name}); } else { diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index b73ca093164..f4e3749bd70 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -16,10 +16,6 @@ #include #include #include -#include -#include "Common/logger_useful.h" -#include "Core/ColumnNumbers.h" -#include "DataTypes/IDataType.h" namespace DB { @@ -57,11 +53,8 @@ static inline void convertToNullable(Block & header, const Names & keys) { auto & column = header.getByName(key); - if (!isAggregateFunction(column.type)) - { - column.type = makeNullable(column.type); - column.column = makeNullable(column.column); - } + column.type = makeNullableSafe(column.type); + column.column = makeNullableSafe(column.column); } } @@ -69,18 +62,7 @@ Block generateOutputHeader(const Block & input_header, const Names & keys, bool { auto header = appendGroupingSetColumn(input_header); if (use_nulls) - { - for (const auto & key : keys) - { - auto & column = header.getByName(key); - - if (!isAggregateFunction(column.type)) - { - column.type = makeNullable(column.type); - column.column = makeNullable(column.column); - } - } - } + convertToNullable(header, keys); return header; } @@ -269,6 +251,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B const auto & missing_columns = grouping_sets_params[set_counter].missing_keys; + auto to_nullable_function = FunctionFactory::instance().get("toNullable", nullptr); for (size_t i = 0; i < output_header.columns(); ++i) { auto & col = output_header.getByPosition(i); @@ -286,22 +269,16 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B else { const auto * column_node = dag->getIndex()[header.getPositionByName(col.name)]; - if (isAggregateFunction(column_node->result_type) || !group_by_use_nulls) - { - index.push_back(column_node); - } + if (group_by_use_nulls && column_node->result_type->canBeInsideNullable()) + index.push_back(&dag->addFunction(to_nullable_function, { column_node }, col.name)); else - { - const auto * node = &dag->addFunction(FunctionFactory::instance().get("toNullable", nullptr), { column_node }, col.name); - index.push_back(node); - } + index.push_back(column_node); } } dag->getIndex().swap(index); auto expression = std::make_shared(dag, settings.getActionsSettings()); auto transform = std::make_shared(header, expression); - LOG_DEBUG(&Poco::Logger::get("AggregatingStep"), "Header for GROUPING SET #{}: {}", set_counter, transform->getOutputPort().getHeader().dumpStructure()); connect(*ports[set_counter], transform->getInputPort()); processors.emplace_back(std::move(transform)); diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index fb838580e0f..71130b65adb 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -3,7 +3,6 @@ #include #include #include -#include "Core/ColumnNumbers.h" namespace DB { diff --git a/src/Processors/Transforms/RollupTransform.cpp b/src/Processors/Transforms/RollupTransform.cpp index 0ab5900447f..a5d67fb2f15 100644 --- a/src/Processors/Transforms/RollupTransform.cpp +++ b/src/Processors/Transforms/RollupTransform.cpp @@ -1,9 +1,7 @@ #include #include #include -#include -#include "Common/logger_useful.h" -#include "Columns/ColumnNullable.h" +#include namespace DB { @@ -25,7 +23,6 @@ GroupByModifierTransform::GroupByModifierTransform(Block header, AggregatingTran auto output_aggregator_params = params->params; output_aggregator = std::make_unique(intermediate_header, output_aggregator_params); } - LOG_DEBUG(&Poco::Logger::get("GroupByModifierTransform"), "Intermediate header: {}", intermediate_header.dumpStructure()); } void GroupByModifierTransform::consume(Chunk chunk) @@ -45,7 +42,7 @@ void GroupByModifierTransform::mergeConsumed() if (use_nulls) { for (auto key : keys) - columns[key] = makeNullable(columns[key]); + columns[key] = makeNullableSafe(columns[key]); } current_chunk = Chunk{ columns, rows }; From 0b18b4ed43f814ea046811ab42d25aa01c6c9dca Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 7 Jul 2022 18:58:46 +0000 Subject: [PATCH 101/659] Use group_by_use_nulls in tests more often --- docker/test/stress/stress | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 76e1ff55c50..6d90b9d5437 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -46,7 +46,7 @@ def get_options(i, backward_compatibility_check): if i == 13: client_options.append("memory_tracker_fault_probability=0.001") - if i % 17 == 1 and not backward_compatibility_check: + if i % 2 == 1 and not backward_compatibility_check: client_options.append("group_by_use_nulls=1") if client_options: From 29fee729e33e8f4e38888163230f736b47678eab Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 7 Jul 2022 15:17:23 -0400 Subject: [PATCH 102/659] revise the way of get job size --- src/Compression/CompressionCodecDeflate.cpp | 19 +++++-------------- src/Compression/CompressionCodecDeflate.h | 2 +- 2 files changed, 6 insertions(+), 15 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 14b3de5dd9f..42ac205451d 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -15,7 +15,6 @@ namespace ErrorCodes extern const int CANNOT_COMPRESS; extern const int CANNOT_DECOMPRESS; } -/// DeflateJobHWPool is resource pool for provide the job objects which is required to save context infomation during offload asynchronous compression to IAA. qpl_job * DeflateJobHWPool::jobPool[JOB_POOL_SIZE]; std::atomic_bool DeflateJobHWPool::jobLocks[JOB_POOL_SIZE]; @@ -24,9 +23,8 @@ bool DeflateJobHWPool::initJobPool() if (job_pool_ready == false) { uint32_t size = 0; - if (qpl_get_job_size(PATH, &size) != QPL_STS_OK) - return false; - + /// get total size required for saving qpl job context + qpl_get_job_size(PATH, &size); for (int i = 0; i < JOB_POOL_SIZE; ++i) { qpl_job * qpl_job_ptr = reinterpret_cast(new uint8_t[size]); @@ -226,20 +224,13 @@ qpl_job * SoftwareCodecDeflate::getJobCodecPtr() { if (nullptr == jobSWPtr) { - qpl_status status; uint32_t size = 0; - // Job initialization - status = qpl_get_job_size(qpl_path_software, &size); - if (status != QPL_STS_OK) - { - throw Exception( - "SoftwareCodecDeflate::getJobCodecPtr -> qpl_get_job_size fail:" + std::to_string(status), ErrorCodes::CANNOT_COMPRESS); - } + qpl_get_job_size(qpl_path_software, &size); jobSWbuffer = std::make_unique(size); jobSWPtr = reinterpret_cast(jobSWbuffer.get()); - - status = qpl_init_job(qpl_path_software, jobSWPtr); + // Job initialization + auto status = qpl_init_job(qpl_path_software, jobSWPtr); if (status != QPL_STS_OK) { throw Exception( diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index 1ff78942796..c5874745882 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -71,7 +71,7 @@ private: /// Returns true if Job pool initialization succeeded, otherwise false bool initJobPool(); - size_t random(uint32_t pool_size) const + static size_t random(uint32_t pool_size) { size_t tsc = 0; unsigned lo, hi; From 2c8f308bbc309375a817bf5e89fcce480979bceb Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 7 Jul 2022 16:05:52 -0400 Subject: [PATCH 103/659] improve buffer allocation --- src/Compression/CompressionCodecDeflate.cpp | 11 ++++++++--- src/Compression/CompressionCodecDeflate.h | 11 ++++------- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 42ac205451d..ba74fe62213 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -25,11 +25,17 @@ bool DeflateJobHWPool::initJobPool() uint32_t size = 0; /// get total size required for saving qpl job context qpl_get_job_size(PATH, &size); + /// allocate buffer for storing all job objects + jobPoolBufferPtr = std::make_unique(size * JOB_POOL_SIZE); + memset(jobPool, 0, JOB_POOL_SIZE*sizeof(qpl_job *)); for (int i = 0; i < JOB_POOL_SIZE; ++i) { - qpl_job * qpl_job_ptr = reinterpret_cast(new uint8_t[size]); + qpl_job * qpl_job_ptr = reinterpret_cast(jobPoolBufferPtr.get() + i*JOB_POOL_SIZE); if ((nullptr == qpl_job_ptr) || (qpl_init_job(PATH, qpl_job_ptr) != QPL_STS_OK)) + { + destroyJobPool(); return false; + } jobPool[i] = qpl_job_ptr; jobLocks[i].store(false); } @@ -227,8 +233,7 @@ qpl_job * SoftwareCodecDeflate::getJobCodecPtr() uint32_t size = 0; qpl_get_job_size(qpl_path_software, &size); - jobSWbuffer = std::make_unique(size); - jobSWPtr = reinterpret_cast(jobSWbuffer.get()); + jobSWPtr = reinterpret_cast((std::make_unique(size)).get()); // Job initialization auto status = qpl_init_job(qpl_path_software, jobSWPtr); if (status != QPL_STS_OK) diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index c5874745882..a02b9624a6c 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -88,15 +88,12 @@ private: void destroyJobPool() { - uint32_t size = 0; - qpl_get_job_size(PATH, &size); - for (uint32_t i = 0; i < JOB_POOL_SIZE && size > 0; ++i) + for (uint32_t i = 0; i < JOB_POOL_SIZE; ++i) { - while (tryLockJob(i) == false); - if (jobPool[i]) + if (jobPool[i] != nullptr) { + while (tryLockJob(i) == false); qpl_fini_job(jobPool[i]); - delete[] jobPool[i]; } jobPool[i] = nullptr; jobLocks[i].store(false); @@ -118,6 +115,7 @@ private: jobLocks[index].store(false); } }; + std::unique_ptr jobPoolBufferPtr; Poco::Logger * log; }; @@ -131,7 +129,6 @@ public: private: qpl_job * jobSWPtr; - std::unique_ptr jobSWbuffer; qpl_job * getJobCodecPtr(); }; From 5681e50da813fafb08e95bda90a843e52598d390 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 7 Jul 2022 17:13:20 -0400 Subject: [PATCH 104/659] improve job pool initialization --- src/Compression/CompressionCodecDeflate.cpp | 75 +++++++++++---------- src/Compression/CompressionCodecDeflate.h | 27 ++------ 2 files changed, 45 insertions(+), 57 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index ba74fe62213..218d9106c51 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -18,55 +18,62 @@ namespace ErrorCodes qpl_job * DeflateJobHWPool::jobPool[JOB_POOL_SIZE]; std::atomic_bool DeflateJobHWPool::jobLocks[JOB_POOL_SIZE]; -bool DeflateJobHWPool::initJobPool() -{ - if (job_pool_ready == false) - { - uint32_t size = 0; - /// get total size required for saving qpl job context - qpl_get_job_size(PATH, &size); - /// allocate buffer for storing all job objects - jobPoolBufferPtr = std::make_unique(size * JOB_POOL_SIZE); - memset(jobPool, 0, JOB_POOL_SIZE*sizeof(qpl_job *)); - for (int i = 0; i < JOB_POOL_SIZE; ++i) - { - qpl_job * qpl_job_ptr = reinterpret_cast(jobPoolBufferPtr.get() + i*JOB_POOL_SIZE); - if ((nullptr == qpl_job_ptr) || (qpl_init_job(PATH, qpl_job_ptr) != QPL_STS_OK)) - { - destroyJobPool(); - return false; - } - jobPool[i] = qpl_job_ptr; - jobLocks[i].store(false); - } - job_pool_ready = true; - } - return job_pool_ready; -} - DeflateJobHWPool & DeflateJobHWPool::instance() { static DeflateJobHWPool ret; return ret; } -DeflateJobHWPool::DeflateJobHWPool():job_pool_ready(false) +DeflateJobHWPool::DeflateJobHWPool():log(&Poco::Logger::get("DeflateJobHWPool")) { - log = &Poco::Logger::get("DeflateJobHWPool"); - if (!initJobPool()) - LOG_WARNING(log, "DeflateJobHWPool is not ready. Please check if IAA hardware support.Auto switch to deflate software codec here"); + uint32_t size = 0; + uint32_t index = 0; + /// get total size required for saving qpl job context + qpl_get_job_size(PATH, &size); + /// allocate buffer for storing all job objects + jobPoolBufferPtr = std::make_unique(size * JOB_POOL_SIZE); + memset(jobPool, 0, JOB_POOL_SIZE*sizeof(qpl_job *)); + for (index = 0; index < JOB_POOL_SIZE; ++index) + { + qpl_job * qpl_job_ptr = reinterpret_cast(jobPoolBufferPtr.get() + index*JOB_POOL_SIZE); + if ((nullptr == qpl_job_ptr) || (qpl_init_job(PATH, qpl_job_ptr) != QPL_STS_OK)) + break; + jobPool[index] = qpl_job_ptr; + jobLocks[index].store(false); + } + + const char * qpl_version = qpl_get_library_version(); + if(JOB_POOL_SIZE == index) + { + jobPoolReady() = true; + LOG_DEBUG(log, "QPL deflate HW codec is ready! QPL Version:{}",qpl_version); + } + else + { + jobPoolReady() = false; + LOG_WARNING(log, "QPL deflate HW codec is not ready! Please check if IAA hardware support. Will fallback to software codec here. QPL Version:{}.",qpl_version); + } } DeflateJobHWPool::~DeflateJobHWPool() { - destroyJobPool(); + for (uint32_t i = 0; i < JOB_POOL_SIZE; ++i) + { + if (jobPool[i] != nullptr) + { + while (!tryLockJob(i)); + qpl_fini_job(jobPool[i]); + } + jobPool[i] = nullptr; + jobLocks[i].store(false); + } } //HardwareCodecDeflate -HardwareCodecDeflate::HardwareCodecDeflate() +HardwareCodecDeflate::HardwareCodecDeflate(): + hwEnabled(DeflateJobHWPool::instance().jobPoolReady()), + log(&Poco::Logger::get("HardwareCodecDeflate")) { - log = &Poco::Logger::get("HardwareCodecDeflate"); - hwEnabled = DeflateJobHWPool::instance().jobPoolReady(); } HardwareCodecDeflate::~HardwareCodecDeflate() diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index a02b9624a6c..a347b522095 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -24,10 +24,7 @@ public: static std::atomic_bool jobLocks[JOB_POOL_SIZE]; bool job_pool_ready = false; - bool jobPoolReady() const - { - return job_pool_ready; - } + bool & jobPoolReady() { return job_pool_ready;} qpl_job * acquireJob(uint32_t * job_id) { @@ -68,9 +65,6 @@ public: } private: - /// Returns true if Job pool initialization succeeded, otherwise false - bool initJobPool(); - static size_t random(uint32_t pool_size) { size_t tsc = 0; @@ -83,30 +77,17 @@ private: bool tryLockJob(size_t index) { bool expected = false; + assert(index < JOB_POOL_SIZE); return jobLocks[index].compare_exchange_strong(expected, true); } - void destroyJobPool() - { - for (uint32_t i = 0; i < JOB_POOL_SIZE; ++i) - { - if (jobPool[i] != nullptr) - { - while (tryLockJob(i) == false); - qpl_fini_job(jobPool[i]); - } - jobPool[i] = nullptr; - jobLocks[i].store(false); - } - } - - struct ReleaseJobObjectGuard + class ReleaseJobObjectGuard { uint32_t index; ReleaseJobObjectGuard() = delete; public: - ReleaseJobObjectGuard(const uint32_t i) : index(i) + ReleaseJobObjectGuard(const uint32_t index_) : index(index_) { } From 6045681bb4d6c0b31a26cda9e42c82b9f87e371d Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 7 Jul 2022 17:21:51 -0400 Subject: [PATCH 105/659] improve jobPoolReady --- src/Compression/CompressionCodecDeflate.cpp | 2 ++ src/Compression/CompressionCodecDeflate.h | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 218d9106c51..9aa5e92a9ee 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -17,6 +17,7 @@ namespace ErrorCodes } qpl_job * DeflateJobHWPool::jobPool[JOB_POOL_SIZE]; std::atomic_bool DeflateJobHWPool::jobLocks[JOB_POOL_SIZE]; +bool DeflateJobHWPool::job_pool_ready; DeflateJobHWPool & DeflateJobHWPool::instance() { @@ -67,6 +68,7 @@ DeflateJobHWPool::~DeflateJobHWPool() jobPool[i] = nullptr; jobLocks[i].store(false); } + jobPoolReady() = false; } //HardwareCodecDeflate diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index a347b522095..e50dc89091e 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -22,7 +22,7 @@ public: static constexpr qpl_path_t PATH = qpl_path_hardware; static qpl_job * jobPool[JOB_POOL_SIZE]; static std::atomic_bool jobLocks[JOB_POOL_SIZE]; - bool job_pool_ready = false; + static bool job_pool_ready; bool & jobPoolReady() { return job_pool_ready;} From e01b6cf8a5bccd30b42d1d5a1f9587422301a493 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 8 Jul 2022 09:25:45 +0800 Subject: [PATCH 106/659] update codes --- src/Storages/Hive/HiveFile.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 112798fea4e..671cb11deaa 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -88,6 +88,7 @@ std::optional IHiveFile::getRows() has_init_rows = true; } } + return rows; } From f8e3bdecafdd1fc9d68f0ce3167ff750261a7d20 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 8 Jul 2022 11:03:06 +0300 Subject: [PATCH 107/659] initial upload --- base/base/Base58.h | 9 +++ src/Core/Base58.h | 14 +++++ src/Core/Types.h | 2 + src/DataTypes/DataTypeBase58.cpp | 34 ++++++++++ src/DataTypes/DataTypeBase58.h | 48 ++++++++++++++ .../Serializations/SerializationBase58.cpp | 63 +++++++++++++++++++ .../Serializations/SerializationBase58.h | 17 +++++ 7 files changed, 187 insertions(+) create mode 100644 base/base/Base58.h create mode 100644 src/Core/Base58.h create mode 100644 src/DataTypes/DataTypeBase58.cpp create mode 100644 src/DataTypes/DataTypeBase58.h create mode 100644 src/DataTypes/Serializations/SerializationBase58.cpp create mode 100644 src/DataTypes/Serializations/SerializationBase58.h diff --git a/base/base/Base58.h b/base/base/Base58.h new file mode 100644 index 00000000000..c8bac92d186 --- /dev/null +++ b/base/base/Base58.h @@ -0,0 +1,9 @@ +#pragma once + +#include +#include + +namespace DB +{ + using Base58 = StrongTypedef; +} diff --git a/src/Core/Base58.h b/src/Core/Base58.h new file mode 100644 index 00000000000..6bf7ddbc07d --- /dev/null +++ b/src/Core/Base58.h @@ -0,0 +1,14 @@ +#pragma once + +#include + + +namespace DB +{ + +namespace Base58Helpers +{ + const Base58 Nil{}; +} + +} diff --git a/src/Core/Types.h b/src/Core/Types.h index 92546d7d07a..d376c036673 100644 --- a/src/Core/Types.h +++ b/src/Core/Types.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -88,6 +89,7 @@ enum class TypeIndex LowCardinality, Map, Object, + Base58, }; #if !defined(__clang__) #pragma GCC diagnostic pop diff --git a/src/DataTypes/DataTypeBase58.cpp b/src/DataTypes/DataTypeBase58.cpp new file mode 100644 index 00000000000..f13236be88e --- /dev/null +++ b/src/DataTypes/DataTypeBase58.cpp @@ -0,0 +1,34 @@ +#include +#include +#include + + +namespace DB +{ + +bool DataTypeBase58::equals(const IDataType & rhs) const +{ + return typeid(rhs) == typeid(*this); +} + +SerializationPtr DataTypeBase58::doGetDefaultSerialization() const +{ + return std::make_shared(); +} + +Field DataTypeUUID::getDefault() const +{ + return UUID{}; +} + +MutableColumnPtr DataTypeUUID::createColumn() const +{ + return ColumnVector::create(); +} + +void registerDataTypeUUID(DataTypeFactory & factory) +{ + factory.registerSimpleDataType("UUID", [] { return DataTypePtr(std::make_shared()); }); +} + +} diff --git a/src/DataTypes/DataTypeBase58.h b/src/DataTypes/DataTypeBase58.h new file mode 100644 index 00000000000..f17e70431ba --- /dev/null +++ b/src/DataTypes/DataTypeBase58.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class DataTypeBase58 : public IDataType +{ +public: + static constexpr bool is_parametric = false; + + using FieldType = Base58; + using ColumnType = ColumnVector; + static constexpr auto type_id = TypeIndex::Base58; + + const char * getFamilyName() const override { return "Base58"; } + TypeIndex getTypeId() const override { return type_id; } + + Field getDefault() const override; + + MutableColumnPtr createColumn() const override; + + bool isParametric() const override { return false; } + bool haveSubtypes() const override { return false; } + + bool equals(const IDataType & rhs) const override; + + bool canBeUsedInBitOperations() const override { return true; } + bool canBeInsideNullable() const override { return true; } + bool canBePromoted() const override { return false; } + bool shouldAlignRightInPrettyFormats() const override { return false; } + bool textCanContainOnlyValidUTF8() const override { return true; } + bool isComparable() const override { return true; } + bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; } + bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const override { return true; } + bool haveMaximumSizeOfValue() const override { return true; } + size_t getSizeOfValueInMemory() const override { return sizeof(Base58); } + bool isCategorial() const override { return true; } + bool canBeInsideLowCardinality() const override { return true; } + + SerializationPtr doGetDefaultSerialization() const override; +}; + +} diff --git a/src/DataTypes/Serializations/SerializationBase58.cpp b/src/DataTypes/Serializations/SerializationBase58.cpp new file mode 100644 index 00000000000..3fe79d9c67e --- /dev/null +++ b/src/DataTypes/Serializations/SerializationBase58.cpp @@ -0,0 +1,63 @@ +#include + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; + extern const int ILLEGAL_COLUMN; +} + +SerializationBase58::SerializationBase58(const SerializationPtr & nested_) + : SerializationCustomSimpleText(nested_) +{ +} + +void SerializationBase58::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const +{ + const auto * col = checkAndGetColumn(&column); + if (!col) + { + throw Exception("IPv4 type can only serialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); + } + + auto + char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'}; + char * ptr = buffer; + formatIPv4(reinterpret_cast(&col->getData()[row_num]), ptr); + + ostr.write(buffer, strlen(buffer)); +} + +void SerializationBase58::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const +{ + ColumnUInt32 * col = typeid_cast(&column); + if (!col) + { + throw Exception("IPv4 type can only deserialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); + } + + char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'}; + istr.read(buffer, sizeof(buffer) - 1); + UInt32 ipv4_value = 0; + + bool parse_result = parseIPv4(buffer, reinterpret_cast(&ipv4_value)); + if (!parse_result && !settings.input_format_ipv4_default_on_conversion_error) + { + throw Exception("Invalid IPv4 value", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); + } + + col->insert(ipv4_value); + + if (whole && !istr.eof()) + throwUnexpectedDataAfterParsedValue(column, istr, settings, "IPv4"); +} diff --git a/src/DataTypes/Serializations/SerializationBase58.h b/src/DataTypes/Serializations/SerializationBase58.h new file mode 100644 index 00000000000..e6586d1fb03 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationBase58.h @@ -0,0 +1,17 @@ +#pragma once + +#include + +namespace DB +{ + +class SerializationBase58 final : public SerializationCustomSimpleText +{ +public: + explicit SerializationBase58(const SerializationPtr & nested_); + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override; +}; + +} From 7a3c21174d07e4955a3dac4013144e0034e282b6 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 8 Jul 2022 11:55:16 -0400 Subject: [PATCH 108/659] use C++17 if initializer --- src/Compression/CompressionCodecDeflate.cpp | 53 ++++++--------------- 1 file changed, 14 insertions(+), 39 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 9aa5e92a9ee..24f25e09eb2 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -93,14 +93,13 @@ HardwareCodecDeflate::~HardwareCodecDeflate() uint32_t HardwareCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const { uint32_t job_id = 0; - qpl_job * job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); - if (job_ptr == nullptr) + qpl_job* job_ptr = nullptr; + uint32_t compressed_size = 0; + if (job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); nullptr == job_ptr) { LOG_WARNING(log, "HardwareCodecDeflate::doCompressData acquireJob fail!"); return 0; } - qpl_status status; - uint32_t compressed_size = 0; job_ptr->op = qpl_op_compress; job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); @@ -109,11 +108,9 @@ uint32_t HardwareCodecDeflate::doCompressData(const char * source, uint32_t sour job_ptr->level = qpl_default_level; job_ptr->available_out = dest_size; job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; - status = qpl_execute_job(job_ptr); - if (QPL_STS_OK == status) - { + + if (auto status = qpl_execute_job(job_ptr); QPL_STS_OK == status) compressed_size = job_ptr->total_out; - } else LOG_WARNING(log, "HardwareCodecDeflate::doCompressData fail ->status: '{}' ", static_cast(status)); @@ -124,14 +121,12 @@ uint32_t HardwareCodecDeflate::doCompressData(const char * source, uint32_t sour uint32_t HardwareCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const { uint32_t job_id = 0; - qpl_job * job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); - if (job_ptr == nullptr) + qpl_job* job_ptr = nullptr; + if (job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); nullptr == job_ptr) { LOG_WARNING(log, "HardwareCodecDeflate::doDecompressData acquireJob fail!"); return 0; } - qpl_status status; - // Performing a decompression operation job_ptr->op = qpl_op_decompress; job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); @@ -140,10 +135,7 @@ uint32_t HardwareCodecDeflate::doDecompressData(const char * source, uint32_t so job_ptr->available_out = uncompressed_size; job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; - // Decompression - status = qpl_execute_job(job_ptr); - - if (status == QPL_STS_OK) + if (auto status = qpl_execute_job(job_ptr); QPL_STS_OK == status) { DeflateJobHWPool::instance().releaseJob(job_id); return job_ptr->total_out; @@ -159,13 +151,12 @@ uint32_t HardwareCodecDeflate::doDecompressData(const char * source, uint32_t so uint32_t HardwareCodecDeflate::doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) { uint32_t job_id = 0; - qpl_job * job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); - if (job_ptr == nullptr) + qpl_job * job_ptr = nullptr; + if (job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); nullptr == job_ptr) { LOG_WARNING(log, "HardwareCodecDeflate::doDecompressDataReq acquireJob fail!"); return 0; } - qpl_status status; // Performing a decompression operation job_ptr->op = qpl_op_decompress; @@ -175,9 +166,7 @@ uint32_t HardwareCodecDeflate::doDecompressDataReq(const char * source, uint32_t job_ptr->available_out = uncompressed_size; job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; - // Decompression - status = qpl_submit_job(job_ptr); - if (QPL_STS_OK == status) + if (auto status = qpl_submit_job(job_ptr); QPL_STS_OK == status) { jobDecompAsyncMap.insert(std::make_pair(job_id, job_ptr)); return job_id; @@ -195,7 +184,6 @@ void HardwareCodecDeflate::flushAsynchronousDecompressRequests() uint32_t job_id = 0; qpl_job * job_ptr = nullptr; - std::map::iterator it; uint32_t n_jobs_processing = jobDecompAsyncMap.size(); it = jobDecompAsyncMap.begin(); @@ -244,19 +232,15 @@ qpl_job * SoftwareCodecDeflate::getJobCodecPtr() jobSWPtr = reinterpret_cast((std::make_unique(size)).get()); // Job initialization - auto status = qpl_init_job(qpl_path_software, jobSWPtr); - if (status != QPL_STS_OK) - { + if (auto status = qpl_init_job(qpl_path_software, jobSWPtr); QPL_STS_OK != status) throw Exception( "SoftwareCodecDeflate::getJobCodecPtr -> qpl_init_job fail:" + std::to_string(status), ErrorCodes::CANNOT_COMPRESS); - } } return jobSWPtr; } uint32_t SoftwareCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) { - qpl_status status; qpl_job * job_ptr = getJobCodecPtr(); // Performing a compression operation job_ptr->op = qpl_op_compress; @@ -267,20 +251,15 @@ uint32_t SoftwareCodecDeflate::doCompressData(const char * source, uint32_t sour job_ptr->level = qpl_default_level; job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; - // Compression - status = qpl_execute_job(job_ptr); - if (status != QPL_STS_OK) - { + if (auto status = qpl_execute_job(job_ptr); QPL_STS_OK != status) throw Exception( "SoftwareCodecDeflate::doCompressData -> qpl_execute_job fail:" + std::to_string(status), ErrorCodes::CANNOT_COMPRESS); - } return job_ptr->total_out; } void SoftwareCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) { - qpl_status status; qpl_job * job_ptr = getJobCodecPtr(); // Performing a decompression operation @@ -291,13 +270,9 @@ void SoftwareCodecDeflate::doDecompressData(const char * source, uint32_t source job_ptr->available_out = uncompressed_size; job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; - // Decompression - status = qpl_execute_job(job_ptr); - if (status != QPL_STS_OK) - { + if (auto status = qpl_execute_job(job_ptr); QPL_STS_OK != status) throw Exception( "SoftwareCodecDeflate::doDecompressData -> qpl_execute_job fail:" + std::to_string(status), ErrorCodes::CANNOT_DECOMPRESS); - } } //CompressionCodecDeflate From 7eb71143db947dbe91659801e8b05f08c905d9d2 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 8 Jul 2022 12:09:58 -0400 Subject: [PATCH 109/659] add a one-line method for resetting a job lock --- src/Compression/CompressionCodecDeflate.cpp | 4 ++-- src/Compression/CompressionCodecDeflate.h | 2 ++ 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 24f25e09eb2..7983ace29f4 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -40,7 +40,7 @@ DeflateJobHWPool::DeflateJobHWPool():log(&Poco::Logger::get("DeflateJobHWPool")) if ((nullptr == qpl_job_ptr) || (qpl_init_job(PATH, qpl_job_ptr) != QPL_STS_OK)) break; jobPool[index] = qpl_job_ptr; - jobLocks[index].store(false); + unLockJob(index); } const char * qpl_version = qpl_get_library_version(); @@ -66,7 +66,7 @@ DeflateJobHWPool::~DeflateJobHWPool() qpl_fini_job(jobPool[i]); } jobPool[i] = nullptr; - jobLocks[i].store(false); + unLockJob(i); } jobPoolReady() = false; } diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index e50dc89091e..473c7c44f6a 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -81,6 +81,8 @@ private: return jobLocks[index].compare_exchange_strong(expected, true); } + void unLockJob(uint32_t index) { jobLocks[index].store(false); } + class ReleaseJobObjectGuard { uint32_t index; From 289c6148e4edb77c85336af130362e5b16ede78b Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 8 Jul 2022 12:20:24 -0400 Subject: [PATCH 110/659] member to be LOWERCASE_UNDERSCORE --- src/Compression/CompressionCodecDeflate.cpp | 60 ++++++++++----------- src/Compression/CompressionCodecDeflate.h | 22 ++++---- 2 files changed, 41 insertions(+), 41 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 7983ace29f4..4ec8727a892 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -15,8 +15,8 @@ namespace ErrorCodes extern const int CANNOT_COMPRESS; extern const int CANNOT_DECOMPRESS; } -qpl_job * DeflateJobHWPool::jobPool[JOB_POOL_SIZE]; -std::atomic_bool DeflateJobHWPool::jobLocks[JOB_POOL_SIZE]; +qpl_job * DeflateJobHWPool::hw_job_pool[JOB_POOL_SIZE]; +std::atomic_bool DeflateJobHWPool::hw_job_locks[JOB_POOL_SIZE]; bool DeflateJobHWPool::job_pool_ready; DeflateJobHWPool & DeflateJobHWPool::instance() @@ -32,14 +32,14 @@ DeflateJobHWPool::DeflateJobHWPool():log(&Poco::Logger::get("DeflateJobHWPool")) /// get total size required for saving qpl job context qpl_get_job_size(PATH, &size); /// allocate buffer for storing all job objects - jobPoolBufferPtr = std::make_unique(size * JOB_POOL_SIZE); - memset(jobPool, 0, JOB_POOL_SIZE*sizeof(qpl_job *)); + hw_job_pool_buffer = std::make_unique(size * JOB_POOL_SIZE); + memset(hw_job_pool, 0, JOB_POOL_SIZE*sizeof(qpl_job *)); for (index = 0; index < JOB_POOL_SIZE; ++index) { - qpl_job * qpl_job_ptr = reinterpret_cast(jobPoolBufferPtr.get() + index*JOB_POOL_SIZE); + qpl_job * qpl_job_ptr = reinterpret_cast(hw_job_pool_buffer.get() + index*JOB_POOL_SIZE); if ((nullptr == qpl_job_ptr) || (qpl_init_job(PATH, qpl_job_ptr) != QPL_STS_OK)) break; - jobPool[index] = qpl_job_ptr; + hw_job_pool[index] = qpl_job_ptr; unLockJob(index); } @@ -60,12 +60,12 @@ DeflateJobHWPool::~DeflateJobHWPool() { for (uint32_t i = 0; i < JOB_POOL_SIZE; ++i) { - if (jobPool[i] != nullptr) + if (hw_job_pool[i] != nullptr) { while (!tryLockJob(i)); - qpl_fini_job(jobPool[i]); + qpl_fini_job(hw_job_pool[i]); } - jobPool[i] = nullptr; + hw_job_pool[i] = nullptr; unLockJob(i); } jobPoolReady() = false; @@ -73,21 +73,21 @@ DeflateJobHWPool::~DeflateJobHWPool() //HardwareCodecDeflate HardwareCodecDeflate::HardwareCodecDeflate(): - hwEnabled(DeflateJobHWPool::instance().jobPoolReady()), + hw_enabled(DeflateJobHWPool::instance().jobPoolReady()), log(&Poco::Logger::get("HardwareCodecDeflate")) { } HardwareCodecDeflate::~HardwareCodecDeflate() { - if (!jobDecompAsyncMap.empty()) + if (!decomp_async_job_map.empty()) { LOG_WARNING(log, "Find un-released job when HardwareCodecDeflate destroy"); - for (auto it : jobDecompAsyncMap) + for (auto it : decomp_async_job_map) { DeflateJobHWPool::instance().releaseJob(it.first); } - jobDecompAsyncMap.clear(); + decomp_async_job_map.clear(); } } uint32_t HardwareCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const @@ -168,7 +168,7 @@ uint32_t HardwareCodecDeflate::doDecompressDataReq(const char * source, uint32_t if (auto status = qpl_submit_job(job_ptr); QPL_STS_OK == status) { - jobDecompAsyncMap.insert(std::make_pair(job_id, job_ptr)); + decomp_async_job_map.insert(std::make_pair(job_id, job_ptr)); return job_id; } else @@ -185,8 +185,8 @@ void HardwareCodecDeflate::flushAsynchronousDecompressRequests() qpl_job * job_ptr = nullptr; std::map::iterator it; - uint32_t n_jobs_processing = jobDecompAsyncMap.size(); - it = jobDecompAsyncMap.begin(); + uint32_t n_jobs_processing = decomp_async_job_map.size(); + it = decomp_async_job_map.begin(); while (n_jobs_processing) { @@ -200,43 +200,43 @@ void HardwareCodecDeflate::flushAsynchronousDecompressRequests() else { DeflateJobHWPool::instance().releaseJob(job_id); - it = jobDecompAsyncMap.erase(it); + it = decomp_async_job_map.erase(it); n_jobs_processing--; if (n_jobs_processing <= 0) break; } - if (it == jobDecompAsyncMap.end()) + if (it == decomp_async_job_map.end()) { - it = jobDecompAsyncMap.begin(); + it = decomp_async_job_map.begin(); _tpause(1, __rdtsc() + 1000); } } } -SoftwareCodecDeflate::SoftwareCodecDeflate():jobSWPtr(nullptr) +SoftwareCodecDeflate::SoftwareCodecDeflate():sw_job(nullptr) { } SoftwareCodecDeflate::~SoftwareCodecDeflate() { - if (nullptr != jobSWPtr) - qpl_fini_job(jobSWPtr); + if (nullptr != sw_job) + qpl_fini_job(sw_job); } qpl_job * SoftwareCodecDeflate::getJobCodecPtr() { - if (nullptr == jobSWPtr) + if (nullptr == sw_job) { uint32_t size = 0; qpl_get_job_size(qpl_path_software, &size); - jobSWPtr = reinterpret_cast((std::make_unique(size)).get()); + sw_job = reinterpret_cast((std::make_unique(size)).get()); // Job initialization - if (auto status = qpl_init_job(qpl_path_software, jobSWPtr); QPL_STS_OK != status) + if (auto status = qpl_init_job(qpl_path_software, sw_job); QPL_STS_OK != status) throw Exception( "SoftwareCodecDeflate::getJobCodecPtr -> qpl_init_job fail:" + std::to_string(status), ErrorCodes::CANNOT_COMPRESS); } - return jobSWPtr; + return sw_job; } uint32_t SoftwareCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) @@ -302,7 +302,7 @@ uint32_t CompressionCodecDeflate::getMaxCompressedDataSize(uint32_t uncompressed uint32_t CompressionCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest) const { uint32_t res = 0; - if (hw_codec->hwEnabled) + if (hw_codec->hw_enabled) res = hw_codec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); if (0 == res) res = sw_codec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); @@ -316,7 +316,7 @@ void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t sou case CodecMode::Synchronous: { uint32_t res = 0; - if (hw_codec->hwEnabled) + if (hw_codec->hw_enabled) res = hw_codec->doDecompressData(source, source_size, dest, uncompressed_size); if (0 == res) sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); @@ -325,7 +325,7 @@ void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t sou case CodecMode::Asynchronous: { uint32_t res = 0; - if (hw_codec->hwEnabled) + if (hw_codec->hw_enabled) res = hw_codec->doDecompressDataReq(source, source_size, dest, uncompressed_size); if (0 == res) sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); @@ -339,7 +339,7 @@ void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t sou void CompressionCodecDeflate::flushAsynchronousDecompressRequests() { - if (hw_codec->hwEnabled) + if (hw_codec->hw_enabled) hw_codec->flushAsynchronousDecompressRequests(); setDecompressMode(CodecMode::Synchronous); } diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index 473c7c44f6a..c8631d843e1 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -20,8 +20,8 @@ public: static DeflateJobHWPool & instance(); static constexpr auto JOB_POOL_SIZE = 1024; static constexpr qpl_path_t PATH = qpl_path_hardware; - static qpl_job * jobPool[JOB_POOL_SIZE]; - static std::atomic_bool jobLocks[JOB_POOL_SIZE]; + static qpl_job * hw_job_pool[JOB_POOL_SIZE]; + static std::atomic_bool hw_job_locks[JOB_POOL_SIZE]; static bool job_pool_ready; bool & jobPoolReady() { return job_pool_ready;} @@ -42,7 +42,7 @@ public: } } *job_id = JOB_POOL_SIZE - index; - return jobPool[index]; + return hw_job_pool[index]; } else { @@ -56,7 +56,7 @@ public: { uint32_t index = JOB_POOL_SIZE - job_id; ReleaseJobObjectGuard _(index); - return jobPool[index]; + return hw_job_pool[index]; } else { @@ -78,10 +78,10 @@ private: { bool expected = false; assert(index < JOB_POOL_SIZE); - return jobLocks[index].compare_exchange_strong(expected, true); + return hw_job_locks[index].compare_exchange_strong(expected, true); } - void unLockJob(uint32_t index) { jobLocks[index].store(false); } + void unLockJob(uint32_t index) { hw_job_locks[index].store(false); } class ReleaseJobObjectGuard { @@ -95,10 +95,10 @@ private: ~ReleaseJobObjectGuard() { - jobLocks[index].store(false); + hw_job_locks[index].store(false); } }; - std::unique_ptr jobPoolBufferPtr; + std::unique_ptr hw_job_pool_buffer; Poco::Logger * log; }; @@ -111,14 +111,14 @@ public: void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); private: - qpl_job * jobSWPtr; + qpl_job * sw_job; qpl_job * getJobCodecPtr(); }; class HardwareCodecDeflate { public: - bool hwEnabled; + bool hw_enabled; HardwareCodecDeflate(); ~HardwareCodecDeflate(); uint32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const; @@ -127,7 +127,7 @@ public: void flushAsynchronousDecompressRequests(); private: - std::map jobDecompAsyncMap; + std::map decomp_async_job_map; Poco::Logger * log; }; class CompressionCodecDeflate : public ICompressionCodec From 46d7d3b5c9dfb32846c2a665a9ccf3a4079baf13 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 8 Jul 2022 15:03:10 -0400 Subject: [PATCH 111/659] constant always goes to the right side --- src/Compression/CompressionCodecDeflate.cpp | 22 ++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 4ec8727a892..4fd365ffd6f 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -37,7 +37,7 @@ DeflateJobHWPool::DeflateJobHWPool():log(&Poco::Logger::get("DeflateJobHWPool")) for (index = 0; index < JOB_POOL_SIZE; ++index) { qpl_job * qpl_job_ptr = reinterpret_cast(hw_job_pool_buffer.get() + index*JOB_POOL_SIZE); - if ((nullptr == qpl_job_ptr) || (qpl_init_job(PATH, qpl_job_ptr) != QPL_STS_OK)) + if ((!qpl_job_ptr) || (qpl_init_job(PATH, qpl_job_ptr) != QPL_STS_OK)) break; hw_job_pool[index] = qpl_job_ptr; unLockJob(index); @@ -95,7 +95,7 @@ uint32_t HardwareCodecDeflate::doCompressData(const char * source, uint32_t sour uint32_t job_id = 0; qpl_job* job_ptr = nullptr; uint32_t compressed_size = 0; - if (job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); nullptr == job_ptr) + if (!(job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id))) { LOG_WARNING(log, "HardwareCodecDeflate::doCompressData acquireJob fail!"); return 0; @@ -109,7 +109,7 @@ uint32_t HardwareCodecDeflate::doCompressData(const char * source, uint32_t sour job_ptr->available_out = dest_size; job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; - if (auto status = qpl_execute_job(job_ptr); QPL_STS_OK == status) + if (auto status = qpl_execute_job(job_ptr); status == QPL_STS_OK) compressed_size = job_ptr->total_out; else LOG_WARNING(log, "HardwareCodecDeflate::doCompressData fail ->status: '{}' ", static_cast(status)); @@ -122,7 +122,7 @@ uint32_t HardwareCodecDeflate::doDecompressData(const char * source, uint32_t so { uint32_t job_id = 0; qpl_job* job_ptr = nullptr; - if (job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); nullptr == job_ptr) + if (!(job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id))) { LOG_WARNING(log, "HardwareCodecDeflate::doDecompressData acquireJob fail!"); return 0; @@ -135,7 +135,7 @@ uint32_t HardwareCodecDeflate::doDecompressData(const char * source, uint32_t so job_ptr->available_out = uncompressed_size; job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; - if (auto status = qpl_execute_job(job_ptr); QPL_STS_OK == status) + if (auto status = qpl_execute_job(job_ptr); status == QPL_STS_OK) { DeflateJobHWPool::instance().releaseJob(job_id); return job_ptr->total_out; @@ -152,7 +152,7 @@ uint32_t HardwareCodecDeflate::doDecompressDataReq(const char * source, uint32_t { uint32_t job_id = 0; qpl_job * job_ptr = nullptr; - if (job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id); nullptr == job_ptr) + if (!(job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id))) { LOG_WARNING(log, "HardwareCodecDeflate::doDecompressDataReq acquireJob fail!"); return 0; @@ -166,7 +166,7 @@ uint32_t HardwareCodecDeflate::doDecompressDataReq(const char * source, uint32_t job_ptr->available_out = uncompressed_size; job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; - if (auto status = qpl_submit_job(job_ptr); QPL_STS_OK == status) + if (auto status = qpl_submit_job(job_ptr); status == QPL_STS_OK) { decomp_async_job_map.insert(std::make_pair(job_id, job_ptr)); return job_id; @@ -225,14 +225,14 @@ SoftwareCodecDeflate::~SoftwareCodecDeflate() qpl_job * SoftwareCodecDeflate::getJobCodecPtr() { - if (nullptr == sw_job) + if (!sw_job) { uint32_t size = 0; qpl_get_job_size(qpl_path_software, &size); sw_job = reinterpret_cast((std::make_unique(size)).get()); // Job initialization - if (auto status = qpl_init_job(qpl_path_software, sw_job); QPL_STS_OK != status) + if (auto status = qpl_init_job(qpl_path_software, sw_job); status != QPL_STS_OK) throw Exception( "SoftwareCodecDeflate::getJobCodecPtr -> qpl_init_job fail:" + std::to_string(status), ErrorCodes::CANNOT_COMPRESS); } @@ -251,7 +251,7 @@ uint32_t SoftwareCodecDeflate::doCompressData(const char * source, uint32_t sour job_ptr->level = qpl_default_level; job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; - if (auto status = qpl_execute_job(job_ptr); QPL_STS_OK != status) + if (auto status = qpl_execute_job(job_ptr); status != QPL_STS_OK) throw Exception( "SoftwareCodecDeflate::doCompressData -> qpl_execute_job fail:" + std::to_string(status), ErrorCodes::CANNOT_COMPRESS); @@ -270,7 +270,7 @@ void SoftwareCodecDeflate::doDecompressData(const char * source, uint32_t source job_ptr->available_out = uncompressed_size; job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; - if (auto status = qpl_execute_job(job_ptr); QPL_STS_OK != status) + if (auto status = qpl_execute_job(job_ptr); status != QPL_STS_OK) throw Exception( "SoftwareCodecDeflate::doDecompressData -> qpl_execute_job fail:" + std::to_string(status), ErrorCodes::CANNOT_DECOMPRESS); } From 50f1edabf6b7d289c83ed56e0c790b8b2d552238 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 8 Jul 2022 19:27:16 +0000 Subject: [PATCH 112/659] add more settings for randomization --- tests/clickhouse-test | 35 +++++++++++++++++++++++++++++++---- 1 file changed, 31 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index cab6daf3a50..4ced036683b 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -426,6 +426,26 @@ class SettingsRandomizer: "read_in_order_two_level_merge_threshold": lambda: random.randint(0, 100), "optimize_aggregation_in_order": lambda: random.randint(0, 1), "aggregation_in_order_max_block_bytes": lambda: random.randint(0, 50000000), + "use_uncompressed_cache": lambda: random.randint(0, 1), + "min_bytes_to_use_direct_io": lambda: 0 + if random.random() < 0.5 + else 1 + if random.random() < 0.2 + else random.randint(1, 1024 * 1024 * 1024), + "min_bytes_to_use_mmap_io": lambda: 0 + if random.random() < 0.5 + else 1 + if random.random() < 0.2 + else random.randint(1, 1024 * 1024 * 1024), + "local_filesystem_read_method": lambda: random.choice( + ["read", "pread", "mmap", "pread_threadpool"] + ), + "remote_filesystem_read_method": lambda: random.choice(["read", "threadpool"]), + "local_filesystem_read_prefetch": lambda: random.randint(0, 1), + "remote_filesystem_read_prefetch": lambda: random.randint(0, 1), + "compile_expressions": lambda: random.randint(0, 1), + "compile_aggregate_expressions": lambda: random.randint(0, 1), + "compile_sort_description": lambda: random.randint(0, 1), } @staticmethod @@ -710,7 +730,9 @@ class TestCase: return None - def process_result_impl(self, proc, stdout: str, stderr: str, debug_log: str, total_time: float): + def process_result_impl( + self, proc, stdout: str, stderr: str, debug_log: str, total_time: float + ): description = "" if proc: @@ -1038,7 +1060,9 @@ class TestCase: server_logs_level, client_options ) - result = self.process_result_impl(proc, stdout, stderr, debug_log, total_time) + result = self.process_result_impl( + proc, stdout, stderr, debug_log, total_time + ) result.check_if_need_retry(args, stdout, stderr, self.runs_count) if result.status == TestStatus.FAIL: result.description = self.add_info_about_settings( @@ -1530,7 +1554,8 @@ def collect_build_flags(args): result.append(BuildFlags.RELEASE) value = clickhouse_execute( - args, "SELECT value FROM system.settings WHERE name = 'allow_deprecated_database_ordinary'" + args, + "SELECT value FROM system.settings WHERE name = 'allow_deprecated_database_ordinary'", ) if value == b"1": result.append(BuildFlags.ORDINARY_DATABASE) @@ -1634,7 +1659,9 @@ def do_run_tests(jobs, test_suite: TestSuite, parallel): queue.close() except Full: - print("Couldn't put test to the queue within timeout. Server probably hung.") + print( + "Couldn't put test to the queue within timeout. Server probably hung." + ) print_stacktraces() queue.close() From 4f4acfabe5ff40e8552c509cd48ccec10ccf43a7 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 8 Jul 2022 17:52:14 -0300 Subject: [PATCH 113/659] c-ares draft wrapper and draft build rules --- .gitmodules | 9 ++-- contrib/CMakeLists.txt | 3 +- contrib/c-ares | 1 + contrib/c-ares-cmake/CMakeLists.txt | 37 ++++++++++++++ contrib/cpp-dns | 1 - contrib/cpp-dns-cmake/CMakeLists.txt | 11 ----- contrib/grpc-cmake/CMakeLists.txt | 5 ++ contrib/udns | 1 - contrib/udns-cmake/CMakeLists.txt | 26 ---------- src/CMakeLists.txt | 4 +- src/Common/CARESPTRResolver.cpp | 69 +++++++++++++++++++++++++++ src/Common/CARESPTRResolver.h | 26 ++++++++++ src/Common/DNSPTRResolver.h | 14 ++++++ src/Common/DNSPTRResolverProvider.cpp | 12 +++++ src/Common/DNSPTRResolverProvider.h | 13 +++++ src/Common/DNSResolver.cpp | 23 ++------- 16 files changed, 186 insertions(+), 69 deletions(-) create mode 160000 contrib/c-ares create mode 100644 contrib/c-ares-cmake/CMakeLists.txt delete mode 160000 contrib/cpp-dns delete mode 100644 contrib/cpp-dns-cmake/CMakeLists.txt delete mode 160000 contrib/udns delete mode 100644 contrib/udns-cmake/CMakeLists.txt create mode 100644 src/Common/CARESPTRResolver.cpp create mode 100644 src/Common/CARESPTRResolver.h create mode 100644 src/Common/DNSPTRResolver.h create mode 100644 src/Common/DNSPTRResolverProvider.cpp create mode 100644 src/Common/DNSPTRResolverProvider.h diff --git a/.gitmodules b/.gitmodules index 0b813722c60..7f88dd3c714 100644 --- a/.gitmodules +++ b/.gitmodules @@ -274,9 +274,6 @@ [submodule "contrib/base-x"] path = contrib/base-x url = https://github.com/ClickHouse/base-x.git -[submodule "contrib/udns"] - path = contrib/udns - url = https://github.com/ClickHouse/udns -[submodule "contrib/cpp-dns"] - path = contrib/cpp-dns - url = https://github.com/ClickHouse/cpp-dns \ No newline at end of file +[submodule "contrib/c-ares"] + path = contrib/c-ares + url = https://github.com/ClickHouse/c-ares diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 06e90dedb5b..460a1d7a3ee 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -156,8 +156,7 @@ endif() add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) add_contrib (base-x-cmake base-x) -add_contrib (udns-cmake udns) -add_contrib (cpp-dns-cmake cpp-dns) +add_contrib(c-ares-cmake c-ares) # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear diff --git a/contrib/c-ares b/contrib/c-ares new file mode 160000 index 00000000000..afee6748b0b --- /dev/null +++ b/contrib/c-ares @@ -0,0 +1 @@ +Subproject commit afee6748b0b99acf4509d42fa37ac8422262f91b diff --git a/contrib/c-ares-cmake/CMakeLists.txt b/contrib/c-ares-cmake/CMakeLists.txt new file mode 100644 index 00000000000..87f2b96fac3 --- /dev/null +++ b/contrib/c-ares-cmake/CMakeLists.txt @@ -0,0 +1,37 @@ +if(NOT TARGET c-ares) + # Choose to build static or shared library for c-ares. + if (USE_STATIC_LIBRARIES) + set(CARES_STATIC ON CACHE BOOL "" FORCE) + set(CARES_SHARED OFF CACHE BOOL "" FORCE) + else () + set(CARES_STATIC OFF CACHE BOOL "" FORCE) + set(CARES_SHARED ON CACHE BOOL "" FORCE) + endif () + + # Disable looking for libnsl on a platforms that has gethostbyname in glibc + # + # c-ares searching for gethostbyname in the libnsl library, however in the + # version that shipped with gRPC it doing it wrong [1], since it uses + # CHECK_LIBRARY_EXISTS(), which will return TRUE even if the function exists in + # another dependent library. The upstream already contains correct macro [2], + # but it is not included in gRPC (even upstream gRPC, not the one that is + # shipped with clickhousee). + # + # [1]: https://github.com/c-ares/c-ares/blob/e982924acee7f7313b4baa4ee5ec000c5e373c30/CMakeLists.txt#L125 + # [2]: https://github.com/c-ares/c-ares/blob/44fbc813685a1fa8aa3f27fcd7544faf612d376a/CMakeLists.txt#L146 + # + # And because if you by some reason have libnsl [3] installed, clickhouse will + # reject to start w/o it. While this is completelly different library. + # + # [3]: https://packages.debian.org/bullseye/libnsl2 + if (NOT CMAKE_SYSTEM_NAME STREQUAL "SunOS") + set(HAVE_LIBNSL OFF CACHE BOOL "" FORCE) + endif() + + add_subdirectory("../c-ares/" "../c-ares/") + + add_library(ch_contrib::c-ares ALIAS c-ares) + + target_link_libraries(c-ares PRIVATE resolv) + +endif() \ No newline at end of file diff --git a/contrib/cpp-dns b/contrib/cpp-dns deleted file mode 160000 index 0ce2dbd40dc..00000000000 --- a/contrib/cpp-dns +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 0ce2dbd40dce456bde1732c4d8f70008f0079db0 diff --git a/contrib/cpp-dns-cmake/CMakeLists.txt b/contrib/cpp-dns-cmake/CMakeLists.txt deleted file mode 100644 index 20e6eb04088..00000000000 --- a/contrib/cpp-dns-cmake/CMakeLists.txt +++ /dev/null @@ -1,11 +0,0 @@ -set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/cpp-dns") - -set(SRCS "${LIBRARY_DIR}/DNSResolver.cpp") - -add_library(_cpp-dns ${SRCS}) - -target_link_libraries(_cpp-dns boost::system boost::headers_only ch_contrib::udns) - -target_include_directories(_cpp-dns SYSTEM BEFORE PUBLIC ${LIBRARY_DIR}) - -add_library(ch_contrib::cpp-dns ALIAS _cpp-dns) \ No newline at end of file diff --git a/contrib/grpc-cmake/CMakeLists.txt b/contrib/grpc-cmake/CMakeLists.txt index 520e04d198e..9609b922840 100644 --- a/contrib/grpc-cmake/CMakeLists.txt +++ b/contrib/grpc-cmake/CMakeLists.txt @@ -92,7 +92,12 @@ if(gRPC_USE_UNSECURE_LIBRARIES) else() set(gRPC_LIBRARIES grpc grpc++) endif() + +target_compile_options(c-ares PRIVATE -Wno-reserved-macro-identifier) + add_library(_ch_contrib_grpc INTERFACE) target_link_libraries(_ch_contrib_grpc INTERFACE ${gRPC_LIBRARIES}) target_include_directories(_ch_contrib_grpc SYSTEM INTERFACE ${gRPC_INCLUDE_DIRS}) add_library(ch_contrib::grpc ALIAS _ch_contrib_grpc) + +target_compile_options(grpc PRIVATE -Wno-reserved-macro-identifier) diff --git a/contrib/udns b/contrib/udns deleted file mode 160000 index f535fbe9b39..00000000000 --- a/contrib/udns +++ /dev/null @@ -1 +0,0 @@ -Subproject commit f535fbe9b39f6bbbf91475834cfefef9c7edab75 diff --git a/contrib/udns-cmake/CMakeLists.txt b/contrib/udns-cmake/CMakeLists.txt deleted file mode 100644 index 1266ea2b847..00000000000 --- a/contrib/udns-cmake/CMakeLists.txt +++ /dev/null @@ -1,26 +0,0 @@ -set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/udns") - -add_library(_udns - "${LIBRARY_DIR}/udns_bl.c" - "${LIBRARY_DIR}/udns_codes.c" - "${LIBRARY_DIR}/udns_dntosp.c" - "${LIBRARY_DIR}/udns_init.c" - "${LIBRARY_DIR}/udns_jran.c" - "${LIBRARY_DIR}/udns_misc.c" - "${LIBRARY_DIR}/udns_parse.c" - "${LIBRARY_DIR}/udns_resolver.c" - "${LIBRARY_DIR}/udns_rr_a.c" - "${LIBRARY_DIR}/udns_rr_mx.c" - "${LIBRARY_DIR}/udns_rr_naptr.c" - "${LIBRARY_DIR}/udns_rr_ptr.c" - "${LIBRARY_DIR}/udns_rr_srv.c" - "${LIBRARY_DIR}/udns_rr_txt.c" - "${LIBRARY_DIR}/udns_XtoX.c" - "${LIBRARY_DIR}/udns_dn.c" - ) - -target_compile_definitions(_udns PRIVATE -DHAVE_CONFIG_H) - -target_include_directories(_udns SYSTEM BEFORE PUBLIC ${LIBRARY_DIR}) - -add_library(ch_contrib::udns ALIAS _udns) \ No newline at end of file diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index dfff3bcfa1d..7f5b97cc52b 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -439,8 +439,8 @@ if (TARGET ch_contrib::avrocpp) dbms_target_link_libraries(PRIVATE ch_contrib::avrocpp) endif () -target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::udns) -target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::cpp-dns) +target_compile_options(clickhouse_common_io PRIVATE -Wno-reserved-identifier) +target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::c-ares) if (TARGET OpenSSL::Crypto) dbms_target_link_libraries (PRIVATE OpenSSL::Crypto) diff --git a/src/Common/CARESPTRResolver.cpp b/src/Common/CARESPTRResolver.cpp new file mode 100644 index 00000000000..eac390c92c5 --- /dev/null +++ b/src/Common/CARESPTRResolver.cpp @@ -0,0 +1,69 @@ +#include +#include "CARESPTRResolver.h" +#include "netdb.h" + +namespace DB { + static void callback(void * arg, int status, int, struct hostent * host) { + auto * ptr_records = reinterpret_cast*>(arg); + if (status == ARES_SUCCESS) { + int i = 0; + while (auto ptr_record = host->h_aliases[i]) { + ptr_records->emplace_back(ptr_record); + i++; + } + } + } + + CARESPTRResolver::CARESPTRResolver() { + init(); + } + + CARESPTRResolver::~CARESPTRResolver() { + deinit(); + } + + std::vector CARESPTRResolver::resolve(const std::string & ip) { + std::vector ptr_records; + + resolve(ip, ptr_records); + wait(); + + return ptr_records; + } + + void CARESPTRResolver::init() { + if (ares_init(&channel) != ARES_SUCCESS){ + throw std::exception {}; + } + } + + void CARESPTRResolver::deinit() { + ares_destroy(channel); + ares_library_cleanup(); + } + + void CARESPTRResolver::resolve(const std::string & ip, std::vector & response) { + in_addr addr; + inet_aton(ip.c_str(), &addr); + + ares_gethostbyaddr(channel, reinterpret_cast(&addr), sizeof(addr), AF_INET, callback, &response); + } + + void CARESPTRResolver::wait() { + for(;;) { + timeval * tvp, tv; + fd_set read_fds, write_fds; + int nfds; + + FD_ZERO(&read_fds); + FD_ZERO(&write_fds); + nfds = ares_fds(channel, &read_fds, &write_fds); + if(nfds == 0){ + break; + } + tvp = ares_timeout(channel, nullptr, &tv); + select(nfds, &read_fds, &write_fds, nullptr, tvp); + ares_process(channel, &read_fds, &write_fds); + } + } +} diff --git a/src/Common/CARESPTRResolver.h b/src/Common/CARESPTRResolver.h new file mode 100644 index 00000000000..645723e9da1 --- /dev/null +++ b/src/Common/CARESPTRResolver.h @@ -0,0 +1,26 @@ +#pragma once + +#include "DNSPTRResolver.h" +#include "ares.h" + +namespace DB { + class CARESPTRResolver : public DNSPTRResolver { + + public: + CARESPTRResolver(); + ~CARESPTRResolver() override; + + std::vector resolve(const std::string & ip) override; + + private: + + void init(); + void deinit(); + void wait(); + + void resolve(const std::string & ip, std::vector & response); + + ares_channel channel; + }; +} + diff --git a/src/Common/DNSPTRResolver.h b/src/Common/DNSPTRResolver.h new file mode 100644 index 00000000000..20a5422767a --- /dev/null +++ b/src/Common/DNSPTRResolver.h @@ -0,0 +1,14 @@ +#pragma once + +#include +#include + +namespace DB { + struct DNSPTRResolver { + + virtual ~DNSPTRResolver() = default; + + virtual std::vector resolve(const std::string & ip) = 0; + + }; +} diff --git a/src/Common/DNSPTRResolverProvider.cpp b/src/Common/DNSPTRResolverProvider.cpp new file mode 100644 index 00000000000..ef19024495f --- /dev/null +++ b/src/Common/DNSPTRResolverProvider.cpp @@ -0,0 +1,12 @@ +#include "DNSPTRResolverProvider.h" +#include "CARESPTRResolver.h" + +namespace DB { + + std::shared_ptr DNSPTRResolverProvider::get() + { + static auto cares_resolver = std::make_shared(); + return cares_resolver; + } + +} diff --git a/src/Common/DNSPTRResolverProvider.h b/src/Common/DNSPTRResolverProvider.h new file mode 100644 index 00000000000..17764d8c91d --- /dev/null +++ b/src/Common/DNSPTRResolverProvider.h @@ -0,0 +1,13 @@ +#pragma once + +#include +#include "DNSPTRResolver.h" + +namespace DB { + class DNSPTRResolverProvider + { + public: + static std::shared_ptr get(); + }; +} + diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index dff1f8e7521..4c32dbf3341 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -13,7 +13,7 @@ #include #include #include -#include +#include "DNSPTRResolverProvider.h" namespace ProfileEvents { @@ -142,25 +142,8 @@ static DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host) static std::vector reverseResolveImpl(const Poco::Net::IPAddress & address) { - boost::asio::io_service io_service; - YukiWorkshop::DNSResolver resolver(io_service); - - std::vector ptr_records; - - resolver.resolve_a4ptr(boost::asio::ip::address_v4::from_string(address.toString()), [&](int err, auto & hosts, auto &, auto &, uint) { - if (err) { - throw Exception("Cannot resolve: " + address.toString() + YukiWorkshop::DNSResolver::error_string(err), ErrorCodes::DNS_ERROR); - } - - for (auto & it : hosts) { - ptr_records.emplace_back(it); - } - - }); - - io_service.run(); - - return ptr_records; + auto ptr_resolver = DB::DNSPTRResolverProvider::get(); + return ptr_resolver->resolve(address.toString()); } struct DNSResolver::Impl From c7967fb721b145958d0948c7596c9105bcc8ee2c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 5 Jul 2022 20:03:51 +0200 Subject: [PATCH 114/659] Add an option to INTO OUTFILE to also print details to standard output. Implementation: - Added a bool to ASTQueryWithOutput & patched the usage in ClientBase. - Added a new buffer TeeWriteBuffer which extends from WriteBufferFromFile (used to write data to the file) and has WriteBufferFromFileDescriptor (used to write data to stdout). The WriteBufferFromFileDescriptor uses the same buffer as TeeWriteBuffer. - Added a new bool select_into_outfile_and_stdout in ClientBase to enable/disable progress rendering. Testing: - Added a test tests/queries/0_stateless/02346_into_outfile_and_stdout.sh Documentation: - Updated the english documentation for the new option in SELECT. --- .../statements/select/into-outfile.md | 3 +- src/Client/ClientBase.cpp | 33 ++++++++----- src/Client/ClientBase.h | 1 + src/IO/TeeWriteBuffer.cpp | 46 +++++++++++++++++++ src/IO/TeeWriteBuffer.h | 34 ++++++++++++++ src/Parsers/ASTQueryWithOutput.h | 1 + src/Parsers/ParserQueryWithOutput.cpp | 7 +++ .../02346_into_outfile_and_stdout.reference | 9 ++++ .../02346_into_outfile_and_stdout.sh | 45 ++++++++++++++++++ 9 files changed, 167 insertions(+), 12 deletions(-) create mode 100644 src/IO/TeeWriteBuffer.cpp create mode 100644 src/IO/TeeWriteBuffer.h create mode 100644 tests/queries/0_stateless/02346_into_outfile_and_stdout.reference create mode 100644 tests/queries/0_stateless/02346_into_outfile_and_stdout.sh diff --git a/docs/en/sql-reference/statements/select/into-outfile.md b/docs/en/sql-reference/statements/select/into-outfile.md index f101c10ff60..eb0cf68eca1 100644 --- a/docs/en/sql-reference/statements/select/into-outfile.md +++ b/docs/en/sql-reference/statements/select/into-outfile.md @@ -11,7 +11,7 @@ Compressed files are supported. Compression type is detected by the extension of **Syntax** ```sql -SELECT INTO OUTFILE file_name [COMPRESSION type [LEVEL level]] +SELECT INTO OUTFILE file_name [AND STDOUT] [COMPRESSION type [LEVEL level]] ``` `file_name` and `type` are string literals. Supported compression types are: `'none'`, `'gzip'`, `'deflate'`, `'br'`, `'xz'`, `'zstd'`, `'lz4'`, `'bz2'`. @@ -23,6 +23,7 @@ SELECT INTO OUTFILE file_name [COMPRESSION type [LEVEL level]] - This functionality is available in the [command-line client](../../../interfaces/cli.md) and [clickhouse-local](../../../operations/utilities/clickhouse-local.md). Thus a query sent via [HTTP interface](../../../interfaces/http.md) will fail. - The query will fail if a file with the same file name already exists. - The default [output format](../../../interfaces/formats.md) is `TabSeparated` (like in the command-line client batch mode). Use [FORMAT](format.md) clause to change it. +- If `AND STDOUT` is mentioned in the query then the output that is written to the file is also displayed on standard output. **Example** diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 0e243f97aaf..d97f01617a6 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -69,6 +69,7 @@ #include #include #include +#include namespace fs = std::filesystem; @@ -404,7 +405,6 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) return; processed_rows += block.rows(); - /// Even if all blocks are empty, we still need to initialize the output stream to write empty resultset. initOutputFormat(block, parsed_query); @@ -415,7 +415,7 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) return; /// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker. - if (need_render_progress && (stdout_is_a_tty || is_interactive) && !select_into_file) + if (need_render_progress && (stdout_is_a_tty || is_interactive) && (!select_into_file || select_into_file_and_stdout)) progress_indication.clearProgressOutput(); try @@ -435,7 +435,7 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) /// Restore progress bar after data block. if (need_render_progress && (stdout_is_a_tty || is_interactive)) { - if (select_into_file) + if (select_into_file && !select_into_file_and_stdout) std::cerr << "\r"; progress_indication.writeProgress(); } @@ -512,7 +512,7 @@ try String current_format = format; select_into_file = false; - + select_into_file_and_stdout = false; /// The query can specify output format or output file. if (const auto * query_with_output = dynamic_cast(parsed_query.get())) { @@ -548,12 +548,23 @@ try range.first, range.second); } - - out_file_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), - compression_method, - compression_level - ); + if (query_with_output->is_stdout_enabled) + { + select_into_file_and_stdout = true; + out_file_buf = wrapWriteBufferWithCompressionMethod( + std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), + compression_method, + compression_level + ); + } + else + { + out_file_buf = wrapWriteBufferWithCompressionMethod( + std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), + compression_method, + compression_level + ); + } // We are writing to file, so default format is the same as in non-interactive mode. if (is_interactive && is_default_format) @@ -579,7 +590,7 @@ try /// It is not clear how to write progress intermixed with data with parallel formatting. /// It may increase code complexity significantly. - if (!need_render_progress || select_into_file) + if (!need_render_progress || (select_into_file && !select_into_file_and_stdout)) output_format = global_context->getOutputFormatParallelIfPossible( current_format, out_file_buf ? *out_file_buf : *out_buf, block); else diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index ec2267a3be6..45c37617647 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -180,6 +180,7 @@ protected: String format; /// Query results output format. bool select_into_file = false; /// If writing result INTO OUTFILE. It affects progress rendering. + bool select_into_file_and_stdout = false; /// If writing result INTO OUTFILE AND STDOUT. It affects progress rendering. bool is_default_format = true; /// false, if format is set in the config or command line. size_t format_max_block_size = 0; /// Max block size for console output. String insert_format; /// Format of INSERT data that is read from stdin in batch mode. diff --git a/src/IO/TeeWriteBuffer.cpp b/src/IO/TeeWriteBuffer.cpp new file mode 100644 index 00000000000..5131fbdeacf --- /dev/null +++ b/src/IO/TeeWriteBuffer.cpp @@ -0,0 +1,46 @@ +#include + +namespace DB +{ + +TeeWriteBuffer::TeeWriteBuffer( + const std::string & file_name_, + size_t buf_size, + int flags, + mode_t mode, + char * existing_memory, + size_t alignment) + : WriteBufferFromFile(file_name_,buf_size,flags,mode,existing_memory,alignment), + stdout_buffer(STDOUT_FILENO,buf_size,working_buffer.begin()) +{ +} + +void TeeWriteBuffer::nextImpl() +{ + try + { + stdout_buffer.position() = position(); + stdout_buffer.next(); + WriteBufferFromFile::nextImpl(); + } + catch (Exception &exception) + { + exception.addMessage("While writing to TeeWriteBuffer "); + throw; + } +} + +void TeeWriteBuffer::finalizeImpl() +{ + if (fd < 0 || stdout_buffer.getFD() < 0) + return; + + next(); +} + +TeeWriteBuffer::~TeeWriteBuffer() +{ + finalize(); +} + +} diff --git a/src/IO/TeeWriteBuffer.h b/src/IO/TeeWriteBuffer.h new file mode 100644 index 00000000000..264ba42b0a8 --- /dev/null +++ b/src/IO/TeeWriteBuffer.h @@ -0,0 +1,34 @@ +#pragma once + +#include + +namespace DB +{ + +/** TeeWriteBuffer extends from WriteBufferFromFile and has + * WriteBufferFromFileDescriptor inside the class which is created + * by using the same buffer as TeeWriteBuffer. So both the data are written + * using same buffer + **/ +class TeeWriteBuffer : public WriteBufferFromFile +{ + +public: + explicit TeeWriteBuffer( + const std::string & file_name_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + int flags = -1, + mode_t mode = 0666, + char * existing_memory = nullptr, + size_t alignment = 0); + + ~TeeWriteBuffer() override; + +protected: + void nextImpl() override; + void finalizeImpl() override; + + WriteBufferFromFileDescriptor stdout_buffer; +}; + +} diff --git a/src/Parsers/ASTQueryWithOutput.h b/src/Parsers/ASTQueryWithOutput.h index a34826d128c..d2a24f1ebe2 100644 --- a/src/Parsers/ASTQueryWithOutput.h +++ b/src/Parsers/ASTQueryWithOutput.h @@ -15,6 +15,7 @@ class ASTQueryWithOutput : public IAST { public: ASTPtr out_file; + bool is_stdout_enabled; ASTPtr format; ASTPtr settings_ast; ASTPtr compression; diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 79203c6d3d1..0f900ed8856 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -107,6 +107,13 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec } query_with_output.children.push_back(query_with_output.out_file); + + ParserKeyword s_stdout("AND STDOUT"); + if (s_stdout.ignore(pos, expected)) + { + query_with_output.is_stdout_enabled = true; + } + } ParserKeyword s_format("FORMAT"); diff --git a/tests/queries/0_stateless/02346_into_outfile_and_stdout.reference b/tests/queries/0_stateless/02346_into_outfile_and_stdout.reference new file mode 100644 index 00000000000..09984d86ca3 --- /dev/null +++ b/tests/queries/0_stateless/02346_into_outfile_and_stdout.reference @@ -0,0 +1,9 @@ +performing test: select +1 2 3 +1 2 3 +performing test: bad_query_incorrect_usage +query failed +performing test: bad_query_no_into_outfile +query failed +performing test: bad_query_file_exists +query failed diff --git a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh new file mode 100644 index 00000000000..76b308d003c --- /dev/null +++ b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh @@ -0,0 +1,45 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function perform() +{ + local test_id=$1 + local query=$2 + + echo "performing test: $test_id" + ${CLICKHOUSE_CLIENT} --query "$query" 2>/dev/null + if [ "$?" -eq 0 ]; then + cat "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_$test_id.out" + else + echo "query failed" + fi + rm -f "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_$test_id.out" +} + +function performFileExists() +{ + local test_id=$1 + local query=$2 + + touch "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_$test_id.out" + + echo "performing test: $test_id" + ${CLICKHOUSE_CLIENT} --query "$query" 2>/dev/null + if [ "$?" -eq 0 ]; then + cat "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_$test_id.out" + else + echo "query failed" + fi + rm -f "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_$test_id.out" +} + +perform "select" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_select.out' AND STDOUT" + +perform "bad_query_incorrect_usage" "SELECT 1, 2, 3 INTO OUTFILE AND STDOUT'${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_incorrect_usage.out'" + +perform "bad_query_no_into_outfile" "SELECT 1, 2, 3 AND STDOUT'" + +performFileExists "bad_query_file_exists" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out' AND STDOUT" \ No newline at end of file From d55d190b9929065c8758a8934c059c4cab0c67d1 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 9 Jul 2022 14:53:53 +0200 Subject: [PATCH 115/659] added execute permission on the 02346_into_outfile_and_stdout.sh test file --- tests/queries/0_stateless/02346_into_outfile_and_stdout.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/02346_into_outfile_and_stdout.sh diff --git a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh old mode 100644 new mode 100755 From 545e6b305c174a775918ed8cfd6088137cf9f773 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sat, 9 Jul 2022 10:09:23 -0400 Subject: [PATCH 116/659] improve log print text --- src/Compression/CompressionCodecDeflate.cpp | 20 ++++++++------------ src/Compression/CompressionCodecDeflate.h | 6 ++++-- 2 files changed, 12 insertions(+), 14 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 4fd365ffd6f..36fb3db9aa3 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -47,12 +47,12 @@ DeflateJobHWPool::DeflateJobHWPool():log(&Poco::Logger::get("DeflateJobHWPool")) if(JOB_POOL_SIZE == index) { jobPoolReady() = true; - LOG_DEBUG(log, "QPL deflate HW codec is ready! QPL Version:{}",qpl_version); + LOG_DEBUG(log, "Hardware-assisted DEFLATE codec is ready! QPL Version:{}",qpl_version); } else { jobPoolReady() = false; - LOG_WARNING(log, "QPL deflate HW codec is not ready! Please check if IAA hardware support. Will fallback to software codec here. QPL Version:{}.",qpl_version); + LOG_WARNING(log, "Initialization of hardware-assisted DEFLATE codec failed, falling back to software DEFLATE codec. Please check if Intel In-Memory Analytics Accelerator (IAA) is properly set up. QPL Version:{}.",qpl_version); } } @@ -213,10 +213,6 @@ void HardwareCodecDeflate::flushAsynchronousDecompressRequests() } } -SoftwareCodecDeflate::SoftwareCodecDeflate():sw_job(nullptr) -{ -} - SoftwareCodecDeflate::~SoftwareCodecDeflate() { if (nullptr != sw_job) @@ -233,8 +229,8 @@ qpl_job * SoftwareCodecDeflate::getJobCodecPtr() sw_job = reinterpret_cast((std::make_unique(size)).get()); // Job initialization if (auto status = qpl_init_job(qpl_path_software, sw_job); status != QPL_STS_OK) - throw Exception( - "SoftwareCodecDeflate::getJobCodecPtr -> qpl_init_job fail:" + std::to_string(status), ErrorCodes::CANNOT_COMPRESS); + throw Exception(ErrorCodes::CANNOT_COMPRESS, + "Initialization of DEFLATE software fallback codec failed. (details: qpl_init_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); } return sw_job; } @@ -252,8 +248,8 @@ uint32_t SoftwareCodecDeflate::doCompressData(const char * source, uint32_t sour job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; if (auto status = qpl_execute_job(job_ptr); status != QPL_STS_OK) - throw Exception( - "SoftwareCodecDeflate::doCompressData -> qpl_execute_job fail:" + std::to_string(status), ErrorCodes::CANNOT_COMPRESS); + throw Exception(ErrorCodes::CANNOT_COMPRESS, + "Execution of DEFLATE software fallback codec failed. (details: qpl_execute_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); return job_ptr->total_out; } @@ -271,8 +267,8 @@ void SoftwareCodecDeflate::doDecompressData(const char * source, uint32_t source job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; if (auto status = qpl_execute_job(job_ptr); status != QPL_STS_OK) - throw Exception( - "SoftwareCodecDeflate::doDecompressData -> qpl_execute_job fail:" + std::to_string(status), ErrorCodes::CANNOT_DECOMPRESS); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, + "Execution of DEFLATE software fallback codec failed. (details: qpl_execute_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); } //CompressionCodecDeflate diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index c8631d843e1..c1611fc95fa 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -105,13 +105,12 @@ private: class SoftwareCodecDeflate { public: - SoftwareCodecDeflate(); ~SoftwareCodecDeflate(); uint32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size); void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); private: - qpl_job * sw_job; + qpl_job * sw_job = nullptr; qpl_job * getJobCodecPtr(); }; @@ -127,6 +126,9 @@ public: void flushAsynchronousDecompressRequests(); private: + /// Asynchronous job map for decompression: job ID - job object. + /// For each submission, push job ID && job object into this map; + /// For flush, pop out job ID && job object from this map. Use job ID to release job lock and use job object to check job status till complete. std::map decomp_async_job_map; Poco::Logger * log; }; From 0fdce07cc66a05d49a819709e962ead53f86c377 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sat, 9 Jul 2022 11:12:41 -0400 Subject: [PATCH 117/659] add dedicated value for HW codec fail --- src/Compression/CompressionCodecDeflate.cpp | 59 ++++++++++----------- src/Compression/CompressionCodecDeflate.h | 10 ++-- 2 files changed, 35 insertions(+), 34 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 36fb3db9aa3..1583cce27ec 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -73,7 +73,6 @@ DeflateJobHWPool::~DeflateJobHWPool() //HardwareCodecDeflate HardwareCodecDeflate::HardwareCodecDeflate(): - hw_enabled(DeflateJobHWPool::instance().jobPoolReady()), log(&Poco::Logger::get("HardwareCodecDeflate")) { } @@ -90,15 +89,16 @@ HardwareCodecDeflate::~HardwareCodecDeflate() decomp_async_job_map.clear(); } } -uint32_t HardwareCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const + +int32_t HardwareCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const { uint32_t job_id = 0; qpl_job* job_ptr = nullptr; uint32_t compressed_size = 0; if (!(job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id))) { - LOG_WARNING(log, "HardwareCodecDeflate::doCompressData acquireJob fail!"); - return 0; + LOG_WARNING(log, "DEFLATE HW codec failed, falling back to SW codec.(Details: doCompressData->acquireJob fail, probably job pool exhausted)"); + return RET_ERROR; } job_ptr->op = qpl_op_compress; @@ -112,20 +112,19 @@ uint32_t HardwareCodecDeflate::doCompressData(const char * source, uint32_t sour if (auto status = qpl_execute_job(job_ptr); status == QPL_STS_OK) compressed_size = job_ptr->total_out; else - LOG_WARNING(log, "HardwareCodecDeflate::doCompressData fail ->status: '{}' ", static_cast(status)); - + LOG_WARNING(log, "DEFLATE HW codec failed, falling back to SW codec.(Details: doCompressData->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); DeflateJobHWPool::instance().releaseJob(job_id); return compressed_size; } -uint32_t HardwareCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const +int32_t HardwareCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const { uint32_t job_id = 0; qpl_job* job_ptr = nullptr; if (!(job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id))) { - LOG_WARNING(log, "HardwareCodecDeflate::doDecompressData acquireJob fail!"); - return 0; + LOG_WARNING(log, "DEFLATE HW codec failed, falling back to SW codec.(Details: doDecompressData->acquireJob fail, probably job pool exhausted)"); + return RET_ERROR; } // Performing a decompression operation job_ptr->op = qpl_op_decompress; @@ -142,20 +141,20 @@ uint32_t HardwareCodecDeflate::doDecompressData(const char * source, uint32_t so } else { - LOG_WARNING(log, "HardwareCodecDeflate::doDecompressData fail ->status: '{}' ", static_cast(status)); + LOG_WARNING(log, "DEFLATE HW codec failed, falling back to SW codec.(Details: doDecompressData->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); DeflateJobHWPool::instance().releaseJob(job_id); - return 0; + return RET_ERROR; } } -uint32_t HardwareCodecDeflate::doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) +int32_t HardwareCodecDeflate::doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) { uint32_t job_id = 0; qpl_job * job_ptr = nullptr; if (!(job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id))) { - LOG_WARNING(log, "HardwareCodecDeflate::doDecompressDataReq acquireJob fail!"); - return 0; + LOG_WARNING(log, "DEFLATE HW codec failed, falling back to SW codec.(Details: doDecompressDataReq->acquireJob fail, probably job pool exhausted)"); + return RET_ERROR; } // Performing a decompression operation @@ -168,14 +167,14 @@ uint32_t HardwareCodecDeflate::doDecompressDataReq(const char * source, uint32_t if (auto status = qpl_submit_job(job_ptr); status == QPL_STS_OK) { - decomp_async_job_map.insert(std::make_pair(job_id, job_ptr)); + decomp_async_job_map.insert({job_id, job_ptr}); return job_id; } else { DeflateJobHWPool::instance().releaseJob(job_id); - LOG_WARNING(log, "HardwareCodecDeflate::doDecompressDataReq fail ->status: '{}' ", static_cast(status)); - return 0; + LOG_WARNING(log, "DEFLATE HW codec failed, falling back to SW codec.(Details: doDecompressDataReq->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + return RET_ERROR; } } @@ -230,7 +229,7 @@ qpl_job * SoftwareCodecDeflate::getJobCodecPtr() // Job initialization if (auto status = qpl_init_job(qpl_path_software, sw_job); status != QPL_STS_OK) throw Exception(ErrorCodes::CANNOT_COMPRESS, - "Initialization of DEFLATE software fallback codec failed. (details: qpl_init_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + "Initialization of DEFLATE software fallback codec failed. (Details: qpl_init_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); } return sw_job; } @@ -249,7 +248,7 @@ uint32_t SoftwareCodecDeflate::doCompressData(const char * source, uint32_t sour if (auto status = qpl_execute_job(job_ptr); status != QPL_STS_OK) throw Exception(ErrorCodes::CANNOT_COMPRESS, - "Execution of DEFLATE software fallback codec failed. (details: qpl_execute_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + "Execution of DEFLATE software fallback codec failed. (Details: qpl_execute_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); return job_ptr->total_out; } @@ -268,7 +267,7 @@ void SoftwareCodecDeflate::doDecompressData(const char * source, uint32_t source if (auto status = qpl_execute_job(job_ptr); status != QPL_STS_OK) throw Exception(ErrorCodes::CANNOT_DECOMPRESS, - "Execution of DEFLATE software fallback codec failed. (details: qpl_execute_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + "Execution of DEFLATE software fallback codec failed. (Details: qpl_execute_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); } //CompressionCodecDeflate @@ -297,10 +296,10 @@ uint32_t CompressionCodecDeflate::getMaxCompressedDataSize(uint32_t uncompressed uint32_t CompressionCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest) const { - uint32_t res = 0; - if (hw_codec->hw_enabled) + int32_t res = HardwareCodecDeflate::RET_ERROR; + if (DeflateJobHWPool::instance().jobPoolReady()) res = hw_codec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); - if (0 == res) + if (res == HardwareCodecDeflate::RET_ERROR) res = sw_codec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); return res; } @@ -311,19 +310,19 @@ void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t sou { case CodecMode::Synchronous: { - uint32_t res = 0; - if (hw_codec->hw_enabled) + int32_t res = HardwareCodecDeflate::RET_ERROR; + if (DeflateJobHWPool::instance().jobPoolReady()) res = hw_codec->doDecompressData(source, source_size, dest, uncompressed_size); - if (0 == res) + if (res == HardwareCodecDeflate::RET_ERROR) sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); break; } case CodecMode::Asynchronous: { - uint32_t res = 0; - if (hw_codec->hw_enabled) + int32_t res = HardwareCodecDeflate::RET_ERROR; + if (DeflateJobHWPool::instance().jobPoolReady()) res = hw_codec->doDecompressDataReq(source, source_size, dest, uncompressed_size); - if (0 == res) + if (res == HardwareCodecDeflate::RET_ERROR) sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); break; } @@ -335,7 +334,7 @@ void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t sou void CompressionCodecDeflate::flushAsynchronousDecompressRequests() { - if (hw_codec->hw_enabled) + if (DeflateJobHWPool::instance().jobPoolReady()) hw_codec->flushAsynchronousDecompressRequests(); setDecompressMode(CodecMode::Synchronous); } diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index c1611fc95fa..1b07d02d569 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -117,12 +117,14 @@ private: class HardwareCodecDeflate { public: - bool hw_enabled; + /// RET_ERROR stands for hardware codec fail,need fallback to software codec. + static constexpr int32_t RET_ERROR = -1; + HardwareCodecDeflate(); ~HardwareCodecDeflate(); - uint32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const; - uint32_t doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const; - uint32_t doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); + int32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const; + int32_t doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const; + int32_t doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); void flushAsynchronousDecompressRequests(); private: From 212f342af552bbfef41404c9f46c97e6dc5c94f4 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sat, 9 Jul 2022 12:03:25 -0400 Subject: [PATCH 118/659] change to std random --- src/Compression/CompressionCodecDeflate.cpp | 5 ++++- src/Compression/CompressionCodecDeflate.h | 16 +++++----------- 2 files changed, 9 insertions(+), 12 deletions(-) diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflate.cpp index 1583cce27ec..c79dc8c42e5 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflate.cpp @@ -25,7 +25,10 @@ DeflateJobHWPool & DeflateJobHWPool::instance() return ret; } -DeflateJobHWPool::DeflateJobHWPool():log(&Poco::Logger::get("DeflateJobHWPool")) +DeflateJobHWPool::DeflateJobHWPool(): + log(&Poco::Logger::get("DeflateJobHWPool")), + random_engine(std::random_device()()), + distribution(0, JOB_POOL_SIZE-1) { uint32_t size = 0; uint32_t index = 0; diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index 1b07d02d569..9e86286b396 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -2,6 +2,7 @@ #include #include +#include namespace Poco { @@ -31,10 +32,10 @@ public: if (jobPoolReady()) { uint32_t retry = 0; - auto index = random(JOB_POOL_SIZE); + auto index = distribution(random_engine); while (tryLockJob(index) == false) { - index = random(JOB_POOL_SIZE); + index = distribution(random_engine); retry++; if (retry > JOB_POOL_SIZE) { @@ -65,15 +66,6 @@ public: } private: - static size_t random(uint32_t pool_size) - { - size_t tsc = 0; - unsigned lo, hi; - __asm__ volatile("rdtsc" : "=a"(lo), "=d"(hi) : :); - tsc = (((static_cast(hi)) << 32) | (static_cast(lo))); - return (static_cast((tsc * 44485709377909ULL) >> 4)) % pool_size; - } - bool tryLockJob(size_t index) { bool expected = false; @@ -100,6 +92,8 @@ private: }; std::unique_ptr hw_job_pool_buffer; Poco::Logger * log; + std::mt19937 random_engine; + std::uniform_int_distribution distribution; }; class SoftwareCodecDeflate From 9e94346e078292bfbc4078e1c34befc22a3608bb Mon Sep 17 00:00:00 2001 From: Vladimir Chebotaryov <108669454+quickhouse@users.noreply.github.com> Date: Sat, 9 Jul 2022 20:51:05 +0300 Subject: [PATCH 119/659] More stable `test_s3_zero_copy_ttl`, weakened requirement to move data to S3 in 0-5 seconds. --- .../integration/test_s3_zero_copy_ttl/test.py | 32 ++++++++++++------- 1 file changed, 20 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_s3_zero_copy_ttl/test.py b/tests/integration/test_s3_zero_copy_ttl/test.py index 14b4664fcc1..9a782aacef6 100644 --- a/tests/integration/test_s3_zero_copy_ttl/test.py +++ b/tests/integration/test_s3_zero_copy_ttl/test.py @@ -68,19 +68,27 @@ def test_ttl_move_and_s3(started_cluster): assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" - time.sleep(5) + for attempt in reversed(range(5)): + time.sleep(5) - print( - node1.query( - "SELECT * FROM system.parts WHERE table = 's3_test_with_ttl' FORMAT Vertical" + print( + node1.query( + "SELECT * FROM system.parts WHERE table = 's3_test_with_ttl' FORMAT Vertical" + ) ) - ) - minio = cluster.minio_client - objects = minio.list_objects(cluster.minio_bucket, "data/", recursive=True) - counter = 0 - for obj in objects: - print("Objectname:", obj.object_name, "metadata:", obj.metadata) - counter += 1 - print("Total objects", counter) + minio = cluster.minio_client + objects = minio.list_objects(cluster.minio_bucket, "data/", recursive=True) + counter = 0 + for obj in objects: + print(f"Objectname: {obj.object_name}, metadata: {obj.metadata}") + counter += 1 + + print(f"Total objects: {counter}") + + if counter == 300: + break + + print(f"Attempts remaining: {attempt}") + assert counter == 300 From 7cb5b35bef9327ba4f4f36203e8cf582d9df7c54 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sat, 9 Jul 2022 13:54:23 -0400 Subject: [PATCH 120/659] remove qpl from fasttest --- docker/test/fasttest/run.sh | 1 - src/Compression/CompressionCodecDeflate.h | 2 ++ src/Compression/ICompressionCodec.h | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 4daf3680f4e..2bbdd978e5e 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -133,7 +133,6 @@ function clone_submodules contrib/NuRaft contrib/jemalloc contrib/replxx - contrib/qpl contrib/wyhash contrib/hashidsxx ) diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflate.h index 9e86286b396..0bf5c46d086 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflate.h @@ -119,6 +119,7 @@ public: int32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const; int32_t doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const; int32_t doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); + /// Flush result for previous asynchronous decompression requests.Must be used following with several calls of doDecompressDataReq. void flushAsynchronousDecompressRequests(); private: @@ -148,6 +149,7 @@ protected: uint32_t doCompressData(const char * source, uint32_t source_size, char * dest) const override; void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; + ///Flush result for previous asynchronous decompression requests on asynchronous mode. void flushAsynchronousDecompressRequests() override; private: diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index b6ef47e6b45..0391fa508a6 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -77,7 +77,7 @@ public: /// Flush result for previous asynchronous decompression requests. /// This function must be called following several requests offload to HW. /// To make sure asynchronous results have been flushed into target buffer completely. - /// Meanwhile, source and target buffer for decompression should not be overwritten until this function execute completely. + /// Meanwhile, source and target buffer for decompression can not be overwritten until this function execute completely. /// Otherwise it would conflict with HW offloading and cause exception. /// For QPL deflate, it support the maximum number of requests equal to DeflateJobHWPool::jobPoolSize virtual void flushAsynchronousDecompressRequests(){} From 24565cb809a07e876157567085943aaadfcb7cc1 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sat, 9 Jul 2022 14:04:17 -0400 Subject: [PATCH 121/659] remove license information --- docs/ru/development/contrib.md | 1 - docs/zh/development/contrib.md | 1 - 2 files changed, 2 deletions(-) diff --git a/docs/ru/development/contrib.md b/docs/ru/development/contrib.md index b1e715e1cd6..0f4d22e90ce 100644 --- a/docs/ru/development/contrib.md +++ b/docs/ru/development/contrib.md @@ -86,7 +86,6 @@ sidebar_label: "Используемые сторонние библиотеки | xz | [Public Domain](https://github.com/xz-mirror/xz/blob/869b9d1b4edd6df07f819d360d306251f8147353/COPYING) | | zlib-ng | [zLib](https://github.com/ClickHouse-Extras/zlib-ng/blob/6a5e93b9007782115f7f7e5235dedc81c4f1facb/LICENSE.md) | | zstd | [BSD](https://github.com/facebook/zstd/blob/a488ba114ec17ea1054b9057c26a046fc122b3b6/LICENSE) | -| qpl | [MIT](https://github.com/intel/qpl/blob/cdc8442f7a5e7a6ff6eea39c69665e0c5034d85d/LICENSE) | Список всех сторонних библиотек можно получить с помощью запроса: diff --git a/docs/zh/development/contrib.md b/docs/zh/development/contrib.md index 0f6f17de29e..8e8efc3c04e 100644 --- a/docs/zh/development/contrib.md +++ b/docs/zh/development/contrib.md @@ -31,4 +31,3 @@ | UnixODBC | [LGPL v2.1](https://github.com/ClickHouse-Extras/UnixODBC/tree/b0ad30f7f6289c12b76f04bfb9d466374bb32168) | | zlib-ng | [Zlib许可证](https://github.com/ClickHouse-Extras/zlib-ng/blob/develop/LICENSE.md) | | zstd | [BSD3-条款许可](https://github.com/facebook/zstd/blob/dev/LICENSE) | -| deflate | [MIT](https://github.com/intel/qpl/blob/develop/LICENSE) \ No newline at end of file From 825b7511e478e95c2cc5197113bdb97ff29d4095 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sat, 9 Jul 2022 14:13:12 -0400 Subject: [PATCH 122/659] improve cosmetics --- contrib/qpl-cmake/CMakeLists.txt | 1 + programs/keeper/CMakeLists.txt | 1 + src/Compression/CompressionFactory.cpp | 6 +++--- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 7b504c235c8..dc90f07a9bc 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -116,6 +116,7 @@ target_compile_options(isal_asm PUBLIC "-I${QPL_SRC_DIR}/isal/include/" PUBLIC "-DQPL_LIB") # AS_FEATURE_LEVEL=10 means "Check SIMD capabilities of the target system at runtime and use up to AVX512 if available". +# AS_FEATURE_LEVEL=5 means "Check SIMD capabilities of the target system at runtime and use up to AVX2 if available". # HAVE_KNOWS_AVX512 means rely on AVX512 being available on the target system. if (ENABLE_AVX512) target_compile_options(isal_asm PUBLIC "-DHAVE_AS_KNOWS_AVX512" "-DAS_FEATURE_LEVEL=10") diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 4da7031ec19..c77b335b615 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -103,6 +103,7 @@ if (BUILD_STANDALONE_KEEPER) # Remove some redundant dependencies target_compile_definitions (clickhouse-keeper PRIVATE -DKEEPER_STANDALONE_BUILD) + target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_SOURCE_DIR}/../../src") # uses includes from src directory target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/../../src/Core/include") # uses some includes from core target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/../../src") # uses some includes from common diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 8667f8e515a..8d85117f20a 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -189,15 +189,15 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecLZ4HC(*this); registerCodecMultiple(*this); #ifndef KEEPER_STANDALONE_BUILD -#ifdef ENABLE_QPL_COMPRESSION - registerCodecDeflate(*this); -#endif registerCodecDelta(*this); registerCodecT64(*this); registerCodecDoubleDelta(*this); registerCodecGorilla(*this); registerCodecEncrypted(*this); registerCodecFPC(*this); + #ifdef ENABLE_QPL_COMPRESSION + registerCodecDeflate(*this); + #endif #endif default_codec = get("LZ4", {}); From 816e974ca2d5e4a8fb227f275ffc1fdbbb5333e1 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sat, 9 Jul 2022 14:42:01 -0400 Subject: [PATCH 123/659] rename deflate to deflate_qpl --- .../settings.md | 2 +- programs/compressor/Compressor.cpp | 10 +- src/CMakeLists.txt | 2 +- ...ate.cpp => CompressionCodecDeflateQpl.cpp} | 132 +++++++++--------- ...Deflate.h => CompressionCodecDeflateQpl.h} | 28 ++-- src/Compression/CompressionFactory.cpp | 4 +- src/Compression/CompressionInfo.h | 2 +- src/Compression/ICompressionCodec.h | 2 +- 8 files changed, 91 insertions(+), 91 deletions(-) rename src/Compression/{CompressionCodecDeflate.cpp => CompressionCodecDeflateQpl.cpp} (58%) rename src/Compression/{CompressionCodecDeflate.h => CompressionCodecDeflateQpl.h} (86%) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index cb44fa6add5..3a957e1e952 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -45,7 +45,7 @@ Configuration template: - `min_part_size` – The minimum size of a data part. - `min_part_size_ratio` – The ratio of the data part size to the table size. -- `method` – Compression method. Acceptable values: `lz4`, `lz4hc`, `zstd`,`deflate`. +- `method` – Compression method. Acceptable values: `lz4`, `lz4hc`, `zstd`,`deflate_qpl`. - `level` – Compression level. See [Codecs](../../sql-reference/statements/create/table.md#create-query-general-purpose-codecs). You can configure multiple `` sections. diff --git a/programs/compressor/Compressor.cpp b/programs/compressor/Compressor.cpp index 34d1641b892..1a9d72e8972 100644 --- a/programs/compressor/Compressor.cpp +++ b/programs/compressor/Compressor.cpp @@ -79,7 +79,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) ("block-size,b", po::value()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size") ("hc", "use LZ4HC instead of LZ4") ("zstd", "use ZSTD instead of LZ4") - ("deflate", "use deflate instead of LZ4") + ("deflate_qpl", "use deflate_qpl instead of LZ4") ("codec", po::value>()->multitoken(), "use codecs combination instead of LZ4") ("level", po::value(), "compression level for codecs specified via flags") ("none", "use no compression instead of LZ4") @@ -104,7 +104,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) bool decompress = options.count("decompress"); bool use_lz4hc = options.count("hc"); bool use_zstd = options.count("zstd"); - bool use_deflate = options.count("deflate"); + bool use_deflate_qpl = options.count("deflate_qpl"); bool stat_mode = options.count("stat"); bool use_none = options.count("none"); unsigned block_size = options["block-size"].as(); @@ -112,7 +112,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) if (options.count("codec")) codecs = options["codec"].as>(); - if ((use_deflate || use_lz4hc || use_zstd || use_none) && !codecs.empty()) + if ((use_deflate_qpl || use_lz4hc || use_zstd || use_none) && !codecs.empty()) throw Exception("Wrong options, codec flags like --zstd and --codec options are mutually exclusive", ErrorCodes::BAD_ARGUMENTS); if (!codecs.empty() && options.count("level")) @@ -124,8 +124,8 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) method_family = "LZ4HC"; else if (use_zstd) method_family = "ZSTD"; - else if (use_deflate) - method_family = "DEFLATE"; + else if (use_deflate_qpl) + method_family = "DEFLATE_QPL"; else if (use_none) method_family = "NONE"; diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index e120433f037..a6f199e1210 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -344,7 +344,7 @@ set_source_files_properties( if (ENABLE_QPL) set_source_files_properties( - Compression/CompressionCodecDeflate.cpp + Compression/CompressionCodecDeflateQpl.cpp PROPERTIES COMPILE_FLAGS "-mwaitpkg") endif () diff --git a/src/Compression/CompressionCodecDeflate.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp similarity index 58% rename from src/Compression/CompressionCodecDeflate.cpp rename to src/Compression/CompressionCodecDeflateQpl.cpp index c79dc8c42e5..5fd92b144e3 100644 --- a/src/Compression/CompressionCodecDeflate.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -1,7 +1,7 @@ #ifdef ENABLE_QPL_COMPRESSION #include #include -#include +#include #include #include #include @@ -15,18 +15,18 @@ namespace ErrorCodes extern const int CANNOT_COMPRESS; extern const int CANNOT_DECOMPRESS; } -qpl_job * DeflateJobHWPool::hw_job_pool[JOB_POOL_SIZE]; -std::atomic_bool DeflateJobHWPool::hw_job_locks[JOB_POOL_SIZE]; -bool DeflateJobHWPool::job_pool_ready; +qpl_job * DeflateQplJobHWPool::hw_job_pool[JOB_POOL_SIZE]; +std::atomic_bool DeflateQplJobHWPool::hw_job_locks[JOB_POOL_SIZE]; +bool DeflateQplJobHWPool::job_pool_ready; -DeflateJobHWPool & DeflateJobHWPool::instance() +DeflateQplJobHWPool & DeflateQplJobHWPool::instance() { - static DeflateJobHWPool ret; + static DeflateQplJobHWPool ret; return ret; } -DeflateJobHWPool::DeflateJobHWPool(): - log(&Poco::Logger::get("DeflateJobHWPool")), +DeflateQplJobHWPool::DeflateQplJobHWPool(): + log(&Poco::Logger::get("DeflateQplJobHWPool")), random_engine(std::random_device()()), distribution(0, JOB_POOL_SIZE-1) { @@ -50,16 +50,16 @@ DeflateJobHWPool::DeflateJobHWPool(): if(JOB_POOL_SIZE == index) { jobPoolReady() = true; - LOG_DEBUG(log, "Hardware-assisted DEFLATE codec is ready! QPL Version:{}",qpl_version); + LOG_DEBUG(log, "Hardware-assisted DeflateQpl codec is ready! QPL Version:{}",qpl_version); } else { jobPoolReady() = false; - LOG_WARNING(log, "Initialization of hardware-assisted DEFLATE codec failed, falling back to software DEFLATE codec. Please check if Intel In-Memory Analytics Accelerator (IAA) is properly set up. QPL Version:{}.",qpl_version); + LOG_WARNING(log, "Initialization of hardware-assisted DeflateQpl codec failed, falling back to software DeflateQpl codec. Please check if Intel In-Memory Analytics Accelerator (IAA) is properly set up. QPL Version:{}.",qpl_version); } } -DeflateJobHWPool::~DeflateJobHWPool() +DeflateQplJobHWPool::~DeflateQplJobHWPool() { for (uint32_t i = 0; i < JOB_POOL_SIZE; ++i) { @@ -74,33 +74,33 @@ DeflateJobHWPool::~DeflateJobHWPool() jobPoolReady() = false; } -//HardwareCodecDeflate -HardwareCodecDeflate::HardwareCodecDeflate(): - log(&Poco::Logger::get("HardwareCodecDeflate")) +//HardwareCodecDeflateQpl +HardwareCodecDeflateQpl::HardwareCodecDeflateQpl(): + log(&Poco::Logger::get("HardwareCodecDeflateQpl")) { } -HardwareCodecDeflate::~HardwareCodecDeflate() +HardwareCodecDeflateQpl::~HardwareCodecDeflateQpl() { if (!decomp_async_job_map.empty()) { - LOG_WARNING(log, "Find un-released job when HardwareCodecDeflate destroy"); + LOG_WARNING(log, "Find un-released job when HardwareCodecDeflateQpl destroy"); for (auto it : decomp_async_job_map) { - DeflateJobHWPool::instance().releaseJob(it.first); + DeflateQplJobHWPool::instance().releaseJob(it.first); } decomp_async_job_map.clear(); } } -int32_t HardwareCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const +int32_t HardwareCodecDeflateQpl::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const { uint32_t job_id = 0; qpl_job* job_ptr = nullptr; uint32_t compressed_size = 0; - if (!(job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id))) + if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(&job_id))) { - LOG_WARNING(log, "DEFLATE HW codec failed, falling back to SW codec.(Details: doCompressData->acquireJob fail, probably job pool exhausted)"); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doCompressData->acquireJob fail, probably job pool exhausted)"); return RET_ERROR; } @@ -115,18 +115,18 @@ int32_t HardwareCodecDeflate::doCompressData(const char * source, uint32_t sourc if (auto status = qpl_execute_job(job_ptr); status == QPL_STS_OK) compressed_size = job_ptr->total_out; else - LOG_WARNING(log, "DEFLATE HW codec failed, falling back to SW codec.(Details: doCompressData->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); - DeflateJobHWPool::instance().releaseJob(job_id); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doCompressData->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + DeflateQplJobHWPool::instance().releaseJob(job_id); return compressed_size; } -int32_t HardwareCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const +int32_t HardwareCodecDeflateQpl::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const { uint32_t job_id = 0; qpl_job* job_ptr = nullptr; - if (!(job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id))) + if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(&job_id))) { - LOG_WARNING(log, "DEFLATE HW codec failed, falling back to SW codec.(Details: doDecompressData->acquireJob fail, probably job pool exhausted)"); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressData->acquireJob fail, probably job pool exhausted)"); return RET_ERROR; } // Performing a decompression operation @@ -139,24 +139,24 @@ int32_t HardwareCodecDeflate::doDecompressData(const char * source, uint32_t sou if (auto status = qpl_execute_job(job_ptr); status == QPL_STS_OK) { - DeflateJobHWPool::instance().releaseJob(job_id); + DeflateQplJobHWPool::instance().releaseJob(job_id); return job_ptr->total_out; } else { - LOG_WARNING(log, "DEFLATE HW codec failed, falling back to SW codec.(Details: doDecompressData->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); - DeflateJobHWPool::instance().releaseJob(job_id); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressData->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + DeflateQplJobHWPool::instance().releaseJob(job_id); return RET_ERROR; } } -int32_t HardwareCodecDeflate::doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) +int32_t HardwareCodecDeflateQpl::doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) { uint32_t job_id = 0; qpl_job * job_ptr = nullptr; - if (!(job_ptr = DeflateJobHWPool::instance().acquireJob(&job_id))) + if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(&job_id))) { - LOG_WARNING(log, "DEFLATE HW codec failed, falling back to SW codec.(Details: doDecompressDataReq->acquireJob fail, probably job pool exhausted)"); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataReq->acquireJob fail, probably job pool exhausted)"); return RET_ERROR; } @@ -175,13 +175,13 @@ int32_t HardwareCodecDeflate::doDecompressDataReq(const char * source, uint32_t } else { - DeflateJobHWPool::instance().releaseJob(job_id); - LOG_WARNING(log, "DEFLATE HW codec failed, falling back to SW codec.(Details: doDecompressDataReq->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + DeflateQplJobHWPool::instance().releaseJob(job_id); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataReq->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); return RET_ERROR; } } -void HardwareCodecDeflate::flushAsynchronousDecompressRequests() +void HardwareCodecDeflateQpl::flushAsynchronousDecompressRequests() { uint32_t job_id = 0; qpl_job * job_ptr = nullptr; @@ -201,7 +201,7 @@ void HardwareCodecDeflate::flushAsynchronousDecompressRequests() } else { - DeflateJobHWPool::instance().releaseJob(job_id); + DeflateQplJobHWPool::instance().releaseJob(job_id); it = decomp_async_job_map.erase(it); n_jobs_processing--; if (n_jobs_processing <= 0) @@ -215,13 +215,13 @@ void HardwareCodecDeflate::flushAsynchronousDecompressRequests() } } -SoftwareCodecDeflate::~SoftwareCodecDeflate() +SoftwareCodecDeflateQpl::~SoftwareCodecDeflateQpl() { if (nullptr != sw_job) qpl_fini_job(sw_job); } -qpl_job * SoftwareCodecDeflate::getJobCodecPtr() +qpl_job * SoftwareCodecDeflateQpl::getJobCodecPtr() { if (!sw_job) { @@ -232,12 +232,12 @@ qpl_job * SoftwareCodecDeflate::getJobCodecPtr() // Job initialization if (auto status = qpl_init_job(qpl_path_software, sw_job); status != QPL_STS_OK) throw Exception(ErrorCodes::CANNOT_COMPRESS, - "Initialization of DEFLATE software fallback codec failed. (Details: qpl_init_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + "Initialization of DeflateQpl software fallback codec failed. (Details: qpl_init_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); } return sw_job; } -uint32_t SoftwareCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) +uint32_t SoftwareCodecDeflateQpl::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) { qpl_job * job_ptr = getJobCodecPtr(); // Performing a compression operation @@ -251,12 +251,12 @@ uint32_t SoftwareCodecDeflate::doCompressData(const char * source, uint32_t sour if (auto status = qpl_execute_job(job_ptr); status != QPL_STS_OK) throw Exception(ErrorCodes::CANNOT_COMPRESS, - "Execution of DEFLATE software fallback codec failed. (Details: qpl_execute_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + "Execution of DeflateQpl software fallback codec failed. (Details: qpl_execute_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); return job_ptr->total_out; } -void SoftwareCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) +void SoftwareCodecDeflateQpl::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) { qpl_job * job_ptr = getJobCodecPtr(); @@ -270,62 +270,62 @@ void SoftwareCodecDeflate::doDecompressData(const char * source, uint32_t source if (auto status = qpl_execute_job(job_ptr); status != QPL_STS_OK) throw Exception(ErrorCodes::CANNOT_DECOMPRESS, - "Execution of DEFLATE software fallback codec failed. (Details: qpl_execute_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + "Execution of DeflateQpl software fallback codec failed. (Details: qpl_execute_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); } -//CompressionCodecDeflate -CompressionCodecDeflate::CompressionCodecDeflate(): - hw_codec(std::make_unique()), - sw_codec(std::make_unique()) +//CompressionCodecDeflateQpl +CompressionCodecDeflateQpl::CompressionCodecDeflateQpl(): + hw_codec(std::make_unique()), + sw_codec(std::make_unique()) { - setCodecDescription("DEFLATE"); + setCodecDescription("DEFLATE_QPL"); } -uint8_t CompressionCodecDeflate::getMethodByte() const +uint8_t CompressionCodecDeflateQpl::getMethodByte() const { - return static_cast(CompressionMethodByte::Deflate); + return static_cast(CompressionMethodByte::DeflateQpl); } -void CompressionCodecDeflate::updateHash(SipHash & hash) const +void CompressionCodecDeflateQpl::updateHash(SipHash & hash) const { getCodecDesc()->updateTreeHash(hash); } -uint32_t CompressionCodecDeflate::getMaxCompressedDataSize(uint32_t uncompressed_size) const +uint32_t CompressionCodecDeflateQpl::getMaxCompressedDataSize(uint32_t uncompressed_size) const { /// Aligned with ZLIB return ((uncompressed_size) + ((uncompressed_size) >> 12) + ((uncompressed_size) >> 14) + ((uncompressed_size) >> 25) + 13); } -uint32_t CompressionCodecDeflate::doCompressData(const char * source, uint32_t source_size, char * dest) const +uint32_t CompressionCodecDeflateQpl::doCompressData(const char * source, uint32_t source_size, char * dest) const { - int32_t res = HardwareCodecDeflate::RET_ERROR; - if (DeflateJobHWPool::instance().jobPoolReady()) + int32_t res = HardwareCodecDeflateQpl::RET_ERROR; + if (DeflateQplJobHWPool::instance().jobPoolReady()) res = hw_codec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); - if (res == HardwareCodecDeflate::RET_ERROR) + if (res == HardwareCodecDeflateQpl::RET_ERROR) res = sw_codec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); return res; } -void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const +void CompressionCodecDeflateQpl::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const { switch (getDecompressMode()) { case CodecMode::Synchronous: { - int32_t res = HardwareCodecDeflate::RET_ERROR; - if (DeflateJobHWPool::instance().jobPoolReady()) + int32_t res = HardwareCodecDeflateQpl::RET_ERROR; + if (DeflateQplJobHWPool::instance().jobPoolReady()) res = hw_codec->doDecompressData(source, source_size, dest, uncompressed_size); - if (res == HardwareCodecDeflate::RET_ERROR) + if (res == HardwareCodecDeflateQpl::RET_ERROR) sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); break; } case CodecMode::Asynchronous: { - int32_t res = HardwareCodecDeflate::RET_ERROR; - if (DeflateJobHWPool::instance().jobPoolReady()) + int32_t res = HardwareCodecDeflateQpl::RET_ERROR; + if (DeflateQplJobHWPool::instance().jobPoolReady()) res = hw_codec->doDecompressDataReq(source, source_size, dest, uncompressed_size); - if (res == HardwareCodecDeflate::RET_ERROR) + if (res == HardwareCodecDeflateQpl::RET_ERROR) sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); break; } @@ -335,16 +335,16 @@ void CompressionCodecDeflate::doDecompressData(const char * source, uint32_t sou } } -void CompressionCodecDeflate::flushAsynchronousDecompressRequests() +void CompressionCodecDeflateQpl::flushAsynchronousDecompressRequests() { - if (DeflateJobHWPool::instance().jobPoolReady()) + if (DeflateQplJobHWPool::instance().jobPoolReady()) hw_codec->flushAsynchronousDecompressRequests(); setDecompressMode(CodecMode::Synchronous); } -void registerCodecDeflate(CompressionCodecFactory & factory) +void registerCodecDeflateQpl(CompressionCodecFactory & factory) { factory.registerSimpleCompressionCodec( - "DEFLATE", static_cast(CompressionMethodByte::Deflate), [&]() { return std::make_shared(); }); + "DEFLATE_QPL", static_cast(CompressionMethodByte::DeflateQpl), [&]() { return std::make_shared(); }); } } #endif diff --git a/src/Compression/CompressionCodecDeflate.h b/src/Compression/CompressionCodecDeflateQpl.h similarity index 86% rename from src/Compression/CompressionCodecDeflate.h rename to src/Compression/CompressionCodecDeflateQpl.h index 0bf5c46d086..5c43d51c896 100644 --- a/src/Compression/CompressionCodecDeflate.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -12,13 +12,13 @@ class Logger; namespace DB { -/// DeflateJobHWPool is resource pool for provide the job objects which is required to save context infomation during offload asynchronous compression to IAA. -class DeflateJobHWPool +/// DeflateQplJobHWPool is resource pool for provide the job objects which is required to save context infomation during offload asynchronous compression to IAA. +class DeflateQplJobHWPool { public: - DeflateJobHWPool(); - ~DeflateJobHWPool(); - static DeflateJobHWPool & instance(); + DeflateQplJobHWPool(); + ~DeflateQplJobHWPool(); + static DeflateQplJobHWPool & instance(); static constexpr auto JOB_POOL_SIZE = 1024; static constexpr qpl_path_t PATH = qpl_path_hardware; static qpl_job * hw_job_pool[JOB_POOL_SIZE]; @@ -96,10 +96,10 @@ private: std::uniform_int_distribution distribution; }; -class SoftwareCodecDeflate +class SoftwareCodecDeflateQpl { public: - ~SoftwareCodecDeflate(); + ~SoftwareCodecDeflateQpl(); uint32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size); void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); @@ -108,14 +108,14 @@ private: qpl_job * getJobCodecPtr(); }; -class HardwareCodecDeflate +class HardwareCodecDeflateQpl { public: /// RET_ERROR stands for hardware codec fail,need fallback to software codec. static constexpr int32_t RET_ERROR = -1; - HardwareCodecDeflate(); - ~HardwareCodecDeflate(); + HardwareCodecDeflateQpl(); + ~HardwareCodecDeflateQpl(); int32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const; int32_t doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const; int32_t doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); @@ -129,10 +129,10 @@ private: std::map decomp_async_job_map; Poco::Logger * log; }; -class CompressionCodecDeflate : public ICompressionCodec +class CompressionCodecDeflateQpl : public ICompressionCodec { public: - CompressionCodecDeflate(); + CompressionCodecDeflateQpl(); uint8_t getMethodByte() const override; void updateHash(SipHash & hash) const override; @@ -154,8 +154,8 @@ protected: private: uint32_t getMaxCompressedDataSize(uint32_t uncompressed_size) const override; - std::unique_ptr hw_codec; - std::unique_ptr sw_codec; + std::unique_ptr hw_codec; + std::unique_ptr sw_codec; }; } diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 8d85117f20a..7291d42f681 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -166,7 +166,7 @@ void registerCodecLZ4(CompressionCodecFactory & factory); void registerCodecLZ4HC(CompressionCodecFactory & factory); void registerCodecZSTD(CompressionCodecFactory & factory); void registerCodecMultiple(CompressionCodecFactory & factory); -void registerCodecDeflate(CompressionCodecFactory & factory); +void registerCodecDeflateQpl(CompressionCodecFactory & factory); /// Keeper use only general-purpose codecs, so we don't need these special codecs /// in standalone build @@ -196,7 +196,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecEncrypted(*this); registerCodecFPC(*this); #ifdef ENABLE_QPL_COMPRESSION - registerCodecDeflate(*this); + registerCodecDeflateQpl(*this); #endif #endif diff --git a/src/Compression/CompressionInfo.h b/src/Compression/CompressionInfo.h index 097cf965264..51136ff825d 100644 --- a/src/Compression/CompressionInfo.h +++ b/src/Compression/CompressionInfo.h @@ -46,7 +46,7 @@ enum class CompressionMethodByte : uint8_t AES_128_GCM_SIV = 0x96, AES_256_GCM_SIV = 0x97, FPC = 0x98, - Deflate = 0x99, + DeflateQpl = 0x99, }; } diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 0391fa508a6..276d20b81db 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -79,7 +79,7 @@ public: /// To make sure asynchronous results have been flushed into target buffer completely. /// Meanwhile, source and target buffer for decompression can not be overwritten until this function execute completely. /// Otherwise it would conflict with HW offloading and cause exception. - /// For QPL deflate, it support the maximum number of requests equal to DeflateJobHWPool::jobPoolSize + /// For QPL deflate, it support the maximum number of requests equal to DeflateQplJobHWPool::jobPoolSize virtual void flushAsynchronousDecompressRequests(){} /// Number of bytes, that will be used to compress uncompressed_size bytes with current codec From eb06c6686915c93c2d4cc72b7485f9b13b0bc5fd Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sat, 9 Jul 2022 14:57:38 -0400 Subject: [PATCH 124/659] cosmetic issue --- .../CompressionCodecDeflateQpl.cpp | 47 +++++++++++++++- src/Compression/CompressionCodecDeflateQpl.h | 55 ++----------------- src/Compression/ICompressionCodec.h | 10 +--- 3 files changed, 52 insertions(+), 60 deletions(-) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 5fd92b144e3..b4f2daacb49 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -63,17 +63,58 @@ DeflateQplJobHWPool::~DeflateQplJobHWPool() { for (uint32_t i = 0; i < JOB_POOL_SIZE; ++i) { - if (hw_job_pool[i] != nullptr) + if (hw_job_pool[i]) { while (!tryLockJob(i)); qpl_fini_job(hw_job_pool[i]); + unLockJob(i); + hw_job_pool[i] = nullptr; } - hw_job_pool[i] = nullptr; - unLockJob(i); } jobPoolReady() = false; } +qpl_job * DeflateQplJobHWPool::acquireJob(uint32_t * job_id) +{ + if (jobPoolReady()) + { + uint32_t retry = 0; + auto index = distribution(random_engine); + while (tryLockJob(index) == false) + { + index = distribution(random_engine); + retry++; + if (retry > JOB_POOL_SIZE) + { + return nullptr; + } + } + *job_id = JOB_POOL_SIZE - index; + return hw_job_pool[index]; + } + else + return nullptr; +} + +qpl_job * DeflateQplJobHWPool::releaseJob(uint32_t job_id) +{ + if (jobPoolReady()) + { + uint32_t index = JOB_POOL_SIZE - job_id; + ReleaseJobObjectGuard _(index); + return hw_job_pool[index]; + } + else + return nullptr; +} + +bool DeflateQplJobHWPool::tryLockJob(size_t index) +{ + bool expected = false; + assert(index < JOB_POOL_SIZE); + return hw_job_locks[index].compare_exchange_strong(expected, true); +} + //HardwareCodecDeflateQpl HardwareCodecDeflateQpl::HardwareCodecDeflateQpl(): log(&Poco::Logger::get("HardwareCodecDeflateQpl")) diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 5c43d51c896..193454a66ab 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -27,51 +27,12 @@ public: bool & jobPoolReady() { return job_pool_ready;} - qpl_job * acquireJob(uint32_t * job_id) - { - if (jobPoolReady()) - { - uint32_t retry = 0; - auto index = distribution(random_engine); - while (tryLockJob(index) == false) - { - index = distribution(random_engine); - retry++; - if (retry > JOB_POOL_SIZE) - { - return nullptr; - } - } - *job_id = JOB_POOL_SIZE - index; - return hw_job_pool[index]; - } - else - { - return nullptr; - } - } + qpl_job * acquireJob(uint32_t * job_id); - qpl_job * releaseJob(uint32_t job_id) - { - if (jobPoolReady()) - { - uint32_t index = JOB_POOL_SIZE - job_id; - ReleaseJobObjectGuard _(index); - return hw_job_pool[index]; - } - else - { - return nullptr; - } - } + qpl_job * releaseJob(uint32_t job_id); private: - bool tryLockJob(size_t index) - { - bool expected = false; - assert(index < JOB_POOL_SIZE); - return hw_job_locks[index].compare_exchange_strong(expected, true); - } + bool tryLockJob(size_t index); void unLockJob(uint32_t index) { hw_job_locks[index].store(false); } @@ -81,15 +42,11 @@ private: ReleaseJobObjectGuard() = delete; public: - ReleaseJobObjectGuard(const uint32_t index_) : index(index_) - { - } + ReleaseJobObjectGuard(const uint32_t index_) : index(index_){} - ~ReleaseJobObjectGuard() - { - hw_job_locks[index].store(false); - } + ~ReleaseJobObjectGuard(){ hw_job_locks[index].store(false); } }; + std::unique_ptr hw_job_pool_buffer; Poco::Logger * log; std::mt19937 random_engine; diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 276d20b81db..51d08ae8f33 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -63,16 +63,10 @@ public: }; /// Get current decompression mode - CodecMode getDecompressMode() const - { - return decompressMode; - } + CodecMode getDecompressMode() const{ return decompressMode; } /// if set mode to CodecMode::Asynchronous, must be followed with flushAsynchronousDecompressRequests - void setDecompressMode(CodecMode mode) - { - decompressMode = mode; - } + void setDecompressMode(CodecMode mode){ decompressMode = mode; } /// Flush result for previous asynchronous decompression requests. /// This function must be called following several requests offload to HW. From 044c14745e26db6ab83a17d6c35e126fe1ffebb7 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sat, 9 Jul 2022 17:17:18 -0400 Subject: [PATCH 125/659] make job pool name comprehensible --- .../CompressionCodecDeflateQpl.cpp | 69 +++++++++---------- src/Compression/CompressionCodecDeflateQpl.h | 12 ++-- 2 files changed, 40 insertions(+), 41 deletions(-) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index b4f2daacb49..ce13b886fd4 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -15,8 +15,8 @@ namespace ErrorCodes extern const int CANNOT_COMPRESS; extern const int CANNOT_DECOMPRESS; } -qpl_job * DeflateQplJobHWPool::hw_job_pool[JOB_POOL_SIZE]; -std::atomic_bool DeflateQplJobHWPool::hw_job_locks[JOB_POOL_SIZE]; +qpl_job * DeflateQplJobHWPool::hw_job_ptr_pool[JOB_NUMBER]; +std::atomic_bool DeflateQplJobHWPool::hw_job_ptr_locks[JOB_NUMBER]; bool DeflateQplJobHWPool::job_pool_ready; DeflateQplJobHWPool & DeflateQplJobHWPool::instance() @@ -28,47 +28,46 @@ DeflateQplJobHWPool & DeflateQplJobHWPool::instance() DeflateQplJobHWPool::DeflateQplJobHWPool(): log(&Poco::Logger::get("DeflateQplJobHWPool")), random_engine(std::random_device()()), - distribution(0, JOB_POOL_SIZE-1) + distribution(0, JOB_NUMBER-1) { - uint32_t size = 0; + uint32_t job_size = 0; uint32_t index = 0; - /// get total size required for saving qpl job context - qpl_get_job_size(PATH, &size); - /// allocate buffer for storing all job objects - hw_job_pool_buffer = std::make_unique(size * JOB_POOL_SIZE); - memset(hw_job_pool, 0, JOB_POOL_SIZE*sizeof(qpl_job *)); - for (index = 0; index < JOB_POOL_SIZE; ++index) + const char * qpl_version = qpl_get_library_version(); + + /// Get size required for saving a single qpl job object + qpl_get_job_size(PATH, &job_size); + /// Allocate entire buffer for storing all job objects + hw_job_buffer = std::make_unique(job_size * JOB_NUMBER); + /// Initialize pool for storing all job object pointers + memset(hw_job_ptr_pool, 0, JOB_NUMBER*sizeof(qpl_job *)); + /// Reallocate buffer for each job object through shifting address offset + for (index = 0; index < JOB_NUMBER; ++index) { - qpl_job * qpl_job_ptr = reinterpret_cast(hw_job_pool_buffer.get() + index*JOB_POOL_SIZE); + qpl_job * qpl_job_ptr = reinterpret_cast(hw_job_buffer.get() + index*job_size); if ((!qpl_job_ptr) || (qpl_init_job(PATH, qpl_job_ptr) != QPL_STS_OK)) - break; - hw_job_pool[index] = qpl_job_ptr; + { + jobPoolReady() = false; + LOG_WARNING(log, "Initialization of hardware-assisted DeflateQpl codec failed, falling back to software DeflateQpl codec. Please check if Intel In-Memory Analytics Accelerator (IAA) is properly set up. QPL Version:{}.",qpl_version); + return; + } + hw_job_ptr_pool[index] = qpl_job_ptr; unLockJob(index); } - const char * qpl_version = qpl_get_library_version(); - if(JOB_POOL_SIZE == index) - { - jobPoolReady() = true; - LOG_DEBUG(log, "Hardware-assisted DeflateQpl codec is ready! QPL Version:{}",qpl_version); - } - else - { - jobPoolReady() = false; - LOG_WARNING(log, "Initialization of hardware-assisted DeflateQpl codec failed, falling back to software DeflateQpl codec. Please check if Intel In-Memory Analytics Accelerator (IAA) is properly set up. QPL Version:{}.",qpl_version); - } + jobPoolReady() = true; + LOG_DEBUG(log, "Hardware-assisted DeflateQpl codec is ready! QPL Version:{}",qpl_version); } DeflateQplJobHWPool::~DeflateQplJobHWPool() { - for (uint32_t i = 0; i < JOB_POOL_SIZE; ++i) + for (uint32_t i = 0; i < JOB_NUMBER; ++i) { - if (hw_job_pool[i]) + if (hw_job_ptr_pool[i]) { while (!tryLockJob(i)); - qpl_fini_job(hw_job_pool[i]); + qpl_fini_job(hw_job_ptr_pool[i]); unLockJob(i); - hw_job_pool[i] = nullptr; + hw_job_ptr_pool[i] = nullptr; } } jobPoolReady() = false; @@ -84,13 +83,13 @@ qpl_job * DeflateQplJobHWPool::acquireJob(uint32_t * job_id) { index = distribution(random_engine); retry++; - if (retry > JOB_POOL_SIZE) + if (retry > JOB_NUMBER) { return nullptr; } } - *job_id = JOB_POOL_SIZE - index; - return hw_job_pool[index]; + *job_id = JOB_NUMBER - index; + return hw_job_ptr_pool[index]; } else return nullptr; @@ -100,9 +99,9 @@ qpl_job * DeflateQplJobHWPool::releaseJob(uint32_t job_id) { if (jobPoolReady()) { - uint32_t index = JOB_POOL_SIZE - job_id; + uint32_t index = JOB_NUMBER - job_id; ReleaseJobObjectGuard _(index); - return hw_job_pool[index]; + return hw_job_ptr_pool[index]; } else return nullptr; @@ -111,8 +110,8 @@ qpl_job * DeflateQplJobHWPool::releaseJob(uint32_t job_id) bool DeflateQplJobHWPool::tryLockJob(size_t index) { bool expected = false; - assert(index < JOB_POOL_SIZE); - return hw_job_locks[index].compare_exchange_strong(expected, true); + assert(index < JOB_NUMBER); + return hw_job_ptr_locks[index].compare_exchange_strong(expected, true); } //HardwareCodecDeflateQpl diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 193454a66ab..5556e7383d2 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -19,10 +19,10 @@ public: DeflateQplJobHWPool(); ~DeflateQplJobHWPool(); static DeflateQplJobHWPool & instance(); - static constexpr auto JOB_POOL_SIZE = 1024; + static constexpr auto JOB_NUMBER = 1024; static constexpr qpl_path_t PATH = qpl_path_hardware; - static qpl_job * hw_job_pool[JOB_POOL_SIZE]; - static std::atomic_bool hw_job_locks[JOB_POOL_SIZE]; + static qpl_job * hw_job_ptr_pool[JOB_NUMBER]; + static std::atomic_bool hw_job_ptr_locks[JOB_NUMBER]; static bool job_pool_ready; bool & jobPoolReady() { return job_pool_ready;} @@ -34,7 +34,7 @@ public: private: bool tryLockJob(size_t index); - void unLockJob(uint32_t index) { hw_job_locks[index].store(false); } + void unLockJob(uint32_t index) { hw_job_ptr_locks[index].store(false); } class ReleaseJobObjectGuard { @@ -44,10 +44,10 @@ private: public: ReleaseJobObjectGuard(const uint32_t index_) : index(index_){} - ~ReleaseJobObjectGuard(){ hw_job_locks[index].store(false); } + ~ReleaseJobObjectGuard(){ hw_job_ptr_locks[index].store(false); } }; - std::unique_ptr hw_job_pool_buffer; + std::unique_ptr hw_job_buffer; Poco::Logger * log; std::mt19937 random_engine; std::uniform_int_distribution distribution; From bc775ecc5b9f690186fef73f606c4daf40152a03 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sat, 9 Jul 2022 17:29:47 -0400 Subject: [PATCH 126/659] add assert for Illegal path --- src/Compression/CompressionCodecDeflateQpl.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index ce13b886fd4..1b6feaa3a42 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -89,6 +89,7 @@ qpl_job * DeflateQplJobHWPool::acquireJob(uint32_t * job_id) } } *job_id = JOB_NUMBER - index; + assert(index < JOB_NUMBER); return hw_job_ptr_pool[index]; } else @@ -100,6 +101,7 @@ qpl_job * DeflateQplJobHWPool::releaseJob(uint32_t job_id) if (jobPoolReady()) { uint32_t index = JOB_NUMBER - job_id; + assert(index < JOB_NUMBER); ReleaseJobObjectGuard _(index); return hw_job_ptr_pool[index]; } @@ -122,6 +124,9 @@ HardwareCodecDeflateQpl::HardwareCodecDeflateQpl(): HardwareCodecDeflateQpl::~HardwareCodecDeflateQpl() { +#ifndef NDEBUG + assert(decomp_async_job_map.empty()); +#else if (!decomp_async_job_map.empty()) { LOG_WARNING(log, "Find un-released job when HardwareCodecDeflateQpl destroy"); @@ -131,6 +136,7 @@ HardwareCodecDeflateQpl::~HardwareCodecDeflateQpl() } decomp_async_job_map.clear(); } +#endif } int32_t HardwareCodecDeflateQpl::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const From bc03e108703ed34c401513230425ba2fe1a3b75a Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sat, 9 Jul 2022 17:50:57 -0400 Subject: [PATCH 127/659] make job pool static member to private --- src/Compression/CompressionCodecDeflateQpl.cpp | 1 + src/Compression/CompressionCodecDeflateQpl.h | 16 +++++++++------- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 1b6feaa3a42..60b60cc33fd 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -18,6 +18,7 @@ namespace ErrorCodes qpl_job * DeflateQplJobHWPool::hw_job_ptr_pool[JOB_NUMBER]; std::atomic_bool DeflateQplJobHWPool::hw_job_ptr_locks[JOB_NUMBER]; bool DeflateQplJobHWPool::job_pool_ready; +std::unique_ptr DeflateQplJobHWPool::hw_job_buffer; DeflateQplJobHWPool & DeflateQplJobHWPool::instance() { diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 5556e7383d2..56130bc95e8 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -18,12 +18,6 @@ class DeflateQplJobHWPool public: DeflateQplJobHWPool(); ~DeflateQplJobHWPool(); - static DeflateQplJobHWPool & instance(); - static constexpr auto JOB_NUMBER = 1024; - static constexpr qpl_path_t PATH = qpl_path_hardware; - static qpl_job * hw_job_ptr_pool[JOB_NUMBER]; - static std::atomic_bool hw_job_ptr_locks[JOB_NUMBER]; - static bool job_pool_ready; bool & jobPoolReady() { return job_pool_ready;} @@ -31,6 +25,8 @@ public: qpl_job * releaseJob(uint32_t job_id); + static DeflateQplJobHWPool & instance(); + private: bool tryLockJob(size_t index); @@ -47,10 +43,16 @@ private: ~ReleaseJobObjectGuard(){ hw_job_ptr_locks[index].store(false); } }; - std::unique_ptr hw_job_buffer; + static constexpr auto JOB_NUMBER = 1024; + static constexpr qpl_path_t PATH = qpl_path_hardware; + static qpl_job * hw_job_ptr_pool[JOB_NUMBER]; + static std::atomic_bool hw_job_ptr_locks[JOB_NUMBER]; + static bool job_pool_ready; + static std::unique_ptr hw_job_buffer; Poco::Logger * log; std::mt19937 random_engine; std::uniform_int_distribution distribution; + }; class SoftwareCodecDeflateQpl From 28d9fc44cfa55656ebe984a69329c647506726e7 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 9 Jul 2022 23:16:16 +0000 Subject: [PATCH 128/659] fix reading with mmap from Log storages --- src/Disks/IO/createReadBufferFromFileBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index 345d0019aa7..d87144dee55 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -52,7 +52,7 @@ std::unique_ptr createReadBufferFromFileBase( { try { - auto res = std::make_unique(*settings.mmap_cache, filename, 0); + auto res = std::make_unique(*settings.mmap_cache, filename, 0, file_size.value_or(-1)); ProfileEvents::increment(ProfileEvents::CreatedReadBufferMMap); return res; } From 43779ec280fa626d6d712c35f503cb8ab5db0bd8 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Mon, 11 Jul 2022 14:59:39 +0200 Subject: [PATCH 129/659] add max_remote_{read,write}_network_bandwidth_for_server settings --- src/Common/Throttler.h | 5 +-- src/Common/Throttler_fwd.h | 11 +++++ src/Core/Settings.h | 2 + src/Disks/ObjectStorages/IObjectStorage.cpp | 24 +++++++++++ src/Disks/ObjectStorages/IObjectStorage.h | 14 ++++++ .../ObjectStorages/S3/S3ObjectStorage.cpp | 43 ++++++++++--------- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 3 ++ src/IO/ReadBufferFromS3.cpp | 6 ++- src/IO/ReadSettings.h | 4 ++ src/IO/WriteBufferFromS3.cpp | 10 ++++- src/IO/WriteBufferFromS3.h | 4 ++ src/IO/WriteSettings.h | 5 +++ src/Interpreters/Context.cpp | 26 ++++++++++- src/Interpreters/Context.h | 2 + src/Storages/StorageS3.cpp | 3 +- 15 files changed, 133 insertions(+), 29 deletions(-) create mode 100644 src/Common/Throttler_fwd.h diff --git a/src/Common/Throttler.h b/src/Common/Throttler.h index 89a83bb23be..6d44ad6ca5f 100644 --- a/src/Common/Throttler.h +++ b/src/Common/Throttler.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include #include @@ -57,7 +59,4 @@ private: std::shared_ptr parent; }; - -using ThrottlerPtr = std::shared_ptr; - } diff --git a/src/Common/Throttler_fwd.h b/src/Common/Throttler_fwd.h new file mode 100644 index 00000000000..1efaf1c85c5 --- /dev/null +++ b/src/Common/Throttler_fwd.h @@ -0,0 +1,11 @@ +#pragma once + +#include + +namespace DB +{ + +class Throttler; +using ThrottlerPtr = std::shared_ptr; + +} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5597d9076a4..733adb4a811 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -96,6 +96,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \ M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited. Only has meaning at server startup.", 0) \ M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited. Only has meaning at server startup.", 0) \ + M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited. Only has meaning at server startup.", 0) \ + M(UInt64, max_remote_write_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited. Only has meaning at server startup.", 0) \ M(Bool, stream_like_engine_allow_direct_select, false, "Allow direct SELECT query for Kafka, RabbitMQ, FileLog, Redis Streams and NATS engines. In case there are attached materialized views, SELECT query is not allowed even if this setting is enabled.", 0) \ M(String, stream_like_engine_insert_queue, "", "When stream like engine reads from multiple queues, user will need to select one queue to insert into when writing. Used by Redis Streams and NATS.", 0) \ \ diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index d29ecc24aeb..4fc8564a592 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -47,4 +47,28 @@ void IObjectStorage::copyObjectToAnotherObjectStorage(const std::string & object out->finalize(); } +void IObjectStorage::applyRemoteThrottlingSettings(ContextPtr context) +{ + std::unique_lock lock{throttlers_mutex}; + read_throttler = context->getRemoteReadThrottler(); + write_throttler = context->getRemoteWriteThrottler(); +} + +ReadSettings IObjectStorage::patchSettings(const ReadSettings & read_settings) const +{ + std::unique_lock lock{throttlers_mutex}; + ReadSettings settings{read_settings}; + settings.throttler = read_throttler; + return settings; +} + +WriteSettings IObjectStorage::patchSettings(const WriteSettings & write_settings) const +{ + std::unique_lock lock{throttlers_mutex}; + WriteSettings settings{write_settings}; + settings.throttler = write_throttler; + return settings; +} + + } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 7532f2a3267..f0b43ad7f8a 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -157,6 +158,19 @@ public: protected: FileCachePtr cache; + +protected: + /// Should be called from implementation of applyNewSettings() + void applyRemoteThrottlingSettings(ContextPtr context); + + /// Should be used by implementation of read* and write* methods + ReadSettings patchSettings(const ReadSettings & read_settings) const; + WriteSettings patchSettings(const WriteSettings & write_settings) const; + +private: + mutable std::mutex throttlers_mutex; + ThrottlerPtr read_throttler; + ThrottlerPtr write_throttler; }; using ObjectStoragePtr = std::unique_ptr; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 9236cde6e93..8b34dbefe3c 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -114,15 +114,7 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT std::optional, std::optional) const { - - ReadSettings disk_read_settings{read_settings}; - if (cache) - { - if (IFileCache::isReadOnly()) - disk_read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true; - - disk_read_settings.remote_fs_cache = cache; - } + ReadSettings disk_read_settings = patchSettings(read_settings); auto settings_ptr = s3_settings.get(); @@ -153,19 +145,10 @@ std::unique_ptr S3ObjectStorage::readObject( /// NOLINT std::optional) const { auto settings_ptr = s3_settings.get(); - ReadSettings disk_read_settings{read_settings}; - if (cache) - { - if (IFileCache::isReadOnly()) - disk_read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true; - - disk_read_settings.remote_fs_cache = cache; - } - + ReadSettings disk_read_settings = patchSettings(read_settings); return std::make_unique(client.get(), bucket, path, version_id, settings_ptr->s3_settings.max_single_read_retries, disk_read_settings); } - std::unique_ptr S3ObjectStorage::writeObject( /// NOLINT const std::string & path, WriteMode mode, // S3 doesn't support append, only rewrite @@ -174,12 +157,14 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN size_t buf_size, const WriteSettings & write_settings) { + WriteSettings disk_write_settings = patchSettings(write_settings); + if (mode != WriteMode::Rewrite) throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 doesn't support append to files"); bool cache_on_write = cache && fs::path(path).extension() != ".tmp" - && write_settings.enable_filesystem_cache_on_write_operations + && disk_write_settings.enable_filesystem_cache_on_write_operations && FileCacheFactory::instance().getSettings(getCacheBasePath()).cache_on_write_operations; auto settings_ptr = s3_settings.get(); @@ -189,7 +174,9 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN path, settings_ptr->s3_settings, attributes, - buf_size, threadPoolCallbackRunner(getThreadPoolWriter()), + buf_size, + threadPoolCallbackRunner(getThreadPoolWriter()), + disk_write_settings, cache_on_write ? cache : nullptr); return std::make_unique(std::move(s3_buffer), std::move(finalize_callback), path); @@ -457,6 +444,19 @@ void S3ObjectStorage::copyObject(const std::string & object_from, const std::str copyObjectImpl(bucket, object_from, bucket, object_to, head, object_to_attributes); } +ReadSettings S3ObjectStorage::patchSettings(const ReadSettings & read_settings) const +{ + ReadSettings settings{read_settings}; + if (cache) + { + if (IFileCache::isReadOnly()) + settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true; + + settings.remote_fs_cache = cache; + } + return IObjectStorage::patchSettings(settings); +} + void S3ObjectStorage::setNewSettings(std::unique_ptr && s3_settings_) { s3_settings.set(std::move(s3_settings_)); @@ -489,6 +489,7 @@ void S3ObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & { s3_settings.set(getSettings(config, config_prefix, context)); client.set(getClient(config, config_prefix, context)); + applyRemoteThrottlingSettings(context); } std::unique_ptr S3ObjectStorage::cloneObjectStorage(const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 5d4300bffd3..ce52f24dc33 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -122,6 +122,9 @@ public: ContextPtr context) override; private: + ReadSettings patchSettings(const ReadSettings & read_settings) const; + WriteSettings patchSettings(const WriteSettings & write_settings) const; + void setNewSettings(std::unique_ptr && s3_settings_); void setNewClient(std::unique_ptr && client_); diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index cf19b6f1980..1e49346c8a2 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -5,12 +5,13 @@ #include #include -#include #include #include #include +#include +#include #include #include @@ -300,6 +301,9 @@ std::unique_ptr ReadBufferFromS3::initialize() { read_result = outcome.GetResultWithOwnership(); + if (read_settings.throttler) + read_settings.throttler->add(read_result.GetContentLength()); + size_t buffer_size = use_external_buffer ? 0 : read_settings.remote_fs_buffer_size; return std::make_unique(read_result.GetBody(), buffer_size); } diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 78d5d6f3d65..97e72bdb147 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -89,6 +90,9 @@ struct ReadSettings FileCachePtr remote_fs_cache; + /// Bandwidth throttler to use during reading + ThrottlerPtr throttler; + size_t http_max_tries = 1; size_t http_retry_initial_backoff_ms = 100; size_t http_retry_max_backoff_ms = 1600; diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 432304d6d5d..7aca544a1a9 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include @@ -61,6 +62,7 @@ WriteBufferFromS3::WriteBufferFromS3( std::optional> object_metadata_, size_t buffer_size_, ScheduleFunc schedule_, + const WriteSettings & write_settings_, FileCachePtr cache_) : BufferWithOwnMemory(buffer_size_, nullptr, 0) , bucket(bucket_) @@ -70,6 +72,7 @@ WriteBufferFromS3::WriteBufferFromS3( , s3_settings(s3_settings_) , object_metadata(std::move(object_metadata_)) , schedule(std::move(schedule_)) + , write_settings(write_settings_) , cache(cache_) { allocateBuffer(); @@ -331,6 +334,8 @@ void WriteBufferFromS3::fillUploadRequest(Aws::S3::Model::UploadPartRequest & re void WriteBufferFromS3::processUploadRequest(UploadPartTask & task) { auto outcome = client_ptr->UploadPart(task.req); + if (write_settings.throttler) + write_settings.throttler->add(bytes); if (outcome.IsSuccess()) { @@ -460,9 +465,12 @@ void WriteBufferFromS3::fillPutRequest(Aws::S3::Model::PutObjectRequest & req) void WriteBufferFromS3::processPutRequest(PutObjectTask & task) { + size_t bytes = task.req.GetContentLength(); auto outcome = client_ptr->PutObject(task.req); - bool with_pool = static_cast(schedule); + if (write_settings.throttler) + write_settings.throttler->add(bytes); + bool with_pool = static_cast(schedule); if (outcome.IsSuccess()) LOG_TRACE(log, "Single part upload has completed. Bucket: {}, Key: {}, Object size: {}, WithPool: {}", bucket, key, task.req.GetContentLength(), with_pool); else diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 4cdc39b80a0..e79051823c4 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -55,6 +56,7 @@ public: std::optional> object_metadata_ = std::nullopt, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, ScheduleFunc schedule_ = {}, + const WriteSettings & write_settings_ = {}, FileCachePtr cache_ = nullptr); ~WriteBufferFromS3() override; @@ -119,6 +121,8 @@ private: Poco::Logger * log = &Poco::Logger::get("WriteBufferFromS3"); + WriteSettings write_settings; + FileCachePtr cache; size_t current_download_offset = 0; std::optional file_segments_holder; diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index 3464bb31664..547ed0eb91b 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -1,5 +1,7 @@ #pragma once +#include + namespace DB { @@ -7,6 +9,9 @@ namespace DB struct WriteSettings { bool enable_filesystem_cache_on_write_operations = false; + + /// Bandwidth throttler to use during writing + ThrottlerPtr throttler; }; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ca6ff02b994..b00d20bae32 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -228,8 +228,10 @@ struct ContextSharedPart mutable std::unique_ptr distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends) mutable std::unique_ptr message_broker_schedule_pool; /// A thread pool that can run different jobs in background (used for message brokers, like RabbitMQ and Kafka) - mutable ThrottlerPtr replicated_fetches_throttler; /// A server-wide throttler for replicated fetches - mutable ThrottlerPtr replicated_sends_throttler; /// A server-wide throttler for replicated sends + mutable ThrottlerPtr replicated_fetches_throttler; /// A server-wide throttler for replicated fetches + mutable ThrottlerPtr replicated_sends_throttler; /// A server-wide throttler for replicated sends + mutable ThrottlerPtr remote_read_throttler; /// A server-wide throttler for remote IO reads + mutable ThrottlerPtr remote_write_throttler; /// A server-wide throttler for remote IO writes MultiVersion macros; /// Substitutions extracted from config. std::unique_ptr ddl_worker; /// Process ddl commands from zk. @@ -1930,6 +1932,26 @@ ThrottlerPtr Context::getReplicatedSendsThrottler() const return shared->replicated_sends_throttler; } +ThrottlerPtr Context::getRemoteReadThrottler() const +{ + auto lock = getLock(); + if (!shared->remote_read_throttler) + shared->remote_read_throttler = std::make_shared( + settings.max_remote_read_network_bandwidth_for_server); + + return shared->remote_read_throttler; +} + +ThrottlerPtr Context::getRemoteWriteThrottler() const +{ + auto lock = getLock(); + if (!shared->remote_write_throttler) + shared->remote_write_throttler = std::make_shared( + settings.max_remote_write_network_bandwidth_for_server); + + return shared->remote_write_throttler; +} + bool Context::hasDistributedDDL() const { return getConfigRef().has("distributed_ddl"); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 83193dd589b..3c88d22f807 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -819,6 +819,8 @@ public: ThrottlerPtr getReplicatedFetchesThrottler() const; ThrottlerPtr getReplicatedSendsThrottler() const; + ThrottlerPtr getRemoteReadThrottler() const; + ThrottlerPtr getRemoteWriteThrottler() const; /// Has distributed_ddl configuration or not. bool hasDistributedDDL() const; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index bed21a9affc..fffd383955f 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -587,7 +587,8 @@ public: s3_configuration_.rw_settings, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, - threadPoolCallbackRunner(IOThreadPool::get())), + threadPoolCallbackRunner(IOThreadPool::get()), + WriteSettings{.throttler = context->getRemoteWriteThrottler()}), compression_method, 3); writer From 5c221464ba488995cd1206cec5cf5baa4d182e59 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 11 Jul 2022 10:10:31 -0300 Subject: [PATCH 130/659] Forward declare ares_channel and apply -wno-reserved-identifier to CARESPTRResolver unit --- src/CMakeLists.txt | 2 +- src/Common/CARESPTRResolver.cpp | 19 ++++++++++--------- src/Common/CARESPTRResolver.h | 7 +++---- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 7f5b97cc52b..c04d679f67b 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -439,7 +439,7 @@ if (TARGET ch_contrib::avrocpp) dbms_target_link_libraries(PRIVATE ch_contrib::avrocpp) endif () -target_compile_options(clickhouse_common_io PRIVATE -Wno-reserved-identifier) +set_source_files_properties(Common/CARESPTRResolver.cpp PROPERTIES COMPILE_FLAGS -Wno-reserved-identifier) target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::c-ares) if (TARGET OpenSSL::Crypto) diff --git a/src/Common/CARESPTRResolver.cpp b/src/Common/CARESPTRResolver.cpp index eac390c92c5..2739e0f602d 100644 --- a/src/Common/CARESPTRResolver.cpp +++ b/src/Common/CARESPTRResolver.cpp @@ -1,20 +1,21 @@ #include #include "CARESPTRResolver.h" #include "netdb.h" +#include "ares.h" namespace DB { static void callback(void * arg, int status, int, struct hostent * host) { auto * ptr_records = reinterpret_cast*>(arg); if (status == ARES_SUCCESS) { int i = 0; - while (auto ptr_record = host->h_aliases[i]) { + while (auto * ptr_record = host->h_aliases[i]) { ptr_records->emplace_back(ptr_record); i++; } } } - CARESPTRResolver::CARESPTRResolver() { + CARESPTRResolver::CARESPTRResolver() : channel(std::make_shared()) { init(); } @@ -32,13 +33,13 @@ namespace DB { } void CARESPTRResolver::init() { - if (ares_init(&channel) != ARES_SUCCESS){ + if (ares_init(channel.get()) != ARES_SUCCESS){ throw std::exception {}; } } void CARESPTRResolver::deinit() { - ares_destroy(channel); + ares_destroy(*channel); ares_library_cleanup(); } @@ -46,7 +47,7 @@ namespace DB { in_addr addr; inet_aton(ip.c_str(), &addr); - ares_gethostbyaddr(channel, reinterpret_cast(&addr), sizeof(addr), AF_INET, callback, &response); + ares_gethostbyaddr(*channel, reinterpret_cast(&addr), sizeof(addr), AF_INET, callback, &response); } void CARESPTRResolver::wait() { @@ -57,13 +58,13 @@ namespace DB { FD_ZERO(&read_fds); FD_ZERO(&write_fds); - nfds = ares_fds(channel, &read_fds, &write_fds); - if(nfds == 0){ + nfds = ares_fds(*channel, &read_fds, &write_fds); + if(nfds == 0) { break; } - tvp = ares_timeout(channel, nullptr, &tv); + tvp = ares_timeout(*channel, nullptr, &tv); select(nfds, &read_fds, &write_fds, nullptr, tvp); - ares_process(channel, &read_fds, &write_fds); + ares_process(*channel, &read_fds, &write_fds); } } } diff --git a/src/Common/CARESPTRResolver.h b/src/Common/CARESPTRResolver.h index 645723e9da1..04861bbb9bd 100644 --- a/src/Common/CARESPTRResolver.h +++ b/src/Common/CARESPTRResolver.h @@ -1,11 +1,11 @@ #pragma once #include "DNSPTRResolver.h" -#include "ares.h" + +using ares_channel = struct ares_channeldata *; namespace DB { class CARESPTRResolver : public DNSPTRResolver { - public: CARESPTRResolver(); ~CARESPTRResolver() override; @@ -13,14 +13,13 @@ namespace DB { std::vector resolve(const std::string & ip) override; private: - void init(); void deinit(); void wait(); void resolve(const std::string & ip, std::vector & response); - ares_channel channel; + std::shared_ptr channel; }; } From 0c6329bdd356dafd0becbb1ea3df9c8b3434b497 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 11 Jul 2022 10:19:25 -0300 Subject: [PATCH 131/659] ares_channel unique_ptr instead of shared_ptr --- src/Common/CARESPTRResolver.cpp | 2 +- src/Common/CARESPTRResolver.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/CARESPTRResolver.cpp b/src/Common/CARESPTRResolver.cpp index 2739e0f602d..9c8a2e4b9b7 100644 --- a/src/Common/CARESPTRResolver.cpp +++ b/src/Common/CARESPTRResolver.cpp @@ -15,7 +15,7 @@ namespace DB { } } - CARESPTRResolver::CARESPTRResolver() : channel(std::make_shared()) { + CARESPTRResolver::CARESPTRResolver() : channel(std::make_unique()) { init(); } diff --git a/src/Common/CARESPTRResolver.h b/src/Common/CARESPTRResolver.h index 04861bbb9bd..cbbbed017cf 100644 --- a/src/Common/CARESPTRResolver.h +++ b/src/Common/CARESPTRResolver.h @@ -19,7 +19,7 @@ namespace DB { void resolve(const std::string & ip, std::vector & response); - std::shared_ptr channel; + std::unique_ptr channel; }; } From 624ae4d8e9166817c3ffa35efe4cdce0e0770217 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 11 Jul 2022 12:15:53 -0300 Subject: [PATCH 132/659] link grpc against project-wide c-ares --- contrib/c-ares-cmake/CMakeLists.txt | 63 ++++++++++++++--------------- contrib/grpc-cmake/CMakeLists.txt | 36 +---------------- 2 files changed, 33 insertions(+), 66 deletions(-) diff --git a/contrib/c-ares-cmake/CMakeLists.txt b/contrib/c-ares-cmake/CMakeLists.txt index 87f2b96fac3..e534b42b963 100644 --- a/contrib/c-ares-cmake/CMakeLists.txt +++ b/contrib/c-ares-cmake/CMakeLists.txt @@ -1,37 +1,36 @@ -if(NOT TARGET c-ares) - # Choose to build static or shared library for c-ares. - if (USE_STATIC_LIBRARIES) - set(CARES_STATIC ON CACHE BOOL "" FORCE) - set(CARES_SHARED OFF CACHE BOOL "" FORCE) - else () - set(CARES_STATIC OFF CACHE BOOL "" FORCE) - set(CARES_SHARED ON CACHE BOOL "" FORCE) - endif () +# Choose to build static or shared library for c-ares. +if (USE_STATIC_LIBRARIES) + set(CARES_STATIC ON CACHE BOOL "" FORCE) + set(CARES_SHARED OFF CACHE BOOL "" FORCE) +else () + set(CARES_STATIC OFF CACHE BOOL "" FORCE) + set(CARES_SHARED ON CACHE BOOL "" FORCE) +endif () - # Disable looking for libnsl on a platforms that has gethostbyname in glibc - # - # c-ares searching for gethostbyname in the libnsl library, however in the - # version that shipped with gRPC it doing it wrong [1], since it uses - # CHECK_LIBRARY_EXISTS(), which will return TRUE even if the function exists in - # another dependent library. The upstream already contains correct macro [2], - # but it is not included in gRPC (even upstream gRPC, not the one that is - # shipped with clickhousee). - # - # [1]: https://github.com/c-ares/c-ares/blob/e982924acee7f7313b4baa4ee5ec000c5e373c30/CMakeLists.txt#L125 - # [2]: https://github.com/c-ares/c-ares/blob/44fbc813685a1fa8aa3f27fcd7544faf612d376a/CMakeLists.txt#L146 - # - # And because if you by some reason have libnsl [3] installed, clickhouse will - # reject to start w/o it. While this is completelly different library. - # - # [3]: https://packages.debian.org/bullseye/libnsl2 - if (NOT CMAKE_SYSTEM_NAME STREQUAL "SunOS") - set(HAVE_LIBNSL OFF CACHE BOOL "" FORCE) - endif() +# Disable looking for libnsl on a platforms that has gethostbyname in glibc +# +# c-ares searching for gethostbyname in the libnsl library, however in the +# version that shipped with gRPC it doing it wrong [1], since it uses +# CHECK_LIBRARY_EXISTS(), which will return TRUE even if the function exists in +# another dependent library. The upstream already contains correct macro [2], +# but it is not included in gRPC (even upstream gRPC, not the one that is +# shipped with clickhousee). +# +# [1]: https://github.com/c-ares/c-ares/blob/e982924acee7f7313b4baa4ee5ec000c5e373c30/CMakeLists.txt#L125 +# [2]: https://github.com/c-ares/c-ares/blob/44fbc813685a1fa8aa3f27fcd7544faf612d376a/CMakeLists.txt#L146 +# +# And because if you by some reason have libnsl [3] installed, clickhouse will +# reject to start w/o it. While this is completelly different library. +# +# [3]: https://packages.debian.org/bullseye/libnsl2 +if (NOT CMAKE_SYSTEM_NAME STREQUAL "SunOS") + set(HAVE_LIBNSL OFF CACHE BOOL "" FORCE) +endif() - add_subdirectory("../c-ares/" "../c-ares/") +add_subdirectory("../c-ares/" "../c-ares/") - add_library(ch_contrib::c-ares ALIAS c-ares) +add_library(ch_contrib::c-ares ALIAS c-ares) - target_link_libraries(c-ares PRIVATE resolv) +find_library(LIBRESOLV_LIBRARY NAMES resolv libresolv REQUIRED) -endif() \ No newline at end of file +target_link_libraries(c-ares PRIVATE ${LIBRESOLV_LIBRARY}) \ No newline at end of file diff --git a/contrib/grpc-cmake/CMakeLists.txt b/contrib/grpc-cmake/CMakeLists.txt index 9609b922840..5156140d88e 100644 --- a/contrib/grpc-cmake/CMakeLists.txt +++ b/contrib/grpc-cmake/CMakeLists.txt @@ -45,38 +45,11 @@ set(_gRPC_SSL_LIBRARIES OpenSSL::Crypto OpenSSL::SSL) # Use abseil-cpp from ClickHouse contrib, not from gRPC third_party. set(gRPC_ABSL_PROVIDER "clickhouse" CACHE STRING "" FORCE) -# Choose to build static or shared library for c-ares. -if (USE_STATIC_LIBRARIES) - set(CARES_STATIC ON CACHE BOOL "" FORCE) - set(CARES_SHARED OFF CACHE BOOL "" FORCE) -else () - set(CARES_STATIC OFF CACHE BOOL "" FORCE) - set(CARES_SHARED ON CACHE BOOL "" FORCE) -endif () - -# Disable looking for libnsl on a platforms that has gethostbyname in glibc -# -# c-ares searching for gethostbyname in the libnsl library, however in the -# version that shipped with gRPC it doing it wrong [1], since it uses -# CHECK_LIBRARY_EXISTS(), which will return TRUE even if the function exists in -# another dependent library. The upstream already contains correct macro [2], -# but it is not included in gRPC (even upstream gRPC, not the one that is -# shipped with clickhousee). -# -# [1]: https://github.com/c-ares/c-ares/blob/e982924acee7f7313b4baa4ee5ec000c5e373c30/CMakeLists.txt#L125 -# [2]: https://github.com/c-ares/c-ares/blob/44fbc813685a1fa8aa3f27fcd7544faf612d376a/CMakeLists.txt#L146 -# -# And because if you by some reason have libnsl [3] installed, clickhouse will -# reject to start w/o it. While this is completelly different library. -# -# [3]: https://packages.debian.org/bullseye/libnsl2 -if (NOT CMAKE_SYSTEM_NAME STREQUAL "SunOS") - set(HAVE_LIBNSL OFF CACHE BOOL "" FORCE) -endif() - # We don't want to build C# extensions. set(gRPC_BUILD_CSHARP_EXT OFF) +set(_gRPC_CARES_LIBRARIES ch_contrib::c-ares) + add_subdirectory("${_gRPC_SOURCE_DIR}" "${_gRPC_BINARY_DIR}") # The contrib/grpc/CMakeLists.txt redefined the PROTOBUF_GENERATE_GRPC_CPP() function for its own purposes, @@ -92,12 +65,7 @@ if(gRPC_USE_UNSECURE_LIBRARIES) else() set(gRPC_LIBRARIES grpc grpc++) endif() - -target_compile_options(c-ares PRIVATE -Wno-reserved-macro-identifier) - add_library(_ch_contrib_grpc INTERFACE) target_link_libraries(_ch_contrib_grpc INTERFACE ${gRPC_LIBRARIES}) target_include_directories(_ch_contrib_grpc SYSTEM INTERFACE ${gRPC_INCLUDE_DIRS}) add_library(ch_contrib::grpc ALIAS _ch_contrib_grpc) - -target_compile_options(grpc PRIVATE -Wno-reserved-macro-identifier) From f176024eb36a25f67bda1ccd79b92f42e959ceef Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 11 Jul 2022 12:29:38 -0300 Subject: [PATCH 133/659] prevent grpc from building c-ares --- contrib/grpc-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc-cmake/CMakeLists.txt b/contrib/grpc-cmake/CMakeLists.txt index 5156140d88e..4e29032f3c7 100644 --- a/contrib/grpc-cmake/CMakeLists.txt +++ b/contrib/grpc-cmake/CMakeLists.txt @@ -49,7 +49,7 @@ set(gRPC_ABSL_PROVIDER "clickhouse" CACHE STRING "" FORCE) set(gRPC_BUILD_CSHARP_EXT OFF) set(_gRPC_CARES_LIBRARIES ch_contrib::c-ares) - +set(gRPC_CARES_PROVIDER "DO NOT BUILD C-ARES") add_subdirectory("${_gRPC_SOURCE_DIR}" "${_gRPC_BINARY_DIR}") # The contrib/grpc/CMakeLists.txt redefined the PROTOBUF_GENERATE_GRPC_CPP() function for its own purposes, From add0d1698e8a505c931a0eff29b27018a69a8339 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Mon, 11 Jul 2022 17:30:23 +0200 Subject: [PATCH 134/659] respect new limits in HDFS --- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 11 ++++---- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 12 +++++++-- src/Storages/HDFS/ReadBufferFromHDFS.h | 4 ++- src/Storages/HDFS/StorageHDFS.cpp | 26 ++++++++++++++++--- src/Storages/HDFS/WriteBufferFromHDFS.cpp | 11 +++++--- src/Storages/HDFS/WriteBufferFromHDFS.h | 2 ++ src/Storages/Hive/HiveFile.cpp | 12 +++++++-- src/Storages/Hive/StorageHive.cpp | 5 +++- 9 files changed, 67 insertions(+), 18 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 1a4ae3f963f..27d7e10f556 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -115,7 +115,7 @@ SeekableReadBufferPtr ReadBufferFromHDFSGather::createImplementationBufferImpl(c auto hdfs_uri = path.substr(0, begin_of_path); LOG_TEST(log, "HDFS uri: {}, path: {}", hdfs_path, hdfs_uri); - return std::make_unique(hdfs_uri, hdfs_path, config); + return std::make_unique(hdfs_uri, hdfs_path, config, settings); } #endif diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index bedd1a83df1..4c2c0743ba0 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -45,7 +45,7 @@ std::unique_ptr HDFSObjectStorage::readObject( /// NOLINT std::optional, std::optional) const { - return std::make_unique(path, path, config, read_settings.remote_fs_buffer_size); + return std::make_unique(path, path, config, patchSettings(read_settings)); } std::unique_ptr HDFSObjectStorage::readObjects( /// NOLINT @@ -54,7 +54,7 @@ std::unique_ptr HDFSObjectStorage::readObjects( /// NOLI std::optional, std::optional) const { - auto hdfs_impl = std::make_unique(config, paths_to_read, read_settings); + auto hdfs_impl = std::make_unique(config, paths_to_read, patchSettings(read_settings)); auto buf = std::make_unique(std::move(hdfs_impl)); return std::make_unique(std::move(buf), settings->min_bytes_for_seek); } @@ -65,7 +65,7 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL std::optional attributes, FinalizeCallback && finalize_callback, size_t buf_size, - const WriteSettings &) + const WriteSettings & write_settings) { if (attributes.has_value()) throw Exception( @@ -74,7 +74,7 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL /// Single O_WRONLY in libhdfs adds O_TRUNC auto hdfs_buffer = std::make_unique( - path, config, settings->replication, buf_size, + path, config, settings->replication, patchSettings(write_settings), buf_size, mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND); return std::make_unique(std::move(hdfs_buffer), std::move(finalize_callback), path); @@ -147,8 +147,9 @@ void HDFSObjectStorage::copyObject( /// NOLINT } -void HDFSObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration &, const std::string &, ContextPtr) +void HDFSObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration &, const std::string &, ContextPtr context) { + applyRemoteThrottlingSettings(context); } std::unique_ptr HDFSObjectStorage::cloneObjectStorage(const std::string &, const Poco::Util::AbstractConfiguration &, const std::string &, ContextPtr) diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 208c8018c64..5d9d76efa87 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -2,6 +2,7 @@ #if USE_HDFS #include +#include #include #include @@ -31,6 +32,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(buf_size_) , hdfs_uri(hdfs_uri_) , hdfs_file_path(hdfs_file_path_) , builder(createHDFSBuilder(hdfs_uri_, config_)) + , read_settings(read_settings_) , read_until_position(read_until_position_) { fs = createHDFSFS(builder.get()); @@ -97,6 +102,8 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemoryadd(bytes_read); return true; } @@ -126,9 +133,10 @@ ReadBufferFromHDFS::ReadBufferFromHDFS( const String & hdfs_uri_, const String & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, + const ReadSettings & read_settings_, size_t buf_size_, size_t read_until_position_) : SeekableReadBuffer(nullptr, 0) - , impl(std::make_unique(hdfs_uri_, hdfs_file_path_, config_, buf_size_, read_until_position_)) + , impl(std::make_unique(hdfs_uri_, hdfs_file_path_, config_, read_settings_, buf_size_, read_until_position_)) { } diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.h b/src/Storages/HDFS/ReadBufferFromHDFS.h index e2929d60464..981262d34e0 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.h +++ b/src/Storages/HDFS/ReadBufferFromHDFS.h @@ -25,8 +25,10 @@ class ReadBufferFromHDFS : public SeekableReadBuffer, public WithFileName, publi struct ReadBufferFromHDFSImpl; public: - ReadBufferFromHDFS(const String & hdfs_uri_, const String & hdfs_file_path_, + ReadBufferFromHDFS(const String & hdfs_uri_, + const String & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, + const ReadSettings & read_settings_ = {}, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, size_t read_until_position_ = 0); diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 5e811f8e42c..a7adc634d7f 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -200,7 +201,13 @@ ColumnsDescription StorageHDFS::getTableStructureFromData( auto compression = chooseCompressionMethod(*it, compression_method); auto zstd_window_log_max = ctx->getSettingsRef().zstd_window_log_max; return wrapReadBufferWithCompressionMethod( - std::make_unique(uri_without_path, *it++, ctx->getGlobalContext()->getConfigRef()), compression, zstd_window_log_max); + std::make_unique( + uri_without_path, + *it++, + ctx->getGlobalContext()->getConfigRef(), + ReadSettings{.throttler = ctx->getRemoteReadThrottler()}), + compression, + zstd_window_log_max); }; return readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, paths.size() > 1, ctx); } @@ -330,7 +337,14 @@ bool HDFSSource::initialize() auto compression = chooseCompressionMethod(path_from_uri, storage->compression_method); const auto zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; - read_buf = wrapReadBufferWithCompressionMethod(std::make_unique(uri_without_path, path_from_uri, getContext()->getGlobalContext()->getConfigRef()), compression, zstd_window_log_max); + read_buf = wrapReadBufferWithCompressionMethod( + std::make_unique( + uri_without_path, + path_from_uri, + getContext()->getGlobalContext()->getConfigRef(), + ReadSettings{.throttler = getContext()->getRemoteReadThrottler()}), + compression, + zstd_window_log_max); auto input_format = getContext()->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size); @@ -410,7 +424,13 @@ public: const CompressionMethod compression_method) : SinkToStorage(sample_block) { - write_buf = wrapWriteBufferWithCompressionMethod(std::make_unique(uri, context->getGlobalContext()->getConfigRef(), context->getSettingsRef().hdfs_replication), compression_method, 3); + write_buf = wrapWriteBufferWithCompressionMethod( + std::make_unique( + uri, + context->getGlobalContext()->getConfigRef(), + context->getSettingsRef().hdfs_replication, + WriteSettings{.throttler = context->getRemoteWriteThrottler()}), + compression_method, 3); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context); } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 42ec3962beb..80acda52ffa 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -4,9 +4,9 @@ #include #include +#include #include - namespace DB { @@ -24,15 +24,18 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl hdfsFile fout; HDFSBuilderWrapper builder; HDFSFSPtr fs; + WriteSettings write_settings; WriteBufferFromHDFSImpl( const std::string & hdfs_uri_, const Poco::Util::AbstractConfiguration & config_, int replication_, + const WriteSettings & write_settings_, int flags) : hdfs_uri(hdfs_uri_) , builder(createHDFSBuilder(hdfs_uri, config_)) , fs(createHDFSFS(builder.get())) + , write_settings(write_settings_) { const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2); const String path = hdfs_uri.substr(begin_of_path); @@ -44,7 +47,6 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl throw Exception("Unable to open HDFS file: " + path + " error: " + std::string(hdfsGetLastError()), ErrorCodes::CANNOT_OPEN_FILE); } - } ~WriteBufferFromHDFSImpl() @@ -56,6 +58,8 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl int write(const char * start, size_t size) const { int bytes_written = hdfsWrite(fs.get(), fout, start, size); + if (write_settings.throttler) + write_settings.throttler->add(bytes_written); if (bytes_written < 0) throw Exception("Fail to write HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), @@ -77,10 +81,11 @@ WriteBufferFromHDFS::WriteBufferFromHDFS( const std::string & hdfs_name_, const Poco::Util::AbstractConfiguration & config_, int replication_, + const WriteSettings & write_settings_, size_t buf_size_, int flags_) : BufferWithOwnMemory(buf_size_) - , impl(std::make_unique(hdfs_name_, config_, replication_, flags_)) + , impl(std::make_unique(hdfs_name_, config_, replication_, write_settings_, flags_)) { } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.h b/src/Storages/HDFS/WriteBufferFromHDFS.h index fe9af7dfba4..3cc11a35186 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.h +++ b/src/Storages/HDFS/WriteBufferFromHDFS.h @@ -4,6 +4,7 @@ #if USE_HDFS #include +#include #include #include #include @@ -24,6 +25,7 @@ public: const String & hdfs_name_, const Poco::Util::AbstractConfiguration & config_, int replication_, + const WriteSettings & write_settings_ = {}, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, int flags = O_WRONLY); diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 57acbdd577b..46bd6aebab7 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -147,7 +147,11 @@ Range HiveORCFile::buildRange(const orc::ColumnStatistics * col_stats) void HiveORCFile::prepareReader() { - in = std::make_unique(namenode_url, path, getContext()->getGlobalContext()->getConfigRef()); + in = std::make_unique( + namenode_url, + path, + getContext()->getGlobalContext()->getConfigRef(), + ReadSettings{.throttler = getContext()->getRemoteReadThrottler()}); auto format_settings = getFormatSettings(getContext()); std::atomic is_stopped{0}; auto result = arrow::adapters::orc::ORCFileReader::Open(asArrowFile(*in, format_settings, is_stopped, "ORC", ORC_MAGIC_BYTES), arrow::default_memory_pool()); @@ -267,7 +271,11 @@ bool HiveParquetFile::useSplitMinMaxIndex() const void HiveParquetFile::prepareReader() { - in = std::make_unique(namenode_url, path, getContext()->getGlobalContext()->getConfigRef()); + in = std::make_unique( + namenode_url, + path, + getContext()->getGlobalContext()->getConfigRef(), + ReadSettings{.throttler = getContext()->getRemoteReadThrottler()}); auto format_settings = getFormatSettings(getContext()); std::atomic is_stopped{0}; THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(*in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES), arrow::default_memory_pool(), &reader)); diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index cea05aee0d5..dcb49b47557 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -218,7 +218,10 @@ public: auto get_raw_read_buf = [&]() -> std::unique_ptr { auto buf = std::make_unique( - hdfs_namenode_url, current_path, getContext()->getGlobalContext()->getConfigRef()); + hdfs_namenode_url, + current_path, + getContext()->getGlobalContext()->getConfigRef(), + ReadSettings{.throttler = getContext()->getRemoteReadThrottler()}); bool thread_pool_read = read_settings.remote_fs_method == RemoteFSReadMethod::threadpool; if (thread_pool_read) From 7b626da080df89b254630c09b23bb884a9bbce53 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 11 Jul 2022 12:43:56 -0300 Subject: [PATCH 135/659] remove cpp-dns and udns from fasttest --- docker/test/fasttest/run.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index ce75e599eac..6b8109a15b2 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -135,8 +135,7 @@ function clone_submodules contrib/replxx contrib/wyhash contrib/hashidsxx - contrib/udns - contrib/cpp-dns + contrib/c-ares ) git submodule sync From bcba581191a4518abe6e8f01295666835d59c6d6 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 11 Jul 2022 12:08:35 -0400 Subject: [PATCH 136/659] fixed cosmetic issues --- .../CompressionCodecDeflateQpl.cpp | 99 ++++++++++--------- src/Compression/CompressionCodecDeflateQpl.h | 32 +++--- 2 files changed, 67 insertions(+), 64 deletions(-) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 60b60cc33fd..b7a5c89b121 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -15,39 +15,39 @@ namespace ErrorCodes extern const int CANNOT_COMPRESS; extern const int CANNOT_DECOMPRESS; } -qpl_job * DeflateQplJobHWPool::hw_job_ptr_pool[JOB_NUMBER]; -std::atomic_bool DeflateQplJobHWPool::hw_job_ptr_locks[JOB_NUMBER]; + +std::array DeflateQplJobHWPool::hw_job_ptr_pool; +std::array DeflateQplJobHWPool::hw_job_ptr_locks; bool DeflateQplJobHWPool::job_pool_ready; -std::unique_ptr DeflateQplJobHWPool::hw_job_buffer; +std::unique_ptr DeflateQplJobHWPool::hw_jobs_buffer; DeflateQplJobHWPool & DeflateQplJobHWPool::instance() { - static DeflateQplJobHWPool ret; - return ret; + static DeflateQplJobHWPool pool; + return pool; } -DeflateQplJobHWPool::DeflateQplJobHWPool(): - log(&Poco::Logger::get("DeflateQplJobHWPool")), - random_engine(std::random_device()()), - distribution(0, JOB_NUMBER-1) +DeflateQplJobHWPool::DeflateQplJobHWPool() + :random_engine(std::random_device()()) + ,distribution(0, MAX_HW_JOB_NUMBER-1) { + Poco::Logger * log = &Poco::Logger::get("DeflateQplJobHWPool"); uint32_t job_size = 0; uint32_t index = 0; const char * qpl_version = qpl_get_library_version(); /// Get size required for saving a single qpl job object - qpl_get_job_size(PATH, &job_size); + qpl_get_job_size(qpl_path_hardware, &job_size); /// Allocate entire buffer for storing all job objects - hw_job_buffer = std::make_unique(job_size * JOB_NUMBER); + hw_jobs_buffer = std::make_unique(job_size * MAX_HW_JOB_NUMBER); /// Initialize pool for storing all job object pointers - memset(hw_job_ptr_pool, 0, JOB_NUMBER*sizeof(qpl_job *)); - /// Reallocate buffer for each job object through shifting address offset - for (index = 0; index < JOB_NUMBER; ++index) + /// Reallocate buffer by shifting address offset for each job object. + for (index = 0; index < MAX_HW_JOB_NUMBER; ++index) { - qpl_job * qpl_job_ptr = reinterpret_cast(hw_job_buffer.get() + index*job_size); - if ((!qpl_job_ptr) || (qpl_init_job(PATH, qpl_job_ptr) != QPL_STS_OK)) + qpl_job * qpl_job_ptr = reinterpret_cast(hw_jobs_buffer.get() + index * job_size); + if (qpl_init_job(qpl_path_hardware, qpl_job_ptr) != QPL_STS_OK) { - jobPoolReady() = false; + job_pool_ready = false; LOG_WARNING(log, "Initialization of hardware-assisted DeflateQpl codec failed, falling back to software DeflateQpl codec. Please check if Intel In-Memory Analytics Accelerator (IAA) is properly set up. QPL Version:{}.",qpl_version); return; } @@ -55,13 +55,13 @@ DeflateQplJobHWPool::DeflateQplJobHWPool(): unLockJob(index); } - jobPoolReady() = true; + job_pool_ready = true; LOG_DEBUG(log, "Hardware-assisted DeflateQpl codec is ready! QPL Version:{}",qpl_version); } DeflateQplJobHWPool::~DeflateQplJobHWPool() { - for (uint32_t i = 0; i < JOB_NUMBER; ++i) + for (uint32_t i = 0; i < MAX_HW_JOB_NUMBER; ++i) { if (hw_job_ptr_pool[i]) { @@ -71,26 +71,26 @@ DeflateQplJobHWPool::~DeflateQplJobHWPool() hw_job_ptr_pool[i] = nullptr; } } - jobPoolReady() = false; + job_pool_ready = false; } qpl_job * DeflateQplJobHWPool::acquireJob(uint32_t * job_id) { - if (jobPoolReady()) + if (isJobPoolReady()) { uint32_t retry = 0; auto index = distribution(random_engine); - while (tryLockJob(index) == false) + while (!tryLockJob(index)) { index = distribution(random_engine); retry++; - if (retry > JOB_NUMBER) + if (retry > MAX_HW_JOB_NUMBER) { return nullptr; } } - *job_id = JOB_NUMBER - index; - assert(index < JOB_NUMBER); + *job_id = MAX_HW_JOB_NUMBER - index; + assert(index < MAX_HW_JOB_NUMBER); return hw_job_ptr_pool[index]; } else @@ -99,10 +99,10 @@ qpl_job * DeflateQplJobHWPool::acquireJob(uint32_t * job_id) qpl_job * DeflateQplJobHWPool::releaseJob(uint32_t job_id) { - if (jobPoolReady()) + if (isJobPoolReady()) { - uint32_t index = JOB_NUMBER - job_id; - assert(index < JOB_NUMBER); + uint32_t index = MAX_HW_JOB_NUMBER - job_id; + assert(index < MAX_HW_JOB_NUMBER); ReleaseJobObjectGuard _(index); return hw_job_ptr_pool[index]; } @@ -113,13 +113,13 @@ qpl_job * DeflateQplJobHWPool::releaseJob(uint32_t job_id) bool DeflateQplJobHWPool::tryLockJob(size_t index) { bool expected = false; - assert(index < JOB_NUMBER); + assert(index < MAX_HW_JOB_NUMBER); return hw_job_ptr_locks[index].compare_exchange_strong(expected, true); } //HardwareCodecDeflateQpl -HardwareCodecDeflateQpl::HardwareCodecDeflateQpl(): - log(&Poco::Logger::get("HardwareCodecDeflateQpl")) +HardwareCodecDeflateQpl::HardwareCodecDeflateQpl() + :log(&Poco::Logger::get("HardwareCodecDeflateQpl")) { } @@ -167,13 +167,13 @@ int32_t HardwareCodecDeflateQpl::doCompressData(const char * source, uint32_t so return compressed_size; } -int32_t HardwareCodecDeflateQpl::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const +int32_t HardwareCodecDeflateQpl::doDecompressDataSynchronous(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const { uint32_t job_id = 0; qpl_job* job_ptr = nullptr; if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(&job_id))) { - LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressData->acquireJob fail, probably job pool exhausted)"); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataSynchronous->acquireJob fail, probably job pool exhausted)"); return RET_ERROR; } // Performing a decompression operation @@ -191,19 +191,19 @@ int32_t HardwareCodecDeflateQpl::doDecompressData(const char * source, uint32_t } else { - LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressData->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataSynchronous->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); DeflateQplJobHWPool::instance().releaseJob(job_id); return RET_ERROR; } } -int32_t HardwareCodecDeflateQpl::doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) +int32_t HardwareCodecDeflateQpl::doDecompressDataAsynchronous(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) { uint32_t job_id = 0; qpl_job * job_ptr = nullptr; if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(&job_id))) { - LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataReq->acquireJob fail, probably job pool exhausted)"); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataAsynchronous->acquireJob fail, probably job pool exhausted)"); return RET_ERROR; } @@ -223,7 +223,7 @@ int32_t HardwareCodecDeflateQpl::doDecompressDataReq(const char * source, uint32 else { DeflateQplJobHWPool::instance().releaseJob(job_id); - LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataReq->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataAsynchronous->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); return RET_ERROR; } } @@ -321,9 +321,9 @@ void SoftwareCodecDeflateQpl::doDecompressData(const char * source, uint32_t sou } //CompressionCodecDeflateQpl -CompressionCodecDeflateQpl::CompressionCodecDeflateQpl(): - hw_codec(std::make_unique()), - sw_codec(std::make_unique()) +CompressionCodecDeflateQpl::CompressionCodecDeflateQpl() + :hw_codec(std::make_unique()) + ,sw_codec(std::make_unique()) { setCodecDescription("DEFLATE_QPL"); } @@ -347,7 +347,7 @@ uint32_t CompressionCodecDeflateQpl::getMaxCompressedDataSize(uint32_t uncompres uint32_t CompressionCodecDeflateQpl::doCompressData(const char * source, uint32_t source_size, char * dest) const { int32_t res = HardwareCodecDeflateQpl::RET_ERROR; - if (DeflateQplJobHWPool::instance().jobPoolReady()) + if (DeflateQplJobHWPool::instance().isJobPoolReady()) res = hw_codec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); if (res == HardwareCodecDeflateQpl::RET_ERROR) res = sw_codec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); @@ -361,30 +361,31 @@ void CompressionCodecDeflateQpl::doDecompressData(const char * source, uint32_t case CodecMode::Synchronous: { int32_t res = HardwareCodecDeflateQpl::RET_ERROR; - if (DeflateQplJobHWPool::instance().jobPoolReady()) - res = hw_codec->doDecompressData(source, source_size, dest, uncompressed_size); + if (DeflateQplJobHWPool::instance().isJobPoolReady()) + res = hw_codec->doDecompressDataSynchronous(source, source_size, dest, uncompressed_size); if (res == HardwareCodecDeflateQpl::RET_ERROR) sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); - break; + return; } case CodecMode::Asynchronous: { int32_t res = HardwareCodecDeflateQpl::RET_ERROR; - if (DeflateQplJobHWPool::instance().jobPoolReady()) - res = hw_codec->doDecompressDataReq(source, source_size, dest, uncompressed_size); + if (DeflateQplJobHWPool::instance().isJobPoolReady()) + res = hw_codec->doDecompressDataAsynchronous(source, source_size, dest, uncompressed_size); if (res == HardwareCodecDeflateQpl::RET_ERROR) sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); - break; + return; } case CodecMode::SoftwareFallback: sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); - break; + return; } + __builtin_unreachable(); } void CompressionCodecDeflateQpl::flushAsynchronousDecompressRequests() { - if (DeflateQplJobHWPool::instance().jobPoolReady()) + if (DeflateQplJobHWPool::instance().isJobPoolReady()) hw_codec->flushAsynchronousDecompressRequests(); setDecompressMode(CodecMode::Synchronous); } diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 56130bc95e8..10d55600f89 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -19,18 +19,20 @@ public: DeflateQplJobHWPool(); ~DeflateQplJobHWPool(); - bool & jobPoolReady() { return job_pool_ready;} - qpl_job * acquireJob(uint32_t * job_id); - qpl_job * releaseJob(uint32_t job_id); + static qpl_job * releaseJob(uint32_t job_id); + + static const bool & isJobPoolReady() { return job_pool_ready; } static DeflateQplJobHWPool & instance(); -private: - bool tryLockJob(size_t index); + static constexpr auto MAX_HW_JOB_NUMBER = 1024; - void unLockJob(uint32_t index) { hw_job_ptr_locks[index].store(false); } +private: + static bool tryLockJob(size_t index); + + static void unLockJob(uint32_t index) { hw_job_ptr_locks[index].store(false); } class ReleaseJobObjectGuard { @@ -43,16 +45,15 @@ private: ~ReleaseJobObjectGuard(){ hw_job_ptr_locks[index].store(false); } }; - static constexpr auto JOB_NUMBER = 1024; - static constexpr qpl_path_t PATH = qpl_path_hardware; - static qpl_job * hw_job_ptr_pool[JOB_NUMBER]; - static std::atomic_bool hw_job_ptr_locks[JOB_NUMBER]; + /// Entire buffer for storing all job objects + static std::unique_ptr hw_jobs_buffer; + /// Job pool for storing all job object pointers + static std::array hw_job_ptr_pool; + /// Locks for accessing each job object pointers + static std::array hw_job_ptr_locks; static bool job_pool_ready; - static std::unique_ptr hw_job_buffer; - Poco::Logger * log; std::mt19937 random_engine; std::uniform_int_distribution distribution; - }; class SoftwareCodecDeflateQpl @@ -76,8 +77,8 @@ public: HardwareCodecDeflateQpl(); ~HardwareCodecDeflateQpl(); int32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const; - int32_t doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const; - int32_t doDecompressDataReq(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); + int32_t doDecompressDataSynchronous(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const; + int32_t doDecompressDataAsynchronous(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); /// Flush result for previous asynchronous decompression requests.Must be used following with several calls of doDecompressDataReq. void flushAsynchronousDecompressRequests(); @@ -88,6 +89,7 @@ private: std::map decomp_async_job_map; Poco::Logger * log; }; + class CompressionCodecDeflateQpl : public ICompressionCodec { public: From 6ed777d2a8c99dd8c516521d664e22222ae885f6 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Mon, 11 Jul 2022 19:19:57 +0200 Subject: [PATCH 137/659] add azure support --- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- .../AzureBlobStorage/AzureObjectStorage.cpp | 16 +++++++++------- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 1 - src/IO/ReadBufferFromAzureBlobStorage.cpp | 8 ++++++-- src/IO/ReadBufferFromAzureBlobStorage.h | 3 ++- src/IO/WriteBufferFromAzureBlobStorage.cpp | 6 ++++++ src/IO/WriteBufferFromAzureBlobStorage.h | 3 +++ src/Storages/HDFS/ReadBufferFromHDFS.cpp | 7 +++---- src/Storages/HDFS/ReadBufferFromHDFS.h | 1 - 10 files changed, 31 insertions(+), 18 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 27d7e10f556..bef02bb833a 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -89,7 +89,7 @@ SeekableReadBufferPtr ReadBufferFromAzureBlobStorageGather::createImplementation path, max_single_read_retries, max_single_download_retries, - settings.remote_fs_buffer_size, + settings, /* use_external_buffer */true, read_until_position); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index c25d1d7470c..59fc8ab7548 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -63,7 +63,7 @@ std::unique_ptr AzureObjectStorage::readObject( /// NOLINT return std::make_unique( client.get(), path, settings_ptr->max_single_read_retries, - settings_ptr->max_single_download_retries, read_settings.remote_fs_buffer_size); + settings_ptr->max_single_download_retries, patchSettings(read_settings)); } std::unique_ptr AzureObjectStorage::readObjects( /// NOLINT @@ -72,18 +72,19 @@ std::unique_ptr AzureObjectStorage::readObjects( /// NOL std::optional, std::optional) const { + ReadSettings disk_read_settings = patchSettings(read_settings); auto settings_ptr = settings.get(); auto reader_impl = std::make_unique( client.get(), paths_to_read, settings_ptr->max_single_read_retries, settings_ptr->max_single_download_retries, - read_settings); + disk_read_settings); - if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) + if (disk_read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { auto reader = getThreadPoolReader(); - return std::make_unique(reader, read_settings, std::move(reader_impl)); + return std::make_unique(reader, disk_read_settings, std::move(reader_impl)); } else { @@ -99,7 +100,7 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO std::optional, FinalizeCallback && finalize_callback, size_t buf_size, - const WriteSettings &) + const WriteSettings & write_settings) { if (mode != WriteMode::Rewrite) throw Exception("Azure storage doesn't support append", ErrorCodes::UNSUPPORTED_METHOD); @@ -108,7 +109,8 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO client.get(), path, settings.get()->max_single_part_upload_size, - buf_size); + buf_size, + patchSettings(write_settings)); return std::make_unique(std::move(buffer), std::move(finalize_callback), path); } @@ -200,7 +202,7 @@ void AzureObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguratio { auto new_settings = getAzureBlobStorageSettings(config, config_prefix, context); settings.set(std::move(new_settings)); - + applyRemoteThrottlingSettings(context); /// We don't update client } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 8b34dbefe3c..44ed7a33b78 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -157,7 +157,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN size_t buf_size, const WriteSettings & write_settings) { - WriteSettings disk_write_settings = patchSettings(write_settings); + WriteSettings disk_write_settings = IObjectStorage::patchSettings(write_settings); if (mode != WriteMode::Rewrite) throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 doesn't support append to files"); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index ce52f24dc33..3f645877fed 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -123,7 +123,6 @@ public: private: ReadSettings patchSettings(const ReadSettings & read_settings) const; - WriteSettings patchSettings(const WriteSettings & write_settings) const; void setNewSettings(std::unique_ptr && s3_settings_); diff --git a/src/IO/ReadBufferFromAzureBlobStorage.cpp b/src/IO/ReadBufferFromAzureBlobStorage.cpp index 2576b10f9ac..0a5b0f8a72d 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/IO/ReadBufferFromAzureBlobStorage.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -25,7 +26,7 @@ ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( const String & path_, size_t max_single_read_retries_, size_t max_single_download_retries_, - size_t tmp_buffer_size_, + const ReadSettings & read_settings_, bool use_external_buffer_, size_t read_until_position_) : SeekableReadBuffer(nullptr, 0) @@ -33,7 +34,8 @@ ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( , path(path_) , max_single_read_retries(max_single_read_retries_) , max_single_download_retries(max_single_download_retries_) - , tmp_buffer_size(tmp_buffer_size_) + , read_settings(read_settings_) + , tmp_buffer_size(read_settings.remote_fs_buffer_size) , use_external_buffer(use_external_buffer_) , read_until_position(read_until_position_) { @@ -75,6 +77,8 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() try { bytes_read = data_stream->ReadToCount(reinterpret_cast(data_ptr), to_read_bytes); + if (read_settings.throttler) + read_settings.throttler->add(bytes_read); break; } catch (const Azure::Storage::StorageException & e) diff --git a/src/IO/ReadBufferFromAzureBlobStorage.h b/src/IO/ReadBufferFromAzureBlobStorage.h index e5a39b84d45..9bdc2769c4c 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.h +++ b/src/IO/ReadBufferFromAzureBlobStorage.h @@ -22,7 +22,7 @@ public: const String & path_, size_t max_single_read_retries_, size_t max_single_download_retries_, - size_t tmp_buffer_size_, + const ReadSettings & read_settings_, bool use_external_buffer_ = false, size_t read_until_position_ = 0); @@ -47,6 +47,7 @@ private: const String path; size_t max_single_read_retries; size_t max_single_download_retries; + ReadSettings read_settings; std::vector tmp_buffer; size_t tmp_buffer_size; bool use_external_buffer; diff --git a/src/IO/WriteBufferFromAzureBlobStorage.cpp b/src/IO/WriteBufferFromAzureBlobStorage.cpp index 51d8bf6aba2..0b7b6323b33 100644 --- a/src/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/IO/WriteBufferFromAzureBlobStorage.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -15,11 +16,13 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( const String & blob_path_, size_t max_single_part_upload_size_, size_t buf_size_, + const WriteSettings & write_settings_, std::optional> attributes_) : BufferWithOwnMemory(buf_size_, nullptr, 0) , blob_container_client(blob_container_client_) , max_single_part_upload_size(max_single_part_upload_size_) , blob_path(blob_path_) + , write_settings(write_settings_) , attributes(attributes_) { } @@ -84,6 +87,9 @@ void WriteBufferFromAzureBlobStorage::nextImpl() } block_blob_client.CommitBlockList(block_ids); + + if (write_settings.throttler) + write_settings.throttler->add(read); } } diff --git a/src/IO/WriteBufferFromAzureBlobStorage.h b/src/IO/WriteBufferFromAzureBlobStorage.h index ef13a24abd8..0005705e68c 100644 --- a/src/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/IO/WriteBufferFromAzureBlobStorage.h @@ -8,6 +8,7 @@ #include #include +#include #include #include @@ -24,6 +25,7 @@ public: const String & blob_path_, size_t max_single_part_upload_size_, size_t buf_size_, + const WriteSettings & write_settings_, std::optional> attributes_ = {}); ~WriteBufferFromAzureBlobStorage() override; @@ -36,6 +38,7 @@ private: std::shared_ptr blob_container_client; size_t max_single_part_upload_size; const String blob_path; + WriteSettings write_settings; std::optional> attributes; }; diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 5d9d76efa87..e79ff75eb42 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -42,9 +42,8 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(buf_size_) + : BufferWithOwnMemory(read_settings_.remote_fs_buffer_size) , hdfs_uri(hdfs_uri_) , hdfs_file_path(hdfs_file_path_) , builder(createHDFSBuilder(hdfs_uri_, config_)) @@ -134,9 +133,9 @@ ReadBufferFromHDFS::ReadBufferFromHDFS( const String & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, const ReadSettings & read_settings_, - size_t buf_size_, size_t read_until_position_) + size_t read_until_position_) : SeekableReadBuffer(nullptr, 0) - , impl(std::make_unique(hdfs_uri_, hdfs_file_path_, config_, read_settings_, buf_size_, read_until_position_)) + , impl(std::make_unique(hdfs_uri_, hdfs_file_path_, config_, read_settings_, read_until_position_)) { } diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.h b/src/Storages/HDFS/ReadBufferFromHDFS.h index 981262d34e0..2f1d7d675df 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.h +++ b/src/Storages/HDFS/ReadBufferFromHDFS.h @@ -29,7 +29,6 @@ public: const String & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, const ReadSettings & read_settings_ = {}, - size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, size_t read_until_position_ = 0); ~ReadBufferFromHDFS() override; From dcc8c646fd6a5e7cf2dce3f799c08f93de698ac8 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 11 Jul 2022 14:29:29 -0300 Subject: [PATCH 138/659] Throw proper DB::Exception in case c-ares initialization fails --- src/Common/CARESPTRResolver.cpp | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Common/CARESPTRResolver.cpp b/src/Common/CARESPTRResolver.cpp index 9c8a2e4b9b7..1977f5cbb7a 100644 --- a/src/Common/CARESPTRResolver.cpp +++ b/src/Common/CARESPTRResolver.cpp @@ -1,9 +1,16 @@ -#include #include "CARESPTRResolver.h" +#include +#include #include "netdb.h" #include "ares.h" namespace DB { + + namespace ErrorCodes + { + extern const int DNS_ERROR; + } + static void callback(void * arg, int status, int, struct hostent * host) { auto * ptr_records = reinterpret_cast*>(arg); if (status == ARES_SUCCESS) { @@ -33,8 +40,8 @@ namespace DB { } void CARESPTRResolver::init() { - if (ares_init(channel.get()) != ARES_SUCCESS){ - throw std::exception {}; + if (ares_init(channel.get()) != ARES_SUCCESS) { + throw DB::Exception("Failed to initialize c-ares", DB::ErrorCodes::DNS_ERROR); } } From 66074b3950e4df04f90570c81b0a71423d7743cf Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 11 Jul 2022 18:11:58 +0000 Subject: [PATCH 139/659] Fix distributed aggregation --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- .../02343_group_by_use_nulls.reference | 20 +++++++++++++++++++ .../0_stateless/02343_group_by_use_nulls.sql | 11 ++++++++++ 3 files changed, 32 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 1fbc42df111..43f2b533605 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -786,7 +786,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (analysis_result.use_grouping_set_key) res.insert({ nullptr, std::make_shared(), "__grouping_set" }); - if (context->getSettingsRef().group_by_use_nulls) + if (context->getSettingsRef().group_by_use_nulls && analysis_result.use_grouping_set_key) { for (const auto & key : query_analyzer->aggregationKeys()) res.insert({nullptr, makeNullableSafe(header.getByName(key.name).type), key.name}); diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.reference b/tests/queries/0_stateless/02343_group_by_use_nulls.reference index 92d36c1a894..ec893b91ce1 100644 --- a/tests/queries/0_stateless/02343_group_by_use_nulls.reference +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.reference @@ -155,3 +155,23 @@ SETTINGS group_by_use_nulls = 0; 7 0 7 8 0 8 9 0 9 +SELECT + CounterID AS k, + quantileBFloat16(0.5)(ResolutionWidth) +FROM remote('127.0.0.{1,2}', datasets, hits_v1) +GROUP BY k +ORDER BY + count() DESC, + CounterID ASC +LIMIT 10 +SETTINGS group_by_use_nulls = 1; +1704509 1384 +732797 1336 +598875 1384 +792887 1336 +3807842 1336 +25703952 1336 +716829 1384 +59183 1336 +33010362 1336 +800784 1336 diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.sql b/tests/queries/0_stateless/02343_group_by_use_nulls.sql index 5256c6bda75..455c53d10d6 100644 --- a/tests/queries/0_stateless/02343_group_by_use_nulls.sql +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.sql @@ -48,3 +48,14 @@ GROUP BY ) ORDER BY (number, number % 2, val) SETTINGS group_by_use_nulls = 0; + +SELECT + CounterID AS k, + quantileBFloat16(0.5)(ResolutionWidth) +FROM remote('127.0.0.{1,2}', datasets, hits_v1) +GROUP BY k +ORDER BY + count() DESC, + CounterID ASC +LIMIT 10 +SETTINGS group_by_use_nulls = 1; From 29cfe33ed7c8301442bd9bb8ef1a505514e6ec0a Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 11 Jul 2022 18:14:49 +0000 Subject: [PATCH 140/659] Small fix --- src/Interpreters/ExpressionAnalyzer.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 9353558926f..58dec59f703 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1513,10 +1513,10 @@ void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAGPtr & befor for (const auto & source_column : source_columns) { - if (isAggregateFunction(source_column.type)) - result_columns.push_back(source_column); - else + if (source_column.type->canBeInsideNullable()) result_columns.emplace_back(makeNullableSafe(source_column.type), source_column.name); + else + result_columns.push_back(source_column); } ExpressionActionsChain::Step & step = chain.lastStep(before_aggregation->getNamesAndTypesList()); From ffa5ef6293fd4668054b80005654eb96273255c3 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 12 Jul 2022 00:14:41 +0000 Subject: [PATCH 141/659] Update tests --- .../02343_group_by_use_nulls.reference | 20 ------------------- .../0_stateless/02343_group_by_use_nulls.sql | 11 ---------- .../00173_group_by_use_nulls.reference | 10 ++++++++++ .../1_stateful/00173_group_by_use_nulls.sql | 10 ++++++++++ 4 files changed, 20 insertions(+), 31 deletions(-) create mode 100644 tests/queries/1_stateful/00173_group_by_use_nulls.reference create mode 100644 tests/queries/1_stateful/00173_group_by_use_nulls.sql diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.reference b/tests/queries/0_stateless/02343_group_by_use_nulls.reference index ec893b91ce1..92d36c1a894 100644 --- a/tests/queries/0_stateless/02343_group_by_use_nulls.reference +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.reference @@ -155,23 +155,3 @@ SETTINGS group_by_use_nulls = 0; 7 0 7 8 0 8 9 0 9 -SELECT - CounterID AS k, - quantileBFloat16(0.5)(ResolutionWidth) -FROM remote('127.0.0.{1,2}', datasets, hits_v1) -GROUP BY k -ORDER BY - count() DESC, - CounterID ASC -LIMIT 10 -SETTINGS group_by_use_nulls = 1; -1704509 1384 -732797 1336 -598875 1384 -792887 1336 -3807842 1336 -25703952 1336 -716829 1384 -59183 1336 -33010362 1336 -800784 1336 diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.sql b/tests/queries/0_stateless/02343_group_by_use_nulls.sql index 455c53d10d6..5256c6bda75 100644 --- a/tests/queries/0_stateless/02343_group_by_use_nulls.sql +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.sql @@ -48,14 +48,3 @@ GROUP BY ) ORDER BY (number, number % 2, val) SETTINGS group_by_use_nulls = 0; - -SELECT - CounterID AS k, - quantileBFloat16(0.5)(ResolutionWidth) -FROM remote('127.0.0.{1,2}', datasets, hits_v1) -GROUP BY k -ORDER BY - count() DESC, - CounterID ASC -LIMIT 10 -SETTINGS group_by_use_nulls = 1; diff --git a/tests/queries/1_stateful/00173_group_by_use_nulls.reference b/tests/queries/1_stateful/00173_group_by_use_nulls.reference new file mode 100644 index 00000000000..02723bf14dd --- /dev/null +++ b/tests/queries/1_stateful/00173_group_by_use_nulls.reference @@ -0,0 +1,10 @@ +1704509 1384 +732797 1336 +598875 1384 +792887 1336 +3807842 1336 +25703952 1336 +716829 1384 +59183 1336 +33010362 1336 +800784 1336 diff --git a/tests/queries/1_stateful/00173_group_by_use_nulls.sql b/tests/queries/1_stateful/00173_group_by_use_nulls.sql new file mode 100644 index 00000000000..7acacc4e579 --- /dev/null +++ b/tests/queries/1_stateful/00173_group_by_use_nulls.sql @@ -0,0 +1,10 @@ +SELECT + CounterID AS k, + quantileBFloat16(0.5)(ResolutionWidth) +FROM remote('127.0.0.{1,2}', test, hits) +GROUP BY k +ORDER BY + count() DESC, + CounterID ASC +LIMIT 10 +SETTINGS group_by_use_nulls = 1; From 2b2e4a16e76d1a6fd780771fe21e2200689a0ada Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Jul 2022 02:30:03 +0000 Subject: [PATCH 142/659] disable random settings for some tests --- tests/queries/0_stateless/01475_read_subcolumns.sql | 2 +- tests/queries/0_stateless/02226_s3_with_cache.sql | 2 +- .../0_stateless/02240_system_remote_filesystem_cache.sql | 2 +- .../0_stateless/02240_system_remote_filesystem_cache_log.sql | 2 +- .../0_stateless/02240_system_remote_filesystem_query_cache.sql | 2 +- tests/queries/0_stateless/02286_drop_filesystem_cache.sql | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01475_read_subcolumns.sql b/tests/queries/0_stateless/01475_read_subcolumns.sql index 4724bec9eff..c287f6d55fa 100644 --- a/tests/queries/0_stateless/01475_read_subcolumns.sql +++ b/tests/queries/0_stateless/01475_read_subcolumns.sql @@ -1,4 +1,4 @@ --- Tags: no-s3-storage +-- Tags: no-s3-storage, no-random-settings SET use_uncompressed_cache = 0; diff --git a/tests/queries/0_stateless/02226_s3_with_cache.sql b/tests/queries/0_stateless/02226_s3_with_cache.sql index 0d0653d4089..3bd4b93155d 100644 --- a/tests/queries/0_stateless/02226_s3_with_cache.sql +++ b/tests/queries/0_stateless/02226_s3_with_cache.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest, long +-- Tags: no-parallel, no-fasttest, long, no-random-settings SET max_memory_usage='20G'; SET enable_filesystem_cache_on_write_operations = 0; diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql index 60a8eba8f3e..00a1f5cf5f4 100644 --- a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest, no-s3-storage +-- Tags: no-parallel, no-fasttest, no-s3-storage, no-random-settings -- { echo } diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache_log.sql b/tests/queries/0_stateless/02240_system_remote_filesystem_cache_log.sql index 871f9305c55..d49096cb9b2 100644 --- a/tests/queries/0_stateless/02240_system_remote_filesystem_cache_log.sql +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache_log.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest, no-s3-storage +-- Tags: no-parallel, no-fasttest, no-s3-storage, no-random-settings -- { echo } diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_query_cache.sql b/tests/queries/0_stateless/02240_system_remote_filesystem_query_cache.sql index 3fa3fdb5926..2a4f4ae219c 100644 --- a/tests/queries/0_stateless/02240_system_remote_filesystem_query_cache.sql +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_query_cache.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest, no-s3-storage +-- Tags: no-parallel, no-fasttest, no-s3-storage, no-random-settings -- { echo } diff --git a/tests/queries/0_stateless/02286_drop_filesystem_cache.sql b/tests/queries/0_stateless/02286_drop_filesystem_cache.sql index 8610299d08a..c0ba0cb3051 100644 --- a/tests/queries/0_stateless/02286_drop_filesystem_cache.sql +++ b/tests/queries/0_stateless/02286_drop_filesystem_cache.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest, no-s3-storage +-- Tags: no-parallel, no-fasttest, no-s3-storage, no-random-settings -- { echo } From 3b45e6e347e6f80bab566cf0ed537933c7d7a5a6 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Jul 2022 03:08:12 +0000 Subject: [PATCH 143/659] fix reading of empty arrays in reverse order with uncompressed cache (addition to #36215) --- src/Compression/CachedCompressedReadBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CachedCompressedReadBuffer.cpp b/src/Compression/CachedCompressedReadBuffer.cpp index 7f6422ae734..8abc16ebb2a 100644 --- a/src/Compression/CachedCompressedReadBuffer.cpp +++ b/src/Compression/CachedCompressedReadBuffer.cpp @@ -89,7 +89,7 @@ void CachedCompressedReadBuffer::seek(size_t offset_in_compressed_file, size_t o { /// Nothing to do if we already at required position if (!owned_cell && file_pos == offset_in_compressed_file - && (offset() == offset_in_decompressed_block || + && ((!buffer().empty() && offset() == offset_in_decompressed_block) || nextimpl_working_buffer_offset == offset_in_decompressed_block)) return; From f35370eac874fb3fcf726cbc821d020599aead4a Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Tue, 12 Jul 2022 19:50:58 +0800 Subject: [PATCH 144/659] rewrite testcase --- .../queries/0_stateless/02169_map_functions.reference | 10 ---------- tests/queries/0_stateless/02169_map_functions.sql | 11 ----------- .../0_stateless/02363_mapupdate_improve.reference | 10 ++++++++++ tests/queries/0_stateless/02363_mapupdate_improve.sql | 11 +++++++++++ 4 files changed, 21 insertions(+), 21 deletions(-) create mode 100644 tests/queries/0_stateless/02363_mapupdate_improve.reference create mode 100644 tests/queries/0_stateless/02363_mapupdate_improve.sql diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index 16fda0bb840..160aebbc852 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -31,13 +31,3 @@ {1:2,2:3} {'x':'y','x':'y'} {'x':'y','x':'y'} -{'fruit':'apple','season':'autumn'} -{'fruit':'apple','season':'autumn'} -{'fruit':'apple','season':'autumn'} -{'fruit':'apple','season':'autumn'} -{'fruit':'apple','season':'autumn'} -{'season':'autumn','fruit':'apple'} -{'season':'autumn','fruit':'apple'} -{'season':'autumn','fruit':'apple'} -{'season':'autumn','fruit':'apple'} -{'season':'autumn','fruit':'apple'} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index 9e93412739f..4cccaa56722 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -37,14 +37,3 @@ SELECT mapUpdate(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } DROP TABLE table_map; - -DROP TABLE IF EXISTS map_test; -CREATE TABLE map_test(`tags` Map(String, String)) ENGINE = MergeTree PRIMARY KEY tags ORDER BY tags SETTINGS index_granularity = 8192; -INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); -INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); -INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); -INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); -INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); -SELECT mapUpdate(mapFilter((k, v) -> (k in ('fruit')), tags), map('season', 'autumn')) FROM map_test; -SELECT mapUpdate(map('season','autumn'), mapFilter((k, v) -> (k in ('fruit')), tags)) FROM map_test; -DROP TABLE map_test; diff --git a/tests/queries/0_stateless/02363_mapupdate_improve.reference b/tests/queries/0_stateless/02363_mapupdate_improve.reference new file mode 100644 index 00000000000..04e2b943929 --- /dev/null +++ b/tests/queries/0_stateless/02363_mapupdate_improve.reference @@ -0,0 +1,10 @@ +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'fruit':'apple','season':'autumn'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} +{'season':'autumn','fruit':'apple'} diff --git a/tests/queries/0_stateless/02363_mapupdate_improve.sql b/tests/queries/0_stateless/02363_mapupdate_improve.sql new file mode 100644 index 00000000000..6b7723cc9b4 --- /dev/null +++ b/tests/queries/0_stateless/02363_mapupdate_improve.sql @@ -0,0 +1,11 @@ +-- Tags: no-backward-compatibility-check +DROP TABLE IF EXISTS map_test; +CREATE TABLE map_test(`tags` Map(String, String)) ENGINE = MergeTree PRIMARY KEY tags ORDER BY tags SETTINGS index_granularity = 8192; +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); +SELECT mapUpdate(mapFilter((k, v) -> (k in ('fruit')), tags), map('season', 'autumn')) FROM map_test; +SELECT mapUpdate(map('season','autumn'), mapFilter((k, v) -> (k in ('fruit')), tags)) FROM map_test; +DROP TABLE map_test; From cfca3db8844dc0f3ccb34aa813c7aafb94fda21b Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 12 Jul 2022 12:15:43 +0000 Subject: [PATCH 145/659] Fix crash with totals --- src/Processors/QueryPlan/CubeStep.cpp | 17 +++++- src/Processors/QueryPlan/RollupStep.cpp | 4 +- .../02343_group_by_use_nulls.reference | 58 +++++++++++++++++++ .../0_stateless/02343_group_by_use_nulls.sql | 12 ++++ 4 files changed, 86 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/CubeStep.cpp b/src/Processors/QueryPlan/CubeStep.cpp index 5559b4e1707..1642b577efc 100644 --- a/src/Processors/QueryPlan/CubeStep.cpp +++ b/src/Processors/QueryPlan/CubeStep.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -36,16 +37,26 @@ CubeStep::CubeStep(const DataStream & input_stream_, Aggregator::Params params_, output_stream->distinct_columns.insert(key); } -ProcessorPtr addGroupingSetForTotals(const Block & header, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number) +ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number) { auto dag = std::make_shared(header.getColumnsWithTypeAndName()); + auto & index = dag->getIndex(); + + auto to_nullable = FunctionFactory::instance().get("toNullable", nullptr); + for (const auto & key : keys) + { + const auto * node = dag->getIndex()[header.getPositionByName(key)]; + if (node->result_type->canBeInsideNullable()) + { + dag->addOrReplaceInIndex(dag->addFunction(to_nullable, { node }, node->result_name)); + } + } auto grouping_col = ColumnUInt64::create(1, grouping_set_number); const auto * grouping_node = &dag->addColumn( {ColumnPtr(std::move(grouping_col)), std::make_shared(), "__grouping_set"}); grouping_node = &dag->materializeNode(*grouping_node); - auto & index = dag->getIndex(); index.insert(index.begin(), grouping_node); auto expression = std::make_shared(dag, settings.getActionsSettings()); @@ -59,7 +70,7 @@ void CubeStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQue pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type == QueryPipelineBuilder::StreamType::Totals) - return addGroupingSetForTotals(header, settings, (UInt64(1) << keys_size) - 1); + return addGroupingSetForTotals(header, params.keys, settings, (UInt64(1) << keys_size) - 1); auto transform_params = std::make_shared(header, std::move(params), final); return std::make_shared(header, std::move(transform_params), use_nulls); diff --git a/src/Processors/QueryPlan/RollupStep.cpp b/src/Processors/QueryPlan/RollupStep.cpp index 3d1e79d9556..293fd79e7c5 100644 --- a/src/Processors/QueryPlan/RollupStep.cpp +++ b/src/Processors/QueryPlan/RollupStep.cpp @@ -34,7 +34,7 @@ RollupStep::RollupStep(const DataStream & input_stream_, Aggregator::Params para output_stream->distinct_columns.insert(key); } -ProcessorPtr addGroupingSetForTotals(const Block & header, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number); +ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number); void RollupStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { @@ -43,7 +43,7 @@ void RollupStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQ pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type == QueryPipelineBuilder::StreamType::Totals) - return addGroupingSetForTotals(header, settings, keys_size); + return addGroupingSetForTotals(header, params.keys, settings, keys_size); auto transform_params = std::make_shared(header, std::move(params), true); return std::make_shared(header, std::move(transform_params), use_nulls); diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.reference b/tests/queries/0_stateless/02343_group_by_use_nulls.reference index 92d36c1a894..24b7bb5277c 100644 --- a/tests/queries/0_stateless/02343_group_by_use_nulls.reference +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.reference @@ -155,3 +155,61 @@ SETTINGS group_by_use_nulls = 0; 7 0 7 8 0 8 9 0 9 +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY ROLLUP(number, number % 2) WITH TOTALS +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; +0 0 0 +0 \N 0 +1 1 1 +1 \N 1 +2 0 2 +2 \N 2 +3 1 3 +3 \N 3 +4 0 4 +4 \N 4 +5 1 5 +5 \N 5 +6 0 6 +6 \N 6 +7 1 7 +7 \N 7 +8 0 8 +8 \N 8 +9 1 9 +9 \N 9 +\N \N 45 + +0 0 45 +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY CUBE(number, number % 2) WITH TOTALS +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; +0 0 0 +0 \N 0 +1 1 1 +1 \N 1 +2 0 2 +2 \N 2 +3 1 3 +3 \N 3 +4 0 4 +4 \N 4 +5 1 5 +5 \N 5 +6 0 6 +6 \N 6 +7 1 7 +7 \N 7 +8 0 8 +8 \N 8 +9 1 9 +9 \N 9 +\N 0 20 +\N 1 25 +\N \N 45 + +0 0 45 diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.sql b/tests/queries/0_stateless/02343_group_by_use_nulls.sql index 5256c6bda75..a14db824013 100644 --- a/tests/queries/0_stateless/02343_group_by_use_nulls.sql +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.sql @@ -48,3 +48,15 @@ GROUP BY ) ORDER BY (number, number % 2, val) SETTINGS group_by_use_nulls = 0; + +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY ROLLUP(number, number % 2) WITH TOTALS +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; + +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY CUBE(number, number % 2) WITH TOTALS +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; From aabf5123d6ae97fa908d005498bf7806e127f8e3 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 12 Jul 2022 13:46:06 +0000 Subject: [PATCH 146/659] Fixup --- src/Processors/QueryPlan/CubeStep.cpp | 17 ++++++++++------- src/Processors/QueryPlan/RollupStep.cpp | 4 ++-- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPlan/CubeStep.cpp b/src/Processors/QueryPlan/CubeStep.cpp index 1642b577efc..52539dec75f 100644 --- a/src/Processors/QueryPlan/CubeStep.cpp +++ b/src/Processors/QueryPlan/CubeStep.cpp @@ -37,18 +37,21 @@ CubeStep::CubeStep(const DataStream & input_stream_, Aggregator::Params params_, output_stream->distinct_columns.insert(key); } -ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number) +ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, bool use_nulls, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number) { auto dag = std::make_shared(header.getColumnsWithTypeAndName()); auto & index = dag->getIndex(); - auto to_nullable = FunctionFactory::instance().get("toNullable", nullptr); - for (const auto & key : keys) + if (use_nulls) { - const auto * node = dag->getIndex()[header.getPositionByName(key)]; - if (node->result_type->canBeInsideNullable()) + auto to_nullable = FunctionFactory::instance().get("toNullable", nullptr); + for (const auto & key : keys) { - dag->addOrReplaceInIndex(dag->addFunction(to_nullable, { node }, node->result_name)); + const auto * node = dag->getIndex()[header.getPositionByName(key)]; + if (node->result_type->canBeInsideNullable()) + { + dag->addOrReplaceInIndex(dag->addFunction(to_nullable, { node }, node->result_name)); + } } } @@ -70,7 +73,7 @@ void CubeStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQue pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type == QueryPipelineBuilder::StreamType::Totals) - return addGroupingSetForTotals(header, params.keys, settings, (UInt64(1) << keys_size) - 1); + return addGroupingSetForTotals(header, params.keys, use_nulls, settings, (UInt64(1) << keys_size) - 1); auto transform_params = std::make_shared(header, std::move(params), final); return std::make_shared(header, std::move(transform_params), use_nulls); diff --git a/src/Processors/QueryPlan/RollupStep.cpp b/src/Processors/QueryPlan/RollupStep.cpp index 293fd79e7c5..3305f24602f 100644 --- a/src/Processors/QueryPlan/RollupStep.cpp +++ b/src/Processors/QueryPlan/RollupStep.cpp @@ -34,7 +34,7 @@ RollupStep::RollupStep(const DataStream & input_stream_, Aggregator::Params para output_stream->distinct_columns.insert(key); } -ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number); +ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, bool use_nulls, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number); void RollupStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { @@ -43,7 +43,7 @@ void RollupStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQ pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type == QueryPipelineBuilder::StreamType::Totals) - return addGroupingSetForTotals(header, params.keys, settings, keys_size); + return addGroupingSetForTotals(header, params.keys, use_nulls, settings, keys_size); auto transform_params = std::make_shared(header, std::move(params), true); return std::make_shared(header, std::move(transform_params), use_nulls); From d48690d455bb182fdff15c21da9e04ba59581f7c Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 12 Jul 2022 14:21:10 -0300 Subject: [PATCH 147/659] Make CaresPTRResolver a singleton through DNSPTRResolverProvider, add comments and address minor comments --- src/Access/Common/AllowedClientHosts.cpp | 13 ++-- src/CMakeLists.txt | 2 +- src/Common/CARESPTRResolver.cpp | 77 ------------------- src/Common/CARESPTRResolver.h | 25 ------- src/Common/CaresPTRResolver.cpp | 95 ++++++++++++++++++++++++ src/Common/CaresPTRResolver.h | 40 ++++++++++ src/Common/DNSPTRResolver.h | 2 + src/Common/DNSPTRResolverProvider.cpp | 8 +- src/Common/DNSPTRResolverProvider.h | 1 - src/Common/DNSResolver.cpp | 8 +- 10 files changed, 155 insertions(+), 116 deletions(-) delete mode 100644 src/Common/CARESPTRResolver.cpp delete mode 100644 src/Common/CARESPTRResolver.h create mode 100644 src/Common/CaresPTRResolver.cpp create mode 100644 src/Common/CaresPTRResolver.h diff --git a/src/Access/Common/AllowedClientHosts.cpp b/src/Access/Common/AllowedClientHosts.cpp index 09058a3c045..e32b218c948 100644 --- a/src/Access/Common/AllowedClientHosts.cpp +++ b/src/Access/Common/AllowedClientHosts.cpp @@ -110,18 +110,18 @@ namespace } /// Returns the host name by its address. - std::vector getHostsByAddress(const IPAddress & address) + Strings getHostsByAddress(const IPAddress & address) { - std::vector hosts = DNSResolver::instance().reverseResolve(address); + auto hosts = DNSResolver::instance().reverseResolve(address); if (hosts.empty()) - throw Exception(address.toString() + " could not be resolved", ErrorCodes::DNS_ERROR); + throw Exception(ErrorCodes::DNS_ERROR, "{} could not be resolved", address.toString()); - for (auto & host : hosts) { + for (const auto & host : hosts) { /// Check that PTR record is resolved back to client address if (!isAddressOfHost(address, host)) - throw Exception("Host " + String(host) + " isn't resolved back to " + address.toString(), ErrorCodes::DNS_ERROR); + throw Exception(ErrorCodes::DNS_ERROR, "Host {} isn't resolved back to {}", host, address.toString()); } return hosts; @@ -532,7 +532,8 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const { if (boost::iequals(name_regexp_, "localhost")) return is_client_local(); - if (!resolved_hosts) { + if (!resolved_hosts) + { resolved_hosts = getHostsByAddress(client_address); } diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index c04d679f67b..8e709acd43e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -439,7 +439,7 @@ if (TARGET ch_contrib::avrocpp) dbms_target_link_libraries(PRIVATE ch_contrib::avrocpp) endif () -set_source_files_properties(Common/CARESPTRResolver.cpp PROPERTIES COMPILE_FLAGS -Wno-reserved-identifier) +set_source_files_properties(Common/CaresPTRResolver.cpp PROPERTIES COMPILE_FLAGS -Wno-reserved-identifier) target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::c-ares) if (TARGET OpenSSL::Crypto) diff --git a/src/Common/CARESPTRResolver.cpp b/src/Common/CARESPTRResolver.cpp deleted file mode 100644 index 1977f5cbb7a..00000000000 --- a/src/Common/CARESPTRResolver.cpp +++ /dev/null @@ -1,77 +0,0 @@ -#include "CARESPTRResolver.h" -#include -#include -#include "netdb.h" -#include "ares.h" - -namespace DB { - - namespace ErrorCodes - { - extern const int DNS_ERROR; - } - - static void callback(void * arg, int status, int, struct hostent * host) { - auto * ptr_records = reinterpret_cast*>(arg); - if (status == ARES_SUCCESS) { - int i = 0; - while (auto * ptr_record = host->h_aliases[i]) { - ptr_records->emplace_back(ptr_record); - i++; - } - } - } - - CARESPTRResolver::CARESPTRResolver() : channel(std::make_unique()) { - init(); - } - - CARESPTRResolver::~CARESPTRResolver() { - deinit(); - } - - std::vector CARESPTRResolver::resolve(const std::string & ip) { - std::vector ptr_records; - - resolve(ip, ptr_records); - wait(); - - return ptr_records; - } - - void CARESPTRResolver::init() { - if (ares_init(channel.get()) != ARES_SUCCESS) { - throw DB::Exception("Failed to initialize c-ares", DB::ErrorCodes::DNS_ERROR); - } - } - - void CARESPTRResolver::deinit() { - ares_destroy(*channel); - ares_library_cleanup(); - } - - void CARESPTRResolver::resolve(const std::string & ip, std::vector & response) { - in_addr addr; - inet_aton(ip.c_str(), &addr); - - ares_gethostbyaddr(*channel, reinterpret_cast(&addr), sizeof(addr), AF_INET, callback, &response); - } - - void CARESPTRResolver::wait() { - for(;;) { - timeval * tvp, tv; - fd_set read_fds, write_fds; - int nfds; - - FD_ZERO(&read_fds); - FD_ZERO(&write_fds); - nfds = ares_fds(*channel, &read_fds, &write_fds); - if(nfds == 0) { - break; - } - tvp = ares_timeout(*channel, nullptr, &tv); - select(nfds, &read_fds, &write_fds, nullptr, tvp); - ares_process(*channel, &read_fds, &write_fds); - } - } -} diff --git a/src/Common/CARESPTRResolver.h b/src/Common/CARESPTRResolver.h deleted file mode 100644 index cbbbed017cf..00000000000 --- a/src/Common/CARESPTRResolver.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include "DNSPTRResolver.h" - -using ares_channel = struct ares_channeldata *; - -namespace DB { - class CARESPTRResolver : public DNSPTRResolver { - public: - CARESPTRResolver(); - ~CARESPTRResolver() override; - - std::vector resolve(const std::string & ip) override; - - private: - void init(); - void deinit(); - void wait(); - - void resolve(const std::string & ip, std::vector & response); - - std::unique_ptr channel; - }; -} - diff --git a/src/Common/CaresPTRResolver.cpp b/src/Common/CaresPTRResolver.cpp new file mode 100644 index 00000000000..6160c2fb3a7 --- /dev/null +++ b/src/Common/CaresPTRResolver.cpp @@ -0,0 +1,95 @@ +#include "CaresPTRResolver.h" +#include +#include +#include "ares.h" +#include "netdb.h" + +namespace DB { + + namespace ErrorCodes + { + extern const int DNS_ERROR; + } + + static void callback(void * arg, int status, int, struct hostent * host) { + auto * ptr_records = reinterpret_cast*>(arg); + if (status == ARES_SUCCESS && host->h_aliases) { + int i = 0; + while (auto * ptr_record = host->h_aliases[i]) { + ptr_records->emplace_back(ptr_record); + i++; + } + } + } + + CaresPTRResolver::CaresPTRResolver(CaresPTRResolver::provider_token) : channel(nullptr) { + /* + * ares_library_init is not thread safe. Currently, the only other usage of c-ares seems to be in grpc. + * In grpc, ares_library_init seems to be called only in Windows. + * See https://github.com/grpc/grpc/blob/master/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.cc#L1187 + * That means it's safe to init it here, but we should be cautious when introducing new code that depends on c-ares and even updates + * to grpc. As discussed in https://github.com/ClickHouse/ClickHouse/pull/37827#discussion_r919189085, c-ares should be adapted to be atomic + * */ + if (ares_library_init(ARES_LIB_INIT_ALL) != ARES_SUCCESS || ares_init(&channel) != ARES_SUCCESS) { + throw DB::Exception("Failed to initialize c-ares", DB::ErrorCodes::DNS_ERROR); + } + } + + CaresPTRResolver::~CaresPTRResolver() { + ares_destroy(channel); + ares_library_cleanup(); + } + + std::vector CaresPTRResolver::resolve(const std::string & ip) { + std::vector ptr_records; + + resolve(ip, ptr_records); + wait(); + + return ptr_records; + } + + std::vector CaresPTRResolver::resolve_v6(const std::string & ip) + { + std::vector ptr_records; + + resolve_v6(ip, ptr_records); + wait(); + + return ptr_records; + } + + void CaresPTRResolver::resolve(const std::string & ip, std::vector & response) { + in_addr addr; + + inet_pton(AF_INET, ip.c_str(), &addr); + + ares_gethostbyaddr(channel, reinterpret_cast(&addr), sizeof(addr), AF_INET, callback, &response); + } + + void CaresPTRResolver::resolve_v6(const std::string & ip, std::vector & response) { + in6_addr addr; + inet_pton(AF_INET6, ip.c_str(), &addr); + + ares_gethostbyaddr(channel, reinterpret_cast(&addr), sizeof(addr), AF_INET6, callback, &response); + } + + void CaresPTRResolver::wait() { + for(;;) { + timeval * tvp, tv; + fd_set read_fds; + fd_set write_fds; + int nfds; + + FD_ZERO(&read_fds); + FD_ZERO(&write_fds); + nfds = ares_fds(channel, &read_fds, &write_fds); + if(nfds == 0) { + break; + } + tvp = ares_timeout(channel, nullptr, &tv); + select(nfds, &read_fds, &write_fds, nullptr, tvp); + ares_process(channel, &read_fds, &write_fds); + } + } +} diff --git a/src/Common/CaresPTRResolver.h b/src/Common/CaresPTRResolver.h new file mode 100644 index 00000000000..23c529f4947 --- /dev/null +++ b/src/Common/CaresPTRResolver.h @@ -0,0 +1,40 @@ +#pragma once + +#include "DNSPTRResolver.h" + +using ares_channel = struct ares_channeldata *; + +namespace DB { + + /* + * Implements reverse DNS resolution using c-ares lib. System reverse DNS resolution via + * gethostbyaddr or getnameinfo does not work reliably as in some systems + * it returns all PTR records for a given IP and in others it returns only one. + * */ + class CaresPTRResolver : public DNSPTRResolver { + friend class DNSPTRResolverProvider; + + /* + * Allow only DNSPTRProvider to instantiate this class + * */ + struct provider_token {}; + + public: + CaresPTRResolver(provider_token); + ~CaresPTRResolver() override; + + std::vector resolve(const std::string & ip) override; + + std::vector resolve_v6(const std::string & ip) override; + + private: + void wait(); + + void resolve(const std::string & ip, std::vector & response); + + void resolve_v6(const std::string & ip, std::vector & response); + + ares_channel channel; + }; +} + diff --git a/src/Common/DNSPTRResolver.h b/src/Common/DNSPTRResolver.h index 20a5422767a..7cb78c15567 100644 --- a/src/Common/DNSPTRResolver.h +++ b/src/Common/DNSPTRResolver.h @@ -10,5 +10,7 @@ namespace DB { virtual std::vector resolve(const std::string & ip) = 0; + virtual std::vector resolve_v6(const std::string & ip) = 0; + }; } diff --git a/src/Common/DNSPTRResolverProvider.cpp b/src/Common/DNSPTRResolverProvider.cpp index ef19024495f..faf1980977c 100644 --- a/src/Common/DNSPTRResolverProvider.cpp +++ b/src/Common/DNSPTRResolverProvider.cpp @@ -1,12 +1,12 @@ #include "DNSPTRResolverProvider.h" -#include "CARESPTRResolver.h" +#include "CaresPTRResolver.h" namespace DB { - std::shared_ptr DNSPTRResolverProvider::get() { - static auto cares_resolver = std::make_shared(); + static auto cares_resolver = std::make_shared( + CaresPTRResolver::provider_token {} + ); return cares_resolver; } - } diff --git a/src/Common/DNSPTRResolverProvider.h b/src/Common/DNSPTRResolverProvider.h index 17764d8c91d..b4fa8ab77a4 100644 --- a/src/Common/DNSPTRResolverProvider.h +++ b/src/Common/DNSPTRResolverProvider.h @@ -10,4 +10,3 @@ namespace DB { static std::shared_ptr get(); }; } - diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 4c32dbf3341..d8967e9661f 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -12,7 +12,6 @@ #include #include #include -#include #include "DNSPTRResolverProvider.h" namespace ProfileEvents @@ -143,7 +142,12 @@ static DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host) static std::vector reverseResolveImpl(const Poco::Net::IPAddress & address) { auto ptr_resolver = DB::DNSPTRResolverProvider::get(); - return ptr_resolver->resolve(address.toString()); + + if (address.family() == Poco::Net::IPAddress::Family::IPv4) { + return ptr_resolver->resolve(address.toString()); + } else { + return ptr_resolver->resolve_v6(address.toString()); + } } struct DNSResolver::Impl From 397d05eb9c1e3e5630221ae82938765864bc4c58 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 12 Jul 2022 14:31:33 -0300 Subject: [PATCH 148/659] Add DNSPTRResolverProvider docs --- src/Common/DNSPTRResolverProvider.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Common/DNSPTRResolverProvider.h b/src/Common/DNSPTRResolverProvider.h index b4fa8ab77a4..5b079c5e376 100644 --- a/src/Common/DNSPTRResolverProvider.h +++ b/src/Common/DNSPTRResolverProvider.h @@ -4,6 +4,11 @@ #include "DNSPTRResolver.h" namespace DB { + /* + * Provides a ready-to-use DNSPTRResolver instance. + * It hides 3rd party lib dependencies, handles initialization and lifetime. + * Since `get` function is static, it can be called from any context. Including cached static functions. + * */ class DNSPTRResolverProvider { public: From b699764bbe93c5c2a2d7568080388f35780fa7ca Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 12 Jul 2022 14:32:53 -0400 Subject: [PATCH 149/659] unified sync implemetation with async --- .../CompressionCodecDeflateQpl.cpp | 40 ++++--------------- src/Compression/CompressionCodecDeflateQpl.h | 1 - 2 files changed, 8 insertions(+), 33 deletions(-) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index b7a5c89b121..343ca430834 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -167,36 +167,6 @@ int32_t HardwareCodecDeflateQpl::doCompressData(const char * source, uint32_t so return compressed_size; } -int32_t HardwareCodecDeflateQpl::doDecompressDataSynchronous(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const -{ - uint32_t job_id = 0; - qpl_job* job_ptr = nullptr; - if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(&job_id))) - { - LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataSynchronous->acquireJob fail, probably job pool exhausted)"); - return RET_ERROR; - } - // Performing a decompression operation - job_ptr->op = qpl_op_decompress; - job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); - job_ptr->next_out_ptr = reinterpret_cast(dest); - job_ptr->available_in = source_size; - job_ptr->available_out = uncompressed_size; - job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; - - if (auto status = qpl_execute_job(job_ptr); status == QPL_STS_OK) - { - DeflateQplJobHWPool::instance().releaseJob(job_id); - return job_ptr->total_out; - } - else - { - LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataSynchronous->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); - DeflateQplJobHWPool::instance().releaseJob(job_id); - return RET_ERROR; - } -} - int32_t HardwareCodecDeflateQpl::doDecompressDataAsynchronous(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) { uint32_t job_id = 0; @@ -362,8 +332,14 @@ void CompressionCodecDeflateQpl::doDecompressData(const char * source, uint32_t { int32_t res = HardwareCodecDeflateQpl::RET_ERROR; if (DeflateQplJobHWPool::instance().isJobPoolReady()) - res = hw_codec->doDecompressDataSynchronous(source, source_size, dest, uncompressed_size); - if (res == HardwareCodecDeflateQpl::RET_ERROR) + { + res = hw_codec->doDecompressDataAsynchronous(source, source_size, dest, uncompressed_size); + if(res != HardwareCodecDeflateQpl::RET_ERROR) + hw_codec->flushAsynchronousDecompressRequests(); + else + sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); + } + else sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); return; } diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 10d55600f89..37e1cff650e 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -77,7 +77,6 @@ public: HardwareCodecDeflateQpl(); ~HardwareCodecDeflateQpl(); int32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const; - int32_t doDecompressDataSynchronous(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const; int32_t doDecompressDataAsynchronous(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); /// Flush result for previous asynchronous decompression requests.Must be used following with several calls of doDecompressDataReq. void flushAsynchronousDecompressRequests(); From 381d60c5109f8b7bbd155052a1f3cc7176f6d576 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 12 Jul 2022 17:55:49 -0300 Subject: [PATCH 150/659] minor adjustments --- src/Common/CaresPTRResolver.cpp | 34 ++++++++++++++++++++------------- 1 file changed, 21 insertions(+), 13 deletions(-) diff --git a/src/Common/CaresPTRResolver.cpp b/src/Common/CaresPTRResolver.cpp index 6160c2fb3a7..b21dec77b4c 100644 --- a/src/Common/CaresPTRResolver.cpp +++ b/src/Common/CaresPTRResolver.cpp @@ -11,7 +11,8 @@ namespace DB { extern const int DNS_ERROR; } - static void callback(void * arg, int status, int, struct hostent * host) { + static void callback(void * arg, int status, int, struct hostent * host) + { auto * ptr_records = reinterpret_cast*>(arg); if (status == ARES_SUCCESS && host->h_aliases) { int i = 0; @@ -22,7 +23,8 @@ namespace DB { } } - CaresPTRResolver::CaresPTRResolver(CaresPTRResolver::provider_token) : channel(nullptr) { + CaresPTRResolver::CaresPTRResolver(CaresPTRResolver::provider_token) : channel(nullptr) + { /* * ares_library_init is not thread safe. Currently, the only other usage of c-ares seems to be in grpc. * In grpc, ares_library_init seems to be called only in Windows. @@ -35,12 +37,14 @@ namespace DB { } } - CaresPTRResolver::~CaresPTRResolver() { + CaresPTRResolver::~CaresPTRResolver() + { ares_destroy(channel); ares_library_cleanup(); } - std::vector CaresPTRResolver::resolve(const std::string & ip) { + std::vector CaresPTRResolver::resolve(const std::string & ip) + { std::vector ptr_records; resolve(ip, ptr_records); @@ -59,7 +63,8 @@ namespace DB { return ptr_records; } - void CaresPTRResolver::resolve(const std::string & ip, std::vector & response) { + void CaresPTRResolver::resolve(const std::string & ip, std::vector & response) + { in_addr addr; inet_pton(AF_INET, ip.c_str(), &addr); @@ -67,23 +72,26 @@ namespace DB { ares_gethostbyaddr(channel, reinterpret_cast(&addr), sizeof(addr), AF_INET, callback, &response); } - void CaresPTRResolver::resolve_v6(const std::string & ip, std::vector & response) { + void CaresPTRResolver::resolve_v6(const std::string & ip, std::vector & response) + { in6_addr addr; inet_pton(AF_INET6, ip.c_str(), &addr); ares_gethostbyaddr(channel, reinterpret_cast(&addr), sizeof(addr), AF_INET6, callback, &response); } - void CaresPTRResolver::wait() { - for(;;) { - timeval * tvp, tv; - fd_set read_fds; - fd_set write_fds; - int nfds; + void CaresPTRResolver::wait() + { + timeval * tvp, tv; + fd_set read_fds; + fd_set write_fds; + int nfds; + for(;;) + { FD_ZERO(&read_fds); FD_ZERO(&write_fds); - nfds = ares_fds(channel, &read_fds, &write_fds); + nfds = ares_fds(channel, &read_fds,&write_fds); if(nfds == 0) { break; } From cb349c05c533704a701726afd13fdb2595133224 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 12 Jul 2022 19:27:44 -0300 Subject: [PATCH 151/659] minor adjustments --- src/Access/Common/AllowedClientHosts.cpp | 2 +- src/Common/CaresPTRResolver.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Access/Common/AllowedClientHosts.cpp b/src/Access/Common/AllowedClientHosts.cpp index e32b218c948..6f5e0392bab 100644 --- a/src/Access/Common/AllowedClientHosts.cpp +++ b/src/Access/Common/AllowedClientHosts.cpp @@ -525,7 +525,7 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const return true; /// Check `name_regexps`. - std::optional> resolved_hosts; + std::optional resolved_hosts; auto check_name_regexp = [&](const String & name_regexp_) { try diff --git a/src/Common/CaresPTRResolver.h b/src/Common/CaresPTRResolver.h index 23c529f4947..9aaed37d514 100644 --- a/src/Common/CaresPTRResolver.h +++ b/src/Common/CaresPTRResolver.h @@ -8,7 +8,7 @@ namespace DB { /* * Implements reverse DNS resolution using c-ares lib. System reverse DNS resolution via - * gethostbyaddr or getnameinfo does not work reliably as in some systems + * gethostbyaddr or getnameinfo does not work reliably because in some systems * it returns all PTR records for a given IP and in others it returns only one. * */ class CaresPTRResolver : public DNSPTRResolver { From d306c1b6f44e5607911ec4b1c4e8e7fa2b8fddab Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 12 Jul 2022 20:37:39 -0300 Subject: [PATCH 152/659] Use Strings alias instead of std::vec> --- src/Common/DNSResolver.cpp | 4 ++-- src/Common/DNSResolver.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index d8967e9661f..a6b096879e9 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -139,7 +139,7 @@ static DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host) return addresses; } -static std::vector reverseResolveImpl(const Poco::Net::IPAddress & address) +static Strings reverseResolveImpl(const Poco::Net::IPAddress & address) { auto ptr_resolver = DB::DNSPTRResolverProvider::get(); @@ -235,7 +235,7 @@ std::vector DNSResolver::resolveAddressList(const std: return addresses; } -std::vector DNSResolver::reverseResolve(const Poco::Net::IPAddress & address) +Strings DNSResolver::reverseResolve(const Poco::Net::IPAddress & address) { if (impl->disable_cache) return reverseResolveImpl(address); diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index d1a8843d234..84c88586636 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -37,7 +37,7 @@ public: std::vector resolveAddressList(const std::string & host, UInt16 port); /// Accepts host IP and resolves its host names - std::vector reverseResolve(const Poco::Net::IPAddress & address); + Strings reverseResolve(const Poco::Net::IPAddress & address); /// Get this server host name String getHostName(); From 1f57f92bf775601436b74d1817ccf67d5c8fe0c2 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 13 Jul 2022 03:01:23 -0400 Subject: [PATCH 153/659] prewhere action should preserve sorting columns --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index b79dc9c80e9..e75fdcd6e3f 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1851,7 +1851,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( { ExpressionActionsChain chain(context); - Names additional_required_columns_after_prewhere; + Names additional_required_columns_after_prewhere = metadata_snapshot->getColumnsRequiredForSortingKey(); if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1)) { From f275b35bba2b72187e3767a6ce9b155e41ef40f5 Mon Sep 17 00:00:00 2001 From: Wangyang Guo Date: Wed, 13 Jul 2022 15:15:54 +0800 Subject: [PATCH 154/659] Dynamic dispatching for best_variant selection --- src/Compression/LZ4_decompress_faster.cpp | 12 ++++++++++-- src/Compression/LZ4_decompress_faster.h | 8 ++++---- 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 4b5eac71428..84471836cfc 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -454,7 +454,9 @@ inline void copyOverlap32Shuffle(UInt8 * op, const UInt8 *& match, const size_t _mm256_load_si256(reinterpret_cast(masks) + offset), _mm256_loadu_si256(reinterpret_cast(match)))); match += masks[offset]; - } else { + } + else + { copyOverlap32(op, match, offset); } #else @@ -638,7 +640,13 @@ bool decompress( /// Don't run timer if the block is too small. if (dest_size >= 32768) { - size_t best_variant = statistics.select(); + size_t variant_size = 4; +#if defined(__AVX512VBMI__) && !defined(MEMORY_SANITIZER) + /// best_variant == 4 only valid when AVX512VBMI available + if (DB::Cpu::CpuFlagsCache::have_AVX512VBMI) + variant_size = 5; +#endif + size_t best_variant = statistics.select(variant_size); /// Run the selected method and measure time. diff --git a/src/Compression/LZ4_decompress_faster.h b/src/Compression/LZ4_decompress_faster.h index 7c6224e939f..08744755916 100644 --- a/src/Compression/LZ4_decompress_faster.h +++ b/src/Compression/LZ4_decompress_faster.h @@ -106,17 +106,17 @@ struct PerformanceStatistics /// To select from different algorithms we use a kind of "bandits" algorithm. /// Sample random values from estimated normal distributions and choose the minimal. - size_t select() + size_t select(size_t max_method = NUM_ELEMENTS) { if (choose_method < 0) { - double samples[NUM_ELEMENTS]; - for (size_t i = 0; i < NUM_ELEMENTS; ++i) + double samples[max_method]; + for (size_t i = 0; i < max_method; ++i) samples[i] = choose_method == -1 ? data[i].sample(rng) : data[i].adjustedCount(); - return std::min_element(samples, samples + NUM_ELEMENTS) - samples; + return std::min_element(samples, samples + max_method) - samples; } else return choose_method; From f8c38b0d3a41489ed360168a13fed33e2bfa6d84 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 12 Jul 2022 08:18:19 +0200 Subject: [PATCH 155/659] Remove obsolete doxygen (the removed functions are implemented elsewhere) --- src/Functions/FunctionsMultiStringPosition.h | 24 ++++---------------- 1 file changed, 4 insertions(+), 20 deletions(-) diff --git a/src/Functions/FunctionsMultiStringPosition.h b/src/Functions/FunctionsMultiStringPosition.h index 855b5448b87..d2a46561b4e 100644 --- a/src/Functions/FunctionsMultiStringPosition.h +++ b/src/Functions/FunctionsMultiStringPosition.h @@ -17,25 +17,11 @@ namespace DB { -/** multiSearchAllPositions(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find first occurrences (positions) of all the const patterns inside haystack +/** + * multiSearchAllPositions(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find first occurrences (positions) of all the const patterns inside haystack * multiSearchAllPositionsUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) * multiSearchAllPositionsCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) * multiSearchAllPositionsCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * - * multiSearchFirstPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first position of the haystack matched by strings or zero if nothing was found - * multiSearchFirstPositionUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchFirstPositionCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchFirstPositionCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * - * multiSearchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1 - * multiSearchAnyUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchAnyCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchAnyCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - - * multiSearchFirstIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found - * multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) */ namespace ErrorCodes @@ -51,10 +37,9 @@ class FunctionsMultiStringPosition : public IFunction { public: static constexpr auto name = Name::name; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } - String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 2; } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } @@ -76,8 +61,6 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { - using ResultType = typename Impl::ResultType; - const ColumnPtr & column_haystack = arguments[0].column; const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); @@ -105,6 +88,7 @@ public: const size_t column_haystack_size = column_haystack->size(); + using ResultType = typename Impl::ResultType; auto col_res = ColumnVector::create(); auto col_offsets = ColumnArray::ColumnOffsets::create(column_haystack_size); From 59a64ac902ec89e2a72384f9c216abab1b030e68 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 12 Jul 2022 08:23:03 +0200 Subject: [PATCH 156/659] Rename some variables (for consistency with Functions/FunctionsMultiStringSearch.h) --- src/Functions/FunctionsMultiStringPosition.h | 36 ++++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Functions/FunctionsMultiStringPosition.h b/src/Functions/FunctionsMultiStringPosition.h index d2a46561b4e..ee9b1a8573b 100644 --- a/src/Functions/FunctionsMultiStringPosition.h +++ b/src/Functions/FunctionsMultiStringPosition.h @@ -61,32 +61,32 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { - const ColumnPtr & column_haystack = arguments[0].column; + const ColumnPtr & haystack_ptr = arguments[0].column; + const ColumnPtr & needles_ptr = arguments[1].column; - const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); + const ColumnString * col_haystack_vector = checkAndGetColumn(&*haystack_ptr); - const ColumnPtr & arr_ptr = arguments[1].column; - const ColumnConst * col_const_arr = checkAndGetColumnConst(arr_ptr.get()); + const ColumnConst * col_needles_const = checkAndGetColumnConst(needles_ptr.get()); - if (!col_const_arr) + if (!col_needles_const) throw Exception( "Illegal column " + arguments[1].column->getName() + ". The array is not const", ErrorCodes::ILLEGAL_COLUMN); - Array src_arr = col_const_arr->getValue(); + Array needles_arr = col_needles_const->getValue(); - if (src_arr.size() > std::numeric_limits::max()) + if (needles_arr.size() > std::numeric_limits::max()) throw Exception( - "Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size()) + "Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(needles_arr.size()) + ", should be at most 255", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - std::vector refs; - refs.reserve(src_arr.size()); - for (const auto & el : src_arr) - refs.emplace_back(el.get()); + std::vector needles; + needles.reserve(needles_arr.size()); + for (const auto & el : needles_arr) + needles.emplace_back(el.get()); - const size_t column_haystack_size = column_haystack->size(); + const size_t column_haystack_size = haystack_ptr->size(); using ResultType = typename Impl::ResultType; auto col_res = ColumnVector::create(); @@ -95,17 +95,17 @@ public: auto & vec_res = col_res->getData(); auto & offsets_res = col_offsets->getData(); - vec_res.resize(column_haystack_size * refs.size()); + vec_res.resize(column_haystack_size * needles.size()); if (col_haystack_vector) - Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res); + Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needles, vec_res); else throw Exception("Illegal column " + arguments[0].column->getName(), ErrorCodes::ILLEGAL_COLUMN); - size_t refs_size = refs.size(); - size_t accum = refs_size; + size_t needles_size = needles.size(); + size_t accum = needles_size; - for (size_t i = 0; i < column_haystack_size; ++i, accum += refs_size) + for (size_t i = 0; i < column_haystack_size; ++i, accum += needles_size) offsets_res[i] = accum; return ColumnArray::create(std::move(col_res), std::move(col_offsets)); From 04c1cb207bb426ab3b4a13615f27cc40bfc5ec4a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 12 Jul 2022 08:27:08 +0200 Subject: [PATCH 157/659] Use fmt-based exceptions in FunctionsMultiStringPosition (for consistency with Functions/FunctionsMultiStringSearch.h) --- src/Functions/FunctionsMultiStringPosition.h | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Functions/FunctionsMultiStringPosition.h b/src/Functions/FunctionsMultiStringPosition.h index ee9b1a8573b..488d8279e5d 100644 --- a/src/Functions/FunctionsMultiStringPosition.h +++ b/src/Functions/FunctionsMultiStringPosition.h @@ -49,12 +49,14 @@ public: { if (!isString(arguments[0])) throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", arguments[0]->getName(), getName()); const DataTypeArray * array_type = checkAndGetDataType(arguments[1].get()); if (!array_type || !checkAndGetDataType(array_type->getNestedType().get())) throw Exception( - "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", arguments[1]->getName(), getName()); return std::make_shared(std::make_shared()); } @@ -70,16 +72,15 @@ public: if (!col_needles_const) throw Exception( - "Illegal column " + arguments[1].column->getName() + ". The array is not const", - ErrorCodes::ILLEGAL_COLUMN); + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {}. The array is not const", arguments[1].column->getName()); Array needles_arr = col_needles_const->getValue(); if (needles_arr.size() > std::numeric_limits::max()) throw Exception( - "Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(needles_arr.size()) - + ", should be at most 255", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be at most 255", getName(), needles_arr.size()); std::vector needles; needles.reserve(needles_arr.size()); @@ -100,7 +101,9 @@ public: if (col_haystack_vector) Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needles, vec_res); else - throw Exception("Illegal column " + arguments[0].column->getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {}", arguments[0].column->getName()); size_t needles_size = needles.size(); size_t accum = needles_size; From 395dada9885016e75c1cc6b0fe1510788836515e Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 13 Jul 2022 09:40:56 -0300 Subject: [PATCH 158/659] minor style adjustments --- src/Access/Common/AllowedClientHosts.cpp | 9 ++++++--- src/Common/CaresPTRResolver.cpp | 15 ++++++++++----- src/Common/CaresPTRResolver.h | 8 +++++--- src/Common/DNSPTRResolver.h | 3 ++- src/Common/DNSResolver.cpp | 6 ++++-- 5 files changed, 27 insertions(+), 14 deletions(-) diff --git a/src/Access/Common/AllowedClientHosts.cpp b/src/Access/Common/AllowedClientHosts.cpp index 6f5e0392bab..efbdf3924e8 100644 --- a/src/Access/Common/AllowedClientHosts.cpp +++ b/src/Access/Common/AllowedClientHosts.cpp @@ -118,7 +118,8 @@ namespace throw Exception(ErrorCodes::DNS_ERROR, "{} could not be resolved", address.toString()); - for (const auto & host : hosts) { + for (const auto & host : hosts) + { /// Check that PTR record is resolved back to client address if (!isAddressOfHost(address, host)) throw Exception(ErrorCodes::DNS_ERROR, "Host {} isn't resolved back to {}", host, address.toString()); @@ -537,10 +538,12 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const resolved_hosts = getHostsByAddress(client_address); } - for (const auto & host : resolved_hosts.value()) { + for (const auto & host : resolved_hosts.value()) + { Poco::RegularExpression re(name_regexp_); Poco::RegularExpression::Match match; - if (re.match(host, match) != 0) { + if (re.match(host, match) != 0) + { return true; } } diff --git a/src/Common/CaresPTRResolver.cpp b/src/Common/CaresPTRResolver.cpp index b21dec77b4c..2915b200433 100644 --- a/src/Common/CaresPTRResolver.cpp +++ b/src/Common/CaresPTRResolver.cpp @@ -4,7 +4,8 @@ #include "ares.h" #include "netdb.h" -namespace DB { +namespace DB +{ namespace ErrorCodes { @@ -14,9 +15,11 @@ namespace DB { static void callback(void * arg, int status, int, struct hostent * host) { auto * ptr_records = reinterpret_cast*>(arg); - if (status == ARES_SUCCESS && host->h_aliases) { + if (status == ARES_SUCCESS && host->h_aliases) + { int i = 0; - while (auto * ptr_record = host->h_aliases[i]) { + while (auto * ptr_record = host->h_aliases[i]) + { ptr_records->emplace_back(ptr_record); i++; } @@ -32,7 +35,8 @@ namespace DB { * That means it's safe to init it here, but we should be cautious when introducing new code that depends on c-ares and even updates * to grpc. As discussed in https://github.com/ClickHouse/ClickHouse/pull/37827#discussion_r919189085, c-ares should be adapted to be atomic * */ - if (ares_library_init(ARES_LIB_INIT_ALL) != ARES_SUCCESS || ares_init(&channel) != ARES_SUCCESS) { + if (ares_library_init(ARES_LIB_INIT_ALL) != ARES_SUCCESS || ares_init(&channel) != ARES_SUCCESS) + { throw DB::Exception("Failed to initialize c-ares", DB::ErrorCodes::DNS_ERROR); } } @@ -92,7 +96,8 @@ namespace DB { FD_ZERO(&read_fds); FD_ZERO(&write_fds); nfds = ares_fds(channel, &read_fds,&write_fds); - if(nfds == 0) { + if(nfds == 0) + { break; } tvp = ares_timeout(channel, nullptr, &tv); diff --git a/src/Common/CaresPTRResolver.h b/src/Common/CaresPTRResolver.h index 9aaed37d514..fd6a1cf7bc5 100644 --- a/src/Common/CaresPTRResolver.h +++ b/src/Common/CaresPTRResolver.h @@ -4,14 +4,16 @@ using ares_channel = struct ares_channeldata *; -namespace DB { +namespace DB +{ /* * Implements reverse DNS resolution using c-ares lib. System reverse DNS resolution via * gethostbyaddr or getnameinfo does not work reliably because in some systems * it returns all PTR records for a given IP and in others it returns only one. * */ - class CaresPTRResolver : public DNSPTRResolver { + class CaresPTRResolver : public DNSPTRResolver + { friend class DNSPTRResolverProvider; /* @@ -20,7 +22,7 @@ namespace DB { struct provider_token {}; public: - CaresPTRResolver(provider_token); + explicit CaresPTRResolver(provider_token); ~CaresPTRResolver() override; std::vector resolve(const std::string & ip) override; diff --git a/src/Common/DNSPTRResolver.h b/src/Common/DNSPTRResolver.h index 7cb78c15567..8266a35b128 100644 --- a/src/Common/DNSPTRResolver.h +++ b/src/Common/DNSPTRResolver.h @@ -4,7 +4,8 @@ #include namespace DB { - struct DNSPTRResolver { + struct DNSPTRResolver + { virtual ~DNSPTRResolver() = default; diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index a6b096879e9..10797b7a809 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -143,9 +143,11 @@ static Strings reverseResolveImpl(const Poco::Net::IPAddress & address) { auto ptr_resolver = DB::DNSPTRResolverProvider::get(); - if (address.family() == Poco::Net::IPAddress::Family::IPv4) { + if (address.family() == Poco::Net::IPAddress::Family::IPv4) + { return ptr_resolver->resolve(address.toString()); - } else { + } else + { return ptr_resolver->resolve_v6(address.toString()); } } From 54a0985c4290ed5706722d9eb0d5b1c4212c9b2f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 13 Jul 2022 12:50:12 +0000 Subject: [PATCH 159/659] randomize one more setting --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 3 +++ tests/clickhouse-test | 1 + 2 files changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 85231aca253..d9fc8ccaf42 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1470,6 +1470,9 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( { // Do exclusion search, where we drop ranges that do not match + if (settings.merge_tree_coarse_index_granularity <= 1) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Setting merge_tree_coarse_index_granularity should be greater than 1"); + size_t min_marks_for_seek = roundRowsOrBytesToMarks( settings.merge_tree_min_rows_for_seek, settings.merge_tree_min_bytes_for_seek, diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 4ced036683b..a28af944868 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -446,6 +446,7 @@ class SettingsRandomizer: "compile_expressions": lambda: random.randint(0, 1), "compile_aggregate_expressions": lambda: random.randint(0, 1), "compile_sort_description": lambda: random.randint(0, 1), + "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), } @staticmethod From 3fd367ea580c51a293ce659779b023433010158c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 13 Jul 2022 10:08:24 -0400 Subject: [PATCH 160/659] check metadata_snapshot is not empty --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index e75fdcd6e3f..6460ed536e9 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1851,7 +1851,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( { ExpressionActionsChain chain(context); - Names additional_required_columns_after_prewhere = metadata_snapshot->getColumnsRequiredForSortingKey(); + Names additional_required_columns_after_prewhere = metadata_snapshot ? metadata_snapshot->getColumnsRequiredForSortingKey() : Names{}; if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1)) { From 3720a3843c30e77e9b1e787aa30e32a3e5b8b29c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Jul 2022 17:03:39 +0300 Subject: [PATCH 161/659] Add merge_algorithm to system.part_log Signed-off-by: Azat Khuzhin --- src/Interpreters/PartLog.cpp | 38 ++++++++++++++++--- src/Interpreters/PartLog.h | 12 ++++++ src/Storages/MergeTree/MergeTreeData.cpp | 5 +++ .../02362_part_log_merge_algorithm.reference | 5 +++ .../02362_part_log_merge_algorithm.sql | 26 +++++++++++++ 5 files changed, 80 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02362_part_log_merge_algorithm.reference create mode 100644 tests/queries/0_stateless/02362_part_log_merge_algorithm.sql diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index f79be6a67e0..c3152f31808 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -25,17 +25,32 @@ PartLogElement::MergeReasonType PartLogElement::getMergeReasonType(MergeType mer { switch (merge_type) { - case MergeType::Regular: - return REGULAR_MERGE; - case MergeType::TTLDelete: - return TTL_DELETE_MERGE; - case MergeType::TTLRecompress: - return TTL_RECOMPRESS_MERGE; + case MergeType::Regular: + return REGULAR_MERGE; + case MergeType::TTLDelete: + return TTL_DELETE_MERGE; + case MergeType::TTLRecompress: + return TTL_RECOMPRESS_MERGE; } throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeType {}", static_cast(merge_type)); } +PartLogElement::PartMergeAlgorithm PartLogElement::getMergeAlgorithm(MergeAlgorithm merge_algorithm_) +{ + switch (merge_algorithm_) + { + case MergeAlgorithm::Undecided: + return UNDECIDED; + case MergeAlgorithm::Horizontal: + return HORIZONTAL; + case MergeAlgorithm::Vertical: + return VERTICAL; + } + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeAlgorithm {}", static_cast(merge_algorithm_)); +} + NamesAndTypesList PartLogElement::getNamesAndTypes() { auto event_type_datatype = std::make_shared( @@ -60,12 +75,22 @@ NamesAndTypesList PartLogElement::getNamesAndTypes() } ); + auto merge_algorithm_datatype = std::make_shared( + DataTypeEnum8::Values + { + {"Undecided", static_cast(UNDECIDED)}, + {"Horizontal", static_cast(HORIZONTAL)}, + {"Vertical", static_cast(VERTICAL)}, + } + ); + ColumnsWithTypeAndName columns_with_type_and_name; return { {"query_id", std::make_shared()}, {"event_type", std::move(event_type_datatype)}, {"merge_reason", std::move(merge_reason_datatype)}, + {"merge_algorithm", std::move(merge_algorithm_datatype)}, {"event_date", std::make_shared()}, {"event_time", std::make_shared()}, @@ -104,6 +129,7 @@ void PartLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(query_id); columns[i++]->insert(event_type); columns[i++]->insert(merge_reason); + columns[i++]->insert(merge_algorithm); columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType()); columns[i++]->insert(event_time); columns[i++]->insert(event_time_microseconds); diff --git a/src/Interpreters/PartLog.h b/src/Interpreters/PartLog.h index 16a7e37ee9d..2ce0dfd76de 100644 --- a/src/Interpreters/PartLog.h +++ b/src/Interpreters/PartLog.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -22,6 +23,14 @@ struct PartLogElement MOVE_PART = 6, }; + /// Copy of MergeAlgorithm since values are written to disk. + enum PartMergeAlgorithm + { + UNDECIDED = 0, + VERTICAL = 1, + HORIZONTAL = 2, + }; + enum MergeReasonType { /// merge_reason is relevant only for event_type = 'MERGE_PARTS', in other cases it is NOT_A_MERGE @@ -38,6 +47,7 @@ struct PartLogElement Type event_type = NEW_PART; MergeReasonType merge_reason = NOT_A_MERGE; + PartMergeAlgorithm merge_algorithm = UNDECIDED; time_t event_time = 0; Decimal64 event_time_microseconds = 0; @@ -72,6 +82,8 @@ struct PartLogElement static std::string name() { return "PartLog"; } static MergeReasonType getMergeReasonType(MergeType merge_type); + static PartMergeAlgorithm getMergeAlgorithm(MergeAlgorithm merge_algorithm_); + static NamesAndTypesList getNamesAndTypes(); static NamesAndAliases getNamesAndAliases() { return {}; } void appendToBlock(MutableColumns & columns) const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 5900ea0fdb7..64aaa40bd4c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6229,8 +6229,13 @@ try part_log_elem.event_type = type; if (part_log_elem.event_type == PartLogElement::MERGE_PARTS) + { if (merge_entry) + { part_log_elem.merge_reason = PartLogElement::getMergeReasonType((*merge_entry)->merge_type); + part_log_elem.merge_algorithm = PartLogElement::getMergeAlgorithm((*merge_entry)->merge_algorithm); + } + } part_log_elem.error = static_cast(execution_status.code); part_log_elem.exception = execution_status.message; diff --git a/tests/queries/0_stateless/02362_part_log_merge_algorithm.reference b/tests/queries/0_stateless/02362_part_log_merge_algorithm.reference new file mode 100644 index 00000000000..91a959d4255 --- /dev/null +++ b/tests/queries/0_stateless/02362_part_log_merge_algorithm.reference @@ -0,0 +1,5 @@ +data_horizontal all_1_1_0 NewPart Undecided +data_horizontal all_1_1_1 MergeParts Horizontal +data_vertical all_1_1_0 NewPart Undecided +data_vertical all_2_2_0 NewPart Undecided +data_vertical all_1_2_1 MergeParts Vertical diff --git a/tests/queries/0_stateless/02362_part_log_merge_algorithm.sql b/tests/queries/0_stateless/02362_part_log_merge_algorithm.sql new file mode 100644 index 00000000000..6446b46c393 --- /dev/null +++ b/tests/queries/0_stateless/02362_part_log_merge_algorithm.sql @@ -0,0 +1,26 @@ +CREATE TABLE data_horizontal ( + key Int +) +Engine=MergeTree() +ORDER BY key; + +INSERT INTO data_horizontal VALUES (1); +OPTIMIZE TABLE data_horizontal FINAL; +SYSTEM FLUSH LOGS; +SELECT table, part_name, event_type, merge_algorithm FROM system.part_log WHERE event_date >= yesterday() AND database = currentDatabase() AND table = 'data_horizontal' ORDER BY event_time_microseconds; + +CREATE TABLE data_vertical +( + key UInt64, + value String +) +ENGINE = MergeTree() +ORDER BY key +SETTINGS index_granularity_bytes = 0, enable_mixed_granularity_parts = 0, min_bytes_for_wide_part = 0, +vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 1; + +INSERT INTO data_vertical VALUES (1, '1'); +INSERT INTO data_vertical VALUES (2, '2'); +OPTIMIZE TABLE data_vertical FINAL; +SYSTEM FLUSH LOGS; +SELECT table, part_name, event_type, merge_algorithm FROM system.part_log WHERE event_date >= yesterday() AND database = currentDatabase() AND table = 'data_vertical' ORDER BY event_time_microseconds; From ad6b50b087086fef8aa6f0f72b3a42f014266763 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Jul 2022 17:34:20 +0300 Subject: [PATCH 162/659] Forbid defining non-default disk with default path from Suppose you have the following configuration: /var/lib/clickhouse/ /var/lib/clickhouse/

data
In this case disks will have two disks: - 'data' disk with path '/var/lib/clickhouse/' - 'default' disk with path '/var/lib/clickhouse/' And in this case MergeTree engine will complain on ATTACH for table that uses 'default' policy: 2022.06.20 07:49:15.165393 [ 242 ] {e8f50978-218a-426f-babc-637a8d03b1c6} TCPHandler: Code: 479. DB::Exception: Part `0_0_0_1` was found on disk `default` which is not defined in the storage policy. (UNKNOWN_DISK), Stack trace (when copying this message, always include the lines below): Signed-off-by: Azat Khuzhin --- src/Disks/DiskLocal.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 31deac88a19..e793f4dfb5a 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -68,6 +68,8 @@ static void loadDiskLocalConfig(const String & name, throw Exception("Disk path can not be empty. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); if (path.back() != '/') throw Exception("Disk path must end with /. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + if (path == context->getPath()) + throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Disk path ('{}') cannot be equal to . Use disk instead.", path); } bool has_space_ratio = config.has(config_prefix + ".keep_free_space_ratio"); From 7f4043a3ed631d17213e92f832622bf5587696ed Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 13 Jul 2022 11:51:43 -0300 Subject: [PATCH 163/659] Force use of c-ares inet_net_pton and style adjustments --- contrib/c-ares-cmake/CMakeLists.txt | 9 ++++----- src/Common/DNSPTRResolver.h | 3 ++- src/Common/DNSPTRResolverProvider.cpp | 3 ++- src/Common/DNSPTRResolverProvider.h | 3 ++- 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/contrib/c-ares-cmake/CMakeLists.txt b/contrib/c-ares-cmake/CMakeLists.txt index e534b42b963..603c1f8b65c 100644 --- a/contrib/c-ares-cmake/CMakeLists.txt +++ b/contrib/c-ares-cmake/CMakeLists.txt @@ -27,10 +27,9 @@ if (NOT CMAKE_SYSTEM_NAME STREQUAL "SunOS") set(HAVE_LIBNSL OFF CACHE BOOL "" FORCE) endif() +# Force use of c-ares inet_net_pton instead of libresolv one +set(HAVE_INET_NET_PTON OFF CACHE BOOL "" FORCE) + add_subdirectory("../c-ares/" "../c-ares/") -add_library(ch_contrib::c-ares ALIAS c-ares) - -find_library(LIBRESOLV_LIBRARY NAMES resolv libresolv REQUIRED) - -target_link_libraries(c-ares PRIVATE ${LIBRESOLV_LIBRARY}) \ No newline at end of file +add_library(ch_contrib::c-ares ALIAS c-ares) \ No newline at end of file diff --git a/src/Common/DNSPTRResolver.h b/src/Common/DNSPTRResolver.h index 8266a35b128..e6cce83f79d 100644 --- a/src/Common/DNSPTRResolver.h +++ b/src/Common/DNSPTRResolver.h @@ -3,7 +3,8 @@ #include #include -namespace DB { +namespace DB +{ struct DNSPTRResolver { diff --git a/src/Common/DNSPTRResolverProvider.cpp b/src/Common/DNSPTRResolverProvider.cpp index faf1980977c..41c73f4f36f 100644 --- a/src/Common/DNSPTRResolverProvider.cpp +++ b/src/Common/DNSPTRResolverProvider.cpp @@ -1,7 +1,8 @@ #include "DNSPTRResolverProvider.h" #include "CaresPTRResolver.h" -namespace DB { +namespace DB +{ std::shared_ptr DNSPTRResolverProvider::get() { static auto cares_resolver = std::make_shared( diff --git a/src/Common/DNSPTRResolverProvider.h b/src/Common/DNSPTRResolverProvider.h index 5b079c5e376..a7f534749e3 100644 --- a/src/Common/DNSPTRResolverProvider.h +++ b/src/Common/DNSPTRResolverProvider.h @@ -3,7 +3,8 @@ #include #include "DNSPTRResolver.h" -namespace DB { +namespace DB +{ /* * Provides a ready-to-use DNSPTRResolver instance. * It hides 3rd party lib dependencies, handles initialization and lifetime. From da2413f7d2f03eec403692d5d22a27fceaaa3534 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 13 Jul 2022 15:10:37 +0000 Subject: [PATCH 164/659] fix reading of sparse columns with prefetch --- src/DataTypes/Serializations/SerializationSparse.cpp | 10 +++++++++- .../queries/0_stateless/01780_column_sparse_tuple.sql | 1 - 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 64db248c5fc..6fa40e460c5 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -263,6 +263,12 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( { auto * state_sparse = checkAndGetState(state); + if (auto cached_column = getFromSubstreamsCache(cache, settings.path)) + { + column = cached_column; + return; + } + if (!settings.continuous_reading) state_sparse->reset(); @@ -281,7 +287,8 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( size_t values_limit = offsets_data.size() - old_size; settings.path.back() = Substream::SparseElements; - nested->deserializeBinaryBulkWithMultipleStreams(values_column, values_limit, settings, state_sparse->nested, cache); + /// Do not use substream cache while reading values column, because ColumnSparse can be cached only in a whole. + nested->deserializeBinaryBulkWithMultipleStreams(values_column, values_limit, settings, state_sparse->nested, nullptr); settings.path.pop_back(); if (offsets_data.size() + 1 != values_column->size()) @@ -291,6 +298,7 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( /// 'insertManyDefaults' just increases size of column. column_sparse.insertManyDefaults(read_rows); column = std::move(mutable_column); + addToSubstreamsCache(cache, settings.path, column); } /// All methods below just wrap nested serialization. diff --git a/tests/queries/0_stateless/01780_column_sparse_tuple.sql b/tests/queries/0_stateless/01780_column_sparse_tuple.sql index e3dfc16fc74..da679f2c7eb 100644 --- a/tests/queries/0_stateless/01780_column_sparse_tuple.sql +++ b/tests/queries/0_stateless/01780_column_sparse_tuple.sql @@ -1,4 +1,3 @@ --- Tags: no-s3-storage DROP TABLE IF EXISTS sparse_tuple; CREATE TABLE sparse_tuple (id UInt64, t Tuple(a UInt64, s String)) From 14aa62f66137e881f083ca55c6b6cd27a6c44a79 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 13 Jul 2022 16:14:51 +0000 Subject: [PATCH 165/659] Fix test output --- tests/queries/0_stateless/01961_roaring_memory_tracking.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01961_roaring_memory_tracking.sql b/tests/queries/0_stateless/01961_roaring_memory_tracking.sql index 64c31472e89..9e14bb9e138 100644 --- a/tests/queries/0_stateless/01961_roaring_memory_tracking.sql +++ b/tests/queries/0_stateless/01961_roaring_memory_tracking.sql @@ -1,4 +1,4 @@ -- Tags: no-replicated-database SET max_memory_usage = '100M'; -SELECT cityHash64(rand() % 1000) as n, groupBitmapState(number) FROM numbers_mt(2000000000) GROUP BY n; -- { serverError 241 } +SELECT cityHash64(rand() % 1000) as n, groupBitmapState(number) FROM numbers_mt(2000000000) GROUP BY n FORMAT Null; -- { serverError 241 } From 60c6a5b23c39520c000ce9f793135176d0bd644a Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 13 Jul 2022 18:36:08 +0200 Subject: [PATCH 166/659] fix s3 read/write throttling instant --- src/IO/ReadBufferFromS3.cpp | 6 ++---- src/IO/WriteBufferFromS3.cpp | 9 ++------- 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 1e49346c8a2..265ce3012e0 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -165,6 +165,8 @@ bool ReadBufferFromS3::nextImpl() ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Bytes, working_buffer.size()); offset += working_buffer.size(); + if (read_settings.throttler) + read_settings.throttler->add(working_buffer.size()); return true; } @@ -300,10 +302,6 @@ std::unique_ptr ReadBufferFromS3::initialize() if (outcome.IsSuccess()) { read_result = outcome.GetResultWithOwnership(); - - if (read_settings.throttler) - read_settings.throttler->add(read_result.GetContentLength()); - size_t buffer_size = use_external_buffer ? 0 : read_settings.remote_fs_buffer_size; return std::make_unique(read_result.GetBody(), buffer_size); } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 7aca544a1a9..27210d50e44 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -124,8 +124,9 @@ void WriteBufferFromS3::nextImpl() } ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Bytes, offset()); - last_part_size += offset(); + if (write_settings.throttler) + write_settings.throttler->add(offset()); /// Data size exceeds singlepart upload threshold, need to use multipart upload. if (multipart_upload_id.empty() && last_part_size > s3_settings.max_single_part_upload_size) @@ -334,8 +335,6 @@ void WriteBufferFromS3::fillUploadRequest(Aws::S3::Model::UploadPartRequest & re void WriteBufferFromS3::processUploadRequest(UploadPartTask & task) { auto outcome = client_ptr->UploadPart(task.req); - if (write_settings.throttler) - write_settings.throttler->add(bytes); if (outcome.IsSuccess()) { @@ -465,11 +464,7 @@ void WriteBufferFromS3::fillPutRequest(Aws::S3::Model::PutObjectRequest & req) void WriteBufferFromS3::processPutRequest(PutObjectTask & task) { - size_t bytes = task.req.GetContentLength(); auto outcome = client_ptr->PutObject(task.req); - if (write_settings.throttler) - write_settings.throttler->add(bytes); - bool with_pool = static_cast(schedule); if (outcome.IsSuccess()) LOG_TRACE(log, "Single part upload has completed. Bucket: {}, Key: {}, Object size: {}, WithPool: {}", bucket, key, task.req.GetContentLength(), with_pool); From f81650e94a6f7f549d48994edb7b9be7c566d5b4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 13 Jul 2022 19:03:48 +0200 Subject: [PATCH 167/659] Fix extremely rare condition in hardlinks creation --- .../FakeMetadataStorageFromDisk.cpp | 79 ----------- .../FakeMetadataStorageFromDisk.h | 8 +- .../MetadataStorageFromDisk.cpp | 41 ++---- .../ObjectStorages/MetadataStorageFromDisk.h | 2 +- ...taStorageFromDiskTransactionOperations.cpp | 96 ++++++++++++-- ...dataStorageFromDiskTransactionOperations.h | 125 +++++++++++++++--- 6 files changed, 204 insertions(+), 147 deletions(-) diff --git a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp index a6b2f6b5f02..76e6aebff1c 100644 --- a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp @@ -114,85 +114,6 @@ const IMetadataStorage & FakeMetadataStorageFromDiskTransaction::getStorageForNo return metadata_storage; } -void FakeMetadataStorageFromDiskTransaction::addOperation(MetadataOperationPtr && operation) -{ - if (state != MetadataFromDiskTransactionState::PREPARING) - throw Exception( - ErrorCodes::FS_METADATA_ERROR, - "Cannot add operations to transaction in {} state, it should be in {} state", - toString(state), toString(MetadataFromDiskTransactionState::PREPARING)); - - operations.emplace_back(std::move(operation)); -} - -void FakeMetadataStorageFromDiskTransaction::commit() -{ - if (state != MetadataFromDiskTransactionState::PREPARING) - throw Exception( - ErrorCodes::FS_METADATA_ERROR, - "Cannot commit transaction in {} state, it should be in {} state", - toString(state), toString(MetadataFromDiskTransactionState::PREPARING)); - - { - std::unique_lock lock(metadata_storage.metadata_mutex); - for (size_t i = 0; i < operations.size(); ++i) - { - try - { - operations[i]->execute(); - } - catch (Exception & ex) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - ex.addMessage(fmt::format("While committing metadata operation #{}", i)); - state = MetadataFromDiskTransactionState::FAILED; - rollback(i); - throw; - } - } - } - - /// Do it in "best effort" mode - for (size_t i = 0; i < operations.size(); ++i) - { - try - { - operations[i]->finalize(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Failed to finalize operation #{}", i)); - } - } - - state = MetadataFromDiskTransactionState::COMMITTED; -} - -void FakeMetadataStorageFromDiskTransaction::rollback(size_t until_pos) -{ - /// Otherwise everything is alright - if (state == MetadataFromDiskTransactionState::FAILED) - { - for (int64_t i = until_pos; i >= 0; --i) - { - try - { - operations[i]->undo(); - } - catch (Exception & ex) - { - state = MetadataFromDiskTransactionState::PARTIALLY_ROLLED_BACK; - ex.addMessage(fmt::format("While rolling back operation #{}", i)); - throw; - } - } - } - else - { - /// Nothing to do, transaction committed or not even started to commit - } -} - void FakeMetadataStorageFromDiskTransaction::writeStringToFile(const std::string & path, const std::string & data) { auto wb = disk->writeFile(path); diff --git a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h index 3fc223ea75b..b6426df1e9a 100644 --- a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h @@ -65,12 +65,6 @@ private: const FakeMetadataStorageFromDisk & metadata_storage; std::vector operations; - MetadataFromDiskTransactionState state{MetadataFromDiskTransactionState::PREPARING}; - - void addOperation(MetadataOperationPtr && operation); - - void rollback(size_t until_pos); - public: FakeMetadataStorageFromDiskTransaction( const FakeMetadataStorageFromDisk & metadata_storage_, DiskPtr disk_) @@ -82,7 +76,7 @@ public: const IMetadataStorage & getStorageForNonTransactionalReads() const final; - void commit() final; + void commit() final {} void writeStringToFile(const std::string & path, const std::string & data) override; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp index a664433a3d0..489772647d1 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp @@ -87,6 +87,14 @@ DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const return metadata; } +DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const std::string & path, std::unique_lock &) const +{ + auto metadata = std::make_unique(disk->getPath(), object_storage_root_path, path); + auto str = readFileToString(path); + metadata->deserializeFromString(str); + return metadata; +} + DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadata(const std::string & path) const { std::shared_lock lock(metadata_mutex); @@ -112,13 +120,7 @@ std::unordered_map MetadataStorageFromDisk::getSerializedMetadat void MetadataStorageFromDiskTransaction::createHardLink(const std::string & path_from, const std::string & path_to) { - auto metadata = metadata_storage.readMetadata(path_from); - - metadata->incrementRefCount(); - - writeStringToFile(path_from, metadata->serializeToString()); - - addOperation(std::make_unique(path_from, path_to, *metadata_storage.getDisk())); + addOperation(std::make_unique(path_from, path_to, *metadata_storage.disk, metadata_storage)); } MetadataTransactionPtr MetadataStorageFromDisk::createTransaction() const @@ -177,12 +179,12 @@ void MetadataStorageFromDiskTransaction::commit() toString(state), toString(MetadataFromDiskTransactionState::PREPARING)); { - std::lock_guard lock(metadata_storage.metadata_mutex); + std::unique_lock lock(metadata_storage.metadata_mutex); for (size_t i = 0; i < operations.size(); ++i) { try { - operations[i]->execute(); + operations[i]->execute(lock); } catch (Exception & ex) { @@ -316,29 +318,12 @@ void MetadataStorageFromDiskTransaction::createMetadataFile(const std::string & void MetadataStorageFromDiskTransaction::addBlobToMetadata(const std::string & path, const std::string & blob_name, uint64_t size_in_bytes) { - DiskObjectStorageMetadataPtr metadata; - if (metadata_storage.exists(path)) - { - metadata = metadata_storage.readMetadata(path); - metadata->addObject(blob_name, size_in_bytes); - writeStringToFile(path, metadata->serializeToString()); - } - else - { - createMetadataFile(path, blob_name, size_in_bytes); - } + addOperation(std::make_unique(path, blob_name, metadata_storage.object_storage_root_path, size_in_bytes, *metadata_storage.disk, metadata_storage)); } void MetadataStorageFromDiskTransaction::unlinkMetadata(const std::string & path) { - auto metadata = metadata_storage.readMetadata(path); - uint32_t ref_count = metadata->getRefCount(); - if (ref_count != 0) - { - metadata->decrementRefCount(); - writeStringToFile(path, metadata->serializeToString()); - } - unlinkFile(path); + addOperation(std::make_unique(path, *metadata_storage.disk, metadata_storage)); } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h index e84b8fe36eb..104e9d54bff 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h @@ -55,9 +55,9 @@ public: std::string getObjectStorageRootPath() const override { return object_storage_root_path; } -private: DiskObjectStorageMetadataPtr readMetadata(const std::string & path) const; + DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::unique_lock & lock) const; DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::shared_lock & lock) const; }; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp index dce4ae2f1f7..7c0a60509ab 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include #include #include @@ -24,7 +26,7 @@ SetLastModifiedOperation::SetLastModifiedOperation(const std::string & path_, Po { } -void SetLastModifiedOperation::execute() +void SetLastModifiedOperation::execute(std::unique_lock &) { old_timestamp = disk.getLastModified(path); disk.setLastModified(path, new_timestamp); @@ -41,7 +43,7 @@ UnlinkFileOperation::UnlinkFileOperation(const std::string & path_, IDisk & disk { } -void UnlinkFileOperation::execute() +void UnlinkFileOperation::execute(std::unique_lock &) { auto buf = disk.readFile(path); readStringUntilEOF(prev_data, *buf); @@ -61,7 +63,7 @@ CreateDirectoryOperation::CreateDirectoryOperation(const std::string & path_, ID { } -void CreateDirectoryOperation::execute() +void CreateDirectoryOperation::execute(std::unique_lock &) { disk.createDirectory(path); } @@ -77,7 +79,7 @@ CreateDirectoryRecursiveOperation::CreateDirectoryRecursiveOperation(const std:: { } -void CreateDirectoryRecursiveOperation::execute() +void CreateDirectoryRecursiveOperation::execute(std::unique_lock &) { namespace fs = std::filesystem; fs::path p(path); @@ -104,7 +106,7 @@ RemoveDirectoryOperation::RemoveDirectoryOperation(const std::string & path_, ID { } -void RemoveDirectoryOperation::execute() +void RemoveDirectoryOperation::execute(std::unique_lock &) { disk.removeDirectory(path); } @@ -121,7 +123,7 @@ RemoveRecursiveOperation::RemoveRecursiveOperation(const std::string & path_, ID { } -void RemoveRecursiveOperation:: execute() +void RemoveRecursiveOperation::execute(std::unique_lock &) { if (disk.isFile(path)) disk.moveFile(path, temp_path); @@ -146,20 +148,31 @@ void RemoveRecursiveOperation::finalize() disk.removeRecursive(path); } -CreateHardlinkOperation::CreateHardlinkOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_) +CreateHardlinkOperation::CreateHardlinkOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_, const MetadataStorageFromDisk & metadata_storage_) : path_from(path_from_) , path_to(path_to_) , disk(disk_) + , metadata_storage(metadata_storage_) { } -void CreateHardlinkOperation::execute() +void CreateHardlinkOperation::execute(std::unique_lock & lock) { + auto metadata = metadata_storage.readMetadataUnlocked(path_from, lock); + + metadata->incrementRefCount(); + + write_operation = std::make_unique(path_from, disk, metadata->serializeToString()); + + write_operation->execute(lock); + disk.createHardLink(path_from, path_to); } void CreateHardlinkOperation::undo() { + if (write_operation) + write_operation->undo(); disk.removeFile(path_to); } @@ -170,7 +183,7 @@ MoveFileOperation::MoveFileOperation(const std::string & path_from_, const std:: { } -void MoveFileOperation::execute() +void MoveFileOperation::execute(std::unique_lock &) { disk.moveFile(path_from, path_to); } @@ -187,7 +200,7 @@ MoveDirectoryOperation::MoveDirectoryOperation(const std::string & path_from_, c { } -void MoveDirectoryOperation::execute() +void MoveDirectoryOperation::execute(std::unique_lock &) { disk.moveDirectory(path_from, path_to); } @@ -197,7 +210,6 @@ void MoveDirectoryOperation::undo() disk.moveDirectory(path_to, path_from); } - ReplaceFileOperation::ReplaceFileOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_) : path_from(path_from_) , path_to(path_to_) @@ -206,7 +218,7 @@ ReplaceFileOperation::ReplaceFileOperation(const std::string & path_from_, const { } -void ReplaceFileOperation::execute() +void ReplaceFileOperation::execute(std::unique_lock &) { if (disk.exists(path_to)) disk.moveFile(path_to, temp_path_to); @@ -232,7 +244,7 @@ WriteFileOperation::WriteFileOperation(const std::string & path_, IDisk & disk_, { } -void WriteFileOperation::execute() +void WriteFileOperation::execute(std::unique_lock &) { if (disk.exists(path)) { @@ -258,4 +270,62 @@ void WriteFileOperation::undo() } } +void AddBlobOperation::execute(std::unique_lock & metadata_lock) +{ + DiskObjectStorageMetadataPtr metadata; + if (metadata_storage.exists(path)) + metadata = metadata_storage.readMetadataUnlocked(path, metadata_lock); + else + metadata = std::make_unique(disk.getPath(), root_path, path); + + metadata->addObject(blob_name, size_in_bytes); + + write_operation = std::make_unique(path, disk, metadata->serializeToString()); + + write_operation->execute(metadata_lock); +} + +void AddBlobOperation::undo() +{ + if (write_operation) + write_operation->undo(); +} + +void UnlinkMetadataFileOperation::execute(std::unique_lock & metadata_lock) +{ + auto metadata = metadata_storage.readMetadataUnlocked(path, metadata_lock); + uint32_t ref_count = metadata->getRefCount(); + if (ref_count != 0) + { + metadata->decrementRefCount(); + write_operation = std::make_unique(path, disk, metadata->serializeToString()); + write_operation->execute(metadata_lock); + } + unlink_operation = std::make_unique(path, disk); + unlink_operation->execute(metadata_lock); +} + +void UnlinkMetadataFileOperation::undo() +{ + if (write_operation) + write_operation->undo(); + + if (unlink_operation) + unlink_operation->undo(); +} + +void SetReadonlyFileOperation::execute(std::unique_lock & metadata_lock) +{ + auto metadata = metadata_storage.readMetadataUnlocked(path, metadata_lock); + metadata->setReadOnly(); + write_operation = std::make_unique(path, disk, metadata->serializeToString()); + write_operation->execute(metadata_lock); +} + +void SetReadonlyFileOperation::undo() +{ + if (write_operation) + write_operation->undo(); +} + } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h index f998771a68f..5f8e772ebc7 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h @@ -4,6 +4,7 @@ namespace DB { +class MetadataStorageFromDisk; class IDisk; /** @@ -12,7 +13,7 @@ class IDisk; struct IMetadataOperation { - virtual void execute() = 0; + virtual void execute(std::unique_lock & metadata_lock) = 0; virtual void undo() = 0; virtual void finalize() {} virtual ~IMetadataOperation() = default; @@ -25,7 +26,7 @@ struct SetLastModifiedOperation final : public IMetadataOperation { SetLastModifiedOperation(const std::string & path_, Poco::Timestamp new_timestamp_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -41,7 +42,7 @@ struct UnlinkFileOperation final : public IMetadataOperation { UnlinkFileOperation(const std::string & path_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -56,7 +57,7 @@ struct CreateDirectoryOperation final : public IMetadataOperation { CreateDirectoryOperation(const std::string & path_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -70,7 +71,7 @@ struct CreateDirectoryRecursiveOperation final : public IMetadataOperation { CreateDirectoryRecursiveOperation(const std::string & path_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -85,7 +86,7 @@ struct RemoveDirectoryOperation final : public IMetadataOperation { RemoveDirectoryOperation(const std::string & path_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -98,7 +99,7 @@ struct RemoveRecursiveOperation final : public IMetadataOperation { RemoveRecursiveOperation(const std::string & path_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -110,12 +111,30 @@ private: std::string temp_path; }; +struct WriteFileOperation final : public IMetadataOperation +{ + WriteFileOperation(const std::string & path_, IDisk & disk_, const std::string & data_); + + void execute(std::unique_lock & metadata_lock) override; + + void undo() override; +private: + std::string path; + IDisk & disk; + std::string data; + bool existed = false; + std::string prev_data; +}; struct CreateHardlinkOperation final : public IMetadataOperation { - CreateHardlinkOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_); + CreateHardlinkOperation( + const std::string & path_from_, + const std::string & path_to_, + IDisk & disk_, + const MetadataStorageFromDisk & metadata_storage_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -123,6 +142,8 @@ private: std::string path_from; std::string path_to; IDisk & disk; + std::unique_ptr write_operation; + const MetadataStorageFromDisk & metadata_storage; }; @@ -130,7 +151,7 @@ struct MoveFileOperation final : public IMetadataOperation { MoveFileOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -145,7 +166,7 @@ struct MoveDirectoryOperation final : public IMetadataOperation { MoveDirectoryOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -160,7 +181,7 @@ struct ReplaceFileOperation final : public IMetadataOperation { ReplaceFileOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_); - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -173,20 +194,86 @@ private: std::string temp_path_to; }; - -struct WriteFileOperation final : public IMetadataOperation +struct AddBlobOperation final : public IMetadataOperation { - WriteFileOperation(const std::string & path_, IDisk & disk_, const std::string & data_); + AddBlobOperation( + const std::string & path_, + const std::string & blob_name_, + const std::string & root_path_, + uint64_t size_in_bytes_, + IDisk & disk_, + const MetadataStorageFromDisk & metadata_storage_) + : path(path_) + , blob_name(blob_name_) + , root_path(root_path_) + , size_in_bytes(size_in_bytes_) + , disk(disk_) + , metadata_storage(metadata_storage_) + {} - void execute() override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; + +private: + std::string path; + std::string blob_name; + std::string root_path; + uint64_t size_in_bytes; + IDisk & disk; + const MetadataStorageFromDisk & metadata_storage; + + std::unique_ptr write_operation; +}; + + +struct UnlinkMetadataFileOperation final : public IMetadataOperation +{ + UnlinkMetadataFileOperation( + const std::string & path_, + IDisk & disk_, + const MetadataStorageFromDisk & metadata_storage_) + : path(path_) + , disk(disk_) + , metadata_storage(metadata_storage_) + { + } + + void execute(std::unique_lock & metadata_lock) override; + + void undo() override; + private: std::string path; IDisk & disk; - std::string data; - bool existed = false; - std::string prev_data; + const MetadataStorageFromDisk & metadata_storage; + + std::unique_ptr write_operation; + std::unique_ptr unlink_operation; +}; + +struct SetReadonlyFileOperation final : public IMetadataOperation +{ + SetReadonlyFileOperation( + const std::string & path_, + IDisk & disk_, + const MetadataStorageFromDisk & metadata_storage_) + : path(path_) + , disk(disk_) + , metadata_storage(metadata_storage_) + { + } + + void execute(std::unique_lock & metadata_lock) override; + + void undo() override; + +private: + std::string path; + IDisk & disk; + const MetadataStorageFromDisk & metadata_storage; + + std::unique_ptr write_operation; }; } From af4a5a5f3ab0777e93e45dab07dcd3e480392abc Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 13 Jul 2022 19:44:29 +0200 Subject: [PATCH 168/659] fix conflicts --- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 1 - src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 9 --------- src/IO/ReadBufferFromAzureBlobStorage.cpp | 1 - src/IO/ReadBufferFromAzureBlobStorage.h | 1 - src/Storages/Hive/HiveFile.cpp | 8 -------- 5 files changed, 20 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 8f70ab876ff..14614871185 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -90,7 +90,6 @@ SeekableReadBufferPtr ReadBufferFromAzureBlobStorageGather::createImplementation settings, max_single_read_retries, max_single_download_retries, - settings, /* use_external_buffer */true, read_until_position); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index da5051f8f47..55c466d45f6 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -193,14 +193,8 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 doesn't support append to files"); bool cache_on_write = cache -<<<<<<< HEAD - && fs::path(path).extension() != ".tmp" - && disk_write_settings.enable_filesystem_cache_on_write_operations - && FileCacheFactory::instance().getSettings(getCacheBasePath()).cache_on_write_operations; -======= && write_settings.enable_filesystem_cache_on_write_operations && FileCacheFactory::instance().getSettings(getCacheBasePath()).cache_on_write_operations; ->>>>>>> master auto settings_ptr = s3_settings.get(); auto s3_buffer = std::make_unique( @@ -211,10 +205,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN attributes, buf_size, threadPoolCallbackRunner(getThreadPoolWriter()), -<<<<<<< HEAD disk_write_settings, -======= ->>>>>>> master cache_on_write ? cache : nullptr); diff --git a/src/IO/ReadBufferFromAzureBlobStorage.cpp b/src/IO/ReadBufferFromAzureBlobStorage.cpp index f624de56dc2..32e70305bea 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/IO/ReadBufferFromAzureBlobStorage.cpp @@ -27,7 +27,6 @@ ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( const ReadSettings & read_settings_, size_t max_single_read_retries_, size_t max_single_download_retries_, - const ReadSettings & read_settings_, bool use_external_buffer_, size_t read_until_position_) : ReadBufferFromFileBase(read_settings_.remote_fs_buffer_size, nullptr, 0) diff --git a/src/IO/ReadBufferFromAzureBlobStorage.h b/src/IO/ReadBufferFromAzureBlobStorage.h index 136f1573fed..5396fcf9719 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.h +++ b/src/IO/ReadBufferFromAzureBlobStorage.h @@ -23,7 +23,6 @@ public: const ReadSettings & read_settings_, size_t max_single_read_retries_, size_t max_single_download_retries_, - const ReadSettings & read_settings_, bool use_external_buffer_ = false, size_t read_until_position_ = 0); diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 23abc074da4..09c3aff4455 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -267,15 +267,7 @@ bool HiveParquetFile::useSplitMinMaxIndex() const void HiveParquetFile::prepareReader() { -<<<<<<< HEAD - in = std::make_unique( - namenode_url, - path, - getContext()->getGlobalContext()->getConfigRef(), - ReadSettings{.throttler = getContext()->getRemoteReadThrottler()}); -======= in = std::make_unique(namenode_url, path, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings()); ->>>>>>> master auto format_settings = getFormatSettings(getContext()); std::atomic is_stopped{0}; THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(*in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES), arrow::default_memory_pool(), &reader)); From 15ab3bc99fb594dc891d93fb8e9086cb9fa7aa60 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 13 Jul 2022 19:48:57 +0200 Subject: [PATCH 169/659] use `context->getWriteSettings()` --- src/Interpreters/Context.cpp | 2 ++ src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/StorageS3.cpp | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index bf011456e89..70a0b201180 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3476,6 +3476,8 @@ WriteSettings Context::getWriteSettings() const res.enable_filesystem_cache_on_write_operations = settings.enable_filesystem_cache_on_write_operations; + res.throttler = getRemoteWriteThrottler(); + return res; } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index d708891d1f6..57e893e9683 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -418,7 +418,7 @@ public: uri, context->getGlobalContext()->getConfigRef(), context->getSettingsRef().hdfs_replication, - WriteSettings{.throttler = context->getRemoteWriteThrottler()}), + context->getWriteSettings()), compression_method, 3); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context); } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index fffd383955f..130bc75a65c 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -588,7 +588,7 @@ public: std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, threadPoolCallbackRunner(IOThreadPool::get()), - WriteSettings{.throttler = context->getRemoteWriteThrottler()}), + context->getWriteSettings()), compression_method, 3); writer From 390b1ac2f7248f7f93fcacc9153a178824eebc9e Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 13 Jul 2022 17:56:34 +0000 Subject: [PATCH 170/659] Improve isNullable/isConstant/isNull/isNotNull performance for LowCardinality argument --- src/Functions/isConstant.cpp | 2 ++ src/Functions/isNotNull.cpp | 13 +++++++++++++ src/Functions/isNull.cpp | 13 +++++++++++++ src/Functions/isNullable.cpp | 4 +++- tests/performance/low_cardinality_argument.xml | 12 ++++++++++++ .../queries/0_stateless/02353_isnullable.reference | 4 ++++ tests/queries/0_stateless/02353_isnullable.sql | 6 ++++++ 7 files changed, 53 insertions(+), 1 deletion(-) create mode 100644 tests/performance/low_cardinality_argument.xml diff --git a/src/Functions/isConstant.cpp b/src/Functions/isConstant.cpp index 6d76cfc1dcc..09b29aaf260 100644 --- a/src/Functions/isConstant.cpp +++ b/src/Functions/isConstant.cpp @@ -29,6 +29,8 @@ public: bool useDefaultImplementationForNothing() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } size_t getNumberOfArguments() const override diff --git a/src/Functions/isNotNull.cpp b/src/Functions/isNotNull.cpp index 49c5964012a..dd6f7de198b 100644 --- a/src/Functions/isNotNull.cpp +++ b/src/Functions/isNotNull.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -32,6 +33,7 @@ public: size_t getNumberOfArguments() const override { return 1; } bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; } @@ -43,6 +45,17 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const ColumnWithTypeAndName & elem = arguments[0]; + if (elem.type->isLowCardinalityNullable()) + { + const auto * low_cardinality_column = checkAndGetColumn(*elem.column); + size_t null_index = low_cardinality_column->getDictionary().getNullValueIndex(); + auto res = DataTypeUInt8().createColumn(); + auto & data = typeid_cast(*res).getData(); + for (size_t i = 0; i != low_cardinality_column->size(); ++i) + data.push_back(low_cardinality_column->getIndexAt(i) != null_index); + return res; + } + if (const auto * nullable = checkAndGetColumn(*elem.column)) { /// Return the negated null map. diff --git a/src/Functions/isNull.cpp b/src/Functions/isNull.cpp index f9111b2dbbb..90f1607578b 100644 --- a/src/Functions/isNull.cpp +++ b/src/Functions/isNull.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -30,6 +31,7 @@ public: size_t getNumberOfArguments() const override { return 1; } bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; } @@ -42,6 +44,17 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override { const ColumnWithTypeAndName & elem = arguments[0]; + if (elem.type->isLowCardinalityNullable()) + { + const auto * low_cardinality_column = checkAndGetColumn(*elem.column); + size_t null_index = low_cardinality_column->getDictionary().getNullValueIndex(); + auto res = DataTypeUInt8().createColumn(); + auto & data = typeid_cast(*res).getData(); + for (size_t i = 0; i != low_cardinality_column->size(); ++i) + data.push_back(low_cardinality_column->getIndexAt(i) == null_index); + return res; + } + if (const auto * nullable = checkAndGetColumn(*elem.column)) { /// Merely return the embedded null map. diff --git a/src/Functions/isNullable.cpp b/src/Functions/isNullable.cpp index 35cefdbfe63..3680ac7ccb0 100644 --- a/src/Functions/isNullable.cpp +++ b/src/Functions/isNullable.cpp @@ -29,6 +29,8 @@ public: bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } @@ -46,7 +48,7 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const auto & elem = arguments[0]; - return ColumnUInt8::create(input_rows_count, isColumnNullable(*elem.column)); + return ColumnUInt8::create(input_rows_count, isColumnNullable(*elem.column) || elem.type->isLowCardinalityNullable()); } }; diff --git a/tests/performance/low_cardinality_argument.xml b/tests/performance/low_cardinality_argument.xml new file mode 100644 index 00000000000..318cfa080cf --- /dev/null +++ b/tests/performance/low_cardinality_argument.xml @@ -0,0 +1,12 @@ + + + CREATE TABLE test_lc (x UInt64, lc LowCardinality(Nullable(String))) ENGINE = MergeTree order by x + + + INSERT INTO test_lc SELECT number, number % 10 ? NULL : toString(number % 10000) FROM numbers(1000000) + + SELECT isNullable(lc) FROM test_lc + SELECT isConstant(lc) FROM test_lc + SELECT isNull(lc) FROM test_lc + SELECT isNotNull(lc) FROM test_lc + diff --git a/tests/queries/0_stateless/02353_isnullable.reference b/tests/queries/0_stateless/02353_isnullable.reference index 74240c4b196..d99961f5c6a 100644 --- a/tests/queries/0_stateless/02353_isnullable.reference +++ b/tests/queries/0_stateless/02353_isnullable.reference @@ -2,3 +2,7 @@ 1 1 1 +0 +1 +0 +1 diff --git a/tests/queries/0_stateless/02353_isnullable.sql b/tests/queries/0_stateless/02353_isnullable.sql index 567e294d498..279eea2520a 100644 --- a/tests/queries/0_stateless/02353_isnullable.sql +++ b/tests/queries/0_stateless/02353_isnullable.sql @@ -3,3 +3,9 @@ SELECT isNullable(toNullable(3)); SELECT isNullable(NULL); SELECT isNullable(materialize(NULL)); + +SELECT isNullable(toLowCardinality(1)); +SELECT isNullable(toNullable(toLowCardinality(1))); + +SELECT isNullable(toLowCardinality(materialize(1))); +SELECT isNullable(toNullable(toLowCardinality(materialize(1)))); From b7f46d954ecc968ee52908007486e1a25d40fcad Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 13 Jul 2022 17:46:09 +0000 Subject: [PATCH 171/659] Cleanup: related to #38719 (SortingStep: deduce way to sort based on input stream sort) --- .../Merges/Algorithms/MergingSortedAlgorithm.cpp | 6 +++--- .../Merges/Algorithms/MergingSortedAlgorithm.h | 4 ++-- src/Processors/Merges/MergingSortedTransform.cpp | 10 +++++++--- src/Processors/Merges/MergingSortedTransform.h | 2 +- src/Processors/Transforms/FinishSortingTransform.cpp | 5 ++--- src/Processors/Transforms/PartialSortingTransform.cpp | 7 +++---- src/Processors/Transforms/PartialSortingTransform.h | 6 +++--- src/Processors/Transforms/SortingTransform.h | 6 +++--- 8 files changed, 24 insertions(+), 22 deletions(-) diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp index 25719166acd..77db1e06d06 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp @@ -10,7 +10,7 @@ namespace DB MergingSortedAlgorithm::MergingSortedAlgorithm( Block header_, size_t num_inputs, - SortDescription description_, + const SortDescription & description_, size_t max_block_size, SortingQueueStrategy sorting_queue_strategy_, UInt64 limit_, @@ -18,7 +18,7 @@ MergingSortedAlgorithm::MergingSortedAlgorithm( bool use_average_block_sizes) : header(std::move(header_)) , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) - , description(std::move(description_)) + , description(description_) , limit(limit_) , out_row_sources_buf(out_row_sources_buf_) , current_inputs(num_inputs) @@ -29,7 +29,7 @@ MergingSortedAlgorithm::MergingSortedAlgorithm( sort_description_types.reserve(description.size()); /// Replace column names in description to positions. - for (auto & column_description : description) + for (const auto & column_description : description) { has_collation |= column_description.collator != nullptr; sort_description_types.emplace_back(header.getByName(column_description.column_name).type); diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h index cf32e5fd4dd..2537c48b128 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h @@ -16,7 +16,7 @@ public: MergingSortedAlgorithm( Block header_, size_t num_inputs, - SortDescription description_, + const SortDescription & description_, size_t max_block_size, SortingQueueStrategy sorting_queue_strategy_, UInt64 limit_ = 0, @@ -38,7 +38,7 @@ private: /// Settings SortDescription description; - UInt64 limit; + const UInt64 limit; bool has_collation = false; /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index 4cb74ffc71e..9ee91ef6db7 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -10,7 +10,7 @@ namespace DB MergingSortedTransform::MergingSortedTransform( const Block & header, size_t num_inputs, - SortDescription description_, + const SortDescription & description_, size_t max_block_size, SortingQueueStrategy sorting_queue_strategy, UInt64 limit_, @@ -19,10 +19,14 @@ MergingSortedTransform::MergingSortedTransform( bool use_average_block_sizes, bool have_all_inputs_) : IMergingTransform( - num_inputs, header, header, have_all_inputs_, limit_, + num_inputs, + header, + header, + have_all_inputs_, + limit_, header, num_inputs, - std::move(description_), + description_, max_block_size, sorting_queue_strategy, limit_, diff --git a/src/Processors/Merges/MergingSortedTransform.h b/src/Processors/Merges/MergingSortedTransform.h index 16e3e2791ee..50586177c6d 100644 --- a/src/Processors/Merges/MergingSortedTransform.h +++ b/src/Processors/Merges/MergingSortedTransform.h @@ -14,7 +14,7 @@ public: MergingSortedTransform( const Block & header, size_t num_inputs, - SortDescription description, + const SortDescription & description, size_t max_block_size, SortingQueueStrategy sorting_queue_strategy, UInt64 limit_ = 0, diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index 3842e034de7..86b5c4c9a00 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -36,9 +36,8 @@ FinishSortingTransform::FinishSortingTransform( /// The target description is modified in SortingTransform constructor. /// To avoid doing the same actions with description_sorted just copy it from prefix of target description. - size_t prefix_size = description_sorted_.size(); - for (size_t i = 0; i < prefix_size; ++i) - description_with_positions.emplace_back(description[i], header_without_constants.getPositionByName(description[i].column_name)); + for (const auto & column_sort_desc : description_sorted_) + description_with_positions.emplace_back(column_sort_desc, header_without_constants.getPositionByName(column_sort_desc.column_name)); } void FinishSortingTransform::consume(Chunk chunk) diff --git a/src/Processors/Transforms/PartialSortingTransform.cpp b/src/Processors/Transforms/PartialSortingTransform.cpp index b0f866cb3fd..3a2cecd3b26 100644 --- a/src/Processors/Transforms/PartialSortingTransform.cpp +++ b/src/Processors/Transforms/PartialSortingTransform.cpp @@ -85,7 +85,7 @@ bool compareWithThreshold(const ColumnRawPtrs & raw_block_columns, size_t min_bl } PartialSortingTransform::PartialSortingTransform( - const Block & header_, SortDescription & description_, UInt64 limit_) + const Block & header_, const SortDescription & description_, UInt64 limit_) : ISimpleTransform(header_, header_, false) , description(description_) , limit(limit_) @@ -93,9 +93,8 @@ PartialSortingTransform::PartialSortingTransform( // Sorting by no columns doesn't make sense. assert(!description_.empty()); - size_t description_size = description.size(); - for (size_t i = 0; i < description_size; ++i) - description_with_positions.emplace_back(description[i], header_.getPositionByName(description[i].column_name)); + for (const auto & column_sort_desc : description) + description_with_positions.emplace_back(column_sort_desc, header_.getPositionByName(column_sort_desc.column_name)); } void PartialSortingTransform::transform(Chunk & chunk) diff --git a/src/Processors/Transforms/PartialSortingTransform.h b/src/Processors/Transforms/PartialSortingTransform.h index 78ce80bdeeb..6dab4497fc7 100644 --- a/src/Processors/Transforms/PartialSortingTransform.h +++ b/src/Processors/Transforms/PartialSortingTransform.h @@ -15,7 +15,7 @@ public: /// limit - if not 0, then you can sort each block not completely, but only `limit` first rows by order. PartialSortingTransform( const Block & header_, - SortDescription & description_, + const SortDescription & description_, UInt64 limit_ = 0); String getName() const override { return "PartialSortingTransform"; } @@ -26,9 +26,9 @@ protected: void transform(Chunk & chunk) override; private: - SortDescription description; + const SortDescription description; SortDescriptionWithPositions description_with_positions; - UInt64 limit; + const UInt64 limit; RowsBeforeLimitCounterPtr read_rows; Columns sort_description_threshold_columns; diff --git a/src/Processors/Transforms/SortingTransform.h b/src/Processors/Transforms/SortingTransform.h index a607e52550d..d9a30699f92 100644 --- a/src/Processors/Transforms/SortingTransform.h +++ b/src/Processors/Transforms/SortingTransform.h @@ -73,8 +73,8 @@ public: ~SortingTransform() override; protected: - Status prepare() override final; - void work() override final; + Status prepare() final; + void work() final; virtual void consume(Chunk chunk) = 0; virtual void generate() = 0; @@ -82,7 +82,7 @@ protected: SortDescription description; size_t max_merged_block_size; - UInt64 limit; + const UInt64 limit; /// Before operation, will remove constant columns from blocks. And after, place constant columns back. /// (to avoid excessive virtual function calls and because constants cannot be serialized in Native format for temporary files) From 2caa3f88036dc41c3d5076915078b97b3e3716c9 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Wed, 13 Jul 2022 11:45:58 -0700 Subject: [PATCH 172/659] Fix LZ4 decompression issue for s390x --- src/Compression/LZ4_decompress_faster.cpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 32d21a37f18..d6384a45740 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -19,6 +19,14 @@ #include #endif +#if defined(__BYTE_ORDER__) && (__BYTE_ORDER__ == __ORDER_BIG_ENDIAN__) +static inline UInt16 LZ4_readLE16(const void* memPtr) +{ + const UInt8* p = reinterpret_cast(memPtr); + return static_cast(p[0]) + (p[1]<<8); +} +#endif + namespace LZ4 { @@ -501,7 +509,11 @@ bool NO_INLINE decompressImpl( /// Get match offset. +#if defined(__BYTE_ORDER__) && (__BYTE_ORDER__ == __ORDER_BIG_ENDIAN__) + size_t offset = LZ4_readLE16(ip); +#else size_t offset = unalignedLoad(ip); +#endif ip += 2; const UInt8 * match = op - offset; From a0055fc47ac751123161fed4e74043116fcd4dd2 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 13 Jul 2022 15:49:22 -0300 Subject: [PATCH 173/659] Add force cache to gRPC_CARES_PROVIDER set --- contrib/grpc-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc-cmake/CMakeLists.txt b/contrib/grpc-cmake/CMakeLists.txt index 4e29032f3c7..b1ed7e464b6 100644 --- a/contrib/grpc-cmake/CMakeLists.txt +++ b/contrib/grpc-cmake/CMakeLists.txt @@ -49,7 +49,7 @@ set(gRPC_ABSL_PROVIDER "clickhouse" CACHE STRING "" FORCE) set(gRPC_BUILD_CSHARP_EXT OFF) set(_gRPC_CARES_LIBRARIES ch_contrib::c-ares) -set(gRPC_CARES_PROVIDER "DO NOT BUILD C-ARES") +set(gRPC_CARES_PROVIDER "clickhouse" CACHE STRING "" FORCE) add_subdirectory("${_gRPC_SOURCE_DIR}" "${_gRPC_BINARY_DIR}") # The contrib/grpc/CMakeLists.txt redefined the PROTOBUF_GENERATE_GRPC_CPP() function for its own purposes, From af682c43794812abb525315d9f14f9c2f6d67180 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Wed, 13 Jul 2022 21:58:25 +0300 Subject: [PATCH 174/659] Empty commit to re-run checks. From 1c6d70f9b32dac7cb5c719f85a1ead99f779487c Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 13 Jul 2022 16:12:15 -0400 Subject: [PATCH 175/659] fixed sw job buffer missing --- src/Compression/CompressionCodecDeflateQpl.cpp | 4 +++- src/Compression/CompressionCodecDeflateQpl.h | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 343ca430834..a1ad0764bc4 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -245,7 +245,9 @@ qpl_job * SoftwareCodecDeflateQpl::getJobCodecPtr() uint32_t size = 0; qpl_get_job_size(qpl_path_software, &size); - sw_job = reinterpret_cast((std::make_unique(size)).get()); + sw_buffer = std::make_unique(size); + sw_job = reinterpret_cast(sw_buffer.get()); + // Job initialization if (auto status = qpl_init_job(qpl_path_software, sw_job); status != QPL_STS_OK) throw Exception(ErrorCodes::CANNOT_COMPRESS, diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 37e1cff650e..b64025c5939 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -65,6 +65,7 @@ public: private: qpl_job * sw_job = nullptr; + std::unique_ptr sw_buffer; qpl_job * getJobCodecPtr(); }; From 089009c996c08fbb7920551b6b41e79e2f13c167 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 13 Jul 2022 20:35:24 +0000 Subject: [PATCH 176/659] fix reading from StorageLog with mmap --- .../MergeTree/MergeTreeWriteAheadLog.cpp | 2 +- src/Storages/StorageLog.cpp | 26 ++++++++++--------- src/Storages/StorageLog.h | 11 ++++---- src/Storages/StorageStripeLog.cpp | 26 ++++++++++--------- src/Storages/StorageStripeLog.h | 10 +++---- .../0_stateless/01533_multiple_nested.sql | 1 + 6 files changed, 41 insertions(+), 35 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index a9dce5b5ebe..3eb638d15c0 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -126,7 +126,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor std::unique_lock lock(write_mutex); MergeTreeData::MutableDataPartsVector parts; - auto in = disk->readFile(path, {}); + auto in = disk->readFile(path); NativeReader block_in(*in, 0); NameSet dropped_parts; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index a03acd3731d..ccb88992732 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -535,15 +535,16 @@ StorageLog::StorageLog( const ConstraintsDescription & constraints_, const String & comment, bool attach, - size_t max_compress_block_size_) + ContextMutablePtr context_) : IStorage(table_id_) + , WithMutableContext(context_) , engine_name(engine_name_) , disk(std::move(disk_)) , table_path(relative_path_) , use_marks_file(engine_name == "Log") , marks_file_path(table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME) , file_checker(disk, table_path + "sizes.json") - , max_compress_block_size(max_compress_block_size_) + , max_compress_block_size(context_->getSettingsRef().max_compress_block_size) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); @@ -750,9 +751,9 @@ static std::chrono::seconds getLockTimeout(ContextPtr context) return std::chrono::seconds{lock_timeout}; } -void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr context, TableExclusiveLockHolder &) +void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) { - WriteLock lock{rwlock, getLockTimeout(context)}; + WriteLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); @@ -769,6 +770,7 @@ void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr marks_loaded = true; num_marks_saved = 0; + getContext()->dropMMappedFileCache(); } @@ -776,14 +778,14 @@ Pipe StorageLog::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & /*query_info*/, - ContextPtr context, + ContextPtr local_context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned num_streams) { storage_snapshot->check(column_names); - auto lock_timeout = getLockTimeout(context); + auto lock_timeout = getLockTimeout(local_context); loadMarks(lock_timeout); ReadLock lock{rwlock, lock_timeout}; @@ -817,7 +819,7 @@ Pipe StorageLog::read( bool limited_by_file_sizes = !use_marks_file; size_t row_limit = std::numeric_limits::max(); - ReadSettings read_settings = context->getReadSettings(); + ReadSettings read_settings = local_context->getReadSettings(); Pipes pipes; for (size_t stream = 0; stream < num_streams; ++stream) @@ -848,18 +850,18 @@ Pipe StorageLog::read( return Pipe::unitePipes(std::move(pipes)); } -SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { - WriteLock lock{rwlock, getLockTimeout(context)}; + WriteLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -CheckResults StorageLog::checkData(const ASTPtr & /* query */, ContextPtr context) +CheckResults StorageLog::checkData(const ASTPtr & /* query */, ContextPtr local_context) { - ReadLock lock{rwlock, getLockTimeout(context)}; + ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); @@ -1114,7 +1116,7 @@ void registerStorageLog(StorageFactory & factory) args.constraints, args.comment, args.attach, - args.getContext()->getSettings().max_compress_block_size); + args.getContext()); }; factory.registerStorage("Log", create_fn, features); diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 778633440a4..2e677dd3161 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -12,6 +12,7 @@ namespace DB { + class IBackup; using BackupPtr = std::shared_ptr; @@ -21,7 +22,7 @@ using BackupPtr = std::shared_ptr; * Also implements TinyLog - a table engine that is suitable for small chunks of the log. * It differs from Log in the absence of mark files. */ -class StorageLog final : public IStorage +class StorageLog final : public IStorage, public WithMutableContext { friend class LogSource; friend class LogSink; @@ -40,7 +41,7 @@ public: const ConstraintsDescription & constraints_, const String & comment, bool attach, - size_t max_compress_block_size_); + ContextMutablePtr context_); ~StorageLog() override; String getName() const override { return engine_name; } @@ -49,16 +50,16 @@ public: const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, + ContextPtr local_context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - CheckResults checkData(const ASTPtr & /* query */, ContextPtr /* context */) override; + CheckResults checkData(const ASTPtr & query, ContextPtr local_context) override; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index eb8bc9b1d51..e3f477936db 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -265,14 +265,15 @@ StorageStripeLog::StorageStripeLog( const ConstraintsDescription & constraints_, const String & comment, bool attach, - size_t max_compress_block_size_) + ContextMutablePtr context_) : IStorage(table_id_) + , WithMutableContext(context_) , disk(std::move(disk_)) , table_path(relative_path_) , data_file_path(table_path + "data.bin") , index_file_path(table_path + "index.mrk") , file_checker(disk, table_path + "sizes.json") - , max_compress_block_size(max_compress_block_size_) + , max_compress_block_size(context_->getSettings().max_compress_block_size) , log(&Poco::Logger::get("StorageStripeLog")) { StorageInMemoryMetadata storage_metadata; @@ -330,9 +331,9 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora } -static std::chrono::seconds getLockTimeout(ContextPtr context) +static std::chrono::seconds getLockTimeout(ContextPtr local_context) { - const Settings & settings = context->getSettingsRef(); + const Settings & settings = local_context->getSettingsRef(); Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) lock_timeout = settings.max_execution_time.totalSeconds(); @@ -344,14 +345,14 @@ Pipe StorageStripeLog::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & /*query_info*/, - ContextPtr context, + ContextPtr local_context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, unsigned num_streams) { storage_snapshot->check(column_names); - auto lock_timeout = getLockTimeout(context); + auto lock_timeout = getLockTimeout(local_context); loadIndices(lock_timeout); ReadLock lock{rwlock, lock_timeout}; @@ -369,7 +370,7 @@ Pipe StorageStripeLog::read( if (num_streams > size) num_streams = size; - ReadSettings read_settings = context->getReadSettings(); + ReadSettings read_settings = local_context->getReadSettings(); Pipes pipes; for (size_t stream = 0; stream < num_streams; ++stream) @@ -390,9 +391,9 @@ Pipe StorageStripeLog::read( } -SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { - WriteLock lock{rwlock, getLockTimeout(context)}; + WriteLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); @@ -400,9 +401,9 @@ SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const Storage } -CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr context) +CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr local_context) { - ReadLock lock{rwlock, getLockTimeout(context)}; + ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); @@ -420,6 +421,7 @@ void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont indices_loaded = true; num_indices_saved = 0; + getContext()->dropMMappedFileCache(); } @@ -686,7 +688,7 @@ void registerStorageStripeLog(StorageFactory & factory) args.constraints, args.comment, args.attach, - args.getContext()->getSettings().max_compress_block_size); + args.getContext()); }, features); } diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 3faffff381d..efdf18c0f7b 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -20,7 +20,7 @@ using BackupPtr = std::shared_ptr; /** Implements a table engine that is suitable for small chunks of the log. * In doing so, stores all the columns in a single Native file, with a nearby index. */ -class StorageStripeLog final : public IStorage +class StorageStripeLog final : public IStorage, public WithMutableContext { friend class StripeLogSource; friend class StripeLogSink; @@ -34,7 +34,7 @@ public: const ConstraintsDescription & constraints_, const String & comment, bool attach, - size_t max_compress_block_size_); + ContextMutablePtr context_); ~StorageStripeLog() override; @@ -44,16 +44,16 @@ public: const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, + ContextPtr local_context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - CheckResults checkData(const ASTPtr & /* query */, ContextPtr /* context */) override; + CheckResults checkData(const ASTPtr & query, ContextPtr ocal_context) override; bool storesDataOnDisk() const override { return true; } Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } diff --git a/tests/queries/0_stateless/01533_multiple_nested.sql b/tests/queries/0_stateless/01533_multiple_nested.sql index 03724ce0b46..a61f13fc807 100644 --- a/tests/queries/0_stateless/01533_multiple_nested.sql +++ b/tests/queries/0_stateless/01533_multiple_nested.sql @@ -4,6 +4,7 @@ DROP TABLE IF EXISTS nested; SET flatten_nested = 0; SET use_uncompressed_cache = 0; +SET local_filesystem_read_method='pread'; CREATE TABLE nested ( From 7a09aa478a0c0ab577a9ec0b0feb018d4eaada2b Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 13 Jul 2022 17:45:09 -0300 Subject: [PATCH 177/659] Add missing include header for osx builds --- src/Common/CaresPTRResolver.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/CaresPTRResolver.cpp b/src/Common/CaresPTRResolver.cpp index 2915b200433..eacc3bffe54 100644 --- a/src/Common/CaresPTRResolver.cpp +++ b/src/Common/CaresPTRResolver.cpp @@ -1,5 +1,6 @@ #include "CaresPTRResolver.h" #include +#include #include #include "ares.h" #include "netdb.h" From ba863b11a4f7a1daa3d397d1cea2517ddde29eef Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Jul 2022 21:06:15 +0000 Subject: [PATCH 178/659] A little fix. Added test. --- src/Interpreters/ExpressionAnalyzer.cpp | 44 +++++++++++-------- .../02354_read_in_order_prewhere.reference | 10 +++++ .../02354_read_in_order_prewhere.sql | 30 +++++++++++++ 3 files changed, 66 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/02354_read_in_order_prewhere.reference create mode 100644 tests/queries/0_stateless/02354_read_in_order_prewhere.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 6460ed536e9..7847f0714ca 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1850,8 +1850,25 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( }; { + bool join_allow_read_in_order = true; + if (hasJoin()) + { + /// You may find it strange but we support read_in_order for HashJoin and do not support for MergeJoin. + join_has_delayed_stream = query_analyzer.analyzedJoin().needStreamWithNonJoinedRows(); + join_allow_read_in_order = typeid_cast(join.get()) && !join_has_delayed_stream; + } + + optimize_read_in_order = + settings.optimize_read_in_order + && storage + && query.orderBy() + && !query_analyzer.hasAggregation() + && !query_analyzer.hasWindow() + && !query.final() + && join_allow_read_in_order; + ExpressionActionsChain chain(context); - Names additional_required_columns_after_prewhere = metadata_snapshot ? metadata_snapshot->getColumnsRequiredForSortingKey() : Names{}; + Names additional_required_columns_after_prewhere; if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1)) { @@ -1867,6 +1884,14 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( columns_for_final.begin(), columns_for_final.end()); } + if (storage && optimize_read_in_order) + { + std::cerr << "!!!!!!!!!!!!!!\n"; + Names columns_for_sorting_key = metadata_snapshot->getColumnsRequiredForSortingKey(); + additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), + columns_for_sorting_key.begin(), columns_for_sorting_key.end()); + } + if (storage && filter_info_) { filter_info = filter_info_; @@ -1951,23 +1976,6 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( } } - bool join_allow_read_in_order = true; - if (hasJoin()) - { - /// You may find it strange but we support read_in_order for HashJoin and do not support for MergeJoin. - join_has_delayed_stream = query_analyzer.analyzedJoin().needStreamWithNonJoinedRows(); - join_allow_read_in_order = typeid_cast(join.get()) && !join_has_delayed_stream; - } - - optimize_read_in_order = - settings.optimize_read_in_order - && storage - && query.orderBy() - && !query_analyzer.hasAggregation() - && !query_analyzer.hasWindow() - && !query.final() - && join_allow_read_in_order; - /// If there is aggregation, we execute expressions in SELECT and ORDER BY on the initiating server, otherwise on the source servers. query_analyzer.appendSelect(chain, only_types || (need_aggregate ? !second_stage : !first_stage)); diff --git a/tests/queries/0_stateless/02354_read_in_order_prewhere.reference b/tests/queries/0_stateless/02354_read_in_order_prewhere.reference new file mode 100644 index 00000000000..7d5543bf9cc --- /dev/null +++ b/tests/queries/0_stateless/02354_read_in_order_prewhere.reference @@ -0,0 +1,10 @@ +1 +1 +1 +2001 +2001 +1 +1 +1 +2001 +2001 diff --git a/tests/queries/0_stateless/02354_read_in_order_prewhere.sql b/tests/queries/0_stateless/02354_read_in_order_prewhere.sql new file mode 100644 index 00000000000..c5abd5945f3 --- /dev/null +++ b/tests/queries/0_stateless/02354_read_in_order_prewhere.sql @@ -0,0 +1,30 @@ +drop table if exists order; + +CREATE TABLE order +( + ID Int64, + Type Int64, + Num UInt64, + t DateTime +) +ENGINE = MergeTree() +PARTITION BY toYYYYMMDD(t) +ORDER BY (ID, Type, Num); + +system stop merges order; + +insert into order select number%2000, 1, number, (1656700561 - intDiv(intHash32(number), 1000)) from numbers(100000); +insert into order select number%2000, 1, number, (1656700561 - intDiv(intHash32(number), 1000)) from numbers(100000); +insert into order select number%2000, 1, number, (1656700561 - intDiv(intHash32(number), 1000)) from numbers(100000); + +SELECT Num +FROM order +WHERE Type = 1 AND ID = 1 +ORDER BY Num ASC limit 5; + +SELECT Num +FROM order +PREWHERE Type = 1 +WHERE ID = 1 +ORDER BY Num ASC limit 5; + From dc20b850781020bb94572d3c3762abd379ab3bae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Jul 2022 21:08:23 +0000 Subject: [PATCH 179/659] Remove debug output. --- src/Interpreters/ExpressionAnalyzer.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index b439583bd0a..e2c106ff796 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1894,7 +1894,6 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (storage && optimize_read_in_order) { - std::cerr << "!!!!!!!!!!!!!!\n"; Names columns_for_sorting_key = metadata_snapshot->getColumnsRequiredForSortingKey(); additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), columns_for_sorting_key.begin(), columns_for_sorting_key.end()); From add169278b65406c71af3f85b4a88fb702e70564 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 13 Jul 2022 19:27:33 -0400 Subject: [PATCH 180/659] fix indent and link --- .../sql-reference/table-functions/remote.md | 70 +++++++++---------- 1 file changed, 35 insertions(+), 35 deletions(-) diff --git a/docs/en/sql-reference/table-functions/remote.md b/docs/en/sql-reference/table-functions/remote.md index 5948fa6f5e1..61018a3d5a7 100644 --- a/docs/en/sql-reference/table-functions/remote.md +++ b/docs/en/sql-reference/table-functions/remote.md @@ -91,50 +91,50 @@ SELECT * FROM remote_table; This example uses one table from a sample dataset. The database is `imdb`, and the table is `actors`. #### On the source ClickHouse system (the system that currently hosts the data) -- Verify the source database and table name (`imdb.actors`) -```sql -show databases -``` +- Verify the source database and table name (`imdb.actors`) + ```sql + show databases + ``` -```sql -show tables in imdb -``` + ```sql + show tables in imdb + ``` - Get the CREATE TABLE statement from the source: -``` -select create_table_query -from system.tables -where database = 'imdb' and table = 'actors' -``` + ``` + select create_table_query + from system.tables + where database = 'imdb' and table = 'actors' + ``` -Response -```sql -CREATE TABLE imdb.actors (`id` UInt32, - `first_name` String, - `last_name` String, - `gender` FixedString(1)) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') - ORDER BY (id, first_name, last_name, gender) - SETTINGS index_granularity = 8192 -``` + Response + ```sql + CREATE TABLE imdb.actors (`id` UInt32, + `first_name` String, + `last_name` String, + `gender` FixedString(1)) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') + ORDER BY (id, first_name, last_name, gender) + SETTINGS index_granularity = 8192 + ``` #### On the destination ClickHouse system: - Create the destination database: -```sql -CREATE DATABASE imdb -``` + ```sql + CREATE DATABASE imdb + ``` - Using the CREATE TABLE statement from the source, create the destination: -```sql -CREATE TABLE imdb.actors (`id` UInt32, - `first_name` String, - `last_name` String, - `gender` FixedString(1)) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') - ORDER BY (id, first_name, last_name, gender) - SETTINGS index_granularity = 8192 -``` + ```sql + CREATE TABLE imdb.actors (`id` UInt32, + `first_name` String, + `last_name` String, + `gender` FixedString(1)) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') + ORDER BY (id, first_name, last_name, gender) + SETTINGS index_granularity = 8192 + ``` #### Back on the source deployment: @@ -145,7 +145,7 @@ remoteSecure('remote.clickhouse.cloud:9440', 'imdb.actors', 'USER', 'PASSWORD', SELECT * from imdb.actors ``` -## Globs in Addresses {globs-in-addresses} +## Globs in Addresses {#globs-in-addresses} Patterns in curly brackets `{ }` are used to generate a set of shards and to specify replicas. If there are multiple pairs of curly brackets, then the direct product of the corresponding sets is generated. The following pattern types are supported. From d108efa2f625a5c76bac0f15a2998cfa4eb12868 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Thu, 14 Jul 2022 01:36:45 -0400 Subject: [PATCH 181/659] merge_algorithm is added --- docs/en/operations/system-tables/part_log.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/operations/system-tables/part_log.md b/docs/en/operations/system-tables/part_log.md index 298667f895b..e1134f6baf6 100644 --- a/docs/en/operations/system-tables/part_log.md +++ b/docs/en/operations/system-tables/part_log.md @@ -19,6 +19,10 @@ The `system.part_log` table contains the following columns: - `REGULAR_MERGE` — Some regular merge. - `TTL_DELETE_MERGE` — Cleaning up expired data. - `TTL_RECOMPRESS_MERGE` — Recompressing data part with the. +- `merge_algorithm` ([Enum8](../../sql-reference/data-types/enum.md)) — Merge algorithm for the event with type `MERGE_PARTS`. Can have one of the following values: + - `UNDECIDED` + - `HORIZONTAL` + - `VERTICAL` - `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time. - `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Event time with microseconds precision. @@ -52,6 +56,7 @@ Row 1: query_id: 983ad9c7-28d5-4ae1-844e-603116b7de31 event_type: NewPart merge_reason: NotAMerge +merge_algorithm: Undecided event_date: 2021-02-02 event_time: 2021-02-02 11:14:28 event_time_microseconds: 2021-02-02 11:14:28.861919 From 887a34869dfa7e691e4b081a66e41854a56029eb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Thu, 14 Jul 2022 01:44:19 -0400 Subject: [PATCH 182/659] merge_reason and merge_algorithm are added --- docs/ru/operations/system-tables/part_log.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/ru/operations/system-tables/part_log.md b/docs/ru/operations/system-tables/part_log.md index 78e9a7c0fbe..73ded8c7355 100644 --- a/docs/ru/operations/system-tables/part_log.md +++ b/docs/ru/operations/system-tables/part_log.md @@ -14,6 +14,15 @@ - `REMOVE_PART` — удаление или отсоединение из таблицы с помощью [DETACH PARTITION](../../sql-reference/statements/alter/partition.md#alter_detach-partition). - `MUTATE_PART` — изменение куска. - `MOVE_PART` — перемещение куска между дисками. +- `merge_reason` ([Enum8](../../sql-reference/data-types/enum.md)) — Причина события с типом `MERGE_PARTS`. Может принимать одно из следующих значений: + - `NOT_A_MERGE` — событие имеет тип иной, чем `MERGE_PARTS`. + - `REGULAR_MERGE` — обычное слияние. + - `TTL_DELETE_MERGE` — очистка истекших данных. + - `TTL_RECOMPRESS_MERGE` — переупаковка куска. +- `merge_algorithm` ([Enum8](../../sql-reference/data-types/enum.md)) — Алгоритм слияния для события с типом `MERGE_PARTS`. Может принимать одно из следующих значений: + - `UNDECIDED` + - `HORIZONTAL` + - `VERTICAL` - `event_date` ([Date](../../sql-reference/data-types/date.md)) — дата события. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время события. - `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — время события с точностью до микросекунд. @@ -46,6 +55,8 @@ Row 1: ────── query_id: 983ad9c7-28d5-4ae1-844e-603116b7de31 event_type: NewPart +merge_reason: NotAMerge +merge_algorithm: Undecided event_date: 2021-02-02 event_time: 2021-02-02 11:14:28 event_time_microseconds: 2021-02-02 11:14:28.861919 From 6ec4f3cf3da836c7e0dde03f584e26bacff41fa1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 12 Jul 2022 09:24:12 +0200 Subject: [PATCH 183/659] Implement non-const needle arguments in multiSearchAllPositions --- .../FunctionsMultiStringFuzzySearch.h | 4 - src/Functions/FunctionsMultiStringPosition.h | 54 +++------ src/Functions/FunctionsMultiStringSearch.h | 4 - src/Functions/MultiSearchAllPositionsImpl.h | 112 ++++++++++++++++-- src/Functions/MultiSearchImpl.h | 2 +- src/Functions/multiSearchAllPositions.cpp | 2 +- ...multiSearchAllPositionsCaseInsensitive.cpp | 2 +- ...iSearchAllPositionsCaseInsensitiveUTF8.cpp | 5 +- src/Functions/multiSearchAllPositionsUTF8.cpp | 2 +- .../00233_position_function_family.reference | 16 +++ .../00233_position_function_family.sql | 8 ++ 11 files changed, 148 insertions(+), 63 deletions(-) diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index 26cafa7d410..5c0c3e12d2a 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -99,23 +99,19 @@ public: /// the implementations are responsible for resizing the output column if (col_needles_const) - { Impl::vectorConstant( col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needles_const->getValue(), vec_res, offsets_res, edit_distance, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); - } else - { Impl::vectorVector( col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needles_vector->getData(), col_needles_vector->getOffsets(), vec_res, offsets_res, edit_distance, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); - } // the combination of const haystack + const needle is not implemented because // useDefaultImplementationForConstants() == true makes upper layers convert both to diff --git a/src/Functions/FunctionsMultiStringPosition.h b/src/Functions/FunctionsMultiStringPosition.h index 488d8279e5d..ec68b33cb2c 100644 --- a/src/Functions/FunctionsMultiStringPosition.h +++ b/src/Functions/FunctionsMultiStringPosition.h @@ -27,22 +27,19 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ILLEGAL_COLUMN; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -template +template class FunctionsMultiStringPosition : public IFunction { public: - static constexpr auto name = Name::name; + static constexpr auto name = Impl::name; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override @@ -67,49 +64,32 @@ public: const ColumnPtr & needles_ptr = arguments[1].column; const ColumnString * col_haystack_vector = checkAndGetColumn(&*haystack_ptr); + assert(col_haystack_vector); + const ColumnArray * col_needles_vector = checkAndGetColumn(needles_ptr.get()); const ColumnConst * col_needles_const = checkAndGetColumnConst(needles_ptr.get()); - - if (!col_needles_const) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {}. The array is not const", arguments[1].column->getName()); - - Array needles_arr = col_needles_const->getValue(); - - if (needles_arr.size() > std::numeric_limits::max()) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be at most 255", getName(), needles_arr.size()); - - std::vector needles; - needles.reserve(needles_arr.size()); - for (const auto & el : needles_arr) - needles.emplace_back(el.get()); - - const size_t column_haystack_size = haystack_ptr->size(); + assert(static_cast(col_needles_vector) ^ static_cast(col_needles_const)); using ResultType = typename Impl::ResultType; auto col_res = ColumnVector::create(); - auto col_offsets = ColumnArray::ColumnOffsets::create(column_haystack_size); + auto col_offsets = ColumnArray::ColumnOffsets::create(); auto & vec_res = col_res->getData(); auto & offsets_res = col_offsets->getData(); + /// the implementations are responsible for resizing the output column - vec_res.resize(column_haystack_size * needles.size()); - - if (col_haystack_vector) - Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needles, vec_res); + if (col_needles_const) + Impl::vectorConstant( + col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), + col_needles_const->getValue(), + vec_res, offsets_res); else - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {}", arguments[0].column->getName()); + Impl::vectorVector( + col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), + col_needles_vector->getData(), col_needles_vector->getOffsets(), + vec_res, offsets_res); - size_t needles_size = needles.size(); - size_t accum = needles_size; - - for (size_t i = 0; i < column_haystack_size; ++i, accum += needles_size) - offsets_res[i] = accum; + // TODO: add comment about const const return ColumnArray::create(std::move(col_res), std::move(col_offsets)); } diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index fb800448bfa..3dd1646c051 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -99,21 +99,17 @@ public: /// the implementations are responsible for resizing the output column if (col_needles_const) - { Impl::vectorConstant( col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needles_const->getValue(), vec_res, offsets_res, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); - } else - { Impl::vectorVector( col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needles_vector->getData(), col_needles_vector->getOffsets(), vec_res, offsets_res, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); - } // the combination of const haystack + const needle is not implemented because // useDefaultImplementationForConstants() == true makes upper layers convert both to diff --git a/src/Functions/MultiSearchAllPositionsImpl.h b/src/Functions/MultiSearchAllPositionsImpl.h index 4356d6110f1..6e1f13d87b6 100644 --- a/src/Functions/MultiSearchAllPositionsImpl.h +++ b/src/Functions/MultiSearchAllPositionsImpl.h @@ -7,17 +7,34 @@ namespace DB { -template +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +template struct MultiSearchAllPositionsImpl { using ResultType = UInt64; + static constexpr auto name = Name::name; static void vectorConstant( const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, - const std::vector & needles, - PaddedPODArray & res) + const Array & needles_arr, + PaddedPODArray & vec_res, + PaddedPODArray & offsets_res) { + if (needles_arr.size() > std::numeric_limits::max()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be at most 255", name, needles_arr.size()); + + std::vector needles; + needles.reserve(needles_arr.size()); + for (const auto & needle : needles_arr) + needles.emplace_back(needle.get()); + auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 { return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); @@ -25,23 +42,96 @@ struct MultiSearchAllPositionsImpl auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_string_size = haystack_offsets.size(); + const size_t haystack_size = haystack_offsets.size(); const size_t needles_size = needles.size(); + vec_res.resize(haystack_size * needles.size()); + offsets_res.resize(haystack_size); + /// Something can be uninitialized after the search itself - std::fill(res.begin(), res.end(), 0); + std::fill(vec_res.begin(), vec_res.end(), 0); while (searcher.hasMoreToSearch()) { - size_t prev_offset = 0; - for (size_t j = 0, from = 0; j < haystack_string_size; ++j, from += needles_size) + size_t prev_haystack_offset = 0; + for (size_t j = 0, from = 0; j < haystack_size; ++j, from += needles_size) { - const auto * haystack = &haystack_data[prev_offset]; - const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; - searcher.searchOneAll(haystack, haystack_end, res.data() + from, res_callback); - prev_offset = haystack_offsets[j]; + const auto * haystack = &haystack_data[prev_haystack_offset]; + const auto * haystack_end = haystack + haystack_offsets[j] - prev_haystack_offset - 1; + searcher.searchOneAll(haystack, haystack_end, vec_res.begin() + from, res_callback); + prev_haystack_offset = haystack_offsets[j]; } } + + size_t accum = needles_size; + for (size_t i = 0; i < haystack_size; ++i) + { + offsets_res[i] = accum; + accum += needles_size; + } + } + + static void vectorVector( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const IColumn & needles_data, + const ColumnArray::Offsets & needles_offsets, + PaddedPODArray & vec_res, + PaddedPODArray & offsets_res) + { + size_t prev_haystack_offset = 0; + size_t prev_needles_offset = 0; + + auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 + { + return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); + }; + + offsets_res.reserve(haystack_offsets.size()); + + const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); + + std::vector needles; + + for (size_t i = 0; i < haystack_offsets.size(); ++i) + { + needles.reserve(needles_offsets[i] - prev_needles_offset); + + for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) + { + needles.emplace_back(needles_data_string->getDataAt(j).toView()); + } + + const size_t needles_size = needles.size(); + + if (needles_size > std::numeric_limits::max()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be at most 255", name, needles_size); + + vec_res.resize(vec_res.size() + needles_size); + + auto searcher = Impl::createMultiSearcherInBigHaystack(needles); /// sub-optimal + + /// Something can be uninitialized after the search itself + std::fill(vec_res.begin() + vec_res.size() - needles_size, vec_res.end(), 0); + + while (searcher.hasMoreToSearch()) + { + const auto * haystack = &haystack_data[prev_haystack_offset]; + const auto * haystack_end = haystack + haystack_offsets[i] - prev_haystack_offset - 1; + searcher.searchOneAll(haystack, haystack_end, vec_res.begin() + vec_res.size() - needles_size, res_callback); + } + + if (offsets_res.empty()) + offsets_res.push_back(needles_size); + else + offsets_res.push_back(offsets_res.back() + needles_size); + + prev_haystack_offset = haystack_offsets[i]; + prev_needles_offset = needles_offsets[i]; + needles.clear(); + } } }; diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index b9ce0293234..d42c2ca43e4 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -38,7 +38,7 @@ struct MultiSearchImpl if (needles_arr.size() > std::numeric_limits::max()) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: passed {}, should be at most {}", - name, std::to_string(needles_arr.size()), std::to_string(std::numeric_limits::max())); + name, needles_arr.size(), std::to_string(std::numeric_limits::max())); std::vector needles; needles.reserve(needles_arr.size()); diff --git a/src/Functions/multiSearchAllPositions.cpp b/src/Functions/multiSearchAllPositions.cpp index 5d9b3f5e2fd..53f3da9cde6 100644 --- a/src/Functions/multiSearchAllPositions.cpp +++ b/src/Functions/multiSearchAllPositions.cpp @@ -15,7 +15,7 @@ struct NameMultiSearchAllPositions }; using FunctionMultiSearchAllPositions - = FunctionsMultiStringPosition, NameMultiSearchAllPositions>; + = FunctionsMultiStringPosition>; } diff --git a/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp b/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp index 9f93284a769..55c112eb093 100644 --- a/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp +++ b/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp @@ -15,7 +15,7 @@ struct NameMultiSearchAllPositionsCaseInsensitive }; using FunctionMultiSearchAllPositionsCaseInsensitive - = FunctionsMultiStringPosition, NameMultiSearchAllPositionsCaseInsensitive>; + = FunctionsMultiStringPosition>; } diff --git a/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp b/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp index 8864a00a8d3..df9de8a17ec 100644 --- a/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp +++ b/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp @@ -14,9 +14,8 @@ struct NameMultiSearchAllPositionsCaseInsensitiveUTF8 static constexpr auto name = "multiSearchAllPositionsCaseInsensitiveUTF8"; }; -using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 = FunctionsMultiStringPosition< - MultiSearchAllPositionsImpl, - NameMultiSearchAllPositionsCaseInsensitiveUTF8>; +using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 + = FunctionsMultiStringPosition>; } diff --git a/src/Functions/multiSearchAllPositionsUTF8.cpp b/src/Functions/multiSearchAllPositionsUTF8.cpp index 3922a859c3a..e5f9a02afcc 100644 --- a/src/Functions/multiSearchAllPositionsUTF8.cpp +++ b/src/Functions/multiSearchAllPositionsUTF8.cpp @@ -15,7 +15,7 @@ struct NameMultiSearchAllPositionsUTF8 }; using FunctionMultiSearchAllPositionsUTF8 - = FunctionsMultiStringPosition, NameMultiSearchAllPositionsUTF8>; + = FunctionsMultiStringPosition>; } diff --git a/tests/queries/0_stateless/00233_position_function_family.reference b/tests/queries/0_stateless/00233_position_function_family.reference index f6cacbc7027..2ccb95615c0 100644 --- a/tests/queries/0_stateless/00233_position_function_family.reference +++ b/tests/queries/0_stateless/00233_position_function_family.reference @@ -23812,3 +23812,19 @@ 1 1 1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/00233_position_function_family.sql b/tests/queries/0_stateless/00233_position_function_family.sql index 36f8a5535d9..d75d09b4716 100644 --- a/tests/queries/0_stateless/00233_position_function_family.sql +++ b/tests/queries/0_stateless/00233_position_function_family.sql @@ -699,6 +699,14 @@ select [1] = multiSearchAllPositions(materialize('abab'), ['ab']); select [1] = multiSearchAllPositions(materialize('abababababababababababab'), ['abab']); select [1] = multiSearchAllPositions(materialize('abababababababababababab'), ['abababababababababa']); +select [1] = multiSearchAllPositions(materialize('abc'), materialize([''])); +select [1] = multiSearchAllPositions(materialize('abc'), materialize([''])) from system.numbers limit 10; +select [1] = multiSearchAllPositions(materialize('abab'), materialize(['ab'])); +select [2] = multiSearchAllPositions(materialize('abab'), materialize(['ba'])); +select [1] = multiSearchAllPositionsCaseInsensitive(materialize('aBaB'), materialize(['abab'])); +select [3] = multiSearchAllPositionsUTF8(materialize('ab€ab'), materialize(['€'])); +select [3] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ab€AB'), materialize(['€ab'])); + select 1 = multiSearchAny(materialize('abcdefgh'), ['b']); select 1 = multiSearchAny(materialize('abcdefgh'), ['bc']); select 1 = multiSearchAny(materialize('abcdefgh'), ['bcd']); From 198abad2847de6b2e4fb72a17c57f6e04215230c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 08:02:49 +0000 Subject: [PATCH 184/659] Disallow const haystack with non-const needle argument --- src/Functions/FunctionsMultiStringPosition.h | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsMultiStringPosition.h b/src/Functions/FunctionsMultiStringPosition.h index ec68b33cb2c..5561af66d2a 100644 --- a/src/Functions/FunctionsMultiStringPosition.h +++ b/src/Functions/FunctionsMultiStringPosition.h @@ -26,6 +26,7 @@ namespace DB namespace ErrorCodes { + extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -64,12 +65,16 @@ public: const ColumnPtr & needles_ptr = arguments[1].column; const ColumnString * col_haystack_vector = checkAndGetColumn(&*haystack_ptr); - assert(col_haystack_vector); + const ColumnConst * col_haystack_const = checkAndGetColumnConst(&*haystack_ptr); + assert(static_cast(col_haystack_vector) ^ static_cast(col_haystack_const)); const ColumnArray * col_needles_vector = checkAndGetColumn(needles_ptr.get()); const ColumnConst * col_needles_const = checkAndGetColumnConst(needles_ptr.get()); assert(static_cast(col_needles_vector) ^ static_cast(col_needles_const)); + if (col_haystack_const && col_needles_vector) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support search with non-constant needles in constant haystack", name); + using ResultType = typename Impl::ResultType; auto col_res = ColumnVector::create(); auto col_offsets = ColumnArray::ColumnOffsets::create(); @@ -89,7 +94,9 @@ public: col_needles_vector->getData(), col_needles_vector->getOffsets(), vec_res, offsets_res); - // TODO: add comment about const const + // the combination of const haystack + const needle is not implemented because + // useDefaultImplementationForConstants() == true makes upper layers convert both to + // non-const columns return ColumnArray::create(std::move(col_res), std::move(col_offsets)); } From 3e77b80b882ee34b502d8ec0caa0405da2a58124 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Jul 2022 11:38:46 +0300 Subject: [PATCH 185/659] Fix fetch of in-memory part with allow_remote_fs_zero_copy_replication CI founds the following error during trying to fetch in-memory part [1]: 2022.07.13 08:06:54.231033 [ 141093 ] {} InterserverIOHTTPHandler: Code: 107. DB::ErrnoException: Cannot open file /var/lib/clickhouse/disks/s3/store/886/88635b40-e4e3-4fe6-a0a0-1b6755463358/all_0_0_0/data.bin, errno: 2, strerror: No such file or directory. (FILE_DOESNT_EXIST), Stack trace (when copying this message, always include the lines below):
stacktrace 2022.07.13 08:06:54.101825 [ 189342 ] {8dbd11b3-f38a-4d5d-9ded-148987adb71d} executeQuery: (from [::1]:54570) (comment: 01643_replicated_merge_tree_fsync_smoke.sql) select 'memory in_memory_parts_insert_sync'; (stage: Complete) 2022.07.13 08:06:54.131309 [ 691 ] {} test_26u6kx.rep_fsync_r2 (39c3823c-22e5-4c05-9dec-cdffd8872c40): Fetching part all_0_0_0 from /clickhouse/tables/test_26u6kx/rep_fsync/replicas/r1 2022.07.13 08:06:54.231033 [ 141093 ] {} InterserverIOHTTPHandler: Code: 107. DB::ErrnoException: Cannot open file /var/lib/clickhouse/disks/s3/store/886/88635b40-e4e3-4fe6-a0a0-1b6755463358/all_0_0_0/data.bin, errno: 2, strerror: No such file or directory. (FILE_DOESNT_EXIST), Stack trace (when copying this message, always include the lines below): 0. DB::Exception::Exception() @ 0xba0191a in /usr/bin/clickhouse 1. DB::throwFromErrnoWithPath() @ 0xba029ca in /usr/bin/clickhouse 2. DB::OpenedFile::open() const @ 0x156e7fb0 in /usr/bin/clickhouse 3. DB::OpenedFile::getFD() const @ 0x156e8003 in /usr/bin/clickhouse 4. DB::ReadBufferFromFilePReadWithDescriptorsCache::ReadBufferFromFilePReadWithDescriptorsCache() @ 0x156e5f23 in /usr/bin/clickhouse 5. ? @ 0x156e53f0 in /usr/bin/clickhouse 6. DB::createReadBufferFromFileBase() @ 0x156e52b5 in /usr/bin/clickhouse 7. DB::DiskLocal::readFile() const @ 0x15e45ea8 in /usr/bin/clickhouse 8. DB::MetadataStorageFromDisk::readFileToString() const @ 0x15e6ab8b in /usr/bin/clickhouse 9. DB::MetadataStorageFromDisk::readMetadataUnlocked() const @ 0x15e6cdeb in /usr/bin/clickhouse 10. DB::MetadataStorageFromDisk::getSerializedMetadata() const @ 0x15e6cfc4 in /usr/bin/clickhouse 11. DB::DiskObjectStorage::getSerializedMetadata() const @ 0x15e19e2e in /usr/bin/clickhouse 12. DB::DiskDecorator::getSerializedMetadata() const @ 0x15e54ed1 in /usr/bin/clickhouse 13. DB::DiskDecorator::getSerializedMetadata() const @ 0x15e54ed1 in /usr/bin/clickhouse 14. DB::DataPartsExchange::Service::sendPartFromDiskRemoteMeta() @ 0x1700bb9e in /usr/bin/clickhouse 15. DB::DataPartsExchange::Service::processQuery(DB::HTMLForm const&, DB::ReadBuffer&, DB::WriteBuffer&, DB::HTTPServerResponse&) @ 0x1700a649 in /usr/bin/clickhouse 16. DB::InterserverIOHTTPHandler::processQuery(DB::HTTPServerRequest&, DB::HTTPServerResponse&, DB::InterserverIOHTTPHandler::Output&) @ 0x17433c53 in /usr/bin/clickhouse 17. DB::InterserverIOHTTPHandler::handleRequest(DB::HTTPServerRequest&, DB::HTTPServerResponse&) @ 0x174344f1 in /usr/bin/clickhouse 18. DB::HTTPServerConnection::run() @ 0x1768714d in /usr/bin/clickhouse 19. Poco::Net::TCPServerConnection::start() @ 0x1a398093 in /usr/bin/clickhouse 20. Poco::Net::TCPServerDispatcher::run() @ 0x1a399411 in /usr/bin/clickhouse 21. Poco::PooledThread::run() @ 0x1a54b7bb in /usr/bin/clickhouse 22. Poco::ThreadImpl::runnableEntry(void*) @ 0x1a548ec0 in /usr/bin/clickhouse 23. ? @ 0x7fdf1c204609 in ? 24. clone @ 0x7fdf1c129133 in ? (version 22.7.1.1781 (official build))
[1]: https://s3.amazonaws.com/clickhouse-test-reports/0/8b6e31cc615ca52c80724b6e5097777cb9514f07/stateless_tests__release__s3_storage__actions_.html Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/DataPartsExchange.cpp | 2 ++ .../0_stateless/01643_replicated_merge_tree_fsync_smoke.sql | 3 +-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 0e655166abf..9e18dbc6281 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -174,6 +174,8 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write std::sregex_token_iterator()); if (data_settings->allow_remote_fs_zero_copy_replication && + /// In memory data part does not have metadata yet. + !isInMemoryPart(part) && client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_ZERO_COPY) { auto disk_type = part->data_part_storage->getDiskType(); diff --git a/tests/queries/0_stateless/01643_replicated_merge_tree_fsync_smoke.sql b/tests/queries/0_stateless/01643_replicated_merge_tree_fsync_smoke.sql index ee0617e42a3..dadd7eaba6c 100644 --- a/tests/queries/0_stateless/01643_replicated_merge_tree_fsync_smoke.sql +++ b/tests/queries/0_stateless/01643_replicated_merge_tree_fsync_smoke.sql @@ -1,6 +1,5 @@ --- Tags: no-parallel, no-s3-storage +-- Tags: no-parallel -- no-parallel -- for flaky check and to avoid "Removing leftovers from table" (for other tables) --- no-s3-storage -- hangs now, need investigation -- Temporarily skip warning 'table was created by another server at the same moment, will retry' set send_logs_level='error'; From 531e3b13ce5d960a94d9a015dcd07a929bfe98bd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Jul 2022 09:06:47 +0300 Subject: [PATCH 186/659] Optimize accesses to system.stack_trace (filter by name before sending signal) Because of failed rebase/push this part of the patch was lost, so resending separatelly (sigh). Follow-up for: #39177 (cc @yakov-olkhovskiy) Signed-off-by: Azat Khuzhin --- src/Storages/System/StorageSystemStackTrace.cpp | 17 ++++++++++++----- .../01051_system_stack_trace.reference | 3 +++ .../0_stateless/01051_system_stack_trace.sql | 2 ++ 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index 4eef69d8634..cdd04964f55 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -206,10 +206,12 @@ namespace filtered_thread_names.emplace(thread_name); } - for (const auto & [tid, name] : tid_to_name) + for (auto it = tid_to_name.begin(); it != tid_to_name.end();) { - if (!filtered_thread_names.contains(name)) - tid_to_name.erase(tid); + if (!filtered_thread_names.contains(it->second)) + it = tid_to_name.erase(it); + else + ++it; } return tid_to_name; @@ -302,8 +304,13 @@ Pipe StorageSystemStackTrace::read( size_t res_index = 0; String thread_name; - if (auto it = thread_names.find(tid); it != thread_names.end()) - thread_name = it->second; + if (read_thread_names) + { + if (auto it = thread_names.find(tid); it != thread_names.end()) + thread_name = it->second; + else + continue; /// was filtered out by "thread_name" condition + } if (!send_signal) { diff --git a/tests/queries/0_stateless/01051_system_stack_trace.reference b/tests/queries/0_stateless/01051_system_stack_trace.reference index ab01c749f41..b82bda76142 100644 --- a/tests/queries/0_stateless/01051_system_stack_trace.reference +++ b/tests/queries/0_stateless/01051_system_stack_trace.reference @@ -13,3 +13,6 @@ SELECT length(query_id) > 0 FROM system.stack_trace WHERE query_id != '' LIMIT 1 -- optimization for thread_name SELECT length(thread_name) > 0 FROM system.stack_trace WHERE thread_name != '' LIMIT 1; 1 +-- enough rows (optimizations works "correctly") +SELECT count() > 100 FROM system.stack_trace; +1 diff --git a/tests/queries/0_stateless/01051_system_stack_trace.sql b/tests/queries/0_stateless/01051_system_stack_trace.sql index c712a1367c0..d018d01fa22 100644 --- a/tests/queries/0_stateless/01051_system_stack_trace.sql +++ b/tests/queries/0_stateless/01051_system_stack_trace.sql @@ -10,3 +10,5 @@ SELECT length(trace) > 0 FROM system.stack_trace LIMIT 1; SELECT length(query_id) > 0 FROM system.stack_trace WHERE query_id != '' LIMIT 1; -- optimization for thread_name SELECT length(thread_name) > 0 FROM system.stack_trace WHERE thread_name != '' LIMIT 1; +-- enough rows (optimizations works "correctly") +SELECT count() > 100 FROM system.stack_trace; From 35d3a0dd1fe6783a73c94060231fb971e5752fa7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 14 Jul 2022 11:08:26 +0200 Subject: [PATCH 187/659] Don't report system.errors when the disk is not rotational --- src/Common/filesystemHelpers.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 00764269269..6f6afcac657 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -87,7 +87,10 @@ BlockDeviceType getBlockDeviceType([[maybe_unused]] const String & device_id) #if defined(OS_LINUX) try { - ReadBufferFromFile in("/sys/dev/block/" + device_id + "/queue/rotational"); + const String path = "/sys/dev/block/" + device_id + "/queue/rotational"; + if (!std::filesystem::exists(path)) + return BlockDeviceType::UNKNOWN; + ReadBufferFromFile in(path); int rotational; readText(rotational, in); return rotational ? BlockDeviceType::ROT : BlockDeviceType::NONROT; From 6d85dcd8a80a2b2a2d73b7bc6452eb811dc5d545 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 14 Jul 2022 12:51:50 +0200 Subject: [PATCH 188/659] Update src/Functions/isNotNull.cpp Co-authored-by: Igor Nikonov <954088+devcrafter@users.noreply.github.com> --- src/Functions/isNotNull.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/isNotNull.cpp b/src/Functions/isNotNull.cpp index dd6f7de198b..87437f49356 100644 --- a/src/Functions/isNotNull.cpp +++ b/src/Functions/isNotNull.cpp @@ -48,7 +48,7 @@ public: if (elem.type->isLowCardinalityNullable()) { const auto * low_cardinality_column = checkAndGetColumn(*elem.column); - size_t null_index = low_cardinality_column->getDictionary().getNullValueIndex(); + const size_t null_index = low_cardinality_column->getDictionary().getNullValueIndex(); auto res = DataTypeUInt8().createColumn(); auto & data = typeid_cast(*res).getData(); for (size_t i = 0; i != low_cardinality_column->size(); ++i) From 1f7fe10313c9f56052e8ee4ca9a560c3c55286be Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 14 Jul 2022 12:54:14 +0200 Subject: [PATCH 189/659] Update low_cardinality_argument.xml --- tests/performance/low_cardinality_argument.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/performance/low_cardinality_argument.xml b/tests/performance/low_cardinality_argument.xml index 318cfa080cf..8ba7b6d09be 100644 --- a/tests/performance/low_cardinality_argument.xml +++ b/tests/performance/low_cardinality_argument.xml @@ -5,8 +5,8 @@ INSERT INTO test_lc SELECT number, number % 10 ? NULL : toString(number % 10000) FROM numbers(1000000) - SELECT isNullable(lc) FROM test_lc - SELECT isConstant(lc) FROM test_lc - SELECT isNull(lc) FROM test_lc - SELECT isNotNull(lc) FROM test_lc + SELECT isNullable(lc) FROM test_lc FORMAT Null + SELECT isConstant(lc) FROM test_lc FORMAT Null + SELECT isNull(lc) FROM test_lc FORMAT Null + SELECT isNotNull(lc) FROM test_lc FORMAT Null From ac5a06d944845cac03299be2ab2ce2cc070c5c4e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 11:00:51 +0000 Subject: [PATCH 190/659] Update doxygen --- src/Functions/FunctionsMultiStringFuzzySearch.h | 6 ++++++ src/Functions/FunctionsMultiStringSearch.h | 5 +++++ 2 files changed, 11 insertions(+) diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index 5c0c3e12d2a..f6ec642fd9b 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -23,6 +23,12 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } +/** + * multiFuzzyMatchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiFuzzyMatchAnyIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiFuzzyMatchAllIndices(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * + */ template class FunctionsMultiStringFuzzySearch : public IFunction diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index 3dd1646c051..2465567b883 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -30,6 +30,11 @@ namespace DB * multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) * multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) * multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * + * multiSearchFirstPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the leftmost offset of the matched string or zero if nothing was found + * multiSearchFirstPositionUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchFirstPositionCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchFirstPositionCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) */ namespace ErrorCodes From 51533adee8530300a773e069e1e24c284d833ddb Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 14 Jul 2022 11:33:23 +0000 Subject: [PATCH 191/659] fix build --- src/Storages/tests/gtest_storage_log.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index f5fdb606018..3fa2f93b484 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -40,7 +40,7 @@ DB::StoragePtr createStorage(DB::DiskPtr & disk) StoragePtr table = std::make_shared( "Log", disk, "table/", StorageID("test", "test"), ColumnsDescription{names_and_types}, - ConstraintsDescription{}, String{}, false, static_cast(1048576)); + ConstraintsDescription{}, String{}, false, getContext().context); table->startup(); From 51fab31f92bff945dcaeaf4f004122690f952499 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Jul 2022 12:07:32 +0000 Subject: [PATCH 192/659] Add comments and description --- src/Core/Settings.h | 6 +++++- src/Core/SettingsChangesHistory.h | 4 ++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c00f04356bf..79848b3cc1b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -35,6 +35,10 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) * * `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. + * + * When adding new settings that control some backward incompatible changes or when changing some settings values, + * consider adding them to settings changes history in SettingsChangesHistory.h for special `compatibility` setting + * to work correctly. */ #define COMMON_SETTINGS(M) \ @@ -593,7 +597,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, allow_deprecated_database_ordinary, false, "Allow to create databases with deprecated Ordinary engine", 0) \ M(Bool, allow_deprecated_syntax_for_merge_tree, false, "Allow to create *MergeTree tables with deprecated engine definition syntax", 0) \ \ - M(String, compatibility, "", "Azaz", 0) \ + M(String, compatibility, "", "Change other settings according to provided ClickHouse version. If we know that we changed some behaviour in ClickHouse by changing some settings in some version, this compatibility setting will control these changed", 0) \ \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 8ccb6ebf86f..c5bd9623aaa 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -69,6 +69,10 @@ struct SettingChangesHistory std::vector changes; }; +/// History of settings changes that controls some backward incompatible changes +/// across all ClickHouse versions. It maps setting name to special struct +/// SettingChangesHistory {initial_value, {{changed_value_1, version1}, {changed_value_2, version_2}, ...}} +/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) const std::unordered_map settings_changes_history = { {"enable_positional_arguments", {false, {{true, "22.7"}}}}, From 83638e34a721c1b5ec285b53151af70846807233 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Jul 2022 12:13:35 +0000 Subject: [PATCH 193/659] Add docs --- docs/en/operations/settings/settings.md | 9 +++++++++ src/Core/Settings.h | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 75c2aa57b32..f40683dd25b 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3329,6 +3329,15 @@ Read more about [memory overcommit](memory-overcommit.md). Default value: `1GiB`. +## compatibility {#compatibility} + +This setting changes other settings according to provided ClickHouse version. +If we know that we changed some behaviour in ClickHouse by changing some settings in some version, this compatibility setting will control these settings if they were not set by user. + +This setting takes ClickHouse version number as a string, like `21.3`, `21.8`. Empty value means that this setting is disabled. + +Disabled by default. + # Format settings {#format-settings} ## input_format_skip_unknown_fields {#input_format_skip_unknown_fields} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 79848b3cc1b..d114fd44a69 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -597,7 +597,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, allow_deprecated_database_ordinary, false, "Allow to create databases with deprecated Ordinary engine", 0) \ M(Bool, allow_deprecated_syntax_for_merge_tree, false, "Allow to create *MergeTree tables with deprecated engine definition syntax", 0) \ \ - M(String, compatibility, "", "Change other settings according to provided ClickHouse version. If we know that we changed some behaviour in ClickHouse by changing some settings in some version, this compatibility setting will control these changed", 0) \ + M(String, compatibility, "", "Changes other settings according to provided ClickHouse version. If we know that we changed some behaviour in ClickHouse by changing some settings in some version, this compatibility setting will control these settings", 0) \ \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ From 8fba500a3a84eb1fe40694d8ddb3483e1ca8c948 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Jul 2022 12:35:32 +0000 Subject: [PATCH 194/659] Pass string_view by value --- src/Core/BaseSettings.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index d8b41109d5e..893c2cfb19e 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -45,7 +45,9 @@ class BaseSettings : public TTraits::Data public: BaseSettings() = default; BaseSettings(const BaseSettings &) = default; + BaseSettings(BaseSettings &&) noexcept = default; BaseSettings & operator=(const BaseSettings &) = default; + BaseSettings & operator=(BaseSettings &&) noexcept = default; virtual ~BaseSettings() = default; using Traits = TTraits; @@ -68,7 +70,7 @@ public: /// Resets all the settings to their default values. void resetToDefault(); /// Resets specified setting to its default value. - void resetToDefault(const std::string_view & name); + void resetToDefault(const std::string_view name); bool has(const std::string_view & name) const { return hasBuiltin(name) || hasCustom(name); } static bool hasBuiltin(const std::string_view & name); @@ -323,7 +325,7 @@ void BaseSettings::resetToDefault() } template -void BaseSettings::resetToDefault(const std::string_view & name) +void BaseSettings::resetToDefault(const std::string_view name) { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) From da4724e7f5701c3fb0478ff1b479b5a772f27c54 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 09:13:08 +0000 Subject: [PATCH 195/659] Enable -Wreturn-std-move-in-c++11 --- cmake/warnings.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index b8fc4229ad9..60b50ae8b38 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -40,7 +40,6 @@ if (COMPILER_CLANG) no_warning(nested-anon-types) no_warning(packed) no_warning(padded) - no_warning(return-std-move-in-c++11) # clang 7+ no_warning(shift-sign-overflow) no_warning(sign-conversion) no_warning(switch-enum) From 439b78615612770afc714ad137d4db59c4b20b78 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 09:54:00 +0000 Subject: [PATCH 196/659] Enable -Wunused-command-line-argument --- cmake/warnings.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 60b50ae8b38..10ccdcc5305 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -23,7 +23,6 @@ if (COMPILER_CLANG) no_warning(vla-extension) no_warning(zero-length-array) no_warning(c11-extensions) - no_warning(unused-command-line-argument) no_warning(c++98-compat-pedantic) no_warning(c++98-compat) no_warning(c99-extensions) From 72d45bdff3d7bc097586ea7271f9c08659c7c420 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 10:17:32 +0000 Subject: [PATCH 197/659] Enable -Wvla-extension --- cmake/warnings.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 10ccdcc5305..9e59b96084c 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -20,7 +20,6 @@ if (COMPILER_CLANG) # We want to get everything out of the compiler for code quality. add_warning(everything) add_warning(pedantic) - no_warning(vla-extension) no_warning(zero-length-array) no_warning(c11-extensions) no_warning(c++98-compat-pedantic) From 3d734a0640891dd153f5366aefa16c7db8f2cbb3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 11:42:25 +0000 Subject: [PATCH 198/659] Enable -Wnested-anon-types --- cmake/warnings.cmake | 1 - src/Common/TaskStatsInfoGetter.cpp | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 9e59b96084c..a4e9adfba12 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -35,7 +35,6 @@ if (COMPILER_CLANG) no_warning(global-constructors) no_warning(missing-prototypes) no_warning(missing-variable-declarations) - no_warning(nested-anon-types) no_warning(packed) no_warning(padded) no_warning(shift-sign-overflow) diff --git a/src/Common/TaskStatsInfoGetter.cpp b/src/Common/TaskStatsInfoGetter.cpp index 36e8a0fce00..304ccc84765 100644 --- a/src/Common/TaskStatsInfoGetter.cpp +++ b/src/Common/TaskStatsInfoGetter.cpp @@ -21,6 +21,7 @@ #if defined(__clang__) #pragma clang diagnostic ignored "-Wgnu-anonymous-struct" + #pragma clang diagnostic ignored "-Wnested-anon-types" #endif /// Basic idea is motivated by "iotop" tool. From e7bc973ffb81143572179452e9d98f503b63dcde Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 12:29:42 +0000 Subject: [PATCH 199/659] Enable -Wshift-sign-overflow --- cmake/warnings.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index a4e9adfba12..ea279870b1d 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -37,7 +37,6 @@ if (COMPILER_CLANG) no_warning(missing-variable-declarations) no_warning(packed) no_warning(padded) - no_warning(shift-sign-overflow) no_warning(sign-conversion) no_warning(switch-enum) no_warning(undefined-func-template) From 97aa9a73710d8d361e3b91a86e12e613fd60a44a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 12:29:54 +0000 Subject: [PATCH 200/659] Enable -Wpacked --- cmake/warnings.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index ea279870b1d..7da73b0dd08 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -35,7 +35,6 @@ if (COMPILER_CLANG) no_warning(global-constructors) no_warning(missing-prototypes) no_warning(missing-variable-declarations) - no_warning(packed) no_warning(padded) no_warning(sign-conversion) no_warning(switch-enum) From 32e48f661ad1b6fa26183fc1050982f9effb6794 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 14 Jul 2022 12:51:56 +0000 Subject: [PATCH 201/659] Fixed query_log result_bytes metric that showed inseted rows instead of bytes --- src/Interpreters/executeQuery.cpp | 2 +- .../0_stateless/02356_insert_query_log_metrics.reference | 1 + tests/queries/0_stateless/02356_insert_query_log_metrics.sql | 4 ++++ 3 files changed, 6 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02356_insert_query_log_metrics.reference create mode 100644 tests/queries/0_stateless/02356_insert_query_log_metrics.sql diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 85c4ea261a0..ae915aab867 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -876,7 +876,7 @@ static std::tuple executeQueryImpl( { auto progress_out = process_list_elem->getProgressOut(); elem.result_rows = progress_out.written_rows; - elem.result_bytes = progress_out.written_rows; + elem.result_bytes = progress_out.written_bytes; } if (elem.read_rows != 0) diff --git a/tests/queries/0_stateless/02356_insert_query_log_metrics.reference b/tests/queries/0_stateless/02356_insert_query_log_metrics.reference new file mode 100644 index 00000000000..0d707252cc7 --- /dev/null +++ b/tests/queries/0_stateless/02356_insert_query_log_metrics.reference @@ -0,0 +1 @@ +1,1,1,1 diff --git a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql new file mode 100644 index 00000000000..3825f37183b --- /dev/null +++ b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql @@ -0,0 +1,4 @@ +CREATE TABLE 02356_destination (a Int64, b String) ENGINE = Memory; + +INSERT INTO 02356_destination (a, b) SELECT * FROM generateRandom('a Int64, b String') LIMIT 100 SETTINGS max_threads=1, max_block_size=100; +SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' FORMAT CSV; \ No newline at end of file From 9955b097d405346d86c33a781fba70b4635b3c33 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 14 Jul 2022 09:00:54 -0400 Subject: [PATCH 202/659] spelling --- docs/en/sql-reference/table-functions/null.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/table-functions/null.md b/docs/en/sql-reference/table-functions/null.md index ae84705cb66..57a885afc54 100644 --- a/docs/en/sql-reference/table-functions/null.md +++ b/docs/en/sql-reference/table-functions/null.md @@ -5,7 +5,7 @@ sidebar_label: null function # null -Creates a temporary table of the specified structure with the [Null](../../engines/table-engines/special/null.md) table engine. According to the `Null`-engine properties, the table data is ignored and the table itself is immediately droped right after the query execution. The function is used for the convenience of test writing and demonstrations. +Creates a temporary table of the specified structure with the [Null](../../engines/table-engines/special/null.md) table engine. According to the `Null`-engine properties, the table data is ignored and the table itself is immediately dropped right after the query execution. The function is used for the convenience of test writing and demonstrations. **Syntax** @@ -40,4 +40,3 @@ See also: - [Null table engine](../../engines/table-engines/special/null.md) -[Original article](https://clickhouse.com/docs/en/sql-reference/table-functions/null/) From 96bb6e0cd2d6195a2bddd684195aecab899b0d13 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 14 Jul 2022 21:00:11 +0800 Subject: [PATCH 203/659] Rename log when rename merge tree tables --- src/Common/logger_useful.h | 15 +++--- src/Storages/MergeTree/MergeTreeData.cpp | 46 ++++++++++++++----- src/Storages/MergeTree/MergeTreeData.h | 11 +++-- ...rename_table_along_with_log_name.reference | 0 .../02360_rename_table_along_with_log_name.sh | 18 ++++++++ 5 files changed, 69 insertions(+), 21 deletions(-) create mode 100644 tests/queries/0_stateless/02360_rename_table_along_with_log_name.reference create mode 100755 tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh diff --git a/src/Common/logger_useful.h b/src/Common/logger_useful.h index ad7d6583f5e..1e84efd8085 100644 --- a/src/Common/logger_useful.h +++ b/src/Common/logger_useful.h @@ -14,8 +14,10 @@ namespace template constexpr auto firstArg(T && x, Ts &&...) { return std::forward(x); } /// For implicit conversion of fmt::basic_runtime<> to char* for std::string ctor template constexpr auto firstArg(fmt::basic_runtime && data, Ts &&...) { return data.str.data(); } -} + [[maybe_unused]] const ::Poco::Logger * getLogger(const ::Poco::Logger * logger) { return logger; }; + [[maybe_unused]] const ::Poco::Logger * getLogger(const std::atomic<::Poco::Logger *> & logger) { return logger.load(); }; +} /// Logs a message to a specified logger with that level. /// If more than one argument is provided, @@ -25,20 +27,21 @@ namespace #define LOG_IMPL(logger, priority, PRIORITY, ...) do \ { \ - const bool is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \ + auto _logger = ::getLogger(logger); \ + const bool _is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \ (DB::CurrentThread::getGroup()->client_logs_level >= (priority)); \ - if ((logger)->is((PRIORITY)) || is_clients_log) \ + if (_logger->is((PRIORITY)) || _is_clients_log) \ { \ std::string formatted_message = numArgs(__VA_ARGS__) > 1 ? fmt::format(__VA_ARGS__) : firstArg(__VA_ARGS__); \ - if (auto channel = (logger)->getChannel()) \ + if (auto _channel = _logger->getChannel()) \ { \ std::string file_function; \ file_function += __FILE__; \ file_function += "; "; \ file_function += __PRETTY_FUNCTION__; \ - Poco::Message poco_message((logger)->name(), formatted_message, \ + Poco::Message poco_message(_logger->name(), formatted_message, \ (PRIORITY), file_function.c_str(), __LINE__); \ - channel->log(poco_message); \ + _channel->log(poco_message); \ } \ } \ } while (false) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 5900ea0fdb7..ef315e865ec 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -216,8 +216,8 @@ MergeTreeData::MergeTreeData( , require_part_metadata(require_part_metadata_) , relative_data_path(relative_data_path_) , broken_part_callback(broken_part_callback_) - , log_name(table_id_.getNameForLogs()) - , log(&Poco::Logger::get(log_name)) + , log_name(std::make_shared(table_id_.getNameForLogs())) + , log(&Poco::Logger::get(*log_name)) , storage_settings(std::move(storage_settings_)) , pinned_part_uuids(std::make_shared()) , data_parts_by_info(data_parts_indexes.get()) @@ -2033,8 +2033,13 @@ void MergeTreeData::rename(const String & new_table_path, const StorageID & new_ relative_data_path = new_table_path; renameInMemory(new_table_id); +} - +void MergeTreeData::renameInMemory(const StorageID & new_table_id) +{ + IStorage::renameInMemory(new_table_id); + std::atomic_store(&log_name, std::make_shared(new_table_id.getNameForLogs())); + log = &Poco::Logger::get(*log_name); } void MergeTreeData::dropAllData() @@ -4734,11 +4739,19 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules( if (!destination_ptr) { if (move_ttl_entry->destination_type == DataDestinationType::VOLUME && !move_ttl_entry->if_exists) - LOG_WARNING(log, "Would like to reserve space on volume '{}' by TTL rule of table '{}' but volume was not found or rule is not applicable at the moment", - move_ttl_entry->destination_name, log_name); + LOG_WARNING( + log, + "Would like to reserve space on volume '{}' by TTL rule of table '{}' but volume was not found or rule is not " + "applicable at the moment", + move_ttl_entry->destination_name, + *std::atomic_load(&log_name)); else if (move_ttl_entry->destination_type == DataDestinationType::DISK && !move_ttl_entry->if_exists) - LOG_WARNING(log, "Would like to reserve space on disk '{}' by TTL rule of table '{}' but disk was not found or rule is not applicable at the moment", - move_ttl_entry->destination_name, log_name); + LOG_WARNING( + log, + "Would like to reserve space on disk '{}' by TTL rule of table '{}' but disk was not found or rule is not applicable " + "at the moment", + move_ttl_entry->destination_name, + *std::atomic_load(&log_name)); } else { @@ -4747,11 +4760,17 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules( return reservation; else if (move_ttl_entry->destination_type == DataDestinationType::VOLUME) - LOG_WARNING(log, "Would like to reserve space on volume '{}' by TTL rule of table '{}' but there is not enough space", - move_ttl_entry->destination_name, log_name); + LOG_WARNING( + log, + "Would like to reserve space on volume '{}' by TTL rule of table '{}' but there is not enough space", + move_ttl_entry->destination_name, + *std::atomic_load(&log_name)); else if (move_ttl_entry->destination_type == DataDestinationType::DISK) - LOG_WARNING(log, "Would like to reserve space on disk '{}' by TTL rule of table '{}' but there is not enough space", - move_ttl_entry->destination_name, log_name); + LOG_WARNING( + log, + "Would like to reserve space on disk '{}' by TTL rule of table '{}' but there is not enough space", + move_ttl_entry->destination_name, + *std::atomic_load(&log_name)); } } @@ -6590,7 +6609,10 @@ bool MergeTreeData::insertQueryIdOrThrowNoLock(const String & query_id, size_t m return false; if (query_id_set.size() >= max_queries) throw Exception( - ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, "Too many simultaneous queries for table {}. Maximum is: {}", log_name, max_queries); + ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, + "Too many simultaneous queries for table {}. Maximum is: {}", + *std::atomic_load(&log_name), + max_queries); query_id_set.insert(query_id); return true; } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 0b6e757ab49..be338d52978 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -456,7 +456,7 @@ public: /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks); - String getLogName() const { return log_name; } + String getLogName() const { return *std::atomic_load(&log_name); } Int64 getMaxBlockNumber() const; @@ -649,6 +649,9 @@ public: /// because changes relative_data_path. void rename(const String & new_table_path, const StorageID & new_table_id) override; + /// Also rename log names. + void renameInMemory(const StorageID & new_table_id) override; + /// Check if the ALTER can be performed: /// - all needed columns are present. /// - all type conversions can be done. @@ -1021,8 +1024,10 @@ protected: /// Engine-specific methods BrokenPartCallback broken_part_callback; - String log_name; - Poco::Logger * log; + /// log_name will change during table RENAME. Use atomic_shared_ptr to allow concurrent RW. + /// NOTE clang-14 doesn't have atomic_shared_ptr yet. Use std::atomic* operations for now. + std::shared_ptr log_name; + std::atomic log; /// Storage settings. /// Use get and set to receive readonly versions. diff --git a/tests/queries/0_stateless/02360_rename_table_along_with_log_name.reference b/tests/queries/0_stateless/02360_rename_table_along_with_log_name.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh b/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh new file mode 100755 index 00000000000..e8c7f844b5c --- /dev/null +++ b/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +[ ! -z "$CLICKHOUSE_CLIENT_REDEFINED" ] && CLICKHOUSE_CLIENT=$CLICKHOUSE_CLIENT_REDEFINED + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS x;" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS y;" +$CLICKHOUSE_CLIENT -q "CREATE TABLE x(i int) ENGINE MergeTree ORDER BY i;" +$CLICKHOUSE_CLIENT -q "RENAME TABLE x TO y;" + +CLICKHOUSE_CLIENT_WITH_LOG=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') +regexp="${CLICKHOUSE_DATABASE}\\.x" # Check if there are still log entries with old table name +$CLICKHOUSE_CLIENT_WITH_LOG --send_logs_source_regexp "$regexp" -q "INSERT INTO y VALUES(1);" + +$CLICKHOUSE_CLIENT -q "DROP TABLE y;" From 0804ff9284fcbc74fa9cd6ea6408ade7379e7db6 Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Thu, 14 Jul 2022 07:12:59 -0700 Subject: [PATCH 204/659] Update as suggested --- contrib/thrift-cmake/CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index 3454b1b87e3..832e104e8a3 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -92,5 +92,5 @@ include_directories("${CMAKE_CURRENT_BINARY_DIR}") add_library(_thrift ${thriftcpp_SOURCES} ${thriftcpp_threads_SOURCES}) add_library(ch_contrib::thrift ALIAS _thrift) -target_include_directories(_thrift SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src" ${CMAKE_CURRENT_BINARY_DIR} "${ClickHouse_SOURCE_DIR}/contrib/boringssl/include") -target_link_libraries (_thrift PUBLIC boost::headers_only) +target_include_directories(_thrift SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src" ${CMAKE_CURRENT_BINARY_DIR}) +target_link_libraries (_thrift PUBLIC OpenSSL::SSL) From b20407fab9e5e4e7dd3e1298a01191d6afe1718b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Jul 2022 16:46:01 +0300 Subject: [PATCH 205/659] Fix NOEXCEPT_SCOPE (before it calls std::terminate and looses the exception) Current implementation of NOEXCEPT_SCOPE will not work, you cannot rethrow exception outside the catch block, this will simply terminate (via std::terminate) the program. In other words NOEXCEPT_SCOPE macro will simply call std::terminate on exception and will lost original exception. But if NOEXCEPT_SCOPE will accept the code that should be runned w/o exceptions, then it can catch exception and log it, rewrite it in this way. Signed-off-by: Azat Khuzhin --- src/Common/noexcept_scope.h | 46 +++---- src/Interpreters/DatabaseCatalog.cpp | 9 +- src/Interpreters/MergeTreeTransaction.cpp | 11 +- src/Interpreters/ThreadStatusExt.cpp | 2 +- src/Interpreters/TransactionLog.cpp | 15 +-- .../MergeTree/MergeTreeBackgroundExecutor.cpp | 61 +++++---- src/Storages/MergeTree/MergeTreeData.cpp | 118 +++++++++--------- 7 files changed, 132 insertions(+), 130 deletions(-) diff --git a/src/Common/noexcept_scope.h b/src/Common/noexcept_scope.h index 56fb44ff0bf..bdd7a98925a 100644 --- a/src/Common/noexcept_scope.h +++ b/src/Common/noexcept_scope.h @@ -1,36 +1,28 @@ #pragma once -#include #include #include - -#define NOEXCEPT_SCOPE_IMPL_CONCAT(n, expected) \ - LockMemoryExceptionInThread lock_memory_tracker##n(VariableContext::Global); \ - SCOPE_EXIT( \ - { \ - const auto uncaught = std::uncaught_exceptions(); \ - assert((expected) == uncaught || (expected) + 1 == uncaught); \ - if ((expected) < uncaught) \ - { \ - tryLogCurrentException("NOEXCEPT_SCOPE"); \ - abort(); \ - } \ - } \ - ) - -#define NOEXCEPT_SCOPE_IMPL(n, expected) NOEXCEPT_SCOPE_IMPL_CONCAT(n, expected) - -#define NOEXCEPT_SCOPE_CONCAT(n) \ - const auto num_curr_exceptions##n = std::uncaught_exceptions(); \ - NOEXCEPT_SCOPE_IMPL(n, num_curr_exceptions##n) - -#define NOEXCEPT_SCOPE_FWD(n) NOEXCEPT_SCOPE_CONCAT(n) - - /// It can be used in critical places to exit on unexpected exceptions. /// SIGABRT is usually better that broken in-memory state with unpredictable consequences. /// It also temporarily disables exception from memory tracker in current thread. /// Strict version does not take into account nested exception (i.e. it aborts even when we're in catch block). -#define NOEXCEPT_SCOPE_STRICT NOEXCEPT_SCOPE_IMPL(__LINE__, 0) -#define NOEXCEPT_SCOPE NOEXCEPT_SCOPE_FWD(__LINE__) +#define NOEXCEPT_SCOPE_IMPL(...) do { \ + LockMemoryExceptionInThread \ + noexcept_lock_memory_tracker(VariableContext::Global); \ + try \ + { \ + __VA_ARGS__; \ + } \ + catch (...) \ + { \ + DB::tryLogCurrentException(__PRETTY_FUNCTION__); \ + std::terminate(); \ + } \ +} while (0) /* to allow leading semi-colon */ + +#define NOEXCEPT_SCOPE_STRICT(...) \ + if (std::uncaught_exceptions()) std::terminate(); \ + NOEXCEPT_SCOPE_IMPL(__VA_ARGS__) + +#define NOEXCEPT_SCOPE(...) NOEXCEPT_SCOPE_IMPL(__VA_ARGS__) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index bae2aed2cd5..aefba3aaa2d 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -403,10 +403,11 @@ void DatabaseCatalog::attachDatabase(const String & database_name, const Databas std::lock_guard lock{databases_mutex}; assertDatabaseDoesntExistUnlocked(database_name); databases.emplace(database_name, database); - NOEXCEPT_SCOPE; - UUID db_uuid = database->getUUID(); - if (db_uuid != UUIDHelpers::Nil) - addUUIDMapping(db_uuid, database, nullptr); + NOEXCEPT_SCOPE({ + UUID db_uuid = database->getUUID(); + if (db_uuid != UUIDHelpers::Nil) + addUUIDMapping(db_uuid, database, nullptr); + }); } diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 432116feaf5..2a57d637333 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -149,11 +149,12 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP checkIsNotCancelled(); part_to_remove->version.lockRemovalTID(tid, context); - NOEXCEPT_SCOPE; - storages.insert(storage); - if (maybe_lock) - table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock)); - removing_parts.push_back(part_to_remove); + NOEXCEPT_SCOPE({ + storages.insert(storage); + if (maybe_lock) + table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock)); + removing_parts.push_back(part_to_remove); + }); } part_to_remove->appendRemovalTIDToVersionMetadata(); diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 53d7fd0457a..9ad3dc7c4a9 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -343,7 +343,7 @@ void ThreadStatus::finalizeQueryProfiler() void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) { - NOEXCEPT_SCOPE; + LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); if (exit_if_already_detached && thread_state == ThreadState::DetachedFromQuery) { diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index a08f940a748..ec7969952f5 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -140,8 +140,7 @@ void TransactionLog::loadEntries(Strings::const_iterator beg, Strings::const_ite } futures.clear(); - NOEXCEPT_SCOPE_STRICT; - { + NOEXCEPT_SCOPE_STRICT({ std::lock_guard lock{mutex}; for (const auto & entry : loaded) { @@ -151,7 +150,8 @@ void TransactionLog::loadEntries(Strings::const_iterator beg, Strings::const_ite tid_to_csn.emplace(entry.first, entry.second); } last_loaded_entry = last_entry; - } + }); + { std::lock_guard lock{running_list_mutex}; latest_snapshot = loaded.back().second.csn; @@ -445,10 +445,11 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn, bool /// Do not allow exceptions between commit point and the and of transaction finalization /// (otherwise it may stuck in COMMITTING state holding snapshot). - NOEXCEPT_SCOPE_STRICT; - /// FIXME Transactions: Sequential node numbers in ZooKeeper are Int32, but 31 bit is not enough for production use - /// (overflow is possible in a several weeks/months of active usage) - allocated_csn = deserializeCSN(csn_path_created.substr(zookeeper_path_log.size() + 1)); + NOEXCEPT_SCOPE_STRICT({ + /// FIXME Transactions: Sequential node numbers in ZooKeeper are Int32, but 31 bit is not enough for production use + /// (overflow is possible in a several weeks/months of active usage) + allocated_csn = deserializeCSN(csn_path_created.substr(zookeeper_path_log.size() + 1)); + }); } return finalizeCommittedTransaction(txn.get(), allocated_csn, state_guard); diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index c3f83771338..1f7ecb39a1d 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -138,18 +138,20 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) } catch (const Exception & e) { - NOEXCEPT_SCOPE; - ALLOW_ALLOCATIONS_IN_SCOPE; - if (e.code() == ErrorCodes::ABORTED) /// Cancelled merging parts is not an error - log as info. - LOG_INFO(log, fmt::runtime(getCurrentExceptionMessage(false))); - else - tryLogCurrentException(__PRETTY_FUNCTION__); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + if (e.code() == ErrorCodes::ABORTED) /// Cancelled merging parts is not an error - log as info. + LOG_INFO(log, fmt::runtime(getCurrentExceptionMessage(false))); + else + tryLogCurrentException(__PRETTY_FUNCTION__); + }); } catch (...) { - NOEXCEPT_SCOPE; - ALLOW_ALLOCATIONS_IN_SCOPE; - tryLogCurrentException(__PRETTY_FUNCTION__); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + tryLogCurrentException(__PRETTY_FUNCTION__); + }); } if (need_execute_again) @@ -162,9 +164,10 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) /// This is significant to order the destructors. { - NOEXCEPT_SCOPE; - ALLOW_ALLOCATIONS_IN_SCOPE; - item->task.reset(); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + item->task.reset(); + }); } item->is_done.set(); item = nullptr; @@ -197,18 +200,20 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) } catch (const Exception & e) { - NOEXCEPT_SCOPE; - ALLOW_ALLOCATIONS_IN_SCOPE; - if (e.code() == ErrorCodes::ABORTED) /// Cancelled merging parts is not an error - log as info. - LOG_INFO(log, fmt::runtime(getCurrentExceptionMessage(false))); - else - tryLogCurrentException(__PRETTY_FUNCTION__); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + if (e.code() == ErrorCodes::ABORTED) /// Cancelled merging parts is not an error - log as info. + LOG_INFO(log, fmt::runtime(getCurrentExceptionMessage(false))); + else + tryLogCurrentException(__PRETTY_FUNCTION__); + }); } catch (...) { - NOEXCEPT_SCOPE; - ALLOW_ALLOCATIONS_IN_SCOPE; - tryLogCurrentException(__PRETTY_FUNCTION__); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + tryLogCurrentException(__PRETTY_FUNCTION__); + }); } @@ -218,9 +223,10 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) /// The thread that shutdowns storage will scan queues in order to find some tasks to wait for, but will find nothing. /// So, the destructor of a task and the destructor of a storage will be executed concurrently. { - NOEXCEPT_SCOPE; - ALLOW_ALLOCATIONS_IN_SCOPE; - item->task.reset(); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + item->task.reset(); + }); } item->is_done.set(); @@ -256,9 +262,10 @@ void MergeTreeBackgroundExecutor::threadFunction() } catch (...) { - NOEXCEPT_SCOPE; - ALLOW_ALLOCATIONS_IN_SCOPE; - tryLogCurrentException(__PRETTY_FUNCTION__); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + tryLogCurrentException(__PRETTY_FUNCTION__); + }); } } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 5900ea0fdb7..b823899d3fe 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4942,78 +4942,78 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData: } } - NOEXCEPT_SCOPE; + NOEXCEPT_SCOPE({ + auto current_time = time(nullptr); - auto current_time = time(nullptr); + size_t add_bytes = 0; + size_t add_rows = 0; + size_t add_parts = 0; - size_t add_bytes = 0; - size_t add_rows = 0; - size_t add_parts = 0; + size_t reduce_bytes = 0; + size_t reduce_rows = 0; + size_t reduce_parts = 0; - size_t reduce_bytes = 0; - size_t reduce_rows = 0; - size_t reduce_parts = 0; - - for (const DataPartPtr & part : precommitted_parts) - { - auto part_in_memory = asInMemoryPart(part); - if (part_in_memory && settings->in_memory_parts_enable_wal) + for (const DataPartPtr & part : precommitted_parts) { - if (!wal) - wal = data.getWriteAheadLog(); - - wal->addPart(part_in_memory); - } - - DataPartPtr covering_part; - DataPartsVector covered_parts = data.getActivePartsToReplace(part->info, part->name, covering_part, *owing_parts_lock); - if (covering_part) - { - LOG_WARNING(data.log, "Tried to commit obsolete part {} covered by {}", part->name, covering_part->getNameWithState()); - - part->remove_time.store(0, std::memory_order_relaxed); /// The part will be removed without waiting for old_parts_lifetime seconds. - data.modifyPartState(part, DataPartState::Outdated); - } - else - { - if (!txn) - MergeTreeTransaction::addNewPartAndRemoveCovered(data.shared_from_this(), part, covered_parts, NO_TRANSACTION_RAW); - - total_covered_parts.insert(total_covered_parts.end(), covered_parts.begin(), covered_parts.end()); - for (const auto & covered_part : covered_parts) + auto part_in_memory = asInMemoryPart(part); + if (part_in_memory && settings->in_memory_parts_enable_wal) { - covered_part->remove_time.store(current_time, std::memory_order_relaxed); + if (!wal) + wal = data.getWriteAheadLog(); - reduce_bytes += covered_part->getBytesOnDisk(); - reduce_rows += covered_part->rows_count; - - data.modifyPartState(covered_part, DataPartState::Outdated); - data.removePartContributionToColumnAndSecondaryIndexSizes(covered_part); + wal->addPart(part_in_memory); } - reduce_parts += covered_parts.size(); + DataPartPtr covering_part; + DataPartsVector covered_parts = data.getActivePartsToReplace(part->info, part->name, covering_part, *owing_parts_lock); + if (covering_part) + { + LOG_WARNING(data.log, "Tried to commit obsolete part {} covered by {}", part->name, covering_part->getNameWithState()); - add_bytes += part->getBytesOnDisk(); - add_rows += part->rows_count; - ++add_parts; + part->remove_time.store(0, std::memory_order_relaxed); /// The part will be removed without waiting for old_parts_lifetime seconds. + data.modifyPartState(part, DataPartState::Outdated); + } + else + { + if (!txn) + MergeTreeTransaction::addNewPartAndRemoveCovered(data.shared_from_this(), part, covered_parts, NO_TRANSACTION_RAW); - data.modifyPartState(part, DataPartState::Active); - data.addPartContributionToColumnAndSecondaryIndexSizes(part); + total_covered_parts.insert(total_covered_parts.end(), covered_parts.begin(), covered_parts.end()); + for (const auto & covered_part : covered_parts) + { + covered_part->remove_time.store(current_time, std::memory_order_relaxed); + + reduce_bytes += covered_part->getBytesOnDisk(); + reduce_rows += covered_part->rows_count; + + data.modifyPartState(covered_part, DataPartState::Outdated); + data.removePartContributionToColumnAndSecondaryIndexSizes(covered_part); + } + + reduce_parts += covered_parts.size(); + + add_bytes += part->getBytesOnDisk(); + add_rows += part->rows_count; + ++add_parts; + + data.modifyPartState(part, DataPartState::Active); + data.addPartContributionToColumnAndSecondaryIndexSizes(part); + } } - } - if (reduce_parts == 0) - { - for (const auto & part : precommitted_parts) - data.updateObjectColumns(part, parts_lock); - } - else - data.resetObjectColumnsFromActiveParts(parts_lock); + if (reduce_parts == 0) + { + for (const auto & part : precommitted_parts) + data.updateObjectColumns(part, parts_lock); + } + else + data.resetObjectColumnsFromActiveParts(parts_lock); - ssize_t diff_bytes = add_bytes - reduce_bytes; - ssize_t diff_rows = add_rows - reduce_rows; - ssize_t diff_parts = add_parts - reduce_parts; - data.increaseDataVolume(diff_bytes, diff_rows, diff_parts); + ssize_t diff_bytes = add_bytes - reduce_bytes; + ssize_t diff_rows = add_rows - reduce_rows; + ssize_t diff_parts = add_parts - reduce_parts; + data.increaseDataVolume(diff_bytes, diff_rows, diff_parts); + }); } clear(); From f8e23c1deb8786729bfd1d9c7358f8264fea97ed Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 14 Jul 2022 17:29:08 +0300 Subject: [PATCH 206/659] Update run.sh --- docker/test/stress/run.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index cb80c8ebdc9..e34195cdd32 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -354,6 +354,8 @@ else # Error messages (we should ignore some errors) # FIXME https://github.com/ClickHouse/ClickHouse/issues/38643 ("Unknown index: idx.") # FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 ("Cannot parse string 'Hello' as UInt64") + # FIXME Not sure if it's expected, but some tests from BC check may not be finished yet when we restarting server. + # Let's just ignore all errors from queries ("} TCPHandler: Code:", "} executeQuery: Code:") echo "Check for Error messages in server log:" zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \ -e "Code: 236. DB::Exception: Cancelled mutating parts" \ @@ -378,6 +380,8 @@ else -e "is lost forever." \ -e "Unknown index: idx." \ -e "Cannot parse string 'Hello' as UInt64" \ + -e "} TCPHandler: Code:" \ + -e "} executeQuery: Code:" \ /var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv From 22abb97b91d0a767478f5ee973e93edb5704bd17 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 12:30:14 +0000 Subject: [PATCH 207/659] Enable -Wc11-extensions --- cmake/warnings.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 7da73b0dd08..0e55f6d32c3 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -21,7 +21,6 @@ if (COMPILER_CLANG) add_warning(everything) add_warning(pedantic) no_warning(zero-length-array) - no_warning(c11-extensions) no_warning(c++98-compat-pedantic) no_warning(c++98-compat) no_warning(c99-extensions) From 62149111d3ccde6cbf293e9cee19fecb3a28bb41 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Jul 2022 14:36:31 +0000 Subject: [PATCH 208/659] Fix build --- programs/copier/ZooKeeperStaff.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/copier/ZooKeeperStaff.h b/programs/copier/ZooKeeperStaff.h index 66036ae2f27..a9e04578607 100644 --- a/programs/copier/ZooKeeperStaff.h +++ b/programs/copier/ZooKeeperStaff.h @@ -20,7 +20,7 @@ public: bool operator<=(const WrappingUInt32 & other) const { - const UInt32 HALF = 1 << 31; + const UInt32 HALF = static_cast(1) << 31; return (value <= other.value && other.value - value < HALF) || (value > other.value && value - other.value > HALF); } From b2317ff710b65eb23bb6e7c87f3c67a0176185f8 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 14 Jul 2022 17:33:22 +0200 Subject: [PATCH 209/659] renaming to be consistent --- src/Disks/ObjectStorages/IObjectStorage.cpp | 8 ++++---- src/Disks/ObjectStorages/IObjectStorage.h | 4 ++-- src/IO/ReadBufferFromAzureBlobStorage.cpp | 4 ++-- src/IO/ReadBufferFromS3.cpp | 4 ++-- src/IO/ReadSettings.h | 2 +- src/IO/WriteBufferFromAzureBlobStorage.cpp | 4 ++-- src/IO/WriteBufferFromS3.cpp | 4 ++-- src/IO/WriteSettings.h | 2 +- src/Interpreters/Context.cpp | 4 ++-- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 4 ++-- src/Storages/HDFS/WriteBufferFromHDFS.cpp | 4 ++-- 11 files changed, 22 insertions(+), 22 deletions(-) diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index 70159b33b87..f9c5c139b95 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -50,15 +50,15 @@ std::string IObjectStorage::getCacheBasePath() const void IObjectStorage::applyRemoteThrottlingSettings(ContextPtr context) { std::unique_lock lock{throttlers_mutex}; - read_throttler = context->getRemoteReadThrottler(); - write_throttler = context->getRemoteWriteThrottler(); + remote_read_throttler = context->getRemoteReadThrottler(); + remote_write_throttler = context->getRemoteWriteThrottler(); } ReadSettings IObjectStorage::patchSettings(const ReadSettings & read_settings) const { std::unique_lock lock{throttlers_mutex}; ReadSettings settings{read_settings}; - settings.throttler = read_throttler; + settings.remote_throttler = remote_read_throttler; return settings; } @@ -66,7 +66,7 @@ WriteSettings IObjectStorage::patchSettings(const WriteSettings & write_settings { std::unique_lock lock{throttlers_mutex}; WriteSettings settings{write_settings}; - settings.throttler = write_throttler; + settings.remote_throttler = remote_write_throttler; return settings; } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index da341a04a9b..06398b11aec 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -178,8 +178,8 @@ protected: private: mutable std::mutex throttlers_mutex; - ThrottlerPtr read_throttler; - ThrottlerPtr write_throttler; + ThrottlerPtr remote_read_throttler; + ThrottlerPtr remote_write_throttler; }; using ObjectStoragePtr = std::shared_ptr; diff --git a/src/IO/ReadBufferFromAzureBlobStorage.cpp b/src/IO/ReadBufferFromAzureBlobStorage.cpp index 32e70305bea..3e6581cd786 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/IO/ReadBufferFromAzureBlobStorage.cpp @@ -77,8 +77,8 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() try { bytes_read = data_stream->ReadToCount(reinterpret_cast(data_ptr), to_read_bytes); - if (read_settings.throttler) - read_settings.throttler->add(bytes_read); + if (read_settings.remote_throttler) + read_settings.remote_throttler->add(bytes_read); break; } catch (const Azure::Storage::StorageException & e) diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index ee21db62507..7fb432eab22 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -165,8 +165,8 @@ bool ReadBufferFromS3::nextImpl() ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Bytes, working_buffer.size()); offset += working_buffer.size(); - if (read_settings.throttler) - read_settings.throttler->add(working_buffer.size()); + if (read_settings.remote_throttler) + read_settings.remote_throttler->add(working_buffer.size()); return true; } diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 97e72bdb147..2a2691e3c06 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -91,7 +91,7 @@ struct ReadSettings FileCachePtr remote_fs_cache; /// Bandwidth throttler to use during reading - ThrottlerPtr throttler; + ThrottlerPtr remote_throttler; size_t http_max_tries = 1; size_t http_retry_initial_backoff_ms = 100; diff --git a/src/IO/WriteBufferFromAzureBlobStorage.cpp b/src/IO/WriteBufferFromAzureBlobStorage.cpp index 0b7b6323b33..bc7b505cd91 100644 --- a/src/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/IO/WriteBufferFromAzureBlobStorage.cpp @@ -88,8 +88,8 @@ void WriteBufferFromAzureBlobStorage::nextImpl() block_blob_client.CommitBlockList(block_ids); - if (write_settings.throttler) - write_settings.throttler->add(read); + if (write_settings.remote_throttler) + write_settings.remote_throttler->add(read); } } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 27210d50e44..51f0c0d0743 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -125,8 +125,8 @@ void WriteBufferFromS3::nextImpl() ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Bytes, offset()); last_part_size += offset(); - if (write_settings.throttler) - write_settings.throttler->add(offset()); + if (write_settings.remote_throttler) + write_settings.remote_throttler->add(offset()); /// Data size exceeds singlepart upload threshold, need to use multipart upload. if (multipart_upload_id.empty() && last_part_size > s3_settings.max_single_part_upload_size) diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index 547ed0eb91b..7530b27794a 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -11,7 +11,7 @@ struct WriteSettings bool enable_filesystem_cache_on_write_operations = false; /// Bandwidth throttler to use during writing - ThrottlerPtr throttler; + ThrottlerPtr remote_throttler; }; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 70a0b201180..bce294d1221 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3458,7 +3458,7 @@ ReadSettings Context::getReadSettings() const res.mmap_threshold = settings.min_bytes_to_use_mmap_io; res.priority = settings.read_priority; - res.throttler = getRemoteReadThrottler(); + res.remote_throttler = getRemoteReadThrottler(); res.http_max_tries = settings.http_max_tries; res.http_retry_initial_backoff_ms = settings.http_retry_initial_backoff_ms; @@ -3476,7 +3476,7 @@ WriteSettings Context::getWriteSettings() const res.enable_filesystem_cache_on_write_operations = settings.enable_filesystem_cache_on_write_operations; - res.throttler = getRemoteWriteThrottler(); + res.remote_throttler = getRemoteWriteThrottler(); return res; } diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 8f22c4c2821..fab810a1e49 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -100,8 +100,8 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemoryadd(bytes_read); + if (read_settings.remote_throttler) + read_settings.remote_throttler->add(bytes_read); return true; } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 80acda52ffa..f8079d95f3c 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -58,8 +58,8 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl int write(const char * start, size_t size) const { int bytes_written = hdfsWrite(fs.get(), fout, start, size); - if (write_settings.throttler) - write_settings.throttler->add(bytes_written); + if (write_settings.remote_throttler) + write_settings.remote_throttler->add(bytes_written); if (bytes_written < 0) throw Exception("Fail to write HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), From 10625e7c5422ccbc3656e0290b714b061625e01e Mon Sep 17 00:00:00 2001 From: Xoel Lopez Barata Date: Thu, 14 Jul 2022 17:41:48 +0200 Subject: [PATCH 210/659] Fix typo Replace "indicies" by "indices" --- docs/en/sql-reference/functions/string-search-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 305e0c8c5cd..86408d8fc93 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -374,7 +374,7 @@ The same as `multiMatchAny`, but returns any index that matches the haystack. ## multiMatchAllIndices(haystack, \[pattern1, pattern2, …, patternn\]) -The same as `multiMatchAny`, but returns the array of all indicies that match the haystack in any order. +The same as `multiMatchAny`, but returns the array of all indices that match the haystack in any order. ## multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern2, …, patternn\]) From 9291d330804730e0426798822f99f51471b21bd1 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Jul 2022 16:11:35 +0000 Subject: [PATCH 211/659] Pass const std::string_view & by value, not by reference --- src/Access/AccessControl.cpp | 4 +- src/Access/AccessRights.cpp | 70 +++++++-------- src/Access/AccessRights.h | 66 +++++++------- src/Access/Authentication.cpp | 8 +- src/Access/Common/AccessFlags.cpp | 14 +-- src/Access/Common/AccessFlags.h | 2 +- src/Access/Common/AccessRightsElement.cpp | 14 +-- src/Access/Common/AccessRightsElement.h | 12 +-- src/Access/Common/AccessType.cpp | 2 +- src/Access/Common/AuthenticationData.cpp | 4 +- src/Access/Common/AuthenticationData.h | 8 +- src/Access/ContextAccess.cpp | 46 +++++----- src/Access/ContextAccess.h | 42 ++++----- src/Access/SettingsConstraints.cpp | 20 ++--- src/Access/SettingsConstraints.h | 20 ++--- src/Common/JSONParsers/DummyJSONParser.h | 4 +- src/Common/JSONParsers/RapidJSONParser.h | 4 +- src/Common/JSONParsers/SimdJSONParser.h | 4 +- src/Common/OpenSSLHelpers.cpp | 4 +- src/Common/OpenSSLHelpers.h | 4 +- src/Common/SettingsChanges.cpp | 10 +-- src/Common/SettingsChanges.h | 10 +-- src/Common/StringUtils/StringUtils.h | 2 +- src/Compression/CompressionCodecEncrypted.cpp | 4 +- src/Core/BaseSettings.cpp | 6 +- src/Core/BaseSettings.h | 88 +++++++++---------- src/Core/Field.cpp | 2 +- src/Core/Field.h | 8 +- src/Core/SettingsFields.cpp | 2 +- src/Core/SettingsFields.h | 12 +-- src/DataTypes/NestedUtils.cpp | 2 +- src/DataTypes/NestedUtils.h | 2 +- src/Formats/ProtobufSerializer.cpp | 88 +++++++++---------- src/Formats/ProtobufWriter.cpp | 2 +- src/Formats/ProtobufWriter.h | 2 +- src/Functions/FunctionsJSON.cpp | 30 +++---- src/IO/ReadHelpers.h | 4 +- src/IO/WriteHelpers.h | 16 ++-- src/Interpreters/Context.cpp | 12 +-- src/Interpreters/Context.h | 12 +-- src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Interpreters/DatabaseCatalog.h | 2 +- .../TranslateQualifiedNamesVisitor.cpp | 2 +- .../TranslateQualifiedNamesVisitor.h | 2 +- .../UserDefinedSQLObjectsLoader.cpp | 2 +- .../UserDefinedSQLObjectsLoader.h | 2 +- .../Access/ParserCreateRowPolicyQuery.cpp | 4 +- src/Storages/MergeTree/KeyCondition.cpp | 2 +- src/Storages/MergeTree/KeyCondition.h | 2 +- .../System/StorageSystemPrivileges.cpp | 2 +- 50 files changed, 344 insertions(+), 344 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 7152820b5bc..abd481f0bb6 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -101,7 +101,7 @@ public: registered_prefixes = prefixes_; } - bool isSettingNameAllowed(const std::string_view & setting_name) const + bool isSettingNameAllowed(std::string_view setting_name) const { if (Settings::hasBuiltin(setting_name)) return true; @@ -116,7 +116,7 @@ public: return false; } - void checkSettingNameIsAllowed(const std::string_view & setting_name) const + void checkSettingNameIsAllowed(std::string_view setting_name) const { if (isSettingNameAllowed(setting_name)) return; diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 7c3139dbb0f..b6fed3ac912 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -252,7 +252,7 @@ public: } template - void grant(const AccessFlags & flags_, const std::string_view & name, const Args &... subnames) + void grant(const AccessFlags & flags_, std::string_view name, const Args &... subnames) { auto & child = getChild(name); child.grant(flags_, subnames...); @@ -279,7 +279,7 @@ public: } template - void revoke(const AccessFlags & flags_, const std::string_view & name, const Args &... subnames) + void revoke(const AccessFlags & flags_, std::string_view name, const Args &... subnames) { auto & child = getChild(name); @@ -306,7 +306,7 @@ public: } template - bool isGranted(const AccessFlags & flags_, const std::string_view & name, const Args &... subnames) const + bool isGranted(const AccessFlags & flags_, std::string_view name, const Args &... subnames) const { AccessFlags flags_to_check = flags_ - min_flags_with_children; if (!flags_to_check) @@ -415,7 +415,7 @@ private: AccessFlags getAllGrantableFlags() const { return ::DB::getAllGrantableFlags(level); } AccessFlags getChildAllGrantableFlags() const { return ::DB::getAllGrantableFlags(static_cast(level + 1)); } - Node * tryGetChild(const std::string_view & name) const + Node * tryGetChild(std::string_view name) const { if (!children) return nullptr; @@ -425,7 +425,7 @@ private: return &it->second; } - Node & getChild(const std::string_view & name) + Node & getChild(std::string_view name) { auto * child = tryGetChild(name); if (child) @@ -819,20 +819,20 @@ void AccessRights::grantImpl(const AccessRightsElements & elements) } void AccessRights::grant(const AccessFlags & flags) { grantImpl(flags); } -void AccessRights::grant(const AccessFlags & flags, const std::string_view & database) { grantImpl(flags, database); } -void AccessRights::grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) { grantImpl(flags, database, table); } -void AccessRights::grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) { grantImpl(flags, database, table, column); } -void AccessRights::grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) { grantImpl(flags, database, table, columns); } -void AccessRights::grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) { grantImpl(flags, database, table, columns); } +void AccessRights::grant(const AccessFlags & flags, std::string_view database) { grantImpl(flags, database); } +void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table) { grantImpl(flags, database, table); } +void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) { grantImpl(flags, database, table, column); } +void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) { grantImpl(flags, database, table, columns); } +void AccessRights::grant(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) { grantImpl(flags, database, table, columns); } void AccessRights::grant(const AccessRightsElement & element) { grantImpl(element); } void AccessRights::grant(const AccessRightsElements & elements) { grantImpl(elements); } void AccessRights::grantWithGrantOption(const AccessFlags & flags) { grantImpl(flags); } -void AccessRights::grantWithGrantOption(const AccessFlags & flags, const std::string_view & database) { grantImpl(flags, database); } -void AccessRights::grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) { grantImpl(flags, database, table); } -void AccessRights::grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) { grantImpl(flags, database, table, column); } -void AccessRights::grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) { grantImpl(flags, database, table, columns); } -void AccessRights::grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) { grantImpl(flags, database, table, columns); } +void AccessRights::grantWithGrantOption(const AccessFlags & flags, std::string_view database) { grantImpl(flags, database); } +void AccessRights::grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) { grantImpl(flags, database, table); } +void AccessRights::grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) { grantImpl(flags, database, table, column); } +void AccessRights::grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) { grantImpl(flags, database, table, columns); } +void AccessRights::grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) { grantImpl(flags, database, table, columns); } void AccessRights::grantWithGrantOption(const AccessRightsElement & element) { grantImpl(element); } void AccessRights::grantWithGrantOption(const AccessRightsElements & elements) { grantImpl(elements); } @@ -892,20 +892,20 @@ void AccessRights::revokeImpl(const AccessRightsElements & elements) } void AccessRights::revoke(const AccessFlags & flags) { revokeImpl(flags); } -void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database) { revokeImpl(flags, database); } -void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) { revokeImpl(flags, database, table); } -void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) { revokeImpl(flags, database, table, column); } -void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) { revokeImpl(flags, database, table, columns); } -void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) { revokeImpl(flags, database, table, columns); } +void AccessRights::revoke(const AccessFlags & flags, std::string_view database) { revokeImpl(flags, database); } +void AccessRights::revoke(const AccessFlags & flags, std::string_view database, std::string_view table) { revokeImpl(flags, database, table); } +void AccessRights::revoke(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) { revokeImpl(flags, database, table, column); } +void AccessRights::revoke(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) { revokeImpl(flags, database, table, columns); } +void AccessRights::revoke(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) { revokeImpl(flags, database, table, columns); } void AccessRights::revoke(const AccessRightsElement & element) { revokeImpl(element); } void AccessRights::revoke(const AccessRightsElements & elements) { revokeImpl(elements); } void AccessRights::revokeGrantOption(const AccessFlags & flags) { revokeImpl(flags); } -void AccessRights::revokeGrantOption(const AccessFlags & flags, const std::string_view & database) { revokeImpl(flags, database); } -void AccessRights::revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) { revokeImpl(flags, database, table); } -void AccessRights::revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) { revokeImpl(flags, database, table, column); } -void AccessRights::revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) { revokeImpl(flags, database, table, columns); } -void AccessRights::revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) { revokeImpl(flags, database, table, columns); } +void AccessRights::revokeGrantOption(const AccessFlags & flags, std::string_view database) { revokeImpl(flags, database); } +void AccessRights::revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) { revokeImpl(flags, database, table); } +void AccessRights::revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) { revokeImpl(flags, database, table, column); } +void AccessRights::revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) { revokeImpl(flags, database, table, columns); } +void AccessRights::revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) { revokeImpl(flags, database, table, columns); } void AccessRights::revokeGrantOption(const AccessRightsElement & element) { revokeImpl(element); } void AccessRights::revokeGrantOption(const AccessRightsElements & elements) { revokeImpl(elements); } @@ -984,20 +984,20 @@ bool AccessRights::isGrantedImpl(const AccessRightsElements & elements) const } bool AccessRights::isGranted(const AccessFlags & flags) const { return isGrantedImpl(flags); } -bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database) const { return isGrantedImpl(flags, database); } -bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return isGrantedImpl(flags, database, table); } -bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return isGrantedImpl(flags, database, table, column); } -bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return isGrantedImpl(flags, database, table, columns); } -bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return isGrantedImpl(flags, database, table, columns); } +bool AccessRights::isGranted(const AccessFlags & flags, std::string_view database) const { return isGrantedImpl(flags, database); } +bool AccessRights::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table) const { return isGrantedImpl(flags, database, table); } +bool AccessRights::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return isGrantedImpl(flags, database, table, column); } +bool AccessRights::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return isGrantedImpl(flags, database, table, columns); } +bool AccessRights::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return isGrantedImpl(flags, database, table, columns); } bool AccessRights::isGranted(const AccessRightsElement & element) const { return isGrantedImpl(element); } bool AccessRights::isGranted(const AccessRightsElements & elements) const { return isGrantedImpl(elements); } bool AccessRights::hasGrantOption(const AccessFlags & flags) const { return isGrantedImpl(flags); } -bool AccessRights::hasGrantOption(const AccessFlags & flags, const std::string_view & database) const { return isGrantedImpl(flags, database); } -bool AccessRights::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return isGrantedImpl(flags, database, table); } -bool AccessRights::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return isGrantedImpl(flags, database, table, column); } -bool AccessRights::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return isGrantedImpl(flags, database, table, columns); } -bool AccessRights::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return isGrantedImpl(flags, database, table, columns); } +bool AccessRights::hasGrantOption(const AccessFlags & flags, std::string_view database) const { return isGrantedImpl(flags, database); } +bool AccessRights::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const { return isGrantedImpl(flags, database, table); } +bool AccessRights::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return isGrantedImpl(flags, database, table, column); } +bool AccessRights::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return isGrantedImpl(flags, database, table, columns); } +bool AccessRights::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return isGrantedImpl(flags, database, table, columns); } bool AccessRights::hasGrantOption(const AccessRightsElement & element) const { return isGrantedImpl(element); } bool AccessRights::hasGrantOption(const AccessRightsElements & elements) const { return isGrantedImpl(elements); } diff --git a/src/Access/AccessRights.h b/src/Access/AccessRights.h index b7499d69f70..80e37561c2b 100644 --- a/src/Access/AccessRights.h +++ b/src/Access/AccessRights.h @@ -39,59 +39,59 @@ public: /// Grants access on a specified database/table/column. /// Does nothing if the specified access has been already granted. void grant(const AccessFlags & flags); - void grant(const AccessFlags & flags, const std::string_view & database); - void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table); - void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column); - void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns); - void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns); + void grant(const AccessFlags & flags, std::string_view database); + void grant(const AccessFlags & flags, std::string_view database, std::string_view table); + void grant(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column); + void grant(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns); + void grant(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns); void grant(const AccessRightsElement & element); void grant(const AccessRightsElements & elements); void grantWithGrantOption(const AccessFlags & flags); - void grantWithGrantOption(const AccessFlags & flags, const std::string_view & database); - void grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table); - void grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column); - void grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns); - void grantWithGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns); + void grantWithGrantOption(const AccessFlags & flags, std::string_view database); + void grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table); + void grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column); + void grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns); + void grantWithGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns); void grantWithGrantOption(const AccessRightsElement & element); void grantWithGrantOption(const AccessRightsElements & elements); /// Revokes a specified access granted earlier on a specified database/table/column. /// For example, revoke(AccessType::ALL) revokes all grants at all, just like clear(); void revoke(const AccessFlags & flags); - void revoke(const AccessFlags & flags, const std::string_view & database); - void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table); - void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column); - void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns); - void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns); + void revoke(const AccessFlags & flags, std::string_view database); + void revoke(const AccessFlags & flags, std::string_view database, std::string_view table); + void revoke(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column); + void revoke(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns); + void revoke(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns); void revoke(const AccessRightsElement & element); void revoke(const AccessRightsElements & elements); void revokeGrantOption(const AccessFlags & flags); - void revokeGrantOption(const AccessFlags & flags, const std::string_view & database); - void revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table); - void revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column); - void revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns); - void revokeGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns); + void revokeGrantOption(const AccessFlags & flags, std::string_view database); + void revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table); + void revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column); + void revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns); + void revokeGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns); void revokeGrantOption(const AccessRightsElement & element); void revokeGrantOption(const AccessRightsElements & elements); /// Whether a specified access granted. bool isGranted(const AccessFlags & flags) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + bool isGranted(const AccessFlags & flags, std::string_view database) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; bool isGranted(const AccessRightsElement & element) const; bool isGranted(const AccessRightsElements & elements) const; bool hasGrantOption(const AccessFlags & flags) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; bool hasGrantOption(const AccessRightsElement & element) const; bool hasGrantOption(const AccessRightsElements & elements) const; @@ -107,9 +107,9 @@ public: const AccessFlags & flags, const AccessFlags & min_flags_with_children, const AccessFlags & max_flags_with_children, - const std::string_view & database, - const std::string_view & table, - const std::string_view & column)>; + std::string_view database, + std::string_view table, + std::string_view column)>; void modifyFlags(const ModifyFlagsFunction & function); void modifyFlagsWithGrantOption(const ModifyFlagsFunction & function); diff --git a/src/Access/Authentication.cpp b/src/Access/Authentication.cpp index 4f304cf5952..c6bbd421c77 100644 --- a/src/Access/Authentication.cpp +++ b/src/Access/Authentication.cpp @@ -26,17 +26,17 @@ namespace return (Util::stringToDigest(password) == password_plaintext); } - bool checkPasswordDoubleSHA1(const std::string_view & password, const Digest & password_double_sha1) + bool checkPasswordDoubleSHA1(std::string_view password, const Digest & password_double_sha1) { return (Util::encodeDoubleSHA1(password) == password_double_sha1); } - bool checkPasswordSHA256(const std::string_view & password, const Digest & password_sha256, const String & salt) + bool checkPasswordSHA256(std::string_view password, const Digest & password_sha256, const String & salt) { return Util::encodeSHA256(String(password).append(salt)) == password_sha256; } - bool checkPasswordDoubleSHA1MySQL(const std::string_view & scramble, const std::string_view & scrambled_password, const Digest & password_double_sha1) + bool checkPasswordDoubleSHA1MySQL(std::string_view scramble, std::string_view scrambled_password, const Digest & password_double_sha1) { /// scrambled_password = SHA1(password) XOR SHA1(scramble SHA1(SHA1(password))) @@ -61,7 +61,7 @@ namespace return calculated_password_double_sha1 == password_double_sha1; } - bool checkPasswordPlainTextMySQL(const std::string_view & scramble, const std::string_view & scrambled_password, const Digest & password_plaintext) + bool checkPasswordPlainTextMySQL(std::string_view scramble, std::string_view scrambled_password, const Digest & password_plaintext) { return checkPasswordDoubleSHA1MySQL(scramble, scrambled_password, Util::encodeDoubleSHA1(password_plaintext)); } diff --git a/src/Access/Common/AccessFlags.cpp b/src/Access/Common/AccessFlags.cpp index 82e1cbfb26b..305ae3f7cf5 100644 --- a/src/Access/Common/AccessFlags.cpp +++ b/src/Access/Common/AccessFlags.cpp @@ -35,7 +35,7 @@ namespace return access_type_to_flags_mapping[static_cast(type)]; } - Flags keywordToFlags(const std::string_view & keyword) const + Flags keywordToFlags(std::string_view keyword) const { auto it = keyword_to_flags_map.find(keyword); if (it == keyword_to_flags_map.end()) @@ -142,14 +142,14 @@ namespace } }; - static String replaceUnderscoreWithSpace(const std::string_view & str) + static String replaceUnderscoreWithSpace(std::string_view str) { String res{str}; boost::replace_all(res, "_", " "); return res; } - static Strings splitAliases(const std::string_view & str) + static Strings splitAliases(std::string_view str) { Strings aliases; boost::split(aliases, str, boost::is_any_of(",")); @@ -160,10 +160,10 @@ namespace static void makeNode( AccessType access_type, - const std::string_view & name, - const std::string_view & aliases, + std::string_view name, + std::string_view aliases, NodeType node_type, - const std::string_view & parent_group_name, + std::string_view parent_group_name, std::unordered_map & nodes, std::unordered_map & owned_nodes, size_t & next_flag) @@ -353,7 +353,7 @@ namespace AccessFlags::AccessFlags(AccessType type) : flags(Helper::instance().accessTypeToFlags(type)) {} -AccessFlags::AccessFlags(const std::string_view & keyword) : flags(Helper::instance().keywordToFlags(keyword)) {} +AccessFlags::AccessFlags(std::string_view keyword) : flags(Helper::instance().keywordToFlags(keyword)) {} AccessFlags::AccessFlags(const std::vector & keywords) : flags(Helper::instance().keywordsToFlags(keywords)) {} AccessFlags::AccessFlags(const Strings & keywords) : flags(Helper::instance().keywordsToFlags(keywords)) {} String AccessFlags::toString() const { return Helper::instance().flagsToString(flags); } diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index 51bf3cd19b0..5124f4ef332 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -21,7 +21,7 @@ public: AccessFlags() = default; /// Constructs from a string like "SELECT". - AccessFlags(const std::string_view & keyword); /// NOLINT + AccessFlags(std::string_view keyword); /// NOLINT /// Constructs from a list of strings like "SELECT, UPDATE, INSERT". AccessFlags(const std::vector & keywords); /// NOLINT diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 9913fc02f4a..69a2354f25d 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -81,7 +81,7 @@ namespace } bool need_comma = false; - for (const std::string_view & keyword : keywords) + for (std::string_view keyword : keywords) { if (need_comma) result.append(", "); @@ -145,18 +145,18 @@ namespace } -AccessRightsElement::AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_) +AccessRightsElement::AccessRightsElement(AccessFlags access_flags_, std::string_view database_) : access_flags(access_flags_), database(database_), any_database(false) { } -AccessRightsElement::AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_) +AccessRightsElement::AccessRightsElement(AccessFlags access_flags_, std::string_view database_, std::string_view table_) : access_flags(access_flags_), database(database_), table(table_), any_database(false), any_table(false) { } AccessRightsElement::AccessRightsElement( - AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const std::string_view & column_) + AccessFlags access_flags_, std::string_view database_, std::string_view table_, std::string_view column_) : access_flags(access_flags_) , database(database_) , table(table_) @@ -169,8 +169,8 @@ AccessRightsElement::AccessRightsElement( AccessRightsElement::AccessRightsElement( AccessFlags access_flags_, - const std::string_view & database_, - const std::string_view & table_, + std::string_view database_, + std::string_view table_, const std::vector & columns_) : access_flags(access_flags_), database(database_), table(table_), any_database(false), any_table(false), any_column(false) { @@ -180,7 +180,7 @@ AccessRightsElement::AccessRightsElement( } AccessRightsElement::AccessRightsElement( - AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const Strings & columns_) + AccessFlags access_flags_, std::string_view database_, std::string_view table_, const Strings & columns_) : access_flags(access_flags_) , database(database_) , table(table_) diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index e5bf76d0017..5f65b6bcd12 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -28,19 +28,19 @@ struct AccessRightsElement explicit AccessRightsElement(AccessFlags access_flags_) : access_flags(access_flags_) {} - AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_); - AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_); + AccessRightsElement(AccessFlags access_flags_, std::string_view database_); + AccessRightsElement(AccessFlags access_flags_, std::string_view database_, std::string_view table_); AccessRightsElement( - AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const std::string_view & column_); + AccessFlags access_flags_, std::string_view database_, std::string_view table_, std::string_view column_); AccessRightsElement( AccessFlags access_flags_, - const std::string_view & database_, - const std::string_view & table_, + std::string_view database_, + std::string_view table_, const std::vector & columns_); AccessRightsElement( - AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const Strings & columns_); + AccessFlags access_flags_, std::string_view database_, std::string_view table_, const Strings & columns_); bool empty() const { return !access_flags || (!any_column && columns.empty()); } diff --git a/src/Access/Common/AccessType.cpp b/src/Access/Common/AccessType.cpp index d44d70d78b2..4df1e1bc77f 100644 --- a/src/Access/Common/AccessType.cpp +++ b/src/Access/Common/AccessType.cpp @@ -35,7 +35,7 @@ namespace #undef ACCESS_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING } - void addToMapping(AccessType type, const std::string_view & str) + void addToMapping(AccessType type, std::string_view str) { String str2{str}; boost::replace_all(str2, "_", " "); diff --git a/src/Access/Common/AuthenticationData.cpp b/src/Access/Common/AuthenticationData.cpp index db0a5d54a63..f3d3bb5b758 100644 --- a/src/Access/Common/AuthenticationData.cpp +++ b/src/Access/Common/AuthenticationData.cpp @@ -71,7 +71,7 @@ const AuthenticationTypeInfo & AuthenticationTypeInfo::get(AuthenticationType ty } -AuthenticationData::Digest AuthenticationData::Util::encodeSHA256(const std::string_view & text [[maybe_unused]]) +AuthenticationData::Digest AuthenticationData::Util::encodeSHA256(std::string_view text [[maybe_unused]]) { #if USE_SSL Digest hash; @@ -86,7 +86,7 @@ AuthenticationData::Digest AuthenticationData::Util::encodeSHA256(const std::str } -AuthenticationData::Digest AuthenticationData::Util::encodeSHA1(const std::string_view & text) +AuthenticationData::Digest AuthenticationData::Util::encodeSHA1(std::string_view text) { Poco::SHA1Engine engine; engine.update(text.data(), text.size()); diff --git a/src/Access/Common/AuthenticationData.h b/src/Access/Common/AuthenticationData.h index 2837e0f10a1..ced9fcd4b6d 100644 --- a/src/Access/Common/AuthenticationData.h +++ b/src/Access/Common/AuthenticationData.h @@ -96,11 +96,11 @@ public: struct Util { - static Digest stringToDigest(const std::string_view & text) { return Digest(text.data(), text.data() + text.size()); } - static Digest encodeSHA256(const std::string_view & text); - static Digest encodeSHA1(const std::string_view & text); + static Digest stringToDigest(std::string_view text) { return Digest(text.data(), text.data() + text.size()); } + static Digest encodeSHA256(std::string_view text); + static Digest encodeSHA1(std::string_view text); static Digest encodeSHA1(const Digest & text) { return encodeSHA1(std::string_view{reinterpret_cast(text.data()), text.size()}); } - static Digest encodeDoubleSHA1(const std::string_view & text) { return encodeSHA1(encodeSHA1(text)); } + static Digest encodeDoubleSHA1(std::string_view text) { return encodeSHA1(encodeSHA1(text)); } static Digest encodeDoubleSHA1(const Digest & text) { return encodeSHA1(encodeSHA1(text)); } }; diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 995a46d07ca..92a5179d861 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -46,7 +46,7 @@ namespace AccessRights addImplicitAccessRights(const AccessRights & access) { - auto modifier = [&](const AccessFlags & flags, const AccessFlags & min_flags_with_children, const AccessFlags & max_flags_with_children, const std::string_view & database, const std::string_view & table, const std::string_view & column) -> AccessFlags + auto modifier = [&](const AccessFlags & flags, const AccessFlags & min_flags_with_children, const AccessFlags & max_flags_with_children, std::string_view database, std::string_view table, std::string_view column) -> AccessFlags { size_t level = !database.empty() + !table.empty() + !column.empty(); AccessFlags res = flags; @@ -141,7 +141,7 @@ namespace std::string_view getDatabase() { return {}; } template - std::string_view getDatabase(const std::string_view & arg1, const OtherArgs &...) { return arg1; } + std::string_view getDatabase(std::string_view arg1, const OtherArgs &...) { return arg1; } } @@ -519,7 +519,7 @@ bool ContextAccess::checkAccessImpl(const AccessFlags & flags) const } template -bool ContextAccess::checkAccessImpl(const AccessFlags & flags, const std::string_view & database, const Args &... args) const +bool ContextAccess::checkAccessImpl(const AccessFlags & flags, std::string_view database, const Args &... args) const { return checkAccessImplHelper(flags, database.empty() ? params.current_database : database, args...); } @@ -564,38 +564,38 @@ bool ContextAccess::checkAccessImpl(const AccessRightsElements & elements) const } bool ContextAccess::isGranted(const AccessFlags & flags) const { return checkAccessImpl(flags); } -bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database) const { return checkAccessImpl(flags, database); } -bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(flags, database, table); } -bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(flags, database, table, column); } -bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } -bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } +bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database) const { return checkAccessImpl(flags, database); } +bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table) const { return checkAccessImpl(flags, database, table); } +bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return checkAccessImpl(flags, database, table, column); } +bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } +bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } bool ContextAccess::isGranted(const AccessRightsElement & element) const { return checkAccessImpl(element); } bool ContextAccess::isGranted(const AccessRightsElements & elements) const { return checkAccessImpl(elements); } bool ContextAccess::hasGrantOption(const AccessFlags & flags) const { return checkAccessImpl(flags); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, const std::string_view & database) const { return checkAccessImpl(flags, database); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(flags, database, table); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(flags, database, table, column); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } +bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database) const { return checkAccessImpl(flags, database); } +bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const { return checkAccessImpl(flags, database, table); } +bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return checkAccessImpl(flags, database, table, column); } +bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } +bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } bool ContextAccess::hasGrantOption(const AccessRightsElement & element) const { return checkAccessImpl(element); } bool ContextAccess::hasGrantOption(const AccessRightsElements & elements) const { return checkAccessImpl(elements); } void ContextAccess::checkAccess(const AccessFlags & flags) const { checkAccessImpl(flags); } -void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database) const { checkAccessImpl(flags, database); } -void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { checkAccessImpl(flags, database, table); } -void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { checkAccessImpl(flags, database, table, column); } -void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { checkAccessImpl(flags, database, table, columns); } -void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { checkAccessImpl(flags, database, table, columns); } +void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database) const { checkAccessImpl(flags, database); } +void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const { checkAccessImpl(flags, database, table); } +void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { checkAccessImpl(flags, database, table, column); } +void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { checkAccessImpl(flags, database, table, columns); } +void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { checkAccessImpl(flags, database, table, columns); } void ContextAccess::checkAccess(const AccessRightsElement & element) const { checkAccessImpl(element); } void ContextAccess::checkAccess(const AccessRightsElements & elements) const { checkAccessImpl(elements); } void ContextAccess::checkGrantOption(const AccessFlags & flags) const { checkAccessImpl(flags); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database) const { checkAccessImpl(flags, database); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { checkAccessImpl(flags, database, table); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { checkAccessImpl(flags, database, table, column); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { checkAccessImpl(flags, database, table, columns); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { checkAccessImpl(flags, database, table, columns); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database) const { checkAccessImpl(flags, database); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const { checkAccessImpl(flags, database, table); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { checkAccessImpl(flags, database, table, column); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { checkAccessImpl(flags, database, table, columns); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { checkAccessImpl(flags, database, table, columns); } void ContextAccess::checkGrantOption(const AccessRightsElement & element) const { checkAccessImpl(element); } void ContextAccess::checkGrantOption(const AccessRightsElements & elements) const { checkAccessImpl(elements); } diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 729574898aa..fa3523977e7 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -101,40 +101,40 @@ public: /// Checks if a specified access is granted, and throws an exception if not. /// Empty database means the current database. void checkAccess(const AccessFlags & flags) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + void checkAccess(const AccessFlags & flags, std::string_view database) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; void checkAccess(const AccessRightsElement & element) const; void checkAccess(const AccessRightsElements & elements) const; void checkGrantOption(const AccessFlags & flags) const; - void checkGrantOption(const AccessFlags & flags, const std::string_view & database) const; - void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; - void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + void checkGrantOption(const AccessFlags & flags, std::string_view database) const; + void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const; + void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; void checkGrantOption(const AccessRightsElement & element) const; void checkGrantOption(const AccessRightsElements & elements) const; /// Checks if a specified access is granted, and returns false if not. /// Empty database means the current database. bool isGranted(const AccessFlags & flags) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + bool isGranted(const AccessFlags & flags, std::string_view database) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; bool isGranted(const AccessRightsElement & element) const; bool isGranted(const AccessRightsElements & elements) const; bool hasGrantOption(const AccessFlags & flags) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - bool hasGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; bool hasGrantOption(const AccessRightsElement & element) const; bool hasGrantOption(const AccessRightsElements & elements) const; @@ -180,7 +180,7 @@ private: bool checkAccessImpl(const AccessFlags & flags) const; template - bool checkAccessImpl(const AccessFlags & flags, const std::string_view & database, const Args &... args) const; + bool checkAccessImpl(const AccessFlags & flags, std::string_view database, const Args &... args) const; template bool checkAccessImpl(const AccessRightsElement & element) const; diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 6084138f306..34f2e10dc83 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -36,12 +36,12 @@ void SettingsConstraints::clear() } -void SettingsConstraints::setMinValue(const std::string_view & setting_name, const Field & min_value) +void SettingsConstraints::setMinValue(std::string_view setting_name, const Field & min_value) { getConstraintRef(setting_name).min_value = Settings::castValueUtil(setting_name, min_value); } -Field SettingsConstraints::getMinValue(const std::string_view & setting_name) const +Field SettingsConstraints::getMinValue(std::string_view setting_name) const { const auto * ptr = tryGetConstraint(setting_name); if (ptr) @@ -51,12 +51,12 @@ Field SettingsConstraints::getMinValue(const std::string_view & setting_name) co } -void SettingsConstraints::setMaxValue(const std::string_view & setting_name, const Field & max_value) +void SettingsConstraints::setMaxValue(std::string_view setting_name, const Field & max_value) { getConstraintRef(setting_name).max_value = Settings::castValueUtil(setting_name, max_value); } -Field SettingsConstraints::getMaxValue(const std::string_view & setting_name) const +Field SettingsConstraints::getMaxValue(std::string_view setting_name) const { const auto * ptr = tryGetConstraint(setting_name); if (ptr) @@ -66,12 +66,12 @@ Field SettingsConstraints::getMaxValue(const std::string_view & setting_name) co } -void SettingsConstraints::setReadOnly(const std::string_view & setting_name, bool read_only) +void SettingsConstraints::setReadOnly(std::string_view setting_name, bool read_only) { getConstraintRef(setting_name).read_only = read_only; } -bool SettingsConstraints::isReadOnly(const std::string_view & setting_name) const +bool SettingsConstraints::isReadOnly(std::string_view setting_name) const { const auto * ptr = tryGetConstraint(setting_name); if (ptr) @@ -81,7 +81,7 @@ bool SettingsConstraints::isReadOnly(const std::string_view & setting_name) cons } -void SettingsConstraints::set(const std::string_view & setting_name, const Field & min_value, const Field & max_value, bool read_only) +void SettingsConstraints::set(std::string_view setting_name, const Field & min_value, const Field & max_value, bool read_only) { auto & ref = getConstraintRef(setting_name); ref.min_value = Settings::castValueUtil(setting_name, min_value); @@ -89,7 +89,7 @@ void SettingsConstraints::set(const std::string_view & setting_name, const Field ref.read_only = read_only; } -void SettingsConstraints::get(const std::string_view & setting_name, Field & min_value, Field & max_value, bool & read_only) const +void SettingsConstraints::get(std::string_view setting_name, Field & min_value, Field & max_value, bool & read_only) const { const auto * ptr = tryGetConstraint(setting_name); if (ptr) @@ -318,7 +318,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh } -SettingsConstraints::Constraint & SettingsConstraints::getConstraintRef(const std::string_view & setting_name) +SettingsConstraints::Constraint & SettingsConstraints::getConstraintRef(std::string_view setting_name) { auto it = constraints.find(setting_name); if (it == constraints.end()) @@ -331,7 +331,7 @@ SettingsConstraints::Constraint & SettingsConstraints::getConstraintRef(const st return it->second; } -const SettingsConstraints::Constraint * SettingsConstraints::tryGetConstraint(const std::string_view & setting_name) const +const SettingsConstraints::Constraint * SettingsConstraints::tryGetConstraint(std::string_view setting_name) const { auto it = constraints.find(setting_name); if (it == constraints.end()) diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index f7bca1eafb3..645a690e051 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -61,17 +61,17 @@ public: void clear(); bool empty() const { return constraints.empty(); } - void setMinValue(const std::string_view & setting_name, const Field & min_value); - Field getMinValue(const std::string_view & setting_name) const; + void setMinValue(std::string_view setting_name, const Field & min_value); + Field getMinValue(std::string_view setting_name) const; - void setMaxValue(const std::string_view & setting_name, const Field & max_value); - Field getMaxValue(const std::string_view & setting_name) const; + void setMaxValue(std::string_view setting_name, const Field & max_value); + Field getMaxValue(std::string_view setting_name) const; - void setReadOnly(const std::string_view & setting_name, bool read_only); - bool isReadOnly(const std::string_view & setting_name) const; + void setReadOnly(std::string_view setting_name, bool read_only); + bool isReadOnly(std::string_view setting_name) const; - void set(const std::string_view & setting_name, const Field & min_value, const Field & max_value, bool read_only); - void get(const std::string_view & setting_name, Field & min_value, Field & max_value, bool & read_only) const; + void set(std::string_view setting_name, const Field & min_value, const Field & max_value, bool read_only); + void get(std::string_view setting_name, Field & min_value, Field & max_value, bool & read_only) const; void merge(const SettingsConstraints & other); @@ -105,8 +105,8 @@ private: }; bool checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const; - Constraint & getConstraintRef(const std::string_view & setting_name); - const Constraint * tryGetConstraint(const std::string_view & setting_name) const; + Constraint & getConstraintRef(std::string_view setting_name); + const Constraint * tryGetConstraint(std::string_view setting_name) const; std::unordered_map constraints; const AccessControl * access_control = nullptr; diff --git a/src/Common/JSONParsers/DummyJSONParser.h b/src/Common/JSONParsers/DummyJSONParser.h index 77b958d1429..3cedd59decd 100644 --- a/src/Common/JSONParsers/DummyJSONParser.h +++ b/src/Common/JSONParsers/DummyJSONParser.h @@ -84,7 +84,7 @@ struct DummyJSONParser static Iterator begin() { return {}; } static Iterator end() { return {}; } static size_t size() { return 0; } - bool find(const std::string_view &, Element &) const { return false; } /// NOLINT + bool find(std::string_view, Element &) const { return false; } /// NOLINT #if 0 /// Optional: Provides access to an object's element by index. @@ -93,7 +93,7 @@ struct DummyJSONParser }; /// Parses a JSON document, returns the reference to its root element if succeeded. - bool parse(const std::string_view &, Element &) { throw Exception{"Functions JSON* are not supported", ErrorCodes::NOT_IMPLEMENTED}; } /// NOLINT + bool parse(std::string_view, Element &) { throw Exception{"Functions JSON* are not supported", ErrorCodes::NOT_IMPLEMENTED}; } /// NOLINT #if 0 /// Optional: Allocates memory to parse JSON documents faster. diff --git a/src/Common/JSONParsers/RapidJSONParser.h b/src/Common/JSONParsers/RapidJSONParser.h index 2d8514868e5..77e8f6b2a74 100644 --- a/src/Common/JSONParsers/RapidJSONParser.h +++ b/src/Common/JSONParsers/RapidJSONParser.h @@ -98,7 +98,7 @@ struct RapidJSONParser ALWAYS_INLINE Iterator end() const { return ptr->MemberEnd(); } ALWAYS_INLINE size_t size() const { return ptr->MemberCount(); } - bool find(const std::string_view & key, Element & result) const + bool find(std::string_view key, Element & result) const { auto it = ptr->FindMember(rapidjson::StringRef(key.data(), key.length())); if (it == ptr->MemberEnd()) @@ -122,7 +122,7 @@ struct RapidJSONParser }; /// Parses a JSON document, returns the reference to its root element if succeeded. - bool parse(const std::string_view & json, Element & result) + bool parse(std::string_view json, Element & result) { rapidjson::MemoryStream ms(json.data(), json.size()); rapidjson::EncodedInputStream, rapidjson::MemoryStream> is(ms); diff --git a/src/Common/JSONParsers/SimdJSONParser.h b/src/Common/JSONParsers/SimdJSONParser.h index 3abeb85fb56..f3bbfe4dfde 100644 --- a/src/Common/JSONParsers/SimdJSONParser.h +++ b/src/Common/JSONParsers/SimdJSONParser.h @@ -105,7 +105,7 @@ struct SimdJSONParser ALWAYS_INLINE Iterator end() const { return object.end(); } ALWAYS_INLINE size_t size() const { return object.size(); } - bool find(const std::string_view & key, Element & result) const + bool find(std::string_view key, Element & result) const { auto x = object.at_key(key); if (x.error()) @@ -131,7 +131,7 @@ struct SimdJSONParser }; /// Parses a JSON document, returns the reference to its root element if succeeded. - bool parse(const std::string_view & json, Element & result) + bool parse(std::string_view json, Element & result) { auto document = parser.parse(json.data(), json.size()); if (document.error()) diff --git a/src/Common/OpenSSLHelpers.cpp b/src/Common/OpenSSLHelpers.cpp index d73e08e79a9..4e7848afc85 100644 --- a/src/Common/OpenSSLHelpers.cpp +++ b/src/Common/OpenSSLHelpers.cpp @@ -10,7 +10,7 @@ namespace DB { #pragma GCC diagnostic warning "-Wold-style-cast" -std::string encodeSHA256(const std::string_view & text) +std::string encodeSHA256(std::string_view text) { return encodeSHA256(text.data(), text.size()); } @@ -21,7 +21,7 @@ std::string encodeSHA256(const void * text, size_t size) encodeSHA256(text, size, reinterpret_cast(out.data())); return out; } -void encodeSHA256(const std::string_view & text, unsigned char * out) +void encodeSHA256(std::string_view text, unsigned char * out) { encodeSHA256(text.data(), text.size(), out); } diff --git a/src/Common/OpenSSLHelpers.h b/src/Common/OpenSSLHelpers.h index f0dbbc10b4c..41f092f0109 100644 --- a/src/Common/OpenSSLHelpers.h +++ b/src/Common/OpenSSLHelpers.h @@ -10,10 +10,10 @@ namespace DB { /// Encodes `text` and returns it. -std::string encodeSHA256(const std::string_view & text); +std::string encodeSHA256(std::string_view text); std::string encodeSHA256(const void * text, size_t size); /// `out` must be at least 32 bytes long. -void encodeSHA256(const std::string_view & text, unsigned char * out); +void encodeSHA256(std::string_view text, unsigned char * out); void encodeSHA256(const void * text, size_t size, unsigned char * out); /// Returns concatenation of error strings for all errors that OpenSSL has recorded, emptying the error queue. diff --git a/src/Common/SettingsChanges.cpp b/src/Common/SettingsChanges.cpp index 370b465eba3..9fb4f361e09 100644 --- a/src/Common/SettingsChanges.cpp +++ b/src/Common/SettingsChanges.cpp @@ -4,7 +4,7 @@ namespace DB { namespace { - SettingChange * find(SettingsChanges & changes, const std::string_view & name) + SettingChange * find(SettingsChanges & changes, std::string_view name) { auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; }); if (it == changes.end()) @@ -12,7 +12,7 @@ namespace return &*it; } - const SettingChange * find(const SettingsChanges & changes, const std::string_view & name) + const SettingChange * find(const SettingsChanges & changes, std::string_view name) { auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; }); if (it == changes.end()) @@ -21,7 +21,7 @@ namespace } } -bool SettingsChanges::tryGet(const std::string_view & name, Field & out_value) const +bool SettingsChanges::tryGet(std::string_view name, Field & out_value) const { const auto * change = find(*this, name); if (!change) @@ -30,7 +30,7 @@ bool SettingsChanges::tryGet(const std::string_view & name, Field & out_value) c return true; } -const Field * SettingsChanges::tryGet(const std::string_view & name) const +const Field * SettingsChanges::tryGet(std::string_view name) const { const auto * change = find(*this, name); if (!change) @@ -38,7 +38,7 @@ const Field * SettingsChanges::tryGet(const std::string_view & name) const return &change->value; } -Field * SettingsChanges::tryGet(const std::string_view & name) +Field * SettingsChanges::tryGet(std::string_view name) { auto * change = find(*this, name); if (!change) diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index 5f6a390d0d2..67cb69f77bf 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -14,8 +14,8 @@ struct SettingChange Field value; SettingChange() = default; - SettingChange(const std::string_view & name_, const Field & value_) : name(name_), value(value_) {} - SettingChange(const std::string_view & name_, Field && value_) : name(name_), value(std::move(value_)) {} + SettingChange(std::string_view name_, const Field & value_) : name(name_), value(value_) {} + SettingChange(std::string_view name_, Field && value_) : name(name_), value(std::move(value_)) {} friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value); } friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } @@ -27,9 +27,9 @@ class SettingsChanges : public std::vector public: using std::vector::vector; - bool tryGet(const std::string_view & name, Field & out_value) const; - const Field * tryGet(const std::string_view & name) const; - Field * tryGet(const std::string_view & name); + bool tryGet(std::string_view name, Field & out_value) const; + const Field * tryGet(std::string_view name) const; + Field * tryGet(std::string_view name); }; } diff --git a/src/Common/StringUtils/StringUtils.h b/src/Common/StringUtils/StringUtils.h index 21df0f5ae8b..b5a081ab693 100644 --- a/src/Common/StringUtils/StringUtils.h +++ b/src/Common/StringUtils/StringUtils.h @@ -147,7 +147,7 @@ inline bool isPunctuationASCII(char c) } -inline bool isValidIdentifier(const std::string_view & str) +inline bool isValidIdentifier(std::string_view str) { return !str.empty() && isValidIdentifierBegin(str[0]) diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index f7e597a0519..bf36fa114fb 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -131,7 +131,7 @@ std::string lastErrorString() /// This function get key and nonce and encrypt text with their help. /// If something went wrong (can't init context or can't encrypt data) it throws exception. /// It returns length of encrypted text. -size_t encrypt(const std::string_view & plaintext, char * ciphertext_and_tag, EncryptionMethod method, const String & key, const String & nonce) +size_t encrypt(std::string_view plaintext, char * ciphertext_and_tag, EncryptionMethod method, const String & key, const String & nonce) { /// Init context for encryption, using key. EVP_AEAD_CTX encrypt_ctx; @@ -160,7 +160,7 @@ size_t encrypt(const std::string_view & plaintext, char * ciphertext_and_tag, En /// This function get key and nonce and encrypt text with their help. /// If something went wrong (can't init context or can't encrypt data) it throws exception. /// It returns length of encrypted text. -size_t decrypt(const std::string_view & ciphertext, char * plaintext, EncryptionMethod method, const String & key, const String & nonce) +size_t decrypt(std::string_view ciphertext, char * plaintext, EncryptionMethod method, const String & key, const String & nonce) { /// Init context for decryption with given key. EVP_AEAD_CTX decrypt_ctx; diff --git a/src/Core/BaseSettings.cpp b/src/Core/BaseSettings.cpp index d4b2d1551b6..f03a59c1342 100644 --- a/src/Core/BaseSettings.cpp +++ b/src/Core/BaseSettings.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -void BaseSettingsHelpers::writeString(const std::string_view & str, WriteBuffer & out) +void BaseSettingsHelpers::writeString(std::string_view str, WriteBuffer & out) { writeStringBinary(str, out); } @@ -39,13 +39,13 @@ BaseSettingsHelpers::Flags BaseSettingsHelpers::readFlags(ReadBuffer & in) } -void BaseSettingsHelpers::throwSettingNotFound(const std::string_view & name) +void BaseSettingsHelpers::throwSettingNotFound(std::string_view name) { throw Exception("Unknown setting " + String{name}, ErrorCodes::UNKNOWN_SETTING); } -void BaseSettingsHelpers::warningSettingNotFound(const std::string_view & name) +void BaseSettingsHelpers::warningSettingNotFound(std::string_view name) { static auto * log = &Poco::Logger::get("Settings"); LOG_WARNING(log, "Unknown setting {}, skipping", name); diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index a4ddc6571ed..3638a036098 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -45,16 +45,16 @@ class BaseSettings : public TTraits::Data public: using Traits = TTraits; - void set(const std::string_view & name, const Field & value); - Field get(const std::string_view & name) const; + void set(std::string_view name, const Field & value); + Field get(std::string_view name) const; - void setString(const std::string_view & name, const String & value); - String getString(const std::string_view & name) const; + void setString(std::string_view name, const String & value); + String getString(std::string_view name) const; - bool tryGet(const std::string_view & name, Field & value) const; - bool tryGetString(const std::string_view & name, String & value) const; + bool tryGet(std::string_view name, Field & value) const; + bool tryGetString(std::string_view name, String & value) const; - bool isChanged(const std::string_view & name) const; + bool isChanged(std::string_view name) const; SettingsChanges changes() const; void applyChange(const SettingChange & change); void applyChanges(const SettingsChanges & changes); @@ -63,22 +63,22 @@ public: /// Resets all the settings to their default values. void resetToDefault(); - bool has(const std::string_view & name) const { return hasBuiltin(name) || hasCustom(name); } - static bool hasBuiltin(const std::string_view & name); - bool hasCustom(const std::string_view & name) const; + bool has(std::string_view name) const { return hasBuiltin(name) || hasCustom(name); } + static bool hasBuiltin(std::string_view name); + bool hasCustom(std::string_view name) const; - const char * getTypeName(const std::string_view & name) const; - const char * getDescription(const std::string_view & name) const; + const char * getTypeName(std::string_view name) const; + const char * getDescription(std::string_view name) const; /// Checks if it's possible to assign a field to a specified value and throws an exception if not. /// This function doesn't change the fields, it performs check only. - static void checkCanSet(const std::string_view & name, const Field & value); - static void checkCanSetString(const std::string_view & name, const String & str); + static void checkCanSet(std::string_view name, const Field & value); + static void checkCanSetString(std::string_view name, const String & str); /// Conversions without changing the fields. - static Field castValueUtil(const std::string_view & name, const Field & value); - static String valueToStringUtil(const std::string_view & name, const Field & value); - static Field stringToValueUtil(const std::string_view & name, const String & str); + static Field castValueUtil(std::string_view name, const Field & value); + static String valueToStringUtil(std::string_view name, const Field & value); + static Field stringToValueUtil(std::string_view name, const String & str); void write(WriteBuffer & out, SettingsWriteFormat format = SettingsWriteFormat::DEFAULT) const; void read(ReadBuffer & in, SettingsWriteFormat format = SettingsWriteFormat::DEFAULT); @@ -164,19 +164,19 @@ public: Iterator end() const { return allChanged().end(); } private: - SettingFieldCustom & getCustomSetting(const std::string_view & name); - const SettingFieldCustom & getCustomSetting(const std::string_view & name) const; - const SettingFieldCustom * tryGetCustomSetting(const std::string_view & name) const; + SettingFieldCustom & getCustomSetting(std::string_view name); + const SettingFieldCustom & getCustomSetting(std::string_view name) const; + const SettingFieldCustom * tryGetCustomSetting(std::string_view name) const; std::conditional_t custom_settings_map; }; struct BaseSettingsHelpers { - [[noreturn]] static void throwSettingNotFound(const std::string_view & name); - static void warningSettingNotFound(const std::string_view & name); + [[noreturn]] static void throwSettingNotFound(std::string_view name); + static void warningSettingNotFound(std::string_view name); - static void writeString(const std::string_view & str, WriteBuffer & out); + static void writeString(std::string_view str, WriteBuffer & out); static String readString(ReadBuffer & in); enum Flags : UInt64 @@ -190,7 +190,7 @@ struct BaseSettingsHelpers }; template -void BaseSettings::set(const std::string_view & name, const Field & value) +void BaseSettings::set(std::string_view name, const Field & value) { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -200,7 +200,7 @@ void BaseSettings::set(const std::string_view & name, const Field & val } template -Field BaseSettings::get(const std::string_view & name) const +Field BaseSettings::get(std::string_view name) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -210,7 +210,7 @@ Field BaseSettings::get(const std::string_view & name) const } template -void BaseSettings::setString(const std::string_view & name, const String & value) +void BaseSettings::setString(std::string_view name, const String & value) { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -220,7 +220,7 @@ void BaseSettings::setString(const std::string_view & name, const Strin } template -String BaseSettings::getString(const std::string_view & name) const +String BaseSettings::getString(std::string_view name) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -230,7 +230,7 @@ String BaseSettings::getString(const std::string_view & name) const } template -bool BaseSettings::tryGet(const std::string_view & name, Field & value) const +bool BaseSettings::tryGet(std::string_view name, Field & value) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -247,7 +247,7 @@ bool BaseSettings::tryGet(const std::string_view & name, Field & value) } template -bool BaseSettings::tryGetString(const std::string_view & name, String & value) const +bool BaseSettings::tryGetString(std::string_view name, String & value) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -264,7 +264,7 @@ bool BaseSettings::tryGetString(const std::string_view & name, String & } template -bool BaseSettings::isChanged(const std::string_view & name) const +bool BaseSettings::isChanged(std::string_view name) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -316,20 +316,20 @@ void BaseSettings::resetToDefault() } template -bool BaseSettings::hasBuiltin(const std::string_view & name) +bool BaseSettings::hasBuiltin(std::string_view name) { const auto & accessor = Traits::Accessor::instance(); return (accessor.find(name) != static_cast(-1)); } template -bool BaseSettings::hasCustom(const std::string_view & name) const +bool BaseSettings::hasCustom(std::string_view name) const { return tryGetCustomSetting(name); } template -const char * BaseSettings::getTypeName(const std::string_view & name) const +const char * BaseSettings::getTypeName(std::string_view name) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -341,7 +341,7 @@ const char * BaseSettings::getTypeName(const std::string_view & name) c } template -const char * BaseSettings::getDescription(const std::string_view & name) const +const char * BaseSettings::getDescription(std::string_view name) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -353,19 +353,19 @@ const char * BaseSettings::getDescription(const std::string_view & name } template -void BaseSettings::checkCanSet(const std::string_view & name, const Field & value) +void BaseSettings::checkCanSet(std::string_view name, const Field & value) { castValueUtil(name, value); } template -void BaseSettings::checkCanSetString(const std::string_view & name, const String & str) +void BaseSettings::checkCanSetString(std::string_view name, const String & str) { stringToValueUtil(name, str); } template -Field BaseSettings::castValueUtil(const std::string_view & name, const Field & value) +Field BaseSettings::castValueUtil(std::string_view name, const Field & value) { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -377,7 +377,7 @@ Field BaseSettings::castValueUtil(const std::string_view & name, const } template -String BaseSettings::valueToStringUtil(const std::string_view & name, const Field & value) +String BaseSettings::valueToStringUtil(std::string_view name, const Field & value) { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) @@ -389,7 +389,7 @@ String BaseSettings::valueToStringUtil(const std::string_view & name, c } template -Field BaseSettings::stringToValueUtil(const std::string_view & name, const String & str) +Field BaseSettings::stringToValueUtil(std::string_view name, const String & str) { try { @@ -521,7 +521,7 @@ bool operator!=(const BaseSettings & left, const BaseSettings } template -SettingFieldCustom & BaseSettings::getCustomSetting(const std::string_view & name) +SettingFieldCustom & BaseSettings::getCustomSetting(std::string_view name) { if constexpr (Traits::allow_custom_settings) { @@ -537,7 +537,7 @@ SettingFieldCustom & BaseSettings::getCustomSetting(const std::string_v } template -const SettingFieldCustom & BaseSettings::getCustomSetting(const std::string_view & name) const +const SettingFieldCustom & BaseSettings::getCustomSetting(std::string_view name) const { if constexpr (Traits::allow_custom_settings) { @@ -549,7 +549,7 @@ const SettingFieldCustom & BaseSettings::getCustomSetting(const std::st } template -const SettingFieldCustom * BaseSettings::tryGetCustomSetting(const std::string_view & name) const +const SettingFieldCustom * BaseSettings::tryGetCustomSetting(std::string_view name) const { if constexpr (Traits::allow_custom_settings) { @@ -780,7 +780,7 @@ bool BaseSettings::SettingFieldRef::isObsolete() const public: \ static const Accessor & instance(); \ size_t size() const { return field_infos.size(); } \ - size_t find(const std::string_view & name) const; \ + size_t find(std::string_view name) const; \ const String & getName(size_t index) const { return field_infos[index].name; } \ const char * getTypeName(size_t index) const { return field_infos[index].type; } \ const char * getDescription(size_t index) const { return field_infos[index].description; } \ @@ -851,7 +851,7 @@ bool BaseSettings::SettingFieldRef::isObsolete() const \ SETTINGS_TRAITS_NAME::Accessor::Accessor() {} \ \ - size_t SETTINGS_TRAITS_NAME::Accessor::find(const std::string_view & name) const \ + size_t SETTINGS_TRAITS_NAME::Accessor::find(std::string_view name) const \ { \ auto it = name_to_index_map.find(name); \ if (it != name_to_index_map.end()) \ diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index 3a4b66e6266..acdfca7a7b2 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -286,7 +286,7 @@ String Field::dump() const return applyVisitor(FieldVisitorDump(), *this); } -Field Field::restoreFromDump(const std::string_view & dump_) +Field Field::restoreFromDump(std::string_view dump_) { auto show_error = [&dump_] { diff --git a/src/Core/Field.h b/src/Core/Field.h index 08274876914..f60b7e4902e 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -346,7 +346,7 @@ public: } /// Create a string inplace. - Field(const std::string_view & str) { create(str.data(), str.size()); } /// NOLINT + Field(std::string_view str) { create(str.data(), str.size()); } /// NOLINT Field(const String & str) { create(std::string_view{str}); } /// NOLINT Field(String && str) { create(std::move(str)); } /// NOLINT Field(const char * str) { create(std::string_view{str}); } /// NOLINT @@ -403,7 +403,7 @@ public: return *this; } - Field & operator= (const std::string_view & str); + Field & operator= (std::string_view str); Field & operator= (const String & str) { return *this = std::string_view{str}; } Field & operator= (String && str); Field & operator= (const char * str) { return *this = std::string_view{str}; } @@ -631,7 +631,7 @@ public: } String dump() const; - static Field restoreFromDump(const std::string_view & dump_); + static Field restoreFromDump(std::string_view dump_); private: std::aligned_union_t()) {} - SettingFieldString & operator =(const std::string_view & str) { value = str; changed = true; return *this; } + SettingFieldString & operator =(std::string_view str) { value = str; changed = true; return *this; } SettingFieldString & operator =(const String & str) { *this = std::string_view{str}; return *this; } SettingFieldString & operator =(String && str) { value = std::move(str); changed = true; return *this; } SettingFieldString & operator =(const char * str) { *this = std::string_view{str}; return *this; } @@ -256,7 +256,7 @@ struct SettingFieldEnum struct SettingFieldEnumHelpers { - static void writeBinary(const std::string_view & str, WriteBuffer & out); + static void writeBinary(std::string_view str, WriteBuffer & out); static String readBinary(ReadBuffer & in); }; @@ -286,7 +286,7 @@ void SettingFieldEnum::readBinary(ReadBuffer & in) { \ using EnumType = ENUM_TYPE; \ static const String & toString(EnumType value); \ - static EnumType fromString(const std::string_view & str); \ + static EnumType fromString(std::string_view str); \ }; \ \ using SettingField##NEW_NAME = SettingFieldEnum; @@ -310,7 +310,7 @@ void SettingFieldEnum::readBinary(ReadBuffer & in) ERROR_CODE_FOR_UNEXPECTED_NAME); \ } \ \ - typename SettingField##NEW_NAME::EnumType SettingField##NEW_NAME##Traits::fromString(const std::string_view & str) \ + typename SettingField##NEW_NAME::EnumType SettingField##NEW_NAME##Traits::fromString(std::string_view str) \ { \ static const std::unordered_map map = [] { \ std::unordered_map res; \ @@ -430,7 +430,7 @@ void SettingFieldMultiEnum::readBinary(ReadBuffer & in) using EnumType = ENUM_TYPE; \ static size_t getEnumSize(); \ static const String & toString(EnumType value); \ - static EnumType fromString(const std::string_view & str); \ + static EnumType fromString(std::string_view str); \ }; \ \ using SettingField##NEW_NAME = SettingFieldMultiEnum; diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index 5dae2b7b413..b28b70f676a 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -54,7 +54,7 @@ std::pair splitName(const std::string & name, bool rev return {name.substr(0, idx), name.substr(idx + 1)}; } -std::pair splitName(const std::string_view & name, bool reverse) +std::pair splitName(std::string_view name, bool reverse) { auto idx = (reverse ? name.find_last_of('.') : name.find_first_of('.')); if (idx == std::string::npos || idx == 0 || idx + 1 == name.size()) diff --git a/src/DataTypes/NestedUtils.h b/src/DataTypes/NestedUtils.h index e7cda541f47..38da382254c 100644 --- a/src/DataTypes/NestedUtils.h +++ b/src/DataTypes/NestedUtils.h @@ -13,7 +13,7 @@ namespace Nested /// Splits name of compound identifier by first/last dot (depending on 'reverse' parameter). std::pair splitName(const std::string & name, bool reverse = false); - std::pair splitName(const std::string_view & name, bool reverse = false); + std::pair splitName(std::string_view name, bool reverse = false); /// Returns the prefix of the name to the first '.'. Or the name is unchanged if there is no dot. std::string extractTableName(const std::string & nested_name); diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 9006c9276d4..b9af9d61da0 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -77,18 +77,18 @@ namespace return convertChar(c1) == convertChar(c2); } - static bool equals(const std::string_view & s1, const std::string_view & s2) + static bool equals(std::string_view s1, std::string_view s2) { return (s1.length() == s2.length()) && std::equal(s1.begin(), s1.end(), s2.begin(), [](char c1, char c2) { return convertChar(c1) == convertChar(c2); }); } - static bool less(const std::string_view & s1, const std::string_view & s2) + static bool less(std::string_view s1, std::string_view s2) { return std::lexicographical_compare(s1.begin(), s1.end(), s2.begin(), s2.end(), [](char c1, char c2) { return convertChar(c1) < convertChar(c2); }); } - static bool startsWith(const std::string_view & s1, const std::string_view & s2) + static bool startsWith(std::string_view s1, std::string_view s2) { return (s1.length() >= s2.length()) && equals(s1.substr(0, s2.length()), s2); } @@ -195,7 +195,7 @@ namespace { protected: ProtobufSerializerSingleValue( - const std::string_view & column_name_, + std::string_view column_name_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) : column_name(column_name_) @@ -264,7 +264,7 @@ namespace return reader->readFixed(); } - void writeStr(const std::string_view & str) + void writeStr(std::string_view str) { if (!str.empty() || !skip_zero_or_empty) writer->writeString(field_tag, str); @@ -274,7 +274,7 @@ namespace void readStrAndAppend(PaddedPODArray & str) { reader->readStringAndAppend(str); } template - DestType parseFromStr(const std::string_view & str) const + DestType parseFromStr(std::string_view str) const { try { @@ -307,7 +307,7 @@ namespace return result; } - [[noreturn]] void incompatibleColumnType(const std::string_view & column_type) const + [[noreturn]] void incompatibleColumnType(std::string_view column_type) const { throw Exception( ErrorCodes::DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD, @@ -318,7 +318,7 @@ namespace field_descriptor.type_name()); } - [[noreturn]] void cannotConvertValue(const std::string_view & src_value, const std::string_view & src_type_name, const std::string_view & dest_type_name) const + [[noreturn]] void cannotConvertValue(std::string_view src_value, std::string_view src_type_name, std::string_view dest_type_name) const { throw Exception( "Could not convert value '" + String{src_value} + "' from type " + String{src_type_name} + " to type " @@ -351,7 +351,7 @@ namespace public: using ColumnType = ColumnVector; - ProtobufSerializerNumber(const std::string_view & column_name_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) + ProtobufSerializerNumber(std::string_view column_name_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) : ProtobufSerializerSingleValue(column_name_, field_descriptor_, reader_or_writer_) { setFunctions(); @@ -590,7 +590,7 @@ namespace using ColumnType = std::conditional_t; ProtobufSerializerString( - const std::string_view & column_name_, + std::string_view column_name_, const std::shared_ptr & fixed_string_data_type_, const google::protobuf::FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) @@ -604,7 +604,7 @@ namespace } ProtobufSerializerString( - const std::string_view & column_name_, + std::string_view column_name_, const google::protobuf::FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) : ProtobufSerializerSingleValue(column_name_, field_descriptor_, reader_or_writer_) @@ -727,7 +727,7 @@ namespace { case FieldTypeId::TYPE_INT32: { - write_function = [this](const std::string_view & str) { writeInt(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeInt(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readInt(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_int32()); }; break; @@ -735,7 +735,7 @@ namespace case FieldTypeId::TYPE_SINT32: { - write_function = [this](const std::string_view & str) { writeSInt(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeSInt(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readSInt(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_int32()); }; break; @@ -743,7 +743,7 @@ namespace case FieldTypeId::TYPE_UINT32: { - write_function = [this](const std::string_view & str) { writeUInt(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeUInt(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readUInt(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_uint32()); }; break; @@ -751,7 +751,7 @@ namespace case FieldTypeId::TYPE_INT64: { - write_function = [this](const std::string_view & str) { writeInt(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeInt(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readInt(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_int64()); }; break; @@ -759,7 +759,7 @@ namespace case FieldTypeId::TYPE_SINT64: { - write_function = [this](const std::string_view & str) { writeSInt(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeSInt(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readSInt(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_int64()); }; break; @@ -767,7 +767,7 @@ namespace case FieldTypeId::TYPE_UINT64: { - write_function = [this](const std::string_view & str) { writeUInt(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeUInt(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readUInt(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_uint64()); }; break; @@ -775,7 +775,7 @@ namespace case FieldTypeId::TYPE_FIXED32: { - write_function = [this](const std::string_view & str) { writeFixed(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeFixed(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readFixed(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_uint32()); }; break; @@ -783,7 +783,7 @@ namespace case FieldTypeId::TYPE_SFIXED32: { - write_function = [this](const std::string_view & str) { writeFixed(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeFixed(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readFixed(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_int32()); }; break; @@ -791,7 +791,7 @@ namespace case FieldTypeId::TYPE_FIXED64: { - write_function = [this](const std::string_view & str) { writeFixed(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeFixed(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readFixed(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_uint64()); }; break; @@ -799,7 +799,7 @@ namespace case FieldTypeId::TYPE_SFIXED64: { - write_function = [this](const std::string_view & str) { writeFixed(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeFixed(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readFixed(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_int64()); }; break; @@ -807,7 +807,7 @@ namespace case FieldTypeId::TYPE_FLOAT: { - write_function = [this](const std::string_view & str) { writeFixed(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeFixed(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readFixed(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_float()); }; break; @@ -815,7 +815,7 @@ namespace case FieldTypeId::TYPE_DOUBLE: { - write_function = [this](const std::string_view & str) { writeFixed(parseFromStr(str)); }; + write_function = [this](std::string_view str) { writeFixed(parseFromStr(str)); }; read_function = [this](PaddedPODArray & str) { toStringAppend(readFixed(), str); }; default_function = [this]() -> String { return toString(field_descriptor.default_value_double()); }; break; @@ -823,7 +823,7 @@ namespace case FieldTypeId::TYPE_BOOL: { - write_function = [this](const std::string_view & str) + write_function = [this](std::string_view str) { if (str == "true") writeUInt(1); @@ -855,7 +855,7 @@ namespace case FieldTypeId::TYPE_STRING: case FieldTypeId::TYPE_BYTES: { - write_function = [this](const std::string_view & str) { writeStr(str); }; + write_function = [this](std::string_view str) { writeStr(str); }; read_function = [this](PaddedPODArray & str) { readStrAndAppend(str); }; default_function = [this]() -> String { return field_descriptor.default_value_string(); }; break; @@ -863,7 +863,7 @@ namespace case FieldTypeId::TYPE_ENUM: { - write_function = [this](const std::string_view & str) { writeInt(stringToProtobufEnumValue(str)); }; + write_function = [this](std::string_view str) { writeInt(stringToProtobufEnumValue(str)); }; read_function = [this](PaddedPODArray & str) { protobufEnumValueToStringAppend(readInt(), str); }; default_function = [this]() -> String { return field_descriptor.default_value_enum()->name(); }; break; @@ -908,7 +908,7 @@ namespace } } - int stringToProtobufEnumValue(const std::string_view & str) const + int stringToProtobufEnumValue(std::string_view str) const { auto it = string_to_protobuf_enum_value_map.find(str); if (it == string_to_protobuf_enum_value_map.end()) @@ -932,7 +932,7 @@ namespace const std::shared_ptr fixed_string_data_type; const size_t n = 0; - std::function write_function; + std::function write_function; std::function &)> read_function; std::function default_function; std::unordered_map string_to_protobuf_enum_value_map; @@ -953,7 +953,7 @@ namespace using BaseClass = ProtobufSerializerNumber; ProtobufSerializerEnum( - const std::string_view & column_name_, + std::string_view column_name_, const std::shared_ptr & enum_data_type_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) @@ -1067,7 +1067,7 @@ namespace protobuf_enum_value_to_enum_data_type_value_map.emplace(protobuf_enum_value, enum_data_type_value); }; - auto iless = [](const std::string_view & s1, const std::string_view & s2) { return ColumnNameWithProtobufFieldNameComparator::less(s1, s2); }; + auto iless = [](std::string_view s1, std::string_view s2) { return ColumnNameWithProtobufFieldNameComparator::less(s1, s2); }; boost::container::flat_map string_to_protobuf_enum_value_map; typename decltype(string_to_protobuf_enum_value_map)::sequence_type string_to_protobuf_enum_value_seq; for (int i : collections::range(enum_descriptor.value_count())) @@ -1133,9 +1133,9 @@ namespace Int64 readInt() { return ProtobufSerializerSingleValue::readInt(); } void writeInt(Int64 value) { ProtobufSerializerSingleValue::writeInt(value); } - void writeStr(const std::string_view & str) { ProtobufSerializerSingleValue::writeStr(str); } + void writeStr(std::string_view str) { ProtobufSerializerSingleValue::writeStr(str); } void readStr(String & str) { ProtobufSerializerSingleValue::readStr(str); } - [[noreturn]] void cannotConvertValue(const std::string_view & src_value, const std::string_view & src_type_name, const std::string_view & dest_type_name) const { ProtobufSerializerSingleValue::cannotConvertValue(src_value, src_type_name, dest_type_name); } + [[noreturn]] void cannotConvertValue(std::string_view src_value, std::string_view src_type_name, std::string_view dest_type_name) const { ProtobufSerializerSingleValue::cannotConvertValue(src_value, src_type_name, dest_type_name); } const std::shared_ptr enum_data_type; std::unordered_map enum_data_type_value_to_protobuf_enum_value_map; @@ -1152,7 +1152,7 @@ namespace using ColumnType = ColumnDecimal; ProtobufSerializerDecimal( - const std::string_view & column_name_, + std::string_view column_name_, const DataTypeDecimalBase & decimal_data_type_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) @@ -1412,7 +1412,7 @@ namespace { public: ProtobufSerializerDate( - const std::string_view & column_name_, + std::string_view column_name_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) : ProtobufSerializerNumber(column_name_, field_descriptor_, reader_or_writer_) @@ -1490,7 +1490,7 @@ namespace { public: ProtobufSerializerDateTime( - const std::string_view & column_name_, + std::string_view column_name_, const DataTypeDateTime & type, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) @@ -1574,7 +1574,7 @@ namespace { public: ProtobufSerializerUUID( - const std::string_view & column_name_, + std::string_view column_name_, const google::protobuf::FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) : ProtobufSerializerSingleValue(column_name_, field_descriptor_, reader_or_writer_) @@ -1654,7 +1654,7 @@ namespace { public: ProtobufSerializerAggregateFunction( - const std::string_view & column_name_, + std::string_view column_name_, const std::shared_ptr & aggregate_function_data_type_, const google::protobuf::FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) @@ -2061,7 +2061,7 @@ namespace { public: ProtobufSerializerTupleAsArray( - const std::string_view & column_name_, + std::string_view column_name_, const std::shared_ptr & tuple_data_type_, const FieldDescriptor & field_descriptor_, std::vector> element_serializers_) @@ -2833,7 +2833,7 @@ namespace return field_names; } - static bool columnNameEqualsToFieldName(const std::string_view & column_name, const FieldDescriptor & field_descriptor) + static bool columnNameEqualsToFieldName(std::string_view column_name, const FieldDescriptor & field_descriptor) { std::string_view suffix; return columnNameStartsWithFieldName(column_name, field_descriptor, suffix) && suffix.empty(); @@ -2844,7 +2844,7 @@ namespace /// which doesn't match to the field's name. /// The function requires that rest part of the column's name to be started with a dot '.' or underline '_', /// but doesn't include those '.' or '_' characters into `suffix`. - static bool columnNameStartsWithFieldName(const std::string_view & column_name, const FieldDescriptor & field_descriptor, std::string_view & suffix) + static bool columnNameStartsWithFieldName(std::string_view column_name, const FieldDescriptor & field_descriptor, std::string_view & suffix) { size_t matching_length = 0; const MessageDescriptor & containing_type = *field_descriptor.containing_type(); @@ -2887,7 +2887,7 @@ namespace /// for that case suffixes are also returned. /// This is only the first filter, buildMessageSerializerImpl() does other checks after calling this function. static bool findFieldsByColumnName( - const std::string_view & column_name, + std::string_view column_name, const MessageDescriptor & message_descriptor, std::vector> & out_field_descriptors_with_suffixes, bool google_wrappers_special_treatment) @@ -3030,7 +3030,7 @@ namespace used_column_indices_sorted.reserve(num_columns); size_t sequential_column_index = 0; - auto add_field_serializer = [&](const std::string_view & column_name_, + auto add_field_serializer = [&](std::string_view column_name_, std::vector && column_indices_, const FieldDescriptor & field_descriptor_, std::unique_ptr field_serializer_) @@ -3243,7 +3243,7 @@ namespace /// Builds a serializer for one-to-one match: /// one column is serialized as one field in the protobuf message. std::unique_ptr buildFieldSerializer( - const std::string_view & column_name, + std::string_view column_name, const DataTypePtr & data_type, const FieldDescriptor & field_descriptor, bool allow_repeat, @@ -3395,7 +3395,7 @@ namespace } } - [[noreturn]] static void throwFieldNotRepeated(const FieldDescriptor & field_descriptor, const std::string_view & column_name) + [[noreturn]] static void throwFieldNotRepeated(const FieldDescriptor & field_descriptor, std::string_view column_name) { if (!field_descriptor.is_repeated()) throw Exception( diff --git a/src/Formats/ProtobufWriter.cpp b/src/Formats/ProtobufWriter.cpp index ece4f78b1c8..da680fae601 100644 --- a/src/Formats/ProtobufWriter.cpp +++ b/src/Formats/ProtobufWriter.cpp @@ -196,7 +196,7 @@ template void ProtobufWriter::writeFixed(int field_number, UInt64 value) template void ProtobufWriter::writeFixed(int field_number, Float32 value); template void ProtobufWriter::writeFixed(int field_number, Float64 value); -void ProtobufWriter::writeString(int field_number, const std::string_view & str) +void ProtobufWriter::writeString(int field_number, std::string_view str) { size_t length = str.length(); size_t old_size = buffer.size(); diff --git a/src/Formats/ProtobufWriter.h b/src/Formats/ProtobufWriter.h index 1dcc8f4ef7c..3ede956e910 100644 --- a/src/Formats/ProtobufWriter.h +++ b/src/Formats/ProtobufWriter.h @@ -30,7 +30,7 @@ public: void writeSInt(int field_number, Int64 value); template void writeFixed(int field_number, T value); - void writeString(int field_number, const std::string_view & str); + void writeString(int field_number, std::string_view str); void startRepeatedPack(); void endRepeatedPack(int field_number, bool skip_if_empty); diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index fa573ac829a..2f10a3c773b 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -304,7 +304,7 @@ private: /// Performs moves of types MoveType::Key and MoveType::ConstKey. template - static bool moveToElementByKey(typename JSONParser::Element & element, const std::string_view & key) + static bool moveToElementByKey(typename JSONParser::Element & element, std::string_view key) { if (!element.isObject()) return false; @@ -504,7 +504,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view) { ColumnVector & col_vec = assert_cast &>(dest); col_vec.insertValue(1); @@ -532,7 +532,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &) { return 0; } - static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view) { /// This function is called only if JSON is valid. /// If JSON isn't valid then `FunctionJSON::Executor::run()` adds default value (=zero) to `dest` without calling this function. @@ -556,7 +556,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { size_t size; if (element.isArray()) @@ -586,7 +586,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view & last_key) + static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view last_key) { if (last_key.empty()) return false; @@ -620,7 +620,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { UInt8 type; if (element.isInt64()) @@ -662,7 +662,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { NumberType value; @@ -737,7 +737,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { if (!element.isBool()) return false; @@ -764,7 +764,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { if (element.isNull()) return false; @@ -1164,7 +1164,7 @@ public: extract_tree = JSONExtractTree::build(function_name, result_type); } - bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { return extract_tree->insertResultToColumn(dest, element); } @@ -1207,7 +1207,7 @@ public: extract_tree = JSONExtractTree::build(function_name, value_type); } - bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { if (!element.isObject()) return false; @@ -1251,7 +1251,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { ColumnString & col_str = assert_cast(dest); auto & chars = col_str.getChars(); @@ -1355,7 +1355,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { if (!element.isArray()) return false; @@ -1387,7 +1387,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { if (!element.isObject()) return false; @@ -1423,7 +1423,7 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { if (!element.isObject()) return false; diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 4cd07dddf25..57283a396d9 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -836,7 +836,7 @@ template inline T parse(const char * data, size_t size); template -inline T parseFromString(const std::string_view & str) +inline T parseFromString(std::string_view str) { return parse(str.data(), str.size()); } @@ -1338,7 +1338,7 @@ inline T parseWithSizeSuffix(const char * data, size_t size) } template -inline T parseWithSizeSuffix(const std::string_view & s) +inline T parseWithSizeSuffix(std::string_view s) { return parseWithSizeSuffix(s.data(), s.size()); } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 5eab75f14b1..c3bbaac097d 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -113,7 +113,7 @@ inline void writeStringBinary(const char * s, WriteBuffer & buf) writeStringBinary(StringRef{s}, buf); } -inline void writeStringBinary(const std::string_view & s, WriteBuffer & buf) +inline void writeStringBinary(std::string_view s, WriteBuffer & buf) { writeStringBinary(StringRef{s}, buf); } @@ -365,7 +365,7 @@ inline void writeJSONString(const StringRef & s, WriteBuffer & buf, const Format writeJSONString(s.data, s.data + s.size, buf, settings); } -inline void writeJSONString(const std::string_view & s, WriteBuffer & buf, const FormatSettings & settings) +inline void writeJSONString(std::string_view s, WriteBuffer & buf, const FormatSettings & settings) { writeJSONString(StringRef{s}, buf, settings); } @@ -440,7 +440,7 @@ inline void writeEscapedString(const StringRef & ref, WriteBuffer & buf) writeEscapedString(ref.data, ref.size, buf); } -inline void writeEscapedString(const std::string_view & ref, WriteBuffer & buf) +inline void writeEscapedString(std::string_view ref, WriteBuffer & buf) { writeEscapedString(ref.data(), ref.size(), buf); } @@ -478,7 +478,7 @@ inline void writeQuotedString(const StringRef & ref, WriteBuffer & buf) writeAnyQuotedString<'\''>(ref, buf); } -inline void writeQuotedString(const std::string_view & ref, WriteBuffer & buf) +inline void writeQuotedString(std::string_view ref, WriteBuffer & buf) { writeAnyQuotedString<'\''>(ref.data(), ref.data() + ref.size(), buf); } @@ -493,7 +493,7 @@ inline void writeDoubleQuotedString(const StringRef & s, WriteBuffer & buf) writeAnyQuotedString<'"'>(s, buf); } -inline void writeDoubleQuotedString(const std::string_view & s, WriteBuffer & buf) +inline void writeDoubleQuotedString(std::string_view s, WriteBuffer & buf) { writeAnyQuotedString<'"'>(s.data(), s.data() + s.size(), buf); } @@ -891,7 +891,7 @@ inline void writeBinary(const T & x, WriteBuffer & buf) { writePODBinary(x, buf) inline void writeBinary(const String & x, WriteBuffer & buf) { writeStringBinary(x, buf); } inline void writeBinary(const StringRef & x, WriteBuffer & buf) { writeStringBinary(x, buf); } -inline void writeBinary(const std::string_view & x, WriteBuffer & buf) { writeStringBinary(x, buf); } +inline void writeBinary(std::string_view x, WriteBuffer & buf) { writeStringBinary(x, buf); } inline void writeBinary(const Decimal32 & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const Decimal64 & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const Decimal128 & x, WriteBuffer & buf) { writePODBinary(x, buf); } @@ -1015,7 +1015,7 @@ inline void writeQuoted(const T & x, WriteBuffer & buf) { writeText(x, buf); } inline void writeQuoted(const String & x, WriteBuffer & buf) { writeQuotedString(x, buf); } -inline void writeQuoted(const std::string_view & x, WriteBuffer & buf) { writeQuotedString(x, buf); } +inline void writeQuoted(std::string_view x, WriteBuffer & buf) { writeQuotedString(x, buf); } inline void writeQuoted(const StringRef & x, WriteBuffer & buf) { writeQuotedString(x, buf); } @@ -1048,7 +1048,7 @@ inline void writeDoubleQuoted(const T & x, WriteBuffer & buf) { writeText(x, buf inline void writeDoubleQuoted(const String & x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); } -inline void writeDoubleQuoted(const std::string_view & x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); } +inline void writeDoubleQuoted(std::string_view x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); } inline void writeDoubleQuoted(const StringRef & x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b83b6420548..3cee7db6434 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -854,13 +854,13 @@ void Context::checkAccessImpl(const Args &... args) const } void Context::checkAccess(const AccessFlags & flags) const { return checkAccessImpl(flags); } -void Context::checkAccess(const AccessFlags & flags, const std::string_view & database) const { return checkAccessImpl(flags, database); } -void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(flags, database, table); } -void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(flags, database, table, column); } -void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } -void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database) const { return checkAccessImpl(flags, database); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const { return checkAccessImpl(flags, database, table); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return checkAccessImpl(flags, database, table, column); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName()); } -void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::string_view & column) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), column); } +void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, std::string_view column) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), column); } void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::vector & columns) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); } void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const Strings & columns) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); } void Context::checkAccess(const AccessRightsElement & element) const { return checkAccessImpl(element); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 37c6b4c9caa..a4f69602a61 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -474,13 +474,13 @@ public: /// Checks access rights. /// Empty database means the current database. void checkAccess(const AccessFlags & flags) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + void checkAccess(const AccessFlags & flags, std::string_view database) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; void checkAccess(const AccessFlags & flags, const StorageID & table_id) const; - void checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::string_view & column) const; + void checkAccess(const AccessFlags & flags, const StorageID & table_id, std::string_view column) const; void checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::vector & columns) const; void checkAccess(const AccessFlags & flags, const StorageID & table_id, const Strings & columns) const; void checkAccess(const AccessRightsElement & element) const; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index bae2aed2cd5..587d58d9ad3 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -234,7 +234,7 @@ void DatabaseCatalog::shutdownImpl() view_dependencies.clear(); } -bool DatabaseCatalog::isPredefinedDatabase(const std::string_view & database_name) +bool DatabaseCatalog::isPredefinedDatabase(std::string_view database_name) { return database_name == TEMPORARY_DATABASE || database_name == SYSTEM_DATABASE || database_name == INFORMATION_SCHEMA || database_name == INFORMATION_SCHEMA_UPPERCASE; diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 133cf0c5126..d82ad56eadd 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -131,7 +131,7 @@ public: static constexpr const char * INFORMATION_SCHEMA_UPPERCASE = "INFORMATION_SCHEMA"; /// Returns true if a passed name is one of the predefined databases' names. - static bool isPredefinedDatabase(const std::string_view & database_name); + static bool isPredefinedDatabase(std::string_view database_name); static DatabaseCatalog & init(ContextMutablePtr global_context_); static DatabaseCatalog & instance(); diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index b58b90b6d47..070fac7ccbd 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -31,7 +31,7 @@ namespace ErrorCodes extern const int UNSUPPORTED_JOIN_KEYS; extern const int LOGICAL_ERROR; } -bool TranslateQualifiedNamesMatcher::Data::matchColumnName(const std::string_view & name, const String & column_name, DataTypePtr column_type) +bool TranslateQualifiedNamesMatcher::Data::matchColumnName(std::string_view name, const String & column_name, DataTypePtr column_type) { if (name.size() < column_name.size()) return false; diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.h b/src/Interpreters/TranslateQualifiedNamesVisitor.h index e0c2f6b6bc0..73e45fc7ea0 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.h +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.h @@ -39,7 +39,7 @@ public: bool hasTable() const { return !tables.empty(); } bool processAsterisks() const { return hasTable() && has_columns; } bool unknownColumn(size_t table_pos, const ASTIdentifier & identifier) const; - static bool matchColumnName(const std::string_view & name, const String & column_name, DataTypePtr column_type); + static bool matchColumnName(std::string_view name, const String & column_name, DataTypePtr column_type); }; static void visit(ASTPtr & ast, Data & data); diff --git a/src/Interpreters/UserDefinedSQLObjectsLoader.cpp b/src/Interpreters/UserDefinedSQLObjectsLoader.cpp index 75b91f3a817..c6f50fc4a0a 100644 --- a/src/Interpreters/UserDefinedSQLObjectsLoader.cpp +++ b/src/Interpreters/UserDefinedSQLObjectsLoader.cpp @@ -43,7 +43,7 @@ UserDefinedSQLObjectsLoader::UserDefinedSQLObjectsLoader() : log(&Poco::Logger::get("UserDefinedSQLObjectsLoader")) {} -void UserDefinedSQLObjectsLoader::loadUserDefinedObject(ContextPtr context, UserDefinedSQLObjectType object_type, const std::string_view & name, const String & path) +void UserDefinedSQLObjectsLoader::loadUserDefinedObject(ContextPtr context, UserDefinedSQLObjectType object_type, std::string_view name, const String & path) { auto name_ref = StringRef(name.data(), name.size()); LOG_DEBUG(log, "Loading user defined object {} from file {}", backQuote(name_ref), path); diff --git a/src/Interpreters/UserDefinedSQLObjectsLoader.h b/src/Interpreters/UserDefinedSQLObjectsLoader.h index 2e747f67a8d..9dfba1181c1 100644 --- a/src/Interpreters/UserDefinedSQLObjectsLoader.h +++ b/src/Interpreters/UserDefinedSQLObjectsLoader.h @@ -29,7 +29,7 @@ public: private: - void loadUserDefinedObject(ContextPtr context, UserDefinedSQLObjectType object_type, const std::string_view & object_name, const String & file_path); + void loadUserDefinedObject(ContextPtr context, UserDefinedSQLObjectType object_type, std::string_view object_name, const String & file_path); Poco::Logger * log; bool enable_persistence = true; }; diff --git a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp index 83156c6a8e1..2c25fc14e7d 100644 --- a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp @@ -75,7 +75,7 @@ namespace { for (auto filter_type : collections::range(RowPolicyFilterType::MAX)) { - const std::string_view & command = RowPolicyFilterTypeInfo::get(filter_type).command; + std::string_view command = RowPolicyFilterTypeInfo::get(filter_type).command; commands.emplace(command); } } @@ -96,7 +96,7 @@ namespace for (auto filter_type : collections::range(RowPolicyFilterType::MAX)) { - const std::string_view & command = RowPolicyFilterTypeInfo::get(filter_type).command; + std::string_view command = RowPolicyFilterTypeInfo::get(filter_type).command; if (ParserKeyword{command.data()}.ignore(pos, expected)) { res_commands.emplace(command); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 40f23fe5294..daf31698aad 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -2050,7 +2050,7 @@ bool KeyCondition::mayBeTrueInRange( } String KeyCondition::RPNElement::toString() const { return toString("column " + std::to_string(key_column), false); } -String KeyCondition::RPNElement::toString(const std::string_view & column_name, bool print_constants) const +String KeyCondition::RPNElement::toString(std::string_view column_name, bool print_constants) const { auto print_wrapped_column = [this, &column_name, print_constants](WriteBuffer & buf) { diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 9a8719afa19..af85a90dd62 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -320,7 +320,7 @@ private: : function(function_), range(range_), key_column(key_column_) {} String toString() const; - String toString(const std::string_view & column_name, bool print_constants) const; + String toString(std::string_view column_name, bool print_constants) const; Function function = FUNCTION_UNKNOWN; diff --git a/src/Storages/System/StorageSystemPrivileges.cpp b/src/Storages/System/StorageSystemPrivileges.cpp index 8cf1accfe34..70163979f72 100644 --- a/src/Storages/System/StorageSystemPrivileges.cpp +++ b/src/Storages/System/StorageSystemPrivileges.cpp @@ -85,7 +85,7 @@ void StorageSystemPrivileges::fillData(MutableColumns & res_columns, ContextPtr, auto & column_parent_group = assert_cast(assert_cast(*res_columns[column_index]).getNestedColumn()).getData(); auto & column_parent_group_null_map = assert_cast(*res_columns[column_index++]).getNullMapData(); - auto add_row = [&](AccessType access_type, const std::string_view & aliases, Level max_level, AccessType parent_group) + auto add_row = [&](AccessType access_type, std::string_view aliases, Level max_level, AccessType parent_group) { column_access_type.push_back(static_cast(access_type)); From 9c443038c7ecd75a10806287a15c83d8a2958cf6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 14 Jul 2022 18:28:25 +0200 Subject: [PATCH 212/659] Update low_cardinality_argument.xml --- tests/performance/low_cardinality_argument.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/low_cardinality_argument.xml b/tests/performance/low_cardinality_argument.xml index 8ba7b6d09be..ba17809ba53 100644 --- a/tests/performance/low_cardinality_argument.xml +++ b/tests/performance/low_cardinality_argument.xml @@ -3,7 +3,7 @@ CREATE TABLE test_lc (x UInt64, lc LowCardinality(Nullable(String))) ENGINE = MergeTree order by x - INSERT INTO test_lc SELECT number, number % 10 ? NULL : toString(number % 10000) FROM numbers(1000000) + INSERT INTO test_lc SELECT number, number % 10 ? NULL : toString(number % 10000) FROM numbers(10000000) SELECT isNullable(lc) FROM test_lc FORMAT Null SELECT isConstant(lc) FROM test_lc FORMAT Null From efb418a7e78d10b6aa6c1cd85fd25545ff6e679f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 14 Jul 2022 16:30:59 +0000 Subject: [PATCH 213/659] fix --- tests/queries/0_stateless/02356_insert_query_log_metrics.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql index 3825f37183b..347073f868f 100644 --- a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql +++ b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql @@ -1,4 +1,4 @@ CREATE TABLE 02356_destination (a Int64, b String) ENGINE = Memory; INSERT INTO 02356_destination (a, b) SELECT * FROM generateRandom('a Int64, b String') LIMIT 100 SETTINGS max_threads=1, max_block_size=100; -SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' FORMAT CSV; \ No newline at end of file +SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' order by event_time desc limit 1 FORMAT CSV; \ No newline at end of file From 3fd327b23a39d97ef4dcb08843f653cc25e8b944 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 14 Jul 2022 19:25:40 +0200 Subject: [PATCH 214/659] Use std::filesystem::path::operator/ --- src/Common/filesystemHelpers.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 6f6afcac657..1e8e53bf1ea 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -87,7 +87,7 @@ BlockDeviceType getBlockDeviceType([[maybe_unused]] const String & device_id) #if defined(OS_LINUX) try { - const String path = "/sys/dev/block/" + device_id + "/queue/rotational"; + const auto path{std::filesystem::path("/sys/dev/block/") / device_id / "queue/rotational"}; if (!std::filesystem::exists(path)) return BlockDeviceType::UNKNOWN; ReadBufferFromFile in(path); @@ -112,7 +112,8 @@ UInt64 getBlockDeviceReadAheadBytes([[maybe_unused]] const String & device_id) #if defined(OS_LINUX) try { - ReadBufferFromFile in("/sys/dev/block/" + device_id + "/queue/read_ahead_kb"); + const auto path{std::filesystem::path("/sys/dev/block/") / device_id / "queue/read_ahead_kb"}; + ReadBufferFromFile in(path); int read_ahead_kb; readText(read_ahead_kb, in); return read_ahead_kb * 1024; From 3c9d4c537a56f5cd026684997c15dbb7d1520675 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Jul 2022 18:36:50 +0000 Subject: [PATCH 215/659] Another try. --- src/Interpreters/ExpressionAnalyzer.cpp | 77 ++++++++++++++----------- src/Interpreters/ExpressionAnalyzer.h | 2 +- 2 files changed, 43 insertions(+), 36 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index e2c106ff796..23258c60099 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1253,7 +1253,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin( } ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( - ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns) + ExpressionActionsChain & chain, bool only_types) { const auto * select_query = getSelectQuery(); if (!select_query->prewhere()) @@ -1290,14 +1290,6 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( NameSet required_source_columns(required_columns.begin(), required_columns.end()); required_source_columns.insert(first_action_names.begin(), first_action_names.end()); - /// Add required columns to required output in order not to remove them after prewhere execution. - /// TODO: add sampling and final execution to common chain. - for (const auto & column : additional_required_columns) - { - if (required_source_columns.contains(column)) - step.addRequiredOutput(column); - } - auto names = step.actions()->getNames(); NameSet name_set(names.begin(), names.end()); @@ -1844,12 +1836,28 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( const Settings & settings = context->getSettingsRef(); const ConstStoragePtr & storage = query_analyzer.storage(); + Names additional_required_columns_after_prewhere; ssize_t prewhere_step_num = -1; ssize_t where_step_num = -1; ssize_t having_step_num = -1; auto finalize_chain = [&](ExpressionActionsChain & chain) { + if (prewhere_step_num >= 0) + { + ExpressionActionsChain::Step & step = *chain.steps.at(prewhere_step_num); + + auto required_columns = prewhere_info->prewhere_actions->getRequiredColumnsNames(); + NameSet required_source_columns(required_columns.begin(), required_columns.end()); + /// Add required columns to required output in order not to remove them after prewhere execution. + /// TODO: add sampling and final execution to common chain. + for (const auto & column : additional_required_columns_after_prewhere) + { + if (required_source_columns.contains(column)) + step.addRequiredOutput(column); + } + } + chain.finalize(); finalize(chain, prewhere_step_num, where_step_num, having_step_num, query); @@ -1858,25 +1866,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( }; { - bool join_allow_read_in_order = true; - if (hasJoin()) - { - /// You may find it strange but we support read_in_order for HashJoin and do not support for MergeJoin. - join_has_delayed_stream = query_analyzer.analyzedJoin().needStreamWithNonJoinedRows(); - join_allow_read_in_order = typeid_cast(join.get()) && !join_has_delayed_stream; - } - - optimize_read_in_order = - settings.optimize_read_in_order - && storage - && query.orderBy() - && !query_analyzer.hasAggregation() - && !query_analyzer.hasWindow() - && !query.final() - && join_allow_read_in_order; - ExpressionActionsChain chain(context); - Names additional_required_columns_after_prewhere; if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1)) { @@ -1892,20 +1882,13 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( columns_for_final.begin(), columns_for_final.end()); } - if (storage && optimize_read_in_order) - { - Names columns_for_sorting_key = metadata_snapshot->getColumnsRequiredForSortingKey(); - additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), - columns_for_sorting_key.begin(), columns_for_sorting_key.end()); - } - if (storage && filter_info_) { filter_info = filter_info_; filter_info->do_remove_column = true; } - if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere)) + if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage)) { /// Prewhere is always the first one. prewhere_step_num = 0; @@ -1983,6 +1966,30 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( } } + bool join_allow_read_in_order = true; + if (hasJoin()) + { + /// You may find it strange but we support read_in_order for HashJoin and do not support for MergeJoin. + join_has_delayed_stream = query_analyzer.analyzedJoin().needStreamWithNonJoinedRows(); + join_allow_read_in_order = typeid_cast(join.get()) && !join_has_delayed_stream; + } + + optimize_read_in_order = + settings.optimize_read_in_order + && storage + && query.orderBy() + && !query_analyzer.hasAggregation() + && !query_analyzer.hasWindow() + && !query.final() + && join_allow_read_in_order; + + if (storage && optimize_read_in_order) + { + Names columns_for_sorting_key = metadata_snapshot->getColumnsRequiredForSortingKey(); + additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), + columns_for_sorting_key.begin(), columns_for_sorting_key.end()); + } + /// If there is aggregation, we execute expressions in SELECT and ORDER BY on the initiating server, otherwise on the source servers. query_analyzer.appendSelect(chain, only_types || (need_aggregate ? !second_stage : !first_stage)); diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index aae45482a97..019cda8b924 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -403,7 +403,7 @@ private: /// remove_filter is set in ExpressionActionsChain::finalize(); /// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier). - ActionsDAGPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns); + ActionsDAGPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types); bool appendWhere(ExpressionActionsChain & chain, bool only_types); bool appendGroupBy(ExpressionActionsChain & chain, bool only_types, bool optimize_aggregation_in_order, ManyExpressionActions &); void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types); From 03897589b90edf05a2eda9af1a9d8cae5a37bfca Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 14 Jul 2022 21:31:17 +0200 Subject: [PATCH 216/659] Fix --- src/IO/ReadWriteBufferFromHTTP.h | 89 +++++++++++++++++++++++++------- 1 file changed, 69 insertions(+), 20 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index c450ffe1747..ab358c8253a 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -44,6 +44,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int CANNOT_SEEK_THROUGH_FILE; extern const int SEEK_POSITION_OUT_OF_BOUND; + extern const int UNKNOWN_FILE_SIZE; } template @@ -119,6 +120,7 @@ namespace detail size_t offset_from_begin_pos = 0; Range read_range; + std::optional file_size; /// Delayed exception in case retries with partial content are not satisfiable. std::exception_ptr exception; @@ -201,11 +203,11 @@ namespace detail size_t getFileSize() override { - if (read_range.end) - return *read_range.end - getRangeBegin(); + if (file_size) + return *file_size; Poco::Net::HTTPResponse response; - for (size_t i = 0; i < 10; ++i) + for (size_t i = 0; i < settings.http_max_tries; ++i) { try { @@ -214,20 +216,30 @@ namespace detail } catch (const Poco::Exception & e) { + if (i == settings.http_max_tries - 1) + throw; + LOG_ERROR(log, "Failed to make HTTP_HEAD request to {}. Error: {}", uri.toString(), e.displayText()); } } if (response.hasContentLength()) - read_range.end = getRangeBegin() + response.getContentLength(); + { + if (!read_range.end) + read_range.end = getRangeBegin() + response.getContentLength(); - return *read_range.end; + file_size = response.getContentLength(); + return *file_size; + } + + throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size for: {}", uri.toString()); } String getFileName() const override { return uri.toString(); } enum class InitializeError { + RETRIABLE_ERROR, /// If error is not retriable, `exception` variable must be set. NON_RETRIABLE_ERROR, /// Allows to skip not found urls for globs @@ -401,19 +413,30 @@ namespace detail saved_uri_redirect = uri_redirect; } + if (response.hasContentLength()) + LOG_DEBUG(log, "Received response with content length: {}", response.getContentLength()); + if (withPartialContent() && response.getStatus() != Poco::Net::HTTPResponse::HTTPStatus::HTTP_PARTIAL_CONTENT) { /// Having `200 OK` instead of `206 Partial Content` is acceptable in case we retried with range.begin == 0. if (read_range.begin && *read_range.begin != 0) { if (!exception) + { exception = std::make_exception_ptr(Exception( ErrorCodes::HTTP_RANGE_NOT_SATISFIABLE, - "Cannot read with range: [{}, {}]", + "Cannot read with range: [{}, {}] (response status: {}, reason: {})", *read_range.begin, - read_range.end ? *read_range.end : '-')); + read_range.end ? toString(*read_range.end) : "-", + toString(response.getStatus()), response.getReason())); + } + + /// Retry 200OK + if (response.getStatus() == Poco::Net::HTTPResponse::HTTPStatus::HTTP_OK) + initialization_error = InitializeError::RETRIABLE_ERROR; + else + initialization_error = InitializeError::NON_RETRIABLE_ERROR; - initialization_error = InitializeError::NON_RETRIABLE_ERROR; return; } else if (read_range.end) @@ -481,6 +504,15 @@ namespace detail bool result = false; size_t milliseconds_to_wait = settings.http_retry_initial_backoff_ms; + auto on_retriable_error = [&]() + { + retry_with_range_header = true; + impl.reset(); + auto http_session = session->getSession(); + http_session->reset(); + sleepForMilliseconds(milliseconds_to_wait); + }; + for (size_t i = 0; i < settings.http_max_tries; ++i) { try @@ -488,14 +520,35 @@ namespace detail if (!impl) { initialize(); - if (initialization_error == InitializeError::NON_RETRIABLE_ERROR) + switch (initialization_error) { - assert(exception); - break; - } - else if (initialization_error == InitializeError::SKIP_NOT_FOUND_URL) - { - return false; + case InitializeError::NON_RETRIABLE_ERROR: + { + assert(exception); + break; + } + case InitializeError::SKIP_NOT_FOUND_URL: + { + return false; + } + case InitializeError::RETRIABLE_ERROR: + { + LOG_ERROR( + log, + "HTTP request to `{}` failed at try {}/{} with bytes read: {}/{}. " + "(Current backoff wait is {}/{} ms)", + uri.toString(), i + 1, settings.http_max_tries, getOffset(), + read_range.end ? toString(*read_range.end) : "unknown", + milliseconds_to_wait, settings.http_retry_max_backoff_ms); + + assert(exception); + on_retriable_error(); + continue; + } + case InitializeError::NONE: + { + break; + } } if (use_external_buffer) @@ -531,12 +584,8 @@ namespace detail milliseconds_to_wait, settings.http_retry_max_backoff_ms); - retry_with_range_header = true; + on_retriable_error(); exception = std::current_exception(); - impl.reset(); - auto http_session = session->getSession(); - http_session->reset(); - sleepForMilliseconds(milliseconds_to_wait); } milliseconds_to_wait = std::min(milliseconds_to_wait * 2, settings.http_retry_max_backoff_ms); From 79e1d39211a5242d2f59d37ced4be55812c325ec Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 14 Jul 2022 18:45:37 +0000 Subject: [PATCH 217/659] do not color logs on client if redirected to file --- src/Client/ClientBase.cpp | 6 +++- src/Client/ClientBase.h | 1 + .../02360_send_logs_level_colors.reference | 3 ++ .../02360_send_logs_level_colors.sh | 31 +++++++++++++++++++ 4 files changed, 40 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02360_send_logs_level_colors.reference create mode 100644 tests/queries/0_stateless/02360_send_logs_level_colors.sh diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 0e243f97aaf..b435b483b71 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -601,6 +601,7 @@ void ClientBase::initLogsOutputStream() { WriteBuffer * wb = out_logs_buf.get(); + bool color_logs = false; if (!out_logs_buf) { if (server_logs_file.empty()) @@ -608,11 +609,13 @@ void ClientBase::initLogsOutputStream() /// Use stderr by default out_logs_buf = std::make_unique(STDERR_FILENO); wb = out_logs_buf.get(); + color_logs = stderr_is_a_tty; } else if (server_logs_file == "-") { /// Use stdout if --server_logs_file=- specified wb = &std_out; + color_logs = stdout_is_a_tty; } else { @@ -622,7 +625,7 @@ void ClientBase::initLogsOutputStream() } } - logs_out_stream = std::make_unique(*wb, stdout_is_a_tty); + logs_out_stream = std::make_unique(*wb, color_logs); } } @@ -2155,6 +2158,7 @@ void ClientBase::init(int argc, char ** argv) stdin_is_a_tty = isatty(STDIN_FILENO); stdout_is_a_tty = isatty(STDOUT_FILENO); + stderr_is_a_tty = isatty(STDERR_FILENO); terminal_width = getTerminalWidth(); Arguments common_arguments{""}; /// 0th argument is ignored. diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index ec2267a3be6..b012680fc3c 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -173,6 +173,7 @@ protected: bool stdin_is_a_tty = false; /// stdin is a terminal. bool stdout_is_a_tty = false; /// stdout is a terminal. + bool stderr_is_a_tty = false; /// stderr is a terminal. uint64_t terminal_width = 0; ServerConnectionPtr connection; diff --git a/tests/queries/0_stateless/02360_send_logs_level_colors.reference b/tests/queries/0_stateless/02360_send_logs_level_colors.reference new file mode 100644 index 00000000000..fe2824243c4 --- /dev/null +++ b/tests/queries/0_stateless/02360_send_logs_level_colors.reference @@ -0,0 +1,3 @@ +ASCII text +ASCII text +ASCII text diff --git a/tests/queries/0_stateless/02360_send_logs_level_colors.sh b/tests/queries/0_stateless/02360_send_logs_level_colors.sh new file mode 100644 index 00000000000..4e5ce057702 --- /dev/null +++ b/tests/queries/0_stateless/02360_send_logs_level_colors.sh @@ -0,0 +1,31 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +file_name="${CLICKHOUSE_TMP}/res_${CLICKHOUSE_DATABASE}.log" +CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') + +# Run query via expect to make isatty() return true +function run() +{ + command=$1 + expect << EOF +log_user 0 +set timeout 3 +match_max 100000 + +spawn bash -c "$command" +expect 1 +EOF + + file "$file_name" | grep -o "ASCII text" + file "$file_name" | grep -o "with escape sequences" +} + +run "$CLICKHOUSE_CLIENT -q 'SELECT 1' 2>$file_name" +run "$CLICKHOUSE_CLIENT -q 'SELECT 1' --server_logs_file=$file_name" +run "$CLICKHOUSE_CLIENT -q 'SELECT 1' --server_logs_file=- >$file_name" + +rm -f "$file_name" From 1bc943264c24f9ec92fa95ff8c787d6377362003 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Jul 2022 21:38:17 +0200 Subject: [PATCH 218/659] Add concurrent backups test --- .../configs/storage_conf.xml | 35 ++++++++++++++ .../test_concurrent_backups_s3/test.py | 48 +++++++++++++++++++ 2 files changed, 83 insertions(+) create mode 100644 tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml create mode 100644 tests/integration/test_concurrent_backups_s3/test.py diff --git a/tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml b/tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml new file mode 100644 index 00000000000..ef55f3a62d7 --- /dev/null +++ b/tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml @@ -0,0 +1,35 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + 33554432 + + + local + / + + + + + +
+ s3 +
+
+
+
+
+ + + 0 + + + hdd + /backups/ + + +
diff --git a/tests/integration/test_concurrent_backups_s3/test.py b/tests/integration/test_concurrent_backups_s3/test.py new file mode 100644 index 00000000000..591715aff17 --- /dev/null +++ b/tests/integration/test_concurrent_backups_s3/test.py @@ -0,0 +1,48 @@ +#!/usr/bin/env python3 +import pytest +import re +import os.path +from multiprocessing.dummy import Pool +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry, TSV +import time + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=["configs/storage_conf.xml"], + with_minio=True, +) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_concurrent_backups(start_cluster): + node.query("DROP TABLE IF EXISTS s3_test NO DELAY") + columns = [f"column_{i} UInt64" for i in range(1000)] + columns_str = ', '.join(columns) + node.query( + f"CREATE TABLE s3_test ({columns_str}) Engine=MergeTree() ORDER BY tuple() SETTINGS storage_policy='s3';" + ) + node.query( + f"INSERT INTO s3_test SELECT * FROM generateRandom('{columns_str}') LIMIT 10000" + ) + + def create_backup(i): + backup_name = f"Disk('hdd', '/backups/{i}')" + node.query(f"BACKUP TABLE s3_test TO {backup_name} ASYNC") + + p = Pool(20) + + p.map(create_backup, range(40)) + + for _ in range(20): + print(node.query("SELECT * FROM system.backups FORMAT Vertical")) + time.sleep(0.1) + + assert node.query("SELECT count() FROM s3_test where not ignore(*)") == "10000\n" From 435d98738d4edeb693faf5d54a2cd20f32012eb9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 14 Jul 2022 22:41:01 +0300 Subject: [PATCH 219/659] Update ReplicatedMergeTreeQueue.cpp --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index f6c80baba05..7967726edca 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1102,9 +1102,8 @@ bool ReplicatedMergeTreeQueue::isCoveredByFuturePartsImpl(const LogEntry & entry if (future_part.isDisjoint(result_part)) continue; - /// Parts are not disjoint, so new_part_name either contains or covers future_part. - if (!(future_part.contains(result_part) || result_part.contains(future_part))) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got unexpected non-disjoint parts: {} and {}", future_part_elem.first, new_part_name); + /// Parts are not disjoint. They can be even intersecting and it's not a problem, + /// because we may have two queue entries producing intersecting parts if there's DROP_RANGE between them (so virtual_parts are ok). /// We cannot execute `entry` (or upgrade its actual_part_name to `new_part_name`) /// while any covered or covering parts are processed. From 1dc2187f9cfbfbdb7aca0c015b03ea4150796ade Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 14 Jul 2022 22:14:46 +0200 Subject: [PATCH 220/659] Addressed review comments Implementation: - Added a new buffer ForkWriteBuffer takes a vector of WriteBuffer and writes data to all of them. It uses the buffer of the first element as its buffer and copies data from first buffer to all the other buffers Testing: - Updated tests/queries/0_stateless/02346_into_outfile_and_stdout.sh Documentation: - Updated the english documentation for SELECT.. INTO OUTFILE with AND STDOUT. --- .../statements/select/into-outfile.md | 2 +- src/Client/ClientBase.cpp | 12 ++- src/IO/ForkWriteBuffer.cpp | 83 +++++++++++++++++++ src/IO/ForkWriteBuffer.h | 35 ++++++++ src/IO/TeeWriteBuffer.cpp | 46 ---------- src/IO/TeeWriteBuffer.h | 34 -------- src/Parsers/ASTQueryWithOutput.h | 2 +- src/Parsers/ParserQueryWithOutput.cpp | 12 +-- .../02346_into_outfile_and_stdout.reference | 13 ++- .../02346_into_outfile_and_stdout.sh | 54 +++++++++--- 10 files changed, 186 insertions(+), 107 deletions(-) create mode 100644 src/IO/ForkWriteBuffer.cpp create mode 100644 src/IO/ForkWriteBuffer.h delete mode 100644 src/IO/TeeWriteBuffer.cpp delete mode 100644 src/IO/TeeWriteBuffer.h diff --git a/docs/en/sql-reference/statements/select/into-outfile.md b/docs/en/sql-reference/statements/select/into-outfile.md index eb0cf68eca1..6e33673a3c0 100644 --- a/docs/en/sql-reference/statements/select/into-outfile.md +++ b/docs/en/sql-reference/statements/select/into-outfile.md @@ -23,7 +23,7 @@ SELECT INTO OUTFILE file_name [AND STDOUT] [COMPRESSION type [LEVEL - This functionality is available in the [command-line client](../../../interfaces/cli.md) and [clickhouse-local](../../../operations/utilities/clickhouse-local.md). Thus a query sent via [HTTP interface](../../../interfaces/http.md) will fail. - The query will fail if a file with the same file name already exists. - The default [output format](../../../interfaces/formats.md) is `TabSeparated` (like in the command-line client batch mode). Use [FORMAT](format.md) clause to change it. -- If `AND STDOUT` is mentioned in the query then the output that is written to the file is also displayed on standard output. +- If `AND STDOUT` is mentioned in the query then the output that is written to the file is also displayed on standard output. If used with compression, the plaintext is displayed on standard output. **Example** diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index d97f01617a6..cbb5ec9f004 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -69,7 +69,7 @@ #include #include #include -#include +#include namespace fs = std::filesystem; @@ -548,14 +548,18 @@ try range.first, range.second); } - if (query_with_output->is_stdout_enabled) + + if (query_with_output->is_into_outfile_with_stdout) { select_into_file_and_stdout = true; - out_file_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), + WriteBufferPtr file_buf = wrapWriteBufferWithCompressionMethod( + std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), compression_method, compression_level ); + + out_file_buf = std::make_unique(std::vector{file_buf, + std::make_shared(STDOUT_FILENO)}); } else { diff --git a/src/IO/ForkWriteBuffer.cpp b/src/IO/ForkWriteBuffer.cpp new file mode 100644 index 00000000000..ef39c9241db --- /dev/null +++ b/src/IO/ForkWriteBuffer.cpp @@ -0,0 +1,83 @@ +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int CANNOT_CREATE_IO_BUFFER; +} + +ForkWriteBuffer::ForkWriteBuffer(WriteBufferPtrs && sources_) + : WriteBuffer(nullptr, 0), sources(std::move(sources_)) +{ + if (sources.empty()) + { + first_buffer = nullptr; + throw Exception("ForkWriteBuffer required WriteBuffer is not provided", ErrorCodes::CANNOT_CREATE_IO_BUFFER); + } + else + { + first_buffer = sources.begin()->get(); + set(first_buffer->buffer().begin(), first_buffer->buffer().size()); + } +} + + +void ForkWriteBuffer::nextImpl() +{ + if (!first_buffer) + return; + + first_buffer->position() = position(); + + try + { + for (auto write_buffer :sources | std::views::reverse) + { + if (write_buffer.get() != first_buffer) + { + //if buffer size if not enough to write, then split the message with buffer length + if (write_buffer->available() < first_buffer->offset()) + { + size_t bytes_written = 0; + auto to_be_written = first_buffer->offset(); + while (to_be_written != 0) + { + int bytes_to_copy = std::min(to_be_written, write_buffer->available()); + write_buffer->write(first_buffer->buffer().begin()+bytes_written, bytes_to_copy); + write_buffer->next(); + bytes_written += bytes_to_copy; + to_be_written -= bytes_to_copy; + } + } + else + write_buffer->write(first_buffer->buffer().begin(), first_buffer->offset()); + } + write_buffer->next(); + } + } + catch (Exception & exception) + { + exception.addMessage("While writing to ForkWriteBuffer"); + throw; + } + +} + +void ForkWriteBuffer::finalizeImpl() +{ + next(); +} + + +ForkWriteBuffer::~ForkWriteBuffer() +{ + finalize(); +} + + +} diff --git a/src/IO/ForkWriteBuffer.h b/src/IO/ForkWriteBuffer.h new file mode 100644 index 00000000000..63267fcd8d7 --- /dev/null +++ b/src/IO/ForkWriteBuffer.h @@ -0,0 +1,35 @@ +#pragma once +#include + + +namespace DB +{ + +namespace ErrorCodes +{ +} + +/** ForkWriteBuffer takes a vector of WriteBuffer and writes data to all of them + * If the vector of WriteBufferPts is empty, then it throws an error + * It uses the buffer of the first element as its buffer and copies data from + * first buffer to all the other buffers + **/ +class ForkWriteBuffer : public WriteBuffer +{ +public: + + using WriteBufferPtrs = std::vector; + + explicit ForkWriteBuffer(WriteBufferPtrs && sources_); + ~ForkWriteBuffer() override; + +protected: + void nextImpl() override; + void finalizeImpl() override; + +private: + WriteBufferPtrs sources; + WriteBuffer *first_buffer; +}; + +} diff --git a/src/IO/TeeWriteBuffer.cpp b/src/IO/TeeWriteBuffer.cpp deleted file mode 100644 index 5131fbdeacf..00000000000 --- a/src/IO/TeeWriteBuffer.cpp +++ /dev/null @@ -1,46 +0,0 @@ -#include - -namespace DB -{ - -TeeWriteBuffer::TeeWriteBuffer( - const std::string & file_name_, - size_t buf_size, - int flags, - mode_t mode, - char * existing_memory, - size_t alignment) - : WriteBufferFromFile(file_name_,buf_size,flags,mode,existing_memory,alignment), - stdout_buffer(STDOUT_FILENO,buf_size,working_buffer.begin()) -{ -} - -void TeeWriteBuffer::nextImpl() -{ - try - { - stdout_buffer.position() = position(); - stdout_buffer.next(); - WriteBufferFromFile::nextImpl(); - } - catch (Exception &exception) - { - exception.addMessage("While writing to TeeWriteBuffer "); - throw; - } -} - -void TeeWriteBuffer::finalizeImpl() -{ - if (fd < 0 || stdout_buffer.getFD() < 0) - return; - - next(); -} - -TeeWriteBuffer::~TeeWriteBuffer() -{ - finalize(); -} - -} diff --git a/src/IO/TeeWriteBuffer.h b/src/IO/TeeWriteBuffer.h deleted file mode 100644 index 264ba42b0a8..00000000000 --- a/src/IO/TeeWriteBuffer.h +++ /dev/null @@ -1,34 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -/** TeeWriteBuffer extends from WriteBufferFromFile and has - * WriteBufferFromFileDescriptor inside the class which is created - * by using the same buffer as TeeWriteBuffer. So both the data are written - * using same buffer - **/ -class TeeWriteBuffer : public WriteBufferFromFile -{ - -public: - explicit TeeWriteBuffer( - const std::string & file_name_, - size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, - int flags = -1, - mode_t mode = 0666, - char * existing_memory = nullptr, - size_t alignment = 0); - - ~TeeWriteBuffer() override; - -protected: - void nextImpl() override; - void finalizeImpl() override; - - WriteBufferFromFileDescriptor stdout_buffer; -}; - -} diff --git a/src/Parsers/ASTQueryWithOutput.h b/src/Parsers/ASTQueryWithOutput.h index d2a24f1ebe2..892d911e2e2 100644 --- a/src/Parsers/ASTQueryWithOutput.h +++ b/src/Parsers/ASTQueryWithOutput.h @@ -15,7 +15,7 @@ class ASTQueryWithOutput : public IAST { public: ASTPtr out_file; - bool is_stdout_enabled; + bool is_into_outfile_with_stdout; ASTPtr format; ASTPtr settings_ast; ASTPtr compression; diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 0f900ed8856..6107bd2a5eb 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -90,6 +90,12 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (!out_file_p.parse(pos, query_with_output.out_file, expected)) return false; + ParserKeyword s_stdout("AND STDOUT"); + if (s_stdout.ignore(pos, expected)) + { + query_with_output.is_into_outfile_with_stdout = true; + } + ParserKeyword s_compression_method("COMPRESSION"); if (s_compression_method.ignore(pos, expected)) { @@ -108,12 +114,6 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query_with_output.children.push_back(query_with_output.out_file); - ParserKeyword s_stdout("AND STDOUT"); - if (s_stdout.ignore(pos, expected)) - { - query_with_output.is_stdout_enabled = true; - } - } ParserKeyword s_format("FORMAT"); diff --git a/tests/queries/0_stateless/02346_into_outfile_and_stdout.reference b/tests/queries/0_stateless/02346_into_outfile_and_stdout.reference index 09984d86ca3..d14e7634f24 100644 --- a/tests/queries/0_stateless/02346_into_outfile_and_stdout.reference +++ b/tests/queries/0_stateless/02346_into_outfile_and_stdout.reference @@ -2,8 +2,15 @@ performing test: select 1 2 3 1 2 3 performing test: bad_query_incorrect_usage -query failed +1 performing test: bad_query_no_into_outfile -query failed +1 performing test: bad_query_file_exists -query failed +1 +performing test: compression +Hello, World! From clickhouse. +Hello, World! From clickhouse. +performing test: bad_query_misplaced_compression +1 +performing test: bad_query_misplaced_format +1 diff --git a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh index 76b308d003c..3879249699f 100755 --- a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh +++ b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh @@ -9,37 +9,67 @@ function perform() local test_id=$1 local query=$2 - echo "performing test: $test_id" - ${CLICKHOUSE_CLIENT} --query "$query" 2>/dev/null + echo "performing test: ${test_id}" + ${CLICKHOUSE_CLIENT} --query "${query}" if [ "$?" -eq 0 ]; then - cat "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_$test_id.out" + cat "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_${test_id}.out" else echo "query failed" fi - rm -f "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_$test_id.out" + rm -f "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_${test_id}.out" +} + +function performBadQuery() +{ + local test_id=$1 + local query=$2 + local error_message=$3 + + echo "performing test: ${test_id}" + ${CLICKHOUSE_CLIENT} --query "${query}" 2>&1 | grep -Fc "${error_message}" } function performFileExists() { local test_id=$1 local query=$2 + local error_message=$3 - touch "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_$test_id.out" + touch "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_${test_id}.out" - echo "performing test: $test_id" - ${CLICKHOUSE_CLIENT} --query "$query" 2>/dev/null + echo "performing test: ${test_id}" + ${CLICKHOUSE_CLIENT} --query "${query}" 2>&1 | grep -Fc "${error_message}" + rm -f "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_${test_id}.out" +} + +function performCompression() +{ + local test_id=$1 + local query=$2 + + echo "performing test: ${test_id}" + ${CLICKHOUSE_CLIENT} --query "${query}" if [ "$?" -eq 0 ]; then - cat "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_$test_id.out" + gunzip "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_${test_id}.gz" + cat "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_${test_id}" else echo "query failed" fi - rm -f "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_$test_id.out" + rm -f "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_${test_id}" } + + perform "select" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_select.out' AND STDOUT" -perform "bad_query_incorrect_usage" "SELECT 1, 2, 3 INTO OUTFILE AND STDOUT'${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_incorrect_usage.out'" +performBadQuery "bad_query_incorrect_usage" "SELECT 1, 2, 3 INTO OUTFILE AND STDOUT'${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_incorrect_usage.out'" "SYNTAX_ERROR" -perform "bad_query_no_into_outfile" "SELECT 1, 2, 3 AND STDOUT'" +performBadQuery "bad_query_no_into_outfile" "SELECT 1, 2, 3 AND STDOUT'" "SYNTAX_ERROR" -performFileExists "bad_query_file_exists" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out' AND STDOUT" \ No newline at end of file +performFileExists "bad_query_file_exists" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out' AND STDOUT" "File exists. (CANNOT_OPEN_FILE)" + +performCompression "compression" "SELECT * FROM (SELECT 'Hello, World! From clickhouse.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_compression.gz' AND STDOUT COMPRESSION 'GZ' FORMAT TabSeparated" + +performBadQuery "bad_query_misplaced_compression" "SELECT 1, 2, 3 INTO OUTFILE 'test.gz' COMPRESSION 'GZ' AND STDOUT'" "SYNTAX_ERROR" + +performBadQuery "bad_query_misplaced_format" "SELECT 1, 2, 3 INTO OUTFILE 'test.gz' FORMAT TabSeparated AND STDOUT'" "SYNTAX_ERROR" \ No newline at end of file From efed13475819f570add3e82e1c0fa40d17a8b975 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 14 Jul 2022 19:41:32 +0000 Subject: [PATCH 221/659] fix test --- docker/test/fasttest/Dockerfile | 1 + docker/test/stateless/Dockerfile | 1 + tests/queries/0_stateless/02360_send_logs_level_colors.sh | 0 3 files changed, 2 insertions(+) mode change 100644 => 100755 tests/queries/0_stateless/02360_send_logs_level_colors.sh diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 03a79b45a10..aa209f197df 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -71,6 +71,7 @@ RUN apt-get update \ software-properties-common \ tzdata \ unixodbc \ + file \ --yes --no-install-recommends RUN pip3 install numpy scipy pandas Jinja2 diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 96c06845812..9141e89d744 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -34,6 +34,7 @@ RUN apt-get update -y \ unixodbc \ wget \ zstd \ + file \ && apt-get clean diff --git a/tests/queries/0_stateless/02360_send_logs_level_colors.sh b/tests/queries/0_stateless/02360_send_logs_level_colors.sh old mode 100644 new mode 100755 From d2b725761717ccd09b243ed3a42ffb35de398fa7 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Fri, 15 Jul 2022 00:12:38 +0200 Subject: [PATCH 222/659] Update tests/queries/0_stateless/02356_insert_query_log_metrics.sql Co-authored-by: Alexander Tokmakov --- tests/queries/0_stateless/02356_insert_query_log_metrics.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql index 347073f868f..08ae22dee95 100644 --- a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql +++ b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql @@ -1,4 +1,5 @@ CREATE TABLE 02356_destination (a Int64, b String) ENGINE = Memory; INSERT INTO 02356_destination (a, b) SELECT * FROM generateRandom('a Int64, b String') LIMIT 100 SETTINGS max_threads=1, max_block_size=100; +SYSTEM FLUSH LOGS; SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' order by event_time desc limit 1 FORMAT CSV; \ No newline at end of file From 4d96ff9b9768a6009d339359635493cb8e222354 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 14 Jul 2022 22:25:10 +0000 Subject: [PATCH 223/659] style --- tests/queries/0_stateless/02356_insert_query_log_metrics.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql index 08ae22dee95..bb37c8b4eb1 100644 --- a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql +++ b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql @@ -2,4 +2,4 @@ CREATE TABLE 02356_destination (a Int64, b String) ENGINE = Memory; INSERT INTO 02356_destination (a, b) SELECT * FROM generateRandom('a Int64, b String') LIMIT 100 SETTINGS max_threads=1, max_block_size=100; SYSTEM FLUSH LOGS; -SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' order by event_time desc limit 1 FORMAT CSV; \ No newline at end of file +SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' and has(databases, currentDatabase()) FORMAT CSV; \ No newline at end of file From c7da3d61bf7be2867ce45679d91852846b501b3d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 00:32:20 +0200 Subject: [PATCH 224/659] More stressful test --- tests/integration/test_concurrent_backups_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_concurrent_backups_s3/test.py b/tests/integration/test_concurrent_backups_s3/test.py index 591715aff17..ed8edbc77d4 100644 --- a/tests/integration/test_concurrent_backups_s3/test.py +++ b/tests/integration/test_concurrent_backups_s3/test.py @@ -37,7 +37,7 @@ def test_concurrent_backups(start_cluster): backup_name = f"Disk('hdd', '/backups/{i}')" node.query(f"BACKUP TABLE s3_test TO {backup_name} ASYNC") - p = Pool(20) + p = Pool(40) p.map(create_backup, range(40)) From 687961cf4242fc28bf0957054716441f4701af70 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 00:38:58 +0200 Subject: [PATCH 225/659] Fix style --- src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp | 1 - .../MetadataStorageFromDiskTransactionOperations.cpp | 1 - tests/integration/test_concurrent_backups_s3/test.py | 4 +++- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp index 76e6aebff1c..d1413bd4e88 100644 --- a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp @@ -11,7 +11,6 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; - extern const int FS_METADATA_ERROR; } FakeMetadataStorageFromDisk::FakeMetadataStorageFromDisk( diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp index 7c0a60509ab..72da240cf8a 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include diff --git a/tests/integration/test_concurrent_backups_s3/test.py b/tests/integration/test_concurrent_backups_s3/test.py index ed8edbc77d4..b1f41f04b99 100644 --- a/tests/integration/test_concurrent_backups_s3/test.py +++ b/tests/integration/test_concurrent_backups_s3/test.py @@ -14,6 +14,7 @@ node = cluster.add_instance( with_minio=True, ) + @pytest.fixture(scope="module") def start_cluster(): try: @@ -22,10 +23,11 @@ def start_cluster(): finally: cluster.shutdown() + def test_concurrent_backups(start_cluster): node.query("DROP TABLE IF EXISTS s3_test NO DELAY") columns = [f"column_{i} UInt64" for i in range(1000)] - columns_str = ', '.join(columns) + columns_str = ", ".join(columns) node.query( f"CREATE TABLE s3_test ({columns_str}) Engine=MergeTree() ORDER BY tuple() SETTINGS storage_policy='s3';" ) From d4a24a5f8ef601598de84ecc9d66b049cf84b44c Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 00:50:03 +0200 Subject: [PATCH 226/659] Better test --- tests/integration/test_concurrent_backups_s3/test.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_concurrent_backups_s3/test.py b/tests/integration/test_concurrent_backups_s3/test.py index b1f41f04b99..c262bc9e331 100644 --- a/tests/integration/test_concurrent_backups_s3/test.py +++ b/tests/integration/test_concurrent_backups_s3/test.py @@ -43,8 +43,13 @@ def test_concurrent_backups(start_cluster): p.map(create_backup, range(40)) - for _ in range(20): - print(node.query("SELECT * FROM system.backups FORMAT Vertical")) + for _ in range(100): + result = node.query( + "SELECT count() FROM system.backups WHERE status != 'BACKUP_COMPLETE' and status != 'FAILED_TO_BACKUP'" + ).strip() + if result == "0": + break + time.sleep(0.1) assert node.query("SELECT count() FROM s3_test where not ignore(*)") == "10000\n" From f3650d8769e9d4daf354df46ece226328fa289ea Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Thu, 14 Jul 2022 18:42:41 -0700 Subject: [PATCH 227/659] Add back accidentally removed line --- contrib/thrift-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index 832e104e8a3..d6aa6b9e5f2 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -93,4 +93,4 @@ include_directories("${CMAKE_CURRENT_BINARY_DIR}") add_library(_thrift ${thriftcpp_SOURCES} ${thriftcpp_threads_SOURCES}) add_library(ch_contrib::thrift ALIAS _thrift) target_include_directories(_thrift SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src" ${CMAKE_CURRENT_BINARY_DIR}) -target_link_libraries (_thrift PUBLIC OpenSSL::SSL) +target_link_libraries (_thrift PUBLIC OpenSSL::SSL boost::headers_only) From a46188216f48d894995f3a0a1d6f68e93c82dca6 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Tue, 7 Jun 2022 13:20:46 +0800 Subject: [PATCH 228/659] Support delete from ... where syntax on mergetree tables --- src/Interpreters/InterpreterDeleteQuery.cpp | 89 +++++++++++++++++++ src/Interpreters/InterpreterDeleteQuery.h | 24 +++++ src/Interpreters/InterpreterFactory.cpp | 6 ++ src/Parsers/ASTDeleteQuery.cpp | 42 +++++++++ src/Parsers/ASTDeleteQuery.h | 21 +++++ src/Parsers/ParserDeleteQuery.cpp | 49 ++++++++++ src/Parsers/ParserDeleteQuery.h | 19 ++++ src/Parsers/ParserQuery.cpp | 3 + ...19_standard_delete_on_merge_tree.reference | 3 + .../02319_standard_delete_on_merge_tree.sql | 21 +++++ 10 files changed, 277 insertions(+) create mode 100644 src/Interpreters/InterpreterDeleteQuery.cpp create mode 100644 src/Interpreters/InterpreterDeleteQuery.h create mode 100644 src/Parsers/ASTDeleteQuery.cpp create mode 100644 src/Parsers/ASTDeleteQuery.h create mode 100644 src/Parsers/ParserDeleteQuery.cpp create mode 100644 src/Parsers/ParserDeleteQuery.h create mode 100644 tests/queries/0_stateless/02319_standard_delete_on_merge_tree.reference create mode 100644 tests/queries/0_stateless/02319_standard_delete_on_merge_tree.sql diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp new file mode 100644 index 00000000000..dff286fc92a --- /dev/null +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -0,0 +1,89 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int TABLE_IS_READ_ONLY; +} + + +InterpreterDeleteQuery::InterpreterDeleteQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) +{ +} + + +BlockIO InterpreterDeleteQuery::execute() +{ + FunctionNameNormalizer().visit(query_ptr.get()); + const ASTDeleteQuery & delete_query = query_ptr->as(); + auto table_id = getContext()->resolveStorageID(delete_query, Context::ResolveOrdinary); + + getContext()->checkAccess(AccessType::ALTER_DELETE, table_id); + + query_ptr->as().setDatabase(table_id.database_name); + + /// First check table storage for validations. + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); + auto storage_merge_tree = std::dynamic_pointer_cast(table); + if (!storage_merge_tree) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only MergeTree tables are supported"); + + checkStorageSupportsTransactionsIfNeeded(table, getContext()); + if (table->isStaticStorage()) + throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only"); + + DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name); + if (typeid_cast(database.get()) + && !getContext()->getClientInfo().is_replicated_database_internal) + { + auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name); + guard->releaseTableLock(); + return typeid_cast(database.get())->tryEnqueueReplicatedDDL(query_ptr, getContext()); + } + + auto table_lock = table->lockForShare(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); + auto metadata_snapshot = table->getInMemoryMetadataPtr(); + + /// Currently do similar as alter table delete. + /// TODO: Mark this delete as lightweight. + MutationCommands mutation_commands; + MutationCommand mut_command; + + mut_command.type = MutationCommand::Type::DELETE; + mut_command.predicate = delete_query.predicate; + + auto command = std::make_shared(); + command->type = ASTAlterCommand::DELETE; + command->predicate = delete_query.predicate; + command->children.push_back(command->predicate); + mut_command.ast = command->ptr(); + + mutation_commands.emplace_back(mut_command); + + if (!mutation_commands.empty()) + { + table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); + MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate(); + table->mutate(mutation_commands, getContext()); + } + + return {}; +} + +} diff --git a/src/Interpreters/InterpreterDeleteQuery.h b/src/Interpreters/InterpreterDeleteQuery.h new file mode 100644 index 00000000000..7f17aa31e37 --- /dev/null +++ b/src/Interpreters/InterpreterDeleteQuery.h @@ -0,0 +1,24 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/** Allows you do lightweight deletion on a MergeTree family table. + */ +class InterpreterDeleteQuery : public IInterpreter, WithContext +{ +public: + InterpreterDeleteQuery(const ASTPtr & query_ptr_, ContextPtr context_); + + BlockIO execute() override; + + bool supportsTransactions() const override { return true; } + +private: + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 6b081467ae7..00183086bf6 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -46,6 +47,7 @@ #include #include #include +#include #include #include #include @@ -314,6 +316,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else { throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY); diff --git a/src/Parsers/ASTDeleteQuery.cpp b/src/Parsers/ASTDeleteQuery.cpp new file mode 100644 index 00000000000..ee7dba8dbf7 --- /dev/null +++ b/src/Parsers/ASTDeleteQuery.cpp @@ -0,0 +1,42 @@ +#include +#include + +namespace DB +{ + +String ASTDeleteQuery::getID(char delim) const +{ + return "DeleteQuery" + (delim + getDatabase()) + delim + getTable(); +} + +ASTPtr ASTDeleteQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (predicate) + { + res->predicate = predicate->clone(); + res->children.push_back(res->predicate); + } + + cloneTableOptions(*res); + return res; +} + +void ASTDeleteQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "DELETE FROM " << (settings.hilite ? hilite_none : ""); + + if (database) + { + settings.ostr << backQuoteIfNeed(getDatabase()); + settings.ostr << "."; + } + settings.ostr << backQuoteIfNeed(getTable()); + + settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : ""); + predicate->formatImpl(settings, state, frame); +} + +} diff --git a/src/Parsers/ASTDeleteQuery.h b/src/Parsers/ASTDeleteQuery.h new file mode 100644 index 00000000000..bcb97639b64 --- /dev/null +++ b/src/Parsers/ASTDeleteQuery.h @@ -0,0 +1,21 @@ +#pragma once + +#include +#include + +namespace DB +{ +/// DELETE FROM [db.]name WHERE ... +class ASTDeleteQuery : public ASTQueryWithTableAndOutput +{ +public: + String getID(char delim) const final; + ASTPtr clone() const final; + + ASTPtr predicate; + +protected: + void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + +} diff --git a/src/Parsers/ParserDeleteQuery.cpp b/src/Parsers/ParserDeleteQuery.cpp new file mode 100644 index 00000000000..bd06c060ac4 --- /dev/null +++ b/src/Parsers/ParserDeleteQuery.cpp @@ -0,0 +1,49 @@ +#include +#include +#include +#include + + +namespace DB +{ + +bool ParserDeleteQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + auto query = std::make_shared(); + node = query; + + ParserKeyword s_delete("DELETE"); + ParserKeyword s_from("FROM"); + ParserKeyword s_where("WHERE"); + ParserExpression parser_exp_elem; + + if (s_delete.ignore(pos, expected)) + { + if (!s_from.ignore(pos, expected)) + return false; + + if (!parseDatabaseAndTableAsAST(pos, expected, query->database, query->table)) + return false; + + if (!s_where.ignore(pos, expected)) + return false; + + if (!parser_exp_elem.parse(pos, query->predicate, expected)) + return false; + } + else + return false; + + if (query->predicate) + query->children.push_back(query->predicate); + + if (query->database) + query->children.push_back(query->database); + + if (query->table) + query->children.push_back(query->table); + + return true; +} + +} diff --git a/src/Parsers/ParserDeleteQuery.h b/src/Parsers/ParserDeleteQuery.h new file mode 100644 index 00000000000..fb7c644f48b --- /dev/null +++ b/src/Parsers/ParserDeleteQuery.h @@ -0,0 +1,19 @@ +#pragma once + +#include + +namespace DB +{ + +/** Query like this: + * DELETE FROM [db.]name WHERE ... + */ + +class ParserDeleteQuery : public IParserBase +{ +protected: + const char * getName() const override{ return "Delete query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index a3cafee65d7..ca837e7dcc5 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -52,6 +53,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserSetRoleQuery set_role_p; ParserExternalDDLQuery external_ddl_p; ParserTransactionControl transaction_control_p; + ParserDeleteQuery delete_p; ParserBackupQuery backup_p; bool res = query_with_output_p.parse(pos, node, expected) @@ -73,6 +75,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || grant_p.parse(pos, node, expected) || external_ddl_p.parse(pos, node, expected) || transaction_control_p.parse(pos, node, expected) + || delete_p.parse(pos, node, expected) || backup_p.parse(pos, node, expected); return res; diff --git a/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.reference b/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.reference new file mode 100644 index 00000000000..b343623df61 --- /dev/null +++ b/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.reference @@ -0,0 +1,3 @@ +99 +95 +0 diff --git a/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.sql new file mode 100644 index 00000000000..419278df291 --- /dev/null +++ b/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS merge_table_standard_delete; + +CREATE TABLE merge_table_standard_delete(id Int32, name String) ENGINE = MergeTree order by id; + +INSERT INTO merge_table_standard_delete select number, toString(number) from numbers(100); + +SET mutations_sync = 1; + +DELETE FROM merge_table_standard_delete WHERE id = 10; + +SELECT COUNT() FROM merge_table_standard_delete; + +DELETE FROM merge_table_standard_delete WHERE name IN ('1','2','3','4'); + +SELECT COUNT() FROM merge_table_standard_delete; + +DELETE FROM merge_table_standard_delete WHERE 1; + +SELECT COUNT() FROM merge_table_standard_delete; + +DROP TABLE merge_table_standard_delete; \ No newline at end of file From 192ca25e877893e260faab0c00692095c43e03f5 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Tue, 7 Jun 2022 14:32:23 +0800 Subject: [PATCH 229/659] Fix style check errors --- src/Interpreters/InterpreterDeleteQuery.cpp | 4 ++-- .../0_stateless/02319_standard_delete_on_merge_tree.sql | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index dff286fc92a..567b91d7577 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -18,7 +18,7 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; extern const int TABLE_IS_READ_ONLY; } @@ -64,7 +64,7 @@ BlockIO InterpreterDeleteQuery::execute() /// TODO: Mark this delete as lightweight. MutationCommands mutation_commands; MutationCommand mut_command; - + mut_command.type = MutationCommand::Type::DELETE; mut_command.predicate = delete_query.predicate; diff --git a/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.sql index 419278df291..13b3a3e2701 100644 --- a/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.sql +++ b/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.sql @@ -18,4 +18,4 @@ DELETE FROM merge_table_standard_delete WHERE 1; SELECT COUNT() FROM merge_table_standard_delete; -DROP TABLE merge_table_standard_delete; \ No newline at end of file +DROP TABLE merge_table_standard_delete; From 8df7b7a030761f4fb3fe4e6192a31297574af01a Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Wed, 8 Jun 2022 10:31:11 +0800 Subject: [PATCH 230/659] Support new mutation type for lightweight --- src/Interpreters/InterpreterDeleteQuery.cpp | 2 +- .../MergeTree/MergeTreeMutationEntry.cpp | 24 +++++++++++++++---- .../MergeTree/MergeTreeMutationEntry.h | 7 +++++- .../MergeTree/MergeTreeMutationStatus.h | 2 ++ .../MergeTree/ReplicatedMergeTreeQueue.cpp | 1 + src/Storages/StorageMergeTree.cpp | 12 +++++++--- src/Storages/StorageMergeTree.h | 5 +++- .../System/StorageSystemMutations.cpp | 2 ++ 8 files changed, 45 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 567b91d7577..fff66402cff 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -80,7 +80,7 @@ BlockIO InterpreterDeleteQuery::execute() { table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate(); - table->mutate(mutation_commands, getContext()); + storage_merge_tree->mutate(mutation_commands, getContext(), MutationType::Lightweight); } return {}; diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index a222f2a8ad8..5b103cbe8d8 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -46,8 +46,9 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) } MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, - const TransactionID & tid_, const WriteSettings & settings) - : create_time(time(nullptr)) + const TransactionID & tid_, const WriteSettings & settings, MutationType type_) + : type(type_) + , create_time(time(nullptr)) , commands(std::move(commands_)) , disk(std::move(disk_)) , path_prefix(path_prefix_) @@ -58,7 +59,8 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP try { auto out = disk->writeFile(std::filesystem::path(path_prefix) / file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, settings); - *out << "format version: 1\n" + *out << "format version: 2\n" + << "type: " << type << "\n" << "create time: " << LocalDateTime(create_time) << "\n"; *out << "commands: "; commands.writeText(*out); @@ -121,7 +123,21 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat block_number = parseFileName(file_name); auto buf = disk->readFile(path_prefix + file_name); - *buf >> "format version: 1\n"; + int format_version; + *buf >> "format version: " >> format_version >> "\n"; + + assert(format_version <= 2); + + type = MutationType::Ordinary; + if (format_version == 2) + { + String type_str; + *buf >> "type: " >> type_str >> "\n"; + + auto type_value = magic_enum::enum_cast(type_str); + if (type_value.has_value()) + type = type_value.value(); + } LocalDateTime create_time_dt; *buf >> "create time: " >> create_time_dt >> "\n"; diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 04297f2852a..06cbd44ed49 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -11,10 +11,15 @@ namespace DB { class IBackupEntry; +enum class MutationType { Ordinary, Lightweight }; + /// A mutation entry for non-replicated MergeTree storage engines. /// Stores information about mutation in file mutation_*.txt. struct MergeTreeMutationEntry { + /// Type of mutation, used for lightweight delete. + MutationType type; + time_t create_time = 0; MutationCommands commands; @@ -38,7 +43,7 @@ struct MergeTreeMutationEntry /// Create a new entry and write it to a temporary file. MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number, - const TransactionID & tid_, const WriteSettings & settings); + const TransactionID & tid_, const WriteSettings & settings, MutationType type_); MergeTreeMutationEntry(const MergeTreeMutationEntry &) = delete; MergeTreeMutationEntry(MergeTreeMutationEntry &&) = default; diff --git a/src/Storages/MergeTree/MergeTreeMutationStatus.h b/src/Storages/MergeTree/MergeTreeMutationStatus.h index acda43b9254..f0949047f6e 100644 --- a/src/Storages/MergeTree/MergeTreeMutationStatus.h +++ b/src/Storages/MergeTree/MergeTreeMutationStatus.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -13,6 +14,7 @@ namespace DB struct MergeTreeMutationStatus { + MutationType type; String id; String command; time_t create_time = 0; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index f6c80baba05..312c4146cb9 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1920,6 +1920,7 @@ std::vector ReplicatedMergeTreeQueue::getMutationsStatu formatAST(*command.ast, buf, false, true); result.push_back(MergeTreeMutationStatus { + MutationType::Ordinary, entry.znode_name, buf.str(), entry.create_time, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 6825698f006..495218d4ef5 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -429,7 +429,7 @@ CurrentlyMergingPartsTagger::~CurrentlyMergingPartsTagger() storage.currently_processing_in_background_condition.notify_all(); } -Int64 StorageMergeTree::startMutation(const MutationCommands & commands, ContextPtr query_context) +Int64 StorageMergeTree::startMutation(const MutationCommands & commands, ContextPtr query_context, MutationType type) { /// Choose any disk, because when we load mutations we search them at each disk /// where storage can be placed. See loadMutations(). @@ -447,7 +447,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context { std::lock_guard lock(currently_processing_in_background_mutex); - MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings()); + MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings(), type); version = increment.get(); entry.commit(version); String mutation_id = entry.file_name; @@ -554,11 +554,16 @@ void StorageMergeTree::setMutationCSN(const String & mutation_id, CSN csn) } void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr query_context) +{ + mutate(commands, query_context, MutationType::Ordinary); +} + +void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr query_context, MutationType type) { /// Validate partition IDs (if any) before starting mutation getPartitionIdsAffectedByCommands(commands, query_context); - Int64 version = startMutation(commands, query_context); + Int64 version = startMutation(commands, query_context, type); if (query_context->getSettingsRef().mutations_sync > 0 || query_context->getCurrentTransaction()) waitForMutation(version); @@ -652,6 +657,7 @@ std::vector StorageMergeTree::getMutationsStatus() cons formatAST(*command.ast, buf, false, true); result.push_back(MergeTreeMutationStatus { + entry.type, entry.file_name, buf.str(), entry.create_time, diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 8ca8ab5d11e..a27925994c9 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -87,6 +87,9 @@ public: void mutate(const MutationCommands & commands, ContextPtr context) override; + /// Support lightweight delete. + void mutate(const MutationCommands & commands, ContextPtr context, MutationType type); + /// Return introspection information about currently processing or recently processed mutations. std::vector getMutationsStatus() const override; @@ -180,7 +183,7 @@ private: /// Allocate block number for new mutation, write mutation to disk /// and into in-memory structures. Wake up merge-mutation task. - Int64 startMutation(const MutationCommands & commands, ContextPtr query_context); + Int64 startMutation(const MutationCommands & commands, ContextPtr query_context, MutationType type = MutationType::Ordinary); /// Wait until mutation with version will finish mutation for all parts void waitForMutation(Int64 version); void waitForMutation(const String & mutation_id) override; diff --git a/src/Storages/System/StorageSystemMutations.cpp b/src/Storages/System/StorageSystemMutations.cpp index fa521c632b8..907376a4936 100644 --- a/src/Storages/System/StorageSystemMutations.cpp +++ b/src/Storages/System/StorageSystemMutations.cpp @@ -20,6 +20,7 @@ NamesAndTypesList StorageSystemMutations::getNamesAndTypes() return { { "database", std::make_shared() }, { "table", std::make_shared() }, + { "is_lightweight", std::make_shared() }, { "mutation_id", std::make_shared() }, { "command", std::make_shared() }, { "create_time", std::make_shared() }, @@ -130,6 +131,7 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, ContextPtr c res_columns[col_num++]->insert(database); res_columns[col_num++]->insert(table); + res_columns[col_num++]->insert(status.type == MutationType::Lightweight); res_columns[col_num++]->insert(status.id); res_columns[col_num++]->insert(status.command); res_columns[col_num++]->insert(UInt64(status.create_time)); From 8696319d62a83ce393c7c64fb5ae01ebe0e20882 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Mon, 20 Jun 2022 17:18:00 +0800 Subject: [PATCH 231/659] Support lightweight delete execution using string as deleted rows mask,also part of select can handle LWD --- src/Interpreters/MutationsInterpreter.cpp | 73 ++++++- src/Interpreters/MutationsInterpreter.h | 9 +- .../MergeTree/FutureMergedMutatedPart.h | 2 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 29 +++ src/Storages/MergeTree/IMergeTreeDataPart.h | 12 ++ .../MergeTree/MergeTreeRangeReader.cpp | 76 +++++++ src/Storages/MergeTree/MergeTreeRangeReader.h | 7 + src/Storages/MergeTree/MutateTask.cpp | 203 +++++++++++++++++- src/Storages/StorageMergeTree.cpp | 6 + 9 files changed, 411 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 8c1d929e409..8753905521d 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -286,15 +286,20 @@ MutationsInterpreter::MutationsInterpreter( const StorageMetadataPtr & metadata_snapshot_, MutationCommands commands_, ContextPtr context_, - bool can_execute_) + bool can_execute_, + bool is_lightweight_) : storage(std::move(storage_)) , metadata_snapshot(metadata_snapshot_) , commands(std::move(commands_)) , context(Context::createCopy(context_)) , can_execute(can_execute_) , select_limits(SelectQueryOptions().analyze(!can_execute).ignoreLimits().ignoreProjections()) + , is_lightweight(is_lightweight_) { - mutation_ast = prepare(!can_execute); + if (is_lightweight) + mutation_ast = prepareLightweightDelete(!can_execute); + else + mutation_ast = prepare(!can_execute); } static NameSet getKeyColumns(const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot) @@ -890,6 +895,70 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & return select; } +/// Prepare for lightweight delete +ASTPtr MutationsInterpreter::prepareLightweightDelete(bool dry_run) +{ + if (is_prepared) + throw Exception("MutationsInterpreter is already prepared. It is a bug.", ErrorCodes::LOGICAL_ERROR); + + if (commands.empty()) + throw Exception("Empty mutation commands list", ErrorCodes::LOGICAL_ERROR); + + /// For lightweight DELETE, we use predicate expression to get deleted rows. + /// Collect predicates in the commands + for (auto & command : commands) + { + if (command.type == MutationCommand::DELETE) + { + mutation_kind.set(MutationKind::MUTATE_OTHER); + if (stages.empty()) + stages.emplace_back(context); + + auto mask_predicate = getPartitionAndPredicateExpressionForMutationCommand(command); + stages.back().filters.push_back(mask_predicate); + } + else + throw Exception("Unsupported lightweight mutation command type: " + DB::toString(command.type), ErrorCodes::UNKNOWN_MUTATION_COMMAND); + } + + /// The updated_header is empty for lightweight delete. + updated_header = std::make_unique(); + + is_prepared = true; + + return prepareInterpreterSelectQueryLightweight(stages, dry_run); +} + +ASTPtr MutationsInterpreter::prepareInterpreterSelectQueryLightweight(std::vector & prepared_stages, bool) +{ + /// Construct a SELECT statement for lightweight delete is like "select _part_offset from db.table where " + auto select = std::make_shared(); + + /// DELETEs only query just need the _part_offset virtual column without real columns + select->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); + select->select()->children.push_back(std::make_shared("_part_offset")); + + ASTPtr where_expression; + if (!prepared_stages[0].filters.empty()) + { + if (prepared_stages[0].filters.size() == 1) + where_expression = prepared_stages[0].filters[0]; + else + { + auto coalesced_predicates = std::make_shared(); + coalesced_predicates->name = "or"; + coalesced_predicates->arguments = std::make_shared(); + coalesced_predicates->children.push_back(coalesced_predicates->arguments); + coalesced_predicates->arguments->children = prepared_stages[0].filters; + where_expression = std::move(coalesced_predicates); + } + + select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where_expression)); + } + + return select; +} + QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const { for (size_t i_stage = 1; i_stage < prepared_stages.size(); ++i_stage) diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 6ce132f300c..360e5aaf17c 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -43,7 +43,8 @@ public: const StorageMetadataPtr & metadata_snapshot_, MutationCommands commands_, ContextPtr context_, - bool can_execute_); + bool can_execute_, + bool is_lightweight_ = false); void validate(); @@ -80,10 +81,13 @@ public: private: ASTPtr prepare(bool dry_run); + ASTPtr prepareLightweightDelete(bool dry_run); struct Stage; ASTPtr prepareInterpreterSelectQuery(std::vector &prepared_stages, bool dry_run); + ASTPtr prepareInterpreterSelectQueryLightweight(std::vector &prepared_stages, bool dry_run); + QueryPipelineBuilder addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const; std::optional getStorageSortDescriptionIfPossible(const Block & header) const; @@ -97,6 +101,9 @@ private: bool can_execute; SelectQueryOptions select_limits; + /// True for lightweight delete. + bool is_lightweight = false; + ASTPtr mutation_ast; /// We have to store interpreter because it use own copy of context diff --git a/src/Storages/MergeTree/FutureMergedMutatedPart.h b/src/Storages/MergeTree/FutureMergedMutatedPart.h index 4447687c3d9..06659249cae 100644 --- a/src/Storages/MergeTree/FutureMergedMutatedPart.h +++ b/src/Storages/MergeTree/FutureMergedMutatedPart.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -23,6 +24,7 @@ struct FutureMergedMutatedPart MergeTreePartInfo part_info; MergeTreeData::DataPartsVector parts; MergeType merge_type = MergeType::Regular; + MutationType mutation_type = MutationType::Ordinary; const MergeTreePartition & getPartition() const { return parts.front()->partition; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 60941108f00..cea8a91e15a 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -648,6 +648,7 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks checkConsistency(require_columns_checksums); loadDefaultCompressionCodec(); + loadDeletedRowMask(); } catch (...) { @@ -1208,6 +1209,34 @@ void IMergeTreeDataPart::loadColumns(bool require) setSerializationInfos(infos); } +void IMergeTreeDataPart::loadDeletedRowMask() +{ + if (part_type == Type::Compact) + return; + + auto path = fs::path(getFullRelativePath()) / DELETED_ROW_MARK_FILE_NAME; + if (volume->getDisk()->exists(path)) + { + has_lightweight_delete = true; + + auto in = openForReading(volume->getDisk(), path); + readString(deleted_rows_mask, *in); + } +} + +void IMergeTreeDataPart::writeLightWeightDeletedMask(String bitmap) const +{ + if (bitmap.empty()) + return; + + auto disk = volume->getDisk(); + String file_name = fs::path(getFullRelativePath()) / DELETED_ROW_MARK_FILE_NAME; + + /// write Non-Empty merged bitmap + auto out = disk->writeFile(file_name); + DB::writeText(bitmap, *out); +} + void IMergeTreeDataPart::assertHasVersionMetadata(MergeTreeTransaction * txn) const { TransactionID expected_tid = txn ? txn->tid : Tx::PrehistoricTID; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 7f3c41ce4c2..f90649b388c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -327,6 +327,10 @@ public: mutable VersionMetadata version; + /// True if the part has deleted_row_mask.bin file used for lightweight delete. + bool has_lightweight_delete = false; + String deleted_rows_mask; + /// For data in RAM ('index') UInt64 getIndexSizeInBytes() const; UInt64 getIndexSizeInAllocatedBytes() const; @@ -404,6 +408,9 @@ public: static inline constexpr auto TXN_VERSION_METADATA_FILE_NAME = "txn_version.txt"; + /// File name for lightweight delete rows mask bitmap file. + static inline constexpr auto DELETED_ROW_MARK_FILE_NAME = "deleted_row_mask.bin"; + /// One of part files which is used to check how many references (I'd like /// to say hardlinks, but it will confuse even more) we have for the part /// for zero copy replication. Sadly it's very complex. @@ -456,6 +463,11 @@ public: /// Check metadata in cache is consistent with actual metadata on disk(if use_metadata_cache is true) std::unordered_map checkMetadata() const; + /// Reads deleted row mask from deleted_row_mask.bin if exists and set has_lightweight_delete. + void loadDeletedRowMask(); + + /// Write lightweight deleted mask to a file. + void writeLightWeightDeletedMask(String bitmap) const; protected: diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 6e14e9c7aa9..1f5589bb75d 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -673,6 +673,9 @@ MergeTreeRangeReader::MergeTreeRangeReader( sample_block.insert(ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), column_name)); } + need_read_deleted_mask = merge_tree_reader->data_part->has_lightweight_delete; + deleted_rows_mask = merge_tree_reader->data_part->deleted_rows_mask; + if (prewhere_info) { const auto & step = *prewhere_info; @@ -852,6 +855,8 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar read_result = startReadingChain(max_rows, ranges); read_result.num_rows = read_result.numReadRows(); + executeDeletedRowMaskFilterColumns(read_result); + if (read_result.num_rows) { /// Physical columns go first and then some virtual columns follow @@ -951,6 +956,10 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t fillPartOffsetColumn(result, leading_begin_part_offset, leading_end_part_offset); } + /// Do similar as part_offset for deleted mask. + if (need_read_deleted_mask) + fillDeletedRowMaskColumn(result, leading_begin_part_offset, leading_end_part_offset); + return result; } @@ -980,6 +989,43 @@ void MergeTreeRangeReader::fillPartOffsetColumn(ReadResult & result, UInt64 lead result.columns.emplace_back(std::move(column)); } +/// Fill deleted_row_mask column, referenced from fillPartOffsetColumn(). +void MergeTreeRangeReader::fillDeletedRowMaskColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset) +{ + size_t num_rows = result.numReadRows(); + + auto mask_column = ColumnUInt8::create(num_rows); + ColumnUInt8::Container & vec = mask_column->getData(); + + UInt8 * pos = vec.data(); + UInt8 * end = &vec[num_rows]; + + while (pos < end && leading_begin_part_offset < leading_end_part_offset) + { + if (deleted_rows_mask[leading_begin_part_offset++] == '0') + *pos++ = 1; + else + *pos++ = 0; + } + + const auto start_ranges = result.startedRanges(); + + for (const auto & start_range : start_ranges) + { + UInt64 start_part_offset = index_granularity->getMarkStartingRow(start_range.range.begin); + UInt64 end_part_offset = index_granularity->getMarkStartingRow(start_range.range.end); + + while (pos < end && start_part_offset < end_part_offset) + { + if (deleted_rows_mask[start_part_offset++] == '0') + *pos++ = 1; + else + *pos++ = 0; + } + } + + result.deleted_mask_filter_holder = std::move(mask_column); +} Columns MergeTreeRangeReader::continueReadingChain(const ReadResult & result, size_t & num_rows) { @@ -1095,6 +1141,36 @@ static ColumnPtr combineFilters(ColumnPtr first, ColumnPtr second) return mut_first; } + +/// Implicitly apply deleted mask filter to columns. +/// If there is no prewhere_info, apply directly the deleted mask filter. +/// If prewhere_info exists, works like row_level_filter and prewhere filter. +void MergeTreeRangeReader::executeDeletedRowMaskFilterColumns(ReadResult & result) +{ + if (prewhere_info || !need_read_deleted_mask || !result.deleted_mask_filter_holder) + return; + + const ColumnUInt8 * mask_filter = typeid_cast(result.deleted_mask_filter_holder.get()); + filterColumns(result.columns, mask_filter->getData()); + + bool has_column = false; + for (auto & column : result.columns) + { + if (column) + { + has_column = true; + result.num_rows = column->size(); + break; + } + } + + /// There is only one filter column. Record the actual number. + if (!has_column) + result.num_rows = result.countBytesInResultFilter(mask_filter->getData()); + + result.need_filter = true; +} + void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result) { if (!prewhere_info) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index b4fb8913122..bd38be47ed9 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -160,6 +160,9 @@ public: /// The number of bytes read from disk. size_t numBytesRead() const { return num_bytes_read; } + /// Similar as filter that you need to apply to newly-read columns + ColumnPtr deleted_mask_filter_holder; + private: /// Only MergeTreeRangeReader is supposed to access ReadResult internals. friend class MergeTreeRangeReader; @@ -251,6 +254,8 @@ private: Columns continueReadingChain(const ReadResult & result, size_t & num_rows); void executePrewhereActionsAndFilterColumns(ReadResult & result); void fillPartOffsetColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset); + void fillDeletedRowMaskColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset); + void executeDeletedRowMaskFilterColumns(ReadResult & result); IMergeTreeReader * merge_tree_reader = nullptr; const MergeTreeIndexGranularity * index_granularity = nullptr; @@ -264,6 +269,8 @@ private: bool last_reader_in_chain = false; bool is_initialized = false; Names non_const_virtual_column_names; + bool need_read_deleted_mask = false; + String deleted_rows_mask; }; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 3a5aa2f8860..25f6fb580a2 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -624,6 +625,8 @@ struct MutationContext MergeTreeData::DataPartPtr source_part; StoragePtr storage_from_source_part; + bool is_lightweight_mutation{0}; + StorageMetadataPtr metadata_snapshot; MutationCommandsConstPtr commands; @@ -1351,6 +1354,193 @@ private: std::unique_ptr part_merger_writer_task{nullptr}; }; +class LightweightDeleteTask : public IExecutableTask +{ +public: + + explicit LightweightDeleteTask(MutationContextPtr ctx_) : ctx(ctx_) {} + + void onCompleted() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + StorageID getStorageID() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + UInt64 getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + + bool executeStep() override + { + switch (state) + { + case State::NEED_PREPARE: + { + prepare(); + + state = State::NEED_EXECUTE; + return true; + } + case State::NEED_EXECUTE: + { + execute(); + + state = State::NEED_FINALIZE; + return true; + } + case State::NEED_FINALIZE: + { + finalize(); + + state = State::SUCCESS; + return true; + } + case State::SUCCESS: + { + return false; + } + } + return false; + } + +private: + + void prepare() + { + if (ctx->execute_ttl_type != ExecuteTTLType::NONE) + ctx->files_to_skip.insert("ttl.txt"); + + ctx->disk->createDirectories(ctx->new_part_tmp_path); + + /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. + TransactionID tid = ctx->txn ? ctx->txn->tid : Tx::PrehistoricTID; + /// NOTE do not pass context for writing to system.transactions_info_log, + /// because part may have temporary name (with temporary block numbers). Will write it later. + ctx->new_data_part->version.setCreationTID(tid, nullptr); + ctx->new_data_part->storeVersionMetadata(); + + NameSet hardlinked_files; + /// Create hardlinks for unchanged files + for (auto it = ctx->disk->iterateDirectory(ctx->source_part->getFullRelativePath()); it->isValid(); it->next()) + { + if (ctx->files_to_skip.contains(it->name())) + continue; + + String destination = ctx->new_part_tmp_path; + String file_name = it->name(); + + destination += file_name; + + if (!ctx->disk->isDirectory(it->path())) + { + ctx->disk->createHardLink(it->path(), destination); + hardlinked_files.insert(file_name); + } + else if (!endsWith(".tmp_proj", file_name)) // ignore projection tmp merge dir + { + // it's a projection part directory + ctx->disk->createDirectories(destination); + for (auto p_it = ctx->disk->iterateDirectory(it->path()); p_it->isValid(); p_it->next()) + { + String p_destination = fs::path(destination) / p_it->name(); + ctx->disk->createHardLink(p_it->path(), p_destination); + hardlinked_files.insert(p_it->name()); + } + } + } + + /// Tracking of hardlinked files required for zero-copy replication. + /// We don't remove them when we delete last copy of source part because + /// new part can use them. + ctx->hardlinked_files.source_table_shared_id = ctx->source_part->storage.getTableSharedID(); + ctx->hardlinked_files.source_part_name = ctx->source_part->name; + ctx->hardlinked_files.hardlinks_from_source_part = hardlinked_files; + + /// Only the _delete mask column will be written. + (*ctx->mutate_entry)->columns_written = 1; + + ctx->new_data_part->checksums = ctx->source_part->checksums; + + ctx->compression_codec = ctx->source_part->default_codec; + + if (ctx->mutating_pipeline_builder.initialized()) + { + QueryPipelineBuilder builder(std::move(ctx->mutating_pipeline_builder)); + + if (ctx->execute_ttl_type == ExecuteTTLType::NORMAL) + builder.addTransform(std::make_shared(builder.getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true)); + + if (ctx->execute_ttl_type == ExecuteTTLType::RECALCULATE) + builder.addTransform(std::make_shared(builder.getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true)); + + ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); + ctx->mutating_pipeline.setProgressCallback(ctx->progress_callback); + /// Is calculated inside MergeProgressCallback. + ctx->mutating_pipeline.disableProfileEventUpdate(); + ctx->mutating_executor = std::make_unique(ctx->mutating_pipeline); + } + } + + void execute() + { + Block block; + bool has_deleted_rows = false; + + /// If this part has already applied lightweight mutation, load the past latest bitmap to merge with current bitmap + if (ctx->source_part->has_lightweight_delete) + { + new_bitmap = ctx->source_part->deleted_rows_mask; + has_deleted_rows = true; + } + else + new_bitmap.resize(ctx->source_part->rows_count, '0'); + + /// Mark the data corresponding to the offset in the as deleted. + while (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor->pull(block)) + { + size_t block_rows = block.rows(); + + if (block_rows && !has_deleted_rows) + has_deleted_rows = true; + + const auto & cols = block.getColumns(); + const auto * offset_col = typeid_cast(cols[0].get()); + const UInt64 * offset = offset_col->getData().data(); + + /// Fill 1 for rows in offset + for (size_t current_row = 0; current_row < block_rows; current_row++) + new_bitmap[offset[current_row]] = '1'; + } + + if (has_deleted_rows) + { + ctx->new_data_part->writeLightWeightDeletedMask(new_bitmap); + ctx->new_data_part->has_lightweight_delete = true; + ctx->new_data_part->deleted_rows_mask = new_bitmap; + } + } + + void finalize() + { + if (ctx->mutating_executor) + { + ctx->mutating_executor.reset(); + ctx->mutating_pipeline.reset(); + } + + MutationHelpers::finalizeMutatedPart(ctx->source_part, ctx->new_data_part, ctx->execute_ttl_type, ctx->compression_codec, ctx->context); + } + + enum class State + { + NEED_PREPARE, + NEED_EXECUTE, + NEED_FINALIZE, + + SUCCESS + }; + + State state{State::NEED_PREPARE}; + + MutationContextPtr ctx; + + String new_bitmap; +}; + MutateTask::MutateTask( FutureMergedMutatedPartPtr future_part_, @@ -1437,8 +1627,10 @@ bool MutateTask::prepare() command.partition, context_for_reading)) ctx->commands_for_part.emplace_back(command); } - - if (ctx->source_part->isStoredOnDisk() && !isStorageTouchedByMutations( + /// Enable lightweight delete for wide part only. + if (isWidePart(ctx->source_part) && (ctx->future_part->mutation_type == MutationType::Lightweight)) + ctx->is_lightweight_mutation = true; + if (ctx->source_part->isStoredOnDisk() && !ctx->is_lightweight_mutation && !isStorageTouchedByMutations( ctx->storage_from_source_part, ctx->metadata_snapshot, ctx->commands_for_part, Context::createCopy(context_for_reading))) { LOG_TRACE(ctx->log, "Part {} doesn't change up to mutation version {}", ctx->source_part->name, ctx->future_part->part_info.mutation); @@ -1457,7 +1649,7 @@ bool MutateTask::prepare() if (!ctx->for_interpreter.empty()) { ctx->interpreter = std::make_unique( - ctx->storage_from_source_part, ctx->metadata_snapshot, ctx->for_interpreter, context_for_reading, true); + ctx->storage_from_source_part, ctx->metadata_snapshot, ctx->for_interpreter, context_for_reading, true, ctx->is_lightweight_mutation); ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices(); ctx->materialized_projections = ctx->interpreter->grabMaterializedProjections(); ctx->mutation_kind = ctx->interpreter->getMutationKind(); @@ -1516,6 +1708,11 @@ bool MutateTask::prepare() { task = std::make_unique(ctx); } + else if (ctx->is_lightweight_mutation) + { + /// We will modify or create only deleted_row_mask for lightweight delete. Other columns and key values are copied as-is. + task = std::make_unique(ctx); + } else /// TODO: check that we modify only non-key columns in this case. { /// We will modify only some of the columns. Other columns and key values can be copied as-is. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 495218d4ef5..83ca5c794e8 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1024,12 +1024,17 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( auto commands = std::make_shared(); size_t current_ast_elements = 0; auto last_mutation_to_apply = mutations_end_it; + MutationType first_mutation_type = mutations_begin_it->second.type; for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { /// Do not squash mutations from different transactions to be able to commit/rollback them independently. if (first_mutation_tid != it->second.tid) break; + /// Do not combine mutations with different types. + if (it->second.type != first_mutation_type) + break; + size_t commands_size = 0; MutationCommands commands_for_size_validation; for (const auto & command : it->second.commands) @@ -1114,6 +1119,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( future_part->part_info = new_part_info; future_part->name = part->getNewName(new_part_info); future_part->type = part->getType(); + future_part->mutation_type = first_mutation_type; tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); return std::make_shared(future_part, std::move(tagger), commands, txn); From b4a37e1e22ae8218eeb48bb300d5db6a1b41bad2 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Tue, 21 Jun 2022 11:07:25 +0800 Subject: [PATCH 232/659] Disable optimizations for count() when lightweight delete exists, add hasLightweightDelete() function in IMergeTreeDataPart --- src/Interpreters/InterpreterSelectQuery.cpp | 1 + src/Storages/IStorage.h | 3 +++ src/Storages/MergeTree/IMergeTreeDataPart.h | 3 +++ src/Storages/MergeTree/MergeTreeData.cpp | 13 ++++++++++++- src/Storages/MergeTree/MergeTreeData.h | 3 +++ src/Storages/MergeTree/MergeTreeRangeReader.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 5 +++++ src/Storages/StorageMergeTree.h | 1 + 9 files changed, 30 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ac31588d210..5852200da2c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1936,6 +1936,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc && !settings.empty_result_for_aggregation_by_empty_set && storage && storage->getName() != "MaterializedMySQL" + && !storage->hasLightweightDelete() && !row_policy_filter && processing_stage == QueryProcessingStage::FetchColumns && query_analyzer->hasAggregation() diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index e265c94eb11..899e7e365ce 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -235,6 +235,9 @@ public: /// Returns true if the storage supports backup/restore for specific partitions. virtual bool supportsBackupPartition() const { return false; } + /// Return true if there are lightweight parts. + virtual bool hasLightweightDelete() const { return false; } + private: StorageID storage_id; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index f90649b388c..ff1a8269cb9 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -463,6 +463,9 @@ public: /// Check metadata in cache is consistent with actual metadata on disk(if use_metadata_cache is true) std::unordered_map checkMetadata() const; + /// True if here is light weight bitmap file in part. + bool hasLightweightDelete() const { return has_lightweight_delete; } + /// Reads deleted row mask from deleted_row_mask.bin if exists and set has_lightweight_delete. void loadDeletedRowMask(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 64aaa40bd4c..6b3ddbf8d49 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1002,6 +1002,7 @@ void MergeTreeData::loadDataPartsFromDisk( size_t suspicious_broken_parts_bytes = 0; std::atomic has_adaptive_parts = false; std::atomic has_non_adaptive_parts = false; + std::atomic has_lightweight_in_parts = false; std::mutex mutex; auto load_part = [&](const String & part_name, const DiskPtr & part_disk_ptr) @@ -1075,6 +1076,10 @@ void MergeTreeData::loadDataPartsFromDisk( else has_adaptive_parts.store(true, std::memory_order_relaxed); + /// Check if there is lightweight delete in part + if (part->hasLightweightDelete()) + has_lightweight_in_parts.store(true, std::memory_order_relaxed); + part->modification_time = part_disk_ptr->getLastModified(fs::path(relative_data_path) / part_name).epochTime(); /// Assume that all parts are Active, covered parts will be detected and marked as Outdated later part->setState(DataPartState::Active); @@ -1149,6 +1154,9 @@ void MergeTreeData::loadDataPartsFromDisk( has_non_adaptive_index_granularity_parts = has_non_adaptive_parts; + if (has_lightweight_in_parts) + has_lightweight_delete_parts.store(true); + if (suspicious_broken_parts > settings->max_suspicious_broken_parts && !skip_sanity_checks) throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS, "Suspiciously many ({} parts, {} in total) broken parts to remove while maximum allowed broken parts count is {}. You can change the maximum value " @@ -2864,6 +2872,9 @@ bool MergeTreeData::renameTempPartAndReplaceImpl( throw Exception("MergeTreeData::Transaction for one table cannot be used with another. It is a bug.", ErrorCodes::LOGICAL_ERROR); + if (part.hasLightweightDelete()) + has_lightweight_delete_parts.store(true); + checkPartCanBeAddedToTable(part, lock); DataPartPtr covering_part; @@ -5672,7 +5683,7 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg ProjectionCandidate * selected_candidate = nullptr; size_t min_sum_marks = std::numeric_limits::max(); - if (metadata_snapshot->minmax_count_projection) + if (metadata_snapshot->minmax_count_projection && !has_lightweight_delete_parts.load(std::memory_order_relaxed)) /// Disable ReadFromStorage for parts with lightweight. add_projection_candidate(*metadata_snapshot->minmax_count_projection, true); std::optional minmax_count_projection_candidate; if (!candidates.empty()) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 0b6e757ab49..e18a3b20b74 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -942,6 +942,9 @@ public: bool has_non_adaptive_index_granularity_parts = false; + /// True if at least one part contains lightweight delete. + mutable std::atomic_bool has_lightweight_delete_parts = false; + /// Parts that currently moving from disk/volume to another. /// This set have to be used with `currently_processing_in_background_mutex`. /// Moving may conflict with merges and mutations, but this is OK, because diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 1f5589bb75d..b37c5d6489f 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -673,7 +673,7 @@ MergeTreeRangeReader::MergeTreeRangeReader( sample_block.insert(ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), column_name)); } - need_read_deleted_mask = merge_tree_reader->data_part->has_lightweight_delete; + need_read_deleted_mask = merge_tree_reader->data_part->hasLightweightDelete(); deleted_rows_mask = merge_tree_reader->data_part->deleted_rows_mask; if (prewhere_info) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 25f6fb580a2..119a216bc88 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1481,7 +1481,7 @@ private: bool has_deleted_rows = false; /// If this part has already applied lightweight mutation, load the past latest bitmap to merge with current bitmap - if (ctx->source_part->has_lightweight_delete) + if (ctx->source_part->hasLightweightDelete()) { new_bitmap = ctx->source_part->deleted_rows_mask; has_deleted_rows = true; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 83ca5c794e8..5480114359e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -569,6 +569,11 @@ void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr quer waitForMutation(version); } +bool StorageMergeTree::hasLightweightDelete() const +{ + return has_lightweight_delete_parts.load(std::memory_order_relaxed); +} + std::optional StorageMergeTree::getIncompleteMutationsStatus(Int64 mutation_version, std::set * mutation_ids) const { std::unique_lock lock(currently_processing_in_background_mutex); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index a27925994c9..6251d0d75e0 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -89,6 +89,7 @@ public: /// Support lightweight delete. void mutate(const MutationCommands & commands, ContextPtr context, MutationType type); + bool hasLightweightDelete() const override; /// Return introspection information about currently processing or recently processed mutations. std::vector getMutationsStatus() const override; From 2c74e9b86667a7945e84cf80e0d1f137bc6cfeb7 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Tue, 21 Jun 2022 13:36:21 +0800 Subject: [PATCH 233/659] Merge support to apply lightweight delete mask --- .../MergeTree/MergeTreeSequentialSource.cpp | 118 ++++++++++++------ 1 file changed, 79 insertions(+), 39 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 7c4cc04ba52..926fde522d0 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -75,53 +75,93 @@ try { const auto & header = getPort().getHeader(); - if (!isCancelled() && current_row < data_part->rows_count) + /// The chunk after deleted mask applied maybe empty. But the empty chunk means done of read rows. + String deleted_rows_mask; + bool need_read_deleted_mask = data_part->hasLightweightDelete(); + if (need_read_deleted_mask) + deleted_rows_mask = data_part->deleted_rows_mask; + + do { - size_t rows_to_read = data_part->index_granularity.getMarkRows(current_mark); - bool continue_reading = (current_mark != 0); - - const auto & sample = reader->getColumns(); - Columns columns(sample.size()); - size_t rows_read = reader->readRows(current_mark, data_part->getMarksCount(), continue_reading, rows_to_read, columns); - - if (rows_read) + if (!isCancelled() && current_row < data_part->rows_count) { - current_row += rows_read; - current_mark += (rows_to_read == rows_read); + size_t rows_to_read = data_part->index_granularity.getMarkRows(current_mark); + bool continue_reading = (current_mark != 0); - bool should_evaluate_missing_defaults = false; - reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_read); + const auto & sample = reader->getColumns(); + Columns columns(sample.size()); + size_t rows_read = reader->readRows(current_mark, data_part->getMarksCount(), continue_reading, rows_to_read, columns); - if (should_evaluate_missing_defaults) + if (rows_read) { - reader->evaluateMissingDefaults({}, columns); + current_row += rows_read; + current_mark += (rows_to_read == rows_read); + + if (need_read_deleted_mask) + { + size_t pos = current_row - rows_read; + + /// Get deleted mask for rows_read + IColumn::Filter deleted_rows_filter(rows_read, true); + for (size_t i = 0; i < rows_read; i++) + { + if (deleted_rows_mask[pos++] == '1') + deleted_rows_filter[i] = 0; + } + + // Filter only if some items were deleted + if (auto num_deleted_rows = std::count(deleted_rows_filter.begin(), deleted_rows_filter.end(), 0)) + { + const auto remaining_rows = deleted_rows_filter.size() - num_deleted_rows; + + /// If we return {} here, it means finished, no reading of the following rows. + /// Continue to read until remaining rows are not zero or reach the end (REAL finish). + if (!remaining_rows) + continue; + + for (auto & col : columns) + col = col->filter(deleted_rows_filter, remaining_rows); + + /// Update rows_read with actual rows in columns + rows_read = remaining_rows; + } + } + + bool should_evaluate_missing_defaults = false; + reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_read); + + if (should_evaluate_missing_defaults) + { + reader->evaluateMissingDefaults({}, columns); + } + + reader->performRequiredConversions(columns); + + /// Reorder columns and fill result block. + size_t num_columns = sample.size(); + Columns res_columns; + res_columns.reserve(num_columns); + + auto it = sample.begin(); + for (size_t i = 0; i < num_columns; ++i) + { + if (header.has(it->name)) + res_columns.emplace_back(std::move(columns[i])); + + ++it; + } + + return Chunk(std::move(res_columns), rows_read); } - - reader->performRequiredConversions(columns); - - /// Reorder columns and fill result block. - size_t num_columns = sample.size(); - Columns res_columns; - res_columns.reserve(num_columns); - - auto it = sample.begin(); - for (size_t i = 0; i < num_columns; ++i) - { - if (header.has(it->name)) - res_columns.emplace_back(std::move(columns[i])); - - ++it; - } - - return Chunk(std::move(res_columns), rows_read); } - } - else - { - finish(); - } + else + { + finish(); + } + + return {}; + } while (true); - return {}; } catch (...) { From 9d27af7ee223335fb9712ada3e944b7fc13663b6 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Thu, 23 Jun 2022 21:02:22 +0800 Subject: [PATCH 234/659] For some columns mutations, skip to apply deleted mask when read some columns. Also add unit test case --- src/Interpreters/Context.h | 6 ++ src/Interpreters/MutationsInterpreter.cpp | 13 +++- src/Interpreters/MutationsInterpreter.h | 4 ++ .../QueryPlan/ReadFromMergeTree.cpp | 1 + src/Storages/MergeTree/IMergeTreeReader.h | 2 + src/Storages/MergeTree/MergeTreeIOSettings.h | 2 + .../MergeTree/MergeTreeRangeReader.cpp | 5 +- src/Storages/MergeTree/MutateTask.cpp | 13 ++-- ...lightweight_delete_on_merge_tree.reference | 31 ++++++++++ ...02319_lightweight_delete_on_merge_tree.sql | 59 +++++++++++++++++++ ...19_standard_delete_on_merge_tree.reference | 3 - .../02319_standard_delete_on_merge_tree.sql | 21 ------- 12 files changed, 129 insertions(+), 31 deletions(-) create mode 100644 tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference create mode 100644 tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql delete mode 100644 tests/queries/0_stateless/02319_standard_delete_on_merge_tree.reference delete mode 100644 tests/queries/0_stateless/02319_standard_delete_on_merge_tree.sql diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 4da5326aca4..c83e38a0ed1 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -360,6 +360,9 @@ private: inline static ContextPtr global_context_instance; + /// A flag, used to mark if reader needs to apply deleted rows mask. + bool skip_deleted_mask = false; + public: // Top-level OpenTelemetry trace context for the query. Makes sense only for a query context. OpenTelemetryTraceContext query_trace_context; @@ -912,6 +915,9 @@ public: bool isInternalQuery() const { return is_internal_query; } void setInternalQuery(bool internal) { is_internal_query = internal; } + bool skipDeletedMask() const { return skip_deleted_mask; } + void setSkipDeletedMask(bool skip) { skip_deleted_mask = skip; } + ActionLocksManagerPtr getActionLocksManager(); enum class ApplicationType diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 8753905521d..cc658bfa764 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1041,7 +1041,18 @@ QueryPipelineBuilder MutationsInterpreter::execute() throw Exception("Cannot execute mutations interpreter because can_execute flag set to false", ErrorCodes::LOGICAL_ERROR); if (!select_interpreter) - select_interpreter = std::make_unique(mutation_ast, context, storage, metadata_snapshot, select_limits); + { + /// Skip to apply deleted mask for MutateSomePartColumn cases when part has lightweight delete. + if (!is_lightweight && skip_deleted_mask) + { + auto context_for_reading = Context::createCopy(context); + context_for_reading->setSkipDeletedMask(skip_deleted_mask); + select_interpreter = std::make_unique(mutation_ast, context_for_reading, storage, metadata_snapshot, select_limits); + } + else + select_interpreter = std::make_unique(mutation_ast, context, storage, metadata_snapshot, select_limits); + } + QueryPlan plan; select_interpreter->buildQueryPlan(plan); diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 360e5aaf17c..fb94c1f5f33 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -79,6 +79,8 @@ public: MutationKind::MutationKindEnum getMutationKind() const { return mutation_kind.mutation_kind; } + void SetSkipDeletedMask(bool skip) { skip_deleted_mask = skip; } + private: ASTPtr prepare(bool dry_run); ASTPtr prepareLightweightDelete(bool dry_run); @@ -103,6 +105,8 @@ private: /// True for lightweight delete. bool is_lightweight = false; + /// True for MutateSomePartColumns on part with lightweight. + bool skip_deleted_mask = false; ASTPtr mutation_ast; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index b4e143cc002..59f6ec558e7 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -62,6 +62,7 @@ static MergeTreeReaderSettings getMergeTreeReaderSettings( .save_marks_in_cache = true, .checksum_on_read = settings.checksum_on_read, .read_in_order = query_info.input_order_info != nullptr, + .skip_deleted_mask = context->skipDeletedMask(), }; } diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index b13db9c3255..229f62da293 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -62,6 +62,8 @@ public: MergeTreeData::DataPartPtr data_part; + bool needReadDeletedMask() { return !settings.skip_deleted_mask && data_part->hasLightweightDelete(); } + protected: /// Returns actual column type in part, which can differ from table metadata. NameAndTypePair getColumnFromPart(const NameAndTypePair & required_column) const; diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index 85cf3e9eda6..02372011876 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -23,6 +23,8 @@ struct MergeTreeReaderSettings bool checksum_on_read = true; /// True if we read in order of sorting key. bool read_in_order = false; + /// Do not apply deleted mask for internal select from mutate some part columns. + bool skip_deleted_mask = false; }; struct MergeTreeWriterSettings diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index b37c5d6489f..5625ea06b7d 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -673,8 +673,9 @@ MergeTreeRangeReader::MergeTreeRangeReader( sample_block.insert(ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), column_name)); } - need_read_deleted_mask = merge_tree_reader->data_part->hasLightweightDelete(); - deleted_rows_mask = merge_tree_reader->data_part->deleted_rows_mask; + need_read_deleted_mask = merge_tree_reader->needReadDeletedMask(); + if (need_read_deleted_mask) + deleted_rows_mask = merge_tree_reader->data_part->deleted_rows_mask; if (prewhere_info) { diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 119a216bc88..21122a42833 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -604,6 +604,7 @@ void finalizeMutatedPart( new_data_part->minmax_idx = source_part->minmax_idx; new_data_part->modification_time = time(nullptr); new_data_part->loadProjections(false, false); + new_data_part->loadDeletedRowMask(); new_data_part->setBytesOnDisk(new_data_part->data_part_storage->calculateTotalSizeOnDisk()); new_data_part->default_codec = codec; new_data_part->calculateColumnsAndSecondaryIndicesSizesOnDisk(); @@ -1509,8 +1510,6 @@ private: if (has_deleted_rows) { ctx->new_data_part->writeLightWeightDeletedMask(new_bitmap); - ctx->new_data_part->has_lightweight_delete = true; - ctx->new_data_part->deleted_rows_mask = new_bitmap; } } @@ -1646,6 +1645,8 @@ bool MutateTask::prepare() ctx->stage_progress = std::make_unique(1.0); + bool need_mutate_all_columns = !isWidePart(ctx->source_part); + if (!ctx->for_interpreter.empty()) { ctx->interpreter = std::make_unique( @@ -1653,6 +1654,11 @@ bool MutateTask::prepare() ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices(); ctx->materialized_projections = ctx->interpreter->grabMaterializedProjections(); ctx->mutation_kind = ctx->interpreter->getMutationKind(); + + /// Skip to apply deleted mask when reading for MutateSomePartColumns. + need_mutate_all_columns = need_mutate_all_columns || (ctx->mutation_kind == MutationsInterpreter::MutationKind::MUTATE_OTHER && ctx->interpreter->isAffectingAllColumns()); + if(!need_mutate_all_columns && ctx->source_part->hasLightweightDelete() && !ctx->is_lightweight_mutation) + ctx->interpreter->SetSkipDeletedMask(true); ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); ctx->progress_callback = MergeProgressCallback((*ctx->mutate_entry)->ptr(), ctx->watch_prev_elapsed, *ctx->stage_progress); @@ -1703,8 +1709,7 @@ bool MutateTask::prepare() /// All columns from part are changed and may be some more that were missing before in part /// TODO We can materialize compact part without copying data - if (!isWidePart(ctx->source_part) - || (ctx->mutation_kind == MutationsInterpreter::MutationKind::MUTATE_OTHER && ctx->interpreter && ctx->interpreter->isAffectingAllColumns())) + if (need_mutate_all_columns) { task = std::make_unique(ctx); } diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference new file mode 100644 index 00000000000..aefc0c88762 --- /dev/null +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference @@ -0,0 +1,31 @@ +99 +95 +0 +-----lightweight mutation type----- +1 DELETE WHERE (c % 5) = 1 1 +1 DELETE WHERE c = 4 1 +0 MATERIALIZE INDEX i_c 1 +0 UPDATE b = -1 WHERE a < 3 1 +0 DROP INDEX i_c 1 +-----Check that select and merge with lightweight delete.----- +7 +0 -1 0 +2 -1 2 +3 3 3 +5 5 5 +7 7 7 +8 8 8 +9 9 9 +t_light 0 0_1_1_0_10 2 +t_light 1 1_2_2_0_10 2 +t_light 2 2_3_3_0_10 2 +t_light 3 3_4_4_0_10 2 +t_light 4 4_5_5_0_10 2 +7 +t_light 0 0_1_1_1_10 2 +t_light 2 2_3_3_1_10 2 +t_light 3 3_4_4_1_10 2 +t_light 4 4_5_5_1_10 1 +-----Test lightweight delete in multi blocks----- +1000 -2 +1005 -2 diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql new file mode 100644 index 00000000000..da302f0f4ff --- /dev/null +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql @@ -0,0 +1,59 @@ +DROP TABLE IF EXISTS merge_table_standard_delete; + +CREATE TABLE merge_table_standard_delete(id Int32, name String) ENGINE = MergeTree order by id settings min_bytes_for_wide_part=0; + +INSERT INTO merge_table_standard_delete select number, toString(number) from numbers(100); + +SET mutations_sync = 1; + +DELETE FROM merge_table_standard_delete WHERE id = 10; + +SELECT COUNT() FROM merge_table_standard_delete; + +DELETE FROM merge_table_standard_delete WHERE name IN ('1','2','3','4'); + +SELECT COUNT() FROM merge_table_standard_delete; + +DELETE FROM merge_table_standard_delete WHERE 1; + +SELECT COUNT() FROM merge_table_standard_delete; + +DROP TABLE merge_table_standard_delete; + +drop table if exists t_light; +create table t_light(a int, b int, c int, index i_c(b) type minmax granularity 4) engine = MergeTree order by a partition by c % 5 settings min_bytes_for_wide_part=0; +INSERT INTO t_light SELECT number, number, number FROM numbers(10); + +SELECT '-----lightweight mutation type-----'; + +DELETE FROM t_light WHERE c%5=1; +DELETE FROM t_light WHERE c=4; +alter table t_light MATERIALIZE INDEX i_c; +alter table t_light update b=-1 where a<3; +alter table t_light drop index i_c; + +SELECT is_lightweight, command, is_done FROM system.mutations WHERE database = currentDatabase() AND table = 't_light'; + +SELECT '-----Check that select and merge with lightweight delete.-----'; +select count(*) from t_light; +select * from t_light order by a; + +select table, partition, name, rows from system.parts where database = currentDatabase() AND active and table ='t_light' order by name; + +optimize table t_light final; +select count(*) from t_light; + +select table, partition, name, rows from system.parts where database = currentDatabase() AND active and table ='t_light' and rows > 0 order by name; + +drop table t_light; + +SELECT '-----Test lightweight delete in multi blocks-----'; +CREATE TABLE t_large(a UInt32, b int) ENGINE=MergeTree order BY a settings min_bytes_for_wide_part=0; +INSERT INTO t_large SELECT number + 1, number + 1 FROM numbers(100000); + +DELETE FROM t_large WHERE a = 50000; +ALTER TABLE t_large UPDATE b = -2 WHERE a between 1000 and 1005; +ALTER TABLE t_large DELETE WHERE a=1; +SELECT * FROM t_large WHERE a in (1,1000,1005,50000) order by a; + +DROP TABLE t_large; diff --git a/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.reference b/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.reference deleted file mode 100644 index b343623df61..00000000000 --- a/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.reference +++ /dev/null @@ -1,3 +0,0 @@ -99 -95 -0 diff --git a/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.sql deleted file mode 100644 index 13b3a3e2701..00000000000 --- a/tests/queries/0_stateless/02319_standard_delete_on_merge_tree.sql +++ /dev/null @@ -1,21 +0,0 @@ -DROP TABLE IF EXISTS merge_table_standard_delete; - -CREATE TABLE merge_table_standard_delete(id Int32, name String) ENGINE = MergeTree order by id; - -INSERT INTO merge_table_standard_delete select number, toString(number) from numbers(100); - -SET mutations_sync = 1; - -DELETE FROM merge_table_standard_delete WHERE id = 10; - -SELECT COUNT() FROM merge_table_standard_delete; - -DELETE FROM merge_table_standard_delete WHERE name IN ('1','2','3','4'); - -SELECT COUNT() FROM merge_table_standard_delete; - -DELETE FROM merge_table_standard_delete WHERE 1; - -SELECT COUNT() FROM merge_table_standard_delete; - -DROP TABLE merge_table_standard_delete; From 11fdea6e4bff0ec55cae329c6f526274575c7b56 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Sun, 26 Jun 2022 20:01:39 +0800 Subject: [PATCH 235/659] Add missing code for deleted_mask_filter_holder --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 5625ea06b7d..857b70ede61 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -321,6 +321,7 @@ void MergeTreeRangeReader::ReadResult::clear() total_rows_per_granule = 0; filter_holder = nullptr; filter = nullptr; + deleted_mask_filter_holder = nullptr; } void MergeTreeRangeReader::ReadResult::shrink(Columns & old_columns) From 8ad2bb7c330a161878fc3a5d768bec0301dc2641 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Tue, 28 Jun 2022 10:07:49 +0800 Subject: [PATCH 236/659] Code changes due to master new fixes, and update reference for mutations table --- .../MergeTree/DataPartStorageOnDisk.cpp | 26 +++++++++++++ .../MergeTree/DataPartStorageOnDisk.h | 3 ++ src/Storages/MergeTree/IDataPartStorage.h | 3 ++ src/Storages/MergeTree/IMergeTreeDataPart.cpp | 15 ++------ src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeRangeReader.cpp | 15 +++++++- src/Storages/MergeTree/MutateTask.cpp | 37 +++++++++++-------- .../02117_show_create_table_system.reference | 1 + 9 files changed, 75 insertions(+), 29 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index f3b228a0748..816d31f44e5 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -368,7 +368,10 @@ void DataPartStorageOnDisk::clearDirectory( request.emplace_back(fs::path(dir) / "delete-on-destroy.txt", true); if (!is_projection) + { request.emplace_back(fs::path(dir) / "txn_version.txt", true); + request.emplace_back(fs::path(dir) / "deleted_row_mask.bin", true); + } disk->removeSharedFiles(request, !can_remove_shared_data, names_not_to_remove); disk->removeDirectory(dir); @@ -649,6 +652,29 @@ bool DataPartStorageOnDisk::shallParticipateInMerges(const IStoragePolicy & stor return !volume_ptr->areMergesAvoided(); } +void DataPartStorageOnDisk::loadDeletedRowMask(String & bitmap) const +{ + String deleted_mask_path = fs::path(getRelativePath()) / "deleted_row_mask.bin"; + auto disk = volume->getDisk(); + auto in = openForReading(disk, deleted_mask_path); + readString(bitmap, *in); +} + +void DataPartStorageOnDisk::writeLightweightDeletedMask(String & bitmap, Poco::Logger * log) const +{ + String deleted_mask_path = fs::path(getRelativePath()) / "deleted_row_mask.bin"; + auto disk = volume->getDisk(); + try + { + auto out = volume->getDisk()->writeFile(deleted_mask_path); + DB::writeText(bitmap, *out); + } + catch (Poco::Exception & e) + { + LOG_ERROR(log, "{} (while writing deleted rows mask file for lightweight delete: {})", e.what(), backQuote(fullPath(disk, deleted_mask_path))); + } +} + void DataPartStorageOnDisk::backup( TemporaryFilesOnDisks & temp_dirs, const MergeTreeDataPartChecksums & checksums, diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index 2426b5eee80..75f11617aef 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -86,6 +86,9 @@ public: bool shallParticipateInMerges(const IStoragePolicy &) const override; + void loadDeletedRowMask(String & bitmap) const override; + void writeLightweightDeletedMask(String & bitmap, Poco::Logger * log) const override; + void backup( TemporaryFilesOnDisks & temp_dirs, const MergeTreeDataPartChecksums & checksums, diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index f0173baecb7..16ff2485a6c 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -169,6 +169,9 @@ public: /// A leak of abstraction virtual bool shallParticipateInMerges(const IStoragePolicy &) const { return true; } + virtual void loadDeletedRowMask(String & bitmap) const = 0; + virtual void writeLightweightDeletedMask(String & bitmap, Poco::Logger * log) const = 0; + /// Create a backup of a data part. /// This method adds a new entry to backup_entries. /// Also creates a new tmp_dir for internal disk (if disk is mentioned the first time). diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index cea8a91e15a..3ee96459185 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1214,27 +1214,20 @@ void IMergeTreeDataPart::loadDeletedRowMask() if (part_type == Type::Compact) return; - auto path = fs::path(getFullRelativePath()) / DELETED_ROW_MARK_FILE_NAME; - if (volume->getDisk()->exists(path)) + if (data_part_storage->exists(DELETED_ROW_MARK_FILE_NAME)) { has_lightweight_delete = true; - auto in = openForReading(volume->getDisk(), path); - readString(deleted_rows_mask, *in); + data_part_storage->loadDeletedRowMask(deleted_rows_mask); } } -void IMergeTreeDataPart::writeLightWeightDeletedMask(String bitmap) const +void IMergeTreeDataPart::writeLightweightDeletedMask(String bitmap) const { if (bitmap.empty()) return; - auto disk = volume->getDisk(); - String file_name = fs::path(getFullRelativePath()) / DELETED_ROW_MARK_FILE_NAME; - - /// write Non-Empty merged bitmap - auto out = disk->writeFile(file_name); - DB::writeText(bitmap, *out); + data_part_storage->writeLightweightDeletedMask(bitmap, storage.log); } void IMergeTreeDataPart::assertHasVersionMetadata(MergeTreeTransaction * txn) const diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index ff1a8269cb9..975e7d6272d 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -470,7 +470,7 @@ public: void loadDeletedRowMask(); /// Write lightweight deleted mask to a file. - void writeLightWeightDeletedMask(String bitmap) const; + void writeLightweightDeletedMask(String bitmap) const; protected: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6b3ddbf8d49..c887552b35b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2872,7 +2872,7 @@ bool MergeTreeData::renameTempPartAndReplaceImpl( throw Exception("MergeTreeData::Transaction for one table cannot be used with another. It is a bug.", ErrorCodes::LOGICAL_ERROR); - if (part.hasLightweightDelete()) + if (part->hasLightweightDelete()) has_lightweight_delete_parts.store(true); checkPartCanBeAddedToTable(part, lock); diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 857b70ede61..d3ac822094a 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -1146,7 +1146,7 @@ static ColumnPtr combineFilters(ColumnPtr first, ColumnPtr second) /// Implicitly apply deleted mask filter to columns. /// If there is no prewhere_info, apply directly the deleted mask filter. -/// If prewhere_info exists, works like row_level_filter and prewhere filter. +/// If prewhere_info exists, only apply to the first prewhere filter. void MergeTreeRangeReader::executeDeletedRowMaskFilterColumns(ReadResult & result) { if (prewhere_info || !need_read_deleted_mask || !result.deleted_mask_filter_holder) @@ -1233,6 +1233,19 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r /// Columns might be projected out. We need to store them here so that default columns can be evaluated later. result.block_before_prewhere = block; + /// Apply deleted mask filter for the first prewhere step + if (!result.getFilter() && result.deleted_mask_filter_holder) + { + auto columns = block.getColumns(); + filterColumns(columns, result.deleted_mask_filter_holder); + if (columns.empty()) + block = block.cloneEmpty(); + else + block.setColumns(columns); + + result.setFilter(result.deleted_mask_filter_holder); + } + if (prewhere_info->actions) prewhere_info->actions->execute(block); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 21122a42833..e22e45da35f 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1405,7 +1405,7 @@ private: if (ctx->execute_ttl_type != ExecuteTTLType::NONE) ctx->files_to_skip.insert("ttl.txt"); - ctx->disk->createDirectories(ctx->new_part_tmp_path); + ctx->data_part_storage_builder->createDirectories(); /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. TransactionID tid = ctx->txn ? ctx->txn->tid : Tx::PrehistoricTID; @@ -1416,29 +1416,36 @@ private: NameSet hardlinked_files; /// Create hardlinks for unchanged files - for (auto it = ctx->disk->iterateDirectory(ctx->source_part->getFullRelativePath()); it->isValid(); it->next()) + for (auto it = ctx->source_part->data_part_storage->iterate(); it->isValid(); it->next()) { if (ctx->files_to_skip.contains(it->name())) continue; - String destination = ctx->new_part_tmp_path; - String file_name = it->name(); + String destination; + destination = it->name(); - destination += file_name; + /// Skip to create hardlink for deleted_row_mask.bin + if (ctx->source_part->hasLightweightDelete() && destination == "deleted_row_mask.bin") + continue; - if (!ctx->disk->isDirectory(it->path())) + if (it->isFile()) { - ctx->disk->createHardLink(it->path(), destination); - hardlinked_files.insert(file_name); + ctx->data_part_storage_builder->createHardLinkFrom( + *ctx->source_part->data_part_storage, it->name(), destination); + hardlinked_files.insert(it->name()); } - else if (!endsWith(".tmp_proj", file_name)) // ignore projection tmp merge dir + else if (!endsWith(".tmp_proj", it->name())) // ignore projection tmp merge dir { // it's a projection part directory - ctx->disk->createDirectories(destination); - for (auto p_it = ctx->disk->iterateDirectory(it->path()); p_it->isValid(); p_it->next()) + ctx->data_part_storage_builder->createProjection(destination); + + auto projection_data_part_storage = ctx->source_part->data_part_storage->getProjection(destination); + auto projection_data_part_storage_builder = ctx->data_part_storage_builder->getProjection(destination); + + for (auto p_it = projection_data_part_storage->iterate(); p_it->isValid(); p_it->next()) { - String p_destination = fs::path(destination) / p_it->name(); - ctx->disk->createHardLink(p_it->path(), p_destination); + projection_data_part_storage_builder->createHardLinkFrom( + *projection_data_part_storage, p_it->name(), p_it->name()); hardlinked_files.insert(p_it->name()); } } @@ -1509,7 +1516,7 @@ private: if (has_deleted_rows) { - ctx->new_data_part->writeLightWeightDeletedMask(new_bitmap); + ctx->new_data_part->writeLightweightDeletedMask(new_bitmap); } } @@ -1521,7 +1528,7 @@ private: ctx->mutating_pipeline.reset(); } - MutationHelpers::finalizeMutatedPart(ctx->source_part, ctx->new_data_part, ctx->execute_ttl_type, ctx->compression_codec, ctx->context); + MutationHelpers::finalizeMutatedPart(ctx->source_part, ctx->data_part_storage_builder, ctx->new_data_part, ctx->execute_ttl_type, ctx->compression_codec, ctx->context); } enum class State diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 6e9d9188962..acc9b08da29 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -378,6 +378,7 @@ CREATE TABLE system.mutations ( `database` String, `table` String, + `is_lightweight` UInt8, `mutation_id` String, `command` String, `create_time` DateTime, From 624848679060cc5ae250864d200335906089e0a0 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Tue, 28 Jun 2022 17:34:03 +0800 Subject: [PATCH 237/659] Fix lightweight delete bugs: add skip files and use source part's columns to avoid metadata updated cases --- src/Storages/MergeTree/MutateTask.cpp | 29 ++++++++++++++++++--------- 1 file changed, 20 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index e22e45da35f..e82becd3166 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1424,10 +1424,6 @@ private: String destination; destination = it->name(); - /// Skip to create hardlink for deleted_row_mask.bin - if (ctx->source_part->hasLightweightDelete() && destination == "deleted_row_mask.bin") - continue; - if (it->isFile()) { ctx->data_part_storage_builder->createHardLinkFrom( @@ -1695,12 +1691,22 @@ bool MutateTask::prepare() /// It shouldn't be changed by mutation. ctx->new_data_part->index_granularity_info = ctx->source_part->index_granularity_info; - auto [new_columns, new_infos] = MutationHelpers::getColumnsForNewDataPart( - ctx->source_part, ctx->updated_header, ctx->storage_columns, - ctx->source_part->getSerializationInfos(), ctx->commands_for_part); + if (ctx->is_lightweight_mutation) + { + /// The metadata alter will update the metadata snapshot, we should use same as source part. + ctx->new_data_part->setColumns(ctx->source_part->getColumns()); + ctx->new_data_part->setSerializationInfos(ctx->source_part->getSerializationInfos()); + } + else + { + auto [new_columns, new_infos] = MutationHelpers::getColumnsForNewDataPart( + ctx->source_part, ctx->updated_header, ctx->storage_columns, + ctx->source_part->getSerializationInfos(), ctx->commands_for_part); + + ctx->new_data_part->setColumns(new_columns); + ctx->new_data_part->setSerializationInfos(new_infos); + } - ctx->new_data_part->setColumns(new_columns); - ctx->new_data_part->setSerializationInfos(new_infos); ctx->new_data_part->partition.assign(ctx->source_part->partition); /// Don't change granularity type while mutating subset of columns @@ -1722,6 +1728,11 @@ bool MutateTask::prepare() } else if (ctx->is_lightweight_mutation) { + ctx->files_to_skip = ctx->source_part->getFileNamesWithoutChecksums(); + /// Skip to create hardlink for deleted_row_mask.bin + if (ctx->source_part->hasLightweightDelete()) + ctx->files_to_skip.insert("deleted_row_mask.bin"); + /// We will modify or create only deleted_row_mask for lightweight delete. Other columns and key values are copied as-is. task = std::make_unique(ctx); } From dcc7367ac4703905c9df59e310c5bb0d587b3e51 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Tue, 28 Jun 2022 18:27:56 +0800 Subject: [PATCH 238/659] Fix code style error --- src/Storages/MergeTree/MutateTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index e82becd3166..08d4e4b47c5 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1660,7 +1660,7 @@ bool MutateTask::prepare() /// Skip to apply deleted mask when reading for MutateSomePartColumns. need_mutate_all_columns = need_mutate_all_columns || (ctx->mutation_kind == MutationsInterpreter::MutationKind::MUTATE_OTHER && ctx->interpreter->isAffectingAllColumns()); - if(!need_mutate_all_columns && ctx->source_part->hasLightweightDelete() && !ctx->is_lightweight_mutation) + if (!need_mutate_all_columns && ctx->source_part->hasLightweightDelete() && !ctx->is_lightweight_mutation) ctx->interpreter->SetSkipDeletedMask(true); ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); From 7e433859eaa96e97f66cc653560b2b159d5c5c60 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Thu, 30 Jun 2022 13:51:48 +0800 Subject: [PATCH 239/659] Change deleted rows mask from String to Native UInt8 format --- src/Compression/CompressionFactory.cpp | 4 +- src/Compression/CompressionFactory.h | 2 +- src/Formats/NativeWriter.cpp | 2 +- src/Formats/NativeWriter.h | 2 + .../MergeTree/DataPartStorageOnDisk.cpp | 33 ++-- .../MergeTree/DataPartStorageOnDisk.h | 4 +- src/Storages/MergeTree/IDataPartStorage.h | 5 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 24 ++- src/Storages/MergeTree/IMergeTreeDataPart.h | 17 +- .../MergeTreeDataPartDeletedMask.cpp | 162 ++++++++++++++++++ .../MergeTree/MergeTreeDataPartDeletedMask.h | 33 ++++ .../MergeTree/MergeTreeRangeReader.cpp | 17 +- src/Storages/MergeTree/MergeTreeRangeReader.h | 1 - .../MergeTree/MergeTreeSequentialSource.cpp | 8 +- src/Storages/MergeTree/MutateTask.cpp | 22 ++- 15 files changed, 272 insertions(+), 64 deletions(-) create mode 100644 src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp create mode 100644 src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index b8a1c5877a4..94fb30af1bc 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -98,14 +98,14 @@ CompressionCodecPtr CompressionCodecFactory::get( } -CompressionCodecPtr CompressionCodecFactory::get(uint8_t byte_code) const +CompressionCodecPtr CompressionCodecFactory::get(uint8_t byte_code, const IDataType * column_type) const { const auto family_code_and_creator = family_code_with_codec.find(byte_code); if (family_code_and_creator == family_code_with_codec.end()) throw Exception("Unknown codec family code: " + toString(byte_code), ErrorCodes::UNKNOWN_CODEC); - return family_code_and_creator->second({}, nullptr); + return family_code_and_creator->second({}, column_type); } diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index a4451f9ed2e..c386784686e 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -63,7 +63,7 @@ public: } /// Get codec by method byte (no params available) - CompressionCodecPtr get(uint8_t byte_code) const; + CompressionCodecPtr get(uint8_t byte_code, const IDataType * column_type = nullptr) const; /// For backward compatibility with config settings CompressionCodecPtr get(const String & family_name, std::optional level) const; diff --git a/src/Formats/NativeWriter.cpp b/src/Formats/NativeWriter.cpp index 77692eec6b6..004c75182a7 100644 --- a/src/Formats/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -46,7 +46,7 @@ void NativeWriter::flush() } -static void writeData(const ISerialization & serialization, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit) +void NativeWriter::writeData(const ISerialization & serialization, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit) { /** If there are columns-constants - then we materialize them. * (Since the data type does not know how to serialize / deserialize constants.) diff --git a/src/Formats/NativeWriter.h b/src/Formats/NativeWriter.h index 010a03ec722..02fc53b60fe 100644 --- a/src/Formats/NativeWriter.h +++ b/src/Formats/NativeWriter.h @@ -32,6 +32,8 @@ public: static String getContentType() { return "application/octet-stream"; } + static void writeData(const ISerialization & serialization, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit); + private: WriteBuffer & ostr; UInt64 client_revision; diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 816d31f44e5..45e0132ddf7 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -370,7 +370,7 @@ void DataPartStorageOnDisk::clearDirectory( if (!is_projection) { request.emplace_back(fs::path(dir) / "txn_version.txt", true); - request.emplace_back(fs::path(dir) / "deleted_row_mask.bin", true); + request.emplace_back(fs::path(dir) / "deleted_rows_mask.bin", true); } disk->removeSharedFiles(request, !can_remove_shared_data, names_not_to_remove); @@ -652,27 +652,30 @@ bool DataPartStorageOnDisk::shallParticipateInMerges(const IStoragePolicy & stor return !volume_ptr->areMergesAvoided(); } -void DataPartStorageOnDisk::loadDeletedRowMask(String & bitmap) const +void DataPartStorageOnDisk::loadDeletedRowsMask(MergeTreeDataPartDeletedMask & deleted_mask) const { - String deleted_mask_path = fs::path(getRelativePath()) / "deleted_row_mask.bin"; + String deleted_mask_path = fs::path(getRelativePath()) / deleted_mask.name; auto disk = volume->getDisk(); - auto in = openForReading(disk, deleted_mask_path); - readString(bitmap, *in); + + if (disk->isFile(deleted_mask_path)) + { + auto read_buf = openForReading(disk, deleted_mask_path); + deleted_mask.read(*read_buf); + assertEOF(*read_buf); + } } -void DataPartStorageOnDisk::writeLightweightDeletedMask(String & bitmap, Poco::Logger * log) const +void DataPartStorageOnDisk::writeDeletedRowsMask(MergeTreeDataPartDeletedMask & deleted_mask) const { - String deleted_mask_path = fs::path(getRelativePath()) / "deleted_row_mask.bin"; - auto disk = volume->getDisk(); - try + const String final_path = fs::path(getRelativePath()) / deleted_mask.name; + const String tmp_path = final_path + ".tmp"; + { - auto out = volume->getDisk()->writeFile(deleted_mask_path); - DB::writeText(bitmap, *out); - } - catch (Poco::Exception & e) - { - LOG_ERROR(log, "{} (while writing deleted rows mask file for lightweight delete: {})", e.what(), backQuote(fullPath(disk, deleted_mask_path))); + auto out = volume->getDisk()->writeFile(tmp_path, 4096); + deleted_mask.write(*out); } + + volume->getDisk()->moveFile(tmp_path, final_path); } void DataPartStorageOnDisk::backup( diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index 75f11617aef..61a91714a3b 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -86,8 +86,8 @@ public: bool shallParticipateInMerges(const IStoragePolicy &) const override; - void loadDeletedRowMask(String & bitmap) const override; - void writeLightweightDeletedMask(String & bitmap, Poco::Logger * log) const override; + void loadDeletedRowsMask(MergeTreeDataPartDeletedMask & deleted_mask) const override; + void writeDeletedRowsMask(MergeTreeDataPartDeletedMask & deleted_mask) const override; void backup( TemporaryFilesOnDisks & temp_dirs, diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 16ff2485a6c..3af9a0ce499 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -169,8 +170,8 @@ public: /// A leak of abstraction virtual bool shallParticipateInMerges(const IStoragePolicy &) const { return true; } - virtual void loadDeletedRowMask(String & bitmap) const = 0; - virtual void writeLightweightDeletedMask(String & bitmap, Poco::Logger * log) const = 0; + virtual void loadDeletedRowsMask(MergeTreeDataPartDeletedMask & deleted_mask) const = 0; + virtual void writeDeletedRowsMask(MergeTreeDataPartDeletedMask & deleted_mask) const = 0; /// Create a backup of a data part. /// This method adds a new entry to backup_entries. diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 3ee96459185..507d815644c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -648,7 +648,7 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks checkConsistency(require_columns_checksums); loadDefaultCompressionCodec(); - loadDeletedRowMask(); + loadDeletedMask(); } catch (...) { @@ -1209,25 +1209,31 @@ void IMergeTreeDataPart::loadColumns(bool require) setSerializationInfos(infos); } -void IMergeTreeDataPart::loadDeletedRowMask() +void IMergeTreeDataPart::loadDeletedMask() { if (part_type == Type::Compact) return; - if (data_part_storage->exists(DELETED_ROW_MARK_FILE_NAME)) + if (data_part_storage->exists(deleted_mask.name)) { has_lightweight_delete = true; - data_part_storage->loadDeletedRowMask(deleted_rows_mask); + data_part_storage->loadDeletedRowsMask(deleted_mask); + + if (deleted_mask.getDeletedRows().size() != rows_count) + throw Exception(ErrorCodes::CORRUPTED_DATA, + "Size of deleted mask loaded from '{}':'{}' doesn't match expected " + "for part {}" + "(loaded {} rows, expected {} rows).", + data_part_storage->getDiskPath(), deleted_mask.name, name, deleted_mask.getDeletedRows().size(), rows_count); } } -void IMergeTreeDataPart::writeLightweightDeletedMask(String bitmap) const +void IMergeTreeDataPart::writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask) { - if (bitmap.empty()) - return; - - data_part_storage->writeLightweightDeletedMask(bitmap, storage.log); + deleted_mask.setDeletedRows(new_mask); + has_lightweight_delete = true; + data_part_storage->writeDeletedRowsMask(deleted_mask); } void IMergeTreeDataPart::assertHasVersionMetadata(MergeTreeTransaction * txn) const diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 975e7d6272d..d5f6322beab 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -327,9 +328,8 @@ public: mutable VersionMetadata version; - /// True if the part has deleted_row_mask.bin file used for lightweight delete. + /// True if the part has deleted_rows_mask.bin file used for lightweight delete. bool has_lightweight_delete = false; - String deleted_rows_mask; /// For data in RAM ('index') UInt64 getIndexSizeInBytes() const; @@ -408,9 +408,6 @@ public: static inline constexpr auto TXN_VERSION_METADATA_FILE_NAME = "txn_version.txt"; - /// File name for lightweight delete rows mask bitmap file. - static inline constexpr auto DELETED_ROW_MARK_FILE_NAME = "deleted_row_mask.bin"; - /// One of part files which is used to check how many references (I'd like /// to say hardlinks, but it will confuse even more) we have for the part /// for zero copy replication. Sadly it's very complex. @@ -466,14 +463,14 @@ public: /// True if here is light weight bitmap file in part. bool hasLightweightDelete() const { return has_lightweight_delete; } - /// Reads deleted row mask from deleted_row_mask.bin if exists and set has_lightweight_delete. - void loadDeletedRowMask(); - - /// Write lightweight deleted mask to a file. - void writeLightweightDeletedMask(String bitmap) const; + const MergeTreeDataPartDeletedMask& getDeletedMask() const { return deleted_mask; } + void writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask); + void loadDeletedMask(); protected: + MergeTreeDataPartDeletedMask deleted_mask {}; + /// Total size of all columns, calculated once in calcuateColumnSizesOnDisk ColumnSize total_columns_size; diff --git a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp new file mode 100644 index 00000000000..fe715f236a5 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp @@ -0,0 +1,162 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB::ErrorCodes +{ + extern const int UNKNOWN_FORMAT_VERSION; + extern const int CORRUPTED_DATA; +} + +namespace DB +{ + +namespace +{ +struct HashValue +{ +private: + char value[16]; + +public: + HashValue() = default; + HashValue(SipHash & hasher) + { + hasher.get128(value); + + static_assert(std::is_pod_v, "Expected to be a POD-type"); + static_assert(sizeof(HashValue) * 8 == 128); + } + + bool operator==(const HashValue & other) const + { + return memcmp(value, other.value, sizeof(value)) == 0; + } +}; + +constexpr UInt8 FORMAT_VERSION = 1; +constexpr UInt8 DEFAULT_CODEC = static_cast(CompressionMethodByte::T64); +constexpr UInt8 PADDING_SIZE = 7; // just in case +constexpr UInt8 HEADER_SIZE = 0 + + sizeof(FORMAT_VERSION) + + sizeof(UInt64) // number of rows in mask + + sizeof(HashValue) // column data hash + + PADDING_SIZE // padding: zero-bytes + + sizeof(HashValue); // header hash +} + +MergeTreeDataPartDeletedMask::MergeTreeDataPartDeletedMask() + : deleted_rows(ColumnUInt8::create()) +{} + +const ColumnUInt8 & MergeTreeDataPartDeletedMask::getDeletedRows() const +{ + return *deleted_rows; +} + +void MergeTreeDataPartDeletedMask::setDeletedRows(DeletedRows new_rows) +{ + deleted_rows.swap(new_rows); +} + +void MergeTreeDataPartDeletedMask::setDeletedRows(size_t rows, bool value) +{ + setDeletedRows(ColumnUInt8::create(rows, value)); +} + +void MergeTreeDataPartDeletedMask::read(ReadBuffer & in) +{ + std::array header_buffer_data; + in.readStrict(header_buffer_data.data(), header_buffer_data.size()); + {// validate hash of the header first + SipHash hash; + hash.update(header_buffer_data.data(), header_buffer_data.size()); + const HashValue computed_hash(hash); + + HashValue read_hash; + readPODBinary(read_hash, in); + if (read_hash != computed_hash) + throw Exception(ErrorCodes::CORRUPTED_DATA, + "Invalid deleted masks file header hash"); + } + + UInt8 format_version = FORMAT_VERSION; + UInt64 stored_rows = 0; + HashValue column_hash; + {// Read header values + ReadBuffer header(header_buffer_data.data(), header_buffer_data.size(), 0); + readBinary(format_version, header); + if (format_version != FORMAT_VERSION) + throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, + "Unknown deleted mask file format version {}", + static_cast(format_version)); + + readBinary(stored_rows, header); + readPODBinary(column_hash, header); + header.ignore(PADDING_SIZE); + assertEOF(header); + } + + auto data_read_buffer = std::make_unique(in); + + auto res_column = DeletedRows(ColumnUInt8::create()); + ColumnPtr res_col_ptr = res_column; + SerializationPtr serialization = DataTypeUInt8().getDefaultSerialization(); + NativeReader::readData(*serialization, res_col_ptr, *data_read_buffer, stored_rows, 0); + assertEOF(*data_read_buffer); + + // we probably don't want to check column hash here, since codec verifies data integrity. + deleted_rows = res_column; +} + +void MergeTreeDataPartDeletedMask::write(WriteBuffer & out) const +{ + {// Header + std::array header_buffer_data; + WriteBuffer header(header_buffer_data.data(), header_buffer_data.size()); + + writeBinary(FORMAT_VERSION, header); + writeBinary(static_cast(deleted_rows->size()), header); + + { + SipHash hash; + deleted_rows->updateHashFast(hash); + writePODBinary(HashValue(hash), header); + } + + { + const char padding[PADDING_SIZE] = {'\0'}; + writePODBinary(padding, header); + } + assert(header_buffer_data.max_size() == header.count()); + + writePODBinary(header_buffer_data, out); + {// header hash + SipHash hash; + hash.update(header_buffer_data.data(), header_buffer_data.size()); + writePODBinary(HashValue(hash), out); + } + } + assert(HEADER_SIZE == out.count()); + + const DataTypeUInt8 col_datatype; + auto codec = CompressionCodecFactory::instance().get(static_cast(DEFAULT_CODEC), &col_datatype); + auto data_write_buffer = std::make_unique(out, codec); + SerializationPtr serialization = col_datatype.getDefaultSerialization(); + + NativeWriter::writeData(*serialization, deleted_rows, *data_write_buffer, 0, deleted_rows->size()); + data_write_buffer->finalize(); +} + +} diff --git a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h new file mode 100644 index 00000000000..b75c9d37c55 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include +#include + +#include + +namespace DB +{ +class ReadBuffer; +class WriteBuffer; + +/// Per-part info about rows deleted by lightweight mutations. +struct MergeTreeDataPartDeletedMask +{ + explicit MergeTreeDataPartDeletedMask(); + using DeletedRows = ColumnUInt8::Ptr; + + std::string name = "deleted_rows_mask.bin"; + + const ColumnUInt8 & getDeletedRows() const; + void setDeletedRows(DeletedRows new_rows); + void setDeletedRows(size_t rows, bool value); + + void read(ReadBuffer & in); + void write(WriteBuffer & out) const; + +private: + ColumnUInt8::Ptr deleted_rows; +}; + +}; diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index d3ac822094a..47433fddadb 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -675,8 +675,6 @@ MergeTreeRangeReader::MergeTreeRangeReader( } need_read_deleted_mask = merge_tree_reader->needReadDeletedMask(); - if (need_read_deleted_mask) - deleted_rows_mask = merge_tree_reader->data_part->deleted_rows_mask; if (prewhere_info) { @@ -1002,12 +1000,15 @@ void MergeTreeRangeReader::fillDeletedRowMaskColumn(ReadResult & result, UInt64 UInt8 * pos = vec.data(); UInt8 * end = &vec[num_rows]; + const auto & deleted_rows_col = merge_tree_reader->data_part->getDeletedMask().getDeletedRows(); + const ColumnUInt8::Container & deleted_rows_mask = deleted_rows_col.getData(); + while (pos < end && leading_begin_part_offset < leading_end_part_offset) { - if (deleted_rows_mask[leading_begin_part_offset++] == '0') - *pos++ = 1; - else + if (deleted_rows_mask[leading_begin_part_offset++]) *pos++ = 0; + else + *pos++ = 1; } const auto start_ranges = result.startedRanges(); @@ -1019,10 +1020,10 @@ void MergeTreeRangeReader::fillDeletedRowMaskColumn(ReadResult & result, UInt64 while (pos < end && start_part_offset < end_part_offset) { - if (deleted_rows_mask[start_part_offset++] == '0') - *pos++ = 1; - else + if (deleted_rows_mask[start_part_offset++]) *pos++ = 0; + else + *pos++ = 1; } } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index bd38be47ed9..2515455bd83 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -270,7 +270,6 @@ private: bool is_initialized = false; Names non_const_virtual_column_names; bool need_read_deleted_mask = false; - String deleted_rows_mask; }; } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 926fde522d0..af31560cb86 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -76,10 +76,7 @@ try const auto & header = getPort().getHeader(); /// The chunk after deleted mask applied maybe empty. But the empty chunk means done of read rows. - String deleted_rows_mask; bool need_read_deleted_mask = data_part->hasLightweightDelete(); - if (need_read_deleted_mask) - deleted_rows_mask = data_part->deleted_rows_mask; do { @@ -99,13 +96,16 @@ try if (need_read_deleted_mask) { + const auto & deleted_rows_col = data_part->getDeletedMask().getDeletedRows(); + const ColumnUInt8::Container & deleted_rows_mask = deleted_rows_col.getData(); + size_t pos = current_row - rows_read; /// Get deleted mask for rows_read IColumn::Filter deleted_rows_filter(rows_read, true); for (size_t i = 0; i < rows_read; i++) { - if (deleted_rows_mask[pos++] == '1') + if (deleted_rows_mask[pos++]) deleted_rows_filter[i] = 0; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 08d4e4b47c5..a6c18147c88 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -604,7 +604,7 @@ void finalizeMutatedPart( new_data_part->minmax_idx = source_part->minmax_idx; new_data_part->modification_time = time(nullptr); new_data_part->loadProjections(false, false); - new_data_part->loadDeletedRowMask(); + new_data_part->loadDeletedMask(); new_data_part->setBytesOnDisk(new_data_part->data_part_storage->calculateTotalSizeOnDisk()); new_data_part->default_codec = codec; new_data_part->calculateColumnsAndSecondaryIndicesSizesOnDisk(); @@ -1484,14 +1484,20 @@ private: Block block; bool has_deleted_rows = false; + auto new_deleted_rows = ColumnUInt8::create(); + auto & data = new_deleted_rows->getData(); + /// If this part has already applied lightweight mutation, load the past latest bitmap to merge with current bitmap if (ctx->source_part->hasLightweightDelete()) { - new_bitmap = ctx->source_part->deleted_rows_mask; + const auto & deleted_rows_col = ctx->source_part->getDeletedMask().getDeletedRows(); + const auto & source_data = deleted_rows_col.getData(); + data.insert(source_data.begin(), source_data.begin() + ctx->source_part->rows_count); + has_deleted_rows = true; } else - new_bitmap.resize(ctx->source_part->rows_count, '0'); + new_deleted_rows->insertManyDefaults(ctx->source_part->rows_count); /// Mark the data corresponding to the offset in the as deleted. while (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor->pull(block)) @@ -1507,12 +1513,12 @@ private: /// Fill 1 for rows in offset for (size_t current_row = 0; current_row < block_rows; current_row++) - new_bitmap[offset[current_row]] = '1'; + data[offset[current_row]] = 1; } if (has_deleted_rows) { - ctx->new_data_part->writeLightweightDeletedMask(new_bitmap); + ctx->new_data_part->writeDeletedMask(ColumnUInt8::Ptr(std::move(new_deleted_rows))); } } @@ -1539,8 +1545,6 @@ private: State state{State::NEED_PREPARE}; MutationContextPtr ctx; - - String new_bitmap; }; @@ -1729,9 +1733,9 @@ bool MutateTask::prepare() else if (ctx->is_lightweight_mutation) { ctx->files_to_skip = ctx->source_part->getFileNamesWithoutChecksums(); - /// Skip to create hardlink for deleted_row_mask.bin + /// Skip to create hardlink for deleted_rows_mask.bin if (ctx->source_part->hasLightweightDelete()) - ctx->files_to_skip.insert("deleted_row_mask.bin"); + ctx->files_to_skip.insert("deleted_rows_mask.bin"); /// We will modify or create only deleted_row_mask for lightweight delete. Other columns and key values are copied as-is. task = std::make_unique(ctx); From 6e6f77ef8ac51684850251e734d4f99768ec974b Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Mon, 4 Jul 2022 08:59:53 +0800 Subject: [PATCH 240/659] Fix compile error in clang tidy build --- src/Interpreters/MutationsInterpreter.h | 2 +- src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index fb94c1f5f33..162039e3fd0 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -88,7 +88,7 @@ private: struct Stage; ASTPtr prepareInterpreterSelectQuery(std::vector &prepared_stages, bool dry_run); - ASTPtr prepareInterpreterSelectQueryLightweight(std::vector &prepared_stages, bool dry_run); + static ASTPtr prepareInterpreterSelectQueryLightweight(std::vector &prepared_stages, bool dry_run); QueryPipelineBuilder addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp index fe715f236a5..e93f3dd65ca 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp @@ -31,7 +31,7 @@ private: public: HashValue() = default; - HashValue(SipHash & hasher) + explicit HashValue(SipHash & hasher) { hasher.get128(value); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a6c18147c88..92a096a7890 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -626,7 +626,7 @@ struct MutationContext MergeTreeData::DataPartPtr source_part; StoragePtr storage_from_source_part; - bool is_lightweight_mutation{0}; + bool is_lightweight_mutation{false}; StorageMetadataPtr metadata_snapshot; From 780cdfb8f0dbdadd6b485267834aa6fd127dca24 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Wed, 6 Jul 2022 18:29:29 +0800 Subject: [PATCH 241/659] Code changes based on review opinions. Make ordinary single alter delete lightwight by default --- src/Core/Settings.h | 2 + src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Interpreters/InterpreterDeleteQuery.cpp | 16 ++++---- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/MutationsInterpreter.h | 4 +- src/Storages/IStorage.h | 4 +- .../MergeTree/DataPartStorageOnDisk.cpp | 2 +- .../MergeTree/DataPartStorageOnDisk.h | 2 +- src/Storages/MergeTree/IDataPartStorage.h | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 +-- src/Storages/MergeTree/IMergeTreeDataPart.h | 9 ++--- .../MergeTree/MergeTreeDataPartDeletedMask.h | 2 +- .../MergeTree/MergeTreeMutationEntry.cpp | 4 +- .../MergeTree/MergeTreeMutationEntry.h | 9 +++-- src/Storages/MergeTree/MutateTask.cpp | 12 ++++-- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- src/Storages/StorageJoin.cpp | 2 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 13 +++++-- src/Storages/StorageMergeTree.h | 4 +- ...lightweight_delete_on_merge_tree.reference | 8 ++++ ...02319_lightweight_delete_on_merge_tree.sql | 37 +++++++++++++++++++ 22 files changed, 102 insertions(+), 44 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bda72f089eb..7c559330e38 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -458,6 +458,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \ M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ + M(Bool, allow_experimental_lightweight_delete, false, "Enable lightweight DELETE mutations for mergetree tables. Work in progress", 0) \ + M(Bool, lightweight_delete_mutation, true, "Enable to make ordinary ALTER DELETE queries lightweight for mergetree tables", 0) \ M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 056a3d9f7b4..03b0e1d5894 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -144,7 +144,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) if (!mutation_commands.empty()) { table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); - MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate(); + MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false, false).validate(); table->mutate(mutation_commands, getContext()); } diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index fff66402cff..3a786997ae3 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int TABLE_IS_READ_ONLY; + extern const int SUPPORT_IS_DISABLED; } @@ -30,6 +31,9 @@ InterpreterDeleteQuery::InterpreterDeleteQuery(const ASTPtr & query_ptr_, Contex BlockIO InterpreterDeleteQuery::execute() { + if (!getContext()->getSettingsRef().allow_experimental_lightweight_delete) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Lightweight delete mutate is experimental. Set `allow_experimental_lightweight_delete` setting to enable it"); + FunctionNameNormalizer().visit(query_ptr.get()); const ASTDeleteQuery & delete_query = query_ptr->as(); auto table_id = getContext()->resolveStorageID(delete_query, Context::ResolveOrdinary); @@ -60,8 +64,7 @@ BlockIO InterpreterDeleteQuery::execute() auto table_lock = table->lockForShare(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = table->getInMemoryMetadataPtr(); - /// Currently do similar as alter table delete. - /// TODO: Mark this delete as lightweight. + /// Convert to MutationCommand MutationCommands mutation_commands; MutationCommand mut_command; @@ -76,12 +79,9 @@ BlockIO InterpreterDeleteQuery::execute() mutation_commands.emplace_back(mut_command); - if (!mutation_commands.empty()) - { - table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); - MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate(); - storage_merge_tree->mutate(mutation_commands, getContext(), MutationType::Lightweight); - } + table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); + MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false, false).validate(); + storage_merge_tree->mutate(mutation_commands, getContext(), MutationType::Lightweight); return {}; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 5852200da2c..ddef35e88cf 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1936,7 +1936,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc && !settings.empty_result_for_aggregation_by_empty_set && storage && storage->getName() != "MaterializedMySQL" - && !storage->hasLightweightDelete() + && !storage->hasLightweightDeletedMask() && !row_policy_filter && processing_stage == QueryProcessingStage::FetchColumns && query_analyzer->hasAggregation() diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 162039e3fd0..49165c6f9ad 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -44,7 +44,7 @@ public: MutationCommands commands_, ContextPtr context_, bool can_execute_, - bool is_lightweight_ = false); + bool is_lightweight_); void validate(); @@ -79,7 +79,7 @@ public: MutationKind::MutationKindEnum getMutationKind() const { return mutation_kind.mutation_kind; } - void SetSkipDeletedMask(bool skip) { skip_deleted_mask = skip; } + void setSkipDeletedMask(bool skip) { skip_deleted_mask = skip; } private: ASTPtr prepare(bool dry_run); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 899e7e365ce..3647941cc57 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -235,8 +235,8 @@ public: /// Returns true if the storage supports backup/restore for specific partitions. virtual bool supportsBackupPartition() const { return false; } - /// Return true if there are lightweight parts. - virtual bool hasLightweightDelete() const { return false; } + /// Return true if there are at least one part containing lightweight deleted mask. + virtual bool hasLightweightDeletedMask() const { return false; } private: diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 45e0132ddf7..1c4d8d9186a 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -665,7 +665,7 @@ void DataPartStorageOnDisk::loadDeletedRowsMask(MergeTreeDataPartDeletedMask & d } } -void DataPartStorageOnDisk::writeDeletedRowsMask(MergeTreeDataPartDeletedMask & deleted_mask) const +void DataPartStorageOnDisk::writeDeletedRowsMask(const MergeTreeDataPartDeletedMask & deleted_mask) const { const String final_path = fs::path(getRelativePath()) / deleted_mask.name; const String tmp_path = final_path + ".tmp"; diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index 61a91714a3b..2362d30a92d 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -87,7 +87,7 @@ public: bool shallParticipateInMerges(const IStoragePolicy &) const override; void loadDeletedRowsMask(MergeTreeDataPartDeletedMask & deleted_mask) const override; - void writeDeletedRowsMask(MergeTreeDataPartDeletedMask & deleted_mask) const override; + void writeDeletedRowsMask(const MergeTreeDataPartDeletedMask & deleted_mask) const override; void backup( TemporaryFilesOnDisks & temp_dirs, diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 3af9a0ce499..cfd29a550e1 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -171,7 +171,7 @@ public: virtual bool shallParticipateInMerges(const IStoragePolicy &) const { return true; } virtual void loadDeletedRowsMask(MergeTreeDataPartDeletedMask & deleted_mask) const = 0; - virtual void writeDeletedRowsMask(MergeTreeDataPartDeletedMask & deleted_mask) const = 0; + virtual void writeDeletedRowsMask(const MergeTreeDataPartDeletedMask & deleted_mask) const = 0; /// Create a backup of a data part. /// This method adds a new entry to backup_entries. diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 507d815644c..b7066ec0cd5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -791,6 +791,9 @@ NameSet IMergeTreeDataPart::getFileNamesWithoutChecksums() const if (data_part_storage->exists(TXN_VERSION_METADATA_FILE_NAME)) result.emplace(TXN_VERSION_METADATA_FILE_NAME); + if (data_part_storage->exists(DELETED_ROWS_MARK_FILE_NAME)) + result.emplace(DELETED_ROWS_MARK_FILE_NAME); + return result; } @@ -1216,8 +1219,6 @@ void IMergeTreeDataPart::loadDeletedMask() if (data_part_storage->exists(deleted_mask.name)) { - has_lightweight_delete = true; - data_part_storage->loadDeletedRowsMask(deleted_mask); if (deleted_mask.getDeletedRows().size() != rows_count) @@ -1232,7 +1233,6 @@ void IMergeTreeDataPart::loadDeletedMask() void IMergeTreeDataPart::writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask) { deleted_mask.setDeletedRows(new_mask); - has_lightweight_delete = true; data_part_storage->writeDeletedRowsMask(deleted_mask); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index d5f6322beab..54f8af6bb5d 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -328,9 +328,6 @@ public: mutable VersionMetadata version; - /// True if the part has deleted_rows_mask.bin file used for lightweight delete. - bool has_lightweight_delete = false; - /// For data in RAM ('index') UInt64 getIndexSizeInBytes() const; UInt64 getIndexSizeInAllocatedBytes() const; @@ -408,6 +405,8 @@ public: static inline constexpr auto TXN_VERSION_METADATA_FILE_NAME = "txn_version.txt"; + static inline constexpr auto DELETED_ROWS_MARK_FILE_NAME = "deleted_rows_mask.bin"; + /// One of part files which is used to check how many references (I'd like /// to say hardlinks, but it will confuse even more) we have for the part /// for zero copy replication. Sadly it's very complex. @@ -460,8 +459,8 @@ public: /// Check metadata in cache is consistent with actual metadata on disk(if use_metadata_cache is true) std::unordered_map checkMetadata() const; - /// True if here is light weight bitmap file in part. - bool hasLightweightDelete() const { return has_lightweight_delete; } + /// True if here is lightweight deleted mask file in part. + bool hasLightweightDelete() const { return deleted_mask.getDeletedRows().size() > 0; } const MergeTreeDataPartDeletedMask& getDeletedMask() const { return deleted_mask; } void writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask); diff --git a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h index b75c9d37c55..2ecdd8e7b28 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h +++ b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h @@ -17,7 +17,7 @@ struct MergeTreeDataPartDeletedMask explicit MergeTreeDataPartDeletedMask(); using DeletedRows = ColumnUInt8::Ptr; - std::string name = "deleted_rows_mask.bin"; + const std::string name = "deleted_rows_mask.bin"; const ColumnUInt8 & getDeletedRows() const; void setDeletedRows(DeletedRows new_rows); diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 5b103cbe8d8..180d78f6ee3 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -47,14 +47,14 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, const TransactionID & tid_, const WriteSettings & settings, MutationType type_) - : type(type_) - , create_time(time(nullptr)) + : create_time(time(nullptr)) , commands(std::move(commands_)) , disk(std::move(disk_)) , path_prefix(path_prefix_) , file_name("tmp_mutation_" + toString(tmp_number) + ".txt") , is_temp(true) , tid(tid_) + , type(type_) { try { diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 06cbd44ed49..37dbca9de7b 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -11,15 +11,15 @@ namespace DB { class IBackupEntry; +/// Type of Mutate. Used to control different mutates during mutates +/// assignment. Also allows to apply special logic during mutate process +/// Stored in FutureMergedMutatedPart and MergeTreeMutationEntry. enum class MutationType { Ordinary, Lightweight }; /// A mutation entry for non-replicated MergeTree storage engines. /// Stores information about mutation in file mutation_*.txt. struct MergeTreeMutationEntry { - /// Type of mutation, used for lightweight delete. - MutationType type; - time_t create_time = 0; MutationCommands commands; @@ -41,6 +41,9 @@ struct MergeTreeMutationEntry /// or UnknownCSN if it's not committed (yet) or RolledBackCSN if it's rolled back or PrehistoricCSN if there is no transaction. CSN csn = Tx::UnknownCSN; + /// Type of mutation, used for lightweight delete. + MutationType type; + /// Create a new entry and write it to a temporary file. MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number, const TransactionID & tid_, const WriteSettings & settings, MutationType type_); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 92a096a7890..a3a6c88f7d6 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -432,6 +432,10 @@ NameSet collectFilesToSkip( { NameSet files_to_skip = source_part->getFileNamesWithoutChecksums(); + /// Remove deleted rows mask file name to create hard link for it when mutate some columns. + if (files_to_skip.contains(IMergeTreeDataPart::DELETED_ROWS_MARK_FILE_NAME)) + files_to_skip.erase(IMergeTreeDataPart::DELETED_ROWS_MARK_FILE_NAME); + /// Skip updated files for (const auto & entry : updated_header) { @@ -1355,6 +1359,9 @@ private: std::unique_ptr part_merger_writer_task{nullptr}; }; +/// LightweightDeleteTask works for lightweight delete mutate. +/// The MutationsInterpreter returns a simple select like "select _part_offset where predicates". +/// The prepare() and execute() has special logics for LWD mutate. class LightweightDeleteTask : public IExecutableTask { public: @@ -1665,7 +1672,7 @@ bool MutateTask::prepare() /// Skip to apply deleted mask when reading for MutateSomePartColumns. need_mutate_all_columns = need_mutate_all_columns || (ctx->mutation_kind == MutationsInterpreter::MutationKind::MUTATE_OTHER && ctx->interpreter->isAffectingAllColumns()); if (!need_mutate_all_columns && ctx->source_part->hasLightweightDelete() && !ctx->is_lightweight_mutation) - ctx->interpreter->SetSkipDeletedMask(true); + ctx->interpreter->setSkipDeletedMask(true); ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); ctx->progress_callback = MergeProgressCallback((*ctx->mutate_entry)->ptr(), ctx->watch_prev_elapsed, *ctx->stage_progress); @@ -1733,9 +1740,6 @@ bool MutateTask::prepare() else if (ctx->is_lightweight_mutation) { ctx->files_to_skip = ctx->source_part->getFileNamesWithoutChecksums(); - /// Skip to create hardlink for deleted_rows_mask.bin - if (ctx->source_part->hasLightweightDelete()) - ctx->files_to_skip.insert("deleted_rows_mask.bin"); /// We will modify or create only deleted_row_mask for lightweight delete. Other columns and key values are copied as-is. task = std::make_unique(ctx); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 312c4146cb9..cd31d356b4b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1920,7 +1920,7 @@ std::vector ReplicatedMergeTreeQueue::getMutationsStatu formatAST(*command.ast, buf, false, true); result.push_back(MergeTreeMutationStatus { - MutationType::Ordinary, + MutationType::Ordinary, /// TODO: ReplicatedMergeTree supports lightweight delete. entry.znode_name, buf.str(), entry.create_time, diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 5e161fc2e6a..0604bb304d0 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -123,7 +123,7 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) // New scope controls lifetime of pipeline. { auto storage_ptr = DatabaseCatalog::instance().getTable(getStorageID(), context); - auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context, true); + auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context, true, false); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index f3f1162287f..fc4a671c071 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -318,7 +318,7 @@ void StorageMemory::mutate(const MutationCommands & commands, ContextPtr context new_context->setSetting("max_streams_to_max_threads_ratio", 1); new_context->setSetting("max_threads", 1); - auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, new_context, true); + auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, new_context, true, false); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 5480114359e..5a178340382 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -332,7 +332,7 @@ void StorageMergeTree::alter( DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata); if (!maybe_mutation_commands.empty()) - mutation_version = startMutation(maybe_mutation_commands, local_context); + mutation_version = startMutation(maybe_mutation_commands, local_context, MutationType::Ordinary); } /// Always execute required mutations synchronously, because alters @@ -555,7 +555,12 @@ void StorageMergeTree::setMutationCSN(const String & mutation_id, CSN csn) void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr query_context) { - mutate(commands, query_context, MutationType::Ordinary); + /// Make ordinary ALTER DELETE queries lightweight to check all tests. + if (query_context->getSettingsRef().lightweight_delete_mutation + && commands.size() == 1 && commands.begin()->type == MutationCommand::DELETE) + mutate(commands, query_context, MutationType::Lightweight); + else + mutate(commands, query_context, MutationType::Ordinary); } void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr query_context, MutationType type) @@ -569,7 +574,7 @@ void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr quer waitForMutation(version); } -bool StorageMergeTree::hasLightweightDelete() const +bool StorageMergeTree::hasLightweightDeletedMask() const { return has_lightweight_delete_parts.load(std::memory_order_relaxed); } @@ -1065,7 +1070,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( fake_query_context->makeQueryContext(); fake_query_context->setCurrentQueryId(""); MutationsInterpreter interpreter( - shared_from_this(), metadata_snapshot, commands_for_size_validation, fake_query_context, false); + shared_from_this(), metadata_snapshot, commands_for_size_validation, fake_query_context, false, false); commands_size += interpreter.evaluateCommandsSize(); } catch (...) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 6251d0d75e0..379e15d5bdf 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -89,7 +89,7 @@ public: /// Support lightweight delete. void mutate(const MutationCommands & commands, ContextPtr context, MutationType type); - bool hasLightweightDelete() const override; + bool hasLightweightDeletedMask() const override; /// Return introspection information about currently processing or recently processed mutations. std::vector getMutationsStatus() const override; @@ -184,7 +184,7 @@ private: /// Allocate block number for new mutation, write mutation to disk /// and into in-memory structures. Wake up merge-mutation task. - Int64 startMutation(const MutationCommands & commands, ContextPtr query_context, MutationType type = MutationType::Ordinary); + Int64 startMutation(const MutationCommands & commands, ContextPtr query_context, MutationType type); /// Wait until mutation with version will finish mutation for all parts void waitForMutation(Int64 version); void waitForMutation(const String & mutation_id) override; diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference index aefc0c88762..bc30d677348 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference @@ -1,7 +1,13 @@ 99 +1 95 +1 0 +1 -----lightweight mutation type----- +1 +1 +1 1 DELETE WHERE (c % 5) = 1 1 1 DELETE WHERE c = 4 1 0 MATERIALIZE INDEX i_c 1 @@ -27,5 +33,7 @@ t_light 2 2_3_3_1_10 2 t_light 3 3_4_4_1_10 2 t_light 4 4_5_5_1_10 1 -----Test lightweight delete in multi blocks----- +1 +1 1000 -2 1005 -2 diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql index da302f0f4ff..3c3df06915f 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql @@ -5,19 +5,32 @@ CREATE TABLE merge_table_standard_delete(id Int32, name String) ENGINE = MergeTr INSERT INTO merge_table_standard_delete select number, toString(number) from numbers(100); SET mutations_sync = 1; +SET allow_experimental_lightweight_delete = 1; DELETE FROM merge_table_standard_delete WHERE id = 10; SELECT COUNT() FROM merge_table_standard_delete; +DETACH TABLE merge_table_standard_delete; +ATTACH TABLE merge_table_standard_delete; +CHECK TABLE merge_table_standard_delete; + DELETE FROM merge_table_standard_delete WHERE name IN ('1','2','3','4'); SELECT COUNT() FROM merge_table_standard_delete; +DETACH TABLE merge_table_standard_delete; +ATTACH TABLE merge_table_standard_delete; +CHECK TABLE merge_table_standard_delete; + DELETE FROM merge_table_standard_delete WHERE 1; SELECT COUNT() FROM merge_table_standard_delete; +DETACH TABLE merge_table_standard_delete; +ATTACH TABLE merge_table_standard_delete; +CHECK TABLE merge_table_standard_delete; + DROP TABLE merge_table_standard_delete; drop table if exists t_light; @@ -27,11 +40,25 @@ INSERT INTO t_light SELECT number, number, number FROM numbers(10); SELECT '-----lightweight mutation type-----'; DELETE FROM t_light WHERE c%5=1; + +DETACH TABLE t_light; +ATTACH TABLE t_light; +CHECK TABLE t_light; + DELETE FROM t_light WHERE c=4; + +DETACH TABLE t_light; +ATTACH TABLE t_light; +CHECK TABLE t_light; + alter table t_light MATERIALIZE INDEX i_c; alter table t_light update b=-1 where a<3; alter table t_light drop index i_c; +DETACH TABLE t_light; +ATTACH TABLE t_light; +CHECK TABLE t_light; + SELECT is_lightweight, command, is_done FROM system.mutations WHERE database = currentDatabase() AND table = 't_light'; SELECT '-----Check that select and merge with lightweight delete.-----'; @@ -52,8 +79,18 @@ CREATE TABLE t_large(a UInt32, b int) ENGINE=MergeTree order BY a settings min_b INSERT INTO t_large SELECT number + 1, number + 1 FROM numbers(100000); DELETE FROM t_large WHERE a = 50000; + +DETACH TABLE t_large; +ATTACH TABLE t_large; +CHECK TABLE t_large; + ALTER TABLE t_large UPDATE b = -2 WHERE a between 1000 and 1005; ALTER TABLE t_large DELETE WHERE a=1; + +DETACH TABLE t_large; +ATTACH TABLE t_large; +CHECK TABLE t_large; + SELECT * FROM t_large WHERE a in (1,1000,1005,50000) order by a; DROP TABLE t_large; From 198674e99483639be3f88c78503a4e3808478d74 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Thu, 7 Jul 2022 11:08:31 +0800 Subject: [PATCH 242/659] Consider lightweight delete as ordinary for compact part in selectPartsToMutate() --- src/Storages/StorageMergeTree.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 5a178340382..ba192f39e2b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1042,7 +1042,8 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( break; /// Do not combine mutations with different types. - if (it->second.type != first_mutation_type) + /// TODO: compact part support lightweight delete. + if (isWidePart(part) && it->second.type != first_mutation_type) break; size_t commands_size = 0; @@ -1129,7 +1130,11 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( future_part->part_info = new_part_info; future_part->name = part->getNewName(new_part_info); future_part->type = part->getType(); - future_part->mutation_type = first_mutation_type; + + if (isWidePart(part)) + future_part->mutation_type = first_mutation_type; + else + future_part->mutation_type = MutationType::Ordinary; tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); return std::make_shared(future_part, std::move(tagger), commands, txn); From 9d1dff515cd2016953d3c1f564c78d4b39086ffe Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Fri, 8 Jul 2022 11:22:25 +0800 Subject: [PATCH 243/659] code changes for strange name and adding exception for mutation entry --- .../MergeTreeDataPartDeletedMask.cpp | 30 +++++++++---------- .../MergeTree/MergeTreeMutationEntry.cpp | 6 +++- 2 files changed, 20 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp index e93f3dd65ca..47e284fced9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp @@ -24,22 +24,22 @@ namespace DB namespace { -struct HashValue +struct DeletedRowsHash { private: char value[16]; public: - HashValue() = default; - explicit HashValue(SipHash & hasher) + DeletedRowsHash() = default; + explicit DeletedRowsHash(SipHash & hasher) { hasher.get128(value); - static_assert(std::is_pod_v, "Expected to be a POD-type"); - static_assert(sizeof(HashValue) * 8 == 128); + static_assert(std::is_pod_v, "Expected to be a POD-type"); + static_assert(sizeof(DeletedRowsHash) * 8 == 128); } - bool operator==(const HashValue & other) const + bool operator==(const DeletedRowsHash & other) const { return memcmp(value, other.value, sizeof(value)) == 0; } @@ -51,9 +51,9 @@ constexpr UInt8 PADDING_SIZE = 7; // just in case constexpr UInt8 HEADER_SIZE = 0 + sizeof(FORMAT_VERSION) + sizeof(UInt64) // number of rows in mask - + sizeof(HashValue) // column data hash + + sizeof(DeletedRowsHash) // column data hash + PADDING_SIZE // padding: zero-bytes - + sizeof(HashValue); // header hash + + sizeof(DeletedRowsHash); // header hash } MergeTreeDataPartDeletedMask::MergeTreeDataPartDeletedMask() @@ -77,14 +77,14 @@ void MergeTreeDataPartDeletedMask::setDeletedRows(size_t rows, bool value) void MergeTreeDataPartDeletedMask::read(ReadBuffer & in) { - std::array header_buffer_data; + std::array header_buffer_data; in.readStrict(header_buffer_data.data(), header_buffer_data.size()); {// validate hash of the header first SipHash hash; hash.update(header_buffer_data.data(), header_buffer_data.size()); - const HashValue computed_hash(hash); + const DeletedRowsHash computed_hash(hash); - HashValue read_hash; + DeletedRowsHash read_hash; readPODBinary(read_hash, in); if (read_hash != computed_hash) throw Exception(ErrorCodes::CORRUPTED_DATA, @@ -93,7 +93,7 @@ void MergeTreeDataPartDeletedMask::read(ReadBuffer & in) UInt8 format_version = FORMAT_VERSION; UInt64 stored_rows = 0; - HashValue column_hash; + DeletedRowsHash column_hash; {// Read header values ReadBuffer header(header_buffer_data.data(), header_buffer_data.size(), 0); readBinary(format_version, header); @@ -123,7 +123,7 @@ void MergeTreeDataPartDeletedMask::read(ReadBuffer & in) void MergeTreeDataPartDeletedMask::write(WriteBuffer & out) const { {// Header - std::array header_buffer_data; + std::array header_buffer_data; WriteBuffer header(header_buffer_data.data(), header_buffer_data.size()); writeBinary(FORMAT_VERSION, header); @@ -132,7 +132,7 @@ void MergeTreeDataPartDeletedMask::write(WriteBuffer & out) const { SipHash hash; deleted_rows->updateHashFast(hash); - writePODBinary(HashValue(hash), header); + writePODBinary(DeletedRowsHash(hash), header); } { @@ -145,7 +145,7 @@ void MergeTreeDataPartDeletedMask::write(WriteBuffer & out) const {// header hash SipHash hash; hash.update(header_buffer_data.data(), header_buffer_data.size()); - writePODBinary(HashValue(hash), out); + writePODBinary(DeletedRowsHash(hash), out); } } assert(HEADER_SIZE == out.count()); diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 180d78f6ee3..08f45e85d23 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -126,7 +126,9 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat int format_version; *buf >> "format version: " >> format_version >> "\n"; - assert(format_version <= 2); + /// Allow format_version = 1 for backward compatibility. + if (format_version != 1 && format_version != 2) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported format version in mutation.txt, expected '1' or '2', got '{}'", format_version); type = MutationType::Ordinary; if (format_version == 2) @@ -137,6 +139,8 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat auto type_value = magic_enum::enum_cast(type_str); if (type_value.has_value()) type = type_value.value(); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported mutation type in mutation.txt, expected 'Lightweight' or 'Ordinary', got '{}'", type_str); } LocalDateTime create_time_dt; From 4268ac8b0e4a8ba6362bc258e7ef1715b0fb53e5 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Fri, 8 Jul 2022 16:12:59 +0800 Subject: [PATCH 244/659] Add supportLightweightDeleteMutate() in IMergeTreeDataPart to disable LWD for part with projections --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 ++++++ src/Storages/MergeTree/IMergeTreeDataPart.h | 3 +++ src/Storages/MergeTree/MutateTask.cpp | 7 ++++--- src/Storages/StorageMergeTree.cpp | 12 +++++------- 4 files changed, 18 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index b7066ec0cd5..2c613122224 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1212,6 +1212,12 @@ void IMergeTreeDataPart::loadColumns(bool require) setSerializationInfos(infos); } +/// Project part / part with project parts / compact part doesn't support LWD. +bool IMergeTreeDataPart::supportLightweightDeleteMutate() const +{ + return part_type == MergeTreeDataPartType::Wide && parent_part == nullptr && projection_parts.size() == 0; +} + void IMergeTreeDataPart::loadDeletedMask() { if (part_type == Type::Compact) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 54f8af6bb5d..07eb910d2ba 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -459,6 +459,9 @@ public: /// Check metadata in cache is consistent with actual metadata on disk(if use_metadata_cache is true) std::unordered_map checkMetadata() const; + /// True if the part supports lightweight delete mutate. + bool supportLightweightDeleteMutate() const; + /// True if here is lightweight deleted mask file in part. bool hasLightweightDelete() const { return deleted_mask.getDeletedRows().size() > 0; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a3a6c88f7d6..b3cbe9e9a29 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1586,6 +1586,9 @@ MutateTask::MutateTask( ctx->source_part = ctx->future_part->parts[0]; ctx->storage_from_source_part = std::make_shared(ctx->source_part); + /// part is checked for lightweight delete in selectPartsToMutate(). + ctx->is_lightweight_mutation = ctx->future_part->mutation_type == MutationType::Lightweight; + auto storage_snapshot = ctx->storage_from_source_part->getStorageSnapshot(ctx->metadata_snapshot, context_); extendObjectColumns(ctx->storage_columns, storage_snapshot->object_columns, /*with_subcolumns=*/ false); } @@ -1640,9 +1643,7 @@ bool MutateTask::prepare() command.partition, context_for_reading)) ctx->commands_for_part.emplace_back(command); } - /// Enable lightweight delete for wide part only. - if (isWidePart(ctx->source_part) && (ctx->future_part->mutation_type == MutationType::Lightweight)) - ctx->is_lightweight_mutation = true; + if (ctx->source_part->isStoredOnDisk() && !ctx->is_lightweight_mutation && !isStorageTouchedByMutations( ctx->storage_from_source_part, ctx->metadata_snapshot, ctx->commands_for_part, Context::createCopy(context_for_reading))) { diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ba192f39e2b..bf731f09428 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1034,7 +1034,9 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( auto commands = std::make_shared(); size_t current_ast_elements = 0; auto last_mutation_to_apply = mutations_end_it; - MutationType first_mutation_type = mutations_begin_it->second.type; + + bool support_lightweight_mutate = part->supportLightweightDeleteMutate(); + MutationType first_mutation_type = support_lightweight_mutate ? mutations_begin_it->second.type : MutationType::Ordinary; for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { /// Do not squash mutations from different transactions to be able to commit/rollback them independently. @@ -1043,7 +1045,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( /// Do not combine mutations with different types. /// TODO: compact part support lightweight delete. - if (isWidePart(part) && it->second.type != first_mutation_type) + if (support_lightweight_mutate && it->second.type != first_mutation_type) break; size_t commands_size = 0; @@ -1130,11 +1132,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( future_part->part_info = new_part_info; future_part->name = part->getNewName(new_part_info); future_part->type = part->getType(); - - if (isWidePart(part)) - future_part->mutation_type = first_mutation_type; - else - future_part->mutation_type = MutationType::Ordinary; + future_part->mutation_type = first_mutation_type; tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); return std::make_shared(future_part, std::move(tagger), commands, txn); From d37152a5d622f09f2027f01c657d7509d0575966 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Mon, 11 Jul 2022 16:22:00 +0800 Subject: [PATCH 245/659] Remove loadDeletedMask() and get deleted mask when needed --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 12 +++++++----- src/Storages/MergeTree/IMergeTreeDataPart.h | 8 +++----- .../MergeTree/MergeTreeDataPartDeletedMask.h | 1 + src/Storages/MergeTree/MergeTreeRangeReader.cpp | 5 +++-- src/Storages/MergeTree/MergeTreeRangeReader.h | 1 + src/Storages/MergeTree/MergeTreeSequentialSource.cpp | 6 ++++-- src/Storages/MergeTree/MutateTask.cpp | 5 ++--- 7 files changed, 21 insertions(+), 17 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 2c613122224..c3919d29ee9 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -648,7 +648,6 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks checkConsistency(require_columns_checksums); loadDefaultCompressionCodec(); - loadDeletedMask(); } catch (...) { @@ -1215,14 +1214,14 @@ void IMergeTreeDataPart::loadColumns(bool require) /// Project part / part with project parts / compact part doesn't support LWD. bool IMergeTreeDataPart::supportLightweightDeleteMutate() const { - return part_type == MergeTreeDataPartType::Wide && parent_part == nullptr && projection_parts.size() == 0; + return part_type == MergeTreeDataPartType::Wide && parent_part == nullptr && projection_parts.empty(); } -void IMergeTreeDataPart::loadDeletedMask() +const MergeTreeDataPartDeletedMask::DeletedRows IMergeTreeDataPart::getDeletedMask() const { - if (part_type == Type::Compact) - return; + MergeTreeDataPartDeletedMask deleted_mask {}; + /// Check if deleted mask file exists. if (data_part_storage->exists(deleted_mask.name)) { data_part_storage->loadDeletedRowsMask(deleted_mask); @@ -1234,10 +1233,13 @@ void IMergeTreeDataPart::loadDeletedMask() "(loaded {} rows, expected {} rows).", data_part_storage->getDiskPath(), deleted_mask.name, name, deleted_mask.getDeletedRows().size(), rows_count); } + + return std::move(deleted_mask.getDeletedRowsPtr()); } void IMergeTreeDataPart::writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask) { + MergeTreeDataPartDeletedMask deleted_mask {}; deleted_mask.setDeletedRows(new_mask); data_part_storage->writeDeletedRowsMask(deleted_mask); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 07eb910d2ba..1f08d42f016 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -463,16 +463,14 @@ public: bool supportLightweightDeleteMutate() const; /// True if here is lightweight deleted mask file in part. - bool hasLightweightDelete() const { return deleted_mask.getDeletedRows().size() > 0; } + bool hasLightweightDelete() const { return data_part_storage->exists(DELETED_ROWS_MARK_FILE_NAME); } - const MergeTreeDataPartDeletedMask& getDeletedMask() const { return deleted_mask; } + /// Read lightweight deleted mask when needed. + const MergeTreeDataPartDeletedMask::DeletedRows getDeletedMask() const; void writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask); - void loadDeletedMask(); protected: - MergeTreeDataPartDeletedMask deleted_mask {}; - /// Total size of all columns, calculated once in calcuateColumnSizesOnDisk ColumnSize total_columns_size; diff --git a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h index 2ecdd8e7b28..4bd2cf77bab 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h +++ b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h @@ -20,6 +20,7 @@ struct MergeTreeDataPartDeletedMask const std::string name = "deleted_rows_mask.bin"; const ColumnUInt8 & getDeletedRows() const; + const DeletedRows & getDeletedRowsPtr() const { return deleted_rows; } void setDeletedRows(DeletedRows new_rows); void setDeletedRows(size_t rows, bool value); diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 47433fddadb..a2ae9978d15 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -675,6 +675,8 @@ MergeTreeRangeReader::MergeTreeRangeReader( } need_read_deleted_mask = merge_tree_reader->needReadDeletedMask(); + if (need_read_deleted_mask) + deleted_rows = merge_tree_reader->data_part->getDeletedMask(); if (prewhere_info) { @@ -1000,8 +1002,7 @@ void MergeTreeRangeReader::fillDeletedRowMaskColumn(ReadResult & result, UInt64 UInt8 * pos = vec.data(); UInt8 * end = &vec[num_rows]; - const auto & deleted_rows_col = merge_tree_reader->data_part->getDeletedMask().getDeletedRows(); - const ColumnUInt8::Container & deleted_rows_mask = deleted_rows_col.getData(); + const ColumnUInt8::Container & deleted_rows_mask = deleted_rows->getData(); while (pos < end && leading_begin_part_offset < leading_end_part_offset) { diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 2515455bd83..d57a94820c4 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -270,6 +270,7 @@ private: bool is_initialized = false; Names non_const_virtual_column_names; bool need_read_deleted_mask = false; + ColumnUInt8::Ptr deleted_rows; }; } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index af31560cb86..0deded95850 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -77,6 +77,9 @@ try /// The chunk after deleted mask applied maybe empty. But the empty chunk means done of read rows. bool need_read_deleted_mask = data_part->hasLightweightDelete(); + ColumnUInt8::Ptr deleted_rows_col; + if (need_read_deleted_mask) + deleted_rows_col = data_part->getDeletedMask(); do { @@ -96,8 +99,7 @@ try if (need_read_deleted_mask) { - const auto & deleted_rows_col = data_part->getDeletedMask().getDeletedRows(); - const ColumnUInt8::Container & deleted_rows_mask = deleted_rows_col.getData(); + const ColumnUInt8::Container & deleted_rows_mask = deleted_rows_col->getData(); size_t pos = current_row - rows_read; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b3cbe9e9a29..9e254361293 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -608,7 +608,6 @@ void finalizeMutatedPart( new_data_part->minmax_idx = source_part->minmax_idx; new_data_part->modification_time = time(nullptr); new_data_part->loadProjections(false, false); - new_data_part->loadDeletedMask(); new_data_part->setBytesOnDisk(new_data_part->data_part_storage->calculateTotalSizeOnDisk()); new_data_part->default_codec = codec; new_data_part->calculateColumnsAndSecondaryIndicesSizesOnDisk(); @@ -1497,8 +1496,8 @@ private: /// If this part has already applied lightweight mutation, load the past latest bitmap to merge with current bitmap if (ctx->source_part->hasLightweightDelete()) { - const auto & deleted_rows_col = ctx->source_part->getDeletedMask().getDeletedRows(); - const auto & source_data = deleted_rows_col.getData(); + const auto & deleted_rows_col = ctx->source_part->getDeletedMask(); + const auto & source_data = deleted_rows_col->getData(); data.insert(source_data.begin(), source_data.begin() + ctx->source_part->rows_count); has_deleted_rows = true; From 8b1114fa609e648895a43c05a28f79c28668bdba Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Tue, 12 Jul 2022 09:38:07 +0800 Subject: [PATCH 246/659] fix clang tidy compile error --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index c3919d29ee9..8f8ea1af025 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1217,7 +1217,7 @@ bool IMergeTreeDataPart::supportLightweightDeleteMutate() const return part_type == MergeTreeDataPartType::Wide && parent_part == nullptr && projection_parts.empty(); } -const MergeTreeDataPartDeletedMask::DeletedRows IMergeTreeDataPart::getDeletedMask() const +MergeTreeDataPartDeletedMask::DeletedRows IMergeTreeDataPart::getDeletedMask() const { MergeTreeDataPartDeletedMask deleted_mask {}; @@ -1237,7 +1237,7 @@ const MergeTreeDataPartDeletedMask::DeletedRows IMergeTreeDataPart::getDeletedMa return std::move(deleted_mask.getDeletedRowsPtr()); } -void IMergeTreeDataPart::writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask) +void IMergeTreeDataPart::writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask) const { MergeTreeDataPartDeletedMask deleted_mask {}; deleted_mask.setDeletedRows(new_mask); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 1f08d42f016..cafdb658216 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -466,8 +466,8 @@ public: bool hasLightweightDelete() const { return data_part_storage->exists(DELETED_ROWS_MARK_FILE_NAME); } /// Read lightweight deleted mask when needed. - const MergeTreeDataPartDeletedMask::DeletedRows getDeletedMask() const; - void writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask); + MergeTreeDataPartDeletedMask::DeletedRows getDeletedMask() const; + void writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask) const; protected: From 0793add9b5604f39fb15904d198759af2ec209f2 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Tue, 12 Jul 2022 15:04:00 +0800 Subject: [PATCH 247/659] Fix clang tidy compile error, remove std::move for const getDeletedRowsPtr() --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 8f8ea1af025..4c69bbb5c8c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1234,7 +1234,7 @@ MergeTreeDataPartDeletedMask::DeletedRows IMergeTreeDataPart::getDeletedMask() c data_part_storage->getDiskPath(), deleted_mask.name, name, deleted_mask.getDeletedRows().size(), rows_count); } - return std::move(deleted_mask.getDeletedRowsPtr()); + return deleted_mask.getDeletedRowsPtr(); } void IMergeTreeDataPart::writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask) const From 5a8d6ba0fc954deedb0d25e98749a928b8608f8c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 15 Jul 2022 09:06:25 +0200 Subject: [PATCH 248/659] clang tidy feedback addressed: updated src/IO/ForkWriteBuffer.cpp to use const reference --- src/IO/ForkWriteBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ForkWriteBuffer.cpp b/src/IO/ForkWriteBuffer.cpp index ef39c9241db..b055ae588a9 100644 --- a/src/IO/ForkWriteBuffer.cpp +++ b/src/IO/ForkWriteBuffer.cpp @@ -36,7 +36,7 @@ void ForkWriteBuffer::nextImpl() try { - for (auto write_buffer :sources | std::views::reverse) + for (const WriteBufferPtr & write_buffer : sources | std::views::reverse) { if (write_buffer.get() != first_buffer) { From 397a4ac888287756cbd55307103cdb62d574095d Mon Sep 17 00:00:00 2001 From: Wangyang Guo Date: Fri, 15 Jul 2022 15:01:16 +0800 Subject: [PATCH 249/659] TargetSpecific: add AVX512VBMI support --- src/Common/TargetSpecific.cpp | 3 +++ src/Common/TargetSpecific.h | 28 ++++++++++++++++++++++++++-- 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/src/Common/TargetSpecific.cpp b/src/Common/TargetSpecific.cpp index c52c8c2bcf0..70b03833775 100644 --- a/src/Common/TargetSpecific.cpp +++ b/src/Common/TargetSpecific.cpp @@ -18,6 +18,8 @@ UInt32 getSupportedArchs() result |= static_cast(TargetArch::AVX512F); if (Cpu::CpuFlagsCache::have_AVX512BW) result |= static_cast(TargetArch::AVX512BW); + if (Cpu::CpuFlagsCache::have_AVX512VBMI) + result |= static_cast(TargetArch::AVX512VBMI); return result; } @@ -37,6 +39,7 @@ String toString(TargetArch arch) case TargetArch::AVX2: return "avx2"; case TargetArch::AVX512F: return "avx512f"; case TargetArch::AVX512BW: return "avx512bw"; + case TargetArch::AVX512VBMI: return "avx512vbmi"; } __builtin_unreachable(); diff --git a/src/Common/TargetSpecific.h b/src/Common/TargetSpecific.h index b045892d2c1..9d0e0d9eede 100644 --- a/src/Common/TargetSpecific.h +++ b/src/Common/TargetSpecific.h @@ -81,6 +81,7 @@ enum class TargetArch : UInt32 AVX2 = (1 << 2), AVX512F = (1 << 3), AVX512BW = (1 << 4), + AVX512VBMI = (1 << 5), }; /// Runtime detection. @@ -95,6 +96,7 @@ String toString(TargetArch arch); #if defined(__clang__) +#define AVX512VBMI_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,avx512vl,avx512vbmi"))) #define AVX512BW_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw"))) #define AVX512_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f"))) #define AVX2_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2"))) @@ -102,6 +104,8 @@ String toString(TargetArch arch); #define SSE42_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt"))) #define DEFAULT_FUNCTION_SPECIFIC_ATTRIBUTE +# define BEGIN_AVX512VBMI_SPECIFIC_CODE \ + _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,avx512vl,avx512vbmi\"))),apply_to=function)") # define BEGIN_AVX512BW_SPECIFIC_CODE \ _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw\"))),apply_to=function)") # define BEGIN_AVX512F_SPECIFIC_CODE \ @@ -121,13 +125,17 @@ String toString(TargetArch arch); # define DUMMY_FUNCTION_DEFINITION [[maybe_unused]] void _dummy_function_definition(); #else +#define AVX512VBMI_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,avx512vl,avx512vbmi,tune=native"))) #define AVX512BW_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,tune=native"))) #define AVX512_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,tune=native"))) #define AVX2_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,tune=native"))) #define AVX_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,tune=native"))) -#define SSE42_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt",tune=native)))) +#define SSE42_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt",tune=native))) #define DEFAULT_FUNCTION_SPECIFIC_ATTRIBUTE +# define BEGIN_AVX512VBMI_SPECIFIC_CODE \ + _Pragma("GCC push_options") \ + _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,avx512vl,avx512vbmi,tune=native\")") # define BEGIN_AVX512BW_SPECIFIC_CODE \ _Pragma("GCC push_options") \ _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,tune=native\")") @@ -196,6 +204,15 @@ namespace TargetSpecific::AVX512BW { \ } \ END_TARGET_SPECIFIC_CODE +#define DECLARE_AVX512VBMI_SPECIFIC_CODE(...) \ +BEGIN_AVX512VBMI_SPECIFIC_CODE \ +namespace TargetSpecific::AVX512VBMI { \ + DUMMY_FUNCTION_DEFINITION \ + using namespace DB::TargetSpecific::AVX512VBMI; \ + __VA_ARGS__ \ +} \ +END_TARGET_SPECIFIC_CODE + #else #define USE_MULTITARGET_CODE 0 @@ -207,6 +224,7 @@ END_TARGET_SPECIFIC_CODE #define DECLARE_AVX2_SPECIFIC_CODE(...) #define DECLARE_AVX512F_SPECIFIC_CODE(...) #define DECLARE_AVX512BW_SPECIFIC_CODE(...) +#define DECLARE_AVX512VBMI_SPECIFIC_CODE(...) #endif @@ -223,7 +241,8 @@ DECLARE_SSE42_SPECIFIC_CODE (__VA_ARGS__) \ DECLARE_AVX_SPECIFIC_CODE (__VA_ARGS__) \ DECLARE_AVX2_SPECIFIC_CODE (__VA_ARGS__) \ DECLARE_AVX512F_SPECIFIC_CODE(__VA_ARGS__) \ -DECLARE_AVX512BW_SPECIFIC_CODE(__VA_ARGS__) +DECLARE_AVX512BW_SPECIFIC_CODE(__VA_ARGS__) \ +DECLARE_AVX512VBMI_SPECIFIC_CODE(__VA_ARGS__) DECLARE_DEFAULT_CODE( constexpr auto BuildArch = TargetArch::Default; /// NOLINT @@ -249,6 +268,11 @@ DECLARE_AVX512BW_SPECIFIC_CODE( constexpr auto BuildArch = TargetArch::AVX512BW; /// NOLINT ) // DECLARE_AVX512BW_SPECIFIC_CODE +DECLARE_AVX512VBMI_SPECIFIC_CODE( + constexpr auto BuildArch = TargetArch::AVX512VBMI; /// NOLINT +) // DECLARE_AVX512VBMI_SPECIFIC_CODE + + /** Runtime Dispatch helpers for class members. * * Example of usage: From ca42f649da7bf5ce2fa57523d23994a7220677c6 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Fri, 15 Jul 2022 15:31:10 +0800 Subject: [PATCH 250/659] Rewrite logic for loading deleted mask related to getDeletedMask() --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 10 ++++++---- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- .../MergeTree/MergeTreeDataPartDeletedMask.cpp | 2 +- .../MergeTree/MergeTreeDataPartDeletedMask.h | 2 +- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 12 ++++++------ src/Storages/MergeTree/MergeTreeRangeReader.h | 5 +++-- .../MergeTree/MergeTreeSequentialSource.cpp | 13 ++++++------- .../MergeTree/MergeTreeSequentialSource.h | 4 ++++ src/Storages/MergeTree/MutateTask.cpp | 15 ++++++++++----- 9 files changed, 38 insertions(+), 27 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 4c69bbb5c8c..b8fd70193f1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1217,12 +1217,12 @@ bool IMergeTreeDataPart::supportLightweightDeleteMutate() const return part_type == MergeTreeDataPartType::Wide && parent_part == nullptr && projection_parts.empty(); } -MergeTreeDataPartDeletedMask::DeletedRows IMergeTreeDataPart::getDeletedMask() const +bool IMergeTreeDataPart::getDeletedMask(MergeTreeDataPartDeletedMask & deleted_mask) const { - MergeTreeDataPartDeletedMask deleted_mask {}; + bool found = false; /// Check if deleted mask file exists. - if (data_part_storage->exists(deleted_mask.name)) + if (data_part_storage->exists(String(deleted_mask.name))) { data_part_storage->loadDeletedRowsMask(deleted_mask); @@ -1232,9 +1232,11 @@ MergeTreeDataPartDeletedMask::DeletedRows IMergeTreeDataPart::getDeletedMask() c "for part {}" "(loaded {} rows, expected {} rows).", data_part_storage->getDiskPath(), deleted_mask.name, name, deleted_mask.getDeletedRows().size(), rows_count); + + found = true; } - return deleted_mask.getDeletedRowsPtr(); + return found; } void IMergeTreeDataPart::writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask) const diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index cafdb658216..7869ca52969 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -466,7 +466,7 @@ public: bool hasLightweightDelete() const { return data_part_storage->exists(DELETED_ROWS_MARK_FILE_NAME); } /// Read lightweight deleted mask when needed. - MergeTreeDataPartDeletedMask::DeletedRows getDeletedMask() const; + bool getDeletedMask(MergeTreeDataPartDeletedMask & deleted_mask) const; void writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask) const; protected: diff --git a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp index 47e284fced9..d1a78623278 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp @@ -117,7 +117,7 @@ void MergeTreeDataPartDeletedMask::read(ReadBuffer & in) assertEOF(*data_read_buffer); // we probably don't want to check column hash here, since codec verifies data integrity. - deleted_rows = res_column; + deleted_rows = std::move(res_column); } void MergeTreeDataPartDeletedMask::write(WriteBuffer & out) const diff --git a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h index 4bd2cf77bab..c8652746d98 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h +++ b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h @@ -17,7 +17,7 @@ struct MergeTreeDataPartDeletedMask explicit MergeTreeDataPartDeletedMask(); using DeletedRows = ColumnUInt8::Ptr; - const std::string name = "deleted_rows_mask.bin"; + static constexpr std::string_view name = "deleted_rows_mask.bin"; const ColumnUInt8 & getDeletedRows() const; const DeletedRows & getDeletedRowsPtr() const { return deleted_rows; } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index a2ae9978d15..2f34d75b2c4 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -674,9 +674,8 @@ MergeTreeRangeReader::MergeTreeRangeReader( sample_block.insert(ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), column_name)); } - need_read_deleted_mask = merge_tree_reader->needReadDeletedMask(); - if (need_read_deleted_mask) - deleted_rows = merge_tree_reader->data_part->getDeletedMask(); + if (merge_tree_reader->needReadDeletedMask()) + need_apply_deleted_mask = merge_tree_reader->data_part->getDeletedMask(deleted_mask); if (prewhere_info) { @@ -959,7 +958,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t } /// Do similar as part_offset for deleted mask. - if (need_read_deleted_mask) + if (need_apply_deleted_mask) fillDeletedRowMaskColumn(result, leading_begin_part_offset, leading_end_part_offset); return result; @@ -1002,7 +1001,8 @@ void MergeTreeRangeReader::fillDeletedRowMaskColumn(ReadResult & result, UInt64 UInt8 * pos = vec.data(); UInt8 * end = &vec[num_rows]; - const ColumnUInt8::Container & deleted_rows_mask = deleted_rows->getData(); + const auto & deleted_rows_col = deleted_mask.getDeletedRows(); + const ColumnUInt8::Container & deleted_rows_mask = deleted_rows_col.getData(); while (pos < end && leading_begin_part_offset < leading_end_part_offset) { @@ -1151,7 +1151,7 @@ static ColumnPtr combineFilters(ColumnPtr first, ColumnPtr second) /// If prewhere_info exists, only apply to the first prewhere filter. void MergeTreeRangeReader::executeDeletedRowMaskFilterColumns(ReadResult & result) { - if (prewhere_info || !need_read_deleted_mask || !result.deleted_mask_filter_holder) + if (prewhere_info || !need_apply_deleted_mask || !result.deleted_mask_filter_holder) return; const ColumnUInt8 * mask_filter = typeid_cast(result.deleted_mask_filter_holder.get()); diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index d57a94820c4..0e6ace18be9 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -269,8 +270,8 @@ private: bool last_reader_in_chain = false; bool is_initialized = false; Names non_const_virtual_column_names; - bool need_read_deleted_mask = false; - ColumnUInt8::Ptr deleted_rows; + bool need_apply_deleted_mask = false; + MergeTreeDataPartDeletedMask deleted_mask; }; } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 0deded95850..47c45058088 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -68,6 +68,9 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( reader = data_part->getReader(columns_for_reader, storage_snapshot->metadata, MarkRanges{MarkRange(0, data_part->getMarksCount())}, /* uncompressed_cache = */ nullptr, mark_cache.get(), reader_settings, {}, {}); + + if (data_part->hasLightweightDelete()) + need_apply_deleted_mask = data_part->getDeletedMask(deleted_mask); } Chunk MergeTreeSequentialSource::generate() @@ -76,11 +79,6 @@ try const auto & header = getPort().getHeader(); /// The chunk after deleted mask applied maybe empty. But the empty chunk means done of read rows. - bool need_read_deleted_mask = data_part->hasLightweightDelete(); - ColumnUInt8::Ptr deleted_rows_col; - if (need_read_deleted_mask) - deleted_rows_col = data_part->getDeletedMask(); - do { if (!isCancelled() && current_row < data_part->rows_count) @@ -97,9 +95,10 @@ try current_row += rows_read; current_mark += (rows_to_read == rows_read); - if (need_read_deleted_mask) + if (need_apply_deleted_mask) { - const ColumnUInt8::Container & deleted_rows_mask = deleted_rows_col->getData(); + const auto & deleted_rows_col = deleted_mask.getDeletedRows(); + const ColumnUInt8::Container & deleted_rows_mask = deleted_rows_col.getData(); size_t pos = current_row - rows_read; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index a3e4f5fa856..5a571825db5 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -58,6 +59,9 @@ private: /// current row at which we stop reading size_t current_row = 0; + bool need_apply_deleted_mask = false; + MergeTreeDataPartDeletedMask deleted_mask {}; + /// Closes readers and unlock part locks void finish(); }; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 9e254361293..30abd546c49 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1496,13 +1496,18 @@ private: /// If this part has already applied lightweight mutation, load the past latest bitmap to merge with current bitmap if (ctx->source_part->hasLightweightDelete()) { - const auto & deleted_rows_col = ctx->source_part->getDeletedMask(); - const auto & source_data = deleted_rows_col->getData(); - data.insert(source_data.begin(), source_data.begin() + ctx->source_part->rows_count); + MergeTreeDataPartDeletedMask deleted_mask {}; + if (ctx->source_part->getDeletedMask(deleted_mask)) + { + const auto & deleted_rows_col = deleted_mask.getDeletedRows(); + const auto & source_data = deleted_rows_col.getData(); + data.insert(source_data.begin(), source_data.begin() + ctx->source_part->rows_count); - has_deleted_rows = true; + has_deleted_rows = true; + } } - else + + if (!has_deleted_rows) new_deleted_rows->insertManyDefaults(ctx->source_part->rows_count); /// Mark the data corresponding to the offset in the as deleted. From 197d3796eca37a734cc82007967d974a4dd10e99 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Fri, 15 Jul 2022 15:37:36 +0800 Subject: [PATCH 251/659] Killed mutation has empty commands may cause SegV --- src/Storages/MergeTree/MutateTask.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 30abd546c49..50c37ba5b08 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -644,7 +644,7 @@ struct MutationContext QueryPipelineBuilder mutating_pipeline_builder; QueryPipeline mutating_pipeline; // in - std::unique_ptr mutating_executor; + std::unique_ptr mutating_executor{nullptr}; ProgressCallback progress_callback; Block updated_header; @@ -1511,7 +1511,7 @@ private: new_deleted_rows->insertManyDefaults(ctx->source_part->rows_count); /// Mark the data corresponding to the offset in the as deleted. - while (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor->pull(block)) + while (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor && ctx->mutating_executor->pull(block)) { size_t block_rows = block.rows(); @@ -1593,6 +1593,10 @@ MutateTask::MutateTask( /// part is checked for lightweight delete in selectPartsToMutate(). ctx->is_lightweight_mutation = ctx->future_part->mutation_type == MutationType::Lightweight; + /// Empty mutation commands mean that the mutation is killed. Just work as ordinary, clone the part. + if (ctx->commands->empty()) + ctx->is_lightweight_mutation = false; + auto storage_snapshot = ctx->storage_from_source_part->getStorageSnapshot(ctx->metadata_snapshot, context_); extendObjectColumns(ctx->storage_columns, storage_snapshot->object_columns, /*with_subcolumns=*/ false); } From c9e15017ac6df237bc376f62dabb0a35a088dc92 Mon Sep 17 00:00:00 2001 From: Wangyang Guo Date: Fri, 15 Jul 2022 16:00:09 +0800 Subject: [PATCH 252/659] lz4 decompress: dynamic dispatch with TargetSpecific --- src/Compression/LZ4_decompress_faster.cpp | 39 +++++++++++------------ 1 file changed, 18 insertions(+), 21 deletions(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 84471836cfc..33726811848 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include @@ -16,7 +16,7 @@ #include #endif -#if defined(__AVX512VBMI__) +#if USE_MULTITARGET_CODE #include #endif @@ -408,9 +408,9 @@ inline void copyOverlap32(UInt8 * op, const UInt8 *& match, const size_t offset) match += shift4[offset]; } +DECLARE_AVX512VBMI_SPECIFIC_CODE( inline void copyOverlap32Shuffle(UInt8 * op, const UInt8 *& match, const size_t offset) { -#if defined(__AVX512VBMI__) && !defined(MEMORY_SANITIZER) static constexpr UInt8 __attribute__((__aligned__(32))) masks[] = { 0, 1, 2, 2, 4, 2, 2, 4, 8, 5, 2, 10, 8, 6, 4, 2, 16, 15, 14, 13, 12, 11, 10, 9, 8, 7, 6, 5, 4, 3, 2, 1, /* offset=0, shift amount index. */ @@ -447,28 +447,25 @@ inline void copyOverlap32Shuffle(UInt8 * op, const UInt8 *& match, const size_t 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 0, }; - if (DB::Cpu::CpuFlagsCache::have_AVX512VBMI) - { - _mm256_storeu_si256(reinterpret_cast<__m256i *>(op), - _mm256_permutexvar_epi8( - _mm256_load_si256(reinterpret_cast(masks) + offset), - _mm256_loadu_si256(reinterpret_cast(match)))); - match += masks[offset]; - } - else - { - copyOverlap32(op, match, offset); - } -#else - copyOverlap32(op, match, offset); -#endif + _mm256_storeu_si256(reinterpret_cast<__m256i *>(op), + _mm256_permutexvar_epi8( + _mm256_load_si256(reinterpret_cast(masks) + offset), + _mm256_loadu_si256(reinterpret_cast(match)))); + match += masks[offset]; } +) /// DECLARE_AVX512VBMI_SPECIFIC_CODE template <> void inline copy<32>(UInt8 * dst, const UInt8 * src) { copy32(dst, src); } template <> void inline wildCopy<32>(UInt8 * dst, const UInt8 * src, UInt8 * dst_end) { wildCopy32(dst, src, dst_end); } template <> void inline copyOverlap<32, false>(UInt8 * op, const UInt8 *& match, const size_t offset) { copyOverlap32(op, match, offset); } -template <> void inline copyOverlap<32, true>(UInt8 * op, const UInt8 *& match, const size_t offset) { copyOverlap32Shuffle(op, match, offset); } +template <> void inline copyOverlap<32, true>(UInt8 * op, const UInt8 *& match, const size_t offset) { +#if USE_MULTITARGET_CODE + TargetSpecific::AVX512VBMI::copyOverlap32Shuffle(op, match, offset); +#else + copyOverlap32(op, match, offset); +#endif +} /// See also https://stackoverflow.com/a/30669632 @@ -641,9 +638,9 @@ bool decompress( if (dest_size >= 32768) { size_t variant_size = 4; -#if defined(__AVX512VBMI__) && !defined(MEMORY_SANITIZER) +#if USE_MULTITARGET_CODE && !defined(MEMORY_SANITIZER) /// best_variant == 4 only valid when AVX512VBMI available - if (DB::Cpu::CpuFlagsCache::have_AVX512VBMI) + if (isArchSupported(DB::TargetArch::AVX512VBMI)) variant_size = 5; #endif size_t best_variant = statistics.select(variant_size); From 1f85358625614c7329d5dafbb53000505214f672 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 15 Jul 2022 08:25:24 +0000 Subject: [PATCH 253/659] Add X86 prefix to x86 performance tests reduces ambiguity with the recently introduced ARM performance tests --- .github/workflows/master.yml | 16 +++++++-------- .github/workflows/pull_request.yml | 32 +++++++++++++++--------------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 66ba8547894..b76ff39f0c8 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -2902,7 +2902,7 @@ jobs: ############################################################################################# #################################### PERFORMANCE TESTS ###################################### ############################################################################################# - PerformanceComparison0: + PerformanceComparisonX86-0: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -2940,7 +2940,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparison1: + PerformanceComparisonX86-1: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -2978,7 +2978,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparison2: + PerformanceComparisonX86-2: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -3016,7 +3016,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparison3: + PerformanceComparisonX86-3: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -3096,10 +3096,10 @@ jobs: - IntegrationTestsTsan1 - IntegrationTestsTsan2 - IntegrationTestsTsan3 - - PerformanceComparison0 - - PerformanceComparison1 - - PerformanceComparison2 - - PerformanceComparison3 + - PerformanceComparisonX86-0 + - PerformanceComparisonX86-1 + - PerformanceComparisonX86-2 + - PerformanceComparisonX86-3 - CompatibilityCheck - ASTFuzzerTestDebug - ASTFuzzerTestAsan diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 9cd8fd6f49d..05fd337e57c 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -3118,7 +3118,7 @@ jobs: ############################################################################################# #################################### PERFORMANCE TESTS ###################################### ############################################################################################# - PerformanceComparison0: + PerformanceComparisonX86-0: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -3156,7 +3156,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparison1: + PerformanceComparisonX86-1: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -3194,7 +3194,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparison2: + PerformanceComparisonX86-2: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -3232,7 +3232,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparison3: + PerformanceComparisonX86-3: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: @@ -3270,7 +3270,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch0: + PerformanceComparisonAarch-0: needs: [BuilderDebAarch64] runs-on: [self-hosted, func-tester-aarch64] steps: @@ -3308,7 +3308,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch1: + PerformanceComparisonAarch-1: needs: [BuilderDebAarch64] runs-on: [self-hosted, func-tester-aarch64] steps: @@ -3346,7 +3346,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch2: + PerformanceComparisonAarch-2: needs: [BuilderDebAarch64] runs-on: [self-hosted, func-tester-aarch64] steps: @@ -3384,7 +3384,7 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch3: + PerformanceComparisonAarch-3: needs: [BuilderDebAarch64] runs-on: [self-hosted, func-tester-aarch64] steps: @@ -3481,14 +3481,14 @@ jobs: - IntegrationTestsTsan1 - IntegrationTestsTsan2 - IntegrationTestsTsan3 - - PerformanceComparison0 - - PerformanceComparison1 - - PerformanceComparison2 - - PerformanceComparison3 - - PerformanceComparisonAarch0 - - PerformanceComparisonAarch1 - - PerformanceComparisonAarch2 - - PerformanceComparisonAarch3 + - PerformanceComparisonX86-0 + - PerformanceComparisonX86-1 + - PerformanceComparisonX86-2 + - PerformanceComparisonX86-3 + - PerformanceComparisonAarch-0 + - PerformanceComparisonAarch-1 + - PerformanceComparisonAarch-2 + - PerformanceComparisonAarch-3 - UnitTestsAsan - UnitTestsTsan - UnitTestsMsan From 1276bfdffd807fc58bb7dbaf296d3ca8e519c044 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 15 Jul 2022 08:53:58 +0000 Subject: [PATCH 254/659] Enable -Wsign-conversion --- cmake/warnings.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 0e55f6d32c3..e9a41dda01d 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -35,7 +35,6 @@ if (COMPILER_CLANG) no_warning(missing-prototypes) no_warning(missing-variable-declarations) no_warning(padded) - no_warning(sign-conversion) no_warning(switch-enum) no_warning(undefined-func-template) no_warning(unused-template) From 429aab2f12ca3994965e6c30ad687283c4a5900d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 15 Jul 2022 11:55:13 +0200 Subject: [PATCH 255/659] Fix making a query scope for async backups. --- src/Backups/BackupsWorker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index bf90d58d009..ba9a5b6ca71 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -107,7 +107,7 @@ UUID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & c { if (async) { - query_scope.emplace(context_in_use); + query_scope.emplace(mutable_context); setThreadName("BackupWorker"); } From c133f27724819ba93dd6c5d31b6a899ccd3df6bd Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 15 Jul 2022 09:56:56 +0000 Subject: [PATCH 256/659] style again --- tests/queries/0_stateless/02356_insert_query_log_metrics.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql index bb37c8b4eb1..dabb898093e 100644 --- a/tests/queries/0_stateless/02356_insert_query_log_metrics.sql +++ b/tests/queries/0_stateless/02356_insert_query_log_metrics.sql @@ -2,4 +2,4 @@ CREATE TABLE 02356_destination (a Int64, b String) ENGINE = Memory; INSERT INTO 02356_destination (a, b) SELECT * FROM generateRandom('a Int64, b String') LIMIT 100 SETTINGS max_threads=1, max_block_size=100; SYSTEM FLUSH LOGS; -SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' and has(databases, currentDatabase()) FORMAT CSV; \ No newline at end of file +SELECT read_rows = written_rows, read_rows = result_rows, read_bytes = written_bytes, read_bytes = result_bytes FROM system.query_log where normalized_query_hash = 1214411238725380014 and type='QueryFinish' and current_database = currentDatabase() FORMAT CSV; \ No newline at end of file From 77487e996c4be698cd09630f287067fd923edb59 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 15 Jul 2022 12:23:28 +0200 Subject: [PATCH 257/659] fix stacktraces in gdb in bc check --- docker/test/stress/run.sh | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index cb80c8ebdc9..085c0c045b7 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -322,7 +322,11 @@ else clickhouse-client --query="SELECT 'Server version: ', version()" # Install new package before running stress test because we should use new clickhouse-client and new clickhouse-test + # But we should leave old binary in /usr/bin/ for gdb (so it will print sane stacktarces) + mv /usr/bin/clickhouse previous_release_package_folder/ install_packages package_folder + mv /usr/bin/clickhouse package_folder/ + mv previous_release_package_folder/clickhouse /usr/bin/ mkdir tmp_stress_output @@ -337,6 +341,7 @@ else mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.stress.log # Start new server + mv package_folder/clickhouse /usr/bin/ configure start 500 clickhouse-client --query "SELECT 'Backward compatibility check: Server successfully started', 'OK'" >> /test_output/test_results.tsv \ From 8d07a1427f35c3e8182e6b56dd378c20efbbb7dc Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 12:28:56 +0200 Subject: [PATCH 258/659] Update storage_conf.xml --- .../test_concurrent_backups_s3/configs/storage_conf.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml b/tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml index ef55f3a62d7..9124d583f8c 100644 --- a/tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml +++ b/tests/integration/test_concurrent_backups_s3/configs/storage_conf.xml @@ -29,7 +29,6 @@ hdd - /backups/ From c4cbefb5211c471c9fb664ff8dd10b9f066676ff Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 15 Jul 2022 13:44:37 +0300 Subject: [PATCH 259/659] Update test.py --- tests/integration/test_s3_zero_copy_replication/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index 39be0d564df..7b7fb9d21ad 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -150,6 +150,7 @@ def test_s3_zero_copy_replication(cluster, policy): node2.query("DROP TABLE IF EXISTS s3_test NO DELAY") +@pytest.mark.skip(reason="Test is flaky (and never was stable)") def test_s3_zero_copy_on_hybrid_storage(cluster): node1 = cluster.instances["node1"] node2 = cluster.instances["node2"] From a7fe1cf5f0a5c90ebc3df9aed646866afbf29f67 Mon Sep 17 00:00:00 2001 From: rnbondarenko Date: Fri, 15 Jul 2022 13:46:52 +0300 Subject: [PATCH 260/659] removed default queue arguments declare queue without default arguments x-max-length and x-overflow. --- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 73f0c8bd44e..f831f81cd22 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -578,16 +578,6 @@ void StorageRabbitMQ::bindQueue(size_t queue_id, AMQP::TcpChannel & rabbit_chann } } - /// Impose default settings if there are no user-defined settings. - if (!queue_settings.contains("x-max-length")) - { - queue_settings["x-max-length"] = queue_size; - } - if (!queue_settings.contains("x-overflow")) - { - queue_settings["x-overflow"] = "reject-publish"; - } - /// If queue_base - a single name, then it can be used as one specific queue, from which to read. /// Otherwise it is used as a generator (unique for current table) of queue names, because it allows to /// maximize performance - via setting `rabbitmq_num_queues`. From 74fc53e5c22b9d5e40dc2c63a32d3ada41b07cef Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 15 Jul 2022 10:58:09 +0000 Subject: [PATCH 261/659] Enable -Wc99-extension --- cmake/warnings.cmake | 1 - src/Parsers/ExpressionListParsers.h | 9 +++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index e9a41dda01d..994f14c6149 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -23,7 +23,6 @@ if (COMPILER_CLANG) no_warning(zero-length-array) no_warning(c++98-compat-pedantic) no_warning(c++98-compat) - no_warning(c99-extensions) no_warning(conversion) no_warning(ctad-maybe-unsupported) # clang 9+, linux-only no_warning(disabled-macro-expansion) diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 2b127dc2607..05c7ec946ee 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -9,6 +9,11 @@ #include #include +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wc99-extensions" +#endif + namespace DB { @@ -564,3 +569,7 @@ protected: }; } + +#ifdef __clang__ +#pragma clang diagnostic pop +#endif From deda29b46b597d87c584521e32f96537b4241712 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 15 Jul 2022 11:15:46 +0000 Subject: [PATCH 262/659] Pass const StringRef by value, not by reference See #39224 --- src/AggregateFunctions/ThetaSketchData.h | 2 +- src/Columns/ColumnUnique.h | 2 +- src/Columns/ReverseIndex.h | 8 ++--- src/Common/HashTable/StringHashMap.h | 6 ++-- src/Common/SpaceSaving.h | 4 +-- src/Common/TLDListsHolder.cpp | 4 +-- src/Common/TLDListsHolder.h | 4 +-- src/Common/quoteString.cpp | 6 ++-- src/Common/quoteString.h | 6 ++-- src/Dictionaries/FlatDictionary.cpp | 4 +-- src/Dictionaries/HashedArrayDictionary.cpp | 4 +-- src/Dictionaries/HashedDictionary.cpp | 4 +-- src/Dictionaries/IPAddressDictionary.cpp | 2 +- src/Dictionaries/RangeHashedDictionary.cpp | 8 ++--- src/Functions/FunctionsAES.cpp | 4 +-- src/Functions/FunctionsAES.h | 10 +++--- src/Functions/isIPAddressContainedIn.cpp | 4 +-- src/IO/ReadHelpers.cpp | 2 +- src/IO/ReadHelpers.h | 2 +- src/IO/WriteHelpers.cpp | 14 ++++---- src/IO/WriteHelpers.h | 34 +++++++++---------- src/Interpreters/Aggregator.h | 10 +++--- src/Interpreters/Context.cpp | 4 +-- src/Interpreters/Context.h | 4 +-- .../Impl/JSONEachRowRowInputFormat.cpp | 4 +-- .../Formats/Impl/JSONEachRowRowInputFormat.h | 4 +-- src/Processors/Merges/Algorithms/Graphite.cpp | 2 +- src/Server/HTTPHandlerRequestFilter.h | 4 +-- 28 files changed, 83 insertions(+), 83 deletions(-) diff --git a/src/AggregateFunctions/ThetaSketchData.h b/src/AggregateFunctions/ThetaSketchData.h index cc35597ba56..f46836ad189 100644 --- a/src/AggregateFunctions/ThetaSketchData.h +++ b/src/AggregateFunctions/ThetaSketchData.h @@ -43,7 +43,7 @@ public: ~ThetaSketchData() = default; /// Insert original value without hash, as `datasketches::update_theta_sketch.update` will do the hash internal. - void insertOriginal(const StringRef & value) + void insertOriginal(StringRef value) { getSkUpdate()->update(value.data, value.size); } diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 33135224e11..3c21a65e404 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -509,7 +509,7 @@ MutableColumnPtr ColumnUnique::uniqueInsertRangeImpl( if (secondary_index) next_position += secondary_index->size(); - auto insert_key = [&](const StringRef & ref, ReverseIndex & cur_index) -> MutableColumnPtr + auto insert_key = [&](StringRef ref, ReverseIndex & cur_index) -> MutableColumnPtr { auto inserted_pos = cur_index.insert(ref); positions[num_added_rows] = inserted_pos; diff --git a/src/Columns/ReverseIndex.h b/src/Columns/ReverseIndex.h index 3f4427e17ad..ba6a014b49d 100644 --- a/src/Columns/ReverseIndex.h +++ b/src/Columns/ReverseIndex.h @@ -92,7 +92,7 @@ struct ReverseIndexHashTableCell /// Special case when we want to compare with something not in index_column. /// When we compare something inside column default keyEquals checks only that row numbers are equal. - bool keyEquals(const StringRef & object, size_t hash_ [[maybe_unused]], const State & state) const + bool keyEquals(StringRef object, size_t hash_ [[maybe_unused]], const State & state) const { auto index = key; if constexpr (has_base_index) @@ -322,7 +322,7 @@ public: static constexpr bool is_numeric_column = isNumericColumn(static_cast(nullptr)); static constexpr bool use_saved_hash = !is_numeric_column; - UInt64 insert(const StringRef & data); + UInt64 insert(StringRef data); /// Returns the found data's index in the dictionary. If index is not built, builds it. UInt64 getInsertionPoint(StringRef data) @@ -383,7 +383,7 @@ private: void buildIndex(); - UInt64 getHash(const StringRef & ref) const + UInt64 getHash(StringRef ref) const { if constexpr (is_numeric_column) { @@ -478,7 +478,7 @@ ColumnUInt64::MutablePtr ReverseIndex::calcHashes() const } template -UInt64 ReverseIndex::insert(const StringRef & data) +UInt64 ReverseIndex::insert(StringRef data) { if (!index) buildIndex(); diff --git a/src/Common/HashTable/StringHashMap.h b/src/Common/HashTable/StringHashMap.h index a3b5c3e9ed0..ada10180786 100644 --- a/src/Common/HashTable/StringHashMap.h +++ b/src/Common/HashTable/StringHashMap.h @@ -12,7 +12,7 @@ struct StringHashMapCell : public HashMapCellvalue.first); } /// NOLINT + StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT // internal static const Key & getKey(const value_type & value_) { return value_.first; } }; @@ -32,7 +32,7 @@ struct StringHashMapCell : public HashMapCellvalue.first.items[1] = 0; } // external - const StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT + StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT // internal static const StringKey16 & getKey(const value_type & value_) { return value_.first; } }; @@ -53,7 +53,7 @@ struct StringHashMapCell : public HashMapCellvalue.first.c = 0; } // external - const StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT + StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT // internal static const StringKey24 & getKey(const value_type & value_) { return value_.first; } }; diff --git a/src/Common/SpaceSaving.h b/src/Common/SpaceSaving.h index 48817d8c926..0f577349722 100644 --- a/src/Common/SpaceSaving.h +++ b/src/Common/SpaceSaving.h @@ -49,12 +49,12 @@ struct SpaceSavingArena template <> struct SpaceSavingArena { - StringRef emplace(const StringRef & key) + StringRef emplace(StringRef key) { return copyStringInArena(arena, key); } - void free(const StringRef & key) + void free(StringRef key) { if (key.data) arena.free(const_cast(key.data), key.size); diff --git a/src/Common/TLDListsHolder.cpp b/src/Common/TLDListsHolder.cpp index 3e5649a5ac6..a3019ac1c49 100644 --- a/src/Common/TLDListsHolder.cpp +++ b/src/Common/TLDListsHolder.cpp @@ -20,13 +20,13 @@ TLDList::TLDList(size_t size) : tld_container(size) , pool(std::make_unique(10 << 20)) {} -bool TLDList::insert(const StringRef & host) +bool TLDList::insert(StringRef host) { bool inserted; tld_container.emplace(DB::ArenaKeyHolder{host, *pool}, inserted); return inserted; } -bool TLDList::has(const StringRef & host) const +bool TLDList::has(StringRef host) const { return tld_container.has(host); } diff --git a/src/Common/TLDListsHolder.h b/src/Common/TLDListsHolder.h index 708d049d5a6..e8acefb1b5e 100644 --- a/src/Common/TLDListsHolder.h +++ b/src/Common/TLDListsHolder.h @@ -23,9 +23,9 @@ public: explicit TLDList(size_t size); /// Return true if the tld_container does not contains such element. - bool insert(const StringRef & host); + bool insert(StringRef host); /// Check is there such TLD - bool has(const StringRef & host) const; + bool has(StringRef host) const; size_t size() const { return tld_container.size(); } private: diff --git a/src/Common/quoteString.cpp b/src/Common/quoteString.cpp index e3e6e0b3249..b464f4837a1 100644 --- a/src/Common/quoteString.cpp +++ b/src/Common/quoteString.cpp @@ -14,7 +14,7 @@ String quoteString(std::string_view x) } -String doubleQuoteString(const StringRef & x) +String doubleQuoteString(StringRef x) { String res(x.size, '\0'); WriteBufferFromString wb(res); @@ -23,7 +23,7 @@ String doubleQuoteString(const StringRef & x) } -String backQuote(const StringRef & x) +String backQuote(StringRef x) { String res(x.size, '\0'); { @@ -34,7 +34,7 @@ String backQuote(const StringRef & x) } -String backQuoteIfNeed(const StringRef & x) +String backQuoteIfNeed(StringRef x) { String res(x.size, '\0'); { diff --git a/src/Common/quoteString.h b/src/Common/quoteString.h index 73c0de03d45..b83988258e2 100644 --- a/src/Common/quoteString.h +++ b/src/Common/quoteString.h @@ -16,12 +16,12 @@ namespace DB } /// Double quote the string. -String doubleQuoteString(const StringRef & x); +String doubleQuoteString(StringRef x); /// Quote the identifier with backquotes. -String backQuote(const StringRef & x); +String backQuote(StringRef x); /// Quote the identifier with backquotes, if required. -String backQuoteIfNeed(const StringRef & x); +String backQuoteIfNeed(StringRef x); } diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index d77f0bf825c..c858618c5ff 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -105,7 +105,7 @@ ColumnPtr FlatDictionary::getColumn( getItemsImpl( attribute, ids, - [&](size_t row, const StringRef value, bool is_null) + [&](size_t row, StringRef value, bool is_null) { (*vec_null_map_to)[row] = is_null; out->insertData(value.data, value.size); @@ -115,7 +115,7 @@ ColumnPtr FlatDictionary::getColumn( getItemsImpl( attribute, ids, - [&](size_t, const StringRef value, bool) { out->insertData(value.data, value.size); }, + [&](size_t, StringRef value, bool) { out->insertData(value.data, value.size); }, default_value_extractor); } else diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index d702a02bc2e..b8ed664e91a 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -585,7 +585,7 @@ ColumnPtr HashedArrayDictionary::getAttributeColumn( getItemsImpl( attribute, keys_object, - [&](size_t row, const StringRef value, bool is_null) + [&](size_t row, StringRef value, bool is_null) { (*vec_null_map_to)[row] = is_null; out->insertData(value.data, value.size); @@ -595,7 +595,7 @@ ColumnPtr HashedArrayDictionary::getAttributeColumn( getItemsImpl( attribute, keys_object, - [&](size_t, const StringRef value, bool) { out->insertData(value.data, value.size); }, + [&](size_t, StringRef value, bool) { out->insertData(value.data, value.size); }, default_value_extractor); } else diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index c5160c0dfa8..9beac59f274 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -117,7 +117,7 @@ ColumnPtr HashedDictionary::getColumn( getItemsImpl( attribute, extractor, - [&](size_t row, const StringRef value, bool is_null) + [&](size_t row, StringRef value, bool is_null) { (*vec_null_map_to)[row] = is_null; out->insertData(value.data, value.size); @@ -127,7 +127,7 @@ ColumnPtr HashedDictionary::getColumn( getItemsImpl( attribute, extractor, - [&](size_t, const StringRef value, bool) { out->insertData(value.data, value.size); }, + [&](size_t, StringRef value, bool) { out->insertData(value.data, value.size); }, default_value_extractor); } else diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 46cba702b5d..2a367323205 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -261,7 +261,7 @@ ColumnPtr IPAddressDictionary::getColumn( getItemsImpl( attribute, key_columns, - [&](const size_t, const StringRef value) { out->insertData(value.data, value.size); }, + [&](const size_t, StringRef value) { out->insertData(value.data, value.size); }, default_value_extractor); } else diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index 261e9166ec8..ad962ca4acc 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -151,7 +151,7 @@ ColumnPtr RangeHashedDictionary::getColumn( getItemsImpl( attribute, modified_key_columns, - [&](size_t row, const StringRef value, bool is_null) + [&](size_t row, StringRef value, bool is_null) { (*vec_null_map_to)[row] = is_null; out->insertData(value.data, value.size); @@ -161,7 +161,7 @@ ColumnPtr RangeHashedDictionary::getColumn( getItemsImpl( attribute, modified_key_columns, - [&](size_t, const StringRef value, bool) + [&](size_t, StringRef value, bool) { out->insertData(value.data, value.size); }, @@ -255,7 +255,7 @@ ColumnPtr RangeHashedDictionary::getColumnInternal( getItemsInternalImpl( attribute, key_to_index, - [&](size_t row, const StringRef value, bool is_null) + [&](size_t row, StringRef value, bool is_null) { (*vec_null_map_to)[row] = is_null; out->insertData(value.data, value.size); @@ -264,7 +264,7 @@ ColumnPtr RangeHashedDictionary::getColumnInternal( getItemsInternalImpl( attribute, key_to_index, - [&](size_t, const StringRef value, bool) + [&](size_t, StringRef value, bool) { out->insertData(value.data, value.size); }); diff --git a/src/Functions/FunctionsAES.cpp b/src/Functions/FunctionsAES.cpp index a2dc7e40489..9ef07e2747d 100644 --- a/src/Functions/FunctionsAES.cpp +++ b/src/Functions/FunctionsAES.cpp @@ -25,7 +25,7 @@ void onError(std::string error_message) throw DB::Exception(error_message, DB::ErrorCodes::OPENSSL_ERROR); } -StringRef foldEncryptionKeyInMySQLCompatitableMode(size_t cipher_key_size, const StringRef & key, std::array & folded_key) +StringRef foldEncryptionKeyInMySQLCompatitableMode(size_t cipher_key_size, StringRef key, std::array & folded_key) { assert(cipher_key_size <= EVP_MAX_KEY_LENGTH); memcpy(folded_key.data(), key.data, cipher_key_size); @@ -38,7 +38,7 @@ StringRef foldEncryptionKeyInMySQLCompatitableMode(size_t cipher_key_size, const return StringRef(folded_key.data(), cipher_key_size); } -const EVP_CIPHER * getCipherByName(const StringRef & cipher_name) +const EVP_CIPHER * getCipherByName(StringRef cipher_name) { // NOTE: cipher obtained not via EVP_CIPHER_fetch() would cause extra work on each context reset // with EVP_CIPHER_CTX_reset() or EVP_EncryptInit_ex(), but using EVP_CIPHER_fetch() diff --git a/src/Functions/FunctionsAES.h b/src/Functions/FunctionsAES.h index d3796081f18..d3c533c804b 100644 --- a/src/Functions/FunctionsAES.h +++ b/src/Functions/FunctionsAES.h @@ -32,9 +32,9 @@ namespace ErrorCodes namespace OpenSSLDetails { [[noreturn]] void onError(std::string error_message); -StringRef foldEncryptionKeyInMySQLCompatitableMode(size_t cipher_key_size, const StringRef & key, std::array & folded_key); +StringRef foldEncryptionKeyInMySQLCompatitableMode(size_t cipher_key_size, StringRef key, std::array & folded_key); -const EVP_CIPHER * getCipherByName(const StringRef & name); +const EVP_CIPHER * getCipherByName(StringRef name); enum class CompatibilityMode { @@ -53,7 +53,7 @@ enum class CipherMode template struct KeyHolder { - inline StringRef setKey(size_t cipher_key_size, const StringRef & key) const + inline StringRef setKey(size_t cipher_key_size, StringRef key) const { if (key.size != cipher_key_size) throw DB::Exception(fmt::format("Invalid key size: {} expected {}", key.size, cipher_key_size), @@ -66,7 +66,7 @@ struct KeyHolder template <> struct KeyHolder { - inline StringRef setKey(size_t cipher_key_size, const StringRef & key) + inline StringRef setKey(size_t cipher_key_size, StringRef key) { if (key.size < cipher_key_size) throw DB::Exception(fmt::format("Invalid key size: {} expected {}", key.size, cipher_key_size), @@ -120,7 +120,7 @@ inline void validateCipherMode(const EVP_CIPHER * evp_cipher) } template -inline void validateIV(const StringRef & iv_value, const size_t cipher_iv_size) +inline void validateIV(StringRef iv_value, const size_t cipher_iv_size) { // In MySQL mode we don't care if IV is longer than expected, only if shorter. if ((mode == CipherMode::MySQLCompatibility && iv_value.size != 0 && iv_value.size < cipher_iv_size) diff --git a/src/Functions/isIPAddressContainedIn.cpp b/src/Functions/isIPAddressContainedIn.cpp index 5ef247f7346..6fdc0dfbee8 100644 --- a/src/Functions/isIPAddressContainedIn.cpp +++ b/src/Functions/isIPAddressContainedIn.cpp @@ -27,7 +27,7 @@ class IPAddressVariant { public: - explicit IPAddressVariant(const StringRef & address_str) + explicit IPAddressVariant(StringRef address_str) { /// IP address parser functions require that the input is /// NULL-terminated so we need to copy it. @@ -75,7 +75,7 @@ struct IPAddressCIDR UInt8 prefix; }; -IPAddressCIDR parseIPWithCIDR(const StringRef cidr_str) +IPAddressCIDR parseIPWithCIDR(StringRef cidr_str) { std::string_view cidr_str_view(cidr_str); size_t pos_slash = cidr_str_view.find('/'); diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index f09292cd349..c2b0a0f65d7 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1053,7 +1053,7 @@ template void readDateTimeTextFallback(time_t &, ReadBuffer &, const DateL template bool readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &); -void skipJSONField(ReadBuffer & buf, const StringRef & name_of_field) +void skipJSONField(ReadBuffer & buf, StringRef name_of_field) { if (buf.eof()) throw Exception("Unexpected EOF for key '" + name_of_field.toString() + "'", ErrorCodes::INCORRECT_DATA); diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 57283a396d9..7a5df1ed5ac 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1238,7 +1238,7 @@ inline void skipWhitespaceIfAny(ReadBuffer & buf, bool one_line = false) } /// Skips json value. -void skipJSONField(ReadBuffer & buf, const StringRef & name_of_field); +void skipJSONField(ReadBuffer & buf, StringRef name_of_field); /** Read serialized exception. diff --git a/src/IO/WriteHelpers.cpp b/src/IO/WriteHelpers.cpp index a6d492b85b0..fae3d21513e 100644 --- a/src/IO/WriteHelpers.cpp +++ b/src/IO/WriteHelpers.cpp @@ -66,7 +66,7 @@ void writeException(const Exception & e, WriteBuffer & buf, bool with_stack_trac /// The same, but quotes apply only if there are characters that do not match the identifier without quotes template -static inline void writeProbablyQuotedStringImpl(const StringRef & s, WriteBuffer & buf, F && write_quoted_string) +static inline void writeProbablyQuotedStringImpl(StringRef s, WriteBuffer & buf, F && write_quoted_string) { if (isValidIdentifier(std::string_view{s}) /// This are valid identifiers but are problematic if present unquoted in SQL query. @@ -79,19 +79,19 @@ static inline void writeProbablyQuotedStringImpl(const StringRef & s, WriteBuffe write_quoted_string(s, buf); } -void writeProbablyBackQuotedString(const StringRef & s, WriteBuffer & buf) +void writeProbablyBackQuotedString(StringRef s, WriteBuffer & buf) { - writeProbablyQuotedStringImpl(s, buf, [](const StringRef & s_, WriteBuffer & buf_) { return writeBackQuotedString(s_, buf_); }); + writeProbablyQuotedStringImpl(s, buf, [](StringRef s_, WriteBuffer & buf_) { return writeBackQuotedString(s_, buf_); }); } -void writeProbablyDoubleQuotedString(const StringRef & s, WriteBuffer & buf) +void writeProbablyDoubleQuotedString(StringRef s, WriteBuffer & buf) { - writeProbablyQuotedStringImpl(s, buf, [](const StringRef & s_, WriteBuffer & buf_) { return writeDoubleQuotedString(s_, buf_); }); + writeProbablyQuotedStringImpl(s, buf, [](StringRef s_, WriteBuffer & buf_) { return writeDoubleQuotedString(s_, buf_); }); } -void writeProbablyBackQuotedStringMySQL(const StringRef & s, WriteBuffer & buf) +void writeProbablyBackQuotedStringMySQL(StringRef s, WriteBuffer & buf) { - writeProbablyQuotedStringImpl(s, buf, [](const StringRef & s_, WriteBuffer & buf_) { return writeBackQuotedStringMySQL(s_, buf_); }); + writeProbablyQuotedStringImpl(s, buf, [](StringRef s_, WriteBuffer & buf_) { return writeBackQuotedStringMySQL(s_, buf_); }); } void writePointerHex(const void * ptr, WriteBuffer & buf) diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index c3bbaac097d..6f35dae8300 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -102,7 +102,7 @@ inline void writeStringBinary(const std::string & s, WriteBuffer & buf) buf.write(s.data(), s.size()); } -inline void writeStringBinary(const StringRef & s, WriteBuffer & buf) +inline void writeStringBinary(StringRef s, WriteBuffer & buf) { writeVarUInt(s.size, buf); buf.write(s.data, s.size); @@ -360,7 +360,7 @@ void writeAnyEscapedString(const char * begin, const char * end, WriteBuffer & b } -inline void writeJSONString(const StringRef & s, WriteBuffer & buf, const FormatSettings & settings) +inline void writeJSONString(StringRef s, WriteBuffer & buf, const FormatSettings & settings) { writeJSONString(s.data, s.data + s.size, buf, settings); } @@ -435,7 +435,7 @@ inline void writeEscapedString(const String & s, WriteBuffer & buf) } -inline void writeEscapedString(const StringRef & ref, WriteBuffer & buf) +inline void writeEscapedString(StringRef ref, WriteBuffer & buf) { writeEscapedString(ref.data, ref.size, buf); } @@ -462,7 +462,7 @@ void writeAnyQuotedString(const String & s, WriteBuffer & buf) template -void writeAnyQuotedString(const StringRef & ref, WriteBuffer & buf) +void writeAnyQuotedString(StringRef ref, WriteBuffer & buf) { writeAnyQuotedString(ref.data, ref.data + ref.size, buf); } @@ -473,7 +473,7 @@ inline void writeQuotedString(const String & s, WriteBuffer & buf) writeAnyQuotedString<'\''>(s, buf); } -inline void writeQuotedString(const StringRef & ref, WriteBuffer & buf) +inline void writeQuotedString(StringRef ref, WriteBuffer & buf) { writeAnyQuotedString<'\''>(ref, buf); } @@ -488,7 +488,7 @@ inline void writeDoubleQuotedString(const String & s, WriteBuffer & buf) writeAnyQuotedString<'"'>(s, buf); } -inline void writeDoubleQuotedString(const StringRef & s, WriteBuffer & buf) +inline void writeDoubleQuotedString(StringRef s, WriteBuffer & buf) { writeAnyQuotedString<'"'>(s, buf); } @@ -499,13 +499,13 @@ inline void writeDoubleQuotedString(std::string_view s, WriteBuffer & buf) } /// Outputs a string in backquotes. -inline void writeBackQuotedString(const StringRef & s, WriteBuffer & buf) +inline void writeBackQuotedString(StringRef s, WriteBuffer & buf) { writeAnyQuotedString<'`'>(s, buf); } /// Outputs a string in backquotes for MySQL. -inline void writeBackQuotedStringMySQL(const StringRef & s, WriteBuffer & buf) +inline void writeBackQuotedStringMySQL(StringRef s, WriteBuffer & buf) { writeChar('`', buf); writeAnyEscapedString<'`', true>(s.data, s.data + s.size, buf); @@ -514,9 +514,9 @@ inline void writeBackQuotedStringMySQL(const StringRef & s, WriteBuffer & buf) /// Write quoted if the string doesn't look like and identifier. -void writeProbablyBackQuotedString(const StringRef & s, WriteBuffer & buf); -void writeProbablyDoubleQuotedString(const StringRef & s, WriteBuffer & buf); -void writeProbablyBackQuotedStringMySQL(const StringRef & s, WriteBuffer & buf); +void writeProbablyBackQuotedString(StringRef s, WriteBuffer & buf); +void writeProbablyDoubleQuotedString(StringRef s, WriteBuffer & buf); +void writeProbablyBackQuotedStringMySQL(StringRef s, WriteBuffer & buf); /** Outputs the string in for the CSV format. @@ -559,7 +559,7 @@ void writeCSVString(const String & s, WriteBuffer & buf) } template -void writeCSVString(const StringRef & s, WriteBuffer & buf) +void writeCSVString(StringRef s, WriteBuffer & buf) { writeCSVString(s.data, s.data + s.size, buf); } @@ -616,7 +616,7 @@ inline void writeXMLStringForTextElementOrAttributeValue(const String & s, Write writeXMLStringForTextElementOrAttributeValue(s.data(), s.data() + s.size(), buf); } -inline void writeXMLStringForTextElementOrAttributeValue(const StringRef & s, WriteBuffer & buf) +inline void writeXMLStringForTextElementOrAttributeValue(StringRef s, WriteBuffer & buf) { writeXMLStringForTextElementOrAttributeValue(s.data, s.data + s.size, buf); } @@ -657,7 +657,7 @@ inline void writeXMLStringForTextElement(const String & s, WriteBuffer & buf) writeXMLStringForTextElement(s.data(), s.data() + s.size(), buf); } -inline void writeXMLStringForTextElement(const StringRef & s, WriteBuffer & buf) +inline void writeXMLStringForTextElement(StringRef s, WriteBuffer & buf) { writeXMLStringForTextElement(s.data, s.data + s.size, buf); } @@ -890,7 +890,7 @@ requires is_arithmetic_v inline void writeBinary(const T & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const String & x, WriteBuffer & buf) { writeStringBinary(x, buf); } -inline void writeBinary(const StringRef & x, WriteBuffer & buf) { writeStringBinary(x, buf); } +inline void writeBinary(StringRef x, WriteBuffer & buf) { writeStringBinary(x, buf); } inline void writeBinary(std::string_view x, WriteBuffer & buf) { writeStringBinary(x, buf); } inline void writeBinary(const Decimal32 & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const Decimal64 & x, WriteBuffer & buf) { writePODBinary(x, buf); } @@ -1017,7 +1017,7 @@ inline void writeQuoted(const String & x, WriteBuffer & buf) { writeQuotedString inline void writeQuoted(std::string_view x, WriteBuffer & buf) { writeQuotedString(x, buf); } -inline void writeQuoted(const StringRef & x, WriteBuffer & buf) { writeQuotedString(x, buf); } +inline void writeQuoted(StringRef x, WriteBuffer & buf) { writeQuotedString(x, buf); } inline void writeQuoted(const LocalDate & x, WriteBuffer & buf) { @@ -1050,7 +1050,7 @@ inline void writeDoubleQuoted(const String & x, WriteBuffer & buf) { writeDouble inline void writeDoubleQuoted(std::string_view x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); } -inline void writeDoubleQuoted(const StringRef & x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); } +inline void writeDoubleQuoted(StringRef x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); } inline void writeDoubleQuoted(const LocalDate & x, WriteBuffer & buf) { diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index feb07727725..716849465de 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -238,7 +238,7 @@ struct AggregationMethodString std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } - static void insertKeyIntoColumns(const StringRef & key, std::vector & key_columns, const Sizes &) + static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) { static_cast(key_columns[0])->insertData(key.data, key.size); } @@ -270,7 +270,7 @@ struct AggregationMethodStringNoCache std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } - static void insertKeyIntoColumns(const StringRef & key, std::vector & key_columns, const Sizes &) + static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) { static_cast(key_columns[0])->insertData(key.data, key.size); } @@ -302,7 +302,7 @@ struct AggregationMethodFixedString std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } - static void insertKeyIntoColumns(const StringRef & key, std::vector & key_columns, const Sizes &) + static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) { static_cast(key_columns[0])->insertData(key.data, key.size); } @@ -333,7 +333,7 @@ struct AggregationMethodFixedStringNoCache std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } - static void insertKeyIntoColumns(const StringRef & key, std::vector & key_columns, const Sizes &) + static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) { static_cast(key_columns[0])->insertData(key.data, key.size); } @@ -501,7 +501,7 @@ struct AggregationMethodSerialized std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } - static void insertKeyIntoColumns(const StringRef & key, std::vector & key_columns, const Sizes &) + static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) { const auto * pos = key.data; for (auto & column : key_columns) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index cbbcc58df2e..fe59215f7d5 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1191,7 +1191,7 @@ void Context::setSettings(const Settings & settings_) } -void Context::setSetting(const StringRef & name, const String & value) +void Context::setSetting(StringRef name, const String & value) { auto lock = getLock(); if (name == "profile") @@ -1206,7 +1206,7 @@ void Context::setSetting(const StringRef & name, const String & value) } -void Context::setSetting(const StringRef & name, const Field & value) +void Context::setSetting(StringRef name, const Field & value) { auto lock = getLock(); if (name == "profile") diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index ca0c218a4c0..e7aba31a1d9 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -607,8 +607,8 @@ public: void setSettings(const Settings & settings_); /// Set settings by name. - void setSetting(const StringRef & name, const String & value); - void setSetting(const StringRef & name, const Field & value); + void setSetting(StringRef name, const String & value); + void setSetting(StringRef name, const Field & value); void applySettingChange(const SettingChange & change); void applySettingsChanges(const SettingsChanges & changes); diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 9eef72f95da..3bcea8a8843 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -62,7 +62,7 @@ const String & JSONEachRowRowInputFormat::columnName(size_t i) const return getPort().getHeader().getByPosition(i).name; } -inline size_t JSONEachRowRowInputFormat::columnIndex(const StringRef & name, size_t key_index) +inline size_t JSONEachRowRowInputFormat::columnIndex(StringRef name, size_t key_index) { /// Optimization by caching the order of fields (which is almost always the same) /// and a quick check to match the next expected field, instead of searching the hash table. @@ -124,7 +124,7 @@ static inline void skipColonDelimeter(ReadBuffer & istr) skipWhitespaceIfAny(istr); } -void JSONEachRowRowInputFormat::skipUnknownField(const StringRef & name_ref) +void JSONEachRowRowInputFormat::skipUnknownField(StringRef name_ref) { if (!format_settings.skip_unknown_fields) throw Exception("Unknown field found while parsing JSONEachRow format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA); diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h index 1da14a532de..1673d55b9fd 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h @@ -40,9 +40,9 @@ private: void syncAfterError() override; const String & columnName(size_t i) const; - size_t columnIndex(const StringRef & name, size_t key_index); + size_t columnIndex(StringRef name, size_t key_index); bool advanceToNextKey(size_t key_index); - void skipUnknownField(const StringRef & name_ref); + void skipUnknownField(StringRef name_ref); StringRef readColumnName(ReadBuffer & buf); void readField(size_t index, MutableColumns & columns); void readJSONObject(MutableColumns & columns); diff --git a/src/Processors/Merges/Algorithms/Graphite.cpp b/src/Processors/Merges/Algorithms/Graphite.cpp index f77bb790332..6c4ca5ef85b 100644 --- a/src/Processors/Merges/Algorithms/Graphite.cpp +++ b/src/Processors/Merges/Algorithms/Graphite.cpp @@ -71,7 +71,7 @@ static const Graphite::Pattern undef_pattern = .type = undef_pattern.TypeUndef, }; -inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, const StringRef path) +inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, StringRef path) { if (params.patterns_typed) { diff --git a/src/Server/HTTPHandlerRequestFilter.h b/src/Server/HTTPHandlerRequestFilter.h index 3236b35d5ae..d0156266fe5 100644 --- a/src/Server/HTTPHandlerRequestFilter.h +++ b/src/Server/HTTPHandlerRequestFilter.h @@ -23,7 +23,7 @@ namespace ErrorCodes using CompiledRegexPtr = std::shared_ptr; -static inline bool checkRegexExpression(const StringRef & match_str, const CompiledRegexPtr & compiled_regex) +static inline bool checkRegexExpression(StringRef match_str, const CompiledRegexPtr & compiled_regex) { int num_captures = compiled_regex->NumberOfCapturingGroups() + 1; @@ -32,7 +32,7 @@ static inline bool checkRegexExpression(const StringRef & match_str, const Compi return compiled_regex->Match(match_input, 0, match_str.size, re2::RE2::Anchor::ANCHOR_BOTH, matches, num_captures); } -static inline bool checkExpression(const StringRef & match_str, const std::pair & expression) +static inline bool checkExpression(StringRef match_str, const std::pair & expression) { if (expression.second) return checkRegexExpression(match_str, expression.second); From 2f47be5da7927e8d5be853173eb99a29a54a874a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 15 Jul 2022 09:42:56 +0200 Subject: [PATCH 263/659] Check that the destination for a backup is not in use. --- src/Backups/BackupFactory.h | 2 +- src/Backups/BackupIO.h | 3 +- src/Backups/BackupIO_Disk.cpp | 21 +++- src/Backups/BackupIO_Disk.h | 3 +- src/Backups/BackupIO_File.cpp | 21 +++- src/Backups/BackupIO_File.h | 3 +- src/Backups/BackupImpl.cpp | 112 ++++++++++++++---- src/Backups/BackupImpl.h | 21 +++- src/Backups/BackupSettings.cpp | 46 ++++++- src/Backups/BackupSettings.h | 4 + src/Backups/BackupsWorker.cpp | 7 +- .../registerBackupEnginesFileAndDisk.cpp | 2 +- .../test_backup_restore_new/test.py | 37 ++++++ .../test_backup_restore_on_cluster/test.py | 57 +++++++++ 14 files changed, 302 insertions(+), 37 deletions(-) diff --git a/src/Backups/BackupFactory.h b/src/Backups/BackupFactory.h index f9a97e3dfc5..9057d2cbfae 100644 --- a/src/Backups/BackupFactory.h +++ b/src/Backups/BackupFactory.h @@ -25,7 +25,6 @@ public: struct CreateParams { OpenMode open_mode = OpenMode::WRITE; - std::optional backup_uuid; BackupInfo backup_info; std::optional base_backup_info; String compression_method; @@ -34,6 +33,7 @@ public: ContextPtr context; bool is_internal_backup = false; std::shared_ptr backup_coordination; + std::optional backup_uuid; }; static BackupFactory & instance(); diff --git a/src/Backups/BackupIO.h b/src/Backups/BackupIO.h index ec0b2301800..433e81a70a2 100644 --- a/src/Backups/BackupIO.h +++ b/src/Backups/BackupIO.h @@ -23,8 +23,9 @@ class IBackupWriter /// BackupWriterFile, BackupWriterDisk, BackupWriterS3 public: virtual ~IBackupWriter() = default; virtual bool fileExists(const String & file_name) = 0; + virtual bool fileContentsEqual(const String & file_name, const String & expected_file_contents) = 0; virtual std::unique_ptr writeFile(const String & file_name) = 0; - virtual void removeFilesAfterFailure(const Strings & file_names) = 0; + virtual void removeFiles(const Strings & file_names) = 0; }; } diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index a5c26bdbed6..537bc667cd4 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -38,6 +38,25 @@ bool BackupWriterDisk::fileExists(const String & file_name) return disk->exists(path / file_name); } +bool BackupWriterDisk::fileContentsEqual(const String & file_name, const String & expected_file_contents) +{ + if (!disk->exists(path / file_name)) + return false; + + try + { + auto in = disk->readFile(path / file_name); + String actual_file_contents(expected_file_contents.size(), ' '); + return (in->read(actual_file_contents.data(), actual_file_contents.size()) == actual_file_contents.size()) + && (actual_file_contents == expected_file_contents) && in->eof(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + return false; + } +} + std::unique_ptr BackupWriterDisk::writeFile(const String & file_name) { auto file_path = path / file_name; @@ -45,7 +64,7 @@ std::unique_ptr BackupWriterDisk::writeFile(const String & file_nam return disk->writeFile(file_path); } -void BackupWriterDisk::removeFilesAfterFailure(const Strings & file_names) +void BackupWriterDisk::removeFiles(const Strings & file_names) { for (const auto & file_name : file_names) disk->removeFileIfExists(path / file_name); diff --git a/src/Backups/BackupIO_Disk.h b/src/Backups/BackupIO_Disk.h index 88d70b0f1db..8ba99470938 100644 --- a/src/Backups/BackupIO_Disk.h +++ b/src/Backups/BackupIO_Disk.h @@ -30,8 +30,9 @@ public: ~BackupWriterDisk() override; bool fileExists(const String & file_name) override; + bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; std::unique_ptr writeFile(const String & file_name) override; - void removeFilesAfterFailure(const Strings & file_names) override; + void removeFiles(const Strings & file_names) override; private: DiskPtr disk; diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index 8e7bfb5b83e..774d493ee38 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -39,6 +39,25 @@ bool BackupWriterFile::fileExists(const String & file_name) return fs::exists(path / file_name); } +bool BackupWriterFile::fileContentsEqual(const String & file_name, const String & expected_file_contents) +{ + if (!fs::exists(path / file_name)) + return false; + + try + { + auto in = createReadBufferFromFileBase(path / file_name, {}); + String actual_file_contents(expected_file_contents.size(), ' '); + return (in->read(actual_file_contents.data(), actual_file_contents.size()) == actual_file_contents.size()) + && (actual_file_contents == expected_file_contents) && in->eof(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + return false; + } +} + std::unique_ptr BackupWriterFile::writeFile(const String & file_name) { auto file_path = path / file_name; @@ -46,7 +65,7 @@ std::unique_ptr BackupWriterFile::writeFile(const String & file_nam return std::make_unique(file_path); } -void BackupWriterFile::removeFilesAfterFailure(const Strings & file_names) +void BackupWriterFile::removeFiles(const Strings & file_names) { for (const auto & file_name : file_names) fs::remove(path / file_name); diff --git a/src/Backups/BackupIO_File.h b/src/Backups/BackupIO_File.h index c4aa20718a9..aebf2bdab73 100644 --- a/src/Backups/BackupIO_File.h +++ b/src/Backups/BackupIO_File.h @@ -27,8 +27,9 @@ public: ~BackupWriterFile() override; bool fileExists(const String & file_name) override; + bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; std::unique_ptr writeFile(const String & file_name) override; - void removeFilesAfterFailure(const Strings & file_names) override; + void removeFiles(const Strings & file_names) override; private: std::filesystem::path path; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 20b7bf37cfc..d445ef9d52c 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -37,6 +37,7 @@ namespace ErrorCodes extern const int BACKUP_ENTRY_ALREADY_EXISTS; extern const int BACKUP_ENTRY_NOT_FOUND; extern const int BACKUP_IS_EMPTY; + extern const int FAILED_TO_SYNC_BACKUP_OR_RESTORE; extern const int LOGICAL_ERROR; } @@ -146,9 +147,9 @@ BackupImpl::BackupImpl( const std::optional & base_backup_info_, std::shared_ptr writer_, const ContextPtr & context_, - const std::optional & backup_uuid_, bool is_internal_backup_, - const std::shared_ptr & coordination_) + const std::shared_ptr & coordination_, + const std::optional & backup_uuid_) : backup_name(backup_name_) , archive_params(archive_params_) , use_archives(!archive_params.archive_name.empty()) @@ -177,42 +178,28 @@ BackupImpl::~BackupImpl() } } - void BackupImpl::open(const ContextPtr & context) { std::lock_guard lock{mutex}; - String file_name_to_check_existence; - if (use_archives) - file_name_to_check_existence = archive_params.archive_name; - else - file_name_to_check_existence = ".backup"; - bool backup_exists = (open_mode == OpenMode::WRITE) ? writer->fileExists(file_name_to_check_existence) : reader->fileExists(file_name_to_check_existence); - - if (open_mode == OpenMode::WRITE) - { - if (backup_exists) - throw Exception(ErrorCodes::BACKUP_ALREADY_EXISTS, "Backup {} already exists", backup_name); - } - else - { - if (!backup_exists) - throw Exception(ErrorCodes::BACKUP_NOT_FOUND, "Backup {} not found", backup_name); - } - if (open_mode == OpenMode::WRITE) { timestamp = std::time(nullptr); if (!uuid) uuid = UUIDHelpers::generateV4(); + lock_file_name = use_archives ? (archive_params.archive_name + ".lock") : ".lock"; writing_finalized = false; + + /// Check that we can write a backup there and create the lock file to own this destination. + checkBackupDoesntExist(); + if (!is_internal_backup) + createLockFile(); + checkLockFile(true); } if (open_mode == OpenMode::READ) readBackupMetadata(); - assert(uuid); /// Backup's UUID must be loaded or generated at this point. - if (base_backup_info) { BackupFactory::CreateParams params; @@ -253,6 +240,8 @@ time_t BackupImpl::getTimestamp() const void BackupImpl::writeBackupMetadata() { + assert(!is_internal_backup); + Poco::AutoPtr config{new Poco::Util::XMLConfiguration()}; config->setUInt("version", CURRENT_BACKUP_VERSION); config->setString("timestamp", toString(LocalDateTime{timestamp})); @@ -308,6 +297,8 @@ void BackupImpl::writeBackupMetadata() config->save(stream); String str = stream.str(); + checkLockFile(true); + std::unique_ptr out; if (use_archives) out = getArchiveWriter("")->writeFile(".backup"); @@ -321,9 +312,17 @@ void BackupImpl::readBackupMetadata() { std::unique_ptr in; if (use_archives) + { + if (!reader->fileExists(archive_params.archive_name)) + throw Exception(ErrorCodes::BACKUP_NOT_FOUND, "Backup {} not found", backup_name); in = getArchiveReader("")->readFile(".backup"); + } else + { + if (!reader->fileExists(".backup")) + throw Exception(ErrorCodes::BACKUP_NOT_FOUND, "Backup {} not found", backup_name); in = reader->readFile(".backup"); + } String str; readStringUntilEOF(str, *in); @@ -387,6 +386,59 @@ void BackupImpl::readBackupMetadata() } } +void BackupImpl::checkBackupDoesntExist() const +{ + String file_name_to_check_existence; + if (use_archives) + file_name_to_check_existence = archive_params.archive_name; + else + file_name_to_check_existence = ".backup"; + + if (writer->fileExists(file_name_to_check_existence)) + throw Exception(ErrorCodes::BACKUP_ALREADY_EXISTS, "Backup {} already exists", backup_name); + + /// Check that no other backup (excluding internal backups) is writing to the same destination. + if (!is_internal_backup) + { + assert(!lock_file_name.empty()); + if (writer->fileExists(lock_file_name)) + throw Exception(ErrorCodes::BACKUP_ALREADY_EXISTS, "Backup {} is being written already", backup_name); + } +} + +void BackupImpl::createLockFile() +{ + /// Internal backup must not create the lock file (it should be created by the initiator). + assert(!is_internal_backup); + + assert(uuid); + auto out = writer->writeFile(lock_file_name); + writeUUIDText(*uuid, *out); +} + +bool BackupImpl::checkLockFile(bool throw_if_failed) const +{ + if (!lock_file_name.empty() && uuid && writer->fileContentsEqual(lock_file_name, toString(*uuid))) + return true; + + if (throw_if_failed) + { + if (!writer->fileExists(lock_file_name)) + throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "Lock file {} suddenly disappeared while writing backup {}", lock_file_name, backup_name); + throw Exception(ErrorCodes::BACKUP_ALREADY_EXISTS, "A concurrent backup writing to the same destination {} detected", backup_name); + } + return false; +} + +void BackupImpl::removeLockFile() +{ + if (is_internal_backup) + return; /// Internal backup must not remove the lock file (it's still used by the initiator). + + if (checkLockFile(false)) + writer->removeFiles({lock_file_name}); +} + Strings BackupImpl::listFiles(const String & directory, bool recursive) const { std::lock_guard lock{mutex}; @@ -648,6 +700,9 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) read_buffer = entry->getReadBuffer(); read_buffer->seek(copy_pos, SEEK_SET); + if (!num_files_written) + checkLockFile(true); + /// Copy the entry's data after `copy_pos`. std::unique_ptr out; if (use_archives) @@ -675,6 +730,7 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) copyData(*read_buffer, *out); out->finalize(); + ++num_files_written; } @@ -694,6 +750,7 @@ void BackupImpl::finalizeWriting() { LOG_TRACE(log, "Finalizing backup {}", backup_name); writeBackupMetadata(); + removeLockFile(); LOG_TRACE(log, "Finalized backup {}", backup_name); } @@ -741,6 +798,9 @@ std::shared_ptr BackupImpl::getArchiveWriter(const String & suff void BackupImpl::removeAllFilesAfterFailure() { + if (is_internal_backup) + return; /// Let the initiator remove unnecessary files. + try { LOG_INFO(log, "Removing all files of backup {} after failure", backup_name); @@ -762,7 +822,11 @@ void BackupImpl::removeAllFilesAfterFailure() files_to_remove.push_back(file_info.data_file_name); } - writer->removeFilesAfterFailure(files_to_remove); + if (!checkLockFile(false)) + return; + + writer->removeFiles(files_to_remove); + removeLockFile(); } catch (...) { diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index f8c5bc0cf5f..ac0662c62c1 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -47,9 +47,9 @@ public: const std::optional & base_backup_info_, std::shared_ptr writer_, const ContextPtr & context_, - const std::optional & backup_uuid_ = {}, bool is_internal_backup_ = false, - const std::shared_ptr & coordination_ = {}); + const std::shared_ptr & coordination_ = {}, + const std::optional & backup_uuid_ = {}); ~BackupImpl() override; @@ -76,12 +76,25 @@ private: void open(const ContextPtr & context); void close(); + + /// Writes the file ".backup" containing backup's metadata. void writeBackupMetadata(); void readBackupMetadata(); + + /// Checks that a new backup doesn't exist yet. + void checkBackupDoesntExist() const; + + /// Lock file named ".lock" and containing the UUID of a backup is used to own the place where we're writing the backup. + /// Thus it will not be allowed to put any other backup to the same place (even if the BACKUP command is executed on a different node). + void createLockFile(); + bool checkLockFile(bool throw_if_failed) const; + void removeLockFile(); + + void removeAllFilesAfterFailure(); + String getArchiveNameWithSuffix(const String & suffix) const; std::shared_ptr getArchiveReader(const String & suffix) const; std::shared_ptr getArchiveWriter(const String & suffix); - void removeAllFilesAfterFailure(); const String backup_name; const ArchiveParams archive_params; @@ -102,6 +115,8 @@ private: mutable std::unordered_map> archive_readers; std::pair> archive_writers[2]; String current_archive_suffix; + String lock_file_name; + size_t num_files_written = 0; bool writing_finalized = false; const Poco::Logger * log; }; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 05ba0676ab8..a9ba7cb5f74 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -15,6 +16,48 @@ namespace ErrorCodes extern const int WRONG_BACKUP_SETTINGS; } + +namespace +{ + struct SettingFieldOptionalUUID + { + std::optional value; + + explicit SettingFieldOptionalUUID(const std::optional & value_) : value(value_) {} + + explicit SettingFieldOptionalUUID(const Field & field) + { + if (field.getType() == Field::Types::Null) + { + value = std::nullopt; + return; + } + + if (field.getType() == Field::Types::String) + { + const String & str = field.get(); + if (str.empty()) + { + value = std::nullopt; + return; + } + + UUID id; + if (tryParse(id, str)) + { + value = id; + return; + } + } + + throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Cannot parse uuid from {}", field); + } + + explicit operator Field() const { return Field(value ? toString(*value) : ""); } + }; +} + + /// List of backup settings except base_backup_name and cluster_host_ids. #define LIST_OF_BACKUP_SETTINGS(M) \ M(String, compression_method) \ @@ -26,7 +69,8 @@ namespace ErrorCodes M(UInt64, replica_num) \ M(Bool, internal) \ M(String, host_id) \ - M(String, coordination_zk_path) + M(String, coordination_zk_path) \ + M(OptionalUUID, backup_uuid) BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) { diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index 8a606ffded8..4e2bad67fce 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -53,6 +53,10 @@ struct BackupSettings /// Path in Zookeeper used to coordinate a distributed backup created by BACKUP ON CLUSTER. String coordination_zk_path; + /// Internal, should not be specified by user. + /// UUID of the backup. If it's not set it will be generated randomly. + std::optional backup_uuid; + static BackupSettings fromBackupQuery(const ASTBackupQuery & query); void copySettingsToQuery(ASTBackupQuery & query) const; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index bf90d58d009..ce3d47e9a30 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -72,12 +72,15 @@ UUID BackupsWorker::start(const ASTPtr & backup_or_restore_query, ContextMutable UUID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & context) { - UUID backup_uuid = UUIDHelpers::generateV4(); auto backup_query = std::static_pointer_cast(query->clone()); auto backup_settings = BackupSettings::fromBackupQuery(*backup_query); auto backup_info = BackupInfo::fromAST(*backup_query->backup_name); bool on_cluster = !backup_query->cluster.empty(); + if (!backup_settings.backup_uuid) + backup_settings.backup_uuid = UUIDHelpers::generateV4(); + UUID backup_uuid = *backup_settings.backup_uuid; + /// Prepare context to use. ContextPtr context_in_use = context; ContextMutablePtr mutable_context; @@ -151,9 +154,9 @@ UUID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & c backup_create_params.compression_method = backup_settings.compression_method; backup_create_params.compression_level = backup_settings.compression_level; backup_create_params.password = backup_settings.password; - backup_create_params.backup_uuid = backup_uuid; backup_create_params.is_internal_backup = backup_settings.internal; backup_create_params.backup_coordination = backup_coordination; + backup_create_params.backup_uuid = backup_uuid; BackupMutablePtr backup = BackupFactory::instance().createBackup(backup_create_params); /// Write the backup. diff --git a/src/Backups/registerBackupEnginesFileAndDisk.cpp b/src/Backups/registerBackupEnginesFileAndDisk.cpp index b1bccb6e914..050a51939b6 100644 --- a/src/Backups/registerBackupEnginesFileAndDisk.cpp +++ b/src/Backups/registerBackupEnginesFileAndDisk.cpp @@ -180,7 +180,7 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory) writer = std::make_shared(path); else writer = std::make_shared(disk, path); - return std::make_unique(backup_name, archive_params, params.base_backup_info, writer, params.context, params.backup_uuid, params.is_internal_backup, params.backup_coordination); + return std::make_unique(backup_name, archive_params, params.base_backup_info, writer, params.context, params.is_internal_backup, params.backup_coordination, params.backup_uuid); } }; diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 3c638efe7cf..e490634e552 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1,4 +1,5 @@ import pytest +import asyncio import re import os.path from helpers.cluster import ClickHouseCluster @@ -323,6 +324,42 @@ def test_async(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" +@pytest.mark.parametrize("interface", ["native", "http"]) +def test_async_backups_to_same_destination(interface): + create_and_fill_table() + backup_name = new_backup_name() + + ids = [] + for _ in range(2): + if interface == "http": + res = instance.http_query(f"BACKUP TABLE test.table TO {backup_name} ASYNC") + else: + res = instance.query(f"BACKUP TABLE test.table TO {backup_name} ASYNC") + ids.append(res.split("\t")[0]) + + [id1, id2] = ids + + assert_eq_with_retry( + instance, + f"SELECT count() FROM system.backups WHERE uuid IN ['{id1}', '{id2}'] AND status != 'BACKUP_COMPLETE' AND status != 'FAILED_TO_BACKUP'", + "0\n", + ) + + assert ( + instance.query(f"SELECT status FROM system.backups WHERE uuid='{id1}'") + == "BACKUP_COMPLETE\n" + ) + + assert ( + instance.query(f"SELECT status FROM system.backups WHERE uuid='{id2}'") + == "FAILED_TO_BACKUP\n" + ) + + instance.query("DROP TABLE test.table") + instance.query(f"RESTORE TABLE test.table FROM {backup_name}") + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + + def test_empty_files_in_backup(): instance.query("CREATE DATABASE test") instance.query( diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 8ba06d9a88c..570e2827871 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -423,6 +423,63 @@ def test_replicated_database_async(): assert node2.query("SELECT * FROM mydb.tbl2 ORDER BY y") == TSV(["a", "bb"]) +@pytest.mark.parametrize( + "interface, on_cluster", [("native", True), ("http", True), ("http", False)] +) +def test_async_backups_to_same_destination(interface, on_cluster): + node1.query( + "CREATE TABLE tbl ON CLUSTER 'cluster' (" + "x UInt8" + ") ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}')" + "ORDER BY x" + ) + + node1.query("INSERT INTO tbl VALUES (1)") + + backup_name = new_backup_name() + + ids = [] + nodes = [node1, node2] + on_cluster_part = "ON CLUSTER 'cluster'" if on_cluster else "" + for node in nodes: + if interface == "http": + res = node.http_query( + f"BACKUP TABLE tbl {on_cluster_part} TO {backup_name} ASYNC" + ) + else: + res = node.query( + f"BACKUP TABLE tbl {on_cluster_part} TO {backup_name} ASYNC" + ) + ids.append(res.split("\t")[0]) + + [id1, id2] = ids + + for i in range(len(nodes)): + assert_eq_with_retry( + nodes[i], + f"SELECT count() FROM system.backups WHERE uuid='{ids[i]}' AND status != 'BACKUP_COMPLETE' AND status != 'FAILED_TO_BACKUP'", + "0\n", + ) + + num_completed_backups = sum( + [ + int( + nodes[i] + .query( + f"SELECT count() FROM system.backups WHERE uuid='{ids[i]}' AND status == 'BACKUP_COMPLETE'" + ) + .strip() + ) + for i in range(len(nodes)) + ] + ) + + assert num_completed_backups == 1 + node1.query("DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY") + node1.query(f"RESTORE TABLE tbl FROM {backup_name}") + assert node1.query("SELECT * FROM tbl") == "1\n" + + def test_required_privileges(): node1.query( "CREATE TABLE tbl ON CLUSTER 'cluster' (" From ce233761d76d616139e2e1e0e69beff07ba55905 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 15 Jul 2022 11:55:13 +0200 Subject: [PATCH 264/659] Fix making a query scope for async backups. --- src/Backups/BackupsWorker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index ce3d47e9a30..09614886f06 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -110,7 +110,7 @@ UUID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & c { if (async) { - query_scope.emplace(context_in_use); + query_scope.emplace(mutable_context); setThreadName("BackupWorker"); } From a06fe4e21b60bf586e017717665e7be3190e16da Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 15 Jul 2022 13:53:20 +0200 Subject: [PATCH 265/659] Fix --- src/IO/ReadWriteBufferFromHTTP.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index ab358c8253a..bbd1f92f0ad 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -551,6 +551,9 @@ namespace detail } } + if (exception) + break; + if (use_external_buffer) { setupExternalBuffer(); From cb6bc63af16a90f8407636811a46b01e1cfac59d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 14:43:27 +0200 Subject: [PATCH 266/659] Better test --- tests/integration/test_concurrent_backups_s3/test.py | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_concurrent_backups_s3/test.py b/tests/integration/test_concurrent_backups_s3/test.py index c262bc9e331..7bf5ea9a5dc 100644 --- a/tests/integration/test_concurrent_backups_s3/test.py +++ b/tests/integration/test_concurrent_backups_s3/test.py @@ -4,7 +4,7 @@ import re import os.path from multiprocessing.dummy import Pool from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry, TSV +from helpers.test_tools import assert_eq_with_retry import time cluster = ClickHouseCluster(__file__) @@ -43,13 +43,5 @@ def test_concurrent_backups(start_cluster): p.map(create_backup, range(40)) - for _ in range(100): - result = node.query( - "SELECT count() FROM system.backups WHERE status != 'BACKUP_COMPLETE' and status != 'FAILED_TO_BACKUP'" - ).strip() - if result == "0": - break - - time.sleep(0.1) - + assert_eq_with_retry(node, "SELECT count() FROM system.backups WHERE status != 'BACKUP_COMPLETE' and status != 'FAILED_TO_BACKUP'", "0", retry_count=100) assert node.query("SELECT count() FROM s3_test where not ignore(*)") == "10000\n" From 6c5de5a867743a1838411bd5b7d2327a21e19653 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 15:34:44 +0200 Subject: [PATCH 267/659] FIX BLACK --- tests/integration/test_concurrent_backups_s3/test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_concurrent_backups_s3/test.py b/tests/integration/test_concurrent_backups_s3/test.py index 7bf5ea9a5dc..608144843d9 100644 --- a/tests/integration/test_concurrent_backups_s3/test.py +++ b/tests/integration/test_concurrent_backups_s3/test.py @@ -43,5 +43,10 @@ def test_concurrent_backups(start_cluster): p.map(create_backup, range(40)) - assert_eq_with_retry(node, "SELECT count() FROM system.backups WHERE status != 'BACKUP_COMPLETE' and status != 'FAILED_TO_BACKUP'", "0", retry_count=100) + assert_eq_with_retry( + node, + "SELECT count() FROM system.backups WHERE status != 'BACKUP_COMPLETE' and status != 'FAILED_TO_BACKUP'", + "0", + retry_count=100, + ) assert node.query("SELECT count() FROM s3_test where not ignore(*)") == "10000\n" From 94b5f1f536195c9b5296a528e91e70b8aef5854c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 15 Jul 2022 15:36:01 +0200 Subject: [PATCH 268/659] disable transactions with Ordinary database --- src/Interpreters/MergeTreeTransaction.cpp | 22 +++++++------------ src/Interpreters/MergeTreeTransaction.h | 1 - .../0_stateless/01133_begin_commit_race.sh | 2 +- .../0_stateless/01167_isolation_hermitage.sh | 2 +- .../0_stateless/01168_mutations_isolation.sh | 2 +- .../01169_alter_partition_isolation_stress.sh | 2 +- .../01170_alter_partition_isolation.sh | 2 +- .../01171_mv_select_insert_isolation_long.sh | 2 +- .../01172_transaction_counters.sql | 2 +- .../01173_transaction_control_queries.sql | 2 ++ .../01174_select_insert_isolation.sh | 2 +- .../02345_implicit_transaction.sql | 2 ++ 12 files changed, 20 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 432116feaf5..18f15f8a250 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -12,16 +12,16 @@ namespace ErrorCodes { extern const int INVALID_TRANSACTION; extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; } -static TableLockHolder getLockForOrdinary(const StoragePtr & storage) +static void checkNotOrdinaryDatabase(const StoragePtr & storage) { if (storage->getStorageID().uuid != UUIDHelpers::Nil) - return {}; + return; - /// Maybe we should just throw an exception and do not support Ordinary database? - auto default_timeout = std::chrono::milliseconds(10 * 1000); - return storage->lockForShare(RWLockImpl::NO_QUERY, default_timeout); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table {} belongs to database with Ordinary engine. " + "This engine is deprecated and is not supported in transactions.", storage->getStorageID().getNameForLogs()); } MergeTreeTransaction::MergeTreeTransaction(CSN snapshot_, LocalTID local_tid_, UUID host_id, std::list::iterator snapshot_it_) @@ -131,18 +131,16 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPartPtr & new_part) { - auto maybe_lock = getLockForOrdinary(storage); + checkNotOrdinaryDatabase(storage); std::lock_guard lock{mutex}; checkIsNotCancelled(); storages.insert(storage); - if (maybe_lock) - table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock)); creating_parts.push_back(new_part); } void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, const TransactionInfoContext & context) { - auto maybe_lock = getLockForOrdinary(storage); + checkNotOrdinaryDatabase(storage); { std::lock_guard lock{mutex}; @@ -151,8 +149,6 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP part_to_remove->version.lockRemovalTID(tid, context); NOEXCEPT_SCOPE; storages.insert(storage); - if (maybe_lock) - table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock)); removing_parts.push_back(part_to_remove); } @@ -161,12 +157,10 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP void MergeTreeTransaction::addMutation(const StoragePtr & table, const String & mutation_id) { - auto maybe_lock = getLockForOrdinary(table); + checkNotOrdinaryDatabase(table); std::lock_guard lock{mutex}; checkIsNotCancelled(); storages.insert(table); - if (maybe_lock) - table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock)); mutations.emplace_back(table, mutation_id); } diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index 7397ea12c12..f2d8d29d244 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -76,7 +76,6 @@ private: /// Lists of changes made by transaction std::unordered_set storages TSA_GUARDED_BY(mutex); - std::vector table_read_locks_for_ordinary_db TSA_GUARDED_BY(mutex); DataPartsVector creating_parts TSA_GUARDED_BY(mutex); DataPartsVector removing_parts TSA_GUARDED_BY(mutex); using RunningMutationsList = std::vector>; diff --git a/tests/queries/0_stateless/01133_begin_commit_race.sh b/tests/queries/0_stateless/01133_begin_commit_race.sh index 7dadb35ccff..2b266527541 100755 --- a/tests/queries/0_stateless/01133_begin_commit_race.sh +++ b/tests/queries/0_stateless/01133_begin_commit_race.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-ordinary-database CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01167_isolation_hermitage.sh b/tests/queries/0_stateless/01167_isolation_hermitage.sh index 7f495801dd0..3f2c8308216 100755 --- a/tests/queries/0_stateless/01167_isolation_hermitage.sh +++ b/tests/queries/0_stateless/01167_isolation_hermitage.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-replicated-database +# Tags: long, no-fasttest, no-replicated-database, no-ordinary-database # Looks like server does not listen https port in fasttest # FIXME Replicated database executes ALTERs in separate context, so transaction info is lost diff --git a/tests/queries/0_stateless/01168_mutations_isolation.sh b/tests/queries/0_stateless/01168_mutations_isolation.sh index 888858edf32..ebfdffdaeee 100755 --- a/tests/queries/0_stateless/01168_mutations_isolation.sh +++ b/tests/queries/0_stateless/01168_mutations_isolation.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-replicated-database +# Tags: no-fasttest, no-replicated-database, no-ordinary-database # Looks like server does not listen https port in fasttest # FIXME Replicated database executes ALTERs in separate context, so transaction info is lost diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index 263b2c84de7..32ad78dead6 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-replicated-database +# Tags: long, no-replicated-database, no-ordinary-database # shellcheck disable=SC2015 diff --git a/tests/queries/0_stateless/01170_alter_partition_isolation.sh b/tests/queries/0_stateless/01170_alter_partition_isolation.sh index 2db178fb6d1..6ac95713800 100755 --- a/tests/queries/0_stateless/01170_alter_partition_isolation.sh +++ b/tests/queries/0_stateless/01170_alter_partition_isolation.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-replicated-database +# Tags: no-fasttest, no-replicated-database, no-ordinary-database # Looks like server does not listen https port in fasttest # FIXME Replicated database executes ALTERs in separate context, so transaction info is lost diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 538f586a673..12b654f4215 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: long, no-parallel, no-ordinary-database # Test is too heavy, avoid parallel run in Flaky Check CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index 8e04b6c89bd..ed40ec3f4b6 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -1,4 +1,4 @@ --- Tags: no-s3-storage, no-tsan +-- Tags: no-s3-storage, no-tsan, no-ordinary-database -- FIXME this test fails with S3 due to a bug in DiskCacheWrapper -- FIXME It became flaky after upgrading to llvm-14 due to obscure freezes in tsan drop table if exists txn_counters; diff --git a/tests/queries/0_stateless/01173_transaction_control_queries.sql b/tests/queries/0_stateless/01173_transaction_control_queries.sql index e23b5ec8657..03c98f50cc4 100644 --- a/tests/queries/0_stateless/01173_transaction_control_queries.sql +++ b/tests/queries/0_stateless/01173_transaction_control_queries.sql @@ -1,3 +1,5 @@ +-- Tags: no-ordinary-database + drop table if exists mt1; drop table if exists mt2; diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index cf1bb23f702..58dadb68117 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-ordinary-database # shellcheck disable=SC2015 diff --git a/tests/queries/0_stateless/02345_implicit_transaction.sql b/tests/queries/0_stateless/02345_implicit_transaction.sql index 677affeec39..e3f9cca37d1 100644 --- a/tests/queries/0_stateless/02345_implicit_transaction.sql +++ b/tests/queries/0_stateless/02345_implicit_transaction.sql @@ -1,3 +1,5 @@ +-- Tags: no-ordinary-database + CREATE TABLE landing (n Int64) engine=MergeTree order by n; CREATE TABLE target (n Int64) engine=MergeTree order by n; CREATE MATERIALIZED VIEW landing_to_target TO target AS From c85b2b5732dfa6bfa07f3c4d1ff77f342c75c9a3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 7 Jul 2022 19:17:07 +0200 Subject: [PATCH 269/659] Add option enabling that SELECT from the system database requires grant. --- programs/server/config.xml | 13 ++++ src/Access/AccessControl.cpp | 13 ++-- src/Access/AccessControl.h | 8 +++ src/Access/AccessRights.cpp | 37 +++++------ src/Access/AccessRights.h | 4 +- src/Access/ContextAccess.cpp | 64 ++++++++++++++++--- .../enable_access_control_improvements.xml | 2 + .../helpers/0_common_instance_config.xml | 2 + 8 files changed, 105 insertions(+), 38 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 203684a9e00..343e8dc7093 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -604,9 +604,22 @@ if this setting is true the user B will see all rows, and if this setting is false the user B will see no rows. By default this setting is false for compatibility with earlier access configurations. --> false + false + + + false + + + false diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index abd481f0bb6..5d3fc558130 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -165,13 +165,12 @@ void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration setNoPasswordAllowed(config_.getBool("allow_no_password", true)); setPlaintextPasswordAllowed(config_.getBool("allow_plaintext_password", true)); - setEnabledUsersWithoutRowPoliciesCanReadRows(config_.getBool( - "access_control_improvements.users_without_row_policies_can_read_rows", - false /* false because we need to be compatible with earlier access configurations */)); - - setOnClusterQueriesRequireClusterGrant(config_.getBool( - "access_control_improvements.on_cluster_queries_require_cluster_grant", - false /* false because we need to be compatible with earlier access configurations */)); + /// Optional improvements in access control system. + /// The default values are false because we need to be compatible with earlier access configurations + setEnabledUsersWithoutRowPoliciesCanReadRows(config_.getBool("access_control_improvements.users_without_row_policies_can_read_rows", false)); + setOnClusterQueriesRequireClusterGrant(config_.getBool("access_control_improvements.on_cluster_queries_require_cluster_grant", false)); + setSelectFromSystemDatabaseRequiresGrant(config_.getBool("access_control_improvements.select_from_system_db_requires_grant", false)); + setSelectFromInformationSchemaDatabaseRequiresGrant(config_.getBool("access_control_improvements.select_from_information_schema_db_requires_grant", false)); addStoragesFromMainConfig(config_, config_path_, get_zookeeper_function_); } diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 22ff0a488f7..e0571ad370e 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -152,6 +152,12 @@ public: void setOnClusterQueriesRequireClusterGrant(bool enable) { on_cluster_queries_require_cluster_grant = enable; } bool doesOnClusterQueriesRequireClusterGrant() const { return on_cluster_queries_require_cluster_grant; } + void setSelectFromSystemDatabaseRequiresGrant(bool enable) { select_from_system_db_requires_grant = enable; } + bool doesSelectFromSystemDatabaseRequireGrant() const { return select_from_system_db_requires_grant; } + + void setSelectFromInformationSchemaDatabaseRequiresGrant(bool enable) { select_from_information_schema_db_requires_grant = enable; } + bool doesSelectFromInformationSchemaDatabaseRequireGrant() const { return select_from_information_schema_db_requires_grant; } + std::shared_ptr getContextAccess( const UUID & user_id, const std::vector & current_roles, @@ -215,6 +221,8 @@ private: std::atomic_bool allow_no_password = true; std::atomic_bool users_without_row_policies_can_read_rows = false; std::atomic_bool on_cluster_queries_require_cluster_grant = false; + std::atomic_bool select_from_system_db_requires_grant = false; + std::atomic_bool select_from_information_schema_db_requires_grant = false; }; } diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index b6fed3ac912..747e7a91b2c 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -388,11 +388,11 @@ public: return res; } - void modifyFlags(const ModifyFlagsFunction & function, bool & flags_added, bool & flags_removed) + void modifyFlags(const ModifyFlagsFunction & function, bool grant_option, bool & flags_added, bool & flags_removed) { flags_added = false; flags_removed = false; - modifyFlagsRec(function, flags_added, flags_removed); + modifyFlagsRec(function, grant_option, flags_added, flags_removed); if (flags_added || flags_removed) optimizeTree(); } @@ -669,11 +669,11 @@ private: } template - void modifyFlagsRec(const ModifyFlagsFunction & function, bool & flags_added, bool & flags_removed, const ParentNames & ... parent_names) + void modifyFlagsRec(const ModifyFlagsFunction & function, bool grant_option, bool & flags_added, bool & flags_removed, const ParentNames & ... parent_names) { - auto invoke = [&function](const AccessFlags & flags_, const AccessFlags & min_flags_with_children_, const AccessFlags & max_flags_with_children_, std::string_view database_ = {}, std::string_view table_ = {}, std::string_view column_ = {}) -> AccessFlags + auto invoke = [function, grant_option](const AccessFlags & flags_, const AccessFlags & min_flags_with_children_, const AccessFlags & max_flags_with_children_, std::string_view database_ = {}, std::string_view table_ = {}, std::string_view column_ = {}) -> AccessFlags { - return function(flags_, min_flags_with_children_, max_flags_with_children_, database_, table_, column_); + return function(flags_, min_flags_with_children_, max_flags_with_children_, database_, table_, column_, grant_option); }; if constexpr (sizeof...(ParentNames) < 3) @@ -683,7 +683,7 @@ private: for (auto & child : *children | boost::adaptors::map_values) { const String & child_name = *child.node_name; - child.modifyFlagsRec(function, flags_added, flags_removed, parent_names..., child_name); + child.modifyFlagsRec(function, grant_option, flags_added, flags_removed, parent_names..., child_name); } } } @@ -1062,24 +1062,21 @@ void AccessRights::modifyFlags(const ModifyFlagsFunction & function) { if (!root) return; + bool flags_added, flags_removed; - root->modifyFlags(function, flags_added, flags_removed); + root->modifyFlags(function, false, flags_added, flags_removed); if (flags_removed && root_with_grant_option) root_with_grant_option->makeIntersection(*root); -} - - -void AccessRights::modifyFlagsWithGrantOption(const ModifyFlagsFunction & function) -{ - if (!root_with_grant_option) - return; - bool flags_added, flags_removed; - root_with_grant_option->modifyFlags(function, flags_added, flags_removed); - if (flags_added) + + if (root_with_grant_option) { - if (!root) - root = std::make_unique(); - root->makeUnion(*root_with_grant_option); + root_with_grant_option->modifyFlags(function, true, flags_added, flags_removed); + if (flags_added) + { + if (!root) + root = std::make_unique(); + root->makeUnion(*root_with_grant_option); + } } } diff --git a/src/Access/AccessRights.h b/src/Access/AccessRights.h index 80e37561c2b..5efffc0037a 100644 --- a/src/Access/AccessRights.h +++ b/src/Access/AccessRights.h @@ -109,9 +109,9 @@ public: const AccessFlags & max_flags_with_children, std::string_view database, std::string_view table, - std::string_view column)>; + std::string_view column, + bool grant_option)>; void modifyFlags(const ModifyFlagsFunction & function); - void modifyFlagsWithGrantOption(const ModifyFlagsFunction & function); friend bool operator ==(const AccessRights & left, const AccessRights & right); friend bool operator !=(const AccessRights & left, const AccessRights & right) { return !(left == right); } diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 92a5179d861..47f6c35ae32 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -44,9 +44,17 @@ namespace } - AccessRights addImplicitAccessRights(const AccessRights & access) + AccessRights addImplicitAccessRights(const AccessRights & access, const AccessControl & access_control) { - auto modifier = [&](const AccessFlags & flags, const AccessFlags & min_flags_with_children, const AccessFlags & max_flags_with_children, std::string_view database, std::string_view table, std::string_view column) -> AccessFlags + AccessFlags max_flags; + + auto modifier = [&](const AccessFlags & flags, + const AccessFlags & min_flags_with_children, + const AccessFlags & max_flags_with_children, + std::string_view database, + std::string_view table, + std::string_view column, + bool /* grant_option */) -> AccessFlags { size_t level = !database.empty() + !table.empty() + !column.empty(); AccessFlags res = flags; @@ -115,17 +123,55 @@ namespace res |= show_databases; } + max_flags |= max_flags_with_children; + return res; }; AccessRights res = access; res.modifyFlags(modifier); - res.modifyFlagsWithGrantOption(modifier); - /// Anyone has access to the "system" and "information_schema" database. - res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE); - res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA); - res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE); + if (access_control.doesSelectFromSystemDatabaseRequireGrant()) + { + res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "one"); + + if (max_flags.contains(AccessType::SHOW_USERS)) + res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "users"); + + if (max_flags.contains(AccessType::SHOW_ROLES)) + res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "roles"); + + if (max_flags.contains(AccessType::SHOW_ROW_POLICIES)) + res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "row_policies"); + + if (max_flags.contains(AccessType::SHOW_SETTINGS_PROFILES)) + res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "settings_profiles"); + + if (max_flags.contains(AccessType::SHOW_QUOTAS)) + res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "quotas"); + + if (max_flags.contains(AccessType::SHOW_COLUMNS)) + res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "columns"); + + if (max_flags.contains(AccessType::SHOW_TABLES)) + res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "tables"); + + if (max_flags.contains(AccessType::SHOW_DATABASES)) + res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "databases"); + } + else + { + /// Anyone has access to the "system" database. + res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE); + } + + if (!access_control.doesSelectFromInformationSchemaDatabaseRequireGrant()) + { + /// Anyone has access to the "information_schema" database. + res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA); + res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE); + } + return res; } @@ -247,7 +293,7 @@ void ContextAccess::setRolesInfo(const std::shared_ptr & void ContextAccess::calculateAccessRights() const { access = std::make_shared(mixAccessRightsFromUserAndRoles(*user, *roles_info)); - access_with_implicit = std::make_shared(addImplicitAccessRights(*access)); + access_with_implicit = std::make_shared(addImplicitAccessRights(*access, *access_control)); if (trace_log) { @@ -342,7 +388,7 @@ std::shared_ptr ContextAccess::getFullAccess() auto full_access = std::shared_ptr(new ContextAccess); full_access->is_full_access = true; full_access->access = std::make_shared(AccessRights::getFullAccess()); - full_access->access_with_implicit = std::make_shared(addImplicitAccessRights(*full_access->access)); + full_access->access_with_implicit = full_access->access; return full_access; }(); return res; diff --git a/tests/config/config.d/enable_access_control_improvements.xml b/tests/config/config.d/enable_access_control_improvements.xml index 052858a9519..3bab0d95144 100644 --- a/tests/config/config.d/enable_access_control_improvements.xml +++ b/tests/config/config.d/enable_access_control_improvements.xml @@ -2,5 +2,7 @@ true true + true + true diff --git a/tests/integration/helpers/0_common_instance_config.xml b/tests/integration/helpers/0_common_instance_config.xml index b6ea21648bb..4bede7767c5 100644 --- a/tests/integration/helpers/0_common_instance_config.xml +++ b/tests/integration/helpers/0_common_instance_config.xml @@ -21,5 +21,7 @@ true + true + true From fbb2e14d543f4871aa2d133c6cc13250277c5a6b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 9 Jul 2022 17:51:59 +0200 Subject: [PATCH 270/659] Add new table function viewIfPermitted(). --- programs/client/Client.cpp | 4 +- src/Client/Suggest.cpp | 76 ++++++------ src/Parsers/ASTFunction.cpp | 19 +++ src/Parsers/ExpressionElementParsers.cpp | 31 ++++- src/Parsers/ExpressionElementParsers.h | 2 +- src/Storages/StorageView.cpp | 10 +- .../TableFunctionViewIfPermitted.cpp | 113 ++++++++++++++++++ .../TableFunctionViewIfPermitted.h | 35 ++++++ src/TableFunctions/registerTableFunctions.cpp | 1 + src/TableFunctions/registerTableFunctions.h | 1 + .../test.py | 35 ++++++ 11 files changed, 281 insertions(+), 46 deletions(-) create mode 100644 src/TableFunctions/TableFunctionViewIfPermitted.cpp create mode 100644 src/TableFunctions/TableFunctionViewIfPermitted.h diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 4e3aa701d95..cf9b7cbafea 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -106,7 +106,9 @@ void Client::processError(const String & query) const std::vector Client::loadWarningMessages() { std::vector messages; - connection->sendQuery(connection_parameters.timeouts, "SELECT message FROM system.warnings", "" /* query_id */, + connection->sendQuery(connection_parameters.timeouts, + "SELECT * FROM viewIfPermitted(SELECT message FROM system.warnings ELSE null('message String'))", + "" /* query_id */, QueryProcessingStage::Complete, &global_context->getSettingsRef(), &global_context->getClientInfo(), false, {}); diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index de09c07f4c1..44e9b1bb735 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -50,52 +50,58 @@ static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggesti { /// NOTE: Once you will update the completion list, /// do not forget to update 01676_clickhouse_client_autocomplete.sh - WriteBufferFromOwnString query; - query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" - "SELECT name FROM system.functions" - " UNION ALL " - "SELECT name FROM system.table_engines" - " UNION ALL " - "SELECT name FROM system.formats" - " UNION ALL " - "SELECT name FROM system.table_functions" - " UNION ALL " - "SELECT name FROM system.data_type_families" - " UNION ALL " - "SELECT name FROM system.merge_tree_settings" - " UNION ALL " - "SELECT name FROM system.settings" - " UNION ALL "; + String query; + + auto add_subquery = [&](std::string_view select, std::string_view result_column_name) + { + if (!query.empty()) + query += " UNION ALL "; + query += fmt::format("SELECT * FROM viewIfPermitted({} ELSE null('{} String'))", select, result_column_name); + }; + + auto add_column = [&](std::string_view column_name, std::string_view table_name, bool distinct, std::optional limit) + { + add_subquery( + fmt::format( + "SELECT {}{} FROM system.{}{}", + (distinct ? "DISTINCT " : ""), + column_name, + table_name, + (limit ? (" LIMIT " + std::to_string(*limit)) : "")), + column_name); + }; + + add_column("name", "functions", false, {}); + add_column("name", "table_engines", false, {}); + add_column("name", "formats", false, {}); + add_column("name", "table_functions", false, {}); + add_column("name", "data_type_families", false, {}); + add_column("name", "merge_tree_settings", false, {}); + add_column("name", "settings", false, {}); + if (!basic_suggestion) { - query << "SELECT cluster FROM system.clusters" - " UNION ALL " - "SELECT macro FROM system.macros" - " UNION ALL " - "SELECT policy_name FROM system.storage_policies" - " UNION ALL "; + add_column("cluster", "clusters", false, {}); + add_column("macro", "macros", false, {}); + add_column("policy_name", "storage_policies", false, {}); } - query << "SELECT concat(func.name, comb.name) FROM system.functions AS func CROSS JOIN system.aggregate_function_combinators AS comb WHERE is_aggregate"; + + add_subquery("SELECT concat(func.name, comb.name) AS x FROM system.functions AS func CROSS JOIN system.aggregate_function_combinators AS comb WHERE is_aggregate", "x"); + /// The user may disable loading of databases, tables, columns by setting suggestion_limit to zero. if (suggestion_limit > 0) { - String limit_str = toString(suggestion_limit); - query << " UNION ALL " - "SELECT name FROM system.databases LIMIT " << limit_str - << " UNION ALL " - "SELECT DISTINCT name FROM system.tables LIMIT " << limit_str - << " UNION ALL "; - + add_column("name", "databases", false, suggestion_limit); + add_column("name", "tables", true, suggestion_limit); if (!basic_suggestion) { - query << "SELECT DISTINCT name FROM system.dictionaries LIMIT " << limit_str - << " UNION ALL "; + add_column("name", "dictionaries", true, suggestion_limit); } - query << "SELECT DISTINCT name FROM system.columns LIMIT " << limit_str; + add_column("name", "columns", true, suggestion_limit); } - query << ") WHERE notEmpty(res)"; - return query.str(); + query = "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" + query + ") WHERE notEmpty(res)"; + return query; } template diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 69927c430dc..39d89f56e91 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -509,6 +509,25 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format settings.ostr << ')'; written = true; } + + if (!written && name == "viewIfPermitted"sv) + { + /// viewIfPermitted() needs special formatting: ELSE instead of comma between arguments, and better indents too. + const auto * nl_or_nothing = settings.one_line ? "" : "\n"; + auto indent0 = settings.one_line ? "" : String(4u * frame.indent, ' '); + auto indent1 = settings.one_line ? "" : String(4u * (frame.indent + 1), ' '); + auto indent2 = settings.one_line ? "" : String(4u * (frame.indent + 2), ' '); + settings.ostr << (settings.hilite ? hilite_function : "") << name << "(" << (settings.hilite ? hilite_none : "") << nl_or_nothing; + FormatStateStacked frame_nested = frame; + frame_nested.need_parens = false; + frame_nested.indent += 2; + arguments->children[0]->formatImpl(settings, state, frame_nested); + settings.ostr << nl_or_nothing << indent1 << (settings.hilite ? hilite_keyword : "") << (settings.one_line ? " " : "") + << "ELSE " << (settings.hilite ? hilite_none : "") << nl_or_nothing << indent2; + arguments->children[1]->formatImpl(settings, state, frame_nested); + settings.ostr << nl_or_nothing << indent0 << ")"; + return; + } } if (!written && arguments->children.size() >= 2) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 045f6aad2b5..bd65305cc52 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1068,13 +1068,16 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool ParserTableFunctionView::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserIdentifier id_parser; - ParserKeyword view("VIEW"); ParserSelectWithUnionQuery select; ASTPtr identifier; ASTPtr query; - if (!view.ignore(pos, expected)) + bool if_permitted = false; + + if (ParserKeyword{"VIEWIFPERMITTED"}.ignore(pos, expected)) + if_permitted = true; + else if (!ParserKeyword{"VIEW"}.ignore(pos, expected)) return false; if (pos->type != TokenType::OpeningRoundBracket) @@ -1094,15 +1097,30 @@ bool ParserTableFunctionView::parseImpl(Pos & pos, ASTPtr & node, Expected & exp return false; } + ASTPtr else_ast; + if (if_permitted) + { + if (!ParserKeyword{"ELSE"}.ignore(pos, expected)) + return false; + + if (!ParserWithOptionalAlias{std::make_unique(true, true), true}.parse(pos, else_ast, expected)) + return false; + } + if (pos->type != TokenType::ClosingRoundBracket) return false; + ++pos; + + auto expr_list = std::make_shared(); + expr_list->children.push_back(query); + if (if_permitted) + expr_list->children.push_back(else_ast); + auto function_node = std::make_shared(); tryGetIdentifierNameInto(identifier, function_node->name); - auto expr_list_with_single_query = std::make_shared(); - expr_list_with_single_query->children.push_back(query); - function_node->name = "view"; - function_node->arguments = expr_list_with_single_query; + function_node->name = if_permitted ? "viewIfPermitted" : "view"; + function_node->arguments = expr_list; function_node->children.push_back(function_node->arguments); node = function_node; return true; @@ -1971,6 +1989,7 @@ const char * ParserAlias::restricted_keywords[] = "WITH", "INTERSECT", "EXCEPT", + "ELSE", nullptr }; diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index f4dfe80f43e..3883631b61c 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -162,7 +162,7 @@ protected: bool is_table_function; }; -// A special function parser for view table function. +// A special function parser for view and viewIfPermitted table functions. // It parses an SELECT query as its argument and doesn't support getColumnName(). class ParserTableFunctionView : public IParserBase { diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index bbbad012547..3377af685f0 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -180,9 +180,13 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_ if (!table_expression->database_and_table_name) { // If it's a view table function, add a fake db.table name. - if (table_expression->table_function && table_expression->table_function->as()->name == "view") - table_expression->database_and_table_name = std::make_shared("__view"); - else + if (table_expression->table_function) + { + auto table_function_name = table_expression->table_function->as()->name; + if ((table_function_name == "view") || (table_function_name == "viewIfPermitted")) + table_expression->database_and_table_name = std::make_shared("__view"); + } + if (!table_expression->database_and_table_name) throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR); } diff --git a/src/TableFunctions/TableFunctionViewIfPermitted.cpp b/src/TableFunctions/TableFunctionViewIfPermitted.cpp new file mode 100644 index 00000000000..72469cf918c --- /dev/null +++ b/src/TableFunctions/TableFunctionViewIfPermitted.cpp @@ -0,0 +1,113 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "registerTableFunctions.h" + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; + extern const int ACCESS_DENIED; +} + + +const ASTSelectWithUnionQuery & TableFunctionViewIfPermitted::getSelectQuery() const +{ + return *create.select; +} + +void TableFunctionViewIfPermitted::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + const auto * function = ast_function->as(); + if (!function || !function->arguments || (function->arguments->children.size() != 2)) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Table function '{}' requires two arguments: a SELECT query and a table function", + getName()); + + const auto & arguments = function->arguments->children; + auto * select = arguments[0]->as(); + if (!select) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}' requires a SELECT query as its first argument", getName()); + create.set(create.select, select->clone()); + + else_ast = arguments[1]; + if (!else_ast->as()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}' requires a table function as its second argument", getName()); + else_table_function = TableFunctionFactory::instance().get(else_ast, context); +} + +ColumnsDescription TableFunctionViewIfPermitted::getActualTableStructure(ContextPtr context) const +{ + return else_table_function->getActualTableStructure(context); +} + +StoragePtr TableFunctionViewIfPermitted::executeImpl( + const ASTPtr & /* ast_function */, ContextPtr context, const std::string & table_name, ColumnsDescription /* cached_columns */) const +{ + StoragePtr storage; + auto columns = getActualTableStructure(context); + + if (isPermitted(context, columns)) + { + storage = std::make_shared(StorageID(getDatabaseName(), table_name), create, columns, ""); + } + else + { + storage = else_table_function->execute(else_ast, context, table_name); + } + + storage->startup(); + return storage; +} + +bool TableFunctionViewIfPermitted::isPermitted(const ContextPtr & context, const ColumnsDescription & else_columns) const +{ + Block sample_block; + + try + { + /// Will throw ACCESS_DENIED if the current user is not allowed to execute the SELECT query. + sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(create.children[0], context); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::ACCESS_DENIED) + return {}; + throw; + } + + /// We check that columns match only if permitted (otherwise we could reveal the structure to an user who must not know it). + ColumnsDescription columns{sample_block.getNamesAndTypesList()}; + if (columns != else_columns) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Table function '{}' requires a SELECT query with the result columns matching a table function after 'ELSE'. " + "Currently the result columns of the SELECT query are {}, and the table function after 'ELSE' gives {}", + getName(), + columns.toString(), + else_columns.toString()); + } + + return true; +} + +void registerTableFunctionViewIfPermitted(TableFunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/TableFunctions/TableFunctionViewIfPermitted.h b/src/TableFunctions/TableFunctionViewIfPermitted.h new file mode 100644 index 00000000000..0fd0b050eaf --- /dev/null +++ b/src/TableFunctions/TableFunctionViewIfPermitted.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/* viewIfPermitted(query ELSE null('structure')) + * Works as "view(query)" if the current user has the permissions required to execute "query"; works as "null('structure')" otherwise. + */ +class TableFunctionViewIfPermitted : public ITableFunction +{ +public: + static constexpr auto name = "viewIfPermitted"; + std::string getName() const override { return name; } + + const ASTSelectWithUnionQuery & getSelectQuery() const; + +private: + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns) const override; + const char * getStorageTypeName() const override { return "ViewIfPermitted"; } + + void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; + ColumnsDescription getActualTableStructure(ContextPtr context) const override; + + bool isPermitted(const ContextPtr & context, const ColumnsDescription & else_columns) const; + + ASTCreateQuery create; + ASTPtr else_ast; + TableFunctionPtr else_table_function; +}; + +} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 12ca4abe113..3ef93c9b69d 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -42,6 +42,7 @@ void registerTableFunctions() registerTableFunctionJDBC(factory); registerTableFunctionView(factory); + registerTableFunctionViewIfPermitted(factory); #if USE_MYSQL registerTableFunctionMySQL(factory); diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 49a1ef60a6b..d7e38403cae 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -40,6 +40,7 @@ void registerTableFunctionODBC(TableFunctionFactory & factory); void registerTableFunctionJDBC(TableFunctionFactory & factory); void registerTableFunctionView(TableFunctionFactory & factory); +void registerTableFunctionViewIfPermitted(TableFunctionFactory & factory); #if USE_MYSQL void registerTableFunctionMySQL(TableFunctionFactory & factory); diff --git a/tests/integration/test_table_functions_access_rights/test.py b/tests/integration/test_table_functions_access_rights/test.py index 705150c8bdd..09a05122c07 100644 --- a/tests/integration/test_table_functions_access_rights/test.py +++ b/tests/integration/test_table_functions_access_rights/test.py @@ -65,3 +65,38 @@ def test_merge(): "it's necessary to have grant SELECT ON default.table2" in instance.query_and_get_error(select_query, user="A") ) + + +def test_view_if_permitted(): + assert ( + instance.query( + "SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('x UInt32'))" + ) + == "1\n" + ) + + expected_error = "requires a SELECT query with the result columns matching a table function after 'ELSE'" + assert expected_error in instance.query_and_get_error( + "SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('x Int32'))" + ) + assert expected_error in instance.query_and_get_error( + "SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('y UInt32'))" + ) + + instance.query("CREATE USER A") + assert ( + instance.query( + "SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('x UInt32'))", + user="A", + ) + == "" + ) + + instance.query("GRANT SELECT ON table1 TO A") + assert ( + instance.query( + "SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('x UInt32'))", + user="A", + ) + == "1\n" + ) From 9e70e025890c43862c115b30b97d5b9c441423c8 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 10 Jul 2022 02:49:45 +0200 Subject: [PATCH 271/659] Table function null() & view() don't require CREATE TEMPORARY TABLE privilege anymore. --- src/TableFunctions/ITableFunction.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index 639240fd105..82b6230dc30 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -23,7 +23,12 @@ StoragePtr ITableFunction::execute(const ASTPtr & ast_function, ContextPtr conte ColumnsDescription cached_columns, bool use_global_context) const { ProfileEvents::increment(ProfileEvents::TableFunctionExecute); - context->checkAccess(AccessType::CREATE_TEMPORARY_TABLE | getSourceAccessType()); + + AccessFlags required_access = getSourceAccessType(); + String function_name = getName(); + if ((function_name != "null") && (function_name != "view") && (function_name != "viewIfPermitted")) + required_access |= AccessType::CREATE_TEMPORARY_TABLE; + context->checkAccess(required_access); auto context_to_use = use_global_context ? context->getGlobalContext() : context; From de34d173ba23d39f165e7c34cf7cf47f74d83a40 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 10 Jul 2022 02:51:31 +0200 Subject: [PATCH 272/659] Remove excessive log messages. --- src/Access/ContextAccess.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 47f6c35ae32..2fc42cf5d0a 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -459,7 +459,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg }; if (is_full_access) - return access_granted(); + return true; if (user_was_dropped) return access_denied("User has been dropped", ErrorCodes::UNKNOWN_USER); @@ -468,7 +468,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg flags &= ~AccessType::CLUSTER; if (!flags) - return access_granted(); + return true; /// Access to temporary tables is controlled in an unusual way, not like normal tables. /// Creating of temporary tables is controlled by AccessType::CREATE_TEMPORARY_TABLES grant, From 6bf7bffbebe6302e8731ccf0d25a454501fa9c5a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 10 Jul 2022 15:15:51 +0200 Subject: [PATCH 273/659] Correct the list of always accessible system tables. --- programs/server/config.xml | 11 ++-- src/Access/AccessControl.cpp | 2 +- src/Access/AccessControl.h | 6 +- src/Access/AccessRights.cpp | 4 +- src/Access/ContextAccess.cpp | 55 ++++++++++++++----- src/Client/Suggest.cpp | 2 +- .../enable_access_control_improvements.xml | 2 +- .../helpers/0_common_instance_config.xml | 3 +- 8 files changed, 56 insertions(+), 29 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 343e8dc7093..40e561c1880 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -611,15 +611,16 @@ + If it's set to true then this query requires "GRANT SELECT ON system." just like as for non-system tables. + Exceptions: a few system tables ("tables", "columns", "databases", and some constant tables like "one", "contributors") + are still accessible for everyone; and if there is a SHOW privilege (e.g. "SHOW USERS") granted the corresponding system + table (i.e. "system.users") will be accessible. --> false - false + If it's set to true then this query requires "GRANT SELECT ON information_schema.
" just like as for ordinary tables. --> + false diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 5d3fc558130..c6729459988 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -170,7 +170,7 @@ void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration setEnabledUsersWithoutRowPoliciesCanReadRows(config_.getBool("access_control_improvements.users_without_row_policies_can_read_rows", false)); setOnClusterQueriesRequireClusterGrant(config_.getBool("access_control_improvements.on_cluster_queries_require_cluster_grant", false)); setSelectFromSystemDatabaseRequiresGrant(config_.getBool("access_control_improvements.select_from_system_db_requires_grant", false)); - setSelectFromInformationSchemaDatabaseRequiresGrant(config_.getBool("access_control_improvements.select_from_information_schema_db_requires_grant", false)); + setSelectFromInformationSchemaRequiresGrant(config_.getBool("access_control_improvements.select_from_information_schema_requires_grant", false)); addStoragesFromMainConfig(config_, config_path_, get_zookeeper_function_); } diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index e0571ad370e..ab9cdba9ad1 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -155,8 +155,8 @@ public: void setSelectFromSystemDatabaseRequiresGrant(bool enable) { select_from_system_db_requires_grant = enable; } bool doesSelectFromSystemDatabaseRequireGrant() const { return select_from_system_db_requires_grant; } - void setSelectFromInformationSchemaDatabaseRequiresGrant(bool enable) { select_from_information_schema_db_requires_grant = enable; } - bool doesSelectFromInformationSchemaDatabaseRequireGrant() const { return select_from_information_schema_db_requires_grant; } + void setSelectFromInformationSchemaRequiresGrant(bool enable) { select_from_information_schema_requires_grant = enable; } + bool doesSelectFromInformationSchemaRequireGrant() const { return select_from_information_schema_requires_grant; } std::shared_ptr getContextAccess( const UUID & user_id, @@ -222,7 +222,7 @@ private: std::atomic_bool users_without_row_policies_can_read_rows = false; std::atomic_bool on_cluster_queries_require_cluster_grant = false; std::atomic_bool select_from_system_db_requires_grant = false; - std::atomic_bool select_from_information_schema_db_requires_grant = false; + std::atomic_bool select_from_information_schema_requires_grant = false; }; } diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 747e7a91b2c..20afc916901 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -1062,12 +1062,12 @@ void AccessRights::modifyFlags(const ModifyFlagsFunction & function) { if (!root) return; - + bool flags_added, flags_removed; root->modifyFlags(function, false, flags_added, flags_removed); if (flags_removed && root_with_grant_option) root_with_grant_option->makeIntersection(*root); - + if (root_with_grant_option) { root_with_grant_option->modifyFlags(function, true, flags_added, flags_removed); diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 2fc42cf5d0a..49736c76994 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -123,7 +123,7 @@ namespace res |= show_databases; } - max_flags |= max_flags_with_children; + max_flags |= res; return res; }; @@ -131,13 +131,48 @@ namespace AccessRights res = access; res.modifyFlags(modifier); + /// If "select_from_system_db_requires_grant" is enabled we provide implicit grants only for a few tables in the system database. if (access_control.doesSelectFromSystemDatabaseRequireGrant()) { - res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "one"); + const char * always_accessible_tables[] = { + /// Constant tables + "one", + + /// "numbers", "numbers_mt", "zeros", "zeros_mt" were excluded because they can generate lots of values and + /// that can decrease performance in some cases. + + "contributors", + "licenses", + "time_zones", + "collations", + + "formats", + "privileges", + "data_type_families", + "table_engines", + "table_functions", + "aggregate_function_combinators", + + "functions", /// Can contain user-defined functions + + /// The following tables hide some rows if the current user doesn't have corresponding SHOW privileges. + "databases", + "tables", + "columns", + + /// Specific to the current session + "settings", + "current_roles", + "enabled_roles", + "quota_usage" + }; + + for (const auto * table_name : always_accessible_tables) + res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, table_name); if (max_flags.contains(AccessType::SHOW_USERS)) res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "users"); - + if (max_flags.contains(AccessType::SHOW_ROLES)) res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "roles"); @@ -149,25 +184,15 @@ namespace if (max_flags.contains(AccessType::SHOW_QUOTAS)) res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "quotas"); - - if (max_flags.contains(AccessType::SHOW_COLUMNS)) - res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "columns"); - - if (max_flags.contains(AccessType::SHOW_TABLES)) - res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "tables"); - - if (max_flags.contains(AccessType::SHOW_DATABASES)) - res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "databases"); } else { - /// Anyone has access to the "system" database. res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE); } - if (!access_control.doesSelectFromInformationSchemaDatabaseRequireGrant()) + /// If "select_from_information_schema_requires_grant" is enabled we don't provide implicit grants for the information_schema database. + if (!access_control.doesSelectFromInformationSchemaRequireGrant()) { - /// Anyone has access to the "information_schema" database. res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA); res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE); } diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 44e9b1bb735..1074adb2bd4 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -87,7 +87,7 @@ static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggesti } add_subquery("SELECT concat(func.name, comb.name) AS x FROM system.functions AS func CROSS JOIN system.aggregate_function_combinators AS comb WHERE is_aggregate", "x"); - + /// The user may disable loading of databases, tables, columns by setting suggestion_limit to zero. if (suggestion_limit > 0) { diff --git a/tests/config/config.d/enable_access_control_improvements.xml b/tests/config/config.d/enable_access_control_improvements.xml index 3bab0d95144..5a186548098 100644 --- a/tests/config/config.d/enable_access_control_improvements.xml +++ b/tests/config/config.d/enable_access_control_improvements.xml @@ -3,6 +3,6 @@ true true true - true + true diff --git a/tests/integration/helpers/0_common_instance_config.xml b/tests/integration/helpers/0_common_instance_config.xml index 4bede7767c5..64f0ce9e361 100644 --- a/tests/integration/helpers/0_common_instance_config.xml +++ b/tests/integration/helpers/0_common_instance_config.xml @@ -21,7 +21,8 @@ true + true true - true + true From ef3a24d20fe6851b9e4e557b0af8a2b26c89e6b2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 10 Jul 2022 18:15:32 +0200 Subject: [PATCH 274/659] Fix tests. --- tests/integration/test_backup_restore_on_cluster/test.py | 2 ++ .../configs/config_allow_databases.xml | 1 + tests/queries/0_stateless/00600_replace_running_query.sh | 1 + tests/queries/0_stateless/01317_no_password_in_command_line.sh | 1 + 4 files changed, 5 insertions(+) diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 8ba06d9a88c..a0e8758ddc5 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -434,6 +434,7 @@ def test_required_privileges(): node1.query("INSERT INTO tbl VALUES (100)") node1.query("CREATE USER u1") + node1.query("GRANT CLUSTER ON *.* TO u1") backup_name = new_backup_name() expected_error = "necessary to have grant BACKUP ON default.tbl" @@ -478,6 +479,7 @@ def test_system_users(): backup_name = new_backup_name() node1.query("CREATE USER u2 SETTINGS allow_backup=false") + node1.query("GRANT CLUSTER ON *.* TO u2") expected_error = "necessary to have grant BACKUP ON system.users" assert expected_error in node1.query_and_get_error( diff --git a/tests/integration/test_config_substitutions/configs/config_allow_databases.xml b/tests/integration/test_config_substitutions/configs/config_allow_databases.xml index be727360dcf..ba38a4f250a 100644 --- a/tests/integration/test_config_substitutions/configs/config_allow_databases.xml +++ b/tests/integration/test_config_substitutions/configs/config_allow_databases.xml @@ -19,6 +19,7 @@ default + system diff --git a/tests/queries/0_stateless/00600_replace_running_query.sh b/tests/queries/0_stateless/00600_replace_running_query.sh index 89c9d1c4279..6a682210489 100755 --- a/tests/queries/0_stateless/00600_replace_running_query.sh +++ b/tests/queries/0_stateless/00600_replace_running_query.sh @@ -9,6 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -q "drop user if exists u_00600" ${CLICKHOUSE_CLIENT} -q "create user u_00600 settings max_execution_time=60, readonly=1" +${CLICKHOUSE_CLIENT} -q "grant select on system.numbers to u_00600" function wait_for_query_to_start() { diff --git a/tests/queries/0_stateless/01317_no_password_in_command_line.sh b/tests/queries/0_stateless/01317_no_password_in_command_line.sh index 5b95f077ea2..7f2e91201a3 100755 --- a/tests/queries/0_stateless/01317_no_password_in_command_line.sh +++ b/tests/queries/0_stateless/01317_no_password_in_command_line.sh @@ -10,6 +10,7 @@ set -e user=user_$CLICKHOUSE_TEST_UNIQUE_NAME $CLICKHOUSE_CLIENT --query "DROP USER IF EXISTS $user" $CLICKHOUSE_CLIENT --query "CREATE USER $user IDENTIFIED WITH PLAINTEXT_PASSWORD BY 'hello'" +$CLICKHOUSE_CLIENT --query "GRANT SELECT ON system.numbers TO $user" trap '$CLICKHOUSE_CLIENT --query "DROP USER $user"' EXIT # Wait for query to start executing. At that time, the password should be cleared. From c7cef91d4d48e24f1a082ab096caba0818f8eba9 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 11 Jul 2022 18:21:44 +0200 Subject: [PATCH 275/659] Two ways to provide access to system.users: "GRANT SHOW USERS ON *.*" and "GRANT SELECT ON system.users" --- src/Storages/System/StorageSystemGrants.cpp | 5 ++++- src/Storages/System/StorageSystemQuotaLimits.cpp | 5 ++++- src/Storages/System/StorageSystemQuotaUsage.cpp | 6 +++++- src/Storages/System/StorageSystemQuotas.cpp | 5 ++++- src/Storages/System/StorageSystemQuotasUsage.cpp | 6 +++++- src/Storages/System/StorageSystemRoleGrants.cpp | 5 ++++- src/Storages/System/StorageSystemRoles.cpp | 5 ++++- src/Storages/System/StorageSystemRowPolicies.cpp | 5 ++++- .../System/StorageSystemSettingsProfileElements.cpp | 5 ++++- src/Storages/System/StorageSystemSettingsProfiles.cpp | 5 ++++- src/Storages/System/StorageSystemUsers.cpp | 5 ++++- 11 files changed, 46 insertions(+), 11 deletions(-) diff --git a/src/Storages/System/StorageSystemGrants.cpp b/src/Storages/System/StorageSystemGrants.cpp index 26bd241023a..461efd7f640 100644 --- a/src/Storages/System/StorageSystemGrants.cpp +++ b/src/Storages/System/StorageSystemGrants.cpp @@ -36,8 +36,11 @@ NamesAndTypesList StorageSystemGrants::getNamesAndTypes() void StorageSystemGrants::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES); + /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); + if (!access_control.doesSelectFromSystemDatabaseRequireGrant()) + context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES); + std::vector ids = access_control.findAll(); boost::range::push_back(ids, access_control.findAll()); diff --git a/src/Storages/System/StorageSystemQuotaLimits.cpp b/src/Storages/System/StorageSystemQuotaLimits.cpp index c98e060a62f..0261d3d2cd9 100644 --- a/src/Storages/System/StorageSystemQuotaLimits.cpp +++ b/src/Storages/System/StorageSystemQuotaLimits.cpp @@ -66,8 +66,11 @@ NamesAndTypesList StorageSystemQuotaLimits::getNamesAndTypes() void StorageSystemQuotaLimits::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - context->checkAccess(AccessType::SHOW_QUOTAS); + /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); + if (!access_control.doesSelectFromSystemDatabaseRequireGrant()) + context->checkAccess(AccessType::SHOW_QUOTAS); + std::vector ids = access_control.findAll(); size_t column_index = 0; diff --git a/src/Storages/System/StorageSystemQuotaUsage.cpp b/src/Storages/System/StorageSystemQuotaUsage.cpp index 54f403803d6..6ba47a86dbf 100644 --- a/src/Storages/System/StorageSystemQuotaUsage.cpp +++ b/src/Storages/System/StorageSystemQuotaUsage.cpp @@ -78,7 +78,11 @@ NamesAndTypesList StorageSystemQuotaUsage::getNamesAndTypesImpl(bool add_column_ void StorageSystemQuotaUsage::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - context->checkAccess(AccessType::SHOW_QUOTAS); + /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. + const auto & access_control = context->getAccessControl(); + if (!access_control.doesSelectFromSystemDatabaseRequireGrant()) + context->checkAccess(AccessType::SHOW_QUOTAS); + auto usage = context->getQuotaUsage(); if (!usage) return; diff --git a/src/Storages/System/StorageSystemQuotas.cpp b/src/Storages/System/StorageSystemQuotas.cpp index 046db151684..17863fa7326 100644 --- a/src/Storages/System/StorageSystemQuotas.cpp +++ b/src/Storages/System/StorageSystemQuotas.cpp @@ -53,8 +53,11 @@ NamesAndTypesList StorageSystemQuotas::getNamesAndTypes() void StorageSystemQuotas::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - context->checkAccess(AccessType::SHOW_QUOTAS); + /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); + if (!access_control.doesSelectFromSystemDatabaseRequireGrant()) + context->checkAccess(AccessType::SHOW_QUOTAS); + std::vector ids = access_control.findAll(); size_t column_index = 0; diff --git a/src/Storages/System/StorageSystemQuotasUsage.cpp b/src/Storages/System/StorageSystemQuotasUsage.cpp index fae0629a209..a3c97247111 100644 --- a/src/Storages/System/StorageSystemQuotasUsage.cpp +++ b/src/Storages/System/StorageSystemQuotasUsage.cpp @@ -15,7 +15,11 @@ NamesAndTypesList StorageSystemQuotasUsage::getNamesAndTypes() void StorageSystemQuotasUsage::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - context->checkAccess(AccessType::SHOW_QUOTAS); + /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. + const auto & access_control = context->getAccessControl(); + if (!access_control.doesSelectFromSystemDatabaseRequireGrant()) + context->checkAccess(AccessType::SHOW_QUOTAS); + auto all_quotas_usage = context->getAccessControl().getAllQuotasUsage(); StorageSystemQuotaUsage::fillDataImpl(res_columns, context, /* add_column_is_current = */ true, all_quotas_usage); } diff --git a/src/Storages/System/StorageSystemRoleGrants.cpp b/src/Storages/System/StorageSystemRoleGrants.cpp index 94ee28cfe83..cf5a24f88cd 100644 --- a/src/Storages/System/StorageSystemRoleGrants.cpp +++ b/src/Storages/System/StorageSystemRoleGrants.cpp @@ -31,8 +31,11 @@ NamesAndTypesList StorageSystemRoleGrants::getNamesAndTypes() void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES); + /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); + if (!access_control.doesSelectFromSystemDatabaseRequireGrant()) + context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES); + std::vector ids = access_control.findAll(); boost::range::push_back(ids, access_control.findAll()); diff --git a/src/Storages/System/StorageSystemRoles.cpp b/src/Storages/System/StorageSystemRoles.cpp index e5b8d53ce7e..5fda021428a 100644 --- a/src/Storages/System/StorageSystemRoles.cpp +++ b/src/Storages/System/StorageSystemRoles.cpp @@ -27,8 +27,11 @@ NamesAndTypesList StorageSystemRoles::getNamesAndTypes() void StorageSystemRoles::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - context->checkAccess(AccessType::SHOW_ROLES); + /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); + if (!access_control.doesSelectFromSystemDatabaseRequireGrant()) + context->checkAccess(AccessType::SHOW_ROLES); + std::vector ids = access_control.findAll(); size_t column_index = 0; diff --git a/src/Storages/System/StorageSystemRowPolicies.cpp b/src/Storages/System/StorageSystemRowPolicies.cpp index 064f610730d..c0bc38edc21 100644 --- a/src/Storages/System/StorageSystemRowPolicies.cpp +++ b/src/Storages/System/StorageSystemRowPolicies.cpp @@ -53,8 +53,11 @@ NamesAndTypesList StorageSystemRowPolicies::getNamesAndTypes() void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - context->checkAccess(AccessType::SHOW_ROW_POLICIES); + /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); + if (!access_control.doesSelectFromSystemDatabaseRequireGrant()) + context->checkAccess(AccessType::SHOW_ROW_POLICIES); + std::vector ids = access_control.findAll(); size_t column_index = 0; diff --git a/src/Storages/System/StorageSystemSettingsProfileElements.cpp b/src/Storages/System/StorageSystemSettingsProfileElements.cpp index 8013a3f2e9e..565ff5e471e 100644 --- a/src/Storages/System/StorageSystemSettingsProfileElements.cpp +++ b/src/Storages/System/StorageSystemSettingsProfileElements.cpp @@ -37,8 +37,11 @@ NamesAndTypesList StorageSystemSettingsProfileElements::getNamesAndTypes() void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES); + /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); + if (!access_control.doesSelectFromSystemDatabaseRequireGrant()) + context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES); + std::vector ids = access_control.findAll(); boost::range::push_back(ids, access_control.findAll()); boost::range::push_back(ids, access_control.findAll()); diff --git a/src/Storages/System/StorageSystemSettingsProfiles.cpp b/src/Storages/System/StorageSystemSettingsProfiles.cpp index d03848ba68b..069c8762154 100644 --- a/src/Storages/System/StorageSystemSettingsProfiles.cpp +++ b/src/Storages/System/StorageSystemSettingsProfiles.cpp @@ -34,8 +34,11 @@ NamesAndTypesList StorageSystemSettingsProfiles::getNamesAndTypes() void StorageSystemSettingsProfiles::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES); + /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); + if (!access_control.doesSelectFromSystemDatabaseRequireGrant()) + context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES); + std::vector ids = access_control.findAll(); size_t column_index = 0; diff --git a/src/Storages/System/StorageSystemUsers.cpp b/src/Storages/System/StorageSystemUsers.cpp index be56abfa3e8..d7cdf280d14 100644 --- a/src/Storages/System/StorageSystemUsers.cpp +++ b/src/Storages/System/StorageSystemUsers.cpp @@ -60,8 +60,11 @@ NamesAndTypesList StorageSystemUsers::getNamesAndTypes() void StorageSystemUsers::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - context->checkAccess(AccessType::SHOW_USERS); + /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); + if (!access_control.doesSelectFromSystemDatabaseRequireGrant()) + context->checkAccess(AccessType::SHOW_USERS); + std::vector ids = access_control.findAll(); size_t column_index = 0; From 5691a859d6228100537cdd424cbbb857c00f064f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 11 Jul 2022 18:22:21 +0200 Subject: [PATCH 276/659] Add tests. --- .../disable_access_control_improvements.xml | 2 + .../configs/users.d/another_user.xml | 3 + .../test_select_from_system_tables.py | 162 +++++++++++++++ .../configs/another_user.xml | 16 ++ .../test_select_from_system_tables.py | 192 ++++++++++++++++++ 5 files changed, 375 insertions(+) create mode 100644 tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py create mode 100644 tests/integration/test_select_access_rights/configs/another_user.xml create mode 100644 tests/integration/test_select_access_rights/test_select_from_system_tables.py diff --git a/tests/integration/test_disabled_access_control_improvements/configs/config.d/disable_access_control_improvements.xml b/tests/integration/test_disabled_access_control_improvements/configs/config.d/disable_access_control_improvements.xml index 0192e211b68..7969c638fd7 100644 --- a/tests/integration/test_disabled_access_control_improvements/configs/config.d/disable_access_control_improvements.xml +++ b/tests/integration/test_disabled_access_control_improvements/configs/config.d/disable_access_control_improvements.xml @@ -1,5 +1,7 @@ + + diff --git a/tests/integration/test_disabled_access_control_improvements/configs/users.d/another_user.xml b/tests/integration/test_disabled_access_control_improvements/configs/users.d/another_user.xml index 19249011968..476072bd138 100644 --- a/tests/integration/test_disabled_access_control_improvements/configs/users.d/another_user.xml +++ b/tests/integration/test_disabled_access_control_improvements/configs/users.d/another_user.xml @@ -13,6 +13,9 @@ default default + + mydb + diff --git a/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py b/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py new file mode 100644 index 00000000000..5d760c9fc2c --- /dev/null +++ b/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py @@ -0,0 +1,162 @@ +import os +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=["configs/config.d/disable_access_control_improvements.xml"], + user_configs=[ + "configs/users.d/another_user.xml", + ], +) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + node.query("CREATE DATABASE mydb") + node.query("CREATE TABLE mydb.table1(x UInt32) ENGINE=Log") + node.query("CREATE TABLE table2(x UInt32) ENGINE=Log") + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def reset_after_test(): + try: + node.query("CREATE USER OR REPLACE sqluser") + yield + finally: + pass + + +def test_system_db(): + assert node.query("SELECT count()>0 FROM system.settings") == "1\n" + assert node.query("SELECT count()>0 FROM system.users") == "1\n" + assert node.query("SELECT count()>0 FROM system.clusters") == "1\n" + assert node.query("SELECT count() FROM system.tables WHERE name='table1'") == "1\n" + assert node.query("SELECT count() FROM system.tables WHERE name='table2'") == "1\n" + + assert node.query("SELECT count()>0 FROM system.settings", user="another") == "1\n" + expected_error = "necessary to have grant SHOW USERS ON *.*" + assert expected_error in node.query_and_get_error( + "SELECT count()>0 FROM system.users", user="another" + ) + assert node.query("SELECT count()>0 FROM system.clusters", user="another") == "1\n" + assert ( + node.query( + "SELECT count() FROM system.tables WHERE name='table1'", user="another" + ) + == "1\n" + ) + assert ( + node.query( + "SELECT count() FROM system.tables WHERE name='table2'", user="another" + ) + == "0\n" + ) + + assert node.query("SELECT count()>0 FROM system.settings", user="sqluser") == "1\n" + expected_error = "necessary to have grant SHOW USERS ON *.*" + assert expected_error in node.query_and_get_error( + "SELECT count()>0 FROM system.users", user="sqluser" + ) + assert node.query("SELECT count()>0 FROM system.clusters", user="sqluser") == "1\n" + assert ( + node.query( + "SELECT count() FROM system.tables WHERE name='table1'", user="sqluser" + ) + == "0\n" + ) + assert ( + node.query( + "SELECT count() FROM system.tables WHERE name='table2'", user="sqluser" + ) + == "0\n" + ) + + node.query("GRANT SHOW USERS ON *.* TO sqluser") + node.query("GRANT SHOW ON mydb.table1 TO sqluser") + node.query("GRANT SHOW ON table2 TO sqluser") + assert node.query("SELECT count()>0 FROM system.settings", user="sqluser") == "1\n" + assert node.query("SELECT count()>0 FROM system.users", user="sqluser") == "1\n" + assert node.query("SELECT count()>0 FROM system.clusters", user="sqluser") == "1\n" + assert ( + node.query( + "SELECT count() FROM system.tables WHERE name='table1'", user="sqluser" + ) + == "1\n" + ) + assert ( + node.query( + "SELECT count() FROM system.tables WHERE name='table2'", user="sqluser" + ) + == "1\n" + ) + + +def test_information_schema(): + assert ( + node.query( + "SELECT count() FROM information_schema.tables WHERE table_name='table1'" + ) + == "1\n" + ) + assert ( + node.query( + "SELECT count() FROM information_schema.tables WHERE table_name='table2'" + ) + == "1\n" + ) + + assert ( + node.query( + "SELECT count() FROM information_schema.tables WHERE table_name='table1'", + user="another", + ) + == "1\n" + ) + assert ( + node.query( + "SELECT count() FROM information_schema.tables WHERE table_name='table2'", + user="another", + ) + == "0\n" + ) + + assert ( + node.query( + "SELECT count() FROM information_schema.tables WHERE table_name='table1'", + user="sqluser", + ) + == "0\n" + ) + assert ( + node.query( + "SELECT count() FROM information_schema.tables WHERE table_name='table2'", + user="sqluser", + ) + == "0\n" + ) + + node.query("GRANT SHOW ON mydb.table1 TO sqluser") + node.query("GRANT SHOW ON table2 TO sqluser") + assert ( + node.query( + "SELECT count() FROM information_schema.tables WHERE table_name='table1'", + user="sqluser", + ) + == "1\n" + ) + assert ( + node.query( + "SELECT count() FROM information_schema.tables WHERE table_name='table2'", + user="sqluser", + ) + == "1\n" + ) diff --git a/tests/integration/test_select_access_rights/configs/another_user.xml b/tests/integration/test_select_access_rights/configs/another_user.xml new file mode 100644 index 00000000000..627ebccdada --- /dev/null +++ b/tests/integration/test_select_access_rights/configs/another_user.xml @@ -0,0 +1,16 @@ + + + + + + + ::/0 + + default + default + + mydb + + + + diff --git a/tests/integration/test_select_access_rights/test_select_from_system_tables.py b/tests/integration/test_select_access_rights/test_select_from_system_tables.py new file mode 100644 index 00000000000..ac938a9694a --- /dev/null +++ b/tests/integration/test_select_access_rights/test_select_from_system_tables.py @@ -0,0 +1,192 @@ +import os +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + user_configs=[ + "configs/another_user.xml", + ], +) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + node.query("CREATE DATABASE mydb") + node.query("CREATE TABLE mydb.table1(x UInt32) ENGINE=Log") + node.query("CREATE TABLE table2(x UInt32) ENGINE=Log") + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def reset_after_test(): + try: + node.query("CREATE USER OR REPLACE sqluser") + yield + finally: + pass + + +def test_system_db(): + assert node.query("SELECT count()>0 FROM system.settings") == "1\n" + assert node.query("SELECT count()>0 FROM system.users") == "1\n" + assert node.query("SELECT count()>0 FROM system.clusters") == "1\n" + assert node.query("SELECT count() FROM system.tables WHERE name='table1'") == "1\n" + assert node.query("SELECT count() FROM system.tables WHERE name='table2'") == "1\n" + + assert node.query("SELECT count()>0 FROM system.settings", user="another") == "1\n" + + expected_error = ( + "necessary to have grant SELECT for at least one column on system.users" + ) + assert expected_error in node.query_and_get_error( + "SELECT count()>0 FROM system.users", user="another" + ) + + expected_error = ( + "necessary to have grant SELECT for at least one column on system.clusters" + ) + assert expected_error in node.query_and_get_error( + "SELECT count()>0 FROM system.clusters", user="another" + ) + assert ( + node.query( + "SELECT count() FROM system.tables WHERE name='table1'", user="another" + ) + == "1\n" + ) + assert ( + node.query( + "SELECT count() FROM system.tables WHERE name='table2'", user="another" + ) + == "0\n" + ) + + assert node.query("SELECT count()>0 FROM system.settings", user="sqluser") == "1\n" + + expected_error = ( + "necessary to have grant SELECT for at least one column on system.users" + ) + assert expected_error in node.query_and_get_error( + "SELECT count()>0 FROM system.users", user="sqluser" + ) + + expected_error = ( + "necessary to have grant SELECT for at least one column on system.clusters" + ) + assert node.query_and_get_error( + "SELECT count()>0 FROM system.clusters", user="sqluser" + ) + + assert ( + node.query( + "SELECT count() FROM system.tables WHERE name='table1'", user="sqluser" + ) + == "0\n" + ) + assert ( + node.query( + "SELECT count() FROM system.tables WHERE name='table2'", user="sqluser" + ) + == "0\n" + ) + + node.query("GRANT SELECT ON system.users TO sqluser") + node.query("GRANT SELECT ON system.clusters TO sqluser") + node.query("GRANT SHOW ON mydb.table1 TO sqluser") + node.query("GRANT SHOW ON table2 TO sqluser") + assert node.query("SELECT count()>0 FROM system.settings", user="sqluser") == "1\n" + assert node.query("SELECT count()>0 FROM system.users", user="sqluser") == "1\n" + assert node.query("SELECT count()>0 FROM system.clusters", user="sqluser") == "1\n" + assert ( + node.query( + "SELECT count() FROM system.tables WHERE name='table1'", user="sqluser" + ) + == "1\n" + ) + assert ( + node.query( + "SELECT count() FROM system.tables WHERE name='table2'", user="sqluser" + ) + == "1\n" + ) + + node.query("REVOKE ALL ON *.* FROM sqluser") + node.query("GRANT SHOW USERS ON *.* TO sqluser") + assert node.query("SELECT count()>0 FROM system.users", user="sqluser") == "1\n" + + +def test_information_schema(): + assert ( + node.query( + "SELECT count() FROM information_schema.tables WHERE table_name='table1'" + ) + == "1\n" + ) + assert ( + node.query( + "SELECT count() FROM information_schema.tables WHERE table_name='table2'" + ) + == "1\n" + ) + + expected_error = ( + "necessary to have grant SELECT(table_name) ON information_schema.tables" + ) + assert expected_error in node.query_and_get_error( + "SELECT count() FROM information_schema.tables WHERE table_name='table1'", + user="another", + ) + assert expected_error in node.query_and_get_error( + "SELECT count() FROM information_schema.tables WHERE table_name='table2'", + user="another", + ) + + assert expected_error in node.query_and_get_error( + "SELECT count() FROM information_schema.tables WHERE table_name='table1'", + user="sqluser", + ) + assert expected_error in node.query_and_get_error( + "SELECT count() FROM information_schema.tables WHERE table_name='table2'", + user="sqluser", + ) + + node.query("GRANT SELECT ON information_schema.* TO sqluser") + assert ( + node.query( + "SELECT count() FROM information_schema.tables WHERE table_name='table1'", + user="sqluser", + ) + == "0\n" + ) + assert ( + node.query( + "SELECT count() FROM information_schema.tables WHERE table_name='table2'", + user="sqluser", + ) + == "0\n" + ) + + node.query("GRANT SHOW ON mydb.table1 TO sqluser") + node.query("GRANT SHOW ON table2 TO sqluser") + assert ( + node.query( + "SELECT count() FROM information_schema.tables WHERE table_name='table1'", + user="sqluser", + ) + == "1\n" + ) + assert ( + node.query( + "SELECT count() FROM information_schema.tables WHERE table_name='table2'", + user="sqluser", + ) + == "1\n" + ) From 3eb847f449dc1e55f883a24fe7083362246bf0f1 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 15 Jul 2022 14:43:15 +0200 Subject: [PATCH 277/659] Small correction. --- src/TableFunctions/TableFunctionViewIfPermitted.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionViewIfPermitted.cpp b/src/TableFunctions/TableFunctionViewIfPermitted.cpp index 72469cf918c..dbc4d40d079 100644 --- a/src/TableFunctions/TableFunctionViewIfPermitted.cpp +++ b/src/TableFunctions/TableFunctionViewIfPermitted.cpp @@ -85,7 +85,7 @@ bool TableFunctionViewIfPermitted::isPermitted(const ContextPtr & context, const catch (Exception & e) { if (e.code() == ErrorCodes::ACCESS_DENIED) - return {}; + return false; throw; } From aea045f29775143eed1cf8b1d20714956ad29d84 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 14 Jul 2022 16:22:06 +0200 Subject: [PATCH 278/659] Improve logging around replicated merges --- src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 13 +++++++++++++ src/Storages/MergeTree/MergeFromLogEntryTask.h | 7 ++++--- src/Storages/MergeTree/MergeList.cpp | 3 ++- src/Storages/MergeTree/MergeList.h | 3 +++ src/Storages/MergeTree/MergePlainMergeTreeTask.h | 5 ++--- src/Storages/MergeTree/MergeTreeData.h | 4 ++-- src/Storages/MergeTree/MutatePlainMergeTreeTask.h | 5 ++--- .../MergeTree/ReplicatedMergeMutateTaskBase.h | 7 ++++--- src/Storages/StorageMergeTree.cpp | 3 ++- 9 files changed, 34 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 0e99d6ce04e..d51cd6aa07d 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -19,6 +19,19 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +MergeFromLogEntryTask::MergeFromLogEntryTask( + ReplicatedMergeTreeQueue::SelectedEntryPtr selected_entry_, + StorageReplicatedMergeTree & storage_, + IExecutableTask::TaskResultCallback & task_result_callback_) + : ReplicatedMergeMutateTaskBase( + &Poco::Logger::get( + storage_.getStorageID().getShortName() + "::" + selected_entry_->log_entry->new_part_name + " (MergeFromLogEntryTask)"), + storage_, + selected_entry_, + task_result_callback_) +{ +} + ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() { diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.h b/src/Storages/MergeTree/MergeFromLogEntryTask.h index d8d2d86148b..9d870cd9a4b 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.h +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.h @@ -17,9 +17,10 @@ namespace DB class MergeFromLogEntryTask : public ReplicatedMergeMutateTaskBase { public: - template - MergeFromLogEntryTask(ReplicatedMergeTreeQueue::SelectedEntryPtr selected_entry_, StorageReplicatedMergeTree & storage_, Callback && task_result_callback_) - : ReplicatedMergeMutateTaskBase(&Poco::Logger::get("MergeFromLogEntryTask"), storage_, selected_entry_, task_result_callback_) {} + MergeFromLogEntryTask( + ReplicatedMergeTreeQueue::SelectedEntryPtr selected_entry_, + StorageReplicatedMergeTree & storage_, + IExecutableTask::TaskResultCallback & task_result_callback_); UInt64 getPriority() override { return priority; } diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index d866345defe..83bcf007928 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -60,6 +60,7 @@ MergeListElement::MergeListElement( , thread_id{getThreadId()} , merge_type{future_part->merge_type} , merge_algorithm{MergeAlgorithm::Undecided} + , description{"to apply mutate/merge in " + query_id} { for (const auto & source_part : future_part->parts) { @@ -77,7 +78,7 @@ MergeListElement::MergeListElement( is_mutation = (result_part_info.getDataVersion() != source_data_version); } - memory_tracker.setDescription("Mutate/Merge"); + memory_tracker.setDescription(description.c_str()); /// MemoryTracker settings should be set here, because /// later (see MemoryTrackerThreadSwitcher) /// parent memory tracker will be changed, and if merge executed from the diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index ac1db503d9b..da6c5184619 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -126,6 +126,9 @@ struct MergeListElement : boost::noncopyable /// Detected after merge already started std::atomic merge_algorithm; + // Description used for logging + const String description{"Mutate/Merge"}; + MergeListElement( const StorageID & table_id_, FutureMergedMutatedPartPtr future_part, diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.h b/src/Storages/MergeTree/MergePlainMergeTreeTask.h index 0f6d38d2cbf..549674060db 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.h @@ -13,7 +13,6 @@ class StorageMergeTree; class MergePlainMergeTreeTask : public IExecutableTask { public: - template MergePlainMergeTreeTask( StorageMergeTree & storage_, StorageMetadataPtr metadata_snapshot_, @@ -21,14 +20,14 @@ public: Names deduplicate_by_columns_, MergeMutateSelectedEntryPtr merge_mutate_entry_, TableLockHolder table_lock_holder_, - Callback && task_result_callback_) + IExecutableTask::TaskResultCallback & task_result_callback_) : storage(storage_) , metadata_snapshot(std::move(metadata_snapshot_)) , deduplicate(deduplicate_) , deduplicate_by_columns(std::move(deduplicate_by_columns_)) , merge_mutate_entry(std::move(merge_mutate_entry_)) , table_lock_holder(std::move(table_lock_holder_)) - , task_result_callback(std::forward(task_result_callback_)) + , task_result_callback(task_result_callback_) { for (auto & item : merge_mutate_entry->future_part->parts) priority += item->getBytesOnDisk(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 0b6e757ab49..68438760ee5 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1094,8 +1094,8 @@ protected: /// Strongly connected with two fields above. /// Every task that is finished will ask to assign a new one into an executor. /// These callbacks will be passed to the constructor of each task. - std::function common_assignee_trigger; - std::function moves_assignee_trigger; + IExecutableTask::TaskResultCallback common_assignee_trigger; + IExecutableTask::TaskResultCallback moves_assignee_trigger; using DataPartIteratorByInfo = DataPartsIndexes::index::type::iterator; using DataPartIteratorByStateAndInfo = DataPartsIndexes::index::type::iterator; diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h index fb3c8318418..e2b019c08ce 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h @@ -22,18 +22,17 @@ class StorageMergeTree; class MutatePlainMergeTreeTask : public IExecutableTask { public: - template MutatePlainMergeTreeTask( StorageMergeTree & storage_, StorageMetadataPtr metadata_snapshot_, MergeMutateSelectedEntryPtr merge_mutate_entry_, TableLockHolder table_lock_holder_, - Callback && task_result_callback_) + IExecutableTask::TaskResultCallback & task_result_callback_) : storage(storage_) , metadata_snapshot(std::move(metadata_snapshot_)) , merge_mutate_entry(std::move(merge_mutate_entry_)) , table_lock_holder(std::move(table_lock_holder_)) - , task_result_callback(std::forward(task_result_callback_)) + , task_result_callback(task_result_callback_) { for (auto & part : merge_mutate_entry->future_part->parts) priority += part->getBytesOnDisk(); diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h index d176f97d929..d8495d35d90 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h @@ -16,19 +16,20 @@ class StorageReplicatedMergeTree; class ReplicatedMergeMutateTaskBase : public IExecutableTask { public: - template ReplicatedMergeMutateTaskBase( Poco::Logger * log_, StorageReplicatedMergeTree & storage_, ReplicatedMergeTreeQueue::SelectedEntryPtr & selected_entry_, - Callback && task_result_callback_) + IExecutableTask::TaskResultCallback & task_result_callback_) : selected_entry(selected_entry_) , entry(*selected_entry->log_entry) , log(log_) , storage(storage_) /// This is needed to ask an asssignee to assign a new merge/mutate operation /// It takes bool argument and true means that current task is successfully executed. - , task_result_callback(task_result_callback_) {} + , task_result_callback(task_result_callback_) + { + } ~ReplicatedMergeMutateTaskBase() override = default; void onCompleted() override; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 6825698f006..bcbfeb290ed 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -938,8 +938,9 @@ bool StorageMergeTree::merge( return false; /// Copying a vector of columns `deduplicate bu columns. + IExecutableTask::TaskResultCallback f = [](bool) {}; auto task = std::make_shared( - *this, metadata_snapshot, deduplicate, deduplicate_by_columns, merge_mutate_entry, table_lock_holder, [](bool){}); + *this, metadata_snapshot, deduplicate, deduplicate_by_columns, merge_mutate_entry, table_lock_holder, f); task->setCurrentTransaction(MergeTreeTransactionHolder{}, MergeTreeTransactionPtr{txn}); From d68b3dfd43264150de5fbbc7b05fccc24ad29113 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 15 Jul 2022 11:40:58 +0200 Subject: [PATCH 279/659] Fix destructor order --- src/Storages/MergeTree/MergeList.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index da6c5184619..d6cabddfec4 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -113,7 +113,6 @@ struct MergeListElement : boost::noncopyable /// Updated only for Vertical algorithm std::atomic columns_written{}; - MemoryTracker memory_tracker{VariableContext::Process}; /// Used to adjust ThreadStatus::untracked_memory_limit UInt64 max_untracked_memory; /// Used to avoid losing any allocation context @@ -126,8 +125,10 @@ struct MergeListElement : boost::noncopyable /// Detected after merge already started std::atomic merge_algorithm; - // Description used for logging + /// Description used for logging + /// Needs to outlive memory_tracker since it's used in its destructor const String description{"Mutate/Merge"}; + MemoryTracker memory_tracker{VariableContext::Process}; MergeListElement( const StorageID & table_id_, From d7fc7a429028b7ebf2910100834d87de91efc0e7 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 15 Jul 2022 16:08:53 +0200 Subject: [PATCH 280/659] Addressed review comments - Removed first_buffer, finalizeImpl and logic to different buffer size from ForkWriteBuffer --- src/IO/ForkWriteBuffer.cpp | 37 ++++--------------------------------- src/IO/ForkWriteBuffer.h | 2 -- 2 files changed, 4 insertions(+), 35 deletions(-) diff --git a/src/IO/ForkWriteBuffer.cpp b/src/IO/ForkWriteBuffer.cpp index b055ae588a9..e91f6ca433b 100644 --- a/src/IO/ForkWriteBuffer.cpp +++ b/src/IO/ForkWriteBuffer.cpp @@ -16,46 +16,23 @@ ForkWriteBuffer::ForkWriteBuffer(WriteBufferPtrs && sources_) { if (sources.empty()) { - first_buffer = nullptr; throw Exception("ForkWriteBuffer required WriteBuffer is not provided", ErrorCodes::CANNOT_CREATE_IO_BUFFER); } - else - { - first_buffer = sources.begin()->get(); - set(first_buffer->buffer().begin(), first_buffer->buffer().size()); - } + set(sources.front()->buffer().begin(), sources.front()->buffer().size()); } void ForkWriteBuffer::nextImpl() { - if (!first_buffer) - return; - - first_buffer->position() = position(); + sources.front()->position() = position(); try { for (const WriteBufferPtr & write_buffer : sources | std::views::reverse) { - if (write_buffer.get() != first_buffer) + if (write_buffer != sources.front()) { - //if buffer size if not enough to write, then split the message with buffer length - if (write_buffer->available() < first_buffer->offset()) - { - size_t bytes_written = 0; - auto to_be_written = first_buffer->offset(); - while (to_be_written != 0) - { - int bytes_to_copy = std::min(to_be_written, write_buffer->available()); - write_buffer->write(first_buffer->buffer().begin()+bytes_written, bytes_to_copy); - write_buffer->next(); - bytes_written += bytes_to_copy; - to_be_written -= bytes_to_copy; - } - } - else - write_buffer->write(first_buffer->buffer().begin(), first_buffer->offset()); + write_buffer->write(sources.front()->buffer().begin(), sources.front()->offset()); } write_buffer->next(); } @@ -68,12 +45,6 @@ void ForkWriteBuffer::nextImpl() } -void ForkWriteBuffer::finalizeImpl() -{ - next(); -} - - ForkWriteBuffer::~ForkWriteBuffer() { finalize(); diff --git a/src/IO/ForkWriteBuffer.h b/src/IO/ForkWriteBuffer.h index 63267fcd8d7..56e9c445842 100644 --- a/src/IO/ForkWriteBuffer.h +++ b/src/IO/ForkWriteBuffer.h @@ -25,11 +25,9 @@ public: protected: void nextImpl() override; - void finalizeImpl() override; private: WriteBufferPtrs sources; - WriteBuffer *first_buffer; }; } From 909e871c4862d88893014f4b2f66206367b2b0fc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 14 Jul 2022 20:57:03 +0200 Subject: [PATCH 281/659] Rewrite cherry_pick.py to PyGithub API --- tests/ci/cherry_pick.py | 507 +++++++++++++++++++-- tests/ci/cherry_pick_utils/__init__.py | 2 - tests/ci/cherry_pick_utils/backport.py | 190 -------- tests/ci/cherry_pick_utils/cherrypick.py | 319 -------------- tests/ci/cherry_pick_utils/local.py | 109 ----- tests/ci/cherry_pick_utils/parser.py | 56 --- tests/ci/cherry_pick_utils/query.py | 532 ----------------------- tests/ci/cherry_pick_utils/readme.md | 3 - tests/ci/git_helper.py | 9 +- tests/ci/github_helper.py | 162 +++++++ 10 files changed, 632 insertions(+), 1257 deletions(-) delete mode 100644 tests/ci/cherry_pick_utils/__init__.py delete mode 100644 tests/ci/cherry_pick_utils/backport.py delete mode 100644 tests/ci/cherry_pick_utils/cherrypick.py delete mode 100644 tests/ci/cherry_pick_utils/local.py delete mode 100644 tests/ci/cherry_pick_utils/parser.py delete mode 100644 tests/ci/cherry_pick_utils/query.py delete mode 100644 tests/ci/cherry_pick_utils/readme.md create mode 100644 tests/ci/github_helper.py diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 745284b2b29..834c9296a59 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -1,72 +1,489 @@ #!/usr/bin/env python3 +""" +A plan: + - Receive GH objects cache from S3, ignore if fails + - Get all open release PRs + - Get all pull-requests between the date of the merge-base for the oldest PR with + labels pr-must-backport and version-specific v21.8-must-backport, but without + pr-backported + - Iterate over gotten PRs: + - for pr-must-backport: + - check if all backport-PRs are created. If yes, + set pr-backported label + - If not, create either cherrypick PRs or merge cherrypick (in the same + stage, if mergable?) and create backport-PRs + - If successfull, set pr-backported label on the PR + + - for version-specific labels: + - the same, check, cherry-pick, backport + +Cherry-pick stage: + - From time to time the cherry-pick fails, if it was done manually. In the + case we should check if it's even needed, and mark the release as done somehow. +""" import argparse import logging import os -import subprocess +import sys +from contextlib import contextmanager +from datetime import date, timedelta +from subprocess import CalledProcessError +from typing import List, Optional -from env_helper import GITHUB_WORKSPACE, TEMP_PATH +from env_helper import TEMP_PATH from get_robot_token import get_best_robot_token +from git_helper import git_runner, is_shallow +from github_helper import ( + GitHub, + PullRequest, + PullRequests, + Repository, +) +from github.Label import Label from ssh import SSHKey -from cherry_pick_utils.backport import Backport -from cherry_pick_utils.cherrypick import CherryPick + +Labels = List[Label] + + +class labels: + LABEL_MUST_BACKPORT = "pr-must-backport" + LABEL_BACKPORT = "pr-backport" + LABEL_BACKPORTED = "pr-backported" + LABEL_CHERRYPICK = "pr-cherrypick" + LABEL_DO_NOT_TEST = "do not test" + + +class ReleaseBranch: + CHERRYPICK_DESCRIPTION = """This pull-request is a first step of an automated \ + backporting. +It contains changes like after calling a local command `git cherry-pick`. +If you intend to continue backporting this changes, then resolve all conflicts if any. +Otherwise, if you do not want to backport them, then just close this pull-request. + +The check results does not matter at this step - you can safely ignore them. +Also this pull-request will be merged automatically as it reaches the mergeable state, \ + but you always can merge it manually. +""" + BACKPORT_DESCRIPTION = """This pull-request is a last step of an automated \ +backporting. +Treat it as a standard pull-request: look at the checks and resolve conflicts. +Merge it only if you intend to backport changes to the target branch, otherwise just \ + close it. +""" + REMOTE = "" + + def __init__(self, name: str, pr: PullRequest): + self.name = name + self.pr = pr + self.cherrypick_branch = f"cherrypick/{name}/{pr.merge_commit_sha}" + self.backport_branch = f"backport/{name}/{pr.number}" + self.cherrypick_pr = None # type: Optional[PullRequest] + self.backport_pr = None # type: Optional[PullRequest] + self._backported = None # type: Optional[bool] + self.git_prefix = ( # All commits to cherrypick are done as robot-clickhouse + "git -c user.email=robot-clickhouse@clickhouse.com " + "-c user.name=robot-clickhouse -c commit.gpgsign=false" + ) + + def pop_prs(self, prs: PullRequests): + to_pop = [] # type: List[int] + for i, pr in enumerate(prs): + if self.name not in pr.head.ref: + continue + if pr.head.ref.startswith(f"cherrypick/{self.name}"): + self.cherrypick_pr = pr + to_pop.append(i) + elif pr.head.ref.startswith(f"backport/{self.name}"): + self.backport_pr = pr + to_pop.append(i) + else: + logging.error( + "PR #%s doesn't head ref starting with known suffix", + pr.number, + ) + for i in reversed(to_pop): + # Going from the tail to keep the order and pop greater index first + prs.pop(i) + + def process(self, dry_run: bool): + if self.backported: + return + if not self.cherrypick_pr: + if dry_run: + logging.info( + "DRY RUN: Would create cherrypick PR for #%s", self.pr.number + ) + return + self.create_cherrypick() + if self.backported: + return + if self.cherrypick_pr is not None: + # Try to merge cherrypick instantly + if self.cherrypick_pr.mergeable and self.cherrypick_pr.state != "closed": + self.cherrypick_pr.merge() + # The PR needs update, since PR.merge doesn't update the object + self.cherrypick_pr.update() + if self.cherrypick_pr.merged: + if dry_run: + logging.info( + "DRY RUN: Would create backport PR for #%s", self.pr.number + ) + return + self.create_backport() + return + elif self.cherrypick_pr.state == "closed": + logging.info( + "The cherrypick PR #%s for PR #%s is discarded", + self.cherrypick_pr.number, + self.pr.number, + ) + self._backported = True + return + logging.info( + "Cherrypick PR #%s for PR #%s have conflicts and unable to be merged", + self.cherrypick_pr.number, + self.pr.number, + ) + + def create_cherrypick(self): + # First, create backport branch: + # Checkout release branch with discarding every change + git_runner(f"{self.git_prefix} checkout -f {self.name}") + # Create or reset backport branch + git_runner(f"{self.git_prefix} checkout -B {self.backport_branch}") + # Merge all changes from PR's the first parent commit w/o applying anything + # It will produce the commit like cherry-pick + first_parent = git_runner(f"git rev-parse {self.pr.merge_commit_sha}^1") + git_runner(f"{self.git_prefix} merge -s ours --no-edit {first_parent}") + + # Second step, create cherrypick branch + git_runner( + f"{self.git_prefix} branch -f " + f"{self.cherrypick_branch} {self.pr.merge_commit_sha}" + ) + + # Check if there actually any changes between branches. If no, then no + # other actions are required. + try: + output = git_runner( + f"{self.git_prefix} merge --no-commit --no-ff {self.cherrypick_branch}" + ) + # 'up-to-date', 'up to date', who knows what else + if output.startswith("Already up") and output.endswith("date."): + # The changes are already in the release branch, we are done here + logging.info( + "Release branch %s already contain changes from %s", + self.name, + self.pr.number, + ) + self._backported = True + return + except CalledProcessError: + # There are most probably conflicts, they'll be resolved in PR + git_runner(f"{self.git_prefix} reset --merge") + else: + # There are changes able to apply, so continue + git_runner(f"{self.git_prefix} reset --merge") + + for branch in [self.cherrypick_branch, self.backport_branch]: + git_runner(f"{self.git_prefix} push -f {self.REMOTE} {branch}:{branch}") + + self.cherrypick_pr = self.pr.base.repo.create_pull( + title=f"Cherry pick #{self.pr.number} to {self.name}: {self.pr.title}", + body=f"Original pull-request #{self.pr.number}\n\n" + f"{self.CHERRYPICK_DESCRIPTION}", + base=self.backport_branch, + head=self.cherrypick_branch, + ) + self.cherrypick_pr.add_to_labels(labels.LABEL_CHERRYPICK) + self.cherrypick_pr.add_to_labels(labels.LABEL_DO_NOT_TEST) + self.cherrypick_pr.add_to_assignees(self.pr.assignee) + self.cherrypick_pr.add_to_assignees(self.pr.user) + + def create_backport(self): + git_runner(f"{self.git_prefix} checkout -f {self.backport_branch}") + git_runner( + f"{self.git_prefix} pull --ff-only {self.REMOTE} {self.backport_branch}" + ) + merge_base = git_runner( + f"{self.git_prefix} merge-base " + f"{self.REMOTE}/{self.name} {self.backport_branch}" + ) + git_runner(f"{self.git_prefix} reset --soft {merge_base}") + title = f"Backport #{self.pr.number} to {self.name}: {self.pr.title}" + git_runner(f"{self.git_prefix} commit -a --allow-empty -F -", input=title) + git_runner( + f"{self.git_prefix} push -f {self.REMOTE} " + f"{self.backport_branch}:{self.backport_branch}" + ) + self.backport_pr = self.pr.base.repo.create_pull( + title=title, + body=f"Original pull-request #{self.pr.number}\n" + f"Cherry-pick pull-request #{self.cherrypick_pr.number}\n\n" + f"{self.BACKPORT_DESCRIPTION}", + base=self.name, + head=self.backport_branch, + ) + self.backport_pr.add_to_labels(labels.LABEL_BACKPORT) + self.backport_pr.add_to_assignees(self.pr.assignee) + self.backport_pr.add_to_assignees(self.pr.user) + + @property + def backported(self) -> bool: + if self._backported is not None: + return self._backported + return self.backport_pr is not None + + def __repr__(self): + return self.name + + +class Backport: + def __init__(self, gh: GitHub, repo: str, dry_run: bool): + self.gh = gh + self._repo_name = repo + self.dry_run = dry_run + self._repo = None # type: Optional[Repository] + self._remote = "" + self._query = f"type:pr repo:{repo}" + self.release_prs = [] # type: PullRequests + self.release_branches = [] # type: List[str] + self.labels_to_backport = [] # type: List[str] + self.prs_for_backport = [] # type: PullRequests + self.error = False + + @property + def remote(self) -> str: + if not self._remote: + # lines of "origin git@github.com:ClickHouse/ClickHouse.git (fetch)" + remotes = git_runner("git remote -v").split("\n") + # We need the first word from the first matching result + self._remote = tuple( + remote.split(maxsplit=1)[0] + for remote in remotes + if f"github.com/{self._repo_name}" in remote # ssh + or f"github.com:{self._repo_name}" in remote # https + )[0] + git_runner(f"git fetch {self._remote}") + ReleaseBranch.REMOTE = self._remote + return self._remote + + def receive_release_prs(self): + logging.info("Getting release PRs") + self.release_prs = self.gh.get_pulls_from_search( + query=f"{self._query} is:open", + sort="created", + order="asc", + type="pr", + label="release", + ) + self.release_branches = [pr.head.ref for pr in self.release_prs] + self.labels_to_backport = [ + f"v{branch}-must-backport" for branch in self.release_branches + ] + logging.info("Active releases: %s", ", ".join(self.release_branches)) + + def receive_prs_for_backport(self): + since_commit = git_runner( + f"git merge-base {self.remote}/{self.release_branches[0]} " + f"{self.remote}/{self.default_branch}" + ) + since_date = date.fromisoformat( + git_runner.run(f"git log -1 --format=format:%cs {since_commit}") + ) + tomorrow = date.today() + timedelta(days=1) + logging.info("Receive PRs suppose to be backported") + self.prs_for_backport = self.gh.get_pulls_from_search( + query=f"{self._query} -label:pr-backported", + label=",".join(self.labels_to_backport + [labels.LABEL_MUST_BACKPORT]), + merged=[since_date, tomorrow], + ) + logging.info( + "PRs to be backported:\n %s", + "\n ".join([pr.html_url for pr in self.prs_for_backport]), + ) + + def process_backports(self): + for pr in self.prs_for_backport: + self.process_pr(pr) + + def process_pr(self, pr: PullRequest): + pr_labels = [label.name for label in pr.labels] + if labels.LABEL_MUST_BACKPORT in pr_labels: + branches = [ + ReleaseBranch(br, pr) for br in self.release_branches + ] # type: List[ReleaseBranch] + else: + branches = [ + ReleaseBranch(br, pr) + for br in [ + label.split("-", 1)[0][1:] # v21.8-must-backport + for label in pr_labels + if label in self.labels_to_backport + ] + ] + if not branches: + # This is definitely some error. There must be at least one branch + # It also make the whole program exit code non-zero + logging.error( + "There are no branches to backport PR #%s, logical error", pr.number + ) + self.error = True + return + + logging.info( + " PR #%s is suppose to be backported to %s", + pr.number, + ", ".join(map(str, branches)), + ) + # All PRs for cherrypick and backport branches as heads + query_suffix = " ".join( + [ + f"head:{branch.backport_branch} head:{branch.cherrypick_branch}" + for branch in branches + ] + ) + bp_cp_prs = self.gh.get_pulls_from_search( + query=f"{self._query} {query_suffix}", + ) + for br in branches: + br.pop_prs(bp_cp_prs) + + if bp_cp_prs: + # This is definitely some error. All prs must be consumed by + # branches with ReleaseBranch.pop_prs. It also make the whole + # program exit code non-zero + logging.error( + "The following PRs are not filtered by release branches:\n%s", + "\n".join(map(str, bp_cp_prs)), + ) + self.error = True + return + + if all(br.backported for br in branches): + # Let's check if the PR is already backported + self.mark_pr_backported(pr) + return + + for br in branches: + try: + br.process(self.dry_run) + except Exception as e: + logging.error( + "During processing the PR #%s error occured: %s", pr.number, e + ) + self.error = True + + if all(br.backported for br in branches): + # And check it after the running + self.mark_pr_backported(pr) + + def mark_pr_backported(self, pr: PullRequest): + if self.dry_run: + logging.info("DRY RUN: would mark PR #%s as done", pr.number) + return + pr.add_to_labels(labels.LABEL_BACKPORTED) + logging.info( + "PR #%s is successfully labeled with `%s`", + pr.number, + labels.LABEL_BACKPORTED, + ) + + @staticmethod + def pr_labels(pr: PullRequest) -> List[str]: + return [label.name for label in pr.labels] + + @property + def repo(self) -> Repository: + if self._repo is None: + try: + self._repo = self.release_prs[0].base.repo + except IndexError as exc: + raise Exception( + "`repo` is available only after the `receive_release_prs`" + ) from exc + return self._repo + + @property + def default_branch(self) -> str: + return self.repo.default_branch def parse_args(): parser = argparse.ArgumentParser("Create cherry-pick and backport PRs") parser.add_argument("--token", help="github token, if not set, used from smm") + parser.add_argument( + "--repo", default="ClickHouse/ClickHouse", help="repo owner/name" + ) parser.add_argument("--dry-run", action="store_true", help="do not create anything") + parser.add_argument( + "--debug-helpers", + action="store_true", + help="add debug logging for git_helper and github_helper", + ) return parser.parse_args() +@contextmanager +def clear_repo(): + orig_ref = git_runner("git branch --show-current") or git_runner( + "git rev-parse HEAD" + ) + try: + yield + except (Exception, KeyboardInterrupt): + git_runner(f"git checkout -f {orig_ref}") + raise + else: + git_runner(f"git checkout -f {orig_ref}") + + +@contextmanager +def stash(): + need_stash = bool(git_runner("git diff HEAD")) + if need_stash: + git_runner("git stash push --no-keep-index -m 'running cherry_pick.py'") + try: + with clear_repo(): + yield + except (Exception, KeyboardInterrupt): + if need_stash: + git_runner("git stash pop") + raise + else: + if need_stash: + git_runner("git stash pop") + + def main(): + if not os.path.exists(TEMP_PATH): + os.makedirs(TEMP_PATH) + args = parse_args() + if args.debug_helpers: + logging.getLogger("github_helper").setLevel(logging.DEBUG) + logging.getLogger("git_helper").setLevel(logging.DEBUG) token = args.token or get_best_robot_token() - bp = Backport( - token, - os.environ.get("REPO_OWNER"), - os.environ.get("REPO_NAME"), - os.environ.get("REPO_TEAM"), - ) - - cherry_pick = CherryPick( - token, - os.environ.get("REPO_OWNER"), - os.environ.get("REPO_NAME"), - os.environ.get("REPO_TEAM"), - 1, - "master", - ) - # Use the same _gh in both objects to have a proper cost - # pylint: disable=protected-access - for key in bp._gh.api_costs: - if key in cherry_pick._gh.api_costs: - bp._gh.api_costs[key] += cherry_pick._gh.api_costs[key] - for key in cherry_pick._gh.api_costs: - if key not in bp._gh.api_costs: - bp._gh.api_costs[key] = cherry_pick._gh.api_costs[key] - cherry_pick._gh = bp._gh - # pylint: enable=protected-access - - def cherrypick_run(pr_data, branch): - cherry_pick.update_pr_branch(pr_data, branch) - return cherry_pick.execute(GITHUB_WORKSPACE, args.dry_run) - - try: - bp.execute(GITHUB_WORKSPACE, "origin", None, cherrypick_run) - except subprocess.CalledProcessError as e: - logging.error(e.output) + gh = GitHub(token, per_page=100) + bp = Backport(gh, args.repo, args.dry_run) + bp.gh.cache_path = str(f"{TEMP_PATH}/gh_cache") + bp.receive_release_prs() + bp.receive_prs_for_backport() + bp.process_backports() + if bp.error: + logging.error("Finished successfully, but errors occured") + sys.exit(1) if __name__ == "__main__": logging.basicConfig(level=logging.INFO) - if not os.path.exists(TEMP_PATH): - os.makedirs(TEMP_PATH) - - if os.getenv("ROBOT_CLICKHOUSE_SSH_KEY", ""): - with SSHKey("ROBOT_CLICKHOUSE_SSH_KEY"): + assert not is_shallow() + with stash(): + if os.getenv("ROBOT_CLICKHOUSE_SSH_KEY", ""): + with SSHKey("ROBOT_CLICKHOUSE_SSH_KEY"): + main() + else: main() - else: - main() diff --git a/tests/ci/cherry_pick_utils/__init__.py b/tests/ci/cherry_pick_utils/__init__.py deleted file mode 100644 index faa18be5bbf..00000000000 --- a/tests/ci/cherry_pick_utils/__init__.py +++ /dev/null @@ -1,2 +0,0 @@ -#!/usr/bin/env python -# -*- coding: utf-8 -*- diff --git a/tests/ci/cherry_pick_utils/backport.py b/tests/ci/cherry_pick_utils/backport.py deleted file mode 100644 index 1bc910886de..00000000000 --- a/tests/ci/cherry_pick_utils/backport.py +++ /dev/null @@ -1,190 +0,0 @@ -# -*- coding: utf-8 -*- - -import argparse -import logging -import os -import re -import sys - -sys.path.append(os.path.dirname(__file__)) - -from cherrypick import CherryPick -from query import Query as RemoteRepo -from local import Repository as LocalRepo - - -class Backport: - def __init__(self, token, owner, name, team): - self._gh = RemoteRepo( - token, owner=owner, name=name, team=team, max_page_size=60, min_page_size=7 - ) - self._token = token - self.default_branch_name = self._gh.default_branch - self.ssh_url = self._gh.ssh_url - - def getPullRequests(self, from_commit): - return self._gh.get_pull_requests(from_commit) - - def getBranchesWithRelease(self): - branches = set() - for pull_request in self._gh.find_pull_requests("release"): - branches.add(pull_request["headRefName"]) - return branches - - def execute(self, repo, upstream, until_commit, run_cherrypick): - repo = LocalRepo(repo, upstream, self.default_branch_name) - all_branches = repo.get_release_branches() # [(branch_name, base_commit)] - - release_branches = self.getBranchesWithRelease() - - branches = [] - # iterate over all branches to preserve their precedence. - for branch in all_branches: - if branch[0] in release_branches: - branches.append(branch) - - if not branches: - logging.info("No release branches found!") - return - - logging.info( - "Found release branches: %s", ", ".join([br[0] for br in branches]) - ) - - if not until_commit: - until_commit = branches[0][1] - pull_requests = self.getPullRequests(until_commit) - - backport_map = {} - pr_map = {pr["number"]: pr for pr in pull_requests} - - RE_MUST_BACKPORT = re.compile(r"^v(\d+\.\d+)-must-backport$") - RE_NO_BACKPORT = re.compile(r"^v(\d+\.\d+)-no-backport$") - RE_BACKPORTED = re.compile(r"^v(\d+\.\d+)-backported$") - - # pull-requests are sorted by ancestry from the most recent. - for pr in pull_requests: - while repo.comparator(branches[-1][1]) >= repo.comparator( - pr["mergeCommit"]["oid"] - ): - logging.info( - "PR #%s is already inside %s. Dropping this branch for further PRs", - pr["number"], - branches[-1][0], - ) - branches.pop() - - logging.info("Processing PR #%s", pr["number"]) - - assert len(branches) != 0 - - branch_set = {branch[0] for branch in branches} - - # First pass. Find all must-backports - for label in pr["labels"]["nodes"]: - if label["name"] == "pr-must-backport": - backport_map[pr["number"]] = branch_set.copy() - continue - matched = RE_MUST_BACKPORT.match(label["name"]) - if matched: - if pr["number"] not in backport_map: - backport_map[pr["number"]] = set() - backport_map[pr["number"]].add(matched.group(1)) - - # Second pass. Find all no-backports - for label in pr["labels"]["nodes"]: - if label["name"] == "pr-no-backport" and pr["number"] in backport_map: - del backport_map[pr["number"]] - break - matched_no_backport = RE_NO_BACKPORT.match(label["name"]) - matched_backported = RE_BACKPORTED.match(label["name"]) - if ( - matched_no_backport - and pr["number"] in backport_map - and matched_no_backport.group(1) in backport_map[pr["number"]] - ): - backport_map[pr["number"]].remove(matched_no_backport.group(1)) - logging.info( - "\tskipping %s because of forced no-backport", - matched_no_backport.group(1), - ) - elif ( - matched_backported - and pr["number"] in backport_map - and matched_backported.group(1) in backport_map[pr["number"]] - ): - backport_map[pr["number"]].remove(matched_backported.group(1)) - logging.info( - "\tskipping %s because it's already backported manually", - matched_backported.group(1), - ) - - for pr, branches in list(backport_map.items()): - statuses = [] - for branch in branches: - branch_status = run_cherrypick(pr_map[pr], branch) - statuses.append(f"{branch}, and the status is: {branch_status}") - logging.info( - "PR #%s needs to be backported to:\n\t%s", pr, "\n\t".join(statuses) - ) - - # print API costs - logging.info("\nGitHub API total costs for backporting per query:") - for name, value in list(self._gh.api_costs.items()): - logging.info("%s : %s", name, value) - - -if __name__ == "__main__": - parser = argparse.ArgumentParser() - parser.add_argument( - "--token", type=str, required=True, help="token for Github access" - ) - parser.add_argument( - "--repo", - type=str, - required=True, - help="path to full repository", - metavar="PATH", - ) - parser.add_argument( - "--til", type=str, help="check PRs from HEAD til this commit", metavar="COMMIT" - ) - parser.add_argument( - "--dry-run", - action="store_true", - help="do not create or merge any PRs", - default=False, - ) - parser.add_argument( - "--verbose", - "-v", - action="store_true", - help="more verbose output", - default=False, - ) - parser.add_argument( - "--upstream", - "-u", - type=str, - help="remote name of upstream in repository", - default="origin", - ) - args = parser.parse_args() - - if args.verbose: - logging.basicConfig( - format="%(message)s", stream=sys.stdout, level=logging.DEBUG - ) - else: - logging.basicConfig(format="%(message)s", stream=sys.stdout, level=logging.INFO) - - cherry_pick = CherryPick( - args.token, "ClickHouse", "ClickHouse", "core", 1, "master" - ) - - def cherrypick_run(pr_data, branch): - cherry_pick.update_pr_branch(pr_data, branch) - return cherry_pick.execute(args.repo, args.dry_run) - - bp = Backport(args.token, "ClickHouse", "ClickHouse", "core") - bp.execute(args.repo, args.upstream, args.til, cherrypick_run) diff --git a/tests/ci/cherry_pick_utils/cherrypick.py b/tests/ci/cherry_pick_utils/cherrypick.py deleted file mode 100644 index c844beaee88..00000000000 --- a/tests/ci/cherry_pick_utils/cherrypick.py +++ /dev/null @@ -1,319 +0,0 @@ -# -*- coding: utf-8 -*- - -""" -Backports changes from PR to release branch. -Requires multiple separate runs as part of the implementation. - -First run should do the following: -1. Merge release branch with a first parent of merge-commit of PR (using 'ours' strategy). (branch: backport/{branch}/{pr}) -2. Create temporary branch over merge-commit to use it for PR creation. (branch: cherrypick/{merge_commit}) -3. Create PR from temporary branch to backport branch (emulating cherry-pick). - -Second run checks PR from previous run to be merged or at least being mergeable. If it's not merged then try to merge it. - -Third run creates PR from backport branch (with merged previous PR) to release branch. -""" - - -import argparse -from enum import Enum -import logging -import os -import subprocess -import sys - -sys.path.append(os.path.dirname(__file__)) - -from query import Query as RemoteRepo - - -class CherryPick: - class Status(Enum): - DISCARDED = "discarded" - NOT_INITIATED = "not started" - FIRST_MERGEABLE = "waiting for 1st stage" - FIRST_CONFLICTS = "conflicts on 1st stage" - SECOND_MERGEABLE = "waiting for 2nd stage" - SECOND_CONFLICTS = "conflicts on 2nd stage" - MERGED = "backported" - - def _run(self, args): - out = subprocess.check_output(args).rstrip() - logging.debug(out) - return out - - def __init__(self, token, owner, name, team, pr_number, target_branch): - self._gh = RemoteRepo(token, owner=owner, name=name, team=team) - self._pr = self._gh.get_pull_request(pr_number) - self.target_branch = target_branch - - self.ssh_url = self._gh.ssh_url - - # TODO: check if pull-request is merged. - self.update_pr_branch(self._pr, self.target_branch) - - def update_pr_branch(self, pr_data, target_branch): - """The method is here to avoid unnecessary creation of new objects""" - self._pr = pr_data - self.target_branch = target_branch - self.merge_commit_oid = self._pr["mergeCommit"]["oid"] - - self.backport_branch = f"backport/{target_branch}/{pr_data['number']}" - self.cherrypick_branch = f"cherrypick/{target_branch}/{self.merge_commit_oid}" - - def getCherryPickPullRequest(self): - return self._gh.find_pull_request( - base=self.backport_branch, head=self.cherrypick_branch - ) - - def createCherryPickPullRequest(self, repo_path): - DESCRIPTION = ( - "This pull-request is a first step of an automated backporting.\n" - "It contains changes like after calling a local command `git cherry-pick`.\n" - "If you intend to continue backporting this changes, then resolve all conflicts if any.\n" - "Otherwise, if you do not want to backport them, then just close this pull-request.\n" - "\n" - "The check results does not matter at this step - you can safely ignore them.\n" - "Also this pull-request will be merged automatically as it reaches the mergeable state, but you always can merge it manually.\n" - ) - - # FIXME: replace with something better than os.system() - git_prefix = [ - "git", - "-C", - repo_path, - "-c", - "user.email=robot-clickhouse@yandex-team.ru", - "-c", - "user.name=robot-clickhouse", - ] - base_commit_oid = self._pr["mergeCommit"]["parents"]["nodes"][0]["oid"] - - # Create separate branch for backporting, and make it look like real cherry-pick. - self._run(git_prefix + ["checkout", "-f", self.target_branch]) - self._run(git_prefix + ["checkout", "-B", self.backport_branch]) - self._run(git_prefix + ["merge", "-s", "ours", "--no-edit", base_commit_oid]) - - # Create secondary branch to allow pull request with cherry-picked commit. - self._run( - git_prefix + ["branch", "-f", self.cherrypick_branch, self.merge_commit_oid] - ) - - self._run( - git_prefix - + [ - "push", - "-f", - "origin", - "{branch}:{branch}".format(branch=self.backport_branch), - ] - ) - self._run( - git_prefix - + [ - "push", - "-f", - "origin", - "{branch}:{branch}".format(branch=self.cherrypick_branch), - ] - ) - - # Create pull-request like a local cherry-pick - title = self._pr["title"].replace('"', r"\"") - pr = self._gh.create_pull_request( - source=self.cherrypick_branch, - target=self.backport_branch, - title=( - f'Cherry pick #{self._pr["number"]} ' - f"to {self.target_branch}: " - f"{title}" - ), - description=f'Original pull-request #{self._pr["number"]}\n\n{DESCRIPTION}', - ) - - # FIXME: use `team` to leave a single eligible assignee. - self._gh.add_assignee(pr, self._pr["author"]) - self._gh.add_assignee(pr, self._pr["mergedBy"]) - - self._gh.set_label(pr, "do not test") - self._gh.set_label(pr, "pr-cherrypick") - - return pr - - def mergeCherryPickPullRequest(self, cherrypick_pr): - return self._gh.merge_pull_request(cherrypick_pr["id"]) - - def getBackportPullRequest(self): - return self._gh.find_pull_request( - base=self.target_branch, head=self.backport_branch - ) - - def createBackportPullRequest(self, cherrypick_pr, repo_path): - DESCRIPTION = ( - "This pull-request is a last step of an automated backporting.\n" - "Treat it as a standard pull-request: look at the checks and resolve conflicts.\n" - "Merge it only if you intend to backport changes to the target branch, otherwise just close it.\n" - ) - - git_prefix = [ - "git", - "-C", - repo_path, - "-c", - "user.email=robot-clickhouse@clickhouse.com", - "-c", - "user.name=robot-clickhouse", - ] - - title = self._pr["title"].replace('"', r"\"") - pr_title = f"Backport #{self._pr['number']} to {self.target_branch}: {title}" - - self._run(git_prefix + ["checkout", "-f", self.backport_branch]) - self._run(git_prefix + ["pull", "--ff-only", "origin", self.backport_branch]) - self._run( - git_prefix - + [ - "reset", - "--soft", - self._run( - git_prefix - + [ - "merge-base", - "origin/" + self.target_branch, - self.backport_branch, - ] - ), - ] - ) - self._run(git_prefix + ["commit", "-a", "--allow-empty", "-m", pr_title]) - self._run( - git_prefix - + [ - "push", - "-f", - "origin", - "{branch}:{branch}".format(branch=self.backport_branch), - ] - ) - - pr = self._gh.create_pull_request( - source=self.backport_branch, - target=self.target_branch, - title=pr_title, - description=f"Original pull-request #{self._pr['number']}\n" - f"Cherry-pick pull-request #{cherrypick_pr['number']}\n\n{DESCRIPTION}", - ) - - # FIXME: use `team` to leave a single eligible assignee. - self._gh.add_assignee(pr, self._pr["author"]) - self._gh.add_assignee(pr, self._pr["mergedBy"]) - - self._gh.set_label(pr, "pr-backport") - - return pr - - def execute(self, repo_path, dry_run=False): - pr1 = self.getCherryPickPullRequest() - if not pr1: - if not dry_run: - pr1 = self.createCherryPickPullRequest(repo_path) - logging.debug( - "Created PR with cherry-pick of %s to %s: %s", - self._pr["number"], - self.target_branch, - pr1["url"], - ) - else: - return CherryPick.Status.NOT_INITIATED - else: - logging.debug( - "Found PR with cherry-pick of %s to %s: %s", - self._pr["number"], - self.target_branch, - pr1["url"], - ) - - if not pr1["merged"] and pr1["mergeable"] == "MERGEABLE" and not pr1["closed"]: - if not dry_run: - pr1 = self.mergeCherryPickPullRequest(pr1) - logging.debug( - "Merged PR with cherry-pick of %s to %s: %s", - self._pr["number"], - self.target_branch, - pr1["url"], - ) - - if not pr1["merged"]: - logging.debug( - "Waiting for PR with cherry-pick of %s to %s: %s", - self._pr["number"], - self.target_branch, - pr1["url"], - ) - - if pr1["closed"]: - return CherryPick.Status.DISCARDED - elif pr1["mergeable"] == "CONFLICTING": - return CherryPick.Status.FIRST_CONFLICTS - else: - return CherryPick.Status.FIRST_MERGEABLE - - pr2 = self.getBackportPullRequest() - if not pr2: - if not dry_run: - pr2 = self.createBackportPullRequest(pr1, repo_path) - logging.debug( - "Created PR with backport of %s to %s: %s", - self._pr["number"], - self.target_branch, - pr2["url"], - ) - else: - return CherryPick.Status.FIRST_MERGEABLE - else: - logging.debug( - "Found PR with backport of %s to %s: %s", - self._pr["number"], - self.target_branch, - pr2["url"], - ) - - if pr2["merged"]: - return CherryPick.Status.MERGED - elif pr2["closed"]: - return CherryPick.Status.DISCARDED - elif pr2["mergeable"] == "CONFLICTING": - return CherryPick.Status.SECOND_CONFLICTS - else: - return CherryPick.Status.SECOND_MERGEABLE - - -if __name__ == "__main__": - logging.basicConfig(format="%(message)s", stream=sys.stdout, level=logging.DEBUG) - - parser = argparse.ArgumentParser() - parser.add_argument( - "--token", "-t", type=str, required=True, help="token for Github access" - ) - parser.add_argument("--pr", type=str, required=True, help="PR# to cherry-pick") - parser.add_argument( - "--branch", - "-b", - type=str, - required=True, - help="target branch name for cherry-pick", - ) - parser.add_argument( - "--repo", - "-r", - type=str, - required=True, - help="path to full repository", - metavar="PATH", - ) - args = parser.parse_args() - - cp = CherryPick( - args.token, "ClickHouse", "ClickHouse", "core", args.pr, args.branch - ) - cp.execute(args.repo) diff --git a/tests/ci/cherry_pick_utils/local.py b/tests/ci/cherry_pick_utils/local.py deleted file mode 100644 index 71923b63c35..00000000000 --- a/tests/ci/cherry_pick_utils/local.py +++ /dev/null @@ -1,109 +0,0 @@ -# -*- coding: utf-8 -*- - -import functools -import logging -import os -import re - -import git - - -class RepositoryBase: - def __init__(self, repo_path): - - self._repo = git.Repo(repo_path, search_parent_directories=(not repo_path)) - - # comparator of commits - def cmp(x, y): - if str(x) == str(y): - return 0 - if self._repo.is_ancestor(x, y): - return -1 - else: - return 1 - - self.comparator = functools.cmp_to_key(cmp) - - def iterate(self, begin, end): - rev_range = f"{begin}...{end}" - for commit in self._repo.iter_commits(rev_range, first_parent=True): - yield commit - - -class Repository(RepositoryBase): - def __init__(self, repo_path, remote_name, default_branch_name): - super().__init__(repo_path) - self._remote = self._repo.remotes[remote_name] - self._remote.fetch() - self._default = self._remote.refs[default_branch_name] - - def get_head_commit(self): - return self._repo.commit(self._default) - - def get_release_branches(self): - """ - Returns sorted list of tuples: - * remote branch (git.refs.remote.RemoteReference), - * base commit (git.Commit), - * head (git.Commit)). - List is sorted by commits in ascending order. - """ - release_branches = [] - - RE_RELEASE_BRANCH_REF = re.compile(r"^refs/remotes/.+/\d+\.\d+$") - - for branch in [ - r for r in self._remote.refs if RE_RELEASE_BRANCH_REF.match(r.path) - ]: - base = self._repo.merge_base(self._default, self._repo.commit(branch)) - if not base: - logging.info( - "Branch %s is not based on branch %s. Ignoring.", - branch.path, - self._default, - ) - elif len(base) > 1: - logging.info( - "Branch %s has more than one base commit. Ignoring.", branch.path - ) - else: - release_branches.append((os.path.basename(branch.name), base[0])) - - return sorted(release_branches, key=lambda x: self.comparator(x[1])) - - -class BareRepository(RepositoryBase): - def __init__(self, repo_path, default_branch_name): - super().__init__(repo_path) - self._default = self._repo.branches[default_branch_name] - - def get_release_branches(self): - """ - Returns sorted list of tuples: - * branch (git.refs.head?), - * base commit (git.Commit), - * head (git.Commit)). - List is sorted by commits in ascending order. - """ - release_branches = [] - - RE_RELEASE_BRANCH_REF = re.compile(r"^refs/heads/\d+\.\d+$") - - for branch in [ - r for r in self._repo.branches if RE_RELEASE_BRANCH_REF.match(r.path) - ]: - base = self._repo.merge_base(self._default, self._repo.commit(branch)) - if not base: - logging.info( - "Branch %s is not based on branch %s. Ignoring.", - branch.path, - self._default, - ) - elif len(base) > 1: - logging.info( - "Branch %s has more than one base commit. Ignoring.", branch.path - ) - else: - release_branches.append((os.path.basename(branch.name), base[0])) - - return sorted(release_branches, key=lambda x: self.comparator(x[1])) diff --git a/tests/ci/cherry_pick_utils/parser.py b/tests/ci/cherry_pick_utils/parser.py deleted file mode 100644 index 29c05e5328f..00000000000 --- a/tests/ci/cherry_pick_utils/parser.py +++ /dev/null @@ -1,56 +0,0 @@ -# -*- coding: utf-8 -*- - - -class Description: - """Parsed description representation""" - - MAP_CATEGORY_TO_LABEL = { - "New Feature": "pr-feature", - "Bug Fix": "pr-bugfix", - "Improvement": "pr-improvement", - "Performance Improvement": "pr-performance", - # 'Backward Incompatible Change': doesn't match anything - "Build/Testing/Packaging Improvement": "pr-build", - "Non-significant (changelog entry is not needed)": "pr-non-significant", - "Non-significant (changelog entry is not required)": "pr-non-significant", - "Non-significant": "pr-non-significant", - "Documentation (changelog entry is not required)": "pr-documentation", - # 'Other': doesn't match anything - } - - def __init__(self, pull_request): - self.label_name = str() - self._parse(pull_request["bodyText"]) - - def _parse(self, text): - lines = text.splitlines() - next_category = False - category = str() - - for line in lines: - stripped = line.strip() - - if not stripped: - continue - - if next_category: - category = stripped - next_category = False - - category_headers = ( - "Category (leave one):", - "Changelog category (leave one):", - "Changelog category:", - "Category:", - ) - - if stripped in category_headers: - next_category = True - - if category in Description.MAP_CATEGORY_TO_LABEL: - self.label_name = Description.MAP_CATEGORY_TO_LABEL[category] - else: - if not category: - print("Cannot find category in pr description") - else: - print(("Unknown category: " + category)) diff --git a/tests/ci/cherry_pick_utils/query.py b/tests/ci/cherry_pick_utils/query.py deleted file mode 100644 index 917f9901287..00000000000 --- a/tests/ci/cherry_pick_utils/query.py +++ /dev/null @@ -1,532 +0,0 @@ -# -*- coding: utf-8 -*- - -import json -import inspect -import logging -import time -from urllib3.util.retry import Retry # type: ignore - -import requests # type: ignore -from requests.adapters import HTTPAdapter # type: ignore - - -class Query: - """ - Implements queries to the Github API using GraphQL - """ - - _PULL_REQUEST = """ -author {{ - ... on User {{ - id - login - }} -}} - -baseRepository {{ - nameWithOwner -}} - -mergeCommit {{ - oid - parents(first: {min_page_size}) {{ - totalCount - nodes {{ - oid - }} - }} -}} - -mergedBy {{ - ... on User {{ - id - login - }} -}} - -baseRefName -closed -headRefName -id -mergeable -merged -number -title -url - """ - - def __init__(self, token, owner, name, team, max_page_size=100, min_page_size=10): - self._PULL_REQUEST = Query._PULL_REQUEST.format(min_page_size=min_page_size) - - self._token = token - self._owner = owner - self._name = name - self._team = team - self._session = None - - self._max_page_size = max_page_size - self._min_page_size = min_page_size - - self.api_costs = {} - - repo = self.get_repository() - self._id = repo["id"] - self.ssh_url = repo["sshUrl"] - self.default_branch = repo["defaultBranchRef"]["name"] - - self.members = set(self.get_members()) - - def get_repository(self): - _QUERY = """ -repository(owner: "{owner}" name: "{name}") {{ - defaultBranchRef {{ - name - }} - id - sshUrl -}} - """ - - query = _QUERY.format(owner=self._owner, name=self._name) - return self._run(query)["repository"] - - def get_members(self): - """Get all team members for organization - - Returns: - members: a map of members' logins to ids - """ - - _QUERY = """ -organization(login: "{organization}") {{ - team(slug: "{team}") {{ - members(first: {max_page_size} {next}) {{ - pageInfo {{ - hasNextPage - endCursor - }} - nodes {{ - id - login - }} - }} - }} -}} - """ - - members = {} - not_end = True - query = _QUERY.format( - organization=self._owner, - team=self._team, - max_page_size=self._max_page_size, - next="", - ) - - while not_end: - result = self._run(query)["organization"]["team"] - if result is None: - break - result = result["members"] - not_end = result["pageInfo"]["hasNextPage"] - query = _QUERY.format( - organization=self._owner, - team=self._team, - max_page_size=self._max_page_size, - next=f'after: "{result["pageInfo"]["endCursor"]}"', - ) - - # Update members with new nodes compatible with py3.8-py3.10 - members = { - **members, - **{node["login"]: node["id"] for node in result["nodes"]}, - } - - return members - - def get_pull_request(self, number): - _QUERY = """ -repository(owner: "{owner}" name: "{name}") {{ - pullRequest(number: {number}) {{ - {pull_request_data} - }} -}} - """ - - query = _QUERY.format( - owner=self._owner, - name=self._name, - number=number, - pull_request_data=self._PULL_REQUEST, - min_page_size=self._min_page_size, - ) - return self._run(query)["repository"]["pullRequest"] - - def find_pull_request(self, base, head): - _QUERY = """ -repository(owner: "{owner}" name: "{name}") {{ - pullRequests( - first: {min_page_size} baseRefName: "{base}" headRefName: "{head}" - ) {{ - nodes {{ - {pull_request_data} - }} - totalCount - }} -}} - """ - - query = _QUERY.format( - owner=self._owner, - name=self._name, - base=base, - head=head, - pull_request_data=self._PULL_REQUEST, - min_page_size=self._min_page_size, - ) - result = self._run(query)["repository"]["pullRequests"] - if result["totalCount"] > 0: - return result["nodes"][0] - else: - return {} - - def find_pull_requests(self, label_name): - """ - Get all pull-requests filtered by label name - """ - _QUERY = """ -repository(owner: "{owner}" name: "{name}") {{ - pullRequests(first: {min_page_size} labels: "{label_name}" states: OPEN) {{ - nodes {{ - {pull_request_data} - }} - }} -}} - """ - - query = _QUERY.format( - owner=self._owner, - name=self._name, - label_name=label_name, - pull_request_data=self._PULL_REQUEST, - min_page_size=self._min_page_size, - ) - return self._run(query)["repository"]["pullRequests"]["nodes"] - - def get_pull_requests(self, before_commit): - """ - Get all merged pull-requests from the HEAD of default branch to the last commit (excluding) - """ - - _QUERY = """ -repository(owner: "{owner}" name: "{name}") {{ - defaultBranchRef {{ - target {{ - ... on Commit {{ - history(first: {max_page_size} {next}) {{ - pageInfo {{ - hasNextPage - endCursor - }} - nodes {{ - oid - associatedPullRequests(first: {min_page_size}) {{ - totalCount - nodes {{ - ... on PullRequest {{ - {pull_request_data} - - labels(first: {min_page_size}) {{ - totalCount - pageInfo {{ - hasNextPage - endCursor - }} - nodes {{ - name - color - }} - }} - }} - }} - }} - }} - }} - }} - }} - }} -}} - """ - - pull_requests = [] - not_end = True - query = _QUERY.format( - owner=self._owner, - name=self._name, - max_page_size=self._max_page_size, - min_page_size=self._min_page_size, - pull_request_data=self._PULL_REQUEST, - next="", - ) - - while not_end: - result = self._run(query)["repository"]["defaultBranchRef"]["target"][ - "history" - ] - not_end = result["pageInfo"]["hasNextPage"] - query = _QUERY.format( - owner=self._owner, - name=self._name, - max_page_size=self._max_page_size, - min_page_size=self._min_page_size, - pull_request_data=self._PULL_REQUEST, - next=f'after: "{result["pageInfo"]["endCursor"]}"', - ) - - for commit in result["nodes"]: - # FIXME: maybe include `before_commit`? - if str(commit["oid"]) == str(before_commit): - not_end = False - break - - # TODO: fetch all pull-requests that were merged in a single commit. - assert ( - commit["associatedPullRequests"]["totalCount"] - <= self._min_page_size - ) - - for pull_request in commit["associatedPullRequests"]["nodes"]: - if ( - pull_request["baseRepository"]["nameWithOwner"] - == f"{self._owner}/{self._name}" - and pull_request["baseRefName"] == self.default_branch - and pull_request["mergeCommit"]["oid"] == commit["oid"] - ): - pull_requests.append(pull_request) - - return pull_requests - - def create_pull_request( - self, source, target, title, description="", draft=False, can_modify=True - ): - _QUERY = """ -createPullRequest(input: {{ - baseRefName: "{target}", - headRefName: "{source}", - repositoryId: "{id}", - title: "{title}", - body: "{body}", - draft: {draft}, - maintainerCanModify: {modify} -}}) {{ - pullRequest {{ - {pull_request_data} - }} -}} - """ - - query = _QUERY.format( - target=target, - source=source, - id=self._id, - title=title, - body=description, - draft="true" if draft else "false", - modify="true" if can_modify else "false", - pull_request_data=self._PULL_REQUEST, - ) - return self._run(query, is_mutation=True)["createPullRequest"]["pullRequest"] - - def merge_pull_request(self, pr_id): - _QUERY = """ -mergePullRequest(input: {{ - pullRequestId: "{pr_id}" -}}) {{ - pullRequest {{ - {pull_request_data} - }} -}} - """ - - query = _QUERY.format(pr_id=pr_id, pull_request_data=self._PULL_REQUEST) - return self._run(query, is_mutation=True)["mergePullRequest"]["pullRequest"] - - # FIXME: figure out how to add more assignees at once - def add_assignee(self, pr, assignee): - _QUERY = """ -addAssigneesToAssignable(input: {{ - assignableId: "{id1}", - assigneeIds: "{id2}" -}}) {{ - clientMutationId -}} - """ - - query = _QUERY.format(id1=pr["id"], id2=assignee["id"]) - self._run(query, is_mutation=True) - - def set_label(self, pull_request, label_name): - """ - Set label by name to the pull request - - Args: - pull_request: JSON object returned by `get_pull_requests()` - label_name (string): label name - """ - - _GET_LABEL = """ -repository(owner: "{owner}" name: "{name}") {{ - labels(first: {max_page_size} {next} query: "{label_name}") {{ - pageInfo {{ - hasNextPage - endCursor - }} - nodes {{ - id - name - color - }} - }} -}} - """ - - _SET_LABEL = """ -addLabelsToLabelable(input: {{ - labelableId: "{pr_id}", - labelIds: "{label_id}" -}}) {{ - clientMutationId -}} - """ - - labels = [] - not_end = True - query = _GET_LABEL.format( - owner=self._owner, - name=self._name, - label_name=label_name, - max_page_size=self._max_page_size, - next="", - ) - - while not_end: - result = self._run(query)["repository"]["labels"] - not_end = result["pageInfo"]["hasNextPage"] - query = _GET_LABEL.format( - owner=self._owner, - name=self._name, - label_name=label_name, - max_page_size=self._max_page_size, - next=f'after: "{result["pageInfo"]["endCursor"]}"', - ) - - labels += list(result["nodes"]) - - if not labels: - return - - query = _SET_LABEL.format(pr_id=pull_request["id"], label_id=labels[0]["id"]) - self._run(query, is_mutation=True) - - @property - def session(self): - if self._session is not None: - return self._session - retries = 5 - self._session = requests.Session() - retry = Retry( - total=retries, - read=retries, - connect=retries, - backoff_factor=1, - status_forcelist=(403, 500, 502, 504), - ) - adapter = HTTPAdapter(max_retries=retry) - self._session.mount("http://", adapter) - self._session.mount("https://", adapter) - return self._session - - def _run(self, query, is_mutation=False): - # Get caller and parameters from the stack to track the progress - frame = inspect.getouterframes(inspect.currentframe(), 2)[1] - caller = frame[3] - f_parameters = inspect.signature(getattr(self, caller)).parameters - parameters = ", ".join(str(frame[0].f_locals[p]) for p in f_parameters) - mutation = "" - if is_mutation: - mutation = ", is mutation" - print(f"---GraphQL request for {caller}({parameters}){mutation}---") - - headers = {"Authorization": f"bearer {self._token}"} - if is_mutation: - query = f""" -mutation {{ - {query} -}} - """ - else: - query = f""" -query {{ - {query} - rateLimit {{ - cost - remaining - }} -}} - """ - - def request_with_retry(retry=0): - max_retries = 5 - # From time to time we face some concrete errors, when it worth to - # retry instead of failing competely - # We should sleep progressively - progressive_sleep = 5 * sum(i + 1 for i in range(retry)) - if progressive_sleep: - logging.warning( - "Retry GraphQL request %s time, sleep %s seconds", - retry, - progressive_sleep, - ) - time.sleep(progressive_sleep) - response = self.session.post( - "https://api.github.com/graphql", json={"query": query}, headers=headers - ) - result = response.json() - if response.status_code == 200: - if "errors" in result: - raise Exception( - f"Errors occurred: {result['errors']}\nOriginal query: {query}" - ) - - if not is_mutation: - if caller not in self.api_costs: - self.api_costs[caller] = 0 - self.api_costs[caller] += result["data"]["rateLimit"]["cost"] - - return result["data"] - elif ( - response.status_code == 403 - and "secondary rate limit" in result["message"] - ): - if retry <= max_retries: - logging.warning("Secondary rate limit reached") - return request_with_retry(retry + 1) - elif response.status_code == 502 and "errors" in result: - too_many_data = any( - True - for err in result["errors"] - if "message" in err - and "This may be the result of a timeout" in err["message"] - ) - if too_many_data: - logging.warning( - "Too many data is requested, decreasing page size %s by 10%%", - self._max_page_size, - ) - self._max_page_size = int(self._max_page_size * 0.9) - return request_with_retry(retry) - - data = json.dumps(result, indent=4) - raise Exception(f"Query failed with code {response.status_code}:\n{data}") - - return request_with_retry() diff --git a/tests/ci/cherry_pick_utils/readme.md b/tests/ci/cherry_pick_utils/readme.md deleted file mode 100644 index 10ae9ca4b0b..00000000000 --- a/tests/ci/cherry_pick_utils/readme.md +++ /dev/null @@ -1,3 +0,0 @@ -# Some scripts for backports implementation - -TODO: Remove copy from utils/github diff --git a/tests/ci/git_helper.py b/tests/ci/git_helper.py index e3ad0eb39c0..77c2fc9cf05 100644 --- a/tests/ci/git_helper.py +++ b/tests/ci/git_helper.py @@ -1,10 +1,13 @@ #!/usr/bin/env python import argparse +import logging import os.path as p import re import subprocess from typing import List, Optional +logger = logging.getLogger(__name__) + # ^ and $ match subline in `multiple\nlines` # \A and \Z match only start and end of the whole string RELEASE_BRANCH_REGEXP = r"\A\d+[.]\d+\Z" @@ -55,6 +58,7 @@ class Runner: def run(self, cmd: str, cwd: Optional[str] = None, **kwargs) -> str: if cwd is None: cwd = self.cwd + logger.debug("Running command: %s", cmd) return subprocess.check_output( cmd, shell=True, cwd=cwd, encoding="utf-8", **kwargs ).strip() @@ -70,6 +74,9 @@ class Runner: return self._cwd = value + def __call__(self, *args, **kwargs): + return self.run(*args, **kwargs) + git_runner = Runner() # Set cwd to abs path of git root @@ -109,8 +116,8 @@ class Git: def update(self): """Is used to refresh all attributes after updates, e.g. checkout or commit""" - self.branch = self.run("git branch --show-current") self.sha = self.run("git rev-parse HEAD") + self.branch = self.run("git branch --show-current") or self.sha self.sha_short = self.sha[:11] # The following command shows the most recent tag in a graph # Format should match TAG_REGEXP diff --git a/tests/ci/github_helper.py b/tests/ci/github_helper.py new file mode 100644 index 00000000000..74793101b17 --- /dev/null +++ b/tests/ci/github_helper.py @@ -0,0 +1,162 @@ +#!/usr/bin/env python +"""Helper for GitHub API requests""" +import logging +from datetime import date, datetime, timedelta +from pathlib import Path +from os import path as p +from time import sleep +from typing import List, Optional + +import github +from github.GithubException import RateLimitExceededException +from github.Issue import Issue +from github.PullRequest import PullRequest +from github.Repository import Repository + +CACHE_PATH = p.join(p.dirname(p.realpath(__file__)), "gh_cache") + +logger = logging.getLogger(__name__) + +PullRequests = List[PullRequest] +Issues = List[Issue] + + +class GitHub(github.Github): + def __init__(self, *args, **kwargs): + # Define meta attribute + self._cache_path = Path(CACHE_PATH) + # And set Path + super().__init__(*args, **kwargs) + + # pylint: disable=signature-differs + def search_issues(self, *args, **kwargs) -> Issues: # type: ignore + """Wrapper around search method with throttling and splitting by date. + + We split only by the first""" + splittable = False + for arg, value in kwargs.items(): + if arg in ["closed", "created", "merged", "updated"]: + if ( + hasattr(value, "__iter__") + and not isinstance(value, str) + and not splittable + ): + assert [True for v in value if isinstance(v, (date, datetime))] + assert len(value) == 2 + preserved_arg = arg + preserved_value = value + middle_value = value[0] + (value[1] - value[0]) / 2 + splittable = middle_value not in value + kwargs[arg] = f"{value[0].isoformat()}..{value[1].isoformat()}" + continue + assert isinstance(value, (date, datetime, str)) + + inter_result = [] # type: Issues + for i in range(3): + try: + logger.debug("Search issues, args=%s, kwards=%s", args, kwargs) + result = super().search_issues(*args, **kwargs) + if result.totalCount == 1000 and splittable: + # The hard limit is 1000. If it's splittable, then we make + # two subrequests requests with less time frames + logger.debug( + "The search result contain exactly 1000 results, " + "splitting %s=%s by middle point %s", + preserved_arg, + kwargs[preserved_arg], + middle_value, + ) + kwargs[preserved_arg] = [preserved_value[0], middle_value] + inter_result.extend(self.search_issues(*args, **kwargs)) + if isinstance(middle_value, date): + # When middle_value is a date, 2022-01-01..2022-01-03 + # is split to 2022-01-01..2022-01-02 and + # 2022-01-02..2022-01-03, so we have results for + # 2022-01-02 twicely. We split it to + # 2022-01-01..2022-01-02 and 2022-01-03..2022-01-03. + # 2022-01-01..2022-01-02 aren't split, see splittable + middle_value += timedelta(days=1) + kwargs[preserved_arg] = [middle_value, preserved_value[1]] + inter_result.extend(self.search_issues(*args, **kwargs)) + return inter_result + + inter_result.extend(result) + return inter_result + except RateLimitExceededException as e: + if i == 2: + exception = e + self.sleep_on_rate_limit() + + raise exception + + # pylint: enable=signature-differs + def get_pulls_from_search(self, *args, **kwargs) -> PullRequests: + """The search api returns actually issues, so we need to fetch PullRequests""" + issues = self.search_issues(*args, **kwargs) + repos = {} + prs = [] # type: PullRequests + for issue in issues: + # See https://github.com/PyGithub/PyGithub/issues/2202, + # obj._rawData doesn't spend additional API requests + # pylint: disable=protected-access + repo_url = issue._rawData["repository_url"] # type: ignore + if repo_url not in repos: + repos[repo_url] = issue.repository + prs.append( + self.get_pull_cached(repos[repo_url], issue.number, issue.updated_at) + ) + return prs + + def sleep_on_rate_limit(self): + for limit, data in self.get_rate_limit().raw_data.items(): + if data["remaining"] == 0: + sleep_time = data["reset"] - int(datetime.now().timestamp()) + 1 + if sleep_time > 0: + logger.warning( + "Faced rate limit for '%s' requests type, sleeping %s", + limit, + sleep_time, + ) + sleep(sleep_time) + return + + def get_pull_cached( + self, repo: Repository, number: int, updated_at: Optional[datetime] = None + ) -> PullRequest: + pr_cache_file = self.cache_path / f"{number}.pickle" + if updated_at is None: + updated_at = datetime.now() - timedelta(hours=-1) + + def _get_pr(path: Path) -> PullRequest: + with open(path, "rb") as prfd: + return self.load(prfd) # type: ignore + + if pr_cache_file.is_file(): + cached_pr = _get_pr(pr_cache_file) + if updated_at <= cached_pr.updated_at: + logger.debug("Getting PR #%s from cache", number) + return cached_pr + for i in range(3): + try: + pr = repo.get_pull(number) + break + except RateLimitExceededException: + if i == 2: + raise + self.sleep_on_rate_limit() + logger.debug("Getting PR #%s from API", number) + with open(pr_cache_file, "wb") as prfd: + self.dump(pr, prfd) # type: ignore + return pr + + @property + def cache_path(self): + return self._cache_path + + @cache_path.setter + def cache_path(self, value: str): + self._cache_path = Path(value) + if self._cache_path.exists(): + assert self._cache_path.is_dir() + else: + self._cache_path.mkdir(parents=True) From 0c766a6cfc545168ffe2d63093134239d59bacc7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 14 Jul 2022 21:04:02 +0200 Subject: [PATCH 282/659] Rename workflow yml to reflect the name, increase frequency --- .github/workflows/{backport.yml => cherry_pick.yml} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename .github/workflows/{backport.yml => cherry_pick.yml} (98%) diff --git a/.github/workflows/backport.yml b/.github/workflows/cherry_pick.yml similarity index 98% rename from .github/workflows/backport.yml rename to .github/workflows/cherry_pick.yml index da42bbae78a..e6a10479c7e 100644 --- a/.github/workflows/backport.yml +++ b/.github/workflows/cherry_pick.yml @@ -8,7 +8,7 @@ concurrency: group: cherry-pick on: # yamllint disable-line rule:truthy schedule: - - cron: '0 */3 * * *' + - cron: '0 * * * *' workflow_dispatch: jobs: From 30b0995532642b98e236bc15526c41fc4ecd20bc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 15 Jul 2022 16:08:29 +0200 Subject: [PATCH 283/659] Add early check if PR is already in a release branch --- tests/ci/cherry_pick.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 834c9296a59..92ac7e2ee3e 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -85,6 +85,15 @@ Merge it only if you intend to backport changes to the target branch, otherwise "git -c user.email=robot-clickhouse@clickhouse.com " "-c user.name=robot-clickhouse -c commit.gpgsign=false" ) + self.pre_check() + + def pre_check(self): + branch_updated = git_runner( + f"git branch -a --contains={self.pr.merge_commit_sha} " + f"{self.REMOTE}/{self.name}" + ) + if branch_updated: + self._backported = True def pop_prs(self, prs: PullRequests): to_pop = [] # type: List[int] From 48797660ffb9dad3c0d19bf80916b390b8f97027 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 15 Jul 2022 16:10:00 +0200 Subject: [PATCH 284/659] Process exceptions per PR to not fail the script --- tests/ci/cherry_pick.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 92ac7e2ee3e..c895612c2dd 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -315,7 +315,13 @@ class Backport: def process_backports(self): for pr in self.prs_for_backport: - self.process_pr(pr) + try: + self.process_pr(pr) + except Exception as e: + logging.error( + "During processing the PR #%s error occured: %s", pr.number, e + ) + self.error = True def process_pr(self, pr: PullRequest): pr_labels = [label.name for label in pr.labels] @@ -376,13 +382,7 @@ class Backport: return for br in branches: - try: - br.process(self.dry_run) - except Exception as e: - logging.error( - "During processing the PR #%s error occured: %s", pr.number, e - ) - self.error = True + br.process(self.dry_run) if all(br.backported for br in branches): # And check it after the running From dc6db5b5d9e34a06e3618892004c8d1d43bcd350 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 15 Jul 2022 16:33:01 +0200 Subject: [PATCH 285/659] Replace sys.exit by Exception to rollback stash and clear_repo --- tests/ci/cherry_pick.py | 27 ++++++++++++--------------- 1 file changed, 12 insertions(+), 15 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index c895612c2dd..aa99c666a14 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -25,7 +25,6 @@ Cherry-pick stage: import argparse import logging import os -import sys from contextlib import contextmanager from datetime import date, timedelta from subprocess import CalledProcessError @@ -260,7 +259,7 @@ class Backport: self.release_branches = [] # type: List[str] self.labels_to_backport = [] # type: List[str] self.prs_for_backport = [] # type: PullRequests - self.error = False + self.error = None # type: Optional[Exception] @property def remote(self) -> str: @@ -321,7 +320,7 @@ class Backport: logging.error( "During processing the PR #%s error occured: %s", pr.number, e ) - self.error = True + self.error = e def process_pr(self, pr: PullRequest): pr_labels = [label.name for label in pr.labels] @@ -341,11 +340,10 @@ class Backport: if not branches: # This is definitely some error. There must be at least one branch # It also make the whole program exit code non-zero - logging.error( - "There are no branches to backport PR #%s, logical error", pr.number + self.error = Exception( + f"There are no branches to backport PR #{pr.number}, logical error" ) - self.error = True - return + raise self.error logging.info( " PR #%s is suppose to be backported to %s", @@ -369,12 +367,11 @@ class Backport: # This is definitely some error. All prs must be consumed by # branches with ReleaseBranch.pop_prs. It also make the whole # program exit code non-zero - logging.error( - "The following PRs are not filtered by release branches:\n%s", - "\n".join(map(str, bp_cp_prs)), + self.error = Exception( + "The following PRs are not filtered by release branches:\n" + "\n".join(map(str, bp_cp_prs)) ) - self.error = True - return + raise self.error if all(br.backported for br in branches): # Let's check if the PR is already backported @@ -481,9 +478,9 @@ def main(): bp.receive_release_prs() bp.receive_prs_for_backport() bp.process_backports() - if bp.error: - logging.error("Finished successfully, but errors occured") - sys.exit(1) + if bp.error is not None: + logging.error("Finished successfully, but errors occured!") + raise bp.error if __name__ == "__main__": From abb550d2ff9289afa39c529a40a0c81e843fe0c7 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 15 Jul 2022 12:42:07 -0300 Subject: [PATCH 286/659] minor style fixes --- src/Common/CaresPTRResolver.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/CaresPTRResolver.cpp b/src/Common/CaresPTRResolver.cpp index eacc3bffe54..f6228e97c02 100644 --- a/src/Common/CaresPTRResolver.cpp +++ b/src/Common/CaresPTRResolver.cpp @@ -92,12 +92,12 @@ namespace DB fd_set write_fds; int nfds; - for(;;) + for (;;) { FD_ZERO(&read_fds); FD_ZERO(&write_fds); nfds = ares_fds(channel, &read_fds,&write_fds); - if(nfds == 0) + if (nfds == 0) { break; } From 6e48b08190310c4f0c1c50b91f913fecccdd60de Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 15 Jul 2022 17:10:37 +0200 Subject: [PATCH 287/659] Clean-up some staff left after finishing --- tests/ci/cherry_pick.py | 62 +++++++++++++++++++++-------------------- 1 file changed, 32 insertions(+), 30 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index aa99c666a14..334a24ed7af 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -1,25 +1,26 @@ #!/usr/bin/env python3 """ A plan: - - Receive GH objects cache from S3, ignore if fails - - Get all open release PRs + - TODO: consider receiving GH objects cache from S3, but it's really a few + of requests to API currently + - Get all open release PRs (20.10, 21.8, 22.5, etc.) - Get all pull-requests between the date of the merge-base for the oldest PR with labels pr-must-backport and version-specific v21.8-must-backport, but without pr-backported - Iterate over gotten PRs: - for pr-must-backport: - check if all backport-PRs are created. If yes, - set pr-backported label + set pr-backported label and finish - If not, create either cherrypick PRs or merge cherrypick (in the same - stage, if mergable?) and create backport-PRs + stage, if mergable) and create backport-PRs - If successfull, set pr-backported label on the PR - for version-specific labels: - - the same, check, cherry-pick, backport + - the same, check, cherry-pick, backport, pr-backported Cherry-pick stage: - From time to time the cherry-pick fails, if it was done manually. In the - case we should check if it's even needed, and mark the release as done somehow. + case we check if it's even needed, and mark the release as done somehow. """ import argparse @@ -39,13 +40,10 @@ from github_helper import ( PullRequests, Repository, ) -from github.Label import Label from ssh import SSHKey -Labels = List[Label] - -class labels: +class Labels: LABEL_MUST_BACKPORT = "pr-must-backport" LABEL_BACKPORT = "pr-backport" LABEL_BACKPORTED = "pr-backported" @@ -161,7 +159,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise # Create or reset backport branch git_runner(f"{self.git_prefix} checkout -B {self.backport_branch}") # Merge all changes from PR's the first parent commit w/o applying anything - # It will produce the commit like cherry-pick + # It will allow to create a merge commit like it would be a cherry-pick first_parent = git_runner(f"git rev-parse {self.pr.merge_commit_sha}^1") git_runner(f"{self.git_prefix} merge -s ours --no-edit {first_parent}") @@ -172,12 +170,13 @@ Merge it only if you intend to backport changes to the target branch, otherwise ) # Check if there actually any changes between branches. If no, then no - # other actions are required. + # other actions are required. It's possible when changes are backported + # manually to the release branch already try: output = git_runner( f"{self.git_prefix} merge --no-commit --no-ff {self.cherrypick_branch}" ) - # 'up-to-date', 'up to date', who knows what else + # 'up-to-date', 'up to date', who knows what else (╯°v°)╯ ^┻━┻ if output.startswith("Already up") and output.endswith("date."): # The changes are already in the release branch, we are done here logging.info( @@ -191,9 +190,10 @@ Merge it only if you intend to backport changes to the target branch, otherwise # There are most probably conflicts, they'll be resolved in PR git_runner(f"{self.git_prefix} reset --merge") else: - # There are changes able to apply, so continue + # There are changes to apply, so continue git_runner(f"{self.git_prefix} reset --merge") + # Push, create the cherrypick PR, lable and assign it for branch in [self.cherrypick_branch, self.backport_branch]: git_runner(f"{self.git_prefix} push -f {self.REMOTE} {branch}:{branch}") @@ -204,12 +204,14 @@ Merge it only if you intend to backport changes to the target branch, otherwise base=self.backport_branch, head=self.cherrypick_branch, ) - self.cherrypick_pr.add_to_labels(labels.LABEL_CHERRYPICK) - self.cherrypick_pr.add_to_labels(labels.LABEL_DO_NOT_TEST) + self.cherrypick_pr.add_to_labels(Labels.LABEL_CHERRYPICK) + self.cherrypick_pr.add_to_labels(Labels.LABEL_DO_NOT_TEST) self.cherrypick_pr.add_to_assignees(self.pr.assignee) self.cherrypick_pr.add_to_assignees(self.pr.user) def create_backport(self): + # Checkout the backport branch from the remote and make all changes to + # apply like they are only one cherry-pick commit on top of release git_runner(f"{self.git_prefix} checkout -f {self.backport_branch}") git_runner( f"{self.git_prefix} pull --ff-only {self.REMOTE} {self.backport_branch}" @@ -221,6 +223,8 @@ Merge it only if you intend to backport changes to the target branch, otherwise git_runner(f"{self.git_prefix} reset --soft {merge_base}") title = f"Backport #{self.pr.number} to {self.name}: {self.pr.title}" git_runner(f"{self.git_prefix} commit -a --allow-empty -F -", input=title) + + # Push with force, create the backport PR, lable and assign it git_runner( f"{self.git_prefix} push -f {self.REMOTE} " f"{self.backport_branch}:{self.backport_branch}" @@ -233,7 +237,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise base=self.name, head=self.backport_branch, ) - self.backport_pr.add_to_labels(labels.LABEL_BACKPORT) + self.backport_pr.add_to_labels(Labels.LABEL_BACKPORT) self.backport_pr.add_to_assignees(self.pr.assignee) self.backport_pr.add_to_assignees(self.pr.user) @@ -252,9 +256,10 @@ class Backport: self.gh = gh self._repo_name = repo self.dry_run = dry_run - self._repo = None # type: Optional[Repository] - self._remote = "" + self._query = f"type:pr repo:{repo}" + self._remote = "" + self._repo = None # type: Optional[Repository] self.release_prs = [] # type: PullRequests self.release_branches = [] # type: List[str] self.labels_to_backport = [] # type: List[str] @@ -270,8 +275,8 @@ class Backport: self._remote = tuple( remote.split(maxsplit=1)[0] for remote in remotes - if f"github.com/{self._repo_name}" in remote # ssh - or f"github.com:{self._repo_name}" in remote # https + if f"github.com/{self._repo_name}" in remote # https + or f"github.com:{self._repo_name}" in remote # ssh )[0] git_runner(f"git fetch {self._remote}") ReleaseBranch.REMOTE = self._remote @@ -283,7 +288,6 @@ class Backport: query=f"{self._query} is:open", sort="created", order="asc", - type="pr", label="release", ) self.release_branches = [pr.head.ref for pr in self.release_prs] @@ -293,6 +297,7 @@ class Backport: logging.info("Active releases: %s", ", ".join(self.release_branches)) def receive_prs_for_backport(self): + # The commit is the oldest open release branch's merge-base since_commit = git_runner( f"git merge-base {self.remote}/{self.release_branches[0]} " f"{self.remote}/{self.default_branch}" @@ -300,11 +305,12 @@ class Backport: since_date = date.fromisoformat( git_runner.run(f"git log -1 --format=format:%cs {since_commit}") ) + # To not have a possible TZ issues tomorrow = date.today() + timedelta(days=1) logging.info("Receive PRs suppose to be backported") self.prs_for_backport = self.gh.get_pulls_from_search( query=f"{self._query} -label:pr-backported", - label=",".join(self.labels_to_backport + [labels.LABEL_MUST_BACKPORT]), + label=",".join(self.labels_to_backport + [Labels.LABEL_MUST_BACKPORT]), merged=[since_date, tomorrow], ) logging.info( @@ -324,7 +330,7 @@ class Backport: def process_pr(self, pr: PullRequest): pr_labels = [label.name for label in pr.labels] - if labels.LABEL_MUST_BACKPORT in pr_labels: + if Labels.LABEL_MUST_BACKPORT in pr_labels: branches = [ ReleaseBranch(br, pr) for br in self.release_branches ] # type: List[ReleaseBranch] @@ -389,17 +395,13 @@ class Backport: if self.dry_run: logging.info("DRY RUN: would mark PR #%s as done", pr.number) return - pr.add_to_labels(labels.LABEL_BACKPORTED) + pr.add_to_labels(Labels.LABEL_BACKPORTED) logging.info( "PR #%s is successfully labeled with `%s`", pr.number, - labels.LABEL_BACKPORTED, + Labels.LABEL_BACKPORTED, ) - @staticmethod - def pr_labels(pr: PullRequest) -> List[str]: - return [label.name for label in pr.labels] - @property def repo(self) -> Repository: if self._repo is None: From 9f1d6814fc2f8c31801d18b2b5d748ef2453f793 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 15 Jul 2022 17:57:22 +0200 Subject: [PATCH 288/659] Fix --- .../DatabaseMaterializedPostgreSQL.cpp | 12 ++++- .../fetchPostgreSQLTableStructure.cpp | 13 ++--- .../PostgreSQLReplicationHandler.cpp | 49 ++++++++++++++++--- .../PostgreSQL/PostgreSQLReplicationHandler.h | 10 +++- 4 files changed, 70 insertions(+), 14 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 8b85d1b9a63..08a0859e6db 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -79,6 +79,7 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() } catch (...) { + tryLogCurrentException(__PRETTY_FUNCTION__); LOG_ERROR(log, "Unable to load replicated tables list"); throw; } @@ -111,7 +112,16 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() } LOG_TRACE(log, "Loaded {} tables. Starting synchronization", materialized_tables.size()); - replication_handler->startup(/* delayed */false); + + try + { + replication_handler->startup(/* delayed */false); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + throw; + } } diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 10cde43e9e1..3405e6e636d 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -39,9 +39,10 @@ std::set fetchPostgreSQLTablesList(T & tx, const String & postgres_schem std::set tables; if (schemas.size() <= 1) { - std::string query = fmt::format("SELECT tablename FROM pg_catalog.pg_tables " - "WHERE schemaname != 'pg_catalog' AND {}", - postgres_schema.empty() ? "schemaname != 'information_schema'" : "schemaname = " + quoteString(postgres_schema)); + std::string query = fmt::format( + "SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname != 'pg_catalog' AND schemaname = {}", + postgres_schema.empty() ? quoteString("public") : quoteString(postgres_schema)); + for (auto table_name : tx.template stream(query)) tables.insert(std::get<0>(table_name)); @@ -53,9 +54,9 @@ std::set fetchPostgreSQLTablesList(T & tx, const String & postgres_schem /// If we add schema to table name then table can be accessed only this way: database_name.`schema_name.table_name` for (const auto & schema : schemas) { - std::string query = fmt::format("SELECT tablename FROM pg_catalog.pg_tables " - "WHERE schemaname != 'pg_catalog' AND {}", - postgres_schema.empty() ? "schemaname != 'information_schema'" : "schemaname = " + quoteString(schema)); + std::string query = fmt::format( + "SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname != 'pg_catalog' AND schemaname = {}", quoteString(schema)); + for (auto table_name : tx.template stream(query)) tables.insert(schema + '.' + std::get<0>(table_name)); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index c9c9877cc93..e0e2acc3436 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -31,6 +31,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; extern const int POSTGRESQL_REPLICATION_INTERNAL_ERROR; + extern const int QUERY_NOT_ALLOWED; } class TemporaryReplicationSlot @@ -188,6 +189,17 @@ void PostgreSQLReplicationHandler::shutdown() } +void PostgreSQLReplicationHandler::assertInitialized() const +{ + if (!replication_handler_initialized) + { + throw Exception( + ErrorCodes::QUERY_NOT_ALLOWED, + "PostgreSQL replication initialization did not finish successfully. Please check logs for error messages"); + } +} + + void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) { postgres::Connection replication_connection(connection_info, /* replication */true); @@ -239,7 +251,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) /// Throw in case of single MaterializedPostgreSQL storage, because initial setup is done immediately /// (unlike database engine where it is done in a separate thread). - if (throw_on_error) + if (throw_on_error && !is_materialized_postgresql_database) throw; } } @@ -314,6 +326,8 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) /// Do not rely anymore on saved storage pointers. materialized_storages.clear(); + + replication_handler_initialized = true; } @@ -393,12 +407,20 @@ void PostgreSQLReplicationHandler::cleanupFunc() cleanup_task->scheduleAfter(CLEANUP_RESCHEDULE_MS); } +PostgreSQLReplicationHandler::ConsumerPtr PostgreSQLReplicationHandler::getConsumer() +{ + if (!consumer) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Consumer not initialized"); + return consumer; +} void PostgreSQLReplicationHandler::consumerFunc() { + assertInitialized(); + std::vector> skipped_tables; - bool schedule_now = consumer->consume(skipped_tables); + bool schedule_now = getConsumer()->consume(skipped_tables); LOG_DEBUG(log, "checking for skipped tables: {}", skipped_tables.size()); if (!skipped_tables.empty()) @@ -603,8 +625,10 @@ void PostgreSQLReplicationHandler::removeTableFromPublication(pqxx::nontransacti void PostgreSQLReplicationHandler::setSetting(const SettingChange & setting) { + assertInitialized(); + consumer_task->deactivate(); - consumer->setSetting(setting); + getConsumer()->setSetting(setting); consumer_task->activateAndSchedule(); } @@ -758,6 +782,15 @@ std::set PostgreSQLReplicationHandler::fetchRequiredTables() { pqxx::nontransaction tx(connection.getRef()); result_tables = fetchPostgreSQLTablesList(tx, schema_list.empty() ? postgres_schema : schema_list); + + std::string tables_string; + for (const auto & table : result_tables) + { + if (!tables_string.empty()) + tables_string += ", "; + tables_string += table; + } + LOG_DEBUG(log, "Tables list was fetched from PostgreSQL directly: {}", tables_string); } } } @@ -824,6 +857,8 @@ PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( void PostgreSQLReplicationHandler::addTableToReplication(StorageMaterializedPostgreSQL * materialized_storage, const String & postgres_table_name) { + assertInitialized(); + /// Note: we have to ensure that replication consumer task is stopped when we reload table, because otherwise /// it can read wal beyond start lsn position (from which this table is being loaded), which will result in losing data. consumer_task->deactivate(); @@ -858,7 +893,7 @@ void PostgreSQLReplicationHandler::addTableToReplication(StorageMaterializedPost } /// Pass storage to consumer and lsn position, from which to start receiving replication messages for this table. - consumer->addNested(postgres_table_name, nested_storage_info, start_lsn); + getConsumer()->addNested(postgres_table_name, nested_storage_info, start_lsn); LOG_TRACE(log, "Table `{}` successfully added to replication", postgres_table_name); } catch (...) @@ -876,6 +911,8 @@ void PostgreSQLReplicationHandler::addTableToReplication(StorageMaterializedPost void PostgreSQLReplicationHandler::removeTableFromReplication(const String & postgres_table_name) { + assertInitialized(); + consumer_task->deactivate(); try { @@ -887,7 +924,7 @@ void PostgreSQLReplicationHandler::removeTableFromReplication(const String & pos } /// Pass storage to consumer and lsn position, from which to start receiving replication messages for this table. - consumer->removeNested(postgres_table_name); + getConsumer()->removeNested(postgres_table_name); } catch (...) { @@ -966,7 +1003,7 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectorgetStorageID().getNameForLogs(), nested_sample_block.dumpStructure()); /// Pass pointer to new nested table into replication consumer, remove current table from skip list and set start lsn position. - consumer->updateNested(table_name, StorageInfo(nested_storage, std::move(table_attributes)), relation_id, start_lsn); + getConsumer()->updateNested(table_name, StorageInfo(nested_storage, std::move(table_attributes)), relation_id, start_lsn); auto table_to_drop = DatabaseCatalog::instance().getTable(StorageID(temp_table_id.database_name, temp_table_id.table_name, table_id.uuid), nested_context); auto drop_table_id = table_to_drop->getStorageID(); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 16e531f5247..89f16457bfe 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -18,6 +18,8 @@ class PostgreSQLReplicationHandler : WithContext friend class TemporaryReplicationSlot; public: + using ConsumerPtr = std::shared_ptr; + PostgreSQLReplicationHandler( const String & replication_identifier, const String & postgres_database_, @@ -87,6 +89,8 @@ private: void consumerFunc(); + ConsumerPtr getConsumer(); + StorageInfo loadFromSnapshot(postgres::Connection & connection, std::string & snapshot_name, const String & table_name, StorageMaterializedPostgreSQL * materialized_storage); void reloadFromSnapshot(const std::vector> & relation_data); @@ -97,6 +101,8 @@ private: std::pair getSchemaAndTableName(const String & table_name) const; + void assertInitialized() const; + Poco::Logger * log; /// If it is not attach, i.e. a create query, then if publication already exists - always drop it. @@ -134,7 +140,7 @@ private: String replication_slot, publication_name; /// Replication consumer. Manages decoding of replication stream and syncing into tables. - std::shared_ptr consumer; + ConsumerPtr consumer; BackgroundSchedulePool::TaskHolder startup_task; BackgroundSchedulePool::TaskHolder consumer_task; @@ -146,6 +152,8 @@ private: MaterializedStorages materialized_storages; UInt64 milliseconds_to_wait; + + bool replication_handler_initialized = false; }; } From 750844827576d269fbb19e298e8d2fa95a139daf Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 15 Jul 2022 16:23:56 +0000 Subject: [PATCH 289/659] Better --- src/Functions/isNotNull.cpp | 1 + src/Functions/isNull.cpp | 1 + 2 files changed, 2 insertions(+) diff --git a/src/Functions/isNotNull.cpp b/src/Functions/isNotNull.cpp index 87437f49356..44ea8aeaeb0 100644 --- a/src/Functions/isNotNull.cpp +++ b/src/Functions/isNotNull.cpp @@ -51,6 +51,7 @@ public: const size_t null_index = low_cardinality_column->getDictionary().getNullValueIndex(); auto res = DataTypeUInt8().createColumn(); auto & data = typeid_cast(*res).getData(); + data.reserve(low_cardinality_column->size()); for (size_t i = 0; i != low_cardinality_column->size(); ++i) data.push_back(low_cardinality_column->getIndexAt(i) != null_index); return res; diff --git a/src/Functions/isNull.cpp b/src/Functions/isNull.cpp index 90f1607578b..e22b1cf469c 100644 --- a/src/Functions/isNull.cpp +++ b/src/Functions/isNull.cpp @@ -50,6 +50,7 @@ public: size_t null_index = low_cardinality_column->getDictionary().getNullValueIndex(); auto res = DataTypeUInt8().createColumn(); auto & data = typeid_cast(*res).getData(); + data.reserve(low_cardinality_column->size()); for (size_t i = 0; i != low_cardinality_column->size(); ++i) data.push_back(low_cardinality_column->getIndexAt(i) == null_index); return res; From 3436fcfda668530012d7812cd729f1ce803aa98c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 15 Jul 2022 18:24:44 +0200 Subject: [PATCH 290/659] Update tests/performance/low_cardinality_argument.xml Co-authored-by: Igor Nikonov <954088+devcrafter@users.noreply.github.com> --- tests/performance/low_cardinality_argument.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/low_cardinality_argument.xml b/tests/performance/low_cardinality_argument.xml index ba17809ba53..89ee58ffd07 100644 --- a/tests/performance/low_cardinality_argument.xml +++ b/tests/performance/low_cardinality_argument.xml @@ -3,7 +3,7 @@ CREATE TABLE test_lc (x UInt64, lc LowCardinality(Nullable(String))) ENGINE = MergeTree order by x - INSERT INTO test_lc SELECT number, number % 10 ? NULL : toString(number % 10000) FROM numbers(10000000) + INSERT INTO test_lc SELECT number, number % 10 ? NULL : toString(number % 10000) FROM numbers(1e7) SELECT isNullable(lc) FROM test_lc FORMAT Null SELECT isConstant(lc) FROM test_lc FORMAT Null From 6318927fba01ab4e9a914586e83f899a7f2b2eea Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 15 Jul 2022 18:37:50 +0200 Subject: [PATCH 291/659] addressed review comment: ForkWriteBuffer for loop --- src/IO/ForkWriteBuffer.cpp | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/src/IO/ForkWriteBuffer.cpp b/src/IO/ForkWriteBuffer.cpp index e91f6ca433b..dd910f7fdf4 100644 --- a/src/IO/ForkWriteBuffer.cpp +++ b/src/IO/ForkWriteBuffer.cpp @@ -1,7 +1,5 @@ #include #include -#include - namespace DB { @@ -28,14 +26,14 @@ void ForkWriteBuffer::nextImpl() try { - for (const WriteBufferPtr & write_buffer : sources | std::views::reverse) + auto & source_buffer = sources.front(); + for (auto it = sources.begin() + 1; it != sources.end(); ++it) { - if (write_buffer != sources.front()) - { - write_buffer->write(sources.front()->buffer().begin(), sources.front()->offset()); - } - write_buffer->next(); + auto & buffer = *it; + buffer->write(source_buffer->buffer().begin(), source_buffer->offset()); + buffer->next(); } + source_buffer->next(); } catch (Exception & exception) { From ec15be27989296a29bd2315422a79b049f1545a2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 19:00:50 +0200 Subject: [PATCH 292/659] Add missed file --- tests/integration/test_concurrent_backups_s3/__init__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_concurrent_backups_s3/__init__.py diff --git a/tests/integration/test_concurrent_backups_s3/__init__.py b/tests/integration/test_concurrent_backups_s3/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From b770f9903d120c691de5840360e8fb5e1b41f65b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 15 Jul 2022 20:03:00 +0300 Subject: [PATCH 293/659] Update run.sh --- docker/test/stress/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index cc76456fac2..06bf05a1727 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -361,6 +361,7 @@ else # FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 ("Cannot parse string 'Hello' as UInt64") # FIXME Not sure if it's expected, but some tests from BC check may not be finished yet when we restarting server. # Let's just ignore all errors from queries ("} TCPHandler: Code:", "} executeQuery: Code:") + # FIXME https://github.com/ClickHouse/ClickHouse/issues/39197 ("Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'") echo "Check for Error messages in server log:" zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \ -e "Code: 236. DB::Exception: Cancelled mutating parts" \ @@ -387,6 +388,7 @@ else -e "Cannot parse string 'Hello' as UInt64" \ -e "} TCPHandler: Code:" \ -e "} executeQuery: Code:" \ + -e "Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'" \ /var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv From d3d0605c00107fe7170db20bc3de7d6c09ae8f0d Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 15 Jul 2022 14:33:51 -0400 Subject: [PATCH 294/659] add words of caution into documentation --- docs/en/sql-reference/statements/create/table.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 2cf57cc2243..0c2e87fbcac 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -248,6 +248,13 @@ ClickHouse supports general purpose codecs and specialized codecs. High compression levels are useful for asymmetric scenarios, like compress once, decompress repeatedly. Higher levels mean better compression and higher CPU usage. +#### DEFLATE_QPL + +`DEFLATE_QPL` — [Deflate compression algorithm](https://github.com/intel/qpl) implemented by Intel® Query Processing Library, which has dependency on Intel Hardware: + +- DEFLATE_QPL is only supported on systems with AVX2/AVX512/IAA. +- DEFLATE_QPL-compressed data can only be transferred between nodes with AVX2/AVX512/IAA. + ### Specialized Codecs These codecs are designed to make compression more effective by using specific features of data. Some of these codecs do not compress data themself. Instead, they prepare the data for a common purpose codec, which compresses it better than without this preparation. From 5104e4200a2fe9e920def024f00048bf5e40f4a8 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 15 Jul 2022 15:32:10 -0400 Subject: [PATCH 295/659] fixed cosmetic issues --- .../CompressionCodecDeflateQpl.cpp | 60 ++++++++++--------- src/Compression/CompressionCodecDeflateQpl.h | 30 +++++----- src/Compression/CompressionInfo.h | 2 +- 3 files changed, 49 insertions(+), 43 deletions(-) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index a1ad0764bc4..137ec711594 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -28,12 +28,12 @@ DeflateQplJobHWPool & DeflateQplJobHWPool::instance() } DeflateQplJobHWPool::DeflateQplJobHWPool() - :random_engine(std::random_device()()) + :job_pool_ready(false) + ,random_engine(std::random_device()()) ,distribution(0, MAX_HW_JOB_NUMBER-1) { Poco::Logger * log = &Poco::Logger::get("DeflateQplJobHWPool"); - uint32_t job_size = 0; - uint32_t index = 0; + UInt32 job_size = 0; const char * qpl_version = qpl_get_library_version(); /// Get size required for saving a single qpl job object @@ -42,7 +42,7 @@ DeflateQplJobHWPool::DeflateQplJobHWPool() hw_jobs_buffer = std::make_unique(job_size * MAX_HW_JOB_NUMBER); /// Initialize pool for storing all job object pointers /// Reallocate buffer by shifting address offset for each job object. - for (index = 0; index < MAX_HW_JOB_NUMBER; ++index) + for (UInt32 index = 0; index < MAX_HW_JOB_NUMBER; ++index) { qpl_job * qpl_job_ptr = reinterpret_cast(hw_jobs_buffer.get() + index * job_size); if (qpl_init_job(qpl_path_hardware, qpl_job_ptr) != QPL_STS_OK) @@ -61,7 +61,7 @@ DeflateQplJobHWPool::DeflateQplJobHWPool() DeflateQplJobHWPool::~DeflateQplJobHWPool() { - for (uint32_t i = 0; i < MAX_HW_JOB_NUMBER; ++i) + for (UInt32 i = 0; i < MAX_HW_JOB_NUMBER; ++i) { if (hw_job_ptr_pool[i]) { @@ -74,11 +74,11 @@ DeflateQplJobHWPool::~DeflateQplJobHWPool() job_pool_ready = false; } -qpl_job * DeflateQplJobHWPool::acquireJob(uint32_t * job_id) +qpl_job * DeflateQplJobHWPool::acquireJob(UInt32 * job_id) { if (isJobPoolReady()) { - uint32_t retry = 0; + UInt32 retry = 0; auto index = distribution(random_engine); while (!tryLockJob(index)) { @@ -97,11 +97,11 @@ qpl_job * DeflateQplJobHWPool::acquireJob(uint32_t * job_id) return nullptr; } -qpl_job * DeflateQplJobHWPool::releaseJob(uint32_t job_id) +qpl_job * DeflateQplJobHWPool::releaseJob(UInt32 job_id) { if (isJobPoolReady()) { - uint32_t index = MAX_HW_JOB_NUMBER - job_id; + UInt32 index = MAX_HW_JOB_NUMBER - job_id; assert(index < MAX_HW_JOB_NUMBER); ReleaseJobObjectGuard _(index); return hw_job_ptr_pool[index]; @@ -110,13 +110,19 @@ qpl_job * DeflateQplJobHWPool::releaseJob(uint32_t job_id) return nullptr; } -bool DeflateQplJobHWPool::tryLockJob(size_t index) +bool DeflateQplJobHWPool::tryLockJob(UInt32 index) { bool expected = false; assert(index < MAX_HW_JOB_NUMBER); return hw_job_ptr_locks[index].compare_exchange_strong(expected, true); } + void DeflateQplJobHWPool::unLockJob(UInt32 index) + { + assert(index < MAX_HW_JOB_NUMBER); + hw_job_ptr_locks[index].store(false); + } + //HardwareCodecDeflateQpl HardwareCodecDeflateQpl::HardwareCodecDeflateQpl() :log(&Poco::Logger::get("HardwareCodecDeflateQpl")) @@ -140,11 +146,11 @@ HardwareCodecDeflateQpl::~HardwareCodecDeflateQpl() #endif } -int32_t HardwareCodecDeflateQpl::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const +Int32 HardwareCodecDeflateQpl::doCompressData(const char * source, UInt32 source_size, char * dest, UInt32 dest_size) const { - uint32_t job_id = 0; + UInt32 job_id = 0; qpl_job* job_ptr = nullptr; - uint32_t compressed_size = 0; + UInt32 compressed_size = 0; if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(&job_id))) { LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doCompressData->acquireJob fail, probably job pool exhausted)"); @@ -167,9 +173,9 @@ int32_t HardwareCodecDeflateQpl::doCompressData(const char * source, uint32_t so return compressed_size; } -int32_t HardwareCodecDeflateQpl::doDecompressDataAsynchronous(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) +Int32 HardwareCodecDeflateQpl::doDecompressDataAsynchronous(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) { - uint32_t job_id = 0; + UInt32 job_id = 0; qpl_job * job_ptr = nullptr; if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(&job_id))) { @@ -200,11 +206,11 @@ int32_t HardwareCodecDeflateQpl::doDecompressDataAsynchronous(const char * sourc void HardwareCodecDeflateQpl::flushAsynchronousDecompressRequests() { - uint32_t job_id = 0; + UInt32 job_id = 0; qpl_job * job_ptr = nullptr; - std::map::iterator it; - uint32_t n_jobs_processing = decomp_async_job_map.size(); + std::map::iterator it; + UInt32 n_jobs_processing = decomp_async_job_map.size(); it = decomp_async_job_map.begin(); while (n_jobs_processing) @@ -242,7 +248,7 @@ qpl_job * SoftwareCodecDeflateQpl::getJobCodecPtr() { if (!sw_job) { - uint32_t size = 0; + UInt32 size = 0; qpl_get_job_size(qpl_path_software, &size); sw_buffer = std::make_unique(size); @@ -256,7 +262,7 @@ qpl_job * SoftwareCodecDeflateQpl::getJobCodecPtr() return sw_job; } -uint32_t SoftwareCodecDeflateQpl::doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) +UInt32 SoftwareCodecDeflateQpl::doCompressData(const char * source, UInt32 source_size, char * dest, UInt32 dest_size) { qpl_job * job_ptr = getJobCodecPtr(); // Performing a compression operation @@ -275,7 +281,7 @@ uint32_t SoftwareCodecDeflateQpl::doCompressData(const char * source, uint32_t s return job_ptr->total_out; } -void SoftwareCodecDeflateQpl::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) +void SoftwareCodecDeflateQpl::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) { qpl_job * job_ptr = getJobCodecPtr(); @@ -310,15 +316,15 @@ void CompressionCodecDeflateQpl::updateHash(SipHash & hash) const getCodecDesc()->updateTreeHash(hash); } -uint32_t CompressionCodecDeflateQpl::getMaxCompressedDataSize(uint32_t uncompressed_size) const +UInt32 CompressionCodecDeflateQpl::getMaxCompressedDataSize(UInt32 uncompressed_size) const { /// Aligned with ZLIB return ((uncompressed_size) + ((uncompressed_size) >> 12) + ((uncompressed_size) >> 14) + ((uncompressed_size) >> 25) + 13); } -uint32_t CompressionCodecDeflateQpl::doCompressData(const char * source, uint32_t source_size, char * dest) const +UInt32 CompressionCodecDeflateQpl::doCompressData(const char * source, UInt32 source_size, char * dest) const { - int32_t res = HardwareCodecDeflateQpl::RET_ERROR; + Int32 res = HardwareCodecDeflateQpl::RET_ERROR; if (DeflateQplJobHWPool::instance().isJobPoolReady()) res = hw_codec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); if (res == HardwareCodecDeflateQpl::RET_ERROR) @@ -326,13 +332,13 @@ uint32_t CompressionCodecDeflateQpl::doCompressData(const char * source, uint32_ return res; } -void CompressionCodecDeflateQpl::doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const +void CompressionCodecDeflateQpl::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { switch (getDecompressMode()) { case CodecMode::Synchronous: { - int32_t res = HardwareCodecDeflateQpl::RET_ERROR; + Int32 res = HardwareCodecDeflateQpl::RET_ERROR; if (DeflateQplJobHWPool::instance().isJobPoolReady()) { res = hw_codec->doDecompressDataAsynchronous(source, source_size, dest, uncompressed_size); @@ -347,7 +353,7 @@ void CompressionCodecDeflateQpl::doDecompressData(const char * source, uint32_t } case CodecMode::Asynchronous: { - int32_t res = HardwareCodecDeflateQpl::RET_ERROR; + Int32 res = HardwareCodecDeflateQpl::RET_ERROR; if (DeflateQplJobHWPool::instance().isJobPoolReady()) res = hw_codec->doDecompressDataAsynchronous(source, source_size, dest, uncompressed_size); if (res == HardwareCodecDeflateQpl::RET_ERROR) diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index b64025c5939..0b04d7185c6 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -19,9 +19,9 @@ public: DeflateQplJobHWPool(); ~DeflateQplJobHWPool(); - qpl_job * acquireJob(uint32_t * job_id); + qpl_job * acquireJob(UInt32 * job_id); - static qpl_job * releaseJob(uint32_t job_id); + static qpl_job * releaseJob(UInt32 job_id); static const bool & isJobPoolReady() { return job_pool_ready; } @@ -30,17 +30,17 @@ public: static constexpr auto MAX_HW_JOB_NUMBER = 1024; private: - static bool tryLockJob(size_t index); + static bool tryLockJob(UInt32 index); - static void unLockJob(uint32_t index) { hw_job_ptr_locks[index].store(false); } + static void unLockJob(UInt32 index); class ReleaseJobObjectGuard { - uint32_t index; + UInt32 index; ReleaseJobObjectGuard() = delete; public: - ReleaseJobObjectGuard(const uint32_t index_) : index(index_){} + ReleaseJobObjectGuard(const UInt32 index_) : index(index_){} ~ReleaseJobObjectGuard(){ hw_job_ptr_locks[index].store(false); } }; @@ -60,8 +60,8 @@ class SoftwareCodecDeflateQpl { public: ~SoftwareCodecDeflateQpl(); - uint32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size); - void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest, UInt32 dest_size); + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size); private: qpl_job * sw_job = nullptr; @@ -73,12 +73,12 @@ class HardwareCodecDeflateQpl { public: /// RET_ERROR stands for hardware codec fail,need fallback to software codec. - static constexpr int32_t RET_ERROR = -1; + static constexpr Int32 RET_ERROR = -1; HardwareCodecDeflateQpl(); ~HardwareCodecDeflateQpl(); - int32_t doCompressData(const char * source, uint32_t source_size, char * dest, uint32_t dest_size) const; - int32_t doDecompressDataAsynchronous(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size); + Int32 doCompressData(const char * source, UInt32 source_size, char * dest, UInt32 dest_size) const; + Int32 doDecompressDataAsynchronous(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size); /// Flush result for previous asynchronous decompression requests.Must be used following with several calls of doDecompressDataReq. void flushAsynchronousDecompressRequests(); @@ -86,7 +86,7 @@ private: /// Asynchronous job map for decompression: job ID - job object. /// For each submission, push job ID && job object into this map; /// For flush, pop out job ID && job object from this map. Use job ID to release job lock and use job object to check job status till complete. - std::map decomp_async_job_map; + std::map decomp_async_job_map; Poco::Logger * log; }; @@ -108,13 +108,13 @@ protected: return true; } - uint32_t doCompressData(const char * source, uint32_t source_size, char * dest) const override; - void doDecompressData(const char * source, uint32_t source_size, char * dest, uint32_t uncompressed_size) const override; + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; ///Flush result for previous asynchronous decompression requests on asynchronous mode. void flushAsynchronousDecompressRequests() override; private: - uint32_t getMaxCompressedDataSize(uint32_t uncompressed_size) const override; + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; std::unique_ptr hw_codec; std::unique_ptr sw_codec; }; diff --git a/src/Compression/CompressionInfo.h b/src/Compression/CompressionInfo.h index 51136ff825d..985d74bbb74 100644 --- a/src/Compression/CompressionInfo.h +++ b/src/Compression/CompressionInfo.h @@ -46,7 +46,7 @@ enum class CompressionMethodByte : uint8_t AES_128_GCM_SIV = 0x96, AES_256_GCM_SIV = 0x97, FPC = 0x98, - DeflateQpl = 0x99, + DeflateQpl = 0x99, }; } From e3735f7464ea14d6467bcc139b6a3ad682de9c16 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 15 Jul 2022 16:12:26 -0400 Subject: [PATCH 296/659] fixed cosmetic issues --- .../CompressionCodecDeflateQpl.cpp | 34 +++++++------------ src/Compression/CompressionCodecDeflateQpl.h | 18 +++------- 2 files changed, 17 insertions(+), 35 deletions(-) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 137ec711594..a467e375746 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -18,7 +18,7 @@ namespace ErrorCodes std::array DeflateQplJobHWPool::hw_job_ptr_pool; std::array DeflateQplJobHWPool::hw_job_ptr_locks; -bool DeflateQplJobHWPool::job_pool_ready; +bool DeflateQplJobHWPool::job_pool_ready = false; std::unique_ptr DeflateQplJobHWPool::hw_jobs_buffer; DeflateQplJobHWPool & DeflateQplJobHWPool::instance() @@ -28,8 +28,7 @@ DeflateQplJobHWPool & DeflateQplJobHWPool::instance() } DeflateQplJobHWPool::DeflateQplJobHWPool() - :job_pool_ready(false) - ,random_engine(std::random_device()()) + :random_engine(std::random_device()()) ,distribution(0, MAX_HW_JOB_NUMBER-1) { Poco::Logger * log = &Poco::Logger::get("DeflateQplJobHWPool"); @@ -48,7 +47,7 @@ DeflateQplJobHWPool::DeflateQplJobHWPool() if (qpl_init_job(qpl_path_hardware, qpl_job_ptr) != QPL_STS_OK) { job_pool_ready = false; - LOG_WARNING(log, "Initialization of hardware-assisted DeflateQpl codec failed, falling back to software DeflateQpl codec. Please check if Intel In-Memory Analytics Accelerator (IAA) is properly set up. QPL Version:{}.",qpl_version); + LOG_WARNING(log, "Initialization of hardware-assisted DeflateQpl codec failed, falling back to software DeflateQpl codec. Please check if Intel In-Memory Analytics Accelerator (IAA) is properly set up. QPL Version: {}.",qpl_version); return; } hw_job_ptr_pool[index] = qpl_job_ptr; @@ -56,7 +55,7 @@ DeflateQplJobHWPool::DeflateQplJobHWPool() } job_pool_ready = true; - LOG_DEBUG(log, "Hardware-assisted DeflateQpl codec is ready! QPL Version:{}",qpl_version); + LOG_DEBUG(log, "Hardware-assisted DeflateQpl codec is ready! QPL Version: {}",qpl_version); } DeflateQplJobHWPool::~DeflateQplJobHWPool() @@ -97,17 +96,10 @@ qpl_job * DeflateQplJobHWPool::acquireJob(UInt32 * job_id) return nullptr; } -qpl_job * DeflateQplJobHWPool::releaseJob(UInt32 job_id) +void DeflateQplJobHWPool::releaseJob(UInt32 job_id) { if (isJobPoolReady()) - { - UInt32 index = MAX_HW_JOB_NUMBER - job_id; - assert(index < MAX_HW_JOB_NUMBER); - ReleaseJobObjectGuard _(index); - return hw_job_ptr_pool[index]; - } - else - return nullptr; + unLockJob(MAX_HW_JOB_NUMBER - job_id); } bool DeflateQplJobHWPool::tryLockJob(UInt32 index) @@ -168,7 +160,7 @@ Int32 HardwareCodecDeflateQpl::doCompressData(const char * source, UInt32 source if (auto status = qpl_execute_job(job_ptr); status == QPL_STS_OK) compressed_size = job_ptr->total_out; else - LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doCompressData->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doCompressData->qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); DeflateQplJobHWPool::instance().releaseJob(job_id); return compressed_size; } @@ -199,7 +191,7 @@ Int32 HardwareCodecDeflateQpl::doDecompressDataAsynchronous(const char * source, else { DeflateQplJobHWPool::instance().releaseJob(job_id); - LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataAsynchronous->qpl_execute_job with error code:{} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataAsynchronous->qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); return RET_ERROR; } } @@ -224,8 +216,8 @@ void HardwareCodecDeflateQpl::flushAsynchronousDecompressRequests() } else { - DeflateQplJobHWPool::instance().releaseJob(job_id); it = decomp_async_job_map.erase(it); + DeflateQplJobHWPool::instance().releaseJob(job_id); n_jobs_processing--; if (n_jobs_processing <= 0) break; @@ -240,7 +232,7 @@ void HardwareCodecDeflateQpl::flushAsynchronousDecompressRequests() SoftwareCodecDeflateQpl::~SoftwareCodecDeflateQpl() { - if (nullptr != sw_job) + if (!sw_job) qpl_fini_job(sw_job); } @@ -257,7 +249,7 @@ qpl_job * SoftwareCodecDeflateQpl::getJobCodecPtr() // Job initialization if (auto status = qpl_init_job(qpl_path_software, sw_job); status != QPL_STS_OK) throw Exception(ErrorCodes::CANNOT_COMPRESS, - "Initialization of DeflateQpl software fallback codec failed. (Details: qpl_init_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + "Initialization of DeflateQpl software fallback codec failed. (Details: qpl_init_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); } return sw_job; } @@ -276,7 +268,7 @@ UInt32 SoftwareCodecDeflateQpl::doCompressData(const char * source, UInt32 sourc if (auto status = qpl_execute_job(job_ptr); status != QPL_STS_OK) throw Exception(ErrorCodes::CANNOT_COMPRESS, - "Execution of DeflateQpl software fallback codec failed. (Details: qpl_execute_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + "Execution of DeflateQpl software fallback codec failed. (Details: qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); return job_ptr->total_out; } @@ -295,7 +287,7 @@ void SoftwareCodecDeflateQpl::doDecompressData(const char * source, UInt32 sourc if (auto status = qpl_execute_job(job_ptr); status != QPL_STS_OK) throw Exception(ErrorCodes::CANNOT_DECOMPRESS, - "Execution of DeflateQpl software fallback codec failed. (Details: qpl_execute_job with error code {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + "Execution of DeflateQpl software fallback codec failed. (Details: qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); } //CompressionCodecDeflateQpl diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 0b04d7185c6..b180d683348 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -17,34 +17,24 @@ class DeflateQplJobHWPool { public: DeflateQplJobHWPool(); + ~DeflateQplJobHWPool(); qpl_job * acquireJob(UInt32 * job_id); - static qpl_job * releaseJob(UInt32 job_id); + static void releaseJob(UInt32 job_id); static const bool & isJobPoolReady() { return job_pool_ready; } static DeflateQplJobHWPool & instance(); - static constexpr auto MAX_HW_JOB_NUMBER = 1024; - private: static bool tryLockJob(UInt32 index); static void unLockJob(UInt32 index); - class ReleaseJobObjectGuard - { - UInt32 index; - ReleaseJobObjectGuard() = delete; - - public: - ReleaseJobObjectGuard(const UInt32 index_) : index(index_){} - - ~ReleaseJobObjectGuard(){ hw_job_ptr_locks[index].store(false); } - }; - + /// Maximum jobs running in parallel supported by IAA hardware + static constexpr auto MAX_HW_JOB_NUMBER = 1024; /// Entire buffer for storing all job objects static std::unique_ptr hw_jobs_buffer; /// Job pool for storing all job object pointers From f05f3cb69abfb8ce68747c60cf6b9c71bb391662 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 15 Jul 2022 17:04:27 -0400 Subject: [PATCH 297/659] fixed cosmetic issues --- .../CompressionCodecDeflateQpl.cpp | 28 +++++++++++-------- src/Compression/CompressionCodecDeflateQpl.h | 2 +- 2 files changed, 17 insertions(+), 13 deletions(-) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index a467e375746..a0fd6b7757e 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -73,7 +73,7 @@ DeflateQplJobHWPool::~DeflateQplJobHWPool() job_pool_ready = false; } -qpl_job * DeflateQplJobHWPool::acquireJob(UInt32 * job_id) +qpl_job * DeflateQplJobHWPool::acquireJob(UInt32 &job_id) { if (isJobPoolReady()) { @@ -88,7 +88,7 @@ qpl_job * DeflateQplJobHWPool::acquireJob(UInt32 * job_id) return nullptr; } } - *job_id = MAX_HW_JOB_NUMBER - index; + job_id = MAX_HW_JOB_NUMBER - index; assert(index < MAX_HW_JOB_NUMBER); return hw_job_ptr_pool[index]; } @@ -143,7 +143,7 @@ Int32 HardwareCodecDeflateQpl::doCompressData(const char * source, UInt32 source UInt32 job_id = 0; qpl_job* job_ptr = nullptr; UInt32 compressed_size = 0; - if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(&job_id))) + if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(job_id))) { LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doCompressData->acquireJob fail, probably job pool exhausted)"); return RET_ERROR; @@ -158,18 +158,24 @@ Int32 HardwareCodecDeflateQpl::doCompressData(const char * source, UInt32 source job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_DYNAMIC_HUFFMAN | QPL_FLAG_LAST | QPL_FLAG_OMIT_VERIFY; if (auto status = qpl_execute_job(job_ptr); status == QPL_STS_OK) + { compressed_size = job_ptr->total_out; + DeflateQplJobHWPool::instance().releaseJob(job_id); + return compressed_size; + } else + { LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doCompressData->qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); - DeflateQplJobHWPool::instance().releaseJob(job_id); - return compressed_size; + DeflateQplJobHWPool::instance().releaseJob(job_id); + return RET_ERROR; + } } Int32 HardwareCodecDeflateQpl::doDecompressDataAsynchronous(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) { UInt32 job_id = 0; qpl_job * job_ptr = nullptr; - if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(&job_id))) + if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(job_id))) { LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataAsynchronous->acquireJob fail, probably job pool exhausted)"); return RET_ERROR; @@ -198,19 +204,17 @@ Int32 HardwareCodecDeflateQpl::doDecompressDataAsynchronous(const char * source, void HardwareCodecDeflateQpl::flushAsynchronousDecompressRequests() { - UInt32 job_id = 0; - qpl_job * job_ptr = nullptr; - - std::map::iterator it; UInt32 n_jobs_processing = decomp_async_job_map.size(); - it = decomp_async_job_map.begin(); + std::map::iterator it = decomp_async_job_map.begin(); while (n_jobs_processing) { + UInt32 job_id = 0; + qpl_job * job_ptr = nullptr; job_id = it->first; job_ptr = it->second; - if (QPL_STS_BEING_PROCESSED == qpl_check_job(job_ptr)) + if (qpl_check_job(job_ptr) == QPL_STS_BEING_PROCESSED) { it++; } diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index b180d683348..f64cf4c0c6c 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -20,7 +20,7 @@ public: ~DeflateQplJobHWPool(); - qpl_job * acquireJob(UInt32 * job_id); + qpl_job * acquireJob(UInt32 &job_id); static void releaseJob(UInt32 job_id); From dbac3a35b26d369104037a2414600de5042b3909 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 15 Jul 2022 17:23:37 -0400 Subject: [PATCH 298/659] fixed cosmetic issues --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- programs/compressor/Compressor.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index e6601d52d8d..0c0c7da330d 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -44,7 +44,7 @@ ClickHouse перезагружает встроенные словари с з - `min_part_size` - Минимальный размер части таблицы. - `min_part_size_ratio` - Отношение размера минимальной части таблицы к полному размеру таблицы. -- `method` - Метод сжатия. Возможные значения: `lz4`, `lz4hc`, `zstd`,`deflate`. +- `method` - Метод сжатия. Возможные значения: `lz4`, `lz4hc`, `zstd`,`deflate_qpl`. - `level` – Уровень сжатия. См. [Кодеки](../../sql-reference/statements/create/table/#create-query-common-purpose-codecs). Можно сконфигурировать несколько разделов ``. diff --git a/programs/compressor/Compressor.cpp b/programs/compressor/Compressor.cpp index 1a9d72e8972..fe8debcee27 100644 --- a/programs/compressor/Compressor.cpp +++ b/programs/compressor/Compressor.cpp @@ -112,7 +112,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) if (options.count("codec")) codecs = options["codec"].as>(); - if ((use_deflate_qpl || use_lz4hc || use_zstd || use_none) && !codecs.empty()) + if ((use_lz4hc || use_zstd || use_deflate_qpl || use_none) && !codecs.empty()) throw Exception("Wrong options, codec flags like --zstd and --codec options are mutually exclusive", ErrorCodes::BAD_ARGUMENTS); if (!codecs.empty() && options.count("level")) From 8a5286ea4c466f2ceeaae2cf7435ba074820f1c5 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Sat, 16 Jul 2022 00:27:26 +0200 Subject: [PATCH 299/659] fix cpu usage metric in client --- src/Client/ClientBase.cpp | 3 +- src/Common/EventRateMeter.h | 70 +++++++++++++++++++++++++++++++ src/Common/ProgressIndication.cpp | 23 +++++----- src/Common/ProgressIndication.h | 10 ++--- 4 files changed, 87 insertions(+), 19 deletions(-) create mode 100644 src/Common/EventRateMeter.h diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index b435b483b71..f0a8794d096 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -983,8 +983,7 @@ void ClientBase::onProfileEvents(Block & block) else if (event_name == MemoryTracker::USAGE_EVENT_NAME) thread_times[host_name][thread_id].memory_usage = value; } - auto elapsed_time = profile_events.watch.elapsedMicroseconds(); - progress_indication.updateThreadEventData(thread_times, elapsed_time); + progress_indication.updateThreadEventData(thread_times); if (need_render_progress) progress_indication.writeProgress(); diff --git a/src/Common/EventRateMeter.h b/src/Common/EventRateMeter.h new file mode 100644 index 00000000000..27cb642a55f --- /dev/null +++ b/src/Common/EventRateMeter.h @@ -0,0 +1,70 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/// Event count measurement with exponential smoothing intended for computing time derivatives +class EventRateMeter { +public: + explicit EventRateMeter(UInt64 period_, UInt64 resolution = 1000) + : period(std::max(period_, 1ul)) + , step(std::max(period / resolution, 1ul)) + , decay(1.0 - 1.0 / resolution) + {} + + /// Add `count` events happened at `now` instant. + /// Previous events that are older than `period` from `now` will be forgotten + /// in a way to keep average event rate the same, using exponential smoothing. + /// NOTE: Adding events into distant past (further than `period`) must be avoided. + void add(UInt64 now, UInt64 count) + { + if (unlikely(end == 0)) + { + // Initialization during the first call + end = now + period; + } + else if (now > end) + { + // Compute number of steps we have to move for `now <= end` to became true + UInt64 steps = (now - end + step - 1) / step; + end += steps * step; + assert(now <= end); + + // Forget old events, assuming all events are distributed evenly throughout whole `period`. + // This assumption leads to exponential decay in case no new events will come. + if (steps == 1) + events *= decay; + else + events *= std::pow(decay, steps); + } + + // Add new events + events += count; + } + + /// Compute average event rate thoughout `[now - period, now]` period. + double rate(UInt64 now) + { + add(now, 0); + return double(events) / period; + } + + void reset() + { + events = 0; + end = 0; + } + +private: + const UInt64 period; + const UInt64 step; + const double decay; + double events = 0; // Estimated number of events in [now - period, now] range + UInt64 end = 0; +}; + +} diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 7bea00f5b1e..e0e63ae864d 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -8,6 +8,7 @@ #include "Common/formatReadable.h" #include #include +#include #include "IO/WriteBufferFromString.h" #include @@ -16,16 +17,16 @@ namespace { constexpr UInt64 ALL_THREADS = 0; - double calculateCPUUsage(DB::ThreadIdToTimeMap times, UInt64 elapsed) + UInt64 aggregateCPUUsageNs(DB::ThreadIdToTimeMap times) { - auto accumulated = std::accumulate(times.begin(), times.end(), 0, + constexpr UInt64 us_to_ns = 1000; + return us_to_ns * std::accumulate(times.begin(), times.end(), 0ull, [](UInt64 acc, const auto & elem) { if (elem.first == ALL_THREADS) return acc; return acc + elem.second.time(); }); - return static_cast(accumulated) / elapsed; } } @@ -55,7 +56,7 @@ void ProgressIndication::resetProgress() write_progress_on_update = false; { std::lock_guard lock(profile_events_mutex); - host_cpu_usage.clear(); + cpu_usage_meter.reset(); thread_data.clear(); } } @@ -82,15 +83,17 @@ void ProgressIndication::addThreadIdToList(String const & host, UInt64 thread_id thread_to_times[thread_id] = {}; } -void ProgressIndication::updateThreadEventData(HostToThreadTimesMap & new_thread_data, UInt64 elapsed_time) +void ProgressIndication::updateThreadEventData(HostToThreadTimesMap & new_thread_data) { std::lock_guard lock(profile_events_mutex); + UInt64 total_cpu_ns = 0; for (auto & new_host_map : new_thread_data) { - host_cpu_usage[new_host_map.first] = calculateCPUUsage(new_host_map.second, elapsed_time); + total_cpu_ns += aggregateCPUUsageNs(new_host_map.second); thread_data[new_host_map.first] = std::move(new_host_map.second); } + cpu_usage_meter.add(clock_gettime_ns(), total_cpu_ns); } size_t ProgressIndication::getUsedThreadsCount() const @@ -104,14 +107,10 @@ size_t ProgressIndication::getUsedThreadsCount() const }); } -double ProgressIndication::getCPUUsage() const +double ProgressIndication::getCPUUsage() { std::lock_guard lock(profile_events_mutex); - - double res = 0; - for (const auto & elem : host_cpu_usage) - res += elem.second; - return res; + return cpu_usage_meter.rate(clock_gettime_ns()); } ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 9ce29ef0d3c..47f67ffb53c 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -7,7 +7,7 @@ #include #include #include - +#include /// http://en.wikipedia.org/wiki/ANSI_escape_code #define CLEAR_TO_END_OF_LINE "\033[K" @@ -59,12 +59,12 @@ public: void addThreadIdToList(String const & host, UInt64 thread_id); - void updateThreadEventData(HostToThreadTimesMap & new_thread_data, UInt64 elapsed_time); + void updateThreadEventData(HostToThreadTimesMap & new_thread_data); private: size_t getUsedThreadsCount() const; - double getCPUUsage() const; + double getCPUUsage(); struct MemoryUsage { @@ -91,7 +91,7 @@ private: bool write_progress_on_update = false; - std::unordered_map host_cpu_usage; + EventRateMeter cpu_usage_meter{1'000'000'000 /*ns*/}; // average cpu utilization per 1 second HostToThreadTimesMap thread_data; /// In case of all of the above: /// - clickhouse-local @@ -100,7 +100,7 @@ private: /// /// It is possible concurrent access to the following: /// - writeProgress() (class properties) (guarded with progress_mutex) - /// - thread_data/host_cpu_usage (guarded with profile_events_mutex) + /// - thread_data/cpu_usage_meter (guarded with profile_events_mutex) mutable std::mutex profile_events_mutex; mutable std::mutex progress_mutex; }; From 82f98cfe6bd9c89275c63901c0954719b51fb216 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Sat, 16 Jul 2022 02:17:18 +0200 Subject: [PATCH 300/659] remove initial linear growth, increase avg period to 3 sec --- src/Common/EventRateMeter.h | 22 ++++++++++++++++------ src/Common/ProgressIndication.cpp | 2 +- src/Common/ProgressIndication.h | 2 +- 3 files changed, 18 insertions(+), 8 deletions(-) diff --git a/src/Common/EventRateMeter.h b/src/Common/EventRateMeter.h index 27cb642a55f..b8582585e4b 100644 --- a/src/Common/EventRateMeter.h +++ b/src/Common/EventRateMeter.h @@ -1,8 +1,10 @@ #pragma once +#include #include #include #include +#include namespace DB { @@ -20,12 +22,14 @@ public: /// Previous events that are older than `period` from `now` will be forgotten /// in a way to keep average event rate the same, using exponential smoothing. /// NOTE: Adding events into distant past (further than `period`) must be avoided. - void add(UInt64 now, UInt64 count) + void add(UInt64 now, UInt64 count = 1) { if (unlikely(end == 0)) { // Initialization during the first call - end = now + period; + if (start == 0) + start = now; + end = start + period; } else if (now > end) { @@ -47,15 +51,20 @@ public: } /// Compute average event rate thoughout `[now - period, now]` period. + /// If measurements are just started (`now - period < start`), then average + /// is computed based on shorter `[start; now]` period to avoid initial linear growth. double rate(UInt64 now) { add(now, 0); - return double(events) / period; + if (unlikely(now <= start)) + return 0; + return double(events) / std::min(period, now - start); } - void reset() + void reset(UInt64 now) { events = 0; + start = now; end = 0; } @@ -63,8 +72,9 @@ private: const UInt64 period; const UInt64 step; const double decay; - double events = 0; // Estimated number of events in [now - period, now] range - UInt64 end = 0; + double events = 0; // Estimated number of events in [end - period, end] range + UInt64 start = 0; // Instant in past without events before it; when measurement started or reset + UInt64 end = 0; // Instant in future to start decay; moving in steps }; } diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index e0e63ae864d..2b4b5d9ade9 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -56,7 +56,7 @@ void ProgressIndication::resetProgress() write_progress_on_update = false; { std::lock_guard lock(profile_events_mutex); - cpu_usage_meter.reset(); + cpu_usage_meter.reset(clock_gettime_ns()); thread_data.clear(); } } diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 47f67ffb53c..6ff89294475 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -91,7 +91,7 @@ private: bool write_progress_on_update = false; - EventRateMeter cpu_usage_meter{1'000'000'000 /*ns*/}; // average cpu utilization per 1 second + EventRateMeter cpu_usage_meter{3'000'000'000 /*ns*/}; // average cpu utilization last 3 second HostToThreadTimesMap thread_data; /// In case of all of the above: /// - clickhouse-local From 1ef9910453a45dd12891a9a8f4838ec95506eff9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 16 Jul 2022 10:50:22 +0300 Subject: [PATCH 301/659] Fix builder image for releases w/o diagnostics tool Signed-off-by: Azat Khuzhin --- docker/packager/binary/build.sh | 26 +++++++++++++++----------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 270c93c105c..87f98df2ad8 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -29,17 +29,21 @@ env if [ -n "$MAKE_DEB" ]; then rm -rf /build/packages/root - if [ -z "$SANITIZER" ]; then - # We need to check if clickhouse-diagnostics is fine and build it - ( - cd /build/programs/diagnostics - make test-no-docker - GOARCH="${DEB_ARCH}" CGO_ENABLED=0 make VERSION="$VERSION_STRING" build - mv clickhouse-diagnostics .. - ) - else - echo -e "#!/bin/sh\necho 'Not implemented for this type of package'" > /build/programs/clickhouse-diagnostics - chmod +x /build/programs/clickhouse-diagnostics + # NOTE: this is for backward compatibility with previous releases, + # that does not diagnostics tool (only script). + if [ -d /build/programs/diagnostics ]; then + if [ -z "$SANITIZER" ]; then + # We need to check if clickhouse-diagnostics is fine and build it + ( + cd /build/programs/diagnostics + make test-no-docker + GOARCH="${DEB_ARCH}" CGO_ENABLED=0 make VERSION="$VERSION_STRING" build + mv clickhouse-diagnostics .. + ) + else + echo -e "#!/bin/sh\necho 'Not implemented for this type of package'" > /build/programs/clickhouse-diagnostics + chmod +x /build/programs/clickhouse-diagnostics + fi fi fi From 9bba55110c87ad359c3bf6788e720d2a95d02c82 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Jul 2022 12:44:47 +0200 Subject: [PATCH 302/659] Fix --- .../fetchPostgreSQLTableStructure.cpp | 14 ++++++----- .../test_postgresql_database_engine/test.py | 23 +++++++++++++++++++ 2 files changed, 31 insertions(+), 6 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 10cde43e9e1..fffd4d4ef7c 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -39,9 +39,10 @@ std::set fetchPostgreSQLTablesList(T & tx, const String & postgres_schem std::set tables; if (schemas.size() <= 1) { - std::string query = fmt::format("SELECT tablename FROM pg_catalog.pg_tables " - "WHERE schemaname != 'pg_catalog' AND {}", - postgres_schema.empty() ? "schemaname != 'information_schema'" : "schemaname = " + quoteString(postgres_schema)); + std::string query = fmt::format( + "SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname != 'pg_catalog' AND schemaname = {}", + postgres_schema.empty() ? quoteString("public") : quoteString(postgres_schema)); + for (auto table_name : tx.template stream(query)) tables.insert(std::get<0>(table_name)); @@ -53,9 +54,10 @@ std::set fetchPostgreSQLTablesList(T & tx, const String & postgres_schem /// If we add schema to table name then table can be accessed only this way: database_name.`schema_name.table_name` for (const auto & schema : schemas) { - std::string query = fmt::format("SELECT tablename FROM pg_catalog.pg_tables " - "WHERE schemaname != 'pg_catalog' AND {}", - postgres_schema.empty() ? "schemaname != 'information_schema'" : "schemaname = " + quoteString(schema)); + std::string query = fmt::format( + "SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname != 'pg_catalog' AND schemaname = {}", + quoteString(schema)); + for (auto table_name : tx.template stream(query)) tables.insert(schema + '.' + std::get<0>(table_name)); } diff --git a/tests/integration/test_postgresql_database_engine/test.py b/tests/integration/test_postgresql_database_engine/test.py index 5619c551c71..d07f62f8a80 100644 --- a/tests/integration/test_postgresql_database_engine/test.py +++ b/tests/integration/test_postgresql_database_engine/test.py @@ -349,6 +349,29 @@ def test_postgres_database_old_syntax(started_cluster): node1.query("DROP DATABASE IF EXISTS postgres_database;") +def test_postgresql_fetch_tables(started_cluster): + conn = get_postgres_conn( + started_cluster.postgres_ip, started_cluster.postgres_port, database=True + ) + cursor = conn.cursor() + + cursor.execute("DROP SCHEMA IF EXISTS test_schema CASCADE") + cursor.execute("CREATE SCHEMA test_schema") + cursor.execute("CREATE TABLE test_schema.table1 (a integer)") + cursor.execute("CREATE TABLE test_schema.table2 (a integer)") + cursor.execute("CREATE TABLE table3 (a integer)") + + node1.query( + "CREATE DATABASE postgres_database ENGINE = PostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')" + ) + + assert node1.query("SHOW TABLES FROM postgres_database") == "table3\n" + assert not node1.contains_in_log("PostgreSQL table table1 does not exist") + + cursor.execute(f"DROP TABLE table3") + cursor.execute("DROP SCHEMA IF EXISTS test_schema CASCADE") + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From accb0e38dafd21c8f8ab392195760ee0a523ed44 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 16 Jul 2022 11:49:02 +0300 Subject: [PATCH 303/659] Remove superior atomic from MergeTreeBackgroundExecutor and annotations for TSA Signed-off-by: Azat Khuzhin --- .../MergeTree/MergeTreeBackgroundExecutor.cpp | 4 ++-- .../MergeTree/MergeTreeBackgroundExecutor.h | 22 +++++++++++-------- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 1f7ecb39a1d..08d39091cfd 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -124,7 +124,7 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) /// All operations with queues are considered no to do any allocations - auto erase_from_active = [this, item] + auto erase_from_active = [this, item]() TSA_REQUIRES(mutex) { active.erase(std::remove(active.begin(), active.end(), item), active.end()); }; @@ -249,7 +249,7 @@ void MergeTreeBackgroundExecutor::threadFunction() TaskRuntimeDataPtr item; { std::unique_lock lock(mutex); - has_tasks.wait(lock, [this](){ return !pending.empty() || shutdown; }); + has_tasks.wait(lock, [this]() TSA_REQUIRES(mutex) { return !pending.empty() || shutdown; }); if (shutdown) break; diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h index de5af5199c0..a2f97d6bbc6 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -15,7 +14,9 @@ #include #include #include +#include #include + namespace DB { namespace ErrorCodes @@ -50,7 +51,8 @@ struct TaskRuntimeData ExecutableTaskPtr task; CurrentMetrics::Metric metric; - std::atomic_bool is_currently_deleting{false}; + /// Guarded by MergeTreeBackgroundExecutor<>::mutex + bool is_currently_deleting{false}; /// Actually autoreset=false is needed only for unit test /// where multiple threads could remove tasks corresponding to the same storage /// This scenario in not possible in reality. @@ -200,19 +202,21 @@ public: private: String name; - size_t threads_count{0}; - size_t max_tasks_count{0}; + size_t threads_count TSA_GUARDED_BY(mutex) = 0; + size_t max_tasks_count TSA_GUARDED_BY(mutex) = 0; CurrentMetrics::Metric metric; void routine(TaskRuntimeDataPtr item); - void threadFunction(); + + /// libc++ does not provide TSA support for std::unique_lock -> TSA_NO_THREAD_SAFETY_ANALYSIS + void threadFunction() TSA_NO_THREAD_SAFETY_ANALYSIS; /// Initially it will be empty - Queue pending{}; - boost::circular_buffer active{0}; + Queue pending TSA_GUARDED_BY(mutex); + boost::circular_buffer active TSA_GUARDED_BY(mutex); mutable std::mutex mutex; - std::condition_variable has_tasks; - std::atomic_bool shutdown{false}; + std::condition_variable has_tasks TSA_GUARDED_BY(mutex); + bool shutdown TSA_GUARDED_BY(mutex) = false; ThreadPool pool; Poco::Logger * log = &Poco::Logger::get("MergeTreeBackgroundExecutor"); }; From 7ef3f5606673631acb343abab6789f80266334ad Mon Sep 17 00:00:00 2001 From: Wangyang Guo Date: Sat, 16 Jul 2022 22:00:13 +0800 Subject: [PATCH 304/659] lz4 decompress: coding style fix --- src/Compression/LZ4_decompress_faster.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 33726811848..56ee62c56b5 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -459,7 +459,8 @@ inline void copyOverlap32Shuffle(UInt8 * op, const UInt8 *& match, const size_t template <> void inline copy<32>(UInt8 * dst, const UInt8 * src) { copy32(dst, src); } template <> void inline wildCopy<32>(UInt8 * dst, const UInt8 * src, UInt8 * dst_end) { wildCopy32(dst, src, dst_end); } template <> void inline copyOverlap<32, false>(UInt8 * op, const UInt8 *& match, const size_t offset) { copyOverlap32(op, match, offset); } -template <> void inline copyOverlap<32, true>(UInt8 * op, const UInt8 *& match, const size_t offset) { +template <> void inline copyOverlap<32, true>(UInt8 * op, const UInt8 *& match, const size_t offset) +{ #if USE_MULTITARGET_CODE TargetSpecific::AVX512VBMI::copyOverlap32Shuffle(op, match, offset); #else From 3f43641b184d6b71088aa5bbe3c9c309f72592e5 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sat, 16 Jul 2022 16:32:33 +0200 Subject: [PATCH 305/659] Update MergedColumnOnlyOutputStream.cpp --- src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index d48a8b90646..deab5e748c7 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -39,7 +39,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( metadata_snapshot_, indices_to_recalc, default_codec, - std::move(writer_settings), + writer_settings, index_granularity); auto * writer_on_disk = dynamic_cast(writer.get()); From 4ab52b6873abdc431751eea2fa434bfe940aa72d Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 16 Jul 2022 18:58:47 +0200 Subject: [PATCH 306/659] added new DataType + fixes --- .gitmodules | 3 - base/base/Base58.h | 9 - contrib/CMakeLists.txt | 1 - contrib/base-x | 1 - contrib/base-x-cmake/CMakeLists.txt | 28 --- docs/en/development/contrib.md | 1 - .../functions/string-functions.md | 17 +- docs/ru/development/contrib.md | 1 - .../functions/string-functions.md | 17 +- src/Common/FieldVisitorToString.cpp | 1 + src/Common/base58.h | 85 +++++++++ src/Core/Base58.h | 14 -- src/Core/Types.h | 2 - src/DataTypes/DataTypeBase58.cpp | 34 ---- src/DataTypes/DataTypeBase58.h | 48 ----- src/DataTypes/DataTypeCustomBase58.cpp | 19 ++ src/DataTypes/DataTypeFactory.cpp | 1 + src/DataTypes/DataTypeFactory.h | 1 + .../Serializations/SerializationBase58.cpp | 49 ++--- src/Functions/CMakeLists.txt | 4 - src/Functions/FunctionBase58Conversion.h | 167 ++++-------------- src/Functions/FunctionsBase58.cpp | 2 - src/Functions/config_functions.h.in | 1 - src/Functions/configure_config.cmake | 3 - src/Functions/registerFunctionsString.cpp | 4 - src/configure_config.cmake | 3 - .../0_stateless/02337_base58.reference | 24 --- tests/queries/0_stateless/02337_base58.sql | 11 +- 28 files changed, 192 insertions(+), 359 deletions(-) delete mode 100644 base/base/Base58.h delete mode 160000 contrib/base-x delete mode 100644 contrib/base-x-cmake/CMakeLists.txt create mode 100644 src/Common/base58.h delete mode 100644 src/Core/Base58.h delete mode 100644 src/DataTypes/DataTypeBase58.cpp delete mode 100644 src/DataTypes/DataTypeBase58.h create mode 100644 src/DataTypes/DataTypeCustomBase58.cpp diff --git a/.gitmodules b/.gitmodules index c46b1c736fc..16099c63368 100644 --- a/.gitmodules +++ b/.gitmodules @@ -271,6 +271,3 @@ [submodule "contrib/liburing"] path = contrib/liburing url = https://github.com/axboe/liburing.git -[submodule "contrib/base-x"] - path = contrib/base-x - url = https://github.com/ClickHouse/base-x.git diff --git a/base/base/Base58.h b/base/base/Base58.h deleted file mode 100644 index c8bac92d186..00000000000 --- a/base/base/Base58.h +++ /dev/null @@ -1,9 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - using Base58 = StrongTypedef; -} diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 1dd28fa90ff..3c10c64441d 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -155,7 +155,6 @@ endif() add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) -add_contrib (base-x-cmake base-x) # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear diff --git a/contrib/base-x b/contrib/base-x deleted file mode 160000 index a85f98fb4ed..00000000000 --- a/contrib/base-x +++ /dev/null @@ -1 +0,0 @@ -Subproject commit a85f98fb4ed52c2f4029a4b6ac1ef0bafdfc56f5 diff --git a/contrib/base-x-cmake/CMakeLists.txt b/contrib/base-x-cmake/CMakeLists.txt deleted file mode 100644 index b0c11a176c6..00000000000 --- a/contrib/base-x-cmake/CMakeLists.txt +++ /dev/null @@ -1,28 +0,0 @@ -option (ENABLE_BASEX "Enable base-x" ${ENABLE_LIBRARIES}) - -if (NOT ENABLE_BASEX) - message(STATUS "Not using base-x") - return() -endif() - -set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/base-x") - -set (SRCS - ${LIBRARY_DIR}/base_x.hh - ${LIBRARY_DIR}/uinteger_t.hh - ) - -add_library(_base-x INTERFACE) -target_include_directories(_base-x SYSTEM BEFORE INTERFACE "${ClickHouse_SOURCE_DIR}/contrib/base-x") - -if (XCODE OR XCODE_VERSION) - # https://gitlab.kitware.com/cmake/cmake/issues/17457 - # Some native build systems may not like targets that have only object files, so consider adding at least one real source file - # This applies to Xcode. - if (NOT EXISTS "${CMAKE_CURRENT_BINARY_DIR}/dummy.c") - file(WRITE "${CMAKE_CURRENT_BINARY_DIR}/dummy.c" "") - endif () - target_sources(_base-x PRIVATE "${CMAKE_CURRENT_BINARY_DIR}/dummy.c") -endif () - -add_library(ch_contrib::base-x ALIAS _base-x) \ No newline at end of file diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index 8c1f6b5fc9e..1fbbff7dcca 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -18,7 +18,6 @@ The list of third-party libraries: | aws-c-common | [Apache](https://github.com/ClickHouse-Extras/aws-c-common/blob/736a82d1697c108b04a277e66438a7f4e19b6857/LICENSE) | | aws-c-event-stream | [Apache](https://github.com/ClickHouse-Extras/aws-c-event-stream/blob/3bc33662f9ccff4f4cbcf9509cc78c26e022fde0/LICENSE) | | aws-checksums | [Apache](https://github.com/ClickHouse-Extras/aws-checksums/blob/519d6d9093819b6cf89ffff589a27ef8f83d0f65/LICENSE) | -| base58 | [MIT](https://github.com/ClickHouse/base-x/blob/3e58874643c087f57e82b0ff03825c933fab945a/LICENSE) | | base64 | [BSD 2-clause](https://github.com/ClickHouse-Extras/Turbo-Base64/blob/af9b331f2b4f30b41c70f3a571ff904a8251c1d3/LICENSE) | | boost | [Boost](https://github.com/ClickHouse-Extras/boost/blob/9cf09dbfd55a5c6202dedbdf40781a51b02c2675/LICENSE_1_0.txt) | | boringssl | [BSD](https://github.com/ClickHouse-Extras/boringssl/blob/a6a2e2ab3e44d97ce98e51c558e989f211de7eb3/LICENSE) | diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 38d247457cf..2fa49cb6853 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -494,22 +494,21 @@ If the ‘s’ string is non-empty and does not contain the ‘c’ character at Returns the string ‘s’ that was converted from the encoding in ‘from’ to the encoding in ‘to’. -## Base58Encode(plaintext[, alphabet_name]), Base58Decode(encoded_text[, alphabet_name]) +## Base58Encode(plaintext), Base58Decode(encoded_text) -Accepts a String and encodes/decodes it using [Base58](https://tools.ietf.org/id/draft-msporny-base58-01.html) encoding scheme using specified alphabet. +Accepts a String and encodes/decodes it using [Base58](https://tools.ietf.org/id/draft-msporny-base58-01.html) encoding scheme using "Bitcoin" alphabet. **Syntax** ```sql -base58Encode(decoded[, alphabet_name]) -base58Decode(encoded[, alphabet_name]) +encodeBase58(decoded) +decodeBase58(encoded) ``` **Arguments** - `decoded` — [String](../../sql-reference/data-types/string.md) column or constant. - `encoded` — [String](../../sql-reference/data-types/string.md) column or constant. If the string is not a valid base58-encoded value, an exception is thrown. -- `alphabet_name` — String constant. Specifies alphabet used for encoding. Possible values: `gmp`, `bitcoin`, `ripple`, `flickr`. Default: `bitcoin`. **Returned value** @@ -522,16 +521,16 @@ Type: [String](../../sql-reference/data-types/string.md). Query: ``` sql -SELECT base58Encode('encode', 'flickr'); -SELECT base58Decode('izCFiDUY', 'ripple'); +SELECT encodeBase58('encode'); +SELECT decodeBase58('izCFiDUY'); ``` Result: ```text -┌─base58Encode('encode', 'flickr')─┐ +┌─encodeBase58('encode', 'flickr')─┐ │ SvyTHb1D │ └──────────────────────────────────┘ -┌─base58Decode('izCFiDUY', 'ripple')─┐ +┌─decodeBase58('izCFiDUY', 'ripple')─┐ │ decode │ └────────────────────────────────────┘ ``` diff --git a/docs/ru/development/contrib.md b/docs/ru/development/contrib.md index 0f4d22e90ce..1b99ec97553 100644 --- a/docs/ru/development/contrib.md +++ b/docs/ru/development/contrib.md @@ -18,7 +18,6 @@ sidebar_label: "Используемые сторонние библиотеки | aws-c-common | [Apache](https://github.com/ClickHouse-Extras/aws-c-common/blob/736a82d1697c108b04a277e66438a7f4e19b6857/LICENSE) | | aws-c-event-stream | [Apache](https://github.com/ClickHouse-Extras/aws-c-event-stream/blob/3bc33662f9ccff4f4cbcf9509cc78c26e022fde0/LICENSE) | | aws-checksums | [Apache](https://github.com/ClickHouse-Extras/aws-checksums/blob/519d6d9093819b6cf89ffff589a27ef8f83d0f65/LICENSE) | -| base58 | [MIT](https://github.com/ClickHouse/base-x/blob/3e58874643c087f57e82b0ff03825c933fab945a/LICENSE) | | base64 | [BSD 2-clause](https://github.com/ClickHouse-Extras/Turbo-Base64/blob/af9b331f2b4f30b41c70f3a571ff904a8251c1d3/LICENSE) | | boost | [Boost](https://github.com/ClickHouse-Extras/boost/blob/9cf09dbfd55a5c6202dedbdf40781a51b02c2675/LICENSE_1_0.txt) | | boringssl | [BSD](https://github.com/ClickHouse-Extras/boringssl/blob/a6a2e2ab3e44d97ce98e51c558e989f211de7eb3/LICENSE) | diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 1acf5ec58b2..c6e6fbdce9a 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -490,22 +490,21 @@ SELECT concat(key1, key2), sum(value) FROM key_val GROUP BY (key1, key2); Возвращает сконвертированную из кодировки from в кодировку to строку s. -## Base58Encode(plaintext[, alphabet_name]), Base58Decode(plaintext[, alphabet_name]) {#base58} +## Base58Encode(plaintext), Base58Decode(encoded_text) {#base58} -Принимает на вход строку или колонку строк и кодирует/раскодирует их с помощью схемы кодирования [Base58](https://tools.ietf.org/id/draft-msporny-base58-01.html) с использованием указанного алфавита. +Принимает на вход строку или колонку строк и кодирует/раскодирует их с помощью схемы кодирования [Base58](https://tools.ietf.org/id/draft-msporny-base58-01.html) с использованием стандартного алфавита Bitcoin. **Синтаксис** ```sql -base58Encode(decoded[, alphabet_name]) -base58Decode(encoded[, alphabet_name]) +encodeBase58(decoded) +decodeBase58(encoded) ``` **Аргументы** - `decoded` — Колонка или строка типа [String](../../sql-reference/data-types/string.md). - `encoded` — Колонка или строка типа [String](../../sql-reference/data-types/string.md). Если входная строка не является корректным кодом для какой-либо другой строки, возникнет исключение `1001`. -- `alphabet_name` — Строковая константа. Указывает алфавит, для которого необходимо получить код. Может принимать одно из следующих значений: `gmp`, `bitcoin`, `ripple`, `flickr`. По умолчанию: `bitcoin`. **Возвращаемое значение** @@ -518,16 +517,16 @@ base58Decode(encoded[, alphabet_name]) Запрос: ``` sql -SELECT base58Encode('encode', 'flickr'); -SELECT base58Decode('izCFiDUY', 'ripple'); +SELECT encodeBase58('encode'); +SELECT decodeBase58('izCFiDUY'); ``` Результат: ```text -┌─base58Encode('encode', 'flickr')─┐ +┌─encodeBase58('encode', 'flickr')─┐ │ SvyTHb1D │ └──────────────────────────────────┘ -┌─base58Decode('izCFiDUY', 'ripple')─┐ +┌─decodeBase58('izCFiDUY', 'ripple')─┐ │ decode │ └────────────────────────────────────┘ ``` diff --git a/src/Common/FieldVisitorToString.cpp b/src/Common/FieldVisitorToString.cpp index 7d93cfba78f..2a03620fb40 100644 --- a/src/Common/FieldVisitorToString.cpp +++ b/src/Common/FieldVisitorToString.cpp @@ -68,6 +68,7 @@ String FieldVisitorToString::operator() (const UUID & x) const { return formatQu String FieldVisitorToString::operator() (const AggregateFunctionStateData & x) const { return formatQuoted(x.data); } String FieldVisitorToString::operator() (const bool & x) const { return x ? "true" : "false"; } + String FieldVisitorToString::operator() (const Array & x) const { WriteBufferFromOwnString wb; diff --git a/src/Common/base58.h b/src/Common/base58.h new file mode 100644 index 00000000000..00eaaa7c5e4 --- /dev/null +++ b/src/Common/base58.h @@ -0,0 +1,85 @@ +#pragma once +#include +#include + +namespace DB +{ + +inline bool encodeBase58(const char8_t * src, char8_t * dst) +{ + const char * base58_encoding_alphabet = "123456789ABCDEFGHJKLMNPQRSTUVWXYZabcdefghijkmnopqrstuvwxyz"; + + size_t idx = 0; + for (; *src; ++src) + { + unsigned int carry = static_cast(*src); + for (size_t j = 0; j < idx; ++j) + { + carry += static_cast(dst[j] << 8); + dst[j] = static_cast(carry % 58); + carry /= 58; + } + while (carry > 0) + { + dst[idx++] = static_cast(carry % 58); + carry /= 58; + } + } + + size_t c_idx = idx >> 1; + for (size_t i = 0; i < c_idx; ++i) + { + char s = base58_encoding_alphabet[static_cast(dst[i])]; + dst[i] = base58_encoding_alphabet[static_cast(dst[idx - (i + 1)])]; + dst[idx - (i + 1)] = s; + } + if ((idx & 1)) + { + dst[c_idx] = base58_encoding_alphabet[static_cast(dst[c_idx])]; + } + dst[idx] = '\0'; + return true; +} + +inline bool decodeBase58(const char8_t * src, char8_t * dst) +{ + const char map_digits[128] + = {-1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 0, 1, 2, 3, 4, 5, 6, 7, 8, -1, -1, -1, -1, -1, -1, + -1, 9, 10, 11, 12, 13, 14, 15, 16, -1, 17, 18, 19, 20, 21, -1, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, -1, -1, -1, -1, -1, + -1, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, -1, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, -1, -1, -1, -1, -1}; + + size_t idx = 0; + + for (; *src; ++src) + { + unsigned int carry = static_cast(map_digits[static_cast(*src)]); + if (carry == UINT_MAX || *src < '1' || map_digits[static_cast(*src)] == map_digits[0]) + { + return false; + } + for (size_t j = 0; j < idx; j++) + { + carry += static_cast(dst[j]) * 58; + dst[j] = static_cast(carry & 0xff); + carry >>= 8; + } + while (carry > 0) + { + dst[idx++] = static_cast(carry & 0xff); + carry >>= 8; + } + } + + size_t c_idx = idx >> 1; + for (size_t i = 0; i < c_idx; ++i) + { + char s = dst[i]; + dst[i] = dst[idx - (i + 1)]; + dst[idx - (i + 1)] = s; + } + dst[idx] = '\0'; + return true; +} + +} diff --git a/src/Core/Base58.h b/src/Core/Base58.h deleted file mode 100644 index 6bf7ddbc07d..00000000000 --- a/src/Core/Base58.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -namespace Base58Helpers -{ - const Base58 Nil{}; -} - -} diff --git a/src/Core/Types.h b/src/Core/Types.h index d376c036673..92546d7d07a 100644 --- a/src/Core/Types.h +++ b/src/Core/Types.h @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB @@ -89,7 +88,6 @@ enum class TypeIndex LowCardinality, Map, Object, - Base58, }; #if !defined(__clang__) #pragma GCC diagnostic pop diff --git a/src/DataTypes/DataTypeBase58.cpp b/src/DataTypes/DataTypeBase58.cpp deleted file mode 100644 index f13236be88e..00000000000 --- a/src/DataTypes/DataTypeBase58.cpp +++ /dev/null @@ -1,34 +0,0 @@ -#include -#include -#include - - -namespace DB -{ - -bool DataTypeBase58::equals(const IDataType & rhs) const -{ - return typeid(rhs) == typeid(*this); -} - -SerializationPtr DataTypeBase58::doGetDefaultSerialization() const -{ - return std::make_shared(); -} - -Field DataTypeUUID::getDefault() const -{ - return UUID{}; -} - -MutableColumnPtr DataTypeUUID::createColumn() const -{ - return ColumnVector::create(); -} - -void registerDataTypeUUID(DataTypeFactory & factory) -{ - factory.registerSimpleDataType("UUID", [] { return DataTypePtr(std::make_shared()); }); -} - -} diff --git a/src/DataTypes/DataTypeBase58.h b/src/DataTypes/DataTypeBase58.h deleted file mode 100644 index f17e70431ba..00000000000 --- a/src/DataTypes/DataTypeBase58.h +++ /dev/null @@ -1,48 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ - -class DataTypeBase58 : public IDataType -{ -public: - static constexpr bool is_parametric = false; - - using FieldType = Base58; - using ColumnType = ColumnVector; - static constexpr auto type_id = TypeIndex::Base58; - - const char * getFamilyName() const override { return "Base58"; } - TypeIndex getTypeId() const override { return type_id; } - - Field getDefault() const override; - - MutableColumnPtr createColumn() const override; - - bool isParametric() const override { return false; } - bool haveSubtypes() const override { return false; } - - bool equals(const IDataType & rhs) const override; - - bool canBeUsedInBitOperations() const override { return true; } - bool canBeInsideNullable() const override { return true; } - bool canBePromoted() const override { return false; } - bool shouldAlignRightInPrettyFormats() const override { return false; } - bool textCanContainOnlyValidUTF8() const override { return true; } - bool isComparable() const override { return true; } - bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; } - bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const override { return true; } - bool haveMaximumSizeOfValue() const override { return true; } - size_t getSizeOfValueInMemory() const override { return sizeof(Base58); } - bool isCategorial() const override { return true; } - bool canBeInsideLowCardinality() const override { return true; } - - SerializationPtr doGetDefaultSerialization() const override; -}; - -} diff --git a/src/DataTypes/DataTypeCustomBase58.cpp b/src/DataTypes/DataTypeCustomBase58.cpp new file mode 100644 index 00000000000..2994f1b5a91 --- /dev/null +++ b/src/DataTypes/DataTypeCustomBase58.cpp @@ -0,0 +1,19 @@ +#include +#include +#include + +namespace DB +{ + +void registerDataTypeBase58(DataTypeFactory & factory) +{ + factory.registerSimpleDataTypeCustom("Base58", [] + { + auto type = DataTypeFactory::instance().get("String"); + return std::make_pair(type, std::make_unique( + std::make_unique("Base58"), std::make_unique(type->getDefaultSerialization()))); + }); + +} + +} diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index dd08656dccc..dcbf8785d31 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -219,6 +219,7 @@ DataTypeFactory::DataTypeFactory() registerDataTypeDomainGeo(*this); registerDataTypeMap(*this); registerDataTypeObject(*this); + registerDataTypeBase58(*this); } DataTypeFactory & DataTypeFactory::instance() diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index 704d8926bf0..1534a398ddb 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -88,5 +88,6 @@ void registerDataTypeDomainBool(DataTypeFactory & factory); void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory); void registerDataTypeDomainGeo(DataTypeFactory & factory); void registerDataTypeObject(DataTypeFactory & factory); +void registerDataTypeBase58(DataTypeFactory & factory); } diff --git a/src/DataTypes/Serializations/SerializationBase58.cpp b/src/DataTypes/Serializations/SerializationBase58.cpp index 3fe79d9c67e..c6cbee843a2 100644 --- a/src/DataTypes/Serializations/SerializationBase58.cpp +++ b/src/DataTypes/Serializations/SerializationBase58.cpp @@ -1,11 +1,11 @@ #include -#include -#include +#include #include #include #include #include +#include namespace DB @@ -17,47 +17,56 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -SerializationBase58::SerializationBase58(const SerializationPtr & nested_) - : SerializationCustomSimpleText(nested_) +SerializationBase58::SerializationBase58(const SerializationPtr & nested_) : SerializationCustomSimpleText(nested_) { } void SerializationBase58::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - const auto * col = checkAndGetColumn(&column); + const ColumnString * col = checkAndGetColumn(&column); if (!col) { - throw Exception("IPv4 type can only serialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Base58 type can only serialize columns of type String." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); } - auto - char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'}; + auto value = col->getDataAtWithTerminatingZero(row_num); + char buffer[value.size * 2 + 1]; char * ptr = buffer; - formatIPv4(reinterpret_cast(&col->getData()[row_num]), ptr); - + encodeBase58(reinterpret_cast(value.data), reinterpret_cast(ptr)); ostr.write(buffer, strlen(buffer)); } void SerializationBase58::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const { - ColumnUInt32 * col = typeid_cast(&column); + ColumnString * col = typeid_cast(&column); if (!col) { - throw Exception("IPv4 type can only deserialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Base58 type can only deserialize columns of type String." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); } - char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'}; - istr.read(buffer, sizeof(buffer) - 1); - UInt32 ipv4_value = 0; + size_t allocated = 32; + std::string encoded(allocated, '\0'); - bool parse_result = parseIPv4(buffer, reinterpret_cast(&ipv4_value)); - if (!parse_result && !settings.input_format_ipv4_default_on_conversion_error) + size_t read_position = 0; + while (istr.read(encoded[read_position])) { - throw Exception("Invalid IPv4 value", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); + ++read_position; + if (read_position == allocated) + { + allocated *= 2; + encoded.resize(allocated, '\0'); + } } - col->insert(ipv4_value); + char buffer[read_position + 1]; + if (!decodeBase58(reinterpret_cast(encoded.c_str()), reinterpret_cast(buffer))) + { + throw Exception("Invalid Base58 encoded value, cannot parse." + column.getName(), ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); + } + + col->insertDataWithTerminatingZero(buffer, read_position+1); if (whole && !istr.eof()) - throwUnexpectedDataAfterParsedValue(column, istr, settings, "IPv4"); + throwUnexpectedDataAfterParsedValue(column, istr, settings, "Base58"); +} } diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 1f728489b73..a37891377f4 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -72,10 +72,6 @@ if (TARGET ch_contrib::llvm) target_link_libraries(clickhouse_functions PRIVATE ch_contrib::llvm) endif () -if (TARGET ch_contrib::base-x) - target_link_libraries(clickhouse_functions PRIVATE ch_contrib::base-x) -endif() - if (TARGET ch_contrib::base64) target_link_libraries(clickhouse_functions PRIVATE ch_contrib::base64) endif() diff --git a/src/Functions/FunctionBase58Conversion.h b/src/Functions/FunctionBase58Conversion.h index 6979ce849f4..7842232961c 100644 --- a/src/Functions/FunctionBase58Conversion.h +++ b/src/Functions/FunctionBase58Conversion.h @@ -1,15 +1,13 @@ #pragma once -#include "config_functions.h" - -#if USE_BASEX -# include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB @@ -26,72 +24,39 @@ struct Base58Encode { static constexpr auto name = "base58Encode"; - static void process(const ColumnString & input, ColumnString::MutablePtr & dst_column, const std::string & alphabet, size_t input_rows_count) + static void process(const ColumnString & src_column, ColumnString::MutablePtr & dst_column, size_t input_rows_count) { auto & dst_data = dst_column->getChars(); auto & dst_offsets = dst_column->getOffsets(); - /// Wikipedia states Base58 has efficiency of 73%, and we take 1.5 scale to avoid reallocation in most cases - size_t current_allocated_size = ceil(1.5 * input.getChars().size()); + /// Base58 has efficiency of 73% (8/11) [https://monerodocs.org/cryptography/base58/], + /// and we take double scale to avoid any reallocation. - dst_data.resize(current_allocated_size); + size_t max_result_size = ceil(2 * src_column.getChars().size() + 1); + + dst_data.resize(max_result_size); dst_offsets.resize(input_rows_count); - const ColumnString::Offsets & src_offsets = input.getOffsets(); + const ColumnString::Offsets & src_offsets = src_column.getOffsets(); - const auto * source = input.getChars().raw_data(); + const auto * source = src_column.getChars().data(); auto * dst = dst_data.data(); auto * dst_pos = dst; size_t src_offset_prev = 0; - size_t processed_size = 0; - const auto& encoder = (alphabet == "bitcoin") ? Base58::bitcoin() : - ((alphabet == "flickr") ? Base58::flickr() : - ((alphabet == "ripple") ? Base58::ripple() : - Base58::base58())); //GMP - - std::string encoded; for (size_t row = 0; row < input_rows_count; ++row) { size_t srclen = src_offsets[row] - src_offset_prev - 1; - /// Why we didn't use char* here? - /// We don't know the size of the result string beforehand (it's not byte-to-byte encoding), - /// so we may need to do many resizes (the worst case -- we'll do it for each row) - /// This way we do exponential resizes and one final resize after whole operation is complete - encoded.clear(); - if (srclen) - try - { - encoder.encode(encoded, source, srclen); - } - catch (const std::invalid_argument& e) - { - throw Exception(e.what(), ErrorCodes::BAD_ARGUMENTS); - } - catch (const std::domain_error& e) - { - throw Exception(e.what(), ErrorCodes::BAD_ARGUMENTS); - } - size_t outlen = encoded.size(); + encodeBase58(source, dst_pos); - if (processed_size + outlen >= current_allocated_size) - { - current_allocated_size += current_allocated_size; - dst_data.resize(current_allocated_size); - auto processed_offset = dst_pos - dst; - dst = dst_data.data(); - dst_pos = dst; - dst_pos += processed_offset; - } - std::memcpy(dst_pos, encoded.c_str(), ++outlen); + size_t encoded_length = strlen(reinterpret_cast(dst_pos)); source += srclen + 1; - dst_pos += outlen; + dst_pos += encoded_length + 1; dst_offsets[row] = dst_pos - dst; src_offset_prev = src_offsets[row]; - processed_size += outlen; } dst_data.resize(dst_pos - dst); @@ -102,72 +67,40 @@ struct Base58Decode { static constexpr auto name = "base58Decode"; - static void process(const ColumnString & input, ColumnString::MutablePtr & dst_column, const std::string & alphabet, size_t input_rows_count) + static void process(const ColumnString & src_column, ColumnString::MutablePtr & dst_column, size_t input_rows_count) { auto & dst_data = dst_column->getChars(); auto & dst_offsets = dst_column->getOffsets(); - /// We allocate probably even more then needed to avoid many resizes - size_t current_allocated_size = input.getChars().size(); + /// Base58 has efficiency of 73% (8/11) [https://monerodocs.org/cryptography/base58/], + /// and decoded value will be no longer than source. - dst_data.resize(current_allocated_size); + size_t max_result_size = src_column.getChars().size() + 1; + + dst_data.resize(max_result_size); dst_offsets.resize(input_rows_count); - const ColumnString::Offsets & src_offsets = input.getOffsets(); + const ColumnString::Offsets & src_offsets = src_column.getOffsets(); - const auto * source = input.getChars().raw_data(); + const auto * source = src_column.getChars().data(); auto * dst = dst_data.data(); auto * dst_pos = dst; size_t src_offset_prev = 0; - size_t processed_size = 0; - const auto& decoder = (alphabet == "bitcoin") ? Base58::bitcoin() : - ((alphabet == "flickr") ? Base58::flickr() : - ((alphabet == "ripple") ? Base58::ripple() : - Base58::base58())); - - std::string decoded; for (size_t row = 0; row < input_rows_count; ++row) { size_t srclen = src_offsets[row] - src_offset_prev - 1; - /// Why we didn't use char* here? - /// We don't know the size of the result string beforehand (it's not byte-to-byte encoding), - /// so we may need to do many resizes (the worst case -- we'll do it for each row) - /// This way we do exponential resizes and one final resize after whole operation is complete - decoded.clear(); - if (srclen) - try - { - decoder.decode(decoded, source, srclen); - } - catch (const std::invalid_argument& e) - { - throw Exception(e.what(), ErrorCodes::BAD_ARGUMENTS); - } - catch (const std::domain_error& e) - { - throw Exception(e.what(), ErrorCodes::BAD_ARGUMENTS); - } - size_t outlen = decoded.size(); + if (!decodeBase58(source, dst_pos)) + throw Exception("Invalid Base58 value, cannot be decoded", ErrorCodes::BAD_ARGUMENTS); - if (processed_size + outlen >= current_allocated_size) - { - current_allocated_size += current_allocated_size; - dst_data.resize(current_allocated_size); - auto processed_offset = dst_pos - dst; - dst = dst_data.data(); - dst_pos = dst; - dst_pos += processed_offset; - } - std::memcpy(dst_pos, decoded.c_str(), ++outlen); + size_t encoded_length = strlen(reinterpret_cast(dst_pos)); source += srclen + 1; - dst_pos += outlen; + dst_pos += encoded_length + 1; dst_offsets[row] = dst_pos - dst; src_offset_prev = src_offsets[row]; - processed_size += outlen; } dst_data.resize(dst_pos - dst); @@ -190,9 +123,7 @@ public: return Func::name; } - bool isVariadic() const override { return true; } - - size_t getNumberOfArguments() const override { return 0; } + size_t getNumberOfArguments() const override { return 1; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } @@ -202,19 +133,12 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() != 1 && arguments.size() != 2) - throw Exception( - "Wrong number of arguments for function " + getName() + ": 1 or 2 expected.", - ErrorCodes::BAD_ARGUMENTS); + if (arguments.size() != 1) + throw Exception("Wrong number of arguments for function " + getName() + ": 1 expected.", ErrorCodes::BAD_ARGUMENTS); if (!isString(arguments[0].type)) throw Exception( - "Illegal type " + arguments[0].type->getName() + " of 1st argument of function " + getName() + ". Must be String.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (arguments.size() == 2 && !isString(arguments[1].type)) - throw Exception( - "Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName() + ". Must be String.", + "Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + ". Must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(); @@ -229,28 +153,11 @@ public: "Illegal column " + arguments[0].column->getName() + " of first argument of function " + getName() + ", must be String", ErrorCodes::ILLEGAL_COLUMN); - std::string alphabet = "bitcoin"; - - if (arguments.size() == 2) - { - const auto * alphabet_column = checkAndGetColumn(arguments[1].column.get()); - - if (!alphabet_column) - throw Exception("Second argument for function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - alphabet = alphabet_column->getValue(); - if (alphabet != "bitcoin" && alphabet != "ripple" && alphabet != "flickr" && alphabet != "gmp") - throw Exception("Second argument for function " + getName() + " must be 'bitcoin', 'ripple', 'gmp' or 'flickr'", ErrorCodes::ILLEGAL_COLUMN); - - } - auto dst_column = ColumnString::create(); - Func::process(*input, dst_column, alphabet, input_rows_count); + Func::process(*input, dst_column, input_rows_count); return dst_column; } }; } - -#endif diff --git a/src/Functions/FunctionsBase58.cpp b/src/Functions/FunctionsBase58.cpp index dc325ff8cfc..a336fbe6b07 100644 --- a/src/Functions/FunctionsBase58.cpp +++ b/src/Functions/FunctionsBase58.cpp @@ -1,5 +1,4 @@ #include -#if USE_BASEX #include namespace DB @@ -14,4 +13,3 @@ void registerFunctionBase58Decode(FunctionFactory & factory) factory.registerFunction>(); } } -#endif diff --git a/src/Functions/config_functions.h.in b/src/Functions/config_functions.h.in index 04ae205aee2..86535d65069 100644 --- a/src/Functions/config_functions.h.in +++ b/src/Functions/config_functions.h.in @@ -2,7 +2,6 @@ // .h autogenerated by cmake! -#cmakedefine01 USE_BASEX #cmakedefine01 USE_BASE64 #cmakedefine01 USE_SIMDJSON #cmakedefine01 USE_RAPIDJSON diff --git a/src/Functions/configure_config.cmake b/src/Functions/configure_config.cmake index df2a8dc174c..33c36412844 100644 --- a/src/Functions/configure_config.cmake +++ b/src/Functions/configure_config.cmake @@ -1,9 +1,6 @@ if (TARGET ch_contrib::fastops) set(USE_FASTOPS 1) endif() -if (TARGET ch_contrib::base-x) - set(USE_BASEX 1) -endif() if (TARGET ch_contrib::base64) set(USE_BASE64 1) endif() diff --git a/src/Functions/registerFunctionsString.cpp b/src/Functions/registerFunctionsString.cpp index 248b6391b4f..9e75e9900b7 100644 --- a/src/Functions/registerFunctionsString.cpp +++ b/src/Functions/registerFunctionsString.cpp @@ -49,10 +49,8 @@ void registerFunctionBase64Decode(FunctionFactory &); void registerFunctionTryBase64Decode(FunctionFactory &); #endif -#if USE_BASEX void registerFunctionBase58Encode(FunctionFactory &); void registerFunctionBase58Decode(FunctionFactory &); -#endif #if USE_NLP void registerFunctionStem(FunctionFactory &); @@ -110,10 +108,8 @@ void registerFunctionsString(FunctionFactory & factory) registerFunctionTryBase64Decode(factory); #endif -#if USE_BASEX registerFunctionBase58Encode(factory); registerFunctionBase58Decode(factory); -#endif #if USE_NLP registerFunctionStem(factory); diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 45e45b505d4..519307ba28a 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -55,9 +55,6 @@ endif() if (TARGET ch_contrib::base64) set(USE_BASE64 1) endif() -if (TARGET ch_contrib::base-x) - set(USE_BASEX 1) -endif() if (TARGET ch_contrib::yaml_cpp) set(USE_YAML_CPP 1) endif() diff --git a/tests/queries/0_stateless/02337_base58.reference b/tests/queries/0_stateless/02337_base58.reference index f4a2c95304b..bc666044388 100644 --- a/tests/queries/0_stateless/02337_base58.reference +++ b/tests/queries/0_stateless/02337_base58.reference @@ -8,30 +8,6 @@ fooba foobar Hello world! -f -fo -foo -foob -fooba -foobar -Hello world! - -f -fo -foo -foob -fooba -foobar -Hello world! - -f -fo -foo -foob -fooba -foobar -Hello world! - 2m 8o8 bQbp diff --git a/tests/queries/0_stateless/02337_base58.sql b/tests/queries/0_stateless/02337_base58.sql index 34da1da4c86..9c9379a2854 100644 --- a/tests/queries/0_stateless/02337_base58.sql +++ b/tests/queries/0_stateless/02337_base58.sql @@ -3,15 +3,10 @@ SET send_logs_level = 'fatal'; SELECT base58Encode('Hold my beer...'); -SELECT base58Encode('Hold my beer...', ''); -- { serverError 44 } -SELECT base58Encode('Hold my beer...', 'gmp', 'third'); -- { serverError 36 } +SELECT base58Encode('Hold my beer...', 'Second arg'); -- { serverError 42 } +SELECT base58Decode('Hold my beer...'); -- { serverError 36 } -SELECT base58Decode(encoded, 'gmp') FROM (SELECT base58Encode(val, 'gmp') as encoded FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar', 'Hello world!']) val)); -SELECT base58Decode(encoded, 'ripple') FROM (SELECT base58Encode(val, 'ripple') as encoded FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar', 'Hello world!']) val)); -SELECT base58Decode(encoded, 'flickr') FROM (SELECT base58Encode(val, 'flickr') as encoded FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar', 'Hello world!']) val)); -SELECT base58Decode(encoded, 'bitcoin') FROM (SELECT base58Encode(val, 'bitcoin') as encoded FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar', 'Hello world!']) val)); +SELECT base58Decode(encoded) FROM (SELECT base58Encode(val) as encoded FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar', 'Hello world!']) val)); SELECT base58Encode(val) FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar']) val); SELECT base58Decode(val) FROM (select arrayJoin(['', '2m', '8o8', 'bQbp', '3csAg9', 'CZJRhmz', 't1Zv2yaZ']) val); - -SELECT base58Decode('Why_not?'); -- { serverError 36 } From 3089da16bebe3d9b365923c752419c1da0f407d1 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 16 Jul 2022 19:07:42 +0200 Subject: [PATCH 307/659] fix identation --- src/Common/FieldVisitorToString.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/FieldVisitorToString.cpp b/src/Common/FieldVisitorToString.cpp index 2a03620fb40..7d93cfba78f 100644 --- a/src/Common/FieldVisitorToString.cpp +++ b/src/Common/FieldVisitorToString.cpp @@ -68,7 +68,6 @@ String FieldVisitorToString::operator() (const UUID & x) const { return formatQu String FieldVisitorToString::operator() (const AggregateFunctionStateData & x) const { return formatQuoted(x.data); } String FieldVisitorToString::operator() (const bool & x) const { return x ? "true" : "false"; } - String FieldVisitorToString::operator() (const Array & x) const { WriteBufferFromOwnString wb; From 283b5980006d366c7d150d54a1a1dcbd9fd6fff5 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 16 Jul 2022 19:29:57 +0200 Subject: [PATCH 308/659] fix unused changes --- src/DataTypes/DataTypeCustomBase58.cpp | 2 +- src/DataTypes/DataTypeFactory.cpp | 2 +- src/DataTypes/DataTypeFactory.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/DataTypeCustomBase58.cpp b/src/DataTypes/DataTypeCustomBase58.cpp index 2994f1b5a91..67c4399c9a0 100644 --- a/src/DataTypes/DataTypeCustomBase58.cpp +++ b/src/DataTypes/DataTypeCustomBase58.cpp @@ -5,7 +5,7 @@ namespace DB { -void registerDataTypeBase58(DataTypeFactory & factory) +void registerDataTypeDomainBase58(DataTypeFactory & factory) { factory.registerSimpleDataTypeCustom("Base58", [] { diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index dcbf8785d31..336934c7a56 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -219,7 +219,7 @@ DataTypeFactory::DataTypeFactory() registerDataTypeDomainGeo(*this); registerDataTypeMap(*this); registerDataTypeObject(*this); - registerDataTypeBase58(*this); + registerDataTypeDomainBase58(*this); } DataTypeFactory & DataTypeFactory::instance() diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index 1534a398ddb..d46dbbade7a 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -88,6 +88,6 @@ void registerDataTypeDomainBool(DataTypeFactory & factory); void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory); void registerDataTypeDomainGeo(DataTypeFactory & factory); void registerDataTypeObject(DataTypeFactory & factory); -void registerDataTypeBase58(DataTypeFactory & factory); +void registerDataTypeDomainBase58(DataTypeFactory & factory); } From 9ac7e41e07fe6d5f2db99ff66e1bdc420590d381 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 16 Jul 2022 19:45:33 +0200 Subject: [PATCH 309/659] updated docs --- .../sql-reference/data-types/domains/base58.md | 12 ++++++++++++ .../sql-reference/functions/string-functions.md | 16 ++++++++-------- 2 files changed, 20 insertions(+), 8 deletions(-) create mode 100644 docs/en/sql-reference/data-types/domains/base58.md diff --git a/docs/en/sql-reference/data-types/domains/base58.md b/docs/en/sql-reference/data-types/domains/base58.md new file mode 100644 index 00000000000..5c2afe106cc --- /dev/null +++ b/docs/en/sql-reference/data-types/domains/base58.md @@ -0,0 +1,12 @@ +--- +sidebar_position: 61 +sidebar_label: Base58 +--- + +## Base58 + +`Base58` is a domain based on `String` type created for storing Base58-encoded strings values. + +Usage is the same as for general `String` type. THe only difference is that string of `Base58` type must be a correct Base58-encoded string. + +[Original article](https://clickhouse.com/docs/en/data_types/domains/base58) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 2fa49cb6853..55ab11763d5 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -501,8 +501,8 @@ Accepts a String and encodes/decodes it using [Base58](https://tools.ietf.org/id **Syntax** ```sql -encodeBase58(decoded) -decodeBase58(encoded) +base58Encode(decoded) +base58Decode(encoded) ``` **Arguments** @@ -521,17 +521,17 @@ Type: [String](../../sql-reference/data-types/string.md). Query: ``` sql -SELECT encodeBase58('encode'); -SELECT decodeBase58('izCFiDUY'); +SELECT base58Encode('Encoded'); +SELECT base58Encode('3dc8KtHrwM'); ``` Result: ```text -┌─encodeBase58('encode', 'flickr')─┐ -│ SvyTHb1D │ +┌─encodeBase58('Encoded')─┐ +│ 3dc8KtHrwM │ └──────────────────────────────────┘ -┌─decodeBase58('izCFiDUY', 'ripple')─┐ -│ decode │ +┌─decodeBase58('3dc8KtHrwM')─┐ +│ Encoded │ └────────────────────────────────────┘ ``` From c07c65615dc7c8b1e405c52b8e8795734aa67063 Mon Sep 17 00:00:00 2001 From: Wangyang Guo Date: Sun, 17 Jul 2022 12:01:53 +0800 Subject: [PATCH 310/659] fix ENABLE_MULTITARGET_CODE not defined warning in some builds --- src/Common/TargetSpecific.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Common/TargetSpecific.h b/src/Common/TargetSpecific.h index 9d0e0d9eede..f078c0e3ffc 100644 --- a/src/Common/TargetSpecific.h +++ b/src/Common/TargetSpecific.h @@ -89,6 +89,10 @@ bool isArchSupported(TargetArch arch); String toString(TargetArch arch); +#ifndef ENABLE_MULTITARGET_CODE +# define ENABLE_MULTITARGET_CODE 0 +#endif + #if ENABLE_MULTITARGET_CODE && defined(__GNUC__) && defined(__x86_64__) /// NOLINTNEXTLINE From 30018ce5d3b41ef4632b36ed6b4c1d9de6cdb4fd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Jul 2022 12:13:36 +0300 Subject: [PATCH 311/659] Fix running cmake with predefined cache (for development only) Right now cmake add the following options only if USE_STATIC_LIBRARIES is OFF: - SPLIT_SHARED_LIBRARIES - CLICKHOUSE_SPLIT_BINARY And this breaks the following usage: $ cmake .. $ cat > debug-build-cache.cmake set(USE_STATIC_LIBRARIES OFF CACHE BOOL "") set(SPLIT_SHARED_LIBRARIES ON CACHE BOOL "") set(CLICKHOUSE_SPLIT_BINARY ON CACHE BOOL "") ^D $ cmake -C debug-build-cache.cmake .. CMake Error at CMakeLists.txt:83 (message): Defining SPLIT_SHARED_LIBRARIES=1 without USE_STATIC_LIBRARIES=0 has no effect. Since with this initial cache we have the following: - USE_STATIC_LIBRARIES=OFF (because it was already set) - SPLIT_SHARED_LIBRARIES=ON (was not set before, so new value) - CLICKHOUSE_SPLIT_BINARY (was not set before, also new value) Yes this is not the common usage, but it seems that it is pretty easy to avoid. Signed-off-by: Azat Khuzhin --- CMakeLists.txt | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 367a88140e5..f2ab66afd9b 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -74,15 +74,10 @@ message (STATUS "CMAKE_BUILD_TYPE: ${CMAKE_BUILD_TYPE}") string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC) option(USE_STATIC_LIBRARIES "Disable to use shared libraries" ON) - -if (NOT USE_STATIC_LIBRARIES) - # DEVELOPER ONLY. - # Faster linking if turned on. - option(SPLIT_SHARED_LIBRARIES "Keep all internal libraries as separate .so files") - - option(CLICKHOUSE_SPLIT_BINARY - "Make several binaries (clickhouse-server, clickhouse-client etc.) instead of one bundled") -endif () +# DEVELOPER ONLY. +# Faster linking if turned on. +option(SPLIT_SHARED_LIBRARIES "Keep all internal libraries as separate .so files" OFF) +option(CLICKHOUSE_SPLIT_BINARY "Make several binaries (clickhouse-server, clickhouse-client etc.) instead of one bundled" OFF) if (USE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES) message(FATAL_ERROR "Defining SPLIT_SHARED_LIBRARIES=1 without USE_STATIC_LIBRARIES=0 has no effect.") From 4b11a718cbd24f5d1be6a6e99d5b75bbe3c7a6f5 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 17 Jul 2022 11:55:34 +0200 Subject: [PATCH 312/659] Update fetchPostgreSQLTableStructure.cpp --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index fffd4d4ef7c..eeae110cddf 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -40,7 +40,7 @@ std::set fetchPostgreSQLTablesList(T & tx, const String & postgres_schem if (schemas.size() <= 1) { std::string query = fmt::format( - "SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname != 'pg_catalog' AND schemaname = {}", + "SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname = {}", postgres_schema.empty() ? quoteString("public") : quoteString(postgres_schema)); for (auto table_name : tx.template stream(query)) @@ -55,7 +55,7 @@ std::set fetchPostgreSQLTablesList(T & tx, const String & postgres_schem for (const auto & schema : schemas) { std::string query = fmt::format( - "SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname != 'pg_catalog' AND schemaname = {}", + "SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname = {}", quoteString(schema)); for (auto table_name : tx.template stream(query)) From 91be72dbb447a1db690d2d6ed4f9815e8d226ab3 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 17 Jul 2022 12:17:16 +0200 Subject: [PATCH 313/659] updated --- contrib/CMakeLists.txt | 1 + contrib/base-x | 1 + contrib/base-x-cmake/CMakeLists.txt | 28 ++++++++++++++++++++++++++++ contrib/jemalloc | 2 +- contrib/libprotobuf-mutator | 2 +- contrib/poco | 2 +- contrib/simdjson | 2 +- contrib/thrift | 2 +- 8 files changed, 35 insertions(+), 5 deletions(-) create mode 160000 contrib/base-x create mode 100644 contrib/base-x-cmake/CMakeLists.txt diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 489f788a512..d4a3f164214 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -156,6 +156,7 @@ endif() add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) +add_contrib (base-x-cmake base-x) # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear diff --git a/contrib/base-x b/contrib/base-x new file mode 160000 index 00000000000..a85f98fb4ed --- /dev/null +++ b/contrib/base-x @@ -0,0 +1 @@ +Subproject commit a85f98fb4ed52c2f4029a4b6ac1ef0bafdfc56f5 diff --git a/contrib/base-x-cmake/CMakeLists.txt b/contrib/base-x-cmake/CMakeLists.txt new file mode 100644 index 00000000000..b0c11a176c6 --- /dev/null +++ b/contrib/base-x-cmake/CMakeLists.txt @@ -0,0 +1,28 @@ +option (ENABLE_BASEX "Enable base-x" ${ENABLE_LIBRARIES}) + +if (NOT ENABLE_BASEX) + message(STATUS "Not using base-x") + return() +endif() + +set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/base-x") + +set (SRCS + ${LIBRARY_DIR}/base_x.hh + ${LIBRARY_DIR}/uinteger_t.hh + ) + +add_library(_base-x INTERFACE) +target_include_directories(_base-x SYSTEM BEFORE INTERFACE "${ClickHouse_SOURCE_DIR}/contrib/base-x") + +if (XCODE OR XCODE_VERSION) + # https://gitlab.kitware.com/cmake/cmake/issues/17457 + # Some native build systems may not like targets that have only object files, so consider adding at least one real source file + # This applies to Xcode. + if (NOT EXISTS "${CMAKE_CURRENT_BINARY_DIR}/dummy.c") + file(WRITE "${CMAKE_CURRENT_BINARY_DIR}/dummy.c" "") + endif () + target_sources(_base-x PRIVATE "${CMAKE_CURRENT_BINARY_DIR}/dummy.c") +endif () + +add_library(ch_contrib::base-x ALIAS _base-x) \ No newline at end of file diff --git a/contrib/jemalloc b/contrib/jemalloc index 78b58379c85..41a859ef732 160000 --- a/contrib/jemalloc +++ b/contrib/jemalloc @@ -1 +1 @@ -Subproject commit 78b58379c854a639df79beb3289351129d863d4b +Subproject commit 41a859ef7325569c6c25f92d294d45123bb81355 diff --git a/contrib/libprotobuf-mutator b/contrib/libprotobuf-mutator index ffd86a32874..a304ec48dcf 160000 --- a/contrib/libprotobuf-mutator +++ b/contrib/libprotobuf-mutator @@ -1 +1 @@ -Subproject commit ffd86a32874e5c08a143019aad1aaf0907294c9f +Subproject commit a304ec48dcf15d942607032151f7e9ee504b5dcf diff --git a/contrib/poco b/contrib/poco index 0e32cb42db7..9fec8e11dbb 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 0e32cb42db76ddaa76848470219056908053b676 +Subproject commit 9fec8e11dbb6a352e1cfba8cc9e23ebd7fb77310 diff --git a/contrib/simdjson b/contrib/simdjson index de196dd7a3a..1075e8609c4 160000 --- a/contrib/simdjson +++ b/contrib/simdjson @@ -1 +1 @@ -Subproject commit de196dd7a3a16e4056b0551ffa3b85c2f52581e1 +Subproject commit 1075e8609c4afa253162d441437af929c29e31bb diff --git a/contrib/thrift b/contrib/thrift index 010ccf0a0c7..2a93df80f27 160000 --- a/contrib/thrift +++ b/contrib/thrift @@ -1 +1 @@ -Subproject commit 010ccf0a0c7023fea0f6bf4e4078ebdff7e61982 +Subproject commit 2a93df80f27739ccabb5b885cb12a8dc7595ecdf From 9cc06ea0e4c68dec641875bcb986bc54ff8b7bb0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Sun, 17 Jul 2022 14:05:21 +0200 Subject: [PATCH 314/659] Fix found points during the review --- tests/ci/github_helper.py | 37 +++++++++++++++++++++++-------------- 1 file changed, 23 insertions(+), 14 deletions(-) diff --git a/tests/ci/github_helper.py b/tests/ci/github_helper.py index 74793101b17..426ab19a520 100644 --- a/tests/ci/github_helper.py +++ b/tests/ci/github_helper.py @@ -27,6 +27,7 @@ class GitHub(github.Github): self._cache_path = Path(CACHE_PATH) # And set Path super().__init__(*args, **kwargs) + self._retries = 0 # pylint: disable=signature-differs def search_issues(self, *args, **kwargs) -> Issues: # type: ignore @@ -36,25 +37,23 @@ class GitHub(github.Github): splittable = False for arg, value in kwargs.items(): if arg in ["closed", "created", "merged", "updated"]: - if ( - hasattr(value, "__iter__") - and not isinstance(value, str) - and not splittable - ): + if hasattr(value, "__iter__") and not isinstance(value, str): assert [True for v in value if isinstance(v, (date, datetime))] assert len(value) == 2 - preserved_arg = arg - preserved_value = value - middle_value = value[0] + (value[1] - value[0]) / 2 - splittable = middle_value not in value kwargs[arg] = f"{value[0].isoformat()}..{value[1].isoformat()}" + if not splittable: + # We split only by the first met splittable argument + preserved_arg = arg + preserved_value = value + middle_value = value[0] + (value[1] - value[0]) / 2 + splittable = middle_value not in value continue assert isinstance(value, (date, datetime, str)) inter_result = [] # type: Issues - for i in range(3): + for i in range(self.retries): try: - logger.debug("Search issues, args=%s, kwards=%s", args, kwargs) + logger.debug("Search issues, args=%s, kwargs=%s", args, kwargs) result = super().search_issues(*args, **kwargs) if result.totalCount == 1000 and splittable: # The hard limit is 1000. If it's splittable, then we make @@ -83,7 +82,7 @@ class GitHub(github.Github): inter_result.extend(result) return inter_result except RateLimitExceededException as e: - if i == 2: + if i == self.retries - 1: exception = e self.sleep_on_rate_limit() @@ -136,12 +135,12 @@ class GitHub(github.Github): if updated_at <= cached_pr.updated_at: logger.debug("Getting PR #%s from cache", number) return cached_pr - for i in range(3): + for i in range(self.retries): try: pr = repo.get_pull(number) break except RateLimitExceededException: - if i == 2: + if i == self.retries - 1: raise self.sleep_on_rate_limit() logger.debug("Getting PR #%s from API", number) @@ -160,3 +159,13 @@ class GitHub(github.Github): assert self._cache_path.is_dir() else: self._cache_path.mkdir(parents=True) + + @property + def retries(self): + if self._retries == 0: + self._retries = 3 + return self._retries + + @retries.setter + def retries(self, value: int): + self._retries = value From d86abba2c6b1c6843ee46202d5225fcfb3b6be79 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Fri, 15 Jul 2022 16:48:34 +0200 Subject: [PATCH 315/659] Fix macosx compilation due to endian.h --- utils/self-extracting-executable/compressor.cpp | 11 +++++++++++ utils/self-extracting-executable/decompressor.cpp | 11 +++++++++++ 2 files changed, 22 insertions(+) diff --git a/utils/self-extracting-executable/compressor.cpp b/utils/self-extracting-executable/compressor.cpp index f6bbc33aa16..5e055a34825 100644 --- a/utils/self-extracting-executable/compressor.cpp +++ b/utils/self-extracting-executable/compressor.cpp @@ -9,7 +9,18 @@ #include #include #include +#if defined __APPLE__ + +// dependencies +#include +#include + +// define 64 bit macros +#define htole64(x) OSSwapHostToLittleInt64(x) + +#else #include +#endif #include "types.h" diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 97bcda763e3..8914f611e10 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -12,7 +12,18 @@ #include #include #include +#if defined __APPLE__ + +// dependencies +#include +#include + +// define 64 bit macros +#define le64toh(x) OSSwapLittleToHostInt64(x) + +#else #include +#endif #include "types.h" From d99851bcbed183f7949a0d34e811a81e6ed97667 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Jul 2022 17:01:47 +0300 Subject: [PATCH 316/659] Fix googletest contrib compilation (due to GTEST_HAS_POSIX_RE=0) By some reason cmake rules for googletest sets GTEST_HAS_POSIX_RE=0 (Compatibilty? But which platform that does support ClickHouse does not have it?) But everything will be okay, if these macros was set PUBLIC (i.e. for compiling googletest library itself and it's users), however it was added as INTERFACE only (so library itself does not know about GTEST_HAS_POSIX_RE=0), and this leads to UB, here ASan report (while I was trying to use ASSERT_EXIT()).
ASan report [ RUN ] Common.LSan ================================================================= ==7566==ERROR: AddressSanitizer: heap-buffer-overflow on address 0x6030005b2388 at pc 0x00000d00924c bp 0x7ffcd3b7cfb0 sp 0x7ffcd3b7c770 WRITE of size 64 at 0x6030005b2388 thread T0 0 0xd00924b in regcomp (/bld/src/unit_tests_dbms+0xd00924b) (BuildId: 40d3fa83125f9047) 1 0x29ca243b in testing::internal::RE::Init(char const*) /bld/./contrib/googletest/googletest/src/gtest-port.cc:750:15 2 0xd4d92b3 in testing::internal::RE::RE(std::__1::basic_string, std::__1::allocator > const&) /bld/./contrib/googletest/googletest/include/gtest/internal/gtest-port.h:896:36 3 0xd4d92b3 in testing::PolymorphicMatcher testing::ContainsRegex(char const* const&) /bld/./contrib/googletest/googletest/include/gtest/gtest-matchers.h:868:28 4 0xd4d813a in testing::internal::MakeDeathTestMatcher(char const*) /bld/./contrib/googletest/googletest/include/gtest/internal/gtest-death-test-internal.h:173:10 5 0xd4d813a in Common_LSan_Test::TestBody() /bld/./src/Common/tests/gtest_lsan.cpp:11:5 0x6030005b2388 is located 0 bytes to the right of 24-byte region [0x6030005b2370,0x6030005b2388) allocated by thread T0 here: 0 0xd066fbd in operator new(unsigned long) (/bld/src/unit_tests_dbms+0xd066fbd) (BuildId: 40d3fa83125f9047) 1 0xd4d913d in testing::PolymorphicMatcher testing::ContainsRegex(char const* const&) /bld/./contrib/googletest/googletest/include/gtest/gtest-matchers.h:868:24 2 0xd4d813a in testing::internal::MakeDeathTestMatcher(char const*) /bld/./contrib/googletest/googletest/include/gtest/internal/gtest-death-test-internal.h:173:10 3 0xd4d813a in Common_LSan_Test::TestBody() /bld/./src/Common/tests/gtest_lsan.cpp:11:5
--- contrib/googletest-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/googletest-cmake/CMakeLists.txt b/contrib/googletest-cmake/CMakeLists.txt index f116eddc337..90fdde0c185 100644 --- a/contrib/googletest-cmake/CMakeLists.txt +++ b/contrib/googletest-cmake/CMakeLists.txt @@ -2,7 +2,7 @@ set (SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest") add_library(_gtest "${SRC_DIR}/src/gtest-all.cc") set_target_properties(_gtest PROPERTIES VERSION "1.0.0") -target_compile_definitions (_gtest INTERFACE GTEST_HAS_POSIX_RE=0) +target_compile_definitions (_gtest PUBLIC GTEST_HAS_POSIX_RE=0) target_include_directories(_gtest SYSTEM PUBLIC "${SRC_DIR}/include") target_include_directories(_gtest PRIVATE "${SRC_DIR}") From 13482af4ee77af196b47ce6de1a076a3c5e8547d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 17 Jul 2022 15:22:12 +0000 Subject: [PATCH 317/659] First try at reducing the use of StringRef - to be replaced by std::string_view - suggested in #39262 --- src/AggregateFunctions/AggregateFunctionMap.h | 2 +- src/Common/CurrentThread.h | 3 +- src/Common/FileSegment.cpp | 4 +-- src/Common/IFileCache.cpp | 4 +-- src/Common/ThreadStatus.h | 2 +- src/Common/TraceSender.cpp | 2 +- src/Daemon/BaseDaemon.cpp | 2 +- src/Dictionaries/IPAddressDictionary.cpp | 2 +- src/Functions/FunctionFile.cpp | 4 +-- src/Functions/FunctionsAES.h | 4 +-- src/Functions/FunctionsJSON.cpp | 2 +- src/Functions/FunctionsMultiStringPosition.h | 1 - src/Functions/URL/domain.h | 28 +++++++++---------- src/Functions/URL/netloc.cpp | 24 ++++++++-------- src/Functions/URL/port.cpp | 8 +++--- src/Functions/URL/protocol.h | 10 +++---- src/Functions/URL/topLevelDomain.cpp | 2 +- src/Functions/countMatches.h | 10 +++---- src/Functions/getSetting.cpp | 2 +- src/Functions/isIPAddressContainedIn.cpp | 17 ++++++----- src/Functions/reinterpretAs.cpp | 4 +-- src/Functions/stringToH3.cpp | 2 +- src/IO/WriteHelpers.cpp | 2 +- src/Interpreters/Context.cpp | 8 +++--- src/Interpreters/Context.h | 4 +-- src/Interpreters/PartLog.cpp | 4 +-- src/Interpreters/TransactionsInfoLog.cpp | 2 +- src/Loggers/ExtendedLogChannel.cpp | 4 +-- src/Parsers/ASTPartition.h | 1 - .../Formats/Impl/AvroRowOutputFormat.cpp | 16 +++++------ .../Formats/Impl/CHColumnToArrowColumn.cpp | 4 +-- .../Formats/Impl/MsgPackRowOutputFormat.cpp | 28 +++++++++---------- .../Formats/Impl/ORCBlockOutputFormat.cpp | 6 ++-- .../Formats/Impl/RawBLOBRowOutputFormat.cpp | 4 +-- src/Server/GRPCServer.cpp | 12 ++++---- src/Server/HTTPHandlerRequestFilter.h | 13 ++++----- src/Storages/MergeTree/MergeList.cpp | 2 +- .../System/StorageSystemStackTrace.cpp | 8 +++--- 38 files changed, 126 insertions(+), 131 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 5ccc9041c36..9ed4b48c281 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -132,7 +132,7 @@ public: key_ref = assert_cast(key_column).getDataAt(offset + i); #ifdef __cpp_lib_generic_unordered_lookup - key = static_cast(key_ref); + key = key_ref.toView(); #else key = key_ref.toString(); #endif diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index 4888adb511a..fa52fafa9e2 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -2,7 +2,6 @@ #include #include -#include #include #include @@ -76,7 +75,7 @@ public: static void finalizePerformanceCounters(); /// Returns a non-empty string if the thread is attached to a query - static StringRef getQueryId() + static std::string_view getQueryId() { if (unlikely(!current_thread)) return {}; diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 1183abc0e22..c16d4658ae5 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -104,10 +104,10 @@ String FileSegment::getCallerId() { if (!CurrentThread::isInitialized() || !CurrentThread::get().getQueryContext() - || CurrentThread::getQueryId().size == 0) + || CurrentThread::getQueryId().empty()) return "None:" + toString(getThreadId()); - return CurrentThread::getQueryId().toString() + ":" + toString(getThreadId()); + return std::string(CurrentThread::getQueryId()) + ":" + toString(getThreadId()); } String FileSegment::getOrSetDownloader() diff --git a/src/Common/IFileCache.cpp b/src/Common/IFileCache.cpp index fb120ae5902..8fe434dd740 100644 --- a/src/Common/IFileCache.cpp +++ b/src/Common/IFileCache.cpp @@ -58,7 +58,7 @@ static bool isQueryInitialized() { return CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() - && CurrentThread::getQueryId().size != 0; + && !CurrentThread::getQueryId().empty(); } bool IFileCache::isReadOnly() @@ -77,7 +77,7 @@ IFileCache::QueryContextPtr IFileCache::getCurrentQueryContext(std::lock_guard & /* cache_lock */) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index c80150a8fe8..7c22d3b8335 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -210,7 +210,7 @@ public: return thread_state.load(std::memory_order_relaxed); } - StringRef getQueryId() const + std::string_view getQueryId() const { return query_id; } diff --git a/src/Common/TraceSender.cpp b/src/Common/TraceSender.cpp index 6f0904bd50f..ce8adb98740 100644 --- a/src/Common/TraceSender.cpp +++ b/src/Common/TraceSender.cpp @@ -47,7 +47,7 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int if (CurrentThread::isInitialized()) { - query_id = CurrentThread::getQueryId(); + query_id = StringRef(CurrentThread::getQueryId()); query_id.size = std::min(query_id.size, QUERY_ID_MAX_LEN); thread_id = CurrentThread::get().thread_id; diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 23835df87ea..1d6acc7eac3 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -298,7 +298,7 @@ private: /// It will allow client to see failure messages directly. if (thread_ptr) { - query_id = thread_ptr->getQueryId().toString(); + query_id = std::string(thread_ptr->getQueryId()); if (auto thread_group = thread_ptr->getThreadGroup()) { diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 2a367323205..efb81849126 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -387,7 +387,7 @@ void IPAddressDictionary::loadData() setAttributeValue(attribute, attribute_column[row]); } - const auto [addr, prefix] = parseIPFromString(std::string_view{key_column_ptr->getDataAt(row)}); + const auto [addr, prefix] = parseIPFromString(key_column_ptr->getDataAt(row).toView()); has_ipv6 = has_ipv6 || (addr.family() == Poco::Net::IPAddress::IPv6); size_t row_number = ip_records.size(); diff --git a/src/Functions/FunctionFile.cpp b/src/Functions/FunctionFile.cpp index 01314b52119..c85051e61c0 100644 --- a/src/Functions/FunctionFile.cpp +++ b/src/Functions/FunctionFile.cpp @@ -67,8 +67,8 @@ public: for (size_t row = 0; row < input_rows_count; ++row) { - StringRef filename = column_src->getDataAt(row); - fs::path file_path(filename.data, filename.data + filename.size); + std::string_view filename = column_src->getDataAt(row).toView(); + fs::path file_path(filename.data(), filename.data() + filename.size()); if (file_path.is_relative()) file_path = user_files_absolute_path / file_path; diff --git a/src/Functions/FunctionsAES.h b/src/Functions/FunctionsAES.h index d3c533c804b..b12fcc00014 100644 --- a/src/Functions/FunctionsAES.h +++ b/src/Functions/FunctionsAES.h @@ -182,7 +182,7 @@ private: const auto mode = arguments[0].column->getDataAt(0); - if (mode.size == 0 || !std::string_view(mode).starts_with("aes-")) + if (mode.size == 0 || !mode.toView().starts_with("aes-")) throw Exception("Invalid mode: " + mode.toString(), ErrorCodes::BAD_ARGUMENTS); const auto * evp_cipher = getCipherByName(mode); @@ -453,7 +453,7 @@ private: using namespace OpenSSLDetails; const auto mode = arguments[0].column->getDataAt(0); - if (mode.size == 0 || !std::string_view(mode).starts_with("aes-")) + if (mode.size == 0 || !mode.toView().starts_with("aes-")) throw Exception("Invalid mode: " + mode.toString(), ErrorCodes::BAD_ARGUMENTS); const auto * evp_cipher = getCipherByName(mode); diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 2f10a3c773b..9ab27c2976a 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -251,7 +251,7 @@ private: } case MoveType::Key: { - key = std::string_view{(*arguments[j + 1].column).getDataAt(row)}; + key = (*arguments[j + 1].column).getDataAt(row).toView(); if (!moveToElementByKey(res_element, key)) return false; break; diff --git a/src/Functions/FunctionsMultiStringPosition.h b/src/Functions/FunctionsMultiStringPosition.h index 855b5448b87..0f9f82c595a 100644 --- a/src/Functions/FunctionsMultiStringPosition.h +++ b/src/Functions/FunctionsMultiStringPosition.h @@ -12,7 +12,6 @@ #include #include #include -#include namespace DB diff --git a/src/Functions/URL/domain.h b/src/Functions/URL/domain.h index 18efe969216..1245bb20182 100644 --- a/src/Functions/URL/domain.h +++ b/src/Functions/URL/domain.h @@ -8,22 +8,22 @@ namespace DB { -inline StringRef checkAndReturnHost(const Pos & pos, const Pos & dot_pos, const Pos & start_of_host) +inline std::string_view checkAndReturnHost(const Pos & pos, const Pos & dot_pos, const Pos & start_of_host) { if (!dot_pos || start_of_host >= pos || pos - dot_pos == 1) - return StringRef{}; + return std::string_view{}; auto after_dot = *(dot_pos + 1); if (after_dot == ':' || after_dot == '/' || after_dot == '?' || after_dot == '#') - return StringRef{}; + return std::string_view{}; - return StringRef(start_of_host, pos - start_of_host); + return std::string_view(start_of_host, pos - start_of_host); } /// Extracts host from given url. /// -/// @return empty StringRef if the host is not valid (i.e. it does not have dot, or there no symbol after dot). -inline StringRef getURLHost(const char * data, size_t size) +/// @return empty string view if the host is not valid (i.e. it does not have dot, or there no symbol after dot). +inline std::string_view getURLHost(const char * data, size_t size) { Pos pos = data; Pos end = data + size; @@ -61,7 +61,7 @@ inline StringRef getURLHost(const char * data, size_t size) case ';': case '=': case '&': - return StringRef{}; + return std::string_view{}; default: goto exloop; } @@ -106,7 +106,7 @@ exloop: if ((scheme_end - pos) > 2 && *pos == ':' && *(pos + 1) == '/' && *(pos case ';': case '=': case '&': - return StringRef{}; + return std::string_view{}; } } @@ -120,20 +120,20 @@ struct ExtractDomain static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) { - StringRef host = getURLHost(data, size); + std::string_view host = getURLHost(data, size); - if (host.size == 0) + if (host.empty()) { res_data = data; res_size = 0; } else { - if (without_www && host.size > 4 && !strncmp(host.data, "www.", 4)) - host = { host.data + 4, host.size - 4 }; + if (without_www && host.size() > 4 && !strncmp(host.data(), "www.", 4)) + host = { host.data() + 4, host.size() - 4 }; - res_data = host.data; - res_size = host.size; + res_data = host.data(); + res_size = host.size(); } } }; diff --git a/src/Functions/URL/netloc.cpp b/src/Functions/URL/netloc.cpp index cb28083f4c7..723eea138c3 100644 --- a/src/Functions/URL/netloc.cpp +++ b/src/Functions/URL/netloc.cpp @@ -12,7 +12,7 @@ struct ExtractNetloc /// We use the same as domain function static size_t getReserveLengthForElement() { return 15; } - static inline StringRef getNetworkLocation(const char * data, size_t size) + static std::string_view getNetworkLocation(const char * data, size_t size) { Pos pos = data; Pos end = data + size; @@ -51,7 +51,7 @@ struct ExtractNetloc case ';': case '=': case '&': - return StringRef{}; + return std::string_view(); default: goto exloop; } @@ -76,18 +76,18 @@ struct ExtractNetloc { case '/': if (has_identification) - return StringRef(start_of_host, pos - start_of_host); + return std::string_view(start_of_host, pos - start_of_host); else slash_pos = pos; break; case '?': if (has_identification) - return StringRef(start_of_host, pos - start_of_host); + return std::string_view(start_of_host, pos - start_of_host); else question_mark_pos = pos; break; case '#': - return StringRef(start_of_host, pos - start_of_host); + return std::string_view(start_of_host, pos - start_of_host); case '@': /// foo:bar@example.ru has_identification = true; break; @@ -108,23 +108,23 @@ struct ExtractNetloc case '=': case '&': return pos > start_of_host - ? StringRef(start_of_host, std::min(std::min(pos - 1, question_mark_pos), slash_pos) - start_of_host) - : StringRef{}; + ? std::string_view(start_of_host, std::min(std::min(pos - 1, question_mark_pos), slash_pos) - start_of_host) + : std::string_view(); } } if (has_identification) - return StringRef(start_of_host, pos - start_of_host); + return std::string_view(start_of_host, pos - start_of_host); else - return StringRef(start_of_host, std::min(std::min(pos, question_mark_pos), slash_pos) - start_of_host); + return std::string_view(start_of_host, std::min(std::min(pos, question_mark_pos), slash_pos) - start_of_host); } static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) { - StringRef host = getNetworkLocation(data, size); + std::string_view host = getNetworkLocation(data, size); - res_data = host.data; - res_size = host.size; + res_data = host.data(); + res_size = host.size(); } }; diff --git a/src/Functions/URL/port.cpp b/src/Functions/URL/port.cpp index e2dbc75ab3f..3bbcdcea560 100644 --- a/src/Functions/URL/port.cpp +++ b/src/Functions/URL/port.cpp @@ -94,13 +94,13 @@ private: const char * p = reinterpret_cast(buf.data()) + offset; const char * end = p + size; - StringRef host = getURLHost(p, size); - if (!host.size) + std::string_view host = getURLHost(p, size); + if (host.empty()) return default_port; - if (host.size == size) + if (host.size() == size) return default_port; - p = host.data + host.size; + p = host.data() + host.size(); if (*p++ != ':') return default_port; diff --git a/src/Functions/URL/protocol.h b/src/Functions/URL/protocol.h index b83eaae7e90..74c0bb820b4 100644 --- a/src/Functions/URL/protocol.h +++ b/src/Functions/URL/protocol.h @@ -8,7 +8,7 @@ namespace DB { /// Extracts scheme from given url. -inline StringRef getURLScheme(const char * data, size_t size) +inline std::string_view getURLScheme(const char * data, size_t size) { // scheme = ALPHA *( ALPHA / DIGIT / "+" / "-" / "." ) const char * pos = data; @@ -24,7 +24,7 @@ inline StringRef getURLScheme(const char * data, size_t size) } } - return StringRef(data, pos - data); + return std::string_view(data, pos - data); } return {}; @@ -42,10 +42,10 @@ struct ExtractProtocol res_data = data; res_size = 0; - StringRef scheme = getURLScheme(data, size); - Pos pos = data + scheme.size; + std::string_view scheme = getURLScheme(data, size); + Pos pos = data + scheme.size(); - if (scheme.size == 0 || (data + size) - pos < 4) + if (scheme.empty() || (data + size) - pos < 4) return; if (pos[0] == ':') diff --git a/src/Functions/URL/topLevelDomain.cpp b/src/Functions/URL/topLevelDomain.cpp index 6aa6e689357..ade9439d8ec 100644 --- a/src/Functions/URL/topLevelDomain.cpp +++ b/src/Functions/URL/topLevelDomain.cpp @@ -11,7 +11,7 @@ struct ExtractTopLevelDomain static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) { - StringRef host = getURLHost(data, size); + StringRef host = StringRef(getURLHost(data, size)); res_data = data; res_size = 0; diff --git a/src/Functions/countMatches.h b/src/Functions/countMatches.h index 397515c8bba..2834b8c764b 100644 --- a/src/Functions/countMatches.h +++ b/src/Functions/countMatches.h @@ -79,7 +79,7 @@ public: current_src_offset = src_offsets[i]; Pos end = reinterpret_cast(&src_chars[current_src_offset]) - 1; - StringRef str(pos, end - pos); + std::string_view str(pos, end - pos); vec_res[i] = countMatches(str, re, matches); } @@ -87,7 +87,7 @@ public: } else if (const ColumnConst * col_const_str = checkAndGetColumnConstStringOrFixedString(column_haystack)) { - StringRef str = col_const_str->getDataColumn().getDataAt(0); + std::string_view str = col_const_str->getDataColumn().getDataAt(0).toView(); uint64_t matches_count = countMatches(str, re, matches); return result_type->createColumnConst(input_rows_count, matches_count); } @@ -95,13 +95,13 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "Error in FunctionCountMatches::getReturnTypeImpl()"); } - static uint64_t countMatches(StringRef src, const Regexps::Regexp & re, OptimizedRegularExpression::MatchVec & matches) + static uint64_t countMatches(std::string_view src, const Regexps::Regexp & re, OptimizedRegularExpression::MatchVec & matches) { /// Only one match is required, no need to copy more. static const unsigned matches_limit = 1; - Pos pos = reinterpret_cast(src.data); - Pos end = reinterpret_cast(src.data + src.size); + Pos pos = reinterpret_cast(src.data()); + Pos end = reinterpret_cast(src.data() + src.size()); uint64_t match_count = 0; while (true) diff --git a/src/Functions/getSetting.cpp b/src/Functions/getSetting.cpp index 7e146448dd6..a27f698d54c 100644 --- a/src/Functions/getSetting.cpp +++ b/src/Functions/getSetting.cpp @@ -56,7 +56,7 @@ private: throw Exception{"The argument of function " + String{name} + " should be a constant string with the name of a setting", ErrorCodes::ILLEGAL_COLUMN}; - std::string_view setting_name{column->getDataAt(0)}; + std::string_view setting_name{column->getDataAt(0).toView()}; return getContext()->getSettingsRef().get(setting_name); } }; diff --git a/src/Functions/isIPAddressContainedIn.cpp b/src/Functions/isIPAddressContainedIn.cpp index 6fdc0dfbee8..1ba719cda63 100644 --- a/src/Functions/isIPAddressContainedIn.cpp +++ b/src/Functions/isIPAddressContainedIn.cpp @@ -75,21 +75,20 @@ struct IPAddressCIDR UInt8 prefix; }; -IPAddressCIDR parseIPWithCIDR(StringRef cidr_str) +IPAddressCIDR parseIPWithCIDR(std::string_view cidr_str) { - std::string_view cidr_str_view(cidr_str); - size_t pos_slash = cidr_str_view.find('/'); + size_t pos_slash = cidr_str.find('/'); if (pos_slash == 0) throw DB::Exception("Error parsing IP address with prefix: " + std::string(cidr_str), DB::ErrorCodes::CANNOT_PARSE_TEXT); if (pos_slash == std::string_view::npos) throw DB::Exception("The text does not contain '/': " + std::string(cidr_str), DB::ErrorCodes::CANNOT_PARSE_TEXT); - std::string_view addr_str = cidr_str_view.substr(0, pos_slash); + std::string_view addr_str = cidr_str.substr(0, pos_slash); IPAddressVariant addr(StringRef{addr_str.data(), addr_str.size()}); uint8_t prefix = 0; - auto prefix_str = cidr_str_view.substr(pos_slash+1); + auto prefix_str = cidr_str.substr(pos_slash+1); const auto * prefix_str_end = prefix_str.data() + prefix_str.size(); auto [parse_end, parse_error] = std::from_chars(prefix_str.data(), prefix_str_end, prefix); @@ -190,7 +189,7 @@ namespace DB const auto & col_cidr = col_cidr_const.getDataColumn(); const auto addr = IPAddressVariant(col_addr.getDataAt(0)); - const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(0)); + const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(0).toView()); ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(1); ColumnUInt8::Container & vec_res = col_res->getData(); @@ -212,7 +211,7 @@ namespace DB for (size_t i = 0; i < input_rows_count; ++i) { - const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i)); + const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i).toView()); vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; } return col_res; @@ -223,7 +222,7 @@ namespace DB { const auto & col_cidr = col_cidr_const.getDataColumn(); - const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(0)); + const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(0).toView()); ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); ColumnUInt8::Container & vec_res = col_res->getData(); @@ -244,7 +243,7 @@ namespace DB for (size_t i = 0; i < input_rows_count; ++i) { const auto addr = IPAddressVariant(col_addr.getDataAt(i)); - const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i)); + const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i).toView()); vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; } diff --git a/src/Functions/reinterpretAs.cpp b/src/Functions/reinterpretAs.cpp index ad357c74402..a31b41b55f2 100644 --- a/src/Functions/reinterpretAs.cpp +++ b/src/Functions/reinterpretAs.cpp @@ -288,9 +288,9 @@ private: ColumnFixedString::Offset offset = 0; for (size_t i = 0; i < rows; ++i) { - StringRef data = src.getDataAt(i); + std::string_view data = src.getDataAt(i).toView(); - memcpy(&data_to[offset], data.data, std::min(n, data.size)); + memcpy(&data_to[offset], data.data(), std::min(n, data.size())); offset += n; } } diff --git a/src/Functions/stringToH3.cpp b/src/Functions/stringToH3.cpp index 1cafd1661f1..b5f8585152f 100644 --- a/src/Functions/stringToH3.cpp +++ b/src/Functions/stringToH3.cpp @@ -84,7 +84,7 @@ private: auto h3index = h3index_source.getWhole(); // convert to std::string and get the c_str to have the delimiting \0 at the end. - auto h3index_str = StringRef(h3index.data, h3index.size).toString(); + auto h3index_str = std::string(std::string_view(h3index.data(), h3index.size())); res_data[row_num] = stringToH3(h3index_str.c_str()); if (res_data[row_num] == 0) diff --git a/src/IO/WriteHelpers.cpp b/src/IO/WriteHelpers.cpp index fae3d21513e..cb341e60a8b 100644 --- a/src/IO/WriteHelpers.cpp +++ b/src/IO/WriteHelpers.cpp @@ -68,7 +68,7 @@ void writeException(const Exception & e, WriteBuffer & buf, bool with_stack_trac template static inline void writeProbablyQuotedStringImpl(StringRef s, WriteBuffer & buf, F && write_quoted_string) { - if (isValidIdentifier(std::string_view{s}) + if (isValidIdentifier(s.toView()) /// This are valid identifiers but are problematic if present unquoted in SQL query. && !(s.size == strlen("distinct") && 0 == strncasecmp(s.data, "distinct", strlen("distinct"))) && !(s.size == strlen("all") && 0 == strncasecmp(s.data, "all", strlen("all")))) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fe59215f7d5..a5629b33d22 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1191,7 +1191,7 @@ void Context::setSettings(const Settings & settings_) } -void Context::setSetting(StringRef name, const String & value) +void Context::setSetting(std::string_view name, const String & value) { auto lock = getLock(); if (name == "profile") @@ -1199,14 +1199,14 @@ void Context::setSetting(StringRef name, const String & value) setCurrentProfile(value); return; } - settings.set(std::string_view{name}, value); + settings.set(name, value); if (name == "readonly" || name == "allow_ddl" || name == "allow_introspection_functions") calculateAccessRights(); } -void Context::setSetting(StringRef name, const Field & value) +void Context::setSetting(std::string_view name, const Field & value) { auto lock = getLock(); if (name == "profile") @@ -1214,7 +1214,7 @@ void Context::setSetting(StringRef name, const Field & value) setCurrentProfile(value.safeGet()); return; } - settings.set(std::string_view{name}, value); + settings.set(name, value); if (name == "readonly" || name == "allow_ddl" || name == "allow_introspection_functions") calculateAccessRights(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index e7aba31a1d9..2d498d81326 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -607,8 +607,8 @@ public: void setSettings(const Settings & settings_); /// Set settings by name. - void setSetting(StringRef name, const String & value); - void setSetting(StringRef name, const Field & value); + void setSetting(std::string_view name, const String & value); + void setSetting(std::string_view name, const Field & value); void applySettingChange(const SettingChange & change); void applySettingsChanges(const SettingsChanges & changes); diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index c3152f31808..d12eca407d9 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -201,8 +201,8 @@ bool PartLog::addNewParts( { PartLogElement elem; - if (query_id.data && query_id.size) - elem.query_id.insert(0, query_id.data, query_id.size); + if (!query_id.empty()) + elem.query_id.insert(0, query_id.data(), query_id.size()); elem.event_type = PartLogElement::NEW_PART; //-V1048 diff --git a/src/Interpreters/TransactionsInfoLog.cpp b/src/Interpreters/TransactionsInfoLog.cpp index 0498ee00e9e..b62cd4672d8 100644 --- a/src/Interpreters/TransactionsInfoLog.cpp +++ b/src/Interpreters/TransactionsInfoLog.cpp @@ -55,7 +55,7 @@ void TransactionsInfoLogElement::fillCommonFields(const TransactionInfoContext * event_time = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); thread_id = getThreadId(); - query_id = CurrentThread::getQueryId().toString(); + query_id = std::string(CurrentThread::getQueryId()); if (!context) return; diff --git a/src/Loggers/ExtendedLogChannel.cpp b/src/Loggers/ExtendedLogChannel.cpp index fa414216aa5..116892b9030 100644 --- a/src/Loggers/ExtendedLogChannel.cpp +++ b/src/Loggers/ExtendedLogChannel.cpp @@ -28,8 +28,8 @@ ExtendedLogMessage ExtendedLogMessage::getFrom(const Poco::Message & base) if (current_thread) { auto query_id_ref = CurrentThread::getQueryId(); - if (query_id_ref.size) - msg_ext.query_id.assign(query_id_ref.data, query_id_ref.size); + if (!query_id_ref.empty()) + msg_ext.query_id.assign(query_id_ref.data(), query_id_ref.size()); } msg_ext.thread_id = getThreadId(); diff --git a/src/Parsers/ASTPartition.h b/src/Parsers/ASTPartition.h index 1bd16d55795..fbe05ce3a8e 100644 --- a/src/Parsers/ASTPartition.h +++ b/src/Parsers/ASTPartition.h @@ -1,7 +1,6 @@ #pragma once #include -#include namespace DB diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index ddee20c187b..dce5a928690 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -179,15 +179,15 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF if (traits->isStringAsString(column_name)) return {avro::StringSchema(), [](const IColumn & column, size_t row_num, avro::Encoder & encoder) { - const StringRef & s = assert_cast(column).getDataAt(row_num); - encoder.encodeString(s.toString()); + const std::string_ref & s = assert_cast(column).getDataAt(row_num).toView(); + encoder.encodeString(std::string(s)); } }; else return {avro::BytesSchema(), [](const IColumn & column, size_t row_num, avro::Encoder & encoder) { - const StringRef & s = assert_cast(column).getDataAt(row_num); - encoder.encodeBytes(reinterpret_cast(s.data), s.size); + const std::string_view & s = assert_cast(column).getDataAt(row_num).toString(); + encoder.encodeBytes(reinterpret_cast(s.data()), s.size()); } }; case TypeIndex::FixedString: @@ -196,8 +196,8 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF auto schema = avro::FixedSchema(size, "fixed_" + toString(type_name_increment)); return {schema, [](const IColumn & column, size_t row_num, avro::Encoder & encoder) { - const StringRef & s = assert_cast(column).getDataAt(row_num); - encoder.encodeFixed(reinterpret_cast(s.data), s.size); + const std::string_view & s = assert_cast(column).getDataAt(row_num).toView(); + encoder.encodeFixed(reinterpret_cast(s.data()), s.size()); }}; } case TypeIndex::Enum8: @@ -343,8 +343,8 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF auto keys_serializer = [](const IColumn & column, size_t row_num, avro::Encoder & encoder) { - const StringRef & s = column.getDataAt(row_num); - encoder.encodeString(s.toString()); + const StringRef & s = column.getDataAt(row_num).toView(); + encoder.encodeString(std::string(s)); }; const auto & values_type = map_type.getValueType(); diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index e3cc896466b..34c120c3f52 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -365,8 +365,8 @@ namespace DB } else { - StringRef string_ref = internal_column.getDataAt(string_i); - status = builder.Append(string_ref.data, string_ref.size); + std::string_view string_ref = internal_column.getDataAt(string_i).toView(); + status = builder.Append(string_ref.data(), string_ref.size()); } checkStatus(status, write_column->getName(), format_name); } diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp index e53aafb4e56..91183ebf633 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp @@ -98,16 +98,16 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr } case TypeIndex::String: { - const StringRef & string = assert_cast(column).getDataAt(row_num); - packer.pack_bin(string.size); - packer.pack_bin_body(string.data, string.size); + const std::string_view & string = assert_cast(column).getDataAt(row_num).toView(); + packer.pack_bin(string.size()); + packer.pack_bin_body(string.data(), string.size()); return; } case TypeIndex::FixedString: { - const StringRef & string = assert_cast(column).getDataAt(row_num); - packer.pack_bin(string.size); - packer.pack_bin_body(string.data, string.size); + const std::string_view & string = assert_cast(column).getDataAt(row_num).toView(); + packer.pack_bin(string.size()); + packer.pack_bin_body(string.data(), string.size()); return; } case TypeIndex::Array: @@ -178,18 +178,18 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr { WriteBufferFromOwnString buf; writeBinary(uuid_column.getElement(row_num), buf); - StringRef uuid_bin = buf.stringRef(); - packer.pack_bin(uuid_bin.size); - packer.pack_bin_body(uuid_bin.data, uuid_bin.size); + std::string_view uuid_bin = buf.stringRef().toView(); + packer.pack_bin(uuid_bin.size()); + packer.pack_bin_body(uuid_bin.data(), uuid_bin.size()); return; } case FormatSettings::MsgPackUUIDRepresentation::STR: { WriteBufferFromOwnString buf; writeText(uuid_column.getElement(row_num), buf); - StringRef uuid_text = buf.stringRef(); - packer.pack_str(uuid_text.size); - packer.pack_bin_body(uuid_text.data, uuid_text.size); + std::string_view uuid_text = buf.stringRef().toView(); + packer.pack_str(uuid_text.size()); + packer.pack_bin_body(uuid_text.data(), uuid_text.size()); return; } case FormatSettings::MsgPackUUIDRepresentation::EXT: @@ -198,9 +198,9 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr UUID value = uuid_column.getElement(row_num); writeBinaryBigEndian(value.toUnderType().items[0], buf); writeBinaryBigEndian(value.toUnderType().items[1], buf); - StringRef uuid_ext = buf.stringRef(); + std::string_view uuid_ext = buf.stringRef().toView(); packer.pack_ext(sizeof(UUID), int8_t(MsgPackExtensionTypes::UUIDType)); - packer.pack_ext_body(uuid_ext.data, uuid_ext.size); + packer.pack_ext_body(uuid_ext.data(), uuid_ext.size()); return; } } diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 5e979c3d35a..1de2acbb3b9 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -225,9 +225,9 @@ void ORCBlockOutputFormat::writeStrings( } string_orc_column.notNull[i] = 1; - const StringRef & string = string_column.getDataAt(i); - string_orc_column.data[i] = const_cast(string.data); - string_orc_column.length[i] = string.size; + const std::string_view & string = string_column.getDataAt(i).toView(); + string_orc_column.data[i] = const_cast(string.data()); + string_orc_column.length[i] = string.size(); } string_orc_column.numElements = string_column.size(); } diff --git a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp index 9a38a11f6f0..1d0e987f0c4 100644 --- a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp @@ -17,8 +17,8 @@ RawBLOBRowOutputFormat::RawBLOBRowOutputFormat( void RawBLOBRowOutputFormat::writeField(const IColumn & column, const ISerialization &, size_t row_num) { - StringRef value = column.getDataAt(row_num); - out.write(value.data, value.size); + std::string_view value = column.getDataAt(row_num).toView(); + out.write(value.data(), value.size()); } diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 4178d0d62da..573e43e9d7a 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1573,14 +1573,14 @@ namespace auto & log_entry = *result.add_logs(); log_entry.set_time(column_time.getElement(row)); log_entry.set_time_microseconds(column_time_microseconds.getElement(row)); - StringRef query_id = column_query_id.getDataAt(row); - log_entry.set_query_id(query_id.data, query_id.size); + std::string_view query_id = column_query_id.getDataAt(row).toView(); + log_entry.set_query_id(query_id.data(), query_id.size()); log_entry.set_thread_id(column_thread_id.getElement(row)); log_entry.set_level(static_cast<::clickhouse::grpc::LogsLevel>(column_level.getElement(row))); - StringRef source = column_source.getDataAt(row); - log_entry.set_source(source.data, source.size); - StringRef text = column_text.getDataAt(row); - log_entry.set_text(text.data, text.size); + std::string_view source = column_source.getDataAt(row).toView(); + log_entry.set_source(source.data(), source.size()); + std::string_view text = column_text.getDataAt(row).toView(); + log_entry.set_text(text.data(), text.size()); } } } diff --git a/src/Server/HTTPHandlerRequestFilter.h b/src/Server/HTTPHandlerRequestFilter.h index d0156266fe5..b04472fbea5 100644 --- a/src/Server/HTTPHandlerRequestFilter.h +++ b/src/Server/HTTPHandlerRequestFilter.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include @@ -23,16 +22,16 @@ namespace ErrorCodes using CompiledRegexPtr = std::shared_ptr; -static inline bool checkRegexExpression(StringRef match_str, const CompiledRegexPtr & compiled_regex) +static inline bool checkRegexExpression(std::string_view match_str, const CompiledRegexPtr & compiled_regex) { int num_captures = compiled_regex->NumberOfCapturingGroups() + 1; re2::StringPiece matches[num_captures]; - re2::StringPiece match_input(match_str.data, match_str.size); - return compiled_regex->Match(match_input, 0, match_str.size, re2::RE2::Anchor::ANCHOR_BOTH, matches, num_captures); + re2::StringPiece match_input(match_str.data(), match_str.size()); + return compiled_regex->Match(match_input, 0, match_str.size(), re2::RE2::Anchor::ANCHOR_BOTH, matches, num_captures); } -static inline bool checkExpression(StringRef match_str, const std::pair & expression) +static inline bool checkExpression(std::string_view match_str, const std::pair & expression) { if (expression.second) return checkRegexExpression(match_str, expression.second); @@ -71,7 +70,7 @@ static inline auto urlFilter(Poco::Util::AbstractConfiguration & config, const s const auto & uri = request.getURI(); const auto & end = find_first_symbols<'?'>(uri.data(), uri.data() + uri.size()); - return checkExpression(StringRef(uri.data(), end - uri.data()), expression); + return checkExpression(std::string_view(uri.data(), end - uri.data()), expression); }; } @@ -93,7 +92,7 @@ static inline auto headersFilter(Poco::Util::AbstractConfiguration & config, con for (const auto & [header_name, header_expression] : headers_expression) { const auto & header_value = request.get(header_name, ""); - if (!checkExpression(StringRef(header_value.data(), header_value.size()), header_expression)) + if (!checkExpression(std::string_view(header_value.data(), header_value.size()), header_expression)) return false; } diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index d866345defe..13237371e9a 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -27,7 +27,7 @@ MemoryTrackerThreadSwitcher::MemoryTrackerThreadSwitcher(MergeListEntry & merge_ prev_untracked_memory = current_thread->untracked_memory; current_thread->untracked_memory = merge_list_entry->untracked_memory; - prev_query_id = current_thread->getQueryId().toString(); + prev_query_id = std::string(current_thread->getQueryId()); current_thread->setQueryId(merge_list_entry->query_id); } diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index cdd04964f55..549ce193137 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -90,10 +90,10 @@ namespace const ucontext_t signal_context = *reinterpret_cast(context); stack_trace = StackTrace(signal_context); - StringRef query_id = CurrentThread::getQueryId(); - query_id_size = std::min(query_id.size, max_query_id_size); - if (query_id.data && query_id.size) - memcpy(query_id_data, query_id.data, query_id_size); + std::string_view query_id = CurrentThread::getQueryId(); + query_id_size = std::min(query_id.size(), max_query_id_size); + if (!query_id.empty()) + memcpy(query_id_data, query_id.data(), query_id_size); /// This is unneeded (because we synchronize through pipe) but makes TSan happy. data_ready_num.store(notification_num, std::memory_order_release); From 228b9e7ec4af15fa92f2301a8d4f7e67d34452d1 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 17 Jul 2022 20:09:18 +0200 Subject: [PATCH 318/659] Addressed review comments - Updated exception message in ForkWriteBuffer - Added test case to tests/queries/0_stateless/02346_into_outfile_and_stdout.sh for calling nextImpl more than once --- src/IO/ForkWriteBuffer.cpp | 2 +- .../0_stateless/02346_into_outfile_and_stdout.reference | 5 +++++ .../queries/0_stateless/02346_into_outfile_and_stdout.sh | 8 +++++--- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/IO/ForkWriteBuffer.cpp b/src/IO/ForkWriteBuffer.cpp index dd910f7fdf4..876a924a72b 100644 --- a/src/IO/ForkWriteBuffer.cpp +++ b/src/IO/ForkWriteBuffer.cpp @@ -14,7 +14,7 @@ ForkWriteBuffer::ForkWriteBuffer(WriteBufferPtrs && sources_) { if (sources.empty()) { - throw Exception("ForkWriteBuffer required WriteBuffer is not provided", ErrorCodes::CANNOT_CREATE_IO_BUFFER); + throw Exception("Expected non-zero number of buffers for `ForkWriteBuffer`", ErrorCodes::CANNOT_CREATE_IO_BUFFER); } set(sources.front()->buffer().begin(), sources.front()->buffer().size()); } diff --git a/tests/queries/0_stateless/02346_into_outfile_and_stdout.reference b/tests/queries/0_stateless/02346_into_outfile_and_stdout.reference index d14e7634f24..6032d8f4b79 100644 --- a/tests/queries/0_stateless/02346_into_outfile_and_stdout.reference +++ b/tests/queries/0_stateless/02346_into_outfile_and_stdout.reference @@ -14,3 +14,8 @@ performing test: bad_query_misplaced_compression 1 performing test: bad_query_misplaced_format 1 +performing test: union_all +1 2 +3 4 +1 2 +3 4 diff --git a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh index 3879249699f..021dc9125d4 100755 --- a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh +++ b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh @@ -10,9 +10,9 @@ function perform() local query=$2 echo "performing test: ${test_id}" - ${CLICKHOUSE_CLIENT} --query "${query}" + ${CLICKHOUSE_CLIENT} --query "${query}" | sort --numeric-sort if [ "$?" -eq 0 ]; then - cat "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_${test_id}.out" + cat "${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_${test_id}.out" | sort --numeric-sort else echo "query failed" fi @@ -72,4 +72,6 @@ performCompression "compression" "SELECT * FROM (SELECT 'Hello, World! From clic performBadQuery "bad_query_misplaced_compression" "SELECT 1, 2, 3 INTO OUTFILE 'test.gz' COMPRESSION 'GZ' AND STDOUT'" "SYNTAX_ERROR" -performBadQuery "bad_query_misplaced_format" "SELECT 1, 2, 3 INTO OUTFILE 'test.gz' FORMAT TabSeparated AND STDOUT'" "SYNTAX_ERROR" \ No newline at end of file +performBadQuery "bad_query_misplaced_format" "SELECT 1, 2, 3 INTO OUTFILE 'test.gz' FORMAT TabSeparated AND STDOUT'" "SYNTAX_ERROR" + +perform "union_all" "SELECT 3, 4 UNION ALL SELECT 1, 2 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_union_all.out' AND STDOUT" \ No newline at end of file From 01b40d5a454bba9105a269af972b53bd04d3fab1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 17 Jul 2022 18:11:37 +0000 Subject: [PATCH 319/659] Split 00233_position_function_family to avoid test timeouts + remove the five most expensive queries --- .../00233_position_function_family.reference | 16522 ---------------- .../00233_position_function_family.sql | 1833 +- ...2364_multiSearch_function_family.reference | 16516 +++++++++++++++ .../02364_multiSearch_function_family.sql | 1047 + 4 files changed, 17564 insertions(+), 18354 deletions(-) create mode 100644 tests/queries/0_stateless/02364_multiSearch_function_family.reference create mode 100644 tests/queries/0_stateless/02364_multiSearch_function_family.sql diff --git a/tests/queries/0_stateless/00233_position_function_family.reference b/tests/queries/0_stateless/00233_position_function_family.reference index 2ccb95615c0..bd970f335e1 100644 --- a/tests/queries/0_stateless/00233_position_function_family.reference +++ b/tests/queries/0_stateless/00233_position_function_family.reference @@ -7306,16525 +7306,3 @@ 1 1 1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/00233_position_function_family.sql b/tests/queries/0_stateless/00233_position_function_family.sql index d75d09b4716..dd7394bc39a 100644 --- a/tests/queries/0_stateless/00233_position_function_family.sql +++ b/tests/queries/0_stateless/00233_position_function_family.sql @@ -1,4 +1,5 @@ SET send_logs_level = 'fatal'; + select 1 = position('', ''); select 1 = position('abc', ''); select 0 = position('', 'abc'); @@ -482,1835 +483,3 @@ select 1 = position('abc', materialize('')) from system.numbers limit 1000; select 1 = position('abab', materialize('ab')); select 1 = position('abababababababababababab', materialize('abab')); select 1 = position('abababababababababababab', materialize('abababababababababa')); - -select 0 = multiSearchAny('\0', CAST([], 'Array(String)')); -select 0 = multiSearchAnyCaseInsensitive('\0', CAST([], 'Array(String)')); -select 0 = multiSearchAnyCaseInsensitiveUTF8('\0', CAST([], 'Array(String)')); -select 0 = multiSearchAnyUTF8('\0', CAST([], 'Array(String)')); -select 0 = multiSearchFirstIndex('\0', CAST([], 'Array(String)')); -select 0 = multiSearchFirstIndexCaseInsensitive('\0', CAST([], 'Array(String)')); -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8('\0', CAST([], 'Array(String)')); -select 0 = multiSearchFirstIndexUTF8('\0', CAST([], 'Array(String)')); -select 0 = multiSearchFirstPosition('\0', CAST([], 'Array(String)')); -select 0 = multiSearchFirstPositionCaseInsensitive('\0', CAST([], 'Array(String)')); -select 0 = multiSearchFirstPositionCaseInsensitiveUTF8('\0', CAST([], 'Array(String)')); -select 0 = multiSearchFirstPositionUTF8('\0', CAST([], 'Array(String)')); -select [] = multiSearchAllPositions('\0', CAST([], 'Array(String)')); -select [] = multiSearchAllPositionsCaseInsensitive('\0', CAST([], 'Array(String)')); -select [] = multiSearchAllPositionsCaseInsensitiveUTF8('\0', CAST([], 'Array(String)')); -select [] = multiSearchAllPositionsUTF8('\0', CAST([], 'Array(String)')); - -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['b']); -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bc']); -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcd']); -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcde']); -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdef']); -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdefg']); -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdefgh']); - -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdefgh']); -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdefg']); -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdef']); -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcde']); -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcd']); -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abc']); -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['ab']); -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['a']); - -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['c']); -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cd']); -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cde']); -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdef']); -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdefg']); -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdefgh']); - -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['defgh']); -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['defg']); -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['def']); -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['de']); -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['d']); - -select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['e']); -select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['ef']); -select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['efg']); -select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['efgh']); - -select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['fgh']); -select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['fg']); -select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['f']); - -select [7] = multiSearchAllPositions(materialize('abcdefgh'), ['g']); -select [7] = multiSearchAllPositions(materialize('abcdefgh'), ['gh']); - -select [8] = multiSearchAllPositions(materialize('abcdefgh'), ['h']); - -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['b']) from system.numbers limit 10; -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bc']) from system.numbers limit 10; -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcd']) from system.numbers limit 10; -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcde']) from system.numbers limit 10; -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdef']) from system.numbers limit 10; -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdefg']) from system.numbers limit 10; -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdefgh']) from system.numbers limit 10; - -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdefgh']) from system.numbers limit 10; -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdefg']) from system.numbers limit 10; -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdef']) from system.numbers limit 10; -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcde']) from system.numbers limit 10; -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcd']) from system.numbers limit 10; -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abc']) from system.numbers limit 10; -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['ab']) from system.numbers limit 10; -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['a']) from system.numbers limit 10; - -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['c']) from system.numbers limit 10; -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cd']) from system.numbers limit 10; -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cde']) from system.numbers limit 10; -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdef']) from system.numbers limit 10; -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdefg']) from system.numbers limit 10; -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdefgh']) from system.numbers limit 10; - -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['defgh']) from system.numbers limit 10; -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['defg']) from system.numbers limit 10; -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['def']) from system.numbers limit 10; -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['de']) from system.numbers limit 10; -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['d']) from system.numbers limit 10; - -select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['e']) from system.numbers limit 10; -select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['ef']) from system.numbers limit 10; -select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['efg']) from system.numbers limit 10; -select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['efgh']) from system.numbers limit 10; - -select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['fgh']) from system.numbers limit 10; -select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['fg']) from system.numbers limit 10; -select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['f']) from system.numbers limit 10; - -select [7] = multiSearchAllPositions(materialize('abcdefgh'), ['g']) from system.numbers limit 10; -select [7] = multiSearchAllPositions(materialize('abcdefgh'), ['gh']) from system.numbers limit 10; - -select [8] = multiSearchAllPositions(materialize('abcdefgh'), ['h']) from system.numbers limit 10; - -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['b']) from system.numbers limit 129; -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bc']) from system.numbers limit 129; -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcd']) from system.numbers limit 10; -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcde']) from system.numbers limit 129; -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdef']) from system.numbers limit 129; -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdefg']) from system.numbers limit 129; -select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdefgh']) from system.numbers limit 129; - -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdefgh']) from system.numbers limit 129; -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdefg']) from system.numbers limit 129; -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdef']) from system.numbers limit 129; -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcde']) from system.numbers limit 129; -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcd']) from system.numbers limit 129; -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abc']) from system.numbers limit 129; -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['ab']) from system.numbers limit 129; -select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['a']) from system.numbers limit 129; - -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['c']) from system.numbers limit 129; -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cd']) from system.numbers limit 129; -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cde']) from system.numbers limit 129; -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdef']) from system.numbers limit 129; -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdefg']) from system.numbers limit 129; -select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdefgh']) from system.numbers limit 129; - -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['defgh']) from system.numbers limit 129; -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['defg']) from system.numbers limit 129; -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['def']) from system.numbers limit 129; -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['de']) from system.numbers limit 129; -select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['d']) from system.numbers limit 129; - -select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['e']) from system.numbers limit 129; -select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['ef']) from system.numbers limit 129; -select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['efg']) from system.numbers limit 129; -select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['efgh']) from system.numbers limit 129; - -select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['fgh']) from system.numbers limit 129; -select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['fg']) from system.numbers limit 129; -select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['f']) from system.numbers limit 129; - -select [7] = multiSearchAllPositions(materialize('abcdefgh'), ['g']) from system.numbers limit 129; -select [7] = multiSearchAllPositions(materialize('abcdefgh'), ['gh']) from system.numbers limit 129; - -select [8] = multiSearchAllPositions(materialize('abcdefgh'), ['h']) from system.numbers limit 129; - -select [2] = multiSearchAllPositions(materialize('abc'), ['b']); -select [2] = multiSearchAllPositions(materialize('abc'), ['bc']); -select [0] = multiSearchAllPositions(materialize('abc'), ['bcde']); -select [0] = multiSearchAllPositions(materialize('abc'), ['bcdef']); -select [0] = multiSearchAllPositions(materialize('abc'), ['bcdefg']); -select [0] = multiSearchAllPositions(materialize('abc'), ['bcdefgh']); - -select [0] = multiSearchAllPositions(materialize('abc'), ['abcdefg']); -select [0] = multiSearchAllPositions(materialize('abc'), ['abcdef']); -select [0] = multiSearchAllPositions(materialize('abc'), ['abcde']); -select [0] = multiSearchAllPositions(materialize('abc'), ['abcd']); -select [1] = multiSearchAllPositions(materialize('abc'), ['abc']); -select [1] = multiSearchAllPositions(materialize('abc'), ['ab']); -select [1] = multiSearchAllPositions(materialize('abc'), ['a']); - -select [3] = multiSearchAllPositions(materialize('abcd'), ['c']); -select [3] = multiSearchAllPositions(materialize('abcd'), ['cd']); -select [0] = multiSearchAllPositions(materialize('abcd'), ['cde']); -select [0] = multiSearchAllPositions(materialize('abcd'), ['cdef']); -select [0] = multiSearchAllPositions(materialize('abcd'), ['cdefg']); -select [0] = multiSearchAllPositions(materialize('abcd'), ['cdefgh']); - -select [0] = multiSearchAllPositions(materialize('abc'), ['defgh']); -select [0] = multiSearchAllPositions(materialize('abc'), ['defg']); -select [0] = multiSearchAllPositions(materialize('abc'), ['def']); -select [0] = multiSearchAllPositions(materialize('abc'), ['de']); -select [0] = multiSearchAllPositions(materialize('abc'), ['d']); - - -select [2] = multiSearchAllPositions(materialize('abc'), ['b']) from system.numbers limit 10; -select [2] = multiSearchAllPositions(materialize('abc'), ['bc']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abc'), ['bcde']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abc'), ['bcdef']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abc'), ['bcdefg']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abc'), ['bcdefgh']) from system.numbers limit 10; - - -select [0] = multiSearchAllPositions(materialize('abc'), ['abcdefg']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abc'), ['abcdef']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abc'), ['abcde']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abc'), ['abcd']) from system.numbers limit 10; -select [1] = multiSearchAllPositions(materialize('abc'), ['abc']) from system.numbers limit 10; -select [1] = multiSearchAllPositions(materialize('abc'), ['ab']) from system.numbers limit 10; -select [1] = multiSearchAllPositions(materialize('abc'), ['a']) from system.numbers limit 10; - -select [3] = multiSearchAllPositions(materialize('abcd'), ['c']) from system.numbers limit 10; -select [3] = multiSearchAllPositions(materialize('abcd'), ['cd']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abcd'), ['cde']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abcd'), ['cdef']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abcd'), ['cdefg']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abcd'), ['cdefgh']) from system.numbers limit 10; - -select [0] = multiSearchAllPositions(materialize('abc'), ['defgh']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abc'), ['defg']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abc'), ['def']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abc'), ['de']) from system.numbers limit 10; -select [0] = multiSearchAllPositions(materialize('abc'), ['d']) from system.numbers limit 10; - -select [1] = multiSearchAllPositions(materialize('abc'), ['']); -select [1] = multiSearchAllPositions(materialize('abc'), ['']) from system.numbers limit 10; -select [1] = multiSearchAllPositions(materialize('abc'), ['']) from system.numbers limit 100; -select [1] = multiSearchAllPositions(materialize('abc'), ['']) from system.numbers limit 1000; - -select [1] = multiSearchAllPositions(materialize('abab'), ['ab']); -select [1] = multiSearchAllPositions(materialize('abababababababababababab'), ['abab']); -select [1] = multiSearchAllPositions(materialize('abababababababababababab'), ['abababababababababa']); - -select [1] = multiSearchAllPositions(materialize('abc'), materialize([''])); -select [1] = multiSearchAllPositions(materialize('abc'), materialize([''])) from system.numbers limit 10; -select [1] = multiSearchAllPositions(materialize('abab'), materialize(['ab'])); -select [2] = multiSearchAllPositions(materialize('abab'), materialize(['ba'])); -select [1] = multiSearchAllPositionsCaseInsensitive(materialize('aBaB'), materialize(['abab'])); -select [3] = multiSearchAllPositionsUTF8(materialize('ab€ab'), materialize(['€'])); -select [3] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ab€AB'), materialize(['€ab'])); - -select 1 = multiSearchAny(materialize('abcdefgh'), ['b']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['bc']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcd']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcde']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdef']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdefg']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdefgh']); - -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdefgh']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdefg']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdef']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcde']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcd']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['abc']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['ab']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['a']); - -select 1 = multiSearchAny(materialize('abcdefgh'), ['c']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['cd']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['cde']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['cdef']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['cdefg']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['cdefgh']); - -select 1 = multiSearchAny(materialize('abcdefgh'), ['defgh']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['defg']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['def']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['de']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['d']); - -select 1 = multiSearchAny(materialize('abcdefgh'), ['e']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['ef']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['efg']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['efgh']); - -select 1 = multiSearchAny(materialize('abcdefgh'), ['fgh']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['fg']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['f']); - -select 1 = multiSearchAny(materialize('abcdefgh'), ['g']); -select 1 = multiSearchAny(materialize('abcdefgh'), ['gh']); - -select 1 = multiSearchAny(materialize('abcdefgh'), ['h']); - -select 1 = multiSearchAny(materialize('abcdefgh'), ['b']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['bc']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcd']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcde']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdef']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdefg']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdefgh']) from system.numbers limit 10; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdefgh']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdefg']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdef']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcde']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcd']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['abc']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['ab']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['a']) from system.numbers limit 10; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['c']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['cd']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['cde']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['cdef']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['cdefg']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['cdefgh']) from system.numbers limit 10; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['defgh']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['defg']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['def']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['de']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['d']) from system.numbers limit 10; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['e']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['ef']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['efg']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['efgh']) from system.numbers limit 10; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['fgh']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['fg']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['f']) from system.numbers limit 10; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['g']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['gh']) from system.numbers limit 10; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['h']) from system.numbers limit 10; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['b']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['bc']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcd']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcde']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdef']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdefg']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdefgh']) from system.numbers limit 129; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdefgh']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdefg']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdef']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcde']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['abcd']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['abc']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['ab']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['a']) from system.numbers limit 129; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['c']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['cd']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['cde']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['cdef']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['cdefg']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['cdefgh']) from system.numbers limit 129; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['defgh']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['defg']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['def']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['de']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['d']) from system.numbers limit 129; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['e']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['ef']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['efg']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['efgh']) from system.numbers limit 129; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['fgh']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['fg']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['f']) from system.numbers limit 129; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['g']) from system.numbers limit 129; -select 1 = multiSearchAny(materialize('abcdefgh'), ['gh']) from system.numbers limit 129; - -select 1 = multiSearchAny(materialize('abcdefgh'), ['h']) from system.numbers limit 129; - -select 1 = multiSearchAny(materialize('abc'), ['b']); -select 1 = multiSearchAny(materialize('abc'), ['bc']); -select 0 = multiSearchAny(materialize('abc'), ['bcde']); -select 0 = multiSearchAny(materialize('abc'), ['bcdef']); -select 0 = multiSearchAny(materialize('abc'), ['bcdefg']); -select 0 = multiSearchAny(materialize('abc'), ['bcdefgh']); - -select 0 = multiSearchAny(materialize('abc'), ['abcdefg']); -select 0 = multiSearchAny(materialize('abc'), ['abcdef']); -select 0 = multiSearchAny(materialize('abc'), ['abcde']); -select 0 = multiSearchAny(materialize('abc'), ['abcd']); -select 1 = multiSearchAny(materialize('abc'), ['abc']); -select 1 = multiSearchAny(materialize('abc'), ['ab']); -select 1 = multiSearchAny(materialize('abc'), ['a']); - -select 1 = multiSearchAny(materialize('abcd'), ['c']); -select 1 = multiSearchAny(materialize('abcd'), ['cd']); -select 0 = multiSearchAny(materialize('abcd'), ['cde']); -select 0 = multiSearchAny(materialize('abcd'), ['cdef']); -select 0 = multiSearchAny(materialize('abcd'), ['cdefg']); -select 0 = multiSearchAny(materialize('abcd'), ['cdefgh']); - -select 0 = multiSearchAny(materialize('abc'), ['defgh']); -select 0 = multiSearchAny(materialize('abc'), ['defg']); -select 0 = multiSearchAny(materialize('abc'), ['def']); -select 0 = multiSearchAny(materialize('abc'), ['de']); -select 0 = multiSearchAny(materialize('abc'), ['d']); - - -select 1 = multiSearchAny(materialize('abc'), ['b']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abc'), ['bc']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abc'), ['bcde']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abc'), ['bcdef']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abc'), ['bcdefg']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abc'), ['bcdefgh']) from system.numbers limit 10; - - -select 0 = multiSearchAny(materialize('abc'), ['abcdefg']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abc'), ['abcdef']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abc'), ['abcde']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abc'), ['abcd']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abc'), ['abc']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abc'), ['ab']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abc'), ['a']) from system.numbers limit 10; - -select 1 = multiSearchAny(materialize('abcd'), ['c']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abcd'), ['cd']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abcd'), ['cde']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abcd'), ['cdef']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abcd'), ['cdefg']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abcd'), ['cdefgh']) from system.numbers limit 10; - -select 0 = multiSearchAny(materialize('abc'), ['defgh']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abc'), ['defg']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abc'), ['def']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abc'), ['de']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('abc'), ['d']) from system.numbers limit 10; - -select 1 = multiSearchAny(materialize('abc'), ['']); -select 1 = multiSearchAny(materialize('abc'), ['']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('abc'), ['']) from system.numbers limit 100; -select 1 = multiSearchAny(materialize('abc'), ['']) from system.numbers limit 1000; - -select 1 = multiSearchAny(materialize('abab'), ['ab']); -select 1 = multiSearchAny(materialize('abababababababababababab'), ['abab']); -select 1 = multiSearchAny(materialize('abababababababababababab'), ['abababababababababa']); - --- select 'some random tests'; - -select [4, 1, 1, 2, 6, 1, 1, 0, 4, 1, 14, 0, 10, 0, 16, 6] = multiSearchAllPositions(materialize('jmdqwjbrxlbatqeixknricfk'), ['qwjbrxlba', 'jmd', '', 'mdqwjbrxlbatqe', 'jbrxlbatqeixknric', 'jmdqwjbrxlbatqeixknri', '', 'fdtmnwtts', 'qwjbrxlba', '', 'qeixknricfk', 'hzjjgrnoilfkvzxaemzhf', 'lb', 'kamz', 'ixknr', 'jbrxlbatq']) from system.numbers limit 10; -select [0, 0, 0, 2, 3, 0, 1, 0, 5, 0, 0, 0, 11, 10, 6, 7] = multiSearchAllPositions(materialize('coxcctuehmzkbrsmodfvx'), ['bkhnp', 'nlypjvriuk', 'rkslxwfqjjivcwdexrdtvjdtvuu', 'oxcctuehm', 'xcctuehmzkbrsm', 'kfrieuocovykjmkwxbdlkgwctwvcuh', 'coxc', 'lbwvetgxyndxjqqwthtkgasbafii', 'ctuehmzkbrsmodfvx', 'obzldxjldxowk', 'ngfikgigeyll', 'wdaejjukowgvzijnw', 'zkbr', 'mzkb', 'tuehm', 'ue']) from system.numbers limit 10; -select [1, 1, 0, 0, 0, 1, 1, 1, 4, 0, 6, 6, 0, 10, 1, 5] = multiSearchAllPositions(materialize('mpswgtljbbrmivkcglamemayfn'), ['', 'm', 'saejhpnfgfq', 'rzanrkdssmmkanqjpfi', 'oputeneprgoowg', 'mp', '', '', 'wgtljbbrmivkcglamemay', 'cbpthtrgrmgfypizi', 'tl', 'tlj', 'xuhs', 'brmivkcglamemayfn', '', 'gtljb']) from system.numbers limit 10; -select [1, 0, 0, 8, 6, 0, 7, 1, 3, 0, 0, 0, 0, 12] = multiSearchAllPositions(materialize('arbphzbbecypbzsqsljurtddve'), ['arbphzb', 'mnrboimjfijnti', 'cikcrd', 'becypbz', 'z', 'uocmqgnczhdcrvtqrnaxdxjjlhakoszuwc', 'bbe', '', 'bp', 'yhltnexlpdijkdzt', 'jkwjmrckvgmccmmrolqvy', 'vdxmicjmfbtsbqqmqcgtnrvdgaucsgspwg', 'witlfqwvhmmyjrnrzttrikhhsrd', 'pbzsqsljurt']) from system.numbers limit 10; -select [7, 0, 0, 8, 0, 2, 0, 0, 6, 0, 2, 0, 3, 1] = multiSearchAllPositions(materialize('aizovxqpzcbbxuhwtiaaqhdqjdei'), ['qpzcbbxuhw', 'jugrpglqbm', 'dspwhzpyjohhtizegrnswhjfpdz', 'pzcbbxuh', 'vayzeszlycke', 'i', 'gvrontcpqavsjxtjwzgwxugiyhkhmhq', 'gyzmeroxztgaurmrqwtmsxcqnxaezuoapatvu', 'xqpzc', 'mjiswsvlvlpqrhhptqq', 'iz', 'hmzjxxfjsvcvdpqwtrdrp', 'zovxqpzcbbxuhwtia', 'ai']) from system.numbers limit 10; -select [0, 0, 0, 19, 14, 22, 10, 0, 0, 13, 0, 8] = multiSearchAllPositions(materialize('ydfgiluhyxwqdfiwtzobwzscyxhuov'), ['srsoubrgghleyheujsbwwwykerzlqphgejpxvog', 'axchkyleddjwkvbuyhmekpbbbztxdlm', 'zqodzvlkmfe', 'obwz', 'fi', 'zsc', 'xwq', 'pvmurvrd', 'uulcdtexckmrsokmgdpkstlkoavyrmxeaacvydxf', 'dfi', 'mxcngttujzgtlssrmluaflmjuv', 'hyxwqdfiwtzobwzscyxhu']) from system.numbers limit 10; -select [6, 1, 1, 0, 0, 5, 1, 0, 8, 0, 5, 0, 2, 12, 0, 15, 0, 0] = multiSearchAllPositions(materialize('pyepgwainvmwekwhhqxxvzdjw'), ['w', '', '', 'gvvkllofjnxvcu', 'kmwwhboplctvzazcyfpxhwtaddfnhekei', 'gwainv', 'pyepgwain', 'ekpnogkzzmbpfynsunwqp', 'invmwe', 'hrxpiplfplqjsstuybksuteoz', 'gwa', 'akfpyduqrwosxcbdemtxrxvundrgse', 'yepgwainvmw', 'wekwhhqxxvzdjw', 'fyimzvedmyriubgoznmcav', 'whhq', 'ozxowbwdqfisuupyzaqynoprgsjhkwlum', 'vpoufrofekajksdp']) from system.numbers limit 10; -select [0, 0, 5, 1, 1, 0, 15, 1, 5, 10, 4, 0, 1, 0, 3, 0, 0, 0] = multiSearchAllPositions(materialize('lqwahffxurkbhhzytequotkfk'), ['rwjqudpuaiufle', 'livwgbnflvy', 'hffxurkbhh', '', '', 'xcajwbqbttzfzfowjubmmgnmssat', 'zytequ', 'lq', 'h', 'rkbhh', 'a', 'immejthwgdr', '', 'llhhnlhcvnxxorzzjt', 'w', 'cvjynqxcivmmmvc', 'wexjomdcmursppjtsweybheyxzleuz', 'fzronsnddfxwlkkzidiknhpjipyrcrzel']) from system.numbers limit 10; -select [0, 1, 11, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4, 1] = multiSearchAllPositions(materialize('nkddriylnakicwgdwrfxpodqea'), ['izwdpgrgpmjlwkanjrffgela', '', 'kicw', 'hltmfymgmrjckdiylkzjlvvyuleksikdjrg', 'yigveskrbidknjxigwilmkgyizewikh', 'xyvzhsnqmuec', 'odcgzlavzrwesjks', 'oilvfgliktoujukpgzvhmokdgkssqgqot', 'llsfsurvimbahwqtbqbp', 'nxj', 'pimydixeobdxmdkvhcyzcgnbhzsydx', 'couzmvxedobuohibgxwoxvmpote', 'driylnakicwgdwrf', 'nkddr']) from system.numbers limit 10; -select [0, 0, 0, 3, 0, 15, 0, 0, 12, 7, 0, 0, 0, 0, 5, 0] = multiSearchAllPositions(materialize('jnckhtjqwycyihuejibqmddrdxe'), ['tajzx', 'vuddoylclxatcjvinusdwt', 'spxkhxvzsljkmnzpeubszjnhqczavgtqopxn', 'ckhtjqwycyi', 'xlbfzdxspldoes', 'u', 'czosfebeznt', 'gzhabdsuyreisxvyfrfrkq', 'yihuejibqmd', 'jqwycyihuejibqm', 'cfbvprgzx', 'hxu', 'vxbhrfpzacgd', 'afoaij', 'htjqwycyihu', 'httzbskqd']) from system.numbers limit 10; -select [0, 0, 12, 4, 4, 0, 13, 23, 0, 1, 0, 2, 0, 0, 0, 3, 0, 0] = multiSearchAllPositions(materialize('dzejajvpoojdkqbnayahygidyrjmb'), ['khwxxvtnqhobbvwgwkpusjlhlzifiuclycml', 'nzvuhtwdaivo', 'dkqbnayahygidyr', 'jajvpoo', 'j', 'wdtbvwmeqgyvetu', 'kqbn', 'idyrjmb', 'tsnxuxevsxrxpgpfdgrkhwqpkse', '', 'efsdgzuefhdzkmquxu', 'zejajvpoojdkqbnayahyg', 'ugwfuighbygrxyctop', 'fcbxzbdugc', 'dxmzzrcplob', 'ejaj', 'wmmupyxrylvawsyfccluiiene', 'ohzmsqhpzbafvbzqwzftbvftei']) from system.numbers limit 10; -select [6, 8, 1, 4, 0, 10, 0, 1, 14, 0, 1, 0, 5, 0, 0, 0, 0, 15, 0, 1] = multiSearchAllPositions(materialize('ffaujlverosspbzaqefjzql'), ['lvero', 'erossp', 'f', 'ujlverosspbz', 'btfimgklzzxlbkbuqyrmnud', 'osspb', 'muqexvtjuaar', 'f', 'bzaq', 'lprihswhwkdhqciqhfaowarn', 'ffaujlve', 'uhbbjrqjb', 'jlver', 'umucyhbbu', 'pjthtzmgxhvpbdphesnnztuu', 'xfqhfdfsbbazactpastzvzqudgk', 'lvovjfoatc', 'z', 'givejzhoqsd', '']) from system.numbers limit 10; -select [5, 7, 0, 1, 6, 0, 0, 1, 1, 2, 0, 1, 4, 2, 0, 6, 0, 0] = multiSearchAllPositions(materialize('hzftozkvquknsahhxefzg'), ['ozkvquknsahhxefzg', 'kv', 'lkdhmafrec', '', 'zkvquknsahh', 'xmjuizyconipirigdmhqclox', 'dqqwolnkkwbyyjicsoshidbay', '', '', 'zf', 'sonvmkapcjcakgpejvn', 'hzftoz', 't', 'zftozkvqukns', 'dyuqohvehxsvdzdlqzl', 'zkvquknsahhx', 'vueohmytvmglqwptfbhxffspf', 'ilkdurxg']) from system.numbers limit 10; -select [1, 7, 6, 4, 0, 1, 0, 0, 0, 9, 7, 1, 1, 0, 0, 0] = multiSearchAllPositions(materialize('aapdygjzrhskntrphianzjob'), ['', 'jz', 'gjzrh', 'dygjzrhskntrphia', 'qcnahphlxmdru', '', 'rnwvzdn', 'isbekwuivytqggsxniqojrvpwjdr', 'sstwvgyavbwxvjojrpg', 'rhskn', 'jzrhskntrp', '', '', 'toilvppgjizaxtidizgbgygubmob', 'vjwzwpvsklkxqgeqqmtssnhlmw', 'znvpjjlydvzhkt']) from system.numbers limit 10; -select [0, 1, 0, 1, 0, 0, 10, 0, 0, 0, 11, 0, 5, 0] = multiSearchAllPositions(materialize('blwpfdjjkxettfetdoxvxbyk'), ['wgylnwqcrojacofrcanjme', 'bl', 'qqcunzpvgi', '', 'ijemdmmdxkakrawwdqrjtrttig', 'qwkaifalc', 'xe', 'zqocnfuvzowuqkmwrfxw', 'xpaayeljvly', 'wvphqqhulpepjjjnxjfudfcomajc', 'ettfetdoxvx', 'ikablovwhnbohibbuhwjshhdemidgreqf', 'fdjjkxett', 'kiairehwbxveqkcfqhgopztgpatljgqp']) from system.numbers limit 10; -select [0, 0, 6, 1, 1, 0, 0, 1, 2, 0, 0, 0, 0, 0] = multiSearchAllPositions(materialize('vghzgedqpnqtvaoonwsz'), ['mfyndhucfpzjxzaezny', 'niejb', 'edqpnqt', '', 'v', 'kivdvealqadzdatziujdnvymmia', 'lvznmgwtlwevcxyfbkqc', 'vghzge', 'gh', 'tbzle', 'vjiqponbvgvguuhqdijbdeu', 'mshlyabasgukboknbqgmmmj', 'kjk', 'abkeftpnpvdkfyrxbrihyfxcfxablv']) from system.numbers limit 10; -select [0, 0, 0, 0, 9, 0, 7, 0, 9, 8, 0, 0] = multiSearchAllPositions(materialize('oaghnutqsqcnwvmzrnxgacsovxiko'), ['upien', 'moqszigvduvvwvmpemupvmmzctbrbtqggrk', 'igeiaccvxejtfvifrmimwpewllcggji', 'wnwjorpzgsqiociw', 'sq', 'rkysegpoej', 'tqsqcnwvmzrnxgacsovxiko', 'ioykypvfjufbicpyrpfuhugk', 's', 'qsqcnwvmzrnxgacsov', 'hhbeisvmpnkwmimgyfmybtljiu', 'kfozjowd']) from system.numbers limit 10; -select [0, 5, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 5, 0, 0, 14, 1, 1, 0, 0] = multiSearchAllPositions(materialize('wbjfsevqspsvbwlzrkhcfuhxddbq'), ['ltgjbz', 's', 'qdfnmggupdfxjfnmvwyrqopxtxf', 'sazlkmaikcltojbzbmdfddu', 'yzanifqxufyfwrxzkhngoxkrrph', 'iwskc', 'xkykshryphyfnwcnmjfqjrixykmzmwm', 'wwpenztbhkdbwidfkypqlxivsjs', 'rlkevy', 'qigywtkezwd', '', 'c', 'sevqspsvbwlzrk', 'gwg', 'iduhrjsrtodxdkjykjoghtjtvplrscitxnvt', 'wlzrkhcfuhxddb', '', 'wbjfsev', 'zytusrcvqbazb', 'tec']) from system.numbers limit 10; -select [0, 1, 5, 0, 6, 8, 0, 3, 2, 0, 0, 9, 0, 4, 0, 0] = multiSearchAllPositions(materialize('mxiifpzlovgfozpgirtio'), ['srullnscuzenzhp', '', 'f', 'apetxezid', 'pzlovgf', 'lo', 'ecbmso', 'i', 'xiifpzlovgfozpgir', 'bnefwypvctubvslsesnctqspdyctq', 'tdncmgbikboss', 'o', 'zmgobcarxlxaho', 'ifpzlovgfozpg', 'dwmjqyylvsxzfr', 'pxhrecconce']) from system.numbers limit 10; -select [0, 0, 0, 2, 0, 0, 2, 0, 8, 0, 0, 0, 7, 0, 0, 0, 21, 3, 1, 8] = multiSearchAllPositions(materialize('jtvnrdpdevgnzexqdrrxqgiujexhm'), ['ibkvzoqmiyfgfztupug', 'iqzeixfykxcghlbgsicxiywlurrgjsywwk', 'vzdffjzlqxgzdcrkgoro', 'tvnrdpdevgnzexqdr', 'nqywueahcmoojtyjlhfpysk', 'iqalixciiidvrtmpzozfb', 'tv', 'rxkfeasoff', 'devgnzexqdrrxqgiuj', 'kvvuvyplboowjrestyvdfrxdjjujvkxy', 'shkhpneekuyyqtxfxutvz', 'yy', 'pdevgnz', 'nplpydxiwnbvlhoorcmqkycqisi', 'jlkxplbftfkxqgnqnaw', 'qdggpjenbrwbjtorbi', 'qgiuje', 'vnrdpd', '', 'dev']) from system.numbers limit 10; -select [14, 0, 0, 7, 20, 6, 0, 13, 0, 0, 20, 0, 20, 2, 0, 8, 2, 11, 2, 0] = multiSearchAllPositions(materialize('asjwxabjrwgcdviokfaoqvqiafz'), ['v', 'zqngytligwwpzxhatyayvdnbbj', 'gjicovfzgbyagiirn', 'bjrwgcdviok', 'oqvqiafz', 'abjrwgc', 'wulrpfzh', 'dviokfao', 'esnchjuiufjadqmdtrpcd', 'tkodqzsjchpaftk', 'oqvq', 'eyoshlrlvmnqjmtmloryvg', 'oqv', 'sjwx', 'uokueelyytnoidplwmmox', 'jrwgcdviokfaoqvqiaf', 'sjwxabjrwgcdviokfaoqvqi', 'gcdviokfa', 'sjwxab', 'zneabsnfucjcwauxmudyxibnmxzfx']) from system.numbers limit 10; -select [0, 16, 8, 0, 10, 0, 0, 0, 0, 1, 0, 6, 0, 1, 0, 4, 0, 6, 0, 0] = multiSearchAllPositions(materialize('soxfqagiuhkaylzootfjy'), ['eveprzxphyenbrnnznpctvxn', 'oo', 'iuhka', 'ikutjhrnvzfb', 'h', 'duyvvjizristnkczgwj', 'ihfrp', 'afpyrlj', 'uonp', 'soxfqagiuhkaylzootfjy', 'qeckxkoxldpzzpmkbvcex', 'agiuhkaylzo', 'tckcumkbsgrgqjvtlijack', '', 'fnfweqlldcdnwfaohqohp', 'fqagiuhkayl', 'pqnvwprxwwrcjqvfsbfimwye', 'agi', 'ta', 'r']) from system.numbers limit 10; -select [3, 7, 1, 6, 0, 1, 0, 11, 0, 9, 17, 1, 18, 12] = multiSearchAllPositions(materialize('ladbcypcbcxahmujwezkvweud'), ['db', 'pcbcxahm', 'lad', 'ypcb', 'atevkzyyxhphtuekymhh', 'lad', 'mltjrwaibetrtwpfa', 'xahmujwezkvweud', 'dg', 'bcxahmujw', 'we', '', 'e', 'ahmujwezkvw']) from system.numbers limit 10; -select [6, 0, 11, 0, 7, 0, 0, 0, 6, 1, 0, 3, 0, 0, 0, 0] = multiSearchAllPositions(materialize('hhkscgmqzmuwltmrhtxnnzsxl'), ['gmqzmuwltmrh', 'qtescwjubeqhurqoqfjauwxdoc', 'uwltmrh', 'qlhyfuspwdtecdbrmrqcnxghhlnbmzs', 'm', 'kcsuocwokvohnqonnfzmeiqtomehksehwc', 'hoxocyilgrxxoek', 'nisnlmbdczjsiw', 'gmqz', '', 'cqzz', 'k', 'utxctwtzelxmtioyqshxedecih', 'ifsmsljxzkyuigdtunwk', 'ojxvxwdosaqjhrnjwisss', 'dz']) from system.numbers limit 10; -select [0, 0, 19, 7, 0, 0, 1, 0, 0, 12, 0, 0, 1, 0, 1, 1, 5, 0, 23, 8] = multiSearchAllPositions(materialize('raxgcqizulxfwivauupqnofbijxfr'), ['sxvhaxlrpviwuinrcebtfepxxkhxxgqu', 'cuodfevkpszuimhymxypktdvicmyxm', 'pqnof', 'i', 'ufpljiniflkctwkwcrsbdhvrvkizticpqkgvq', 'osojyhejhrlhjvqrtobwthjgw', '', 'anzlevtxre', 'ufnpkjvgidirrnpvbsndfnovebdily', 'fwivauupqnofbi', 'rywyadwcvk', 'ltnlhftdfefmkenadahcpxw', '', 'xryluzlhnsqk', 'r', '', 'cqizulxfwivauupqnofb', 'y', 'fb', 'zulxfwivauupqnofbijxf']) from system.numbers limit 10; -select [4, 0, 0, 0, 0, 24, 1, 2, 0, 2, 0, 0, 8, 0] = multiSearchAllPositions(materialize('cwcqyjjodlepauupgobsgrzdvii'), ['q', 'yjppewylsqbnjwnhokzqtauggsjhhhkkkqsy', 'uutltzhjtc', 'pkmuptmzzeqhichaikwbggronli', 'erzgcuxnec', 'dvii', '', 'w', 'fkmpha', 'wcqyjjodlepauupgobsgrz', 'cbnmwirigaf', 'fcumlot', 'odlepauu', 'lthautlklktfukpt']) from system.numbers limit 10; -select [1, 1, 1, 1, 22, 0, 0, 8, 18, 15] = multiSearchAllPositions(materialize('vpscxxibyhvtmrdzrocvdngpb'), ['', '', '', '', 'n', 'agrahemfuhmftacvpnaxkx', 'dqqwvfsrqv', 'byhvtmrdzrocv', 'ocvdn', 'dzrocvdngpb']) from system.numbers limit 10; -select [1, 1, 1, 15, 10, 0, 0, 0, 0, 2] = multiSearchAllPositions(materialize('nfoievsrpvheprosjdsoiz'), ['', 'nfo', '', 'osjd', 'vheprosjdsoiz', 'az', 'blhvdycvjnxaipvxybs', 'umgxmpkvuvuvdaczkz', 'gfspmnzidixcjgjw', 'f']) from system.numbers limit 10; -select [0, 0, 2, 2, 0, 0, 0, 11, 10, 4, 9, 1, 6, 4, 0, 0] = multiSearchAllPositions(materialize('bdmfwdisdlgbcidshnhautsye'), ['uxdceftnmnqpveljer', 'xdnh', 'dmf', 'dmfwdisdlgbc', 'cpwnaijpkpyjgaq', 'doquvlrzhusjbxyqcqxvwr', 'llppnnmtqggyfoxtawnngsiiunvjjxxsufh', 'gbcidshnhau', 'lgbcids', 'f', 'dlgbc', 'bdmfwdisdlgbcids', 'disdlgbcidshnhautsy', 'fwdisdlgbcidshn', 'zfpbfc', 'triqajlyfmxlredivqiambigmge']) from system.numbers limit 10; -select [0, 0, 16, 0, 0, 0, 14, 6, 2, 1, 0, 0, 1, 0, 10, 12, 0, 0, 0, 0] = multiSearchAllPositions(materialize('absimumlxdlxuzpyrunivcb'), ['jglfzroni', 'wzfmtbjlcdxlbpialqjafjwz', 'yrun', 'fgmljkkp', 'nniob', 'fdektoyhxrumiycvkwekphypgti', 'zp', 'um', 'bsimu', '', 'yslsnfisaebuujltpgcskhhqcucdhb', 'xlaphsqgqsfykhilddctrawerneqoigb', '', 'pdvcfxdlurmegspidojt', 'd', 'xu', 'fdp', 'xjrqmybmccjbjtvyvdh', 'nvhdfatqi', 'neubuiykajzcrzdbvpwjhlpdmd']) from system.numbers limit 10; -select [0, 0, 0, 9, 0, 0, 1, 1, 1, 1] = multiSearchAllPositions(materialize('lvyenvktdnylszlypuwqecohy'), ['ihlsiynj', 'ctcnhbkumvbgfdclwjhsswpqyfrx', 'rpgqwkydwlfclcuupoynwrfffogxesvmbj', 'dnyl', 'coeqgdtbemkhgplprfxgwpl', 'dkbshktectbduxlcaptlzspq', 'l', 'lvyenvktdnylszlypuw', 'lvyenvk', '']) from system.numbers limit 10; -select [1, 0, 0, 0, 0, 1, 2, 22, 8, 17, 1, 13, 0, 0, 0, 0, 0, 5] = multiSearchAllPositions(materialize('wphcobonpgaqwgfenotzadgqezx'), ['', 'qeuycfhkfjwokxgrkaodqioaotkepzlhnrv', 'taehtytq', 'gejlcipocalc', 'poyvvvntrvqazixkwigtairjvxkgouiuva', '', 'phc', 'dg', 'npgaqwg', 'notzadgqe', '', 'wgfe', 'smipuxgvntys', 'qhrfdytbfeujzievelffzrv', 'cfmzw', 'hcywnyguzjredwjbqtwyuhtewuhzkc', 'tssfeinoykdauderpjyxtmb', 'obonpgaqwgfen']) from system.numbers limit 10; -select [0, 0, 0, 0, 0, 6, 6, 0, 0, 2, 0, 5, 2, 0, 6, 3] = multiSearchAllPositions(materialize('qvslufpsddtfudzrzlvrzdra'), ['jxsgyzgnjwyd', 'hqhxzhskwivpuqkjheywwfhthm', 'kbwlwadilqhgwlcpxkadkamsnzngms', 'fxunda', 'nlltydufobnfxjyhch', 'fpsddtfudzrzl', 'fp', 'ykhxjyqtvjbykskbejpnmbxpumknqucu', 'iyecekjcbkowdothxc', 'vslufpsddtfu', 'mjgtofkjeknlikrugkfhxlioicevil', 'uf', 'vslufpsdd', 'cxizdzygyu', 'fpsddtfudzrz', 'slufp']) from system.numbers limit 10; -select [12, 0, 0, 0, 0, 1, 6, 0, 1, 2] = multiSearchAllPositions(materialize('ydsbycnifbcforymknzfi'), ['forymkn', 'vgxtcdkfmjhc', 'ymugjvtmtzvghmifolzdihutqoisl', 'fzooddrlhi', 'bdefmxxdepcqi', '', 'cnif', 'ilzbhegpcnkdkooopaguljlie', '', 'dsbycnifbcforym']) from system.numbers limit 10; -select [0, 2, 4, 1, 1, 3, 0, 0, 0, 7] = multiSearchAllPositions(materialize('sksoirfwdhpdyxrkklhc'), ['vuixtegnp', 'ks', 'oirfwdhpd', 'sksoirf', 'skso', 'soi', 'eoxpa', 'vpfmzovgatllf', 'txsezmqvduxbmwu', 'fw']) from system.numbers limit 10; -select [2, 21, 8, 10, 6, 0, 1, 11, 0, 0, 21, 4, 29, 0] = multiSearchAllPositions(materialize('wlkublfclrvgixpbvgliylzbuuoyai'), ['l', 'ylzbuu', 'clr', 'rvgi', 'lf', 'bqtzaqjdfhvgddyaywaiybk', '', 'vgixpbv', 'ponnohwdvrq', 'dqioxovlbvobwkgeghlqxtwre', 'y', 'ublfclrvgix', 'a', 'eoxxbkaawwsdgzfweci']) from system.numbers limit 10; -select [0, 0, 2, 1, 1, 9, 1, 0, 0, 1] = multiSearchAllPositions(materialize('llpbsbgmfiadwvvsciak'), ['knyjtntotuldifbndcpxzsdwdduv', 'lfhofdxavpsiporpdyfziqzcni', 'lpbsbgmf', 'llpbsbgmfi', 'llpbsbgmfiadwvv', 'fia', '', 'uomksovcuhfmztuqwzwchmwvonk', 'ujbasmokvghmredszgwe', '']) from system.numbers limit 10; -select [3, 0, 0, 0, 6, 1, 7, 0, 2, 1, 1, 0, 7, 0, 1, 0, 1, 1, 5, 11] = multiSearchAllPositions(materialize('hnmrouevovxrzrejesigfukkmbiid'), ['m', 'apqlvipphjbui', 'wkepvtnpu', 'amjvdpudkdsddjgsmzhzovnwjrzjirdoxk', 'ue', '', 'evov', 'qoplzddxjejvbmthnplyha', 'nmrouevovxrz', '', 'hnmrouev', 'hnzevrvlmxnjmvhitgdhgd', 'evovxrzrejesig', 'yvlxrjaqdaizishkftgcuikt', '', 'buyrmbkvqukochjteumqchrhxgtmuorsdgzlfn', '', 'hnmrouevov', 'ouevovx', 'xr']) from system.numbers limit 10; -select [0, 13, 0, 0, 0, 0, 0, 14, 0, 0, 1, 12, 0, 1] = multiSearchAllPositions(materialize('uwfgpemgdjimotxuxrxxoynxoaw'), ['uzcevfdfy', 'otxuxrxxoynxoa', 'xeduvwhrogxccwhnzkiolksry', 'pxdszcyzxlrvkymhomz', 'vhsacxoaymycvcevuujpvozsqklahstmvgt', 'zydsajykft', 'vdvqynfhlhoilkhjjkcehnpmwgdtfkspk', 'txuxrx', 'slcaryelankprkeyzaucfhe', 'iocwevqwpkbrbqvddaob', 'uwfg', 'motxuxrxx', 'kpzbg', '']) from system.numbers limit 10; -select [1, 1, 0, 6, 6, 0, 0, 0, 8, 0, 8, 14, 1, 5, 6, 0, 0, 1] = multiSearchAllPositions(materialize('epudevopgooprmhqzjdvjvqm'), ['ep', 'epudevopg', 'tlyinfnhputxggivtyxgtupzs', 'vopgoop', 'v', 'hjfcoemfk', 'zjyhmybeuzxkuwaxtcut', 'txrxzndoxyzgnzepjzagc', 'pgooprmhqzj', 'wmtqcbsofbe', 'pgo', 'm', '', 'evopgooprmhqzjdv', 'vopgooprmhqzjdv', 'gmvqubpsnvrabixk', 'wjevqrrywloomnpsjbuybhkhzdeamj', '']) from system.numbers limit 10; -select [15, 4, 4, 0, 0, 1, 1, 0, 0, 0, 0, 20, 0, 10, 1, 1, 0, 2, 4, 3] = multiSearchAllPositions(materialize('uogsfbdefogwnekfoeobtkrgiceksz'), ['kfoeobtkrgice', 'sfbd', 'sfbdefogwn', 'zwtenhiqavmqoolkvjiqjfb', 'vnjkshyvpwhrauackplqllakcjyamvsuokrxbfv', 'uog', '', 'qtzuhdcdymytgtscvzlzswdlrqidreuuuqk', 'vlridmjlbxyiljpgxsctzygzyawqqysf', 'xsnkwyrmjaaaryvrdgtoshdxpvgsjjrov', 'fanchgljgwosfamgscuuriwospheze', 'btkrgicek', 'ohsclekvizgfoatxybxbjoxpsd', 'ogwnekfoeobtkr', '', '', 'vtzcobbhadfwubkcd', 'og', 's', 'gs']) from system.numbers limit 10; -select [0, 0, 5, 1, 0, 5, 1, 6, 0, 1, 9, 0, 1, 1] = multiSearchAllPositions(materialize('aoiqztelubikzmxchloa'), ['blc', 'p', 'ztelubikzmxchlo', 'aoiqztelubi', 'uckqledkyfboolq', 'ztelubikzmxch', 'a', 'telubikzm', 'powokpdraslpadpwvrqpbb', 'aoiqztelu', 'u', 'kishbitagsxnhyyswn', '', '']) from system.numbers limit 10; -select [5, 11, 0, 0, 0, 5, 0, 0, 0, 1, 16, 0, 0, 0, 0, 0] = multiSearchAllPositions(materialize('egxmimubhidowgnfziwgnlqiw'), ['imubhidowgnfzi', 'dowgnf', 'yqpcpfvnfpxetozraxbmzxxcvtzm', 'xkbaqvzlqjyjoiqourezbzwaqkfyekcfie', 'jjctusdmxr', 'imubhi', 'zawnslbfrtqohnztmnssxscymonlhkitq', 'oxcitennfpuoptwrlmc', 'ac', 'egxmi', 'fziwgn', 'rt', 'fuxfuctdmawmhxxxg', 'suulqkrsfgynruygjckrmizsksjcfwath', 'slgsq', 'zcbqjpehilwyztumebmdrsl']) from system.numbers limit 10; -select [20, 0, 9, 0, 0, 14, 0, 5, 8, 3, 0, 0, 0, 4] = multiSearchAllPositions(materialize('zczprzdcvcqzqdnhubyoblg'), ['obl', 'lzrjyezgqqoiydn', 'vc', 'nbvwfpmqlziedob', 'pnezljnnujjbyviqsdpaqkkrlogeht', 'dn', 'irvgeaq', 'rzdcvcqzqdnh', 'cvcqzqdnh', 'zprzdcv', 'wvvgoexuevmqjeqavsianoviubfixdpe', 'aeavhqipsvfkcynyrtlxwpegwqmnd', 'blckyiacwgfaoarfkptwcei', 'prz']) from system.numbers limit 10; -select [2, 1, 1, 9, 10, 5, 0, 0, 0, 2, 9, 7, 9, 0, 1, 9, 7, 0] = multiSearchAllPositions(materialize('mvovpvuhjwdzjwojcxxrbxy'), ['vo', '', '', 'jwdz', 'wdzj', 'pvu', 'ocxprubxhjnji', 'phzfbtacrg', 'jguuqhhxbrwbo', 'vovpvuhjwd', 'jw', 'u', 'jwdzjwojcx', 'nlwfvolaklizslylbvcgicbjw', '', 'jwd', 'uhjwdz', 'bbcsuvtru']) from system.numbers limit 10; -select [2, 0, 21, 0, 0, 0, 3, 0, 0, 0, 0, 10, 1, 18] = multiSearchAllPositions(materialize('nmdkwvafhcbipwoqtsrzitwxsnabwf'), ['m', 'ohlfouwyucostahqlwlbkjgmdhdyagnihtmlt', 'itwx', 'jjkyhungzqqyzxrq', 'abkqvxxpu', 'lvzgnaxzctaarxuqowcski', 'dkwvafhcb', 'xuxjexmeeqvyjmpznpdmcn', 'vklvpoaakfnhtkprnijihxdbbhbllnz', 'fpcdgmcrwmdbflnijjmljlhtkszkocnafzaubtxp', 'hmysdmmhnebmhpjrrqpjdqsgeuutsj', 'cbipwoqtsrzitwxsna', 'nm', 'srzitwx']) from system.numbers limit 10; -select [17, 5, 0, 13, 0, 0, 10, 1, 0, 19, 10, 8, 0, 4] = multiSearchAllPositions(materialize('gfvndbztroigxfujasvcdgfbh'), ['asvcdgf', 'dbztroigxfujas', 'pr', 'xfujas', 'nxwdmqsobxgm', 'wdvoepclqfhy', 'oigxfu', '', 'flgcghcfeiqvhvqiriciywbkhrxraxvneu', 'vcd', 'oigxfu', 'troigxfuj', 'gbnyvjhptuehkefhwjo', 'ndbz']) from system.numbers limit 10; -select [0, 14, 1, 0, 0, 1, 1, 11, 0, 8, 6, 0, 3, 19, 7, 0] = multiSearchAllPositions(materialize('nofwsbvvzgijgskbqjwyjmtfdogzzo'), ['kthjocfzvys', 'skbqjwyjmtfdo', 'nof', 'mfapvffuhueofutby', 'vqmkgjldhqohipgecie', 'nofwsbv', '', 'ijgs', 'telzjcbsloysamquwsoaso', 'vzgijgskbqjwyjmt', 'bvvzgijgskbqjwyjmtfd', 'hdlvuoylcmoicsejofcgnvddx', 'fwsbvvzgijgskb', 'wyjm', 'vvzgijg', 'fwzysuvkjtdiufetvlfwf']) from system.numbers limit 10; -select [10, 2, 13, 0, 0, 0, 2, 0, 9, 2, 4, 1, 1, 0, 1, 6] = multiSearchAllPositions(materialize('litdbgdtgtbkyflsvpjbqwsg'), ['tbky', 'itdbgdtgtb', 'yflsvpjb', 'ikbylslpoqxeqoqurbdehlroympy', 'hxejlgsbthvjalqjybc', 'sontq', 'itdbgd', 'ozqwgcjqmqqlkiaqppitsvjztwkh', 'gtbkyf', 'itdbgdtgtbkyfls', 'dbg', 'litdb', '', 'qesbakrnkbtfvwu', 'litd', 'g']) from system.numbers limit 10; -select [0, 0, 1, 1, 5, 0, 8, 12, 0, 2, 0, 7, 0, 6] = multiSearchAllPositions(materialize('ijzojxumpvcxwgekqimrkomvuzl'), ['xirqhjqibnirldvbfsb', 'htckarpuctrasdxoosutyxqioizsnzi', '', '', 'jxu', 'dskssv', 'mpvcxwgekqi', 'xwgek', 'qsuexmzfcxlrhkvlzwceqxfkyzogpoku', 'jzojx', 'carjpqihtpjniqz', 'umpvcxwgekq', 'krpkzzrxxtvfhdopjpqcyxfnbas', 'xumpvcxwg']) from system.numbers limit 10; -select [0, 0, 0, 6, 0, 8, 0, 2, 0, 0, 0, 0, 14, 0, 0, 1, 1, 0, 0, 0] = multiSearchAllPositions(materialize('zpplelzzxsjwktedrrtqhfmoufv'), ['jzzlntsokwlm', 'cb', 'wuxotyiegupflu', 'lzzxsjwkte', 'owbxgndpcmfuizpcduvucnntgryn', 'zxsjwktedrrtqhf', 'kystlupelnmormqmqclgjakfwnyt', 'pple', 'lishqmxa', 'mulwlrbizkmtbved', 'uchtfzizjiooetgjfydhmzbtmqsyhayd', 'hrzgjifkinwyxnazokuhicvloaygeinpd', 'tedrrt', 'shntwxsuxux', 'evrjehtdzzoxkismtfnqp', 'z', '', 'nxtybut', 'vfdchgqclhxpqpmitppysbvxepzhxv', 'wxmvmvjlrrehwylgqhpehzotgrzkgi']) from system.numbers limit 10; - -select [15, 19, 0, 0, 15, 0, 0, 1, 2, 6] = multiSearchAllPositionsUTF8(materialize('зжерхмчсйирдчрришкраоддцфгх'), ['ришкра', 'раоддц', 'фттиалусгоцжлтщзвумрдчи', 'влййи', 'ришкра', 'цгфжуцгивй', 'ккгжхрггчфглх', 'з', 'жерхмчсйи', 'мчсйирдчрришкраоддц']) from system.numbers limit 10; -select [0, 0, 0, 1, 4, 0, 14, 0, 1, 8, 8, 9, 0, 0, 4, 0] = multiSearchAllPositionsUTF8(materialize('етвхйчдобкчукхпщлмжпфайтфдоизщ'), ['амфшужперосрфщфлижйййжжжй', 'ххкбщшзлмщггтшцпсдйкдшйвхскемц', 'ергйплгпнглккшкарещимгапхг', '', 'хйчдо', 'вввбжовшзйбгуоиждепйабаххеквщижтйиухос', 'хпщл', 'жфуомщуххнедзхищнгхрквлпмзауеегз', 'етвхй', 'о', 'о', 'бк', 'цфецккифж', 'аизлокл', 'х', 'слщгеивлевбчнчбтшгфмжрфка']) from system.numbers limit 10; -select [0, 0, 1, 2, 0, 0, 14, 0, 3, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('йбемооабурнирйофшдгпснж'), ['гпфцл', 'нчбперпмцкввдчсщвзйрдфнф', '', 'бем', 'ч', 'жгш', 'йофшдгпснж', 'шасгафчг', 'емооабур', 'пиохцжццгппщчопзйлмуотз', 'рпдомнфвопхкшешйишумбацтл', 'нисиийфррбдоц']) from system.numbers limit 10; -select [1, 18, 12, 0, 0, 1, 1, 3, 7, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('гсщнфийтфзжцйпфбйалущ'), ['', 'алущ', 'цйпфбйал', 'цвбфцйвсвлицсчнргпцнр', 'х', 'гс', '', 'щн', 'й', 'дгйрвцщтп', 'уитвквоффвцхфишрлерйцувф', 'кфтййлпнзжчижвглзкижн']) from system.numbers limit 10; -select [14, 0, 5, 5, 0, 6, 0, 16, 0, 0] = multiSearchAllPositionsUTF8(materialize('ефщнйнуйебнснлрцгкеитбг'), ['лрцгкеитб', 'епклжфцпнфопе', 'йнуйебн', 'й', 'тлт', 'нуйебнснлрцгкеит', 'глечршгвотумкимтлм', 'цгк', 'щгйчой', 'звкцкчк']) from system.numbers limit 10; -select [0, 1, 18, 6, 0, 3, 0, 0, 25, 0, 0, 1, 16, 5, 1, 7, 0, 0] = multiSearchAllPositionsUTF8(materialize('пумгмцшмжштсшлачсжарерфиозиг'), ['чсуубфийемквмоотванухмбрфхжоест', '', 'жар', 'цшмжш', 'жртещтинтвпочнкдткцза', 'м', 'адзгтбаскщгдшжл', 'штфжшллезпджигщфлезфгзчайанхктицштйй', 'о', 'етадаарйсцейдошшцечхзлшлрртсрггцртспд', 'зтвшалрпфлщбцд', 'пу', 'ч', 'мцшмжштсшлачсж', '', 'шмжшт', 'ещтжшйтчзчаноемрбц', 'тевбусешйрйчшзо']) from system.numbers limit 10; -select [7, 10, 0, 0, 0, 0, 1, 12, 9, 2, 0, 0, 0, 4, 1, 1, 0, 6] = multiSearchAllPositionsUTF8(materialize('дупгвндвйжмаузнллнзл'), ['двйжмаузн', 'жмаузнлл', 'емйжркоблновцгпезрдавкбелцщста', 'щзкгм', 'лебрпцрсутшриащгайвц', 'лзнмл', 'д', 'ауз', 'йжмау', 'упгвндвйж', 'жщсббфвихг', 'всигсеигцбгаелтчкирлнзшзцжещнс', 'рмшиеиесрлщципщхкхтоцщчйоо', 'гвн', '', '', 'йадеоцлпшпвщзещзкхйрейопмажбб', 'ндв']) from system.numbers limit 10; -select [0, 0, 0, 8, 3, 10, 22, 0, 13, 11, 0, 1, 18, 0, 1, 0] = multiSearchAllPositionsUTF8(materialize('жшзфппавввслфцлнщшопкдшку'), ['саоткнхфодзаа', 'кйхванкзаисйбврщве', 'бчоуучватхфукчф', 'вввслфц', 'з', 'вслфцлнщшопк', 'дшк', 'из', 'фцл', 'с', 'зртмцтпощпщхк', 'жшзфппавввслфц', 'шопк', 'збтхрсдтатхпрзлхдооощифачхчфн', '', 'жщшийугз']) from system.numbers limit 10; -select [2, 4, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 21, 0, 6, 0] = multiSearchAllPositionsUTF8(materialize('пчботухвгдчекмжндбоожш'), ['чботухвгдчекмжндб', 'от', 'гвсжжйлбтщчучнхсмдйни', 'жцжзмшлибшефуоуомпацбщщу', 'онхфлуцйлхтбмц', 'йтепжу', 'хтдрпвкщрли', 'аддайф', 'нхегщккбфедндоацкиз', 'йгкцзртфжгв', 'буелрщмхйохгибжндфшщвшрлдччрмфмс', 'цщцтзфнщ', 'уч', 'пчб', 'жш', 'пнфббтшйгхйрочнлксщпгвжтч', 'ухвг', 'лсцппузазщрйхймщбзоршощбзленхп']) from system.numbers limit 10; -select [0, 0, 4, 11, 0, 0, 0, 0, 0, 11, 2, 4, 6, 0, 0, 1, 2, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('тжрмчпваухрхуфбгнифгбопфт'), ['дпмгкекщлнемссаицщпащтиуцхкфчихтц', 'акйиуоатунтчф', 'мчпва', 'рхуфбгнифгб', 'кнаишж', 'пчвотенеафкухжцешбцхг', 'опеа', 'ушчадфтчхечеуркбтел', 'ашшптаударчжчмвалтдхкимищпф', 'рхуфбгниф', 'ж', 'мчпваухрхуфбгнифг', 'пваухрху', 'зргачбтцдахвймсбсврбндзтнущхвп', 'асбфцавбгуолг', 'тж', 'жрмчпваухрх', 'мрвзцгоб', 'чрцснчсдхтзжвнздзфцвхеилишдбж', 'кчт']) from system.numbers limit 10; -select [0, 2, 4, 0, 6, 0, 0, 0, 0, 19, 7, 1, 0, 1, 0, 0, 2, 10, 0, 1] = multiSearchAllPositionsUTF8(materialize('опрурпгабеарушиойцрхвбнсщ'), ['йошуоесдщеж', 'пр', 'урпгабеарушиой', 'хщиаршблашфажщметчзи', 'пгабеарушиойцрхвб', 'щцбдвц', 'еечрззвкожзсдурйщувмцйшихдц', 'офхачгсзашфзозрлба', 'айдфжджшжлрргмабапткбцпиизигдтс', 'рх', 'габ', '', 'цнкдбфчщшмчулврбцчакщвзхлазфа', '', 'екбтфпфилсаванхфкмчнпумехиищди', 'епвщхаклшомвцжбф', 'прурпгабе', 'еарушиойцрхв', 'црвтгрзтитц', 'опрурпг']) from system.numbers limit 10; -select [0, 10, 1, 0, 0, 0, 0, 0, 10, 0, 15, 2] = multiSearchAllPositionsUTF8(materialize('угпщлзчжшбзвууцшатпщцр'), ['цоуарцжсз', 'бз', '', 'пщфтзрч', 'лфуипмсдмнхнгйнтк', 'айжунцйбйцасчфдхй', 'щдфщлцптплсачв', 'грв', 'бзвууц', 'бумййшдшфашцгзфвчвзвтсувнжс', 'цшатпщ', 'гпщлзчжшб']) from system.numbers limit 10; -select [0, 15, 0, 1, 5, 0, 0, 5, 0, 0, 0, 1, 0, 0] = multiSearchAllPositionsUTF8(materialize('цнлеодлмдцдйснитвдчтхжизв'), ['ивкчсзшугоцжчохщцабл', 'итвдчт', 'кнх', '', 'одлм', 'ктшфзбщзцуймагсоукщщудвуфо', 'ххеаефудгчхр', 'одлмдцдйснитвдчт', 'умцлпкв', 'зщсокйтцзачщафвбповжгнлавсгйг', 'бкибм', '', 'охсоихнцчцшевчеележтука', 'фаийхгжнсгищгщц']) from system.numbers limit 10; -select [0, 0, 0, 2, 0, 0, 0, 0, 3, 2, 3, 6, 0, 0, 0, 12, 4, 1] = multiSearchAllPositionsUTF8(materialize('бгдбувдужщвоошлтчрбй'), ['щвбаиф', 'итчднесжкчжвпжйвл', 'мм', 'г', 'хктзгтзазфгщшфгбеулцмдмдбдпчзх', 'сфуак', 'злйфцощегзекщб', 'фшлдтолрщфзжчмих', 'дбувдужщ', 'гдб', 'дбувдужщ', 'в', 'лчищкечнжщисцичбнзшмулпмлп', 'чжцсгмгфвлиецахзнрбмщин', 'обпжвй', 'о', 'бувдужщвоош', '']) from system.numbers limit 10; -select [0, 2, 5, 3, 2, 0, 1, 0, 0, 4, 2, 0, 0, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('шсушлорзфжзудбсейенм'), ['чнзпбновтршеумбвщчлх', 'су', 'лорзфж', 'ушлорзфжзудб', 'сушлорзфжзудбсейенм', 'ткдрхфнб', '', 'пщд', 'чбдцмщ', 'шлорзфж', 'су', 'сккигркедчожжемгнайвйчтдмхлтти', 'мц', 'пхнхрхйцйсйбхчлктз', 'иафжстлйфцр', 'алщщлангнбнйхлшлфшйонщек']) from system.numbers limit 10; -select [12, 1, 0, 5, 0, 10, 1, 0, 7, 4, 0, 1, 12, 1, 1, 1, 0, 1, 15, 0] = multiSearchAllPositionsUTF8(materialize('ощзллчубоггцвжриуардрулащйпу'), ['цвжр', '', 'нмзкаиудзтиффззшзканжвулт', 'лчубоггцвжриуардрулащйпу', 'чтцлзшуижолибаоххвшихбфжйхетивп', 'ггцвжри', '', 'йдгнвс', 'у', 'л', 'зпщнжуойдлдвхокцжнзйсйзе', '', 'цв', '', '', '', 'ехлцзгвф', '', 'риу', 'уйжгтжноомонгщ']) from system.numbers limit 10; -select [0, 12, 13, 20, 0, 1, 0, 0, 3, 4] = multiSearchAllPositionsUTF8(materialize('цбкифйтшузажопнжщарбштвдерзтдш'), ['щлмлижтншчсмксгтнсврро', 'жопнжщарбштвд', 'опнжщарб', 'бштвдерзтд', 'пуфслейщбкжмпнш', 'ц', 'маве', 'кмйхойрдлшцхишдтищвйбцкщуигваещгтнхйц', 'кифй', 'и']) from system.numbers limit 10; -select [0, 6, 0, 0, 0, 8, 0, 3, 6, 0] = multiSearchAllPositionsUTF8(materialize('еачачгбмомоххкгнвштггпчудл'), ['ндзчфчвжтцщпхщуккбеф', 'г', 'рткнфвчтфннхлжфцкгштймгмейжй', 'йчннбщфкщф', 'лсртщиндшшкичзррущвдйвнаркмешерв', 'момоххк', 'рфафчмсизлрхзуа', 'ч', 'гбмомоххкгнвштг', 'валжпошзбгзлвевчнтз']) from system.numbers limit 10; -select [0, 0, 10, 0, 8, 13, 0, 0, 19, 15, 3, 1] = multiSearchAllPositionsUTF8(materialize('зокимчгхухшкшмтшцчффвззкалпва'), ['цалфжажщщширнрвтпвмщжннрагвойм', 'оукзрдцсадешжмз', 'хшкшмтшцч', 'ауилтсаомуркпаркбцркугм', 'хухшкшмтшцчффв', 'шмтшцч', 'зщгшпцхзгцншднпеусмтжбцшч', 'щлраащсйлщрд', 'ффвзз', 'тшцчффвззкалпв', 'кимчгхухшкш', '']) from system.numbers limit 10; -select [0, 0, 1, 0, 6, 0, 6, 0, 5, 0, 13, 0, 0, 6] = multiSearchAllPositionsUTF8(materialize('йдйндиибщекгтчбфйдредпхв'), ['тдршвтцихцичощнцницшдхйбогбчубие', 'акппакуцйсхцдххнотлгирввоу', '', 'улщвзхохблтксчтб', 'и', 'ибейзчшклепзриж', 'иибщекгт', 'шидббеухчпшусцнрз', 'диибщекгтчбфйд', 'дейуонечзйзлдкшщрцйбйклччсцуй', 'тч', 'лшицлшме', 'чйнжчоейасмрщегтхвйвеевбма', 'ии']) from system.numbers limit 10; -select [15, 3, 3, 2, 0, 11, 0, 0, 0, 2, 0, 4, 0, 1, 1, 3, 0, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('нхгбфчшджсвхлкхфвтдтлж'), ['хфвтдтлж', 'гбфчшд', 'гбфчш', 'х', 'ачдгбккжра', 'вхлк', 'мщчвещлвшдщпдиимлшрвнщнфсзгщм', 'жчоббгшзщлгеепщжкчецумегпйчт', 'жжд', 'хг', 'мтсааролшгмоуйфйгщгтрв', 'бфчшд', 'чейрбтофпшишгуасоодлакчдф', 'н', 'нхгбфч', 'гбф', 'гдежсх', 'йифжацзгжбклх', 'ещпзущпбаолплвевфиаибшйубйцсзгт', 'жезгчжатзтучжб']) from system.numbers limit 10; -select [0, 10, 1, 0, 0, 0, 4, 0, 13, 1, 12, 1, 0, 6] = multiSearchAllPositionsUTF8(materialize('акбдестрдшерунпвойзв'), ['нркчх', 'шерунп', '', 'зжвахслфббтоиоцрзаззасгнфчх', 'шлжмдг', 'тлйайвцжчсфтцйрчосмижт', 'дестрдшерунп', 'мвамйшцбдщпчлрщд', 'у', 'акбдестрд', 'рунпвойз', '', 'айздцоилсйшцфнчтхбн', 'с']) from system.numbers limit 10; -select [1, 0, 0, 3, 2, 1, 0, 0, 1, 10, 7, 0, 5, 0, 8, 4, 1, 0, 8, 1] = multiSearchAllPositionsUTF8(materialize('кйхпукаеуддтйччхлнпсуклрф'), ['кйхпукаеуддтйччхл', 'йатлрйкстлхфхз', 'фгихслшкж', 'хпу', 'йхпукаеу', '', 'сруакбфоа', 'оажуз', 'кйхпукаеуддтйччх', 'ддтйччхлн', 'аеуддтйччхл', 'тмажиойщтпуцглхфишеиф', 'укаеуддтйччхлнпс', 'ретифе', 'еуддтйччхлнпсуклр', 'пукаеуд', 'кйхпу', 'таппфггвджлцпжшпишбпциуохсцх', 'еуд', '']) from system.numbers limit 10; -select [2, 3, 3, 16, 5, 13, 0, 0, 0, 18, 0, 6, 0, 16, 0, 10, 3, 0] = multiSearchAllPositionsUTF8(materialize('плврйщовкзнбзлбжнсатрцщщучтйач'), ['лврйщовкзнбзлбж', 'врйщовкзнбзлбжнса', 'врйщовкзнбз', 'жнсатрцщщучтйач', 'йщовкзнбзлбжнсатрцщщуч', 'злбжнсатрцщ', 'ввтбрдт', 'нжйапойг', 'ннцппгперхйвдхоеожупйебочуежбвб', 'сатрцщщу', 'деваийтна', 'щ', 'вкжйгкужжгтевлцм', 'жнс', 'датг', 'знбзлбжнсатрцщщучтйа', 'врйщовк', 'оашмкгчдзщефм']) from system.numbers limit 10; -select [3, 1, 19, 1, 0, 0, 0, 0, 11, 3, 0, 0] = multiSearchAllPositionsUTF8(materialize('фчдеахвщжхутхрккхасвсхепщ'), ['деах', '', 'свсхепщ', '', 'анчнсржйоарвтщмрж', 'нечбтшщвркгд', 'вштчцгшж', 'з', 'у', 'деахвщ', 'ххкцжрвзкжзжчугнфцшуиаклтмц', 'фцкжшо']) from system.numbers limit 10; -select [16, 0, 0, 1, 8, 14, 0, 12, 12, 5, 0, 0, 16, 0, 11, 0] = multiSearchAllPositionsUTF8(materialize('щмнжчввбжцчммчшсрхйшбктш'), ['срхйшбк', 'йлзцнржчууочвселцхоучмщфчмнфос', 'еижлафатшхщгшейххжтубзвшпгзмзцод', '', 'бжцчммчшсрхй', 'чшсрхй', 'влемчммйтителщвзган', 'ммч', 'ммчшсрх', 'чввбж', 'нобзжучшошмбщешлхжфгдхлпнгпопип', 'цгт', 'срхйш', 'лкклмйжтеа', 'чммчшсрхйшбктш', 'йежффзнфтнжхфедгбоахпг']) from system.numbers limit 10; -select [1, 12, 9, 5, 1, 0, 6, 3, 0, 1] = multiSearchAllPositionsUTF8(materialize('кжнщсашдзитдмщцхуоебтфжл'), ['', 'дмщцхуоебт', 'зитдмщцхуоебт', 'сашдзитдмщцхуое', 'кжнщ', 'тхкйтшебчигбтмглшеужззоббдилмдм', 'ашдзитдмщцхуоебтф', 'нщсашдз', 'аузщшр', 'кжнщсашдз']) from system.numbers limit 10; -select [2, 0, 0, 0, 1, 0, 2, 0, 0, 17, 0, 8, 7, 14, 0, 0, 0, 7, 9, 23] = multiSearchAllPositionsUTF8(materialize('закуфгхчтшивзчжаппбжнтслщввущ'), ['а', 'днойвхфрммтж', 'внтлжрхзрпчбтуркшдатннглечг', 'ахиеушжтфкгцщтзхмжнрхдшт', '', 'тцчгрззржмдшйщфдцрбшжеичч', 'а', 'ктиечцпршнфнбчуолипацчдсосцнлфаццм', 'аусрлхдцегферуо', 'ппбжнт', 'жкццуосгвп', 'чтшивзчжаппб', 'хчтшивзчжаппб', 'чжаппбжнтслщ', 'ччрлфдмлу', 'щзршффбфчзо', 'ущуймшддннрхзийлваежщухч', 'хчтши', 'тшивзчжаппбжнтсл', 'слщв']) from system.numbers limit 10; -select [1, 1, 9, 2, 0, 3, 7, 0, 0, 19, 2, 2, 0, 8] = multiSearchAllPositionsUTF8(materialize('мвкзккупнокченйнзкшбдрай'), ['м', '', 'н', 'вкз', 'гдпертшйбртотунур', 'к', 'упнокченйнзкшбдр', 'нфшрг', 'нмждрйббдцлйемжпулдвкещхтжч', 'ш', 'вкзккупнокченйнзкшбдр', 'вкзккупнокченйнзкшбдрай', 'адииксвеавогтйторчтцвемвойшпгбнз', 'пнокченй']) from system.numbers limit 10; -select [15, 0, 0, 1, 12, 1, 0, 0, 1, 11, 0, 4, 0, 2] = multiSearchAllPositionsUTF8(materialize('отарлшпсабждфалпшножид'), ['лпшно', 'вт', 'лпжшосндутхорлиифжаакш', 'отарлшпсабждфалпшнож', 'дфал', '', 'бкцжучншжбгзжхщпзхирртнбийбтж', 'уцвцкшдзревпршурбсвйнемоетчс', '', 'ждфал', 'тлскхрнпмойчбцпфущфгф', 'рлшпсабж', 'нхнмк', 'тарлшпса']) from system.numbers limit 10; -select [0, 2, 0, 20, 0, 17, 18, 0, 1, 1, 21, 1, 0, 1, 6, 26] = multiSearchAllPositionsUTF8(materialize('ачйвцштвобижнзжнчбппйеабтцнйн'), ['сзхшзпетншйисщкшрвйшжуогцвбл', 'чйвцштво', 'евз', 'пй', 'хуждапрахитйажрищуллйзвчт', 'чбппйе', 'бппйеабтцнйн', 'схш', 'а', 'ачйвцштвобижнзжнчбпп', 'йеабтцнй', '', 'ег', '', 'штвобижнзжнчбпп', 'цн']) from system.numbers limit 10; -select [1, 0, 0, 3, 4, 12, 0, 9, 0, 12, 0, 0, 8, 0, 10, 3, 4, 1, 1, 9] = multiSearchAllPositionsUTF8(materialize('жмхоужежйуфцзеусеоднчкечфмемба'), ['', 'идосйксзнщйервосогф', 'тхмсйлвкул', 'хоужежйуф', 'оужежйуфцзеусеоднчкечфм', 'цзеусеоднчкеч', 'бецвдиубххвхйкажуурщщшщфбзххт', 'йуфцзеусеодн', 'мглкфтуеайсржисстнпкгебфцпа', 'цзеусео', 'уехцфучецчгшйиржтсмгхакчшввохочжпухс', 'дчвмсбткзталшбу', 'жйуфцзеусеоднчке', 'ччшщтдбпвчд', 'уфцзеусеоднчкечфмем', 'хоужежйуфцзеусеоднчкечф', 'оуже', '', 'жмхоужежйуфцзеу', 'й']) from system.numbers limit 10; -select [0, 0, 0, 3, 0, 0, 0, 0, 1, 0, 1, 0, 1, 2, 0, 0, 0, 6] = multiSearchAllPositionsUTF8(materialize('лшпцхкмтресзпзйвцфрз'), ['енрнцепацлщлблкццжсч', 'ецжужлуфаееоггрчохпчн', 'зхзнгасхебнаейбддсфб', 'пцхкмтресзпзйв', 'фчетгеодщтавиииухцундпнхлчте', 'шшгсдошкфлгдвкурбуохзчзучбжйк', 'мцщщцп', 'рх', '', 'зйошвщцгхбж', '', 'ввлпнамуцвлпзеух', '', 'шпцхкмтре', 'маабтруздрфйпзшлсжшгож', 'фдчптишмштссщшдшгх', 'оллохфпкаем', 'кмтресзпз']) from system.numbers limit 10; -select [2, 5, 0, 0, 6, 0, 0, 0, 0, 0, 0, 0, 1, 1, 12, 0, 0, 0, 4, 8] = multiSearchAllPositionsUTF8(materialize('есипзсвшемлхчзмйрсфз'), ['с', 'з', 'пщчсмаиахппферзжбпвиибаачй', 'гтщкзоиежав', 'свшемлхчзм', 'шийанбке', 'зхе', 'авркудфаусзквкфффйцпзжщввенттб', 'ножцваушапиж', 'иизкежлщиафицкчщмалнпсащсднкс', 'вчмв', 'кщеурмуужжлшррце', '', '', 'х', 'алзебзпчеложихашжвхмйхрицн', 'тпзмумчшдпицпдшиаог', 'сулксфчоштаййзбзшкджббщшсей', 'пзсвшемлхчзм', 'ш']) from system.numbers limit 10; -select [0, 1, 2, 4, 0, 0, 14, 1, 13, 4, 0, 0, 1, 1] = multiSearchAllPositionsUTF8(materialize('сзиимонзффичвфжоеулсадону'), ['зфтшебтршхддмеесчд', '', 'зиимонзф', 'имон', 'езбдйшжичценлгршщшаумайаицй', 'птпщемтбмднацлг', 'фжоеулса', '', 'вфжоеулсадону', 'имонзфф', 'йщвдфдиркважгйджгжашарчучйххйднпт', 'дй', '', '']) from system.numbers limit 10; -select [12, 0, 24, 0, 9, 0, 1, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('ижсщщрзжфнгццпзкфбвезгбохлж'), ['ццпзкфбвез', 'ацррвхоптаоснулнжкщжел', 'охлж', 'тнсхбпшщнб', 'фнг', 'урйвг', '', 'цохс', 'щбйрйкжчмйзачуефч', 'афа']) from system.numbers limit 10; -select [9, 0, 0, 0, 1, 0, 7, 7, 0, 0, 1, 0, 7, 0, 0, 8, 0, 3, 0, 0] = multiSearchAllPositionsUTF8(materialize('рерфвирачйнашхрмцебфдйааеммд'), ['чйнашхрмцебфдйааеммд', 'сжщзснвкущлжплцзлизаомдизцнжлмййбохрцч', 'еппбжджмримфчйеаолидпцруоовх', 'едтжкоийггснехшсчйлвфбкцжжрчтш', '', 'пжахфднхсотй', 'ра', 'рач', 'вчримуцнхбкуйжрвфиугзфсзг', 'кщфехрххциаашщсифвашгйцвхевцщнйахтбпжщ', '', 'ртщиобчжстовйчфабалзц', 'рачйнашхрмцебфдйаае', 'ощгжосччфкуг', 'гехвжнщжссидмрфчйтнепдсртбажм', 'а', 'ицжлсрсиатевбвнжрдмзцувввтзцфтвгвш', 'рф', 'прсмлча', 'ндлхшцааурмзфгверуфниац']) from system.numbers limit 10; -select [2, 14, 10, 0, 6, 15, 1, 0, 0, 4, 5, 17, 0, 0, 3, 0, 3, 0, 9, 0] = multiSearchAllPositionsUTF8(materialize('влфощсшкщумчллфшшвбшинфппкчуи'), ['лфощ', 'лфшшвбшинфпп', 'умчллфшшвбшинф', 'слмтнг', 'сшкщумчллфшшвбшинф', 'фшшвб', '', 'рчфбчййсффнодцтнтнбцмолф', 'щфнщокхжккшкудлцжрлжкнп', 'ощ', 'щсшкщумчлл', 'швбшинфппкч', 'септзкщотишсехийлоцчапщжшжсфмщхсацг', 'нт', 'фощсшкщумчллфшшвбшинфп', 'нщпдш', 'фощс', 'мивсмча', 'щумч', 'щчйнткжпмгавфтйтибпхх']) from system.numbers limit 10; -select [0, 10, 0, 0, 0, 0, 0, 3, 0, 0, 0, 2, 0, 11, 0, 0] = multiSearchAllPositionsUTF8(materialize('еаиалмзхцгфунфеагшчцд'), ['йнш', 'гфун', 'жлйудмхнсвфхсуедспщбтутс', 'елмуийгдйучшфлтхцппамфклйг', 'евйдецц', 'пчтфцоучфбсйщпвдацмчриуцжлтжк', 'нстмпумчспцвцмахб', 'иалмз', 'зифчп', 'чогфщимоопт', 'фдйблзеп', 'аиа', 'щугмзужзлйдктш', 'фунфеагшч', 'нйхшмсгцфжчхжвхгдхцуппдц', 'асмвмтнрейшгардллмсрзгзфйи']) from system.numbers limit 10; -select [23, 0, 8, 0, 0, 0, 0, 0, 0, 4, 0, 5, 7, 1, 9, 4] = multiSearchAllPositionsUTF8(materialize('зузйфзлхходфрхгтбпржшрктпйхеоп'), ['ктпйхео', 'лжитуддикчсмкглдфнзцроцбзтсугпвмхзллжж', 'х', 'меуфтено', 'фтдшбшрпоцедктсийка', 'кхтоомтбчвеонксабшйптаихжбтирпзшймчемжим', 'чиаущлрдкухцрдумсвивпафгмр', 'фрнпродв', 'тдгтишхйсашвмдгкчбмшн', 'йфзлхходфрхгтбпржшр', 'бежшлрйврзмумеуооплкицхлйажвцчнчсеакм', 'ф', 'лхходфрхгтб', '', 'ходфрхгтбпржшр', 'й']) from system.numbers limit 10; -select [0, 0, 0, 1, 0, 1, 22, 1, 0, 0, 0, 0, 18, 1, 0, 0, 0, 1] = multiSearchAllPositionsUTF8(materialize('чфгвчхчпщазтгмбнплдгщикойчднж'), ['мштцгтмблаезочкхзвхгрбпкбмзмтбе', 'канбжгсшхшз', 'кзинвщйччажацзйнсанкнщ', 'чфгвчхчпщазтгмбнп', 'етйцгтбнщзнржнйхж', '', 'ик', '', 'еизщвпрохдгхир', 'псумйгшфбвгщдмхжтц', 'слмжопинйхнштх', 'йшралцицммбщлквмгхцввизопнт', 'л', 'чфгвчхчпщазтгмбнплдгщ', 'пбзмхжнпгикиищжтшботкцеолчцгхпбвхи', 'хзкцгрмшгхпхуоцгоудойнжлсоййосссмрткцес', 'ажуофйпщратдйцбржжлжнжащцикжиа', '']) from system.numbers limit 10; -select [6, 0, 2, 5, 2, 9, 10, 0, 0, 4, 0, 6, 3, 2] = multiSearchAllPositionsUTF8(materialize('ишогпсисжашфшлйичлба'), ['сисжашфшлй', 'пднещбгзпмшепкфосовбеге', 'шогп', 'пс', 'шогпси', 'жаш', 'аш', 'деисмжатуклдшфлщчубфс', 'грмквкщзур', 'гпсис', 'кйпкбцмисчхдмшбу', 'сисжашф', 'о', 'шо']) from system.numbers limit 10; -select [8, 15, 13, 0, 1, 2, 5, 2, 9, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('нсчщчвсанпрлисблснокзагансхм'), ['анпрлисблснокзагансхм', 'блснокз', 'исб', 'дрмгвснпл', '', 'счщчвса', 'чвсанпрлисблснокзагансх', 'счщчвсанпрлис', 'нпрли', 'пциишуецнймуодасмжсойглретиефо', 'фхимщвкехшлг', 'слщмаимшжчфхзпрцмхшуниврлуйлжмфжц']) from system.numbers limit 10; -select [0, 5, 0, 0, 14, 0, 12, 0, 2, 3, 0, 3, 21, 5] = multiSearchAllPositionsUTF8(materialize('хажуижанндвблищдтлорпзчфзк'), ['щуфхл', 'и', 'фцежлакчннуувпаму', 'щесщжрчиктфсмтжнхекзфс', 'ищдтлорпзчф', 'дееичч', 'блищ', 'гиефгйзбдвишхбкбнфпкддмбтзиутч', 'ажуижа', 'жуижанндвблищдтлорпзчфзк', 'чщщдзетвщтччмудвзчгг', 'ж', 'пзчфз', 'ижанн']) from system.numbers limit 10; -select [0, 0, 0, 9, 15, 0, 0, 0, 1, 3, 0, 0, 1, 0, 10, 0, 4, 0, 0, 7] = multiSearchAllPositionsUTF8(materialize('россроапцмцагвиигнозхзчотус'), ['ошажбчвхсншсвйршсашкм', 'пфдчпдчдмауцгкйдажрйефапввшжлшгд', 'иеаочутввжмемчушлуч', 'цмцагвиигно', 'ииг', 'ммпжщожфйкакбущчирзоммагеиучнщмтвгихк', 'укррхбпезбжууеипрзжсло', 'ншопзжфзббилйбувгпшшиохврнфчч', '', 'ссроап', 'лийщфшдн', 'йчкбцциснгначдцйчпа', 'россроапцмцагвииг', 'кштндцтсшорввжсфщчмщчжфжквзралнивчзт', 'мца', 'нбтзетфтздцао', 'сроа', 'мщсфие', 'дткодбошенищйтрподублжскенлдик', 'апцмцагвиигноз']) from system.numbers limit 10; -select [16, 0, 0, 2, 1, 1, 0, 1, 9, 0, 0, 3] = multiSearchAllPositionsUTF8(materialize('тйсдйилфзчфплсджбарйиолцус'), ['жбарйиолцу', 'цназщжждефлбрджктеглщпунйжддгпммк', 'хгжоашцшсзкеазуцесудифчнощр', 'йс', '', 'тйсдйилфзчфп', 'ивфсплшвслфмлтххжчсстзл', '', 'зчфплсдж', 'йртопзлодбехрфижчдцйс', 'цлащцкенмшеоерееиуноп', 'с']) from system.numbers limit 10; -select [3, 2, 1, 1, 0, 0, 0, 14, 6, 0] = multiSearchAllPositionsUTF8(materialize('нсцннйрмщфбшщховвццбдеишиохл'), ['цннйр', 'сцннйрм', 'н', 'нс', 'двтфхйзгеиеиауимбчхмщрцутф', 'пчтмшйцзсфщзшгнхщсутфжтлпаввфгххв', 'лшмусе', 'ховвццбд', 'йрмщфбшщховвццбдеи', 'гндруущрфзсфжикшзцжбил']) from system.numbers limit 10; -select [0, 18, 0, 1, 2, 0, 0, 0, 1, 7, 10, 0, 1, 0, 2, 0, 0, 18] = multiSearchAllPositionsUTF8(materialize('щидмфрсготсгхбомлмущлаф'), ['тлтфхпмфдлуоцгчскусфжчкфцхдухм', 'мущла', 'емлвзузхгндгафги', '', 'идмфрсготсгхбомлмущла', 'зфаргзлщолисцфдщсеайапибд', 'кдхоорхзжтсйимкггйлжни', 'лчгупсзждплаблаеклсвчвгвдмхклщк', 'щидмфр', 'сготсгхбомлму', 'тсгхбомлмущла', 'хсзафйлкчлди', '', 'й', 'ид', 'щлйпмздйхфзайсщсасейлфцгхфк', 'шдщчбшжбмййзеормнрноейй', 'мущ']) from system.numbers limit 10; -select [0, 13, 0, 0, 1, 0, 7, 7, 8, 0, 2, 0, 3, 0, 0, 13] = multiSearchAllPositionsUTF8(materialize('трцмлщввадлжввзчфипп'), ['хшзйийфжмдпуигсбтглй', 'ввзчфи', 'нсцчцгзегммтсшбатщзузпкшрг', 'гувйддежзфилйтш', '', 'хгзечиа', 'ввадлжввз', 'ввадлжввзчфи', 'ва', 'щтшсамклегш', 'рцмлщ', 'учзмиерфбтцучйдглбщсз', 'цмлщввадлжввзчфи', 'орйжччцнаррбоабцжзйлл', 'квпжматпцсхзузхвмйч', 'ввзчфип']) from system.numbers limit 10; -select [0, 1, 1, 0, 11, 4, 1, 2, 0, 0] = multiSearchAllPositionsUTF8(materialize('инкщблбвнскцдндбмсщщш'), ['жхрбсусахрфкафоилмецчебржкписуз', 'инкщблбвнс', '', 'зисгжфлашймлджинаоджруй', 'кцднд', 'щблбвнскцдндбмсщщ', 'инкщблбвнс', 'н', 'зб', 'фчпупшйфшбдфенгитатхч']) from system.numbers limit 10; -select [6, 0, 4, 20, 1, 0, 5, 0, 1, 0] = multiSearchAllPositionsUTF8(materialize('рзтецуйхлоорйхдбжашнларнцт'), ['у', 'бпгййекцчглпдвсцсещщкакцзтцбччввл', 'ецуйхлоо', 'нлар', 'рз', 'ккнжзшекфирфгсгбрнвжчл', 'цуйхлоорйхдбжашн', 'йнучгрчдлйвводт', 'рзте', 'нткрввтубчлщк']) from system.numbers limit 10; - -select [1, 1, 0, 0, 1, 0, 0, 3, 3, 3, 1, 0, 8, 0, 8, 1, 0, 1] = multiSearchAllPositionsCaseInsensitive(materialize('OTMMDcziXMLglehgkklbcGeAZkkdh'), ['', 'OTmmDCZiX', 'SfwUmhcGTvdYgxlzsBJpikOxVrg', 'ngqLQNIkqwguAHyqA', '', 'VVZPhzGizPnKJAkRPbosoNGJTeO', 'YHpLYTVkHnhTxMODfABor', 'mMdcZi', 'MmdCZI', 'MMdCZixmlg', '', 'hgaQHHHkIQRpPjv', 'ixMLgLeHgkkL', 'uKozJxZBorYWjrx', 'i', '', 'WSOYdEKatHkWiCtlwsCbKRnXuKcLggbkBxoq', '']) from system.numbers limit 10; -select [4, 15, 0, 0, 0, 0, 5, 0, 5, 1, 0, 1, 13, 0, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('VcrBhHvWSFXnSEdYCYpU'), ['bhhVwSfXnSEd', 'DycyP', 'kEbKocUxLxmIAFQDiUNoAmJd', 'bsOjljbyCEcedqL', 'uJZxIXwICFBPDlUPRyDHMmTxv', 'BCIPfyArrdtv', 'hHv', 'eEMkLteHsuwsxkJKG', 'hHVWsFxNseDy', '', 'HsFlleAQfyVVCoOSLQqTNTaA', '', 'sEDY', 'UMCKQJY', 'j', 'rBhHvw']) from system.numbers limit 10; -select [1, 1, 0, 0, 1, 0, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('wZyCLyiWnNNdNAPWeGSQZcdqk'), ['w', '', 'vlgiXgFTplwqRbnwBumAjHvQuM', 'QoIRVKDHMlapLNiIZXvwYxluUivjY', 'WZY', 'gAFpUfPDAwgzARCIMrtbZUsNcR', 'egkLWqqdNiETeETsMG', 'dzSlJaoHKlQmENIboow', 'vPNBhcaIfsgLH', 'mlWPTCBDVTdKHxlvIUVcJXBrmTcJokAls']) from system.numbers limit 10; -select [0, 10, 0, 1, 7, 1, 6, 1, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('pqliUxqpRcOOKMjtrZSEsdW'), ['YhskuppNFdWaTaZo', 'Coo', 'mTEADzHXPeSMCQaYbKpikXBqcfIGKs', 'PQLiUxq', 'qpRCoOK', 'PQLIu', 'XQPrcoOK', '', 'pR', 'cTmgRtcSdRIklNQVcGZthwfarLtAYh']) from system.numbers limit 10; -select [16, 1, 1, 1, 1, 4, 17, 0, 0, 0, 1, 0, 0, 0, 20, 0] = multiSearchAllPositionsCaseInsensitive(materialize('kJyseeDFCeUWoqMfubYqJqWA'), ['fub', 'kJY', '', '', 'Kj', 's', 'uBYQJq', 'sUqCmHUZIBtZPswObXSrYCwrdxdznM', 'mtZDCJENYuikJnCcJfRcSCDYDPXU', 'IDXjRjHhmjqXmCOlQ', '', 'jiEwAxIsJDu', 'YXqcEKbHxlgUliIALorSKDMlGGWeCO', 'OstKrLpYuASEUrIlIuHIRdwLr', 'qJq', 'tnmvMTFvjsW']) from system.numbers limit 10; -select [11, 3, 1, 0, 9, 0, 0, 0, 0, 8, 3, 0] = multiSearchAllPositionsCaseInsensitive(materialize('EBSPtFpDaCIydASuyreS'), ['iyD', 'sptfpdAciyDAsuyR', 'EbS', 'IJlqfAcPMTUsTFXkvmtsma', 'AcIYda', 'fbWuKoCaCpRMddUr', 'srlRzZKeOQGGLtTLOwylLNpVM', 'ZeIgfTFxUyNwDkbnpeiPxQumD', 'j', 'daciydA', 'sp', 'dyGFtyfnngIIbcCRQzphoqIgIMt']) from system.numbers limit 10; -select [6, 0, 0, 0, 10, 0, 1, 4, 0, 15, 0, 2, 2, 6] = multiSearchAllPositionsCaseInsensitive(materialize('QvlLEEsgpydemRZAZcYbqPZHx'), ['eSgpYDEMRzAzcyBQPzH', 'NUabuIKDlDxoPXoZOKbUMdioqwQjQAiArv', 'pRFrIAGTrggEOBBxFmnZKRPtsUHEMUEg', 'CDvyjef', 'YdEMrzaZc', 'BO', '', 'leEsgPyDEmRzaZCYBqPz', 'EzcTkEbqVXaVKXNuoxqNWHM', 'Z', 'cuuHNcHCcLGb', 'V', 'vllEes', 'eS']) from system.numbers limit 10; -select [0, 0, 0, 0, 0, 3, 0, 0, 0, 0, 5, 7, 5, 0, 11, 1] = multiSearchAllPositionsCaseInsensitive(materialize('eiCZvPdGJSmwxMIrZvEzfYFOFJmV'), ['lSydrmJDeXDYHGFFiFOOJGyCbCCDbLzbSbub', 'ewsAVflvcTBQFtvWBwuZOJKkrUArIg', 'fpEkBWaBkRWypFWtMz', 'YatSURyNtcSuerWWlTBSdBNClO', 'YO', 'CZvpdg', 'uoH', 'gtGwQSVqSJDVROmsBIxjuVNfrQnxDhWGXLBH', 'IKNs', 'HElLuRMlsRgINaNp', 'V', 'DGjsMW', 'vPDgJSmW', 'SGCwNiAmNfHSwLGZkRYEqrxBTaDRAWcyHZYzn', 'mWXMiRZvezfYf', '']) from system.numbers limit 10; -select [23, 1, 0, 17, 0, 0, 9, 3, 0, 2] = multiSearchAllPositionsCaseInsensitive(materialize('BizUwoENfLxIIYVDflhOaxyPJw'), ['yPJ', '', 'gExRSJWtZwOptFTkNlBGuxyQrAu', 'FLH', 'hCqo', 'oVGcArersxMUCNewhTMmjpyZYAIU', 'FlXIiYVdflHoAX', 'ZuWOe', 'bhfAfNdgEAtGdHylxkjgvU', 'IZUWo']) from system.numbers limit 10; -select [0, 9, 0, 0, 0, 0, 1, 0, 0, 1, 3, 0, 13, 0, 3, 5] = multiSearchAllPositionsCaseInsensitive(materialize('loKxfFSIAjbRcguvSnCdTdyk'), ['UWLIDIermdFaQVqEsdpPpAJ', 'ajBrcg', 'xmDmuYoRpGu', 'wlNjlKhVzpC', 'MxIjTspHAQCDbGrIdepFmLHgQzfO', 'FybQUvFFJwMxpVQRrsKSNHfKyyf', '', 'vBWzlOChNgEf', 'DiCssjczvdDYZVXdCfdSDrWaxmgpPXDiD', '', 'kxFFSIAjBRCGUVSNcD', 'LrPRUqeehMZapsyNJdu', 'cGuVSNcdTdy', 'NmZpHGkBIHVSoOcj', 'KxffSIAjBr', 'ffsIaJB']) from system.numbers limit 10; -select [14, 0, 11, 0, 10, 0, 0, 0, 13, 1, 2, 11, 5, 0] = multiSearchAllPositionsCaseInsensitive(materialize('uijOrdZfWXamCseueEbq'), ['sE', 'VV', 'AmcsEu', 'fUNjxmUKgnDLHbbezdTOzyLaknQ', 'XAmCsE', 'HqprIpxIcOTkDIKcVK', 'NbmirQlNsTHnAVKlF', 'VVDNOxFKSnQGKPsTqgtwLhZnIPkL', 'c', '', 'IJ', 'aM', 'rDzF', 'YFwP']) from system.numbers limit 10; -select [0, 8, 17, 0, 1, 0, 0, 0, 0, 0, 5, 0] = multiSearchAllPositionsCaseInsensitive(materialize('PzIxktujxHZsaDlwSGQPgvA'), ['zrYlZdnUxlPrVJJeZEASwdCHlNEm', 'jxhZS', 'sGQPgV', 'MZMChmRBgsxhdgspUhALoxmrkZVp', 'pzIxktuJxHzsADlw', 'xavwOAibQuoKg', 'vuuETOrWLBNLhrMeWLgGQpeFPdcWmWu', 'TZrAgmdorqZIdudhyCMypHYKFO', 'ztcCyGxRKrcUTv', 'OUvwdMZrcZuwGtjuEBeGU', 'k', 'rFTpnfGIOCfwktWnyOMeXQZelkYwqZ']) from system.numbers limit 10; -select [3, 1, 4, 1, 0, 17, 13, 0, 0, 0, 0, 0, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('pUOaQLUvgmqvxaMsfJpud'), ['OaqLUvGm', '', 'aQ', '', 'VajqJSlkmQTOYcedjiwZwqNH', 'f', 'xaMsfj', 'CirvGMezpiIoacBGAGQhTJyr', 'vucKngiFjTlzltKHexFVFuUlVbey', 'ppalHtIYycBCEjsgsXbFeecpkQMNr', 'nEgIYVoGkhTsFgBUSHJvIcYCYbuOBP', 'efjBVRVzknGrikGHxExlFEtYf', 'v', 'QgRBCaGlwNYWRslDylOrfPxZxAOF']) from system.numbers limit 10; -select [14, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 1, 0, 20, 5, 0, 4, 0] = multiSearchAllPositionsCaseInsensitive(materialize('WZNWOCjFkCAAzIptkUtyPCyC'), ['iPTkuT', 'BngeNlFbKymzMYmNPfV', 'XKEjbLtADFMqS', 'dbRQKJGSFhzljAiZV', 'wZnwoCjFKCAAzIPTKuTYpc', 'yBaUvSSGOEL', 'iEYopROOYKxBwPdCgbPNPAsMwVksHgagnO', 'TljXPJVebHqrnhSiTGwpMaNeKy', 'wzNWocjF', 'bLxLrZnOCeIfxkfZEOcqDteUvc', 'CtHYpAZDANEv', '', 'XMAMpGYMiOb', 'y', 'o', 'floswnnFjXDTxantSvDYPSnaORL', 'WOcjFkcAaZIp', 'buqBHbZsLDnCUDhLdgd']) from system.numbers limit 10; -select [0, 20, 14, 0, 2, 0, 1, 14, 0, 0, 0, 1, 0, 26, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('XJMggEHaxfddDadtwKMCcPsMlSFVJ'), ['NzbUAZvCsnRnuzTglTsoT', 'ccP', 'ADTwKmc', 'JaUzcvWHMotuEMUtjsTfJzrsXqKf', 'jMGgEHaXfdddAdTWKMCcpsM', 'SMnb', '', 'AdTWkMccPSMlsfv', 'fVjPVafkp', 'goqsYAFqhhnCkGwhg', 'CNHNPZHZreFwhRMr', '', 'vcimNhmdbtoiCgVzNuvdgZG', 'sfvJ', 'AqKmroxmRMSFAKjfhwrzxmNSSjMHxKow', 'Xhub']) from system.numbers limit 10; -select [0, 0, 7, 0, 1, 1, 0, 0, 13, 0, 1, 1, 5, 0] = multiSearchAllPositionsCaseInsensitive(materialize('VQuEWycGbGcTcCCvWkujgdoWjKgVYy'), ['UevGaXmEAtBdWsPhBfqp', 'aQOrNMPmoVGSu', 'c', 'TMhzvbNJCaxtGNUgRBmTFEqgNBIBpSJ', '', 'vq', 'pVNUTCqXr', 'QSvkansbdPbvVmQpcQXDk', 'cCCvwkUjgdOWjKgVYy', 'EtCGaEzsSbJ', 'V', '', 'WycgBgCTCcCvwkujgdoWJKgv', 'xPBJqKrZbZHJawYvPxgqrgxPN']) from system.numbers limit 10; -select [4, 1, 0, 0, 0, 0, 0, 0, 0, 18] = multiSearchAllPositionsCaseInsensitive(materialize('LODBfQsqxfeNuoGtzvrUMRVWNKUKKs'), ['Bf', 'lOdbfQs', 'ZDSDfKXABsFiZRwsebyU', 'DT', 'GEUukPEwWZ', 'GNSbrGYqEDWNNCFRYokZbZEzGzc', 'kYCF', 'Kh', 'jRMxqdmGYpTkePeReXJNdnxagceitMJlmbbro', 'VrumrvWnKU']) from system.numbers limit 10; -select [1, 1, 3, 1, 10, 0, 9, 2, 2, 0, 0, 0, 0, 0, 8, 0, 1, 11, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('lStPVtsQypFlZQoQhCuP'), ['', '', 'tpV', 'L', 'PF', 'pGPggwbkQMZandXugTpUorlPOubk', 'yPFlz', 'sTPVTsQyPfLzQOqhCU', 'StPVtSq', 'cbCxBjAfJXYgueqMFNIoSguFm', 'AosIZKMPduRfumDZ', 'AGcNTHObH', 'oPaGpsQ', 'kwQCczyY', 'q', 'HHUYdzGAzVJyn', '', 'fLZQoqHcUp', 'q', 'SSonzfqLVwIGzdHtj']) from system.numbers limit 10; -select [0, 1, 2, 0, 0, 0, 13, 1, 27, 1, 0, 1, 3, 1, 0, 1, 3, 0] = multiSearchAllPositionsCaseInsensitive(materialize('NhKJtvBUddKWpseWwRiMyBsTWmlk'), ['toBjODDZoRAjFeppAdsne', '', 'HKjTvBu', 'QpFOZJzUHHQAExAqkdoBpSbXzPnTzuPd', 'gE', 'hLmXhcEOwCkatUrLGuEIJRkjATPlqBjKPOV', 'Ps', 'NH', 'l', '', 'aSZiWpmNKfglqAbMZpEwZKmIVNjyJTtDianY', 'NhKJTvBUDDkwpS', 'KJtvbUDDKWPSewwrimYbstwm', 'NHKJTvbudDKwpSEwwR', 'hmMeWEpksVAaXd', 'NHkJTvBUDd', 'kjTvbudd', 'kmwUzfEpWSIWkEylDeRPpJDGb']) from system.numbers limit 10; -select [0, 5, 0, 0, 0, 1, 1, 15, 2, 3, 4, 5] = multiSearchAllPositionsCaseInsensitive(materialize('NAfMyPcNINKcgsShJMascJunjJva'), ['ftHhHaJoHcALmFYVvNaazowvQlgxwqdTBkIF', 'yp', 'zDEdjPPkAdtkBqgLpBfCtsepRZScuQKbyxeYP', 'yPPTvdFcwNsUSeqdAUGySOGVIhxsJhMkZRGI', 'JQEqJOlnSSam', 'nAFmy', '', 'sHJmaScjUnJj', 'afmY', 'FmYpcnINKCg', 'MYPCniNkcgSS', 'YPCNiNkCgSsHjmasCJuNjJ']) from system.numbers limit 10; -select [0, 0, 6, 3, 2, 0, 8, 2, 2, 10, 0, 0, 14, 0, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('hgpZVERvggiLOpjMJhgUhpBKaN'), ['Nr', 'jMcd', 'e', 'PZVeRvggiLOPjmjh', 'GpZVe', 'cVbWQeTQGhYcWEANtAiihYzVGUoHKH', 'VGgilOPj', 'GPZVervgGiLopjmjHGuHp', 'GP', 'gil', 'fzwDPTewvwuCvpxNZDi', 'gLLycXDitSXUZTgwyeQgMSyC', 'PJmjh', 'bTQdrFiMiBtYBcEnYbKlqpTvGLmo', 'ggHxiDatVcGTiMogkIWDxmNnKyVDJth', 'pzv']) from system.numbers limit 10; -select [7, 1, 9, 3, 0, 0, 2, 0, 1, 11] = multiSearchAllPositionsCaseInsensitive(materialize('xUHVawrEvgeYyUZGmGZejClfinvNS'), ['RevGeYyuz', 'XUHvAWrev', 'Vg', 'hvawR', 'eRQbWyincvqjohEcYHMwmDbjU', 'nuQCxaoxEdadhptAhZMxkZl', 'UhVAwREvGEy', 'lHtwTFqlcQcoOAkujHSaj', '', 'eYYUzgMgzEjCLfIn']) from system.numbers limit 10; -select [0, 0, 8, 5, 9, 1, 0, 4, 12, 6, 4, 0, 0, 12] = multiSearchAllPositionsCaseInsensitive(materialize('DbtStWzfvScJMGVPQEGkGFoS'), ['CSjYiEgihaqQDxZsOiSDCWXPrBdiVg', 'aQukOYRCSLiildgifpuUXvepbXuAXnYMyk', 'fvsCjmgv', 'TWZFV', 'VscjMgVpQ', 'dBtSTwZfVsCjmGVP', 'wqpMklzJiEvqRFnZYMfd', 'StwZfVScJ', 'j', 'wzfVsCjmGV', 'STWZfVS', 'kdrDcqSnKFvKGAcsjcAPEwUUGWxh', 'UtrcmrgonvUlLnzWXvZI', 'jMgvP']) from system.numbers limit 10; -select [0, 0, 0, 0, 7, 3, 0, 11, 1, 10, 0, 0, 7, 1, 4, 0, 17, 3, 15, 0] = multiSearchAllPositionsCaseInsensitive(materialize('YSBdcQkWhYJMtqdEXFoLfDmSFeQrf'), ['TnclcrBJjLBtkdVtecaZQTUZjkXBC', 'SPwzygXYMrxKzdmBRTbppBQSvDADMUIWSEpVI', 'QnMXyFwUouXBoCGLtbBPDSxyaLTcjLcf', 'dOwcYyLWtJEhlXxiQLRYQBcU', 'KWhYjMtqdEXFo', 'BD', 'nnPsgvdYUIhjaMRVcbpPGWOgVjJxoUsliZi', 'j', '', 'YjmtQdeXF', 'peeOAjH', 'agVscUvPQNDwxyFfXpuUVPJZOjpSBv', 'kWh', '', 'dcQKWHYjmTQD', 'qjWSZOgiTCJyEvXYqaPFqbwvrwadJsGVTOhD', 'xfoL', 'b', 'DeXf', 'HyBR']) from system.numbers limit 10; -select [4, 0, 0, 13, 1, 0, 3, 13, 16, 1, 0, 1, 16, 1, 12, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('SoVPMQNqmaTGuzYxDvZvapSuPiaP'), ['pMqNQMAtGuzYxDVz', 'TEJtgLhyredMnIpoZfmWvNwpkxnm', 'XRWmsfWVOCHhk', 'u', '', 'HvkXtxFdhVIyccpzFFSL', 'VPM', 'uZyXDVzvAPsUpIaP', 'xDvzV', 'sovpmqNQmATguZYx', 'wEG', 'soVPmQnQ', 'XDVzV', '', 'GUZyXdvzva', 'FetUahWwGtwEpVdlJCJntL', 'B', 'lSCUttZM']) from system.numbers limit 10; -select [1, 0, 1, 2, 15, 0, 0, 0, 1, 0] = multiSearchAllPositionsCaseInsensitive(materialize('zFWmqRMtsDjSeWBSFoqvWsrV'), ['', 'GItrPyYRBwNUqwSaUBpbHJ', '', 'f', 'BsfOQvWsR', 'JgvsMUZzWaddD', 'wxRECkgoCBPjSMRorZpBwuOQL', 'xHKLLxUoWexAM', '', 'YlckoSedfStmFOumjm']) from system.numbers limit 10; -select [11, 1, 1, 1, 0, 0, 1, 0, 4, 0, 0, 0, 1, 0, 5, 8] = multiSearchAllPositionsCaseInsensitive(materialize('THBuPkHbMokPQgchYfBFFXme'), ['KpqGchyfBF', '', '', 'TH', 'NjnC', 'ssbzgYTybNDbtuwJnvCCM', 'tHbupKHBMOkPQgcHy', 'RpOBhT', 'uPKHbMoKpq', 'oNQLkpSKwocBuPglKvciSjttK', 'TaCqLisKvOjznOxnTuZe', 'HmQJhFyZrcfeWbXVXsnqpcgRlg', 'tHB', 'gkFGbYje', 'pkhbMokPq', 'Bm']) from system.numbers limit 10; -select [7, 10, 0, 0, 9, 0, 0, 3, 0, 10] = multiSearchAllPositionsCaseInsensitive(materialize('ESKeuHuVsDbiNtvxUrfPFjxblv'), ['uvsDBiNtV', 'DbInTvxu', 'YcLzbvwQghvrCtCGTWVuosE', 'cGMNo', 'SDb', 'nFIRTLImfrLpxsVFMBJKHBKdSeBy', 'EUSiPjqCXVOFOJkGnKYdrpuxzlbKizCURgQ', 'KeUHU', 'gStFdxQlrDcUEbOlhLjdtQlddJ', 'DBInTVx']) from system.numbers limit 10; -select [1, 0, 2, 18, 1, 3, 15, 8, 0, 0, 1, 3, 0, 23, 2, 0, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('TzczIDSFtrkjCmDQyHxSlvYTNVKjMT'), ['', 'AmIFsYdYFaIYObkyiXtxgvnwMVZxLNlmytkSqAyb', 'ZcZI', 'HXsLVYTnvKjm', '', 'CZiDsFtRKJ', 'DQYhxSl', 'fTRKjCmdqYHxsLvYtNvk', 'hxVpKFQojYDnGjPaTNPhGkRFzkNhnMUeDLKnd', 'RBVNIxIvzjGYmQBNFhubBMOMvInMQMqXQnjnzyw', '', 'c', 'vcvyskDmNYOobeNSfmlWcpfpXHfdAdgZNXzNm', 'ytnvKJM', 'ZcZidsFtRKjcmdqy', 'IRNETsfz', 'fTR', 'POwVxuBifnvZmtBICqOWhbOmrcU']) from system.numbers limit 10; -select [14, 16, 10, 2, 6, 1, 0, 8, 0, 0, 12, 1, 0, 1, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('tejdZOLhjpFLkGBWTGPfmk'), ['GBWtgPF', 'Wt', 'PflkgBWTgpFmK', 'ejdZOLhJPFlKgb', 'o', 'TejDZ', 'HlQfCP', 'hJP', 'ydiyWEfPGyRwcKGfGVdYxAXmkY', 'QsOyrgkTGMpVUAmLjtnWEIW', 'LKGBw', 'tejDzolHJpFLKgbWT', 'IK', '', 'WrzLpcmudcIJEBapkToDbYSazKTwilW', 'DmEWOxoieDsQHYsLNelMc']) from system.numbers limit 10; -select [9, 0, 1, 4, 13, 0, 0, 1, 3, 7, 9, 0, 1, 1, 0, 7] = multiSearchAllPositionsCaseInsensitive(materialize('ZWHpzwUiXxltWPAIGGxIcJB'), ['XxLTWpA', 'YOv', '', 'pzwUIXXl', 'wp', 'lpMMLDAuflLnWMFrETXRethzCUZOWfQ', 'la', '', 'HPZ', 'UixxlTw', 'xXLTWP', 'YlfpbSBqkbddrVwTEmXxgymedH', '', '', 'QZWlplahlCRTMjmNBeoSlcBoKBTnNZAS', 'UiXxlTwPAiGG']) from system.numbers limit 10; -select [0, 9, 6, 0, 4, 0, 3, 0, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('NytxaLUvmiojEepjuCzwUYPoWL'), ['LcOnnmjbZSifx', 'm', 'lUvMIOjeE', 'vuZsNMSsutiLCDbClPUSsrziohmoZaQeXtKG', 'XaLuvm', 'hlUevDfTSEGOjvLNdRTYjJQvMvwrMpwy', 'TXALuVmioJeePjUczw', 'pKaQKZg', 'PAdX', 'FKLMfNAwNqeZeWplTLjd', 'DODpbzUmMCzfGZwfkjH', 'HMcEGRHLspYdJIiJXqwjDUBp']) from system.numbers limit 10; -select [2, 1, 0, 16, 8, 1, 6, 0, 0, 1, 8, 0, 7, 0, 9, 1, 1, 0, 0, 1] = multiSearchAllPositionsCaseInsensitive(materialize('WGVvkXuhsbzkLqiIEOuyiRfomy'), ['GVv', '', 'VbldWXHWzdziNcJKqIkDWrO', 'iEOUyIRFomy', 'hsBZklqiieOuy', '', 'X', 'emXjmIqLvXsNz', 'rxhVkujX', 'wgvvK', 'HsBzKLQiie', 'wVzJBMSdKOqjiNrXrfLEjjXozolCgYv', 'UHsbzklQiiEouyirf', 'UOvUsiKtUnwIt', 'SBZKLqiIEoUYIrfom', 'wg', '', 'BefhETEirL', 'WyTCSmbKLbkQ', '']) from system.numbers limit 10; -select [8, 1, 2, 8, 1, 0, 5, 0, 0, 4, 0, 1, 14, 0, 0, 7, 0, 1] = multiSearchAllPositionsCaseInsensitive(materialize('uyWhVSwxUFitYoVQqUaCVlsZN'), ['XufitYOVqqUACVlszn', '', 'ywH', 'XUFIT', 'uywHvSWXuFIt', 'dGhpjGRnQlrZhzGeInmOj', 'vswXuFitYovqQuA', 'dHCfJRAAQJUZeMJNXLqrqYCygdozjAC', 'rojpIwYfNLECl', 'hVswxufiTYov', 'bgJdgRoye', '', 'ovQ', 'AdVrJlq', 'krJFOKilvBTGZ', 'WxuFITYOV', 'AsskQjNPViwyTF', 'u']) from system.numbers limit 10; -select [0, 2, 0, 0, 0, 6, 0, 5, 0, 15, 0, 0, 3, 0] = multiSearchAllPositionsCaseInsensitive(materialize('BEKRRKLkptaZQvBxKoBL'), ['HTwmOxzMykTOkDVKjSbOqaAbg', 'eKrRKl', 'UrLKPVVwK', 'TyuqYmTlQDMXJUfbiTCr', 'fyHrUaoMGdq', 'KLkPtaZq', 'cPUJp', 'RKLk', 'yMnNgUOpDdP', 'BX', 'tXZScAuxcwYEfSKXzyfioYPWsrpuZz', 'dsiqhlAKbCXkyTjBbXGxOENd', 'k', 'juPjORNFlAoEeMAUVH']) from system.numbers limit 10; -select [9, 0, 0, 0, 1, 4, 2, 0, 0, 0, 0, 8, 0, 2, 0, 3, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('PFkLcrbouhBTisTkuUcO'), ['UhBtistKU', 'ioQunYMFWHD', 'VgYHTKZazRtfgRtvywtIgVoBqNBwVn', 'ijSNLKch', 'pFKlcrBOuhbtIsTku', 'lCRboUHBtI', 'fKLCRBOu', 'XTeBYUCBQVFwqRkElrvDOpZiZYmh', 'KzXfBUupnT', 'OgIjgQO', 'icmYVdmekJlUGSmPLXHc', 'OuH', 'BWDGzBZFhTKQErIRCbtUDIIjzw', 'F', 'LuWyPfSdNHIAOYwRMFhP', 'kL', 'PQmvXDCkEhrlFBkUmRqqWBxYi', 'kLcrbo']) from system.numbers limit 10; -select [0, 1, 1, 6, 14, 3, 0, 1, 9, 1, 9, 0, 1, 10, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('pfynpJvgIjSqXWlZzqSGPTTW'), ['ZzeqsJPmHmpoYyTnKcWJGReOSUCITAX', '', 'P', 'jvGIj', 'wLZzQsgP', 'YnPjVGij', 'DmpcmWsyilwHwAFcKpLhkiV', '', 'I', 'pFy', 'IjsqxwLZzqSgpT', 'pKpe', 'PfynpJvgiJSqXwlzZ', 'jsQXwLZZqs', 'onQyQzglEOJwMCO', 'GV']) from system.numbers limit 10; -select [1, 17, 1, 20, 0, 0, 5, 0, 0, 0, 24, 0] = multiSearchAllPositionsCaseInsensitive(materialize('BLNRADHLMQstZkAlKJVylmBUDHqEVa'), ['bLnRaDhLm', 'kJVYlmbuD', 'bLnr', 'yLMbU', 'eAZtcqAMoqPEgwtcrHTgooQcOOCmn', 'jPmVwqZfp', 'aDHlmqS', 'fmaauDbUAQsTeijxJFhpRFjkbYPX', 'aqIXStybzbcMjyDKRUFBrhfRcNjauljlqolfDX', 'WPIuzORuNbTGTNb', 'uDhqeVa', 'fQRglSARIviYABcjGeLK']) from system.numbers limit 10; -select [2, 0, 4, 5, 1, 15, 1, 9, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('BEwjMzphoTMoGikbrjTVyqDq'), ['E', 'sClgniMsZoGTEuLO', 'jmzphotmoGIKBRjtv', 'MZPhOtmo', '', 'Kb', '', 'otm', 'tVpxYRttoVpRLencV', 'SJAhAuMttGaeMsalRjeelAGG']) from system.numbers limit 10; -select [1, 0, 0, 0, 0, 0, 4, 0, 0, 19, 0, 7] = multiSearchAllPositionsCaseInsensitive(materialize('yNnYRQfcyemQdxUEPOiwRn'), ['', 'SJteoGNeIAMPWWBltkNKMrWDiVfR', 'kKnnKQhIPiekpnqTXJuyHfvWL', 'GPDUQEMWKzEEpvjLaIRYiuNfpzxsnSBX', 'oPrngRKwruyH', 'ukTSzFePSeVoeZeLQlAaOUe', 'yRqfcyemQDXUepo', 'CwmxidvpPHIbkJnVfSpbiZY', 'FUxmQdFVISApa', 'iwr', 'ciGHzDpMGNQbytsKpRP', 'Fcy']) from system.numbers limit 10; -select [0, 1, 0, 11, 2, 0, 1, 3, 0, 0, 0, 21] = multiSearchAllPositionsCaseInsensitive(materialize('EgGWQFaRsjTzAzejYhVrboju'), ['DVnaLFtCeuFJsFMLsfk', '', 'thaqudWdT', 'Tzazejy', 'GGW', 'RolbbeLLHOJpzmUgCN', '', 'gwqfarsjtzaZeJYHvR', 'KkaoIcijmfILoe', 'UofWvICTEbwVgISstVjIzkdrrGryxNB', 'UJEvDeESWShjvsJeioXMddXDkaWkOiCV', 'B']) from system.numbers limit 10; -select [0, 5, 2, 0, 0, 7, 0, 0, 0, 11, 0, 12, 22, 10, 0, 12] = multiSearchAllPositionsCaseInsensitive(materialize('ONgpDBjfRUCmkAOabDkgHXICkKuuL'), ['XiMhnzJKAulYUCAUkHa', 'dbj', 'nGpDbJFRU', 'xwbyFAiJjkohARSeXmaU', 'QgsJHnGqKZOsFCfxXEBexQHrNpewEBFgme', 'JFruCM', 'DLiobjNSVmQk', 'vx', 'HYQYzwiCArqkVOwnjoVNZxhbjFaMK', 'Cm', 'ckHlrEXBPMrVIlyD', 'M', 'xI', 'UcmkAOabdKg', 'jursqSsWYOLbXMLQAEhvnuHclcrNcKqB', 'mKaoaBdKghxiCkkUUL']) from system.numbers limit 10; -select [0, 1, 0, 1, 0, 0, 0, 0, 7, 21] = multiSearchAllPositionsCaseInsensitive(materialize('WhdlibCbKUmdiGbJRshgdOWe'), ['kDPiHmzbHUZB', '', 'CukBhVOzElTdbEBHyrspj', '', 'QOmMle', 'wiRqgNwjpdfgyQabxzksjg', 'RgilTJqakLrXnlWMn', 'bSPXSjkbypwqyazFLQ', 'CBkuMDiGbJRShGdOWe', 'dow']) from system.numbers limit 10; -select [0, 8, 0, 1, 1, 0, 1, 7, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('oOccAoDDoPzHUyRqdWhJxNmATEqtE'), ['LFuvoQkVx', 'DoPzh', 'YaBSTdWvmUzlgRloppaShkRmLC', 'oO', '', 'eeEpOSLSXbyaOxTscOPoaTcKcchPmSGThk', '', 'dDO', 'oFXmyIJtmcSnebywDlKruvPUgmPFzEnMvA', 'vCs', 'MsxHLTgQcaQYZdPWJshIMWbk', 'yqrjIzvrxd']) from system.numbers limit 10; -select [0, 16, 0, 0, 0, 0, 7, 1, 0, 0, 1, 2, 1, 4, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('FtjOSBIjcnZecmFEoECoep'), ['FQQwzxsyauVUBufEBdLTKKSdxSxoMFpL', 'EOecoEP', 'HGWzNTDfHxLtKrIODGnDehl', 'ZxirLbookpoHaxvASAMfiZUhYlfuJJN', 'mKh', 'GZaxbwVOEEsApJgkLFBRXvmrymSp', 'Ij', '', 'X', 'AnCEVAe', 'fTj', 'tjOSbIjcNZECMfeoEC', '', 'OsBIjcN', 'LtdJpFximOmwYmawvlAIadIstt', 'JOsBiJCNzEc']) from system.numbers limit 10; -select [0, 2, 0, 0, 19, 0, 0, 12, 1, 0, 3, 1, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('ugpnWWncvqSLsYUCVXRZk'), ['yOWnQmZuhppRVZamgmRIXXMDQdeUich', 'gPNww', 'jlyFSbvmjaYPsMe', 'fQUeGVxgQdmPbVH', 'rZk', 'ariCX', 'grAffMPlefMQvugtAzN', 'LsYuCVX', '', 'jZFoQdWEWJFfSmNDqxIyNjvxnZJ', 'P', 'UgPN', 'JmKMsbegxNvusaiGGAZKglq', 'qArXLxzdYvabPv']) from system.numbers limit 10; -select [0, 0, 0, 0, 0, 0, 8, 0, 0, 1, 1, 15, 0, 1, 7, 0] = multiSearchAllPositionsCaseInsensitive(materialize('nxwotjpplUAXvoQaHgQzr'), ['ABiEhaADbBLzPwhSfhu', 'TbIqtlkCnFdPgvXAYpUuLjqnnDjDD', 'oPszWpzxuhcyuWxiOyfMBi', 'fLkacEEeHXCYuGYQXbDHKTBntqCQOnD', 'GHGZkWVqyooxtKtFTh', 'CvHcLTbMOQBKNCizyEXIZSgFxJY', 'PlUAxVoQah', 'zrhYwNUzoYjUSswEFEQKvkI', 'c', 'NXWOt', '', 'qAhG', 'JNqCpsMJfOcDxWLVhSSqyNauaRxC', '', 'PpLuaxV', 'DLITYGE']) from system.numbers limit 10; -select [2, 0, 0, 1, 0, 0, 28, 1, 16, 1] = multiSearchAllPositionsCaseInsensitive(materialize('undxzJRxBhUkJpInxxJZvcUkINlya'), ['ndxzjRxbhuKjP', 'QdJVLzIyWazIfRcXU', 'oiXcYEsTIKdDZSyQ', 'U', 'dRLPRY', 'jTQRHyW', 'Y', '', 'nxxJZVcU', '']) from system.numbers limit 10; -select [1, 4, 1, 0, 4, 1, 0, 1, 16, 1, 0, 0, 0, 8, 12, 14, 0, 2] = multiSearchAllPositionsCaseInsensitive(materialize('lrDgweYHmpzOASVeiFcrDQUsv'), ['', 'gwEYhMP', 'LrDGwEyHmPzOaSVEifC', 'oMN', 'gwEYhMpZO', 'lrdGWEy', 'pOKrxN', 'lrDgwEyhmpZoaSv', 'eifcrdqU', 'LrDgw', 'dUvarZ', 'giYIvswNbNaBWprMd', 'pPPqKPhVaBhNdmZqrBmb', 'hmPzoASVEiF', 'O', 'SVEi', 'gIGLmHnctIkFsDFfeJWahtjDzjPXwY', 'rDGweyHmP']) from system.numbers limit 10; -select [0, 0, 11, 1, 1, 1, 0, 16, 0, 1, 5, 0, 0, 0, 2, 0, 2, 0] = multiSearchAllPositionsCaseInsensitive(materialize('XAtDvcDVPxZSQsnmVSXMvHcKVab'), ['bFLmyGwEdXiyNfnzjKxUlhweubGMeuHxaL', 'IhXOeTDqcamcAHzSh', 'ZSQsNMvsxmVHcK', '', '', '', 'dbrLiMzYMQotrvgwjh', 'MvsxMV', 'zMp', 'XaTDvCdvpXzsqSNMVSxm', 'v', 'LkUkcjfrhyFmgPXPmXNkuDjGYlSfzPi', 'ULpAlGowytswrAqYdaufOyWybVOhWMQrvxqMs', 'wGdptUwQtNaS', 'ATdVcdVPXzSqsnmVSXMvHcKVab', 'JnhhGhONmMlUvrKGjQcsWbQGgDCYSDOlor', 'atdvCdvpXzsqSnMVSxMVhCkvAb', 'ybNczkKjdlMoOavqBaouwI']) from system.numbers limit 10; -select [8, 0, 0, 0, 4, 0, 0, 5, 5, 2] = multiSearchAllPositionsCaseInsensitive(materialize('XPquCTjqgYymRuwolcgmcIqS'), ['qgyYMruW', 'tPWiStuETZYRkfjfqBeTfYlhmsjRjMVLJZ', 'PkTdqDkRpPpQAMksmkRNXydKBmrlOAzIKe', 'wDUMtn', 'UcTJQgYYMRuWoLCgMcI', 'PieFD', 'kCBaCC', 'Ct', 'C', 'pQuctjqgyymRuwOLCgmc']) from system.numbers limit 10; - -select [1, 0, 7, 1, 0, 24, 17, 0, 0, 0, 2, 0, 1, 7, 4, 1, 12, 8] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('гГБаДнФбпнЩврЩшЩЩМщЕБшЩПЖПчдт'), ['', 'таОХхрзИДжЛСдЖКЧжБВЩжЛкКХУКждАКРеаЗТгч', 'Ф', '', 'ЙЩИФМфАГщХзКЩЧТЙжмуГшСЛ', 'ПЖпчдТ', 'ЩМщЕбшЩПжПч', 'ФгА', 'гУД', 'зУцкжРоППЖчиШйЗЕшаНаЧаЦх', 'гбаДНФбПНЩВРЩШЩщМЩеБшЩпжПЧд', 'РДЧЖАбрФЦ', 'гГ', 'ФбпНщвр', 'адНфБПнщвРщШщщМщЕбШщ', 'ггб', 'ВРЩ', 'бПНщврЩш']) from system.numbers limit 10; -select [0, 12, 8, 0, 12, 0, 0, 10, 0, 8, 4, 6] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('айРВбЧБжКВИхБкчФЖЖНВнпФйФБДфЗ'), ['ЛрЦфуУДВК', 'хБкчфЖжНвнпфйфБдФ', 'жКВИХБкчФЖжНвнПф', 'кЖчвУцВСфЗБТИфбСжТИдРкшгзХвщ', 'хбк', 'штДезйААУЛчнЖофМисНЗо', 'нлнШЧВЙхОПежкцевчлКрайдХНчНб', 'вИХбкчфжжНВН', 'ЩдзЦТуоЛДСеШГфЦ', 'ЖКВихбКЧфжЖ', 'вбЧбЖкВихБкЧфЖжНВ', 'Чб']) from system.numbers limit 10; -select [18, 15, 0, 0, 0, 0, 5, 0, 14, 1, 0, 0, 0, 0, 0, 15] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('пМИОкоЗжГйНТПЙацччЧАЩгЕВБбЕ'), ['ЧЧАЩгЕВБ', 'а', 'ФбРВщшййпХдфаЗЖлЛСЗПРШПпАОинЧКзЩхждН', 'ЛфРКДЙВСУСЙОчтнИкРЗбСГфкЩреИхЛлчХчШСч', 'ШйвБПАдФдФепЗТкНУрААйеЧПВйТоЧмБГДгс', 'ФтЙлЖЕсИАХИФЗаЕМшсшуцлцАМФМгбО', 'КО', 'лиШБнлпОХИнБаФЩдмцпжЗИЛнвсЩЙ', 'йацччЧАщгевбБЕ', 'ПмИоКозжГйНТП', 'ИГНннСчКАИСБщцП', 'ПнжмЙЛвШтЩейХЛутОРЩжифбЗчгМУЛруГпх', 'ХжЗПлГЖЛйсбпрЩОТИеБвулДСиГзлЛНГ', 'учклЦНЕгжмщлжАшщжМд', 'ЩеПОЙтЖзСифОУ', 'АЦЧ']) from system.numbers limit 10; -select [10, 0, 1, 1, 6, 1, 7, 6, 0, 0, 0, 2, 12, 0, 6, 0, 4, 8, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('квхБнцхйзЕпйИмтЙхфзвгдФ'), ['еПйИМт', 'хгкиМжСБИТНщенЩИщНСкй', '', 'Квхб', 'цхЙЗЕПйИмТйХФЗ', 'к', 'хйЗЕПЙИмтй', 'Цх', 'нКлШбМЖГйШкРзадрЛ', 'ДштШвБШТг', 'СЦКйЕамЦщПглдСзМлоНШарУтМднЕтв', 'ВхБнцхйЗЕПйимТ', 'йимтЙХФЗВГД', 'жчссунЙаРцМкЖУЦщнцОЕхнРж', 'цХЙЗЕП', 'ОгНФдМЛПТИдшцмХИеКйРЛД', 'бнЦхЙ', 'ЙЗе', 'згЩищШ', 'фХлФчлХ']) from system.numbers limit 10; -select [0, 0, 0, 12, 0, 0, 27, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('хНпсРТХВдтоЦчдлеФПвнЛгЗКлПйнМВ'), ['ШиБфЗШПДЧхОЩшхфщЗЩ', 'иГйСЧЗтШЛуч', 'АЗХЦхедхОуРАСВЙС', 'цчдЛЕфП', 'СДбйГйВЕРмЙЩЛщнжен', 'НДлцСфТшАщижгфмуЖицжчзегЕСЕНп', 'й', '', 'йлчМкРИЙиМКЙжссЦТцРГзщнхТмОР', 'ПРцГувЧкйУХггОгЖНРРсшГДрлЧНжГМчрХЗфЧЕ']) from system.numbers limit 10; -select [0, 0, 2, 0, 10, 7, 1, 1, 0, 9, 0, 2, 0, 17, 0, 0, 0, 6, 5, 2] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЙзЗжпжДЕСУхчйдттСЙзоЗо'), ['щОЙУшееЧщкхГККреБкВ', 'жВ', 'ззЖпждЕсУХчЙДТТсЙ', 'ЙЦШЦЙЖзХШРвнкЕд', 'УхчйДтТсйЗОз', 'дЕСу', '', '', 'дсцеррищндЗдНкжаНЦ', 'сУхчЙдттсйзОзО', 'ЦЖРжмц', 'ЗЗ', 'СгЛГАГЕЖНгщОеЖЦДмБССцЩафзЗ', 'Сйзоз', 'ЦГХТЕвЕЗБМА', 'пмВоиеХГжВшдфАЖАшТйуСщШчИДРЙБнФц', 'Оа', 'ждЕ', 'ПжДесу', 'ЗзЖПждЕСУ']) from system.numbers limit 10; -select [0, 0, 0, 0, 5, 1, 0, 6, 0, 1, 17, 15, 1, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('уФШЙбШТоХВбзЦцЖОЕКТлщхнЖГ'), ['цЛ', 'ууМ', 'ТИгЙолМФсибтЕМнетквЦИЩИБккйн', 'оФОаМогсХЧЦооДТПхб', 'бШтОХВбЗцЦЖоЕКтЛ', 'уфШйбШтоХ', 'фдтщрФОЦсшигдПУхЛцнХрЦл', 'ШтО', 'НИкИТрбФБГИДКфшзЕмЙнДЖОсЙпЩцщкеЖхкР', 'уфШЙБш', 'екТлщ', 'ЖоекТл', 'уфШйБшТоХвбз', 'ТуОхдЗмгФеТаафЙм']) from system.numbers limit 10; -select [0, 1, 6, 1, 0, 1, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чМЩБЛЛПРлщкВУбПефХВФлАЗШао'), ['гаТкЛВнрвдПМоеКПОйр', 'ч', 'ЛпрЛЩКвуБпе', 'ЧмЩб', 'ц', '', 'жгаччЖйГЧацмдсИИВЩЩжВЛо', 'йГеЙнБзгнкЦЛБКдОЕЧ', 'ПоЦРвпЕЗСАШж', 'ЙОНЦОбиееО']) from system.numbers limit 10; -select [2, 0, 17, 1, 0, 0, 0, 5, 0, 4, 0, 0, 0, 0, 0, 2] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЕаЩичщМЦЖиЗБЛЧжуНМЧК'), ['АЩиЧЩ', 'ИлУсшДБнжщаатуРТтраПОЙКЩйТГ', 'НМЧк', 'Еа', 'зАВФЛЩбФрМВШбПФГгВЕвЖббИТйе', 'РЗНРБЩ', 'ЦдЙНГпефзЛчпУ', 'ч', 'НШШчПЗР', 'ИчЩмЦжИЗБлЧЖУНМч', 'аннвГДлмОнТЖЗЙ', 'ШдчЩшЕБвхПУсШпг', 'гФИШНфЖПжймРчхАБШкЖ', 'ЖзгЖАБлШЗДпд', 'Д', 'ащиЧ']) from system.numbers limit 10; -select [4, 1, 0, 7, 0, 7, 1, 1, 0, 3, 7, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('иОцХКЙвувМИжШдУМУЕйНсБ'), ['ХкйвуВмИжШдУм', '', 'звМАОМЩщЙПшкиТчЩдгТЦмфзеИ', 'вуВМиж', 'КДщчшЙВЕ', 'в', '', 'ИоЦхКЙВувМижШ', 'ЕвТАРи', 'цхКЙвувмИЖШДумуе', 'вУвМи', 'зПШИХчУщШХУвврХйсуЙЗеВЧКНмКШ']) from system.numbers limit 10; -select [0, 5, 0, 0, 0, 0, 0, 12, 0, 11] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЦОфбчУФсвТймЦчдщгЩжИАБ'), ['йлрк', 'ЧуФсвтйМцчдЩгщ', 'МНлЕжорв', 'иНзТЖМсмх', 'шЕМЖжпИчсБжмтЧЙчщФХб', 'жШХДнФКАЩГсОЩвЕаам', 'НпКЦХулЛвФчШЕЗкхХо', 'мЦчДЩгЩжиАб', 'мпцгВАЕ', 'Й']) from system.numbers limit 10; -select [1, 0, 0, 0, 8, 0, 2, 0, 0, 7] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чТХЙНщФфцИНБаеЖкОвлиУДР'), ['', 'рВХмжКцНцИЙраштМппсодЛнЧАКуЩ', 'ИХфХЖЧХВкзЩВЙхчфМрчдтКздиОфЙжУ', 'Гзлр', 'фЦи', 'абПф', 'тХЙНщффЦИн', 'нссГбВеЖх', 'амлЗщрсУ', 'фФ']) from system.numbers limit 10; -select [0, 9, 11, 0, 11, 1, 0, 0, 0, 1, 6, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('зДЗпщАцвТгРдврщхЩфЖл'), ['йХЛ', 'Т', 'рд', 'АИЦщгниДфВОе', 'Р', 'здзпщ', 'вКТвВШмгч', 'ввирАйбЗЕЕНПс', 'тХиХоОтхПК', '', 'аЦВТгРДврщ', '', 'уЗЗЖвУЕйтчудноЕКМЖцВРаНТЙЗСОиЕ', 'оЕфПхЕДжАаНхЕцЖжжофЦхкШоБЙр']) from system.numbers limit 10; -select [1, 1, 0, 0, 1, 7, 0, 0, 0, 2] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('йЛПЛшмЦШНЖРрЧрМцкЖзЕНжЧДелФжАн'), ['', 'йЛПлшМЦшНЖррч', 'ПНКдфтДейуиШзЗХАРУХизВ', 'ПценмщЧОФУСЙЖв', '', 'ЦшнжрРчрМЦКЖЗе', 'МрПзЕАгжРбТЧ', 'ЕДФмаФНвТЦгКТЧЦжцЛбещЛ', 'УтПУвЛкТасдЦкеИмОещНИАоИжЖдЛРгБЩнвЖКЛЕП', 'Л']) from system.numbers limit 10; -select [1, 5, 1, 1, 0, 0, 1, 1, 0, 2, 19, 0, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('сйДпмжнДРщКБгфцЖОчтГНБ'), ['', 'МЖнДРщ', 'Сй', '', 'пУщ', 'йгВИАЦнозаемТиХВвожКАПТдкПИаж', 'Сйд', 'СЙДпмжНдРщ', 'ФПщБцАпетаЙФГ', 'ЙдпМжНдрЩКбГфЦжОЧТГНб', 'т', 'гллрБВМнвУБгНаЙцМцТйЙФпзЧОЙЛвчЙ', 'йДПМжндРЩкБ', 'ЗмфОмГСНПщшЧкиССдГБУсчМ']) from system.numbers limit 10; -select [0, 18, 10, 5, 0, 2, 8, 1, 4, 11] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ХпИРддХрмВНйфчвгШиЧМКП'), ['хЗФДлДУБЙаЦтжРБЗсуйнЦпш', 'иЧмК', 'внЙ', 'д', 'зиМУЩГиГ', 'ПИр', 'РМвнЙфчвгШич', '', 'РдДхРМ', 'нЙфчВГШИ']) from system.numbers limit 10; -select [18, 0, 0, 1, 0, 0, 6, 0, 0, 9] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('нГгФкдуФШуИТбшпХфтаГт'), ['Таг', 'рРпшУйчГд', 'гК', '', 'лаВНбездпШШ', 'ЕБРйаНрОБожкКИсв', 'ДУфШУитБ', 'ГРиГШфШтйфЖлРФзфбащМЗ', 'мхЩжЛнК', 'ШуИтБШ']) from system.numbers limit 10; -select [13, 0, 0, 7, 0, 15, 0, 0, 15, 0, 0, 5, 6, 0, 18, 21, 11, 1] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('рлобшдПЦИхжФуХщжгПФукшзт'), ['УхщжГ', 'ТВщЦфФсчЩГ', 'ЕжФШойжуЛРМчУвк', 'пцИХжфуХЩж', 'бР', 'щЖГПфуКШЗТ', 'йжРГгЛуШКдлил', 'ТщЖГкбШНИщЩеЩлаАГхрАфЙНцЦгВкб', 'щжГПфУ', 'бкаДБЛХ', 'АЗ', 'шДПЦихжфух', 'дП', 'вфнЙобСцвЩмКОбЦсИббФКзЩ', 'пФУкшзТ', 'К', 'жфу', '']) from system.numbers limit 10; -select [12, 19, 8, 1, 0, 0, 0, 15, 0, 0, 12, 2, 0, 4, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЦкЛЗепкЕХЩГлКФрБдТрлвйАхдООШ'), ['лК', 'рЛв', 'Ехщ', '', 'еаПКБгЦЩАоЗВонйТЗгМхццСАаодМЕЩГ', 'ишОНиеБидфбФБЖриУЩЩ', 'дуж', 'РбДТ', 'пЗсГХКсгРущкЙРФкАНЩОржФвбЦнЩНЖЩ', 'щрОУАГФащзхффКвЕйизцсйВТШКбнБПеОГ', 'лкФрБдТРлвЙа', 'КЛзеп', 'УЛФЗРшкРщзеФуМвгПасШЧЛАЦр', 'зеПКеХщглкфР', 'ЦЖЗдХеМЕ', 'зЖжрт', 'уЩФрйрЖдЦз', 'МфцУГЩтвПАЦжтМТоеищЕфнЖй']) from system.numbers limit 10; -select [0, 0, 1, 0, 1, 0, 0, 7, 0, 5, 1, 6, 1, 1, 1, 5, 6, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('РННЕШвжМКФтшДЙлфЛИзЙ'), ['ГаМРош', 'Дтфс', '', 'еБбиаКщГхххШвхМЖКзЛАезФУчХо', 'РНн', 'сВбТМ', 'ЖЗЦПБчиСйе', 'жМкфтШДЙл', 'нЖХуеДзтЧтулиСХпТпеМлИа', 'ШВжМкФТШдЙлфл', '', 'вЖМКфТ', '', '', '', 'швЖМКфтШДЙЛфлИЗй', 'вЖмКФТ', 'еМ']) from system.numbers limit 10; -select [0, 0, 15, 1, 0, 0, 8, 1, 0, 0, 0, 4, 8, 10] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('РиучГийдХутДЕЙДпфиуд'), ['ЩмгцлЖрц', 'ЕСжСлЩЧИЖгЗЛлф', 'дП', '', 'щГЦаБтПШВзЦСрриСЙбД', 'тдРгОЛТШ', 'д', '', 'КЕбЗКСХЦТщЦДЖХпфаЧйоХАл', 'мТвзелНКрЖЧЦПпЕЙвдШтеШйБ', 'ЙОТКрБСШпШд', 'ЧГ', 'ДХУТДЕЙд', 'УТд']) from system.numbers limit 10; -select [0, 0, 0, 0, 15, 0, 0, 0, 11, 0, 0, 5, 1, 1, 0, 2, 3, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('МшазшргхОПивОлбмДоебАшцН'), ['ЦИшштН', 'еМСЗкФЕКДйОНМ', 'ЛСчГрбеРЕбЩМПМЗЦИп', 'ХнИПЧжЗдзФщЗ', 'бмдоЕ', 'гМОдйсбТСЦЩбФВЗШзшщбчегаЕмЕБаХаРР', 'фщнР', 'щмТчФчсМАОгчБщшг', 'иВ', 'УщцГОшТзпУХКоКЖБеМШ', 'мйаАЛцАегСмПОаСТИСфбЧДБКоИВчбЦЙ', 'шРгхоп', '', '', 'еИпАЩпнЛцФжЩХИрЧаИИТЛвшиСНЩ', 'шаЗ', 'АЗ', 'ФгдтфвКЩБреногуир', 'ДБжШгщШБЩпЖИЛК', 'ЧдРЩрбфЛзЙклхдМСФУЙЛн']) from system.numbers limit 10; -select [5, 0, 0, 18, 13, 0, 2, 7, 0, 0, 1, 15, 1, 0, 0, 0, 3, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('хщеКЗПчуНЙтрЧЩгфСбоКЕАДТййАрр'), ['зп', 'хчПЦшпДбзСфНВЧзНжЕМФОП', 'ЧЖхЕУк', 'БОКеАдтЙЙа', 'чЩГфС', 'шллддЩщеМжШйкЩн', 'щЕкзпЧуНЙТ', 'ЧунйтРЧщгФс', 'ввНздЙуоТЖРаВЙчМИчхРвфЛЖБН', 'ЗХМХПщПкктцАзщЙкдпжф', '', 'ГФСбОкеАДтйЙа', '', 'МБХВЕчпБМчуххРбнИМЛТшЩИщЙгаДцзЛАМвйаО', 'ЛкОзц', 'ЕцпАДЗСРрсЕвтВщДвцбЗузУннТИгХжхрцПДРДПм', 'екЗПЧунЙТРчщгФсбоК', 'шпИфЕчгШжцГВСйм', 'ЛхйЧбЧД', 'ВзЗоМцкЩНХГж']) from system.numbers limit 10; -select [0, 0, 6, 20, 0, 10, 0, 0, 0, 9, 10, 3, 23, 1, 0, 0, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('лцапШиХчЛДшдксСНИбшгикзчЙанми'), ['ХууатТдтбодМГЧгщЧнклШтЗПНчкЦОаЙг', 'МЦЧчпИхКЛаФхщХдРУДщжУчфлжахц', 'иХЧлдшдкСсНИбШГикзЧЙ', 'гикЗчйА', 'ГсТзЛОфИББлекЩАсЛвмБ', 'Д', 'ЦХрТЖощНрУШфнужзжецсНХВфЩБбДУоМШШиГйж', 'йуВдЕзоггПВДЖб', 'ЙфБГйХМбжоакЖЛфБаГИаБФСнБЖсТшбмЗЙТГОДКИ', 'ЛДШдКССНИБшГикзч', 'ДШдКССниБ', 'аПШИХчЛДШДКсс', 'з', '', 'ФоохПЩОГЖоУШлКшзЙДоуп', 'хАДХЩхлвУИсшчрбРШУдФА', 'ЦА', 'гвптУФлчУуРхпрмЖКИрБеЩКчН']) from system.numbers limit 10; -select [0, 4, 5, 7, 15, 3, 3, 17, 7, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('зЗАЩлЕЕЕПИохЧчШвКЧйрсКХдд'), ['пКРбуШОНТЙБГНзИРвЖБсхрЛщчИрлЧУ', 'ЩЛЕЕЕПиоХЧ', 'ЛеЕеп', 'Еепио', 'швкЧйрС', 'ащЛеееПИох', 'АЩлеЕЕпиОхЧЧШвкЧЙРсК', 'КчйРскхД', 'ЕЕПИохччшВКчй', 'у']) from system.numbers limit 10; -select [1, 12, 0, 8, 1, 1, 0, 1, 5, 0, 1, 0, 0, 0, 0, 3, 1, 0, 4, 5] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ПмКСйСКЖККмШеоигЙчПфжТ'), ['', 'Шео', 'РчвлдЙЙлПщуКмтН', 'жкКмшЕоИГЙЧ', '', '', 'йРмМЖнПиЙ', '', 'йс', 'тфФРСцл', '', 'щлЩХиКсС', 'кпнТЖпФЩиЙЛ', 'абкКптбИВгмЧкцфЦртЛДЦФФВоУхЗБн', 'чНшоВСГДМйДлтвфмхХВВуеЩЦВтЖтв', 'кС', '', 'фидБлйеЙЧШРЗЗОулщеЕЩщЙсЙшА', 'СЙс', 'йсКжкКМшЕо']) from system.numbers limit 10; -select [0, 0, 1, 0, 2, 2, 1, 2, 7, 0, 1, 2, 1, 0, 6, 8] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('УгЖЕугАЩХйчидаррлжНпфФГшр'), ['утвШ', 'кЕвФч', 'угжеУг', 'тШлТвЕШЗчЖеЛНджЦазЩХцж', 'гЖеугаЩхй', 'ГжЕугаЩХйЧидАР', 'УгжЕУГаЩХЙЧИда', 'гЖеу', 'ащхЙчИ', 'мЧлщгкЛдмЙЩРЧДИу', '', 'ГжеугАщХйЧиДаРРЛЖНП', '', 'зЕМвИКбУГКЩФшоГЧГ', 'ГАЩХйчИДАррлЖНпФфг', 'ЩХЙчИдАррЛЖНпфФгш']) from system.numbers limit 10; -select [1, 0, 0, 7, 0, 6, 0, 11, 0, 0, 0, 2, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЗЕГЛЩПцГНтзЕЦШЧхНКГТХЙЙФШ'), ['', 'шзкиЗсаИщАБмаз', 'Ж', 'ц', 'гШуЕжЛСПодРнхе', 'пцГНтЗЕЦ', 'щРкЩАеНржЙПМАизшщКвЗщглТкКИф', 'ЗеЦшчхнКГтхЙЙ', 'пелгЩКкцвтфнжЖУуКосЙлкЛ', 'рф', 'хНШчНрАХМШщфЧкЩБНзХУкилЙмП', 'ЕгЛЩПЦгнтзецШЧ', 'ЩУчБчРнЖугабУоиХоИККтО', 'СГмЦШтФШЛмЙЩ', 'ауТПЛШВадоХМПиБу', 'ЩЩйр']) from system.numbers limit 10; -select [2, 2, 1, 0, 0, 0, 0, 0, 1, 0, 7, 9, 0, 15, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('гЙЧЙФХнЖБвомгАШГбОВГксИйцз'), ['ЙЧйфхНЖбвО', 'Й', 'гЙЧйфхнЖбв', 'хсЩмШЙЙММВЦмУБТчгзУЛР', 'зктшп', 'дЕоиЖлгШж', 'хКкаНЛБ', 'ЗКйСчсоЗшскГЩбИта', '', 'у', 'НжбВОмгашГ', 'БВо', 'ещфРШлчСчмаЖШПЧфоК', 'шгбо', 'ЙСтШШДЩшзМмдпЧдЙЖевТвоУСЕп', 'Л']) from system.numbers limit 10; -select [0, 9, 0, 0, 18, 13, 13, 11, 0, 0, 4, 1] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЙЛмоЦСдТаоФчШКЖЦСНРаРЦзоС'), ['ДфгЗАасВфаМмшхчлмР', 'аоФчШкЖцСнРАРЦзОС', 'зЩзнйтФРТЙжУлхФВт', 'чЦкШВчЕщДУМкхЛУЩФшА', 'н', 'Шк', 'шКЖцсНРаРцЗос', 'фчшкЖцснрАРЦз', 'лку', 'пЧШМЦквоемЕщ', 'о', 'йЛМоцСДТАофЧшкжЦСнРаРЦзос']) from system.numbers limit 10; -select [21, 0, 0, 17, 1, 11, 0, 2, 0, 7] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('кЧЖнЕбМЛпШЗХиЙжиМщлнСФрПЧЖВН'), ['сФ', 'гцХаШЛсаШЛкшфЧОКЛцзешХСиЩоаЕОш', 'Г', 'МщЛНСФРпч', '', 'зХ', 'ОАДепНпСГшгФАЦмлуНуШШЗфдЧРШфрБЛчРМ', 'чЖне', 'СфЕАбФн', 'М']) from system.numbers limit 10; -select [4, 0, 1, 1, 0, 2, 4, 16, 3, 6, 5, 0, 0, 6, 1, 0, 5, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('кдАпЩСШИСцРхтеСиФЖЧСсОоц'), ['пщСшиСцрХТЕсифЖчССоОц', 'рхнкикДТКДВШчиЖЦнВм', '', '', 'жПЛСнЦцн', 'дА', 'ПщсШИсцрХтЕс', 'иФжЧсСоОЦ', 'ап', 'с', 'щсШИ', 'МАзашДРПЩПзРТЛАсБцкСШнЕРЙцИЩлТЛеУ', 'ичцпДбАК', 'сшИСЦрхтЕсифжчСсООц', 'КдАПЩСшИСЦРХТЕсИфЖЧСсо', 'ЛнБсИПоМЩвЛпиЩЗЖСд', 'щс', 'шщДНБаСщЗАхкизжнЛАХЙ']) from system.numbers limit 10; -select [0, 13, 0, 2, 16, 1, 3, 0, 9, 0, 2, 0, 1, 4, 0, 0, 0, 1] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('иНхеЕкхЩщмгзМГхсгРБсхОКцУгуНБ'), ['ДиоУлФЖЛисУСЕтсЕалщн', 'МгХсгрБСХО', 'ЖХНцршПшгйО', 'нХЕЕкхЩ', 'сГРбсхОКцУг', '', 'х', 'Ж', 'щМгЗмгхСг', 'СрпхДГОУ', 'НхеЕкХщ', 'ПМтБцЦЙЖАЙКВБпФ', 'ИнхеЕ', 'еЕКхЩ', 'мМГлРзш', 'гтдоЙБСВещкЩАЩЦйТВИгоАЦлчКнНРНПДЖшСЧа', 'ЖшеН', '']) from system.numbers limit 10; -select [1, 5, 0, 0, 3, 0, 2, 0, 14, 14, 1, 0, 17, 13, 3, 25] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('айлзсЗБоГйтГжЙРККФхКшлНРОрЦкфо'), ['', 'с', 'Д', 'шиБраНИЦЧуИжп', 'Лз', 'ДРБСУфКСшцГДц', 'йЛЗСЗбОгЙтГЖйРК', 'ЕЙЦсвРЕШшщЕЗб', 'ЙркКфхкшЛнРОР', 'ЙРкКФхкШ', 'а', 'ГдоДКшСудНл', 'КФхКшлНРоР', 'ж', 'лзСзБогйТГЖйрККф', 'оР']) from system.numbers limit 10; -select [6, 0, 8, 10, 1, 0, 1, 13, 0, 0, 0, 2, 2, 0, 4, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('РучУлрХчЗУпИчДТЕфщИЙщрНлн'), ['РХЧ', 'оДсГСЛЙшйиЧРСКзчХВоХарцНШ', 'ЧЗУпИ', 'УПичдТе', 'Р', 'ВЙЩхжАутПСНЦфхКщеЩИуЧдчусцАесзМпмУв', '', 'ЧдТ', 'ООсШИ', 'ФШсВжХтБУШз', 'ЕЩуДдшкМУРЕБшщпДОСАцйауи', 'УЧ', 'УЧУЛрХчзуПИчдТеФщий', 'йнЦцДСхйШВЛнШКМСфмдЩВйлнеЖуВдС', 'улрхчзупиЧдтефщИ', 'СХТЧШшГТВвлЕИчНОВи']) from system.numbers limit 10; -select [0, 0, 0, 2, 1, 1, 0, 1, 19, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('УецжлЦЦщМшРГгЩЩдБмхЖЗЧзШЙб'), ['НзИуАузуРЗРуКфоТМмлПкрсмЕЕЕнТ', 'ЕЩГХхЧш', 'ХоЙпООчфЖввИжЙшЖжЕФОтБхлВен', 'ЕЦЖЛЦцщ', '', '', 'ухогСИФвемдпаШЗуЛтлизОЧ', 'УецЖ', 'ХЖзЧЗ', 'П', 'мБкзХ', 'уБуОБхШ']) from system.numbers limit 10; -select [6, 1, 15, 5, 0, 0, 0, 3, 2, 4, 0, 12, 0, 2, 0, 3, 1, 6, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ГЖФеачМаКчПСпкВкхсПтг'), ['чмАкЧ', '', 'ВкХс', 'ачМА', 'КлтжУлОЛршБЕблФЩ', 'тцуМфж', 'л', 'фе', 'Жф', 'ЕАЧМак', 'лЖЕРТнФбЧЙТййвзШМСплИхбЙЛЖзДпм', 'СпкВК', 'ЩзчжИш', 'жФеАчМ', 'КбЦбйЕШмКтЩЕКдуЩтмпИЕВТЖл', 'ФЕаЧмАКчПСПквкхспТ', 'гжФеАЧмаКчпСп', 'ЧмАК', 'дцкДННМБцйЕгайхшжПГх', 'ТЩбвЦЖАНшрАШФДчОщй']) from system.numbers limit 10; -select [1, 6, 0, 1, 0, 0, 3, 1, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('МФННЧйОнцЛИЧЕПШПЧйоГФО'), ['', 'йОн', 'шУлгИЛЛРЙАсфЗоИЙЗРхуПбОЙсшдхо', 'МФННчЙоНц', 'лзВжбЦзфкзтуОйзуЗ', 'ЖГДщшЦзсжщцЦЖеЧвРфНИНОСАОщг', 'ННчйОНЦлИчЕПШ', '', 'Ф', 'ЩрИдНСлЙуАНЗвЕчмчАКмФУипндиП']) from system.numbers limit 10; -select [5, 0, 8, 13, 0, 0, 0, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('зВйймХЩМзЦГЕкЕКфоСтхПблуКМхц'), ['МХщмз', 'НАНрШоНДмурМлО', 'мзцгЕкек', 'кеКфоСтХПбЛУК', 'СУУксО', 'ЦоШжЧфйШЦаГЧйбЛШГЙггцРРчт', 'НбтвВбМ', '', 'тЩФкСтоСЧЦЦЙаСДЩСГЙГРИФЗОЗфбТДЙИб', 'ВГж']) from system.numbers limit 10; -select [0, 0, 0, 8, 19, 0, 3, 12, 1, 4] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ДпбЙЖНЗбПнЛбахБаХТуабШ'), ['цИаЩвгеИР', 'Ф', 'РЖиА', 'БпнЛб', 'У', 'Тфн', 'Б', 'БА', '', 'ЙЖНзБПнлбАхбаХ']) from system.numbers limit 10; -select [0, 0, 0, 0, 0, 1, 0, 17, 1, 0, 1, 1, 1, 11, 0, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ТЦмЩОинХзоДДпПНЩигрРщОзКц'), ['ЕжЙВпПл', 'ВКфКТ', 'ШкДсЖхшфоПИадУбхФЩБчОАкпУеБхи', 'НТЕЙОШЦЖоЩбзВзшс', 'учГгуКФзлУдНУУуПУлкаЦЕ', '', 'фАПМКуЧйБЧзСоЗргШДб', 'ИГРрщОзк', '', 'йупОМшУйзВиВрЛЩЕеЩмп', '', '', '', 'дДППнщИгРР', 'ШФвИЧакеЦвШ', 'ТцМЩоинхЗОДдппнЩ', 'мрОгЩшЩеЧ', 'еЖРиркуаОТсАолЩДББВАМБфРфпШшРРРм']) from system.numbers limit 10; -select [3, 0, 0, 0, 0, 0, 1, 0, 0, 14, 0, 1, 0, 1, 1, 1, 0, 7] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('аОкиЛгКйхаОГОУзЦЛрбцш'), ['кИЛГкйхАогоУЗЦл', 'щЧДпХИхбпсГвфДФХкчХ', 'ШвАмБЗлДОИПткжхФТФН', 'щфсхФмЦсЛеувЙО', 'лВУЖц', 'еИщРшозЖАдцтКииДУлДОУФв', 'а', 'ХгЦРШ', 'ФзрЖкРЗЩЧИеЧцКФИфЧЧжаооИФк', 'уЗ', 'фЦФдцжжМчЗЖлиСЧзлщжжЦт', '', 'МдхжизИХфвбМААрйФНХдЕжп', 'аОкиЛг', 'АОКИЛгкйХАОГОУЗЦ', '', 'МбЖйрсумщиеОЩк', 'КйХАоГоУЗцлРБЦШ']) from system.numbers limit 10; -select [0, 0, 2, 1, 0, 0, 12, 0, 17, 0, 0, 0, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('КУчЛХФчЛХшвбМЦинРвНрФМРкмиеЕп'), ['ТБЩБзхАмщПщЧПИФПашгЕТиКЦМБМпСЩСуЩМчтшеш', 'йлВЕЙшфшаШЗШЩВХЦчЛБс', 'УЧл', '', 'ЛДсЖщмНЦсКуфЗуГиука', 'РТТОТфГЕлЩЕгЛтДфлВЖШГзЦЖвнЗ', 'БМцИНРвнРф', 'ОЕИЕдИсАНаифТПмузЧчЖфШЕуеЩсслСШМоЖуЩЛМп', 'рвНРфМркМи', 'ЦзБМСиКчУжКУЩИИПУДвлбдБИОЙКТЛвтз', 'злСГе', 'ВдтцвОИРМЕжХО', 'учЛХфЧл', 'БшччШбУзЕТзфКпиШжнезвоеК']) from system.numbers limit 10; -select [0, 7, 0, 0, 0, 0, 7, 6, 0, 16, 12, 12, 15, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('оЖиогсфклШМСДрбхРбМбрЕщНЙЗйод'), ['иПмДКейууОклНХГЗсбаЙдШ', 'ФКлШмсДрБХРбМбрещНЙЗЙОд', 'арчжтСТнк', 'чбТНЛЕжооЗшзОУ', 'ощАЩучРСУгауДхГКлмОхЙцЕо', 'аЛбкиЦаКМбКхБМДнмФМкйРвРр', 'ФКлШмСДрбХРбм', 'СфклШ', 'еДйилкУлиИчХЙШтхцЗБУ', 'хрБ', 'СДрбХрбМБР', 'СдрбхРБ', 'бхрБМБРЕщНйз', 'КИб']) from system.numbers limit 10; -select [22, 1, 8, 0, 0, 1, 0, 3, 0, 6, 20, 0, 0, 0, 4, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЕЖДФбКужЙЦЦмсЖГГжБзеЙнПйЙри'), ['НПййР', '', 'Жй', 'Щ', 'ФхУО', 'ЕЖДфБКУЖйЦЦмСжГГ', 'НФЙзщЩГЧпфсфЦШОМЕЗгцрс', 'д', 'ЦтщДДЖтбвкгКонСк', 'кУЖЙЦЦм', 'ЕйНПййРИ', 'РчеЙйичФбдЦОтпчлТЖИлДучЙПгЗр', 'внчзшЗзОнФфхДгфзХТеНПШРшфБТЖДйф', 'кНснгмулМуГНурщЕББСузВмбнЧаХ', 'фбКУЖйЦцМсЖГгЖб', 'ЩСЕ']) from system.numbers limit 10; -select [0, 0, 0, 1, 10, 4, 0, 0, 5, 0, 1, 0, 7, 0, 3, 7, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чБхлжгКЖХлЙнкКЦфжЕгЖАндЧ'), ['ПдмРрЖАтВнСдСБШпПЗГгшИ', 'цшцг', 'тчАЙЧОеЕАвГпЗцЖЧгдХуЛСЛНрвАЖщ', '', 'Лй', 'Л', 'ОйррцУжчуЦБАжтшл', 'вХУКк', 'жгКжхЛЙН', 'уцбЕЕОЧГКУПуШХВЕчГБнт', '', 'ПсАжБИКштЕаН', 'КжхлЙН', 'ЩгШухЦПАТКежхгХксгокбщФЙПсдТНШФЦ', 'Х', 'кЖХЛйНккЦФжЕГЖ', 'ЙзРДСПднаСтбЧЖхощ', 'пАПОУЧмИпслБЗПфУ']) from system.numbers limit 10; -select [0, 0, 0, 5, 2, 16, 4, 4, 11, 0, 0, 3, 3, 0, 0, 6] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('кпМаоуГГфвощолЦЩщЧПРОКепеА'), ['ЗзуФжНшщПТнЧЦКВОиАУсЧХОШбк', 'тмПкАпеайзуХсурШй', 'АЕЦавбШиСДвВДумВкиИУБШЕ', 'о', 'ПМаОУггФВощоЛЦЩЩЧПрокЕПеа', 'щЩ', 'аоУг', 'аОуГгФВ', 'оЩоЛЦЩщчПРОК', 'виХЛшчБсщ', 'УчАМаЦкйДЦфКСмГУЧт', 'мАоУ', 'МАО', 'щФФА', 'Н', 'У']) from system.numbers limit 10; -select [0, 3, 10, 8, 3, 0, 4, 0, 9, 4, 1, 9] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('щЙЧРпшИцхпргЦНуДййусЧЧнЖ'), ['ДлУцтееЖБКХгМзСВжА', 'чРпШИЦ', 'пргЦнУДЙЙУ', 'Ц', 'ЧРПш', 'нЩрЕвмрМеРйхтшЩче', 'РпШИЦхПРГцнУд', 'ПНоЙтПкоаОКгПОМЦпДЛФЩДНКПбСгЗНЗ', 'ХПРГцНудЙЙ', 'рПши', '', 'ХПРГ']) from system.numbers limit 10; -select [11, 4, 1, 0, 1, 0, 0, 0, 0, 12, 0, 9, 5, 0, 16, 0, 12, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('пкзщщЛНОНбфЦноИЧфхбФ'), ['ф', 'щщл', 'ПКзЩщЛНОн', 'ЩшФйЧБНДОИзМхеЖНЦцеЛлУЧ', '', 'сЗоЙТклйДШкДИЗгЖ', 'орЛФХПвБбУхНс', 'доЗмЩВу', 'ШиЕ', 'ЦНО', 'ндЩдРУЖШМпнзНссЖШДЦФвпТмуМЙйцН', 'НбФЦнОИч', 'ЩлНонБФ', 'ЛдРжКММЙм', 'чфх', 'ЦматДйиСфЦфааЦо', 'ЦНОИчФх', 'иржЦщн']) from system.numbers limit 10; -select [0, 0, 0, 0, 0, 1, 0, 0, 0, 1, 0, 1, 0, 3, 0, 5] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чЖажцВбшЛттзДааАугШщАйПгщП'), ['ШгУТсчГОВЦЦеЛАСфдЗоЗЦВЛйлТДзчвЛва', 'УшЕшищЖткрвРСйиФЗйТФТЛЗаЗ', 'ВдикЙббщузоФХщХХГтЗоДпхбЕкМщц', 'срйеХ', 'рАшуПсЙоДнхчВкПЖ', '', 'гНЗбКРНСБВрАВФлнДШг', 'фХЧгмКнлПШлЩР', 'мкйЗбИФрЗахжгАдвЕ', 'чжаЖцВБШлТ', 'лХЕСрлПрОс', '', 'ЗЧПтчЙОцвОФУФО', 'ажцвБшЛТт', 'уНчЖШчМЕА', 'ц']) from system.numbers limit 10; -select [7, 1, 0, 7, 1, 19, 8, 6, 3, 0, 2, 13, 6, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('НТКПпмБжДцбАКПНСЖоиТФД'), ['б', '', 'аУщЛМХЖбвИтНчГБМГдДнч', 'Б', 'НТкппм', 'и', 'Жд', 'МБждЦбАкП', 'кппмБждцБа', 'ПЕрнЦпМЦВгЧЧгГ', 'ткПпМБЖДцбаКпнСжО', 'кПнСЖоИ', 'МБжДцБакпН', 'гхОХжГуОвШШАкфКМщсшФДШеИжоАйг']) from system.numbers limit 10; - -select 0 = multiSearchAny(materialize('mpnsguhwsitzvuleiwebwjfitmsg'), ['wbirxqoabpblrnvvmjizj', 'cfcxhuvrexyzyjsh', 'oldhtubemyuqlqbwvwwkwin', 'bumoozxdkjglzu', 'intxlfohlxmajjomw', 'dxkeghohv', 'arsvmwwkjeopnlwnan', 'ouugllgowpqtaxslcopkytbfhifaxbgt', 'hkedmjlbcrzvryaopjqdjjc', 'tbqkljywstuahzh', 'o', 'wowoclosyfcuwotmvjygzuzhrery', 'vpefjiffkhlggntcu', 'ytdixvasrorhripzfhjdmlhqksmctyycwp']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('qjjzqexjpgkglgxpzrbqbnskq'), ['vaiatcjacmlffdzsejpdareqzy', 'xspcfzdufkmecud', 'bcvtbuqtctq', 'nkcopwbfytgemkqcfnnno', 'dylxnzuyhq', 'tno', 'scukuhufly', 'cdyquzuqlptv', 'ohluyfeksyxepezdhqmtfmgkvzsyph', 'ualzwtahvqvtijwp', 'jg', 'gwbawqlngzcknzgtmlj', 'qimvjcgbkkp', 'eaedbcgyrdvv', 'qcwrncjoewwedyyewcdkh', 'uqcvhngoqngmitjfxpznqomertqnqcveoqk', 'ydrgjiankgygpm', 'axepgap']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('fdkmtqmxnegwvnjhghjq'), ['vynkybvdmhgeezybbdqfrukibisj', 'knazzamgjjpavwhvdkwigykh', 'peumnifrmdhhmrqqnemw', 'lmsnyvqoisinlaqobxojlwfbi', 'oqwfzs', 'dymudxxeodwjpgbibnkvr', 'vomtfsnizkplgzktqyoiw', 'yoyfuhlpgrzds', 'cefao', 'gi', 'srpgxfjwl', 'etsjusdeiwbfe', 'ikvtzdopxo', 'ljfkavrau', 'soqdhxtenfrkmeic', 'ktprjwfcelzbup', 'pcvuoddqwsaurcqdtjfnczekwni', 'agkqkqxkfbkfgyqliahsljim']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('khljxzxlpcrxpkrfybbfk'), ['', 'lpc', 'rxpkrfybb', 'crxp', '', 'pkr', 'jxzxlpcrxpkrf', '', 'xzxlpcr', 'xpk', 'fyb', 'xzxlpcrxpkrfybbfk', 'k', 'lpcrxp', 'ljxzxlpcr', 'r', 'pkr', 'fk']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('rbrizgjbigvzfnpgmpkqxoqxvdj'), ['ee', 'cohqnb', 'msol', 'yhlujcvhklnhuomy', 'ietn', 'vgmnlkcsybtokrepzrm', 'wspiryefojxysgrzsxyrluykxfnnbzdstcel', 'mxisnsivndbefqxwznimwgazuulupbaihavg', 'vpzdjvqqeizascxmzdhuq', 'pgvncohlxcqjhfkm', 'mbaypcnfapltsegquurahlsruqvipfhrhq', 'ioxjbcyyqujfveujfhnfdfokfcrlsincjbdt', 'cnvlujyowompdrqjwjx', 'wobwed', 'kdfhaoxiuifotmptcmdbk', 'leoamsnorcvtlmokdomkzuo', 'jjw', 'ogugysetxuqmvggneosbsfbonszepsatq']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('uymwxzyjbfegbhgswiqhinf'), ['lizxzbzlwljkr', 'ukxygktlpzuyijcqeqktxenlaqi', 'onperabgbdiafsxwbvpjtyt', 'xfqgoqvhqph', 'aflmcwabtwgmajmmqelxwkaolyyhmdlc', 'yfz', 'meffuiaicvwed', 'hhzvgmifzamgftkifaeowayjrnnzw', 'nwewybtajv', 'ectiye', 'epjeiljegmqqjncubj', 'zsjgftqjrn', 'pssng', 'raqoarfhdoeujulvqmdo']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('omgghgnzjmecpzqmtcvw'), ['fjhlzbszodmzavzg', 'gfofrnwrxprkfiokv', 'jmjiiqpgznlmyrxwewzqzbe', 'pkyrsqkltlmxr', 'crqgkgqkkyujcyoc', 'endagbcxwqhueczuasykmajfsvtcmh', 'xytmxtrnkdysuwltqomehddp', 'etmdxyyfotfyifwvbykghijvwv', 'mwqtgrncyhkfhjdg', 'iuvymofrqpp', 'pgllsdanlhzqhkstwsmzzftp', 'disjylcceufxtjdvhy']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('mznihnmshftvnmmhnrulizzpslq'), ['nrul', 'mshftvnmmhnr', 'z', 'mhnrulizzps', 'hftvnmmhnrul', 'ihnmshftvnmmhnrulizzp', 'izz', '', 'uli', 'nihnmshftvnmmhnru', 'hnrulizzp', 'nrulizz']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('ruqmqrsxrbftvruvahonradau'), ['uqmqrsxrbft', 'ftv', 'tvruvahonrad', 'mqrsxrbftvruvahon', 'rbftvruvah', 'qrsxrbftvru', 'o', 'ahonradau', 'a', 'ft', '', 'u', 'rsxrbftvruvahonradau', 'ruvahon', 'bftvruvahonradau', 'qrsxrbftvru', 't', 'vahonrada', 'vruvahonradau', 'onra']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('gpsevxtcoeexrltyzduyidmtzxf'), ['exrltyzduyid', 'vxtcoeexrltyz', 'xr', 'ltyzduyidmt', 'yzduy', 'exr', 'coeexrltyzduy', 'coeexrltyzduy', 'rlty', 'rltyzduyidm', 'exrltyz', 'xtcoeexrlty', 'vxtcoeexrltyzduyidm', '', 'coeexrl', 'sevxtcoeexrltyzdu', 'dmt', '']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('dyhycfhzyewaikgursyxfkuv'), ['sktnofpugrmyxmbizzrivmhn', 'fhlgadpoqcvktbfzncxbllvwutdawmw', 'eewzjpcgzrqmltbgmhafwlwqb', 'tpogbkyj', 'rtllntxjgkzs', 'mirbvsqexscnzglogigbujgdwjvcv', 'iktwpgjsakemewmahgqza', 'xgfvzkvqgiuoihjjnxwwpznxhz', 'nxaumpaknreklbwynvxdsmatjekdlxvklh', 'zadzwqhgfxqllihuudozxeixyokhny', 'tdqpgfpzexlkslodps', 'slztannufxaabqfcjyfquafgfhfb', 'xvjldhfuwurvkb', 'aecv', 'uycfsughpikqsbcmwvqygdyexkcykhbnau', 'jr']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('vbcsettndwuntnruiyclvvwoo'), ['dwuntnru', '', 'ttndwuntnruiyclvv', 'ntnr', 'nruiyclvvw', 'wo', '', 'bcsettndwuntnruiycl', 'yc', 'untnruiyclvvw', 'csettndwuntnr', 'ntnruiyclvvwo']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('pqqnugshlczcuxhpjxjbcnro'), ['dpeedqy', 'rtsc', 'jdgla', 'qkgudqjiyzvlvsj', 'xmfxawhijgxxtydbd', 'ebgzazqthb', 'wyrjhvhwzhmpybnylirrn', 'iviqbyuclayqketooztwegtkgwnsezfl', 'bhvidy', 'hijctxxweboq', 't', 'osnzfbziidteiaifgaanm']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('loqchlxspwuvvccucskuytr'), ['', 'k', 'qchlxspwu', 'u', 'hlxspwuvv', 'wuvvccucsku', 'vcc', 'uyt', 'uvv', 'spwu', 'ytr', 'wuvvccucs', 'xspwuv', 'lxspwuvvccuc', 'spwuvvccu', 'oqchlxspwuvvccucskuy']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('pjjyzupzwllshlnatiujmwvaofr'), ['lnatiujmwvao', '', 'zupzwllsh', 'nati', 'wllshl', 'hlnatiujmwv', 'mwvao', 'shlnat', 'ati', 'wllshlnatiujmwvao', 'wllshlnatiujmwvaofr', 'nat']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('iketunkleyaqaxdlocci'), ['nkleyaqaxd', 'etunkleyaq', 'yaqaxdlocci', 'tunkleyaq', 'eyaqaxdlocc', 'leyaq', 'nkleyaqaxdl', 'tunkleya', 'kleyaqa', 'etunkleya', 'leyaqa', 'dlo', 'yaqa', 'leyaqaxd', 'etunkleyaq', '']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('drqianqtangmgbdwruvblkqd'), ['wusajejyucamkyl', 'wsgibljugzrpkniliy', 'lhwqqiuafwffyersqjgjvvvfurx', 'jfokpzzxfdonelorqu', 'ccwkpcgac', 'jmyulqpndkmzbfztobwtm', 'rwrgfkccgxht', 'ggldjecrgbngkonphtcxrkcviujihidjx', 'spwweavbiokizv', 'lv', 'krb', 'vstnhvkbwlqbconaxgbfobqky', 'pvxwdc', 'thrl', 'ahsblffdveamceonqwrbeyxzccmux', 'yozji', 'oejtaxwmeovtqtz', 'zsnzznvqpxdvdxhznxrjn', 'hse', 'kcmkrccxmljzizracxwmpoaggywhdfpxkq']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('yasnpckniistxcejowfijjsvkdajz'), ['slkpxhtsmrtvtm', 'crsbq', 'rdeshtxbfrlfwpsqojassxmvlfbzefldavmgme', 'ipetilcbpsfroefkjirquciwtxhrimbmwnlyv', 'knjpwkmdwbvdbapuyqbtsw', 'horueidziztxovqhsicnklmharuxhtgrsr', 'ofohrgpz', 'oneqnwyevbaqsonrcpmxcynflojmsnix', 'shg', 'nglqzczevgevwawdfperpeytuodjlf']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('ueptpscfgxhplwsueckkxs'), ['ohhygchclbpcdwmftperprn', 'dvpjdqmqckekndvcerqrpkxen', 'lohhvarnmyi', 'zppd', 'qmqxgfewitsunbuhffozcpjtc', 'hsjbioisycsrawktqssjovkmltxodjgv', 'dbzuunwbkrtosyvctdujqtvaawfnvuq', 'gupbvpqthqxae', 'abjdmijaaiasnccgxttmqdsz', 'uccyumqoyqe', 'kxxliepyzlc', 'wbqcqtbyyjbqcgdbpkmzugksmcxhvr', 'piedxm', 'uncpphzoif', 'exkdankwck', 'qeitzozdrqopsergzr', 'hesgrhaftgesnzflrrtjdobxhbepjoas', 'wfpexx']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('ldrzgttlqaphekkkdukgngl'), ['gttlqaphekkkdukgn', 'ekkkd', 'gttlqaphe', 'qaphek', 'h', 'kdu', 'he', 'phek', '', 'drzgttlqaphekkkd']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('ololo'), ['ololo', 'ololo', 'ololo']); - -select 1 = multiSearchAnyUTF8(materialize('иечбпрхгебилцмпфвжцс'), ['лцмпфвж', 'ечбпрхгебилц', 'фвж', 'мпфвж', 'вжцс', 'пфвжцс', 'ц', 'чбпрхгебил', 'илцмп', 'фвж', 'ечбпрхгеби', '', 'б', 'хгеб', '', '', 'ил', 'ебилцмпфвжцс']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('змейдмоодкшуищвеишчддуцпх'), ['здсщесгдкзмчбжчщчиоо', 'чфззцмудщхтфрмсзрвшйщ', 'рлунбнзрфубуббдочтвлзмпгскузохк', 'ктзлебцам', 'вчспмж', 'нгкк', 'гпзйа', 'щпйкччнабакцтлапсбваихншхфридб', 'афсузжнайхфи', 'йрздеучфдбсвпжохрз', 'ошбечпзлг', 'полшхидфр']) from system.numbers limit 10; -select 1 = multiSearchAnyUTF8(materialize('лшнуухевгплвйужчошгнкнгбпщф'), ['гбпщф', 'б', 'ф', 'чошгнкнг', 'йужчо', 'гплвйужчошгнкн', 'бпщф', 'плвйужч', 'шгнкнг', 'хевгплвй', 'плвйужчошгн', 'вй', 'лвйужчошгнкнгбпщф', 'лвйужчошгнкн']) from system.numbers limit 10; -select 1 = multiSearchAnyUTF8(materialize('кцпгуоойвщталпобщафибирад'), ['ойвщталпобща', 'щта', 'пгуоойвщтал', 'ф', 'общ', 'цпгуоойвщталпобща', 'побщ', 'ф', 'цпгуоойвщталпобщафиб', 'побщаф', 'лпобщафи', 'цпгуоойвщталпобщафи', 'пгуоойвщталпобщаф', 'талпоб', 'уоойвщталпо', 'гуоойвщтал', 'уоойвщталп', 'щ', '', 'цпгуоойвщталпобщафибирад']) from system.numbers limit 10; -select 1 = multiSearchAnyUTF8(materialize('фвгйсеккзбщвфтмблщходео'), ['еккзбщвфтмблщходе', 'йсеккзбщвфтм', 'вфтмблщходео', 'вгйсеккзбщ', '', 'йсеккзбщвфт', 'бщвфтмблщход', 'ккзбщвфтмблщход', 'ккзбщвфтм', 'еккзбщвфтмблщходе', 'еккзбщвфтмблщх', 'вгйсеккзбщвф', 'оде', 'оде', '', 'бщвфтмблщх', 'б', 'йсеккзбщвфтмблщходео', 'вфтмблщ', 'кзбщ']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('хбаипфшнкнлтбшрскшщдувчтг'), ['хгшгднфуркшщвфгдглххс', 'цогчщки', 'тдмщшйзйхиквмб', 'етелфмшвмтзгеурнтбгчнщпмйпйжжциш', 'чсбк', 'ибащлшздеуревжйфуепфхкузбзао', 'дкмбщдсбжййсвгкхбхпшноншлщ', 'щхбеехнцегрфжжу', 'збфлпгсмащр', 'скчдигцнсзфрещйлвзнбнл', 'освзелагррдоортлрз', 'утхрч', 'йкбрвруенчччпшрнгмхобщимантешищщбж', 'жгивтеншхкцаргдасгирфанебкзаспбдшж', 'ййекжшщцщ', 'ефдсфбунйчдбуй', 'бвжцирзшмзщ', 'випжцщйзхнгахчсцвфгщзкдтвчйцемшлй', 'лдрфгвднеиопннтчсйффвлхемввег', 'бмтцжжеоебщупфчазпгхггцегнрутр']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('фбуоойпцщишщлбхчрсллзвг'), ['уччхщ', 'вщчсарфмйшгшпйфгмжугмщжкцщгйжзфл', 'кклл', 'лпнжирпсиуо', 'нчипзфщхнтштхйхщрпзитко', 'вйпсдергвцзсцсгмхпбз', 'чфщдфоилгцевпц', 'чааиае', 'чгингршжтчпу', 'щетбнгутшйсгмвмучдхстнбрптничихб']) from system.numbers limit 10; -select 1 = multiSearchAnyUTF8(materialize('лйвзжфснтлгбгцерлзсжфещ'), ['зсжф', '', 'бгц', 'зжфснтлгбгц', 'л', 'цер', 'жфснтлгбгц', 'тлгбг', 'це', 'гбгцерл', 'нтлгбгцерлзсж', 'жфещ', 'взжфснтлг', 'фснтлгбгцерлзсжфещ', 'нтлгбгцерлзсж', 'зжфснтлгбг', 'взжфснтлгбгцерлз', 'взжфснтлгбгце']) from system.numbers limit 10; -select 1 = multiSearchAnyUTF8(materialize('нфдцжбхуучеинивсжуеблмйрзцршз'), ['чеинивсжуеблм', 'жуебл', 'блмйрзцрш', 'цр', 'м', 'фдцжбхуучеинивсжуеблмйрзцр', 'нивсж', 'ивсжуеблмй', 'й', 'всжуеблмйрзцршз']) from system.numbers limit 10; -select 1 = multiSearchAnyUTF8(materialize('всщромуцйсхрпчщрхгбцмхшуиоб'), ['муцйсхрп', '', 'уцйсхрп', 'сщромуцйсхрпчщ', 'схрпчщр', 'сщромуцйсхрп', '', 'уцйсхрпчщрхгбцмх', '', 'цмхшуиоб', 'гбц', 'пчщр', 'цйсхрпчщр', 'омуцйсхрпч', 'схрпчщрхгбцм', 'йсхрпчщрхгбцм', '', 'пчщрхгбцм', 'уцйсхрпчщрхгбцмх', 'омуцйсхрпчщ']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('уузшсржоцчтсачтедебозцвчвс'), ['бомбсзхйхкх', 'отвгстзихфойукарацуздшгбщеховпзкй', 'мфнев', 'вйийшшггилцохнзбхрлхи', 'втинбтпсщрбевзуокб', 'оиойвулхкзлифкзиххт', 'зацччзвибшицщрзиптвицзхщхкбйгшфи', 'кнузршшднмвтощрцвтрулхцх', 'рчбкагчкпзжвтбажиабиркдсройцл', 'щргчкзожийтпдзфч', 'щбошгщзсжтнжцтлкщитеееигзцлцсмч', 'сцкк']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('щчбслгзвйдйжрнщчвфшй'), ['пдашзбалйнзвузкдвймц', 'щхтшйоч', 'фднвфигозржаз', 'рйфопхкшщвщдвл', 'цдкйхтусожпешпджпатфуиткп', 'щпбчсслгщййлвскшц', 'жпснс', 'уиицуувешвмчмиеднлекшснчлйц', 'пххаедштхмчщчбч', 'ичтмжз', 'лсбкчу', 'бгфдвпзрл', 'йицц', 'цфйвфлнвопкмщк', 'бгщцвбелхефв', 'мймсвзаелхнжйчохомлизенфш', 'трйднхндшсщмпвщомашчнгхд', 'жфцнифлгдзйе', 'зспкшщщенбцжгл', 'рщтб']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('шщпееасбтхогвгвцниуевисгшгбч'), ['гпа', 'стимсркзебхрвфпиемзчзу', 'нзгофухвекудблкадбшшусбеулрлмгфнйгиух', 'кфиашфобакщворувгвкчавфзшх', 'гфпгщгедкмтгрдодфпуйддхзчждихгрчтб', 'тцтжр', 'рцйна', 'йцбпбдрреаолг', 'житсфосшлтгсщдцидгсгфтвлз', 'жвтнжедцфцтхжчщч']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('вхкшгфпфмнщаохтмизпврйопцуйзмк'), ['дтчбкхащаткифружжейабфйкйтрскбощиеч', 'фтоуабхмдааиснрбраттклмйонлфна', 'цадзиднщймшкщолттпгщбх', 'кштбчжтждпкцнтщвмухнлби', 'микудпдпумцдцгфахгб', 'ирик', 'емлжухвмк', 'чгуросфйдцшигцхжрухжпшдкфгдклмдцнмодкп', 'ттбнллквдувтфжвчттжщажзчлнбждчщцонцлуж', 'елцофйамкхзегхклйгглаувфтуувее', 'двкзчсифвтекб', 'шсус']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('йхцглкцвзтшщочпзмнчтуеао'), ['йечдай', 'дащжщзлосмй', 'афуккгугаазшрчпцнхщцтмлфф', 'чфтфскрфйщк', 'жлччкцшнфижтехппафхвщфс', 'бзжчв', 'щкщймнкщлпедидсу', 'оцбажцзшзйпптгщтфекртдпдзшодвойвох', 'йжддбссерхичгнчлкидвгбдзуфембрц', 'ктщвшкрщмдшчогхфхусдотсщтцхтищ', 'пшстккамнбнардпзчлшечхундргтоегцзр', 'нсрнфузгжррчнжначучиелебрб', 'шгжмквршжтккднгаткзтпвкгзхшйр', 'змквцефтулхфохбнхбакдичудфмйчп']) from system.numbers limit 10; -select 1 = multiSearchAnyUTF8(materialize('шждйрчйавщбйфвмнжоржмвдфжх'), ['ор', '', 'йрчйавщбйфвмнжо', 'вщбйфвмнжорж', 'ждйрчйавщбйфвмнжорж', 'йавщбйф', 'дф', 'вщбйф', 'бйфвмнжорж', 'мнж']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('кдшнсйршгвлицбенйбцфрсаччетфм'), ['асмун', 'йогкдчодиф', 'лштйбжнзфкикмпбитжшгкбоослщгзнщо', 'улштжцисцажзчштгжтфффабйлофедуфме', 'дрпгкчджихшзммймиамзфнуиорлищзгйвху', 'йиоршнйоввквбдвдзасма', 'члмвасмфрхжсхрбцро', 'лшкизщушборшчшастйсцкжцбонсшейрщ', 'масдфкршлупасвйфщфважсуфсейшзлащхрж', 'дгхшщферодщцнйна', 'цзфзждбавкжрткст', 'рфбожзееаце', 'кошомвгпрщсдквазчавожпечдиуйлщадфкгфи', 'бшпхнхсгшикеавааизцсйажсдийаачбхч']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('хтиелйтарквурйлжпеегфш'), ['зпмйвзуднцпвжкбмйрпушдуавднвцх', 'фбссчгчвжакуагдвижйтщтшоабпхабжш', 'щхшибаскрщбшрндххщт', 'сммрсцзмптисвим', 'цсргщфж', 'восжбшйштезвлкммвдхд', 'вбсапкефецщжквплуо', 'даеуфчвеби', 'бтптлжпин', 'шчддтнсйкщйщ', 'фжхщецпзчбйкц', 'цсвфпздхрщхцбуцвтег']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('апрчвзфжмбутццрйщкар'), ['индхжз', 'жилцовщччгстби', 'ажс', 'фктйамйтаг', 'шммнзачггоннксцушпчн', 'чдлйтзтоцдгзццисц', 'пнбтувщцдсчнщмсакрлгфмгрй', 'овмсйнщзушвщгуитщрхвйодф', 'бзлштезвлаижхбмигйзалчолшеунлц', 'фкжпеввгшгащз', 'тменбщжмсхщсогттршгек', 'чап', 'х', 'шкомегурлнйпшбщглав']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('двхопооллаеийтпцчфжштнргкк'), ['йймчнздешхбццбжибопгктрнркевпиз', 'фйрохсамщцнмф', 'ййхфдпецжзгнуорвбплоахрфиле', 'пкллкацнвдббогг', 'йщдезамтжйзихщжмцлх', 'гдзувмщиеулиддердшпитвд', 'фхтунйшзхтщжтзхгцорошднпбс', 'фнситбеелцдкйщойлатиуухгффдвищсше', 'нзщщщндцрнищпхйвтбвмцтнуадцбву', 'вбщкапшнв', 'зйлмуимчскщнивтшлчмуузщепшйр', 'шжбвйдр', 'гддждбкначдттфшжшхпфиклртпгм', 'еншащцфафчнгбнщххнзочбтпушщорегшцзб', 'уунеущкззоетбучкц', 'щасифзоажребийещ', 'пщбххсдгйтт', 'хшсчуотрт', 'жкднйрозбцшужчшбкккагрщчхат', 'шачефцгч']) from system.numbers limit 10; - -select 0 = multiSearchAnyCaseInsensitive(materialize('QWyWngrQGrDmZxgRnlOMYHBtuMW'), ['ZnvckNbkeVHnIBwAwpPZIr', 'NCzFhWQmOqIGQzMORw', 'tDYaxfQXWpKNLsawBUUOmik', 'IMveCViyAvmoTEQqmbcTbdfjULnnl', 'NRvsdotmmfwumsDpDtZU', 'mnqVnwWOvMiD', 'HXpHrMvGQpbuhVgnUkfFPqjpoRdhXBrFB', 'awtr', 'IMIdOmMHZccbOZHhWOKcKjkwwgkJSfxHDCzR', 'jPLISbIwWJEKPwgvajTxVLws', 'HBfRrzEC', 'VXsysGnAsFbqNOvIaR', 'upCaeaIOK', 'GUDFkrzBiqrbZVnS', 'MoCOePXRlVqCQpSCaIKpEXkH', 'rfF', 'fjhMEpySIpevBVWLOpqi', 'KdeskLSktU', 'vjUuNUlBEGkQyRuojZLyrmf', 'SvSxotkTKCeVzNICcSZLsScKsf']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('gcDqqBCNqhQgVVgsxMXkevYIAxNl'), ['BHnoKRqOoKgmOVkjtehGSsInDvavDWOhkKAUL', 'nYqpmKPTWGdnyMcg', 'TIplHzsSXUz', 'SiQwpQgEdZ', 'YoJTWBJgsbJvq', 'CwyazvXERUFMCJWhTjvltxFBkkvMwAysRLe', 'tXUxqmPbYFeLUlNrNlvKFKAwLhCXg', 'vUbNusJGlwsOyAqxPS', 'ME', 'ASUzpELipnYwAknh', 'VtTdMpsQALpibryKQfPBzDFNLz', 'KmujbORrULAYfSBDyYvA', 'BaLGNBliWdgmqnzUx', 'IzwKIbbSUiwhFQrujMgRcigX', 'pnS', 'UKSZbRGwGtFyLMSxcinKvBvaX']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('HCPOPUUEVVsuZDbyRnbowGuOMhQ'), ['UzDbYrNBoWgUo', '', 'pUUEVVsUzdByrNB', 'nBO', 'SUZdbYrNbOWgUoMH', 'pOpuUevVSUZDbYRnb', 'bowGUoMh', 'VsUZDbyrNbo', 'suzdBYrN', 'uueVvsUZDBYRnBoW', 'gUom', 'eVvsuzDBYRNBoWgUOM']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('RIDPJWYYSGBFWyXikHofbTcZAnj'), ['aFxQyVe', 'OcnZBgPsA', 'iBQaH', 'oesSvsWtgQprSSIPaDHdW', 'EfytiMfW', 'qHiFjeUvQRm', 'LfQkfmhTMUfoTOmGJUnJpevIoPpfpzMuKKjv', 'scYbCYNzJhEMMg', 'yTLwClSbqklywqDiSKmEdyfU', 'HYlGFMM', 'TMQhjOMTImXbCv', 'AVtzpxurFkmpVkddQANedlyVlQsCXWcRjEr']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('GEsmYgXgMWWYsdhZaVvikXZiN'), ['wySd', 'smYgxGMWWYsDHZ', 'vIk', 'smyGxgmwWysDHzAvvikxZi', 'WYsdHZAvVI', 'YGxGmwWYSDhzavvI', 'XzI', 'ySDhZAvvIK', '', 'myGXgmwWySdHz', 'MYGxgmwWySdHZaVvik', 'wYsDhzAvvikXz', 'wwYsdHzav', 'Z']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('XKCeCpxYeaYOWzIDcreyPWJWdrck'), ['tTRLUYJTkSWOabLJlIBshARIkwVRKemt', 'jQgn', 'wdNRsKIVunGlvwqkwn', 'BsbKGBJlkWQDBwqqeIjENvtkQue', 'yLuUru', 'zoLGzThznNmsitmJFIjQ', 'WFKnfdrnoxOWcXBqxkvqrFbahQx', 'QHbgRXcfuESPcMkwGJuDN', 'NPqfqLS', 'bi', 'HnccYFPObXjeGYtrmAEHDZQiXTvbNcOiesqRPS', 'KobVCJewfUsjBXDfgSnPxzeJhz', 'AqYNUPOYDZjwXx', 'xbZydBGZFFYFsFHwm']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('AnIhBNnXKYQwRSuSqrDCnI'), ['', 'HBNNxkyqWRS', 'xKyqwrSUSQR', 'yQwr', 'ihbnnxKYQWrsUS', 'bnnXkYqwrSuS', 'qWRs', 'nXKyqWRSUS', 'qrdcN', 'NiHBnNXkYQWrS', 'NnXkYQwRSUsqRDCn', 'rSusqRd']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('OySHBUpomaqcWHcHgyufm'), ['lihJlyBiOyyqzeveErImIJuJlfl', 'WyfAXSwZPcxOEDtiCGBJvkCHNnYfA', 'hZ', 'fDQzngAutwHSVeoGVihUyvHXmAE', 'aCpcZqWKdNqTdLwBnQENgQptIyRuOT', 'PFQVrlctEwb', 'ggpNUNnWqoubvmAFdjhLXzohmT', 'VFsfaLwcwNME', 'nHuIzNMciJjmK', 'OryyjtFfIaxViPXRyzKiMu', 'XufDMKXzqKjYynmmZzZHcDm', 'xWbDgq', 'ArElRZqdLQmN', 'obzvBzKQuJXZHMVmEBgFdnnQvtZSV', 'ZEHSnSmlbfsjc', 'gjmWPiLylEkYMTFCOVFB']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('NwMuwbdjhSYlzKoAZIceDx'), ['ZKOaZ', 'wBDJhsYlZKo', 'hSy', 'MUwbDjHsyl', 'sYlzK', 'ylZKOAZ', 'y', 'lZKoaZICEdX', 'azIce', 'djHSylZkoAzice', 'djHsYLZKoAzi', 'dJHSYlZK', 'muWbDJHsYLzKOaziC', 'zi']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('gtBXzVqRbepHJVsMocOxn'), ['DidFXiqhRVBCHBVklLHudA', 'yEhumIpaYXlj', 'iaEmViTRLPM', 'vTwKBlbpaJZGYGdMifOVd', 'zvgfzWeLsMQNLutdAdCeuAgEBhy', 'Ca', 'iHabiaRoIeiJgSx', 'EBfgrJnzHbuinysDBKc', 'kT', 'SGIT', 'BTRuKgHDuXMzxwwEgvE', 'OWJIeTLqLfaPT', 'BQM', 'yMimBqutKovoBIvMBok', 'zIBCYVNYAwu', 'EFDEFWGqvuxygsLszSwSiWYEqJu', 'QJDIXvPOYtvhPyfIKqebhTfL', 'ssALaXRxjguUIVKMCdWRPkivww']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('MowjvqBkjnVTelCcXpoSuUowuzF'), ['Su', 'vqBkJNvTelC', 'Elccxp', 'vtElc', 'JVqBkJnVTELCcxpOsU', 'OsUuOWUz', 'ElccxPoSU', 'wJVQbkJNVtElCC', 'xpOSUUo', 'VQbkJnvTELCCXp', '', 'TeLcCxPOsuuO']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('VfVQmlYIDdGBpRyfoeuLffUUpMordC'), ['vqMLyIddgBPrYFoEulFFu', 'lyIDdgBPrYFOeul', 'dGBPRYFOeUlffUupmOrD', 'OEulffU', 'pMordc', 'FVqmlyiDdgBpRyFoeUlFfuUpMOrD', 'PmO', 'o', 'YiDDgbPRYFOe', 'DGBPryfoeU', 'yIDdgbpRyFOeULfFU', 'lyIddgBPryfoeulfFuU', 'gbPrYfOeUlFfuupmO', 'yFoeULF']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('CdnrzjzmwtMMPLjgcXWsbtrBs'), ['RfgIUeerlPIozKpRQR', 'QRoYzjZlgngJxX', 'mEbqlBIzTQH', 'UmrfJxKyTllktPfyHA', 'ukoZeOPA', 'pbbRaUcJijcxt', 'Rg', 'lSBG', 'HvuwuiqVy', 'Fo', 'aGpUVjaFCrOwFCvjc', 'zKhfkgymcWmXdsSrqAHBnxJhvcpplgUecg', 'ioTdwUnrJBGUEESnxKuaRM', 'QciYRCjRDUxPkafN']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('miTQkQcxbKMwGOyzzRJpfXLyGx'), ['yMwgQQJkeshUugm', 'wGVe', 'XncShWqjp', 'KWjGQCOsfMKWRcgCfebkXZwZ', 'SFWbU', 'WdFDMIcfWeApTteNfcDsHIjEB', 'XRuUJznPOCQbK', 'tibBMGZHiIKVAKuUAIwuRAAfG', 'VVCqVGGObZLQsuqUjrXrsBSQJKChGpZxb', 'bWYAOLuwMcwWYeECkpVYLGeWHRrIp', 'SLzCgfkRWmZQQcQzP', 'VvfOhFBhfiVezUSPdIbr']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('KXoTIgVktxiXoEwfoLCENiEhz'), ['oLCENie', 'xix', 'en', 'IgvktxIXoEWFOLCEnieHz', 'xOEWFoL', 'LC', 'ktxIxoEwfolCenie', 'ce', 'oTIGvktXIXOE', 'eW', 'otigVKTXIXOEwFolC', 'E', 'CEni', 'gVKtxIxoEwfOLCENieh']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('DXKzSivrdLuBdCrEYfMEgPhOZ'), ['', 'sIVRDlUBdcr', 'luBDcrE', 'rDLUbDCreY', 'KzSiVRdLuBDCr', 'dcREYFme', 'lUbdCReyFMEgph', 'sivrDlubdCr', 'BdcreYfMEgP', 'ZSiVrdluBDCrEYfmegpHOZ']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('lTvINMXVojkokvNBXPZOm'), ['ZQOJMEJfrjm', 'vIpmXnGlmWze', 'wbdDKcjrrIzBHypzJU', 'omotHOYbZjWfyVNeNtyOsfXPALJG', 'SXxu', 'yZPDFsZq', 'OVYVWUjQDSQTKRgKoHSovXbROLRQ', 'RnXWZfZwHipewOJimTeRoNRYIdcZGzv', 'sizoEJibbfzwqFb', 'vgFmePQYlajiqSyBpvaKdmMYZohM', 'ENsFoFCxDQofsBSkLZRtOcJNU', 'nG']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('LsTqxiGRdvQClVNBCGMOUHOAmOqPEC'), ['NdFuUQEUWaxS', 'fdOHzUzineBDnWJJvhPNZgB', 'rYAWGIBPxOLrjuquqGjLLoIHrHqSFmjh', 'IVgYBJARY', 'ToivVgUJAxRJoCIFo', 'yQXGrRjhIqFtC', 'PNYdEPsWVqjZOhanGNAq', 'nrQIDDOfETr', 'usJcPtiHKhgKtYO', 'vPKqumGhPbmAJGAoiyZHJvNBd', 'eXINlP', 'WQeESQJcJJV']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('gRzzQYOwLNiDcMFjXzSFleV'), ['XZSfLe', 'wLnIdcMFjxZSf', 'F', 'm', 'Le', 'qYoWLNidcMFjXzsf', 'zqyoWlNIdcMFj', '', 'oWlnIDCMfJxzsfL', 'wlNIdCmfjXzS']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('cYnMXJMJCdibMXoUQHEw'), ['BFrGFZRgzwHGkUVbBiZMe', 'piORdVIWHMBsBDeJRLbGZAHGBrzNg', 'bmDePbTPnFQiCFfBJUxAEYNSbgrOoM', 'gtzeAGwqjFrasTQUgAscfcangexE', 'okLG', 'l', 'EBkkGYNZZURgFgJPlb', 'HDQVngp', 'vEHhtBqWhZHCOrqEKO', 'fgqdFc', 'COig', 'VftTpSXAmTmvnShHJqJTdEFcyKPUN', 'WDI', 'knBm']) from system.numbers limit 10; - -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('мтдчЛВЖАгвзщущвкфИКмТбжВ'), ['щУщвкФИкМ', 'чЛвжАГвЗЩуЩвКФикм', 'ДчлвЖАГвзЩУЩвКфИКМтБЖВ', 'ЖагвзщуЩВКФикМТБжВ', 'ВжагВзЩУ', 'гВЗщущвкфИКмТБж', 'ГвЗщ', 'щВкФикМТБЖВ', 'вЖАГВзщущ', 'взЩуЩвКФИкМТ', 'ЧЛВЖагвЗщуЩВк', 'тДчлвЖагвзЩуЩвкфИк', 'ТДЧлвжаГВзЩущВ', 'тДчлВжАГВЗЩУ']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('дтрцФхИнпиОШфдАгзктвбУвсб'), ['чТрВиУРФсРпДЩОащчзЦНцхИДА', 'ЗжмПВтмиойУГхАЦПиДУЦноНуййЩХаФТофшЩ', 'уБшлОЙцМПгетЖЧетШжу', 'ЧзИАУХобФрачТеХОШбМщЖСамиМВАКРщАЦ', 'ВйвТзхЙФЧоАЖвщиушАз', 'ЦшИфххкжиФйСЛЛНЛчВоЙВПпхиИ', 'ОатЕтщкЦпбСБйцОшГШРОшхБцщЙЧиУЩЕеФлщ', 'цСПпЧА', 'ШЧНфПмФсКМКДВЦАоФчОУеТЦИзЦ', 'зАбдЛНДГИ', 'фхЩлЗДНСсКЖИФлУАбЛеТФЕпЖлпПхЙиТЕ', 'иВшкНслТКМШЗиДПйфвйНкМЛхеФДзИм', 'лпушПБванпцев', 'ЧОшЧЧмшЦЛЙйГСДФйЛАв']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('нщМаНдЧЛмиВврПокПШмКБичкхвРГ'), ['АЙбаЙйШЛЙРЦмЗчВеИЕощсЦ', 'щЦФдВжчТСЩВКЦСпачЙсумщАтЩувеиниХПДоМС', 'иоАкДРршуойиЩищпрфВаЦПж', 'еЖПйШкГжЧтоГЙМВ', 'ЩПалиБ', 'ТвВлт', 'оХжйЛФеКчхЗВвЕ', 'ерцЩ', 'ШХЖОАрзеп', 'ККМрфктКГишпГЩхаллхДиВИИЛЗДеКйХмжШ']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('вШЙчоМгОттЧАЕнЧаВеЦщчЧошМУ'), ['ЧОмГотТчАЕН', 'ОмГотТчАЕнчАвецЩчч', 'ЧАВецЩч', 'ТЧАеНЧаВ', 'ттчаЕнча', 'ТчАЕ', 'мготтЧАенчавЕЦЩ', 'НЧаВец', 'тТЧаенчАвецщчЧошм', 'Ав', 'ТЧаЕнчавецщчЧоШму', 'аЕнЧав', 'АеНЧав', 'шйЧомГОТТчаЕнчАВЕ', 'шйчоМгОтТЧаЕНчаВеЦщЧчош', 'МУ', 'ошМ', 'гОТтЧаеНЧА']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('фйадзЧмщЖШйЖЛшцГигцШ'), ['НТХеМРшДНУЗгадцуЧИ', 'жпСИКЩМлНлиоктлЦИвНЛ', 'КхшКРчХ', 'кгТЗаШИарХЧЛЖмСЖм', 'ОмиЛй', 'жЕРбФЩНуЕКЕАВоБМОнАЕнКщшзйПкОЗ', 'гиЗдадкбжХМЗслшВИШай', 'двтЗйЙНгПуТзД', 'ТНкмаВЕФ', 'Шеа']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ШЕшхмеЦХеАСКощеКИфлсТЧИЗЛ'), ['КифЛсТ', 'ХеаСКощЕк', 'КифлсТЧ', 'шХМеЦхЕаскОЩеКИ', 'ЕшхмЕцХеаСК', 'ХЕасКоЩ', 'чИ', 'ЕцхеАсКОЩек', 'ЩЕкИфлс', 'асКощЕкифЛсТ']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('шоКнВЕрОЖЛпУйХзСугКПВжиРсЙпо'), ['игВербфНахчжЙггч', 'лтимрдфЕг', 'нкеаЖАшНБвйСдКИВГДшАГиАТнФШ', 'МжсТЙМГОииУКВГнцткДнцсоАд', 'ХтпгУСдБдцАЖЛАННоЕцзЕшштккз', 'ншУЦгФСЖшмс', 'нЩшМ', 'гоЖхМшаЕмаДРЧБЛИТпмЗОоД', 'фГКШхчФбЕГЛйкчПИЙххуМГНШзхг', 'ХпХщПЦАзщтг']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('кЧбоЗХфвБХхусмШгНаШШаГзш'), ['Ури', 'лММшткфНзцЦСВАдЩПМШфйОМшефигЖлуЕП', 'сМтЕдчЦафйСТЖЗфлРЙПЦдипжШскцВКХЦЖ', 'АУкжИФцшЛБЦЧм', 'ФПлнАаДСХзфоХПСБоСгМТОкЗЧйЛ', 'ЦшСГЛрцДмнНнХщивППттжв', 'жзЕгнциФ', 'МШЛсЙЧтЛАГжд', 'уИиЕжцоРНх', 'ЧбйГуХтшОНкрЧИеПД', 'ЦдЩЕкКвРЦжщЧциекЗРйхрббЖуЧ', 'иВжен', 'ГчОржвБГсжштРЕБ', 'ШоЖдуЙфчсЧегумщс', 'йчЙГ', 'РДедвТ']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ткРНбЩаРкгГчХшецИкНЕнСЖкйзАуУЖ'), ['ХлЖхУИллрРННйЗйсРуШЧвМбЧЧщфФЦц', 'СЛчКБцСФДшлфщаФлЙСзШабмбхуБжТСТ', 'УКУиввЗЩуВМцпчбпнДГбпЕЖрПбИДркМРОеЧмЧдГ', 'ПчщвШЩвГсЛмММГБ', 'хКЦЧсчжХЩИЖХеНнтоФЦлнмЛЧРФКпмСшгСЧДБ', 'удсЗйУДНЧУнтЕйЦЗЖзВСх', 'хПЖЙИрцхмУкКоСмГсвПаДОаЦНЖПп', 'сВОей', 'ЩЦжщоабнСгдчрХнЩиМХзжЩмФцррвД', 'ЦИсйнЦДоЕДглЕЦД', 'жзйПфБфУФоцзмКЩГПЧХхщщПТпдодмап', 'ДНХГНипжШлСхХхСнШЩЛИснУйЧЩЖДССФфиС', 'ОйЩНнйЕшцФчБГЛвхЖ', 'КЧРВшИуШйВфрпБНМсУмнСЦРпхЗАщЗУСвЧйБХтшХЧ', 'зЛбНу', 'ЗСрзпшЕйРржПСсФсШиМдйМЦГхдйтРКЩКНцкбмгС', 'СУццБуКнчОищГ', 'уЕГЧлЗБНпУисЕЛ']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ВЦХсЖЗЧЙБЗНбРитщстеМНжвВ'), ['итщст', 'ЧйБЗНбрИтщстЕМнЖ', 'ХСЖЗЧйбзНБриТщ', 'Темнж', 'сЖзЧЙБзнб', 'хСжЗчйБзнБрИтЩстЕм', 'БзнБРиТщ', 'ЗчЙбзНбрИТщ', 'чйбЗНбри', 'зЧйбзНБРИ', 'нБРитщсТе', 'зНб', 'цхСжзчйБЗнБРИТЩСтЕм', 'жЗЧЙБЗнбрит']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ХцМШКАБАОххЕижгГХЩГиНциД'), ['ОРАБЕРВомЛфГНМИКупбхЛаАкЗдМзтш', 'лЗУЩнлбмиЛАфсгМРкцВтлснййишИНАС', 'ТлжлУоУгжукФжЖва', 'жоСШПоУНЩшРМГшОЛзЦБЛиЛдТхПДнфжн', 'чнСУЗбДаГогжДфвШКеЙПБПутрРпсалцоБ', 'ЙозоПщчакщаАлРХбЦгац', 'иаИСсчЙЧБШорлгЧТнчцйзоВБХбхЙФтоЩ', 'ПСзсБЗЕщурфДЛХйГИеПНрмииаРнвСФч', 'ЦйЖЕуТфЖбхЩМтйсЙОгЛбхгтКЕЩСАЩ', 'гтЗуЩлужДУцФВПЛмрБТсСНпА', 'тГвлбчЗМасМЖхдЕгхмЩксоЩдрквук', 'ВРаг']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('тУйВЖдНнщцЗЖфКгфжГфиХСБЕЩ'), ['КгФЖГФи', 'сБе', 'ЖФ', 'гфжгФИхсбе', 'ВЖДНнщЦзжфКГфЖгфИхсбещ', 'ВЖДНнЩЦзжфкГ', 'вЖДННЩЦзжФКГфЖгФ', 'ф', 'НщЦЗж', 'нщЦЗЖФк', 'Их', 'дННщцзЖФКгф', '', 'нщцзжФкг']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ШкКРаоПеЗалРсТОиовРжгЙЧМКЛШ'), ['рчсажЕК', 'пЧТМфУрУММждЛйжзУрбкмам', 'бАШеНмВШзлзтушШШсхОсцрчЙПКИБнКжфЧЕХ', 'ЖМЛшбсУМкшфзочщАЖцМбмШСбВб', 'гтРХсщхАИОащчлИЧуйиСпСДФПбРл', 'ЧуОРУаоойГбУппМйЩФДКПВ', 'уУпугйРЕетвцБес', 'ЙЖЦТбСЖж', 'ИБКЛ', 'ТДтвОШСХГКУИПСмФМтНМзвбЦрднлхвДРсРФ', 'вВгНЙХИрвйЕЗпчРГЩ', 'ПчмТуивШб']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('РлчгхзуВШежХЦуМмнВЙщдцО'), ['ХшвМЦДШпЩОСшЦПдруа', 'ФИЦчУвРкпнПшИЕСЧАувиХд', 'фшвбЦОИЗфпИУМщзОЧЗфВцЙПнмтаТгг', 'мЖЩйавтнМСЛ', 'НВбШ', 'ааФДДрВвЙТдПд', 'ЗнчЧущшхЙС', 'рзуСзнеДфЩПуХЙЕл', 'ШСЩсАгдЦбНиШмшКрКс', 'ггнЕфБГзрОнАГЙзЧеИП', 'вшТИпЧдЖРкМНшзпиоиЩчзДмлШКТдпЦчж', 'фЦТЙц', 'ОтУшмбптТКЗеПлЧцЛОкЩБпккфгИн', 'ЩпвхпЗлШБЦ']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ЙбйнхНщЧЖщчГОАпчФнЛШФбгЛа'), ['щчг', '', 'апЧфНЛШфб', 'ЙнхНЩЧЖщчгОАПЧф', 'ХНщЧжЩЧгоАпч', 'ХНщЧжщчГо', 'нщЧжщчГОа', 'чЖЩЧГоапЧФНл', 'оапчФ', 'щЧГОАпЧФНлшФ', 'ЩЧГОАпЧФНЛшфБг', 'БЙНхнщчЖщчГоаПЧФНЛШФБгЛ', 'ОапЧфн', 'ф', 'БглА', 'ш', 'шфбГ', 'ХнЩЧЖщчГоА', 'ХНщчжщЧгоапч', 'хНЩчжщЧГоапчфнлшФбгЛ']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('кдЙДТЩеВЕфйКЩЦДиКБМф'), ['щфЛ', 'фЧЩЩичрКйЦКхеИИАпоБВЙЗбДАФио', 'мИтиЦРоВЙсБбСлНзиЛЧОфФевТмижщК', 'тЙгнКШфНТЕБЛцтГШЦхШхБ', 'уаабРГрМЙпМаБуЗпБЙчНивЦеДК', 'мпВЛНДеКПУгРЛЛинзуЕщиВШ', 'ЩжКйШшпгллщУ', 'пршЙПцхХЗжБС', 'нбЗНЙШБш', 'йцхИщиоцаМРсвнНфКБекзЛкчТ', 'хсмЦмнТрЩкДТЖиХщцкЦМх', 'ГмЛАбМщЗцЦйаОНвзуЗмЕКПБЙмАЕЛГ', 'ОЦХРЗРмкжмРИЖИЙ', 'з', 'лЕТкпкдЗчЗшжНфо', 'ИТПфйгЖЛзУТсЩ', 'ОфрбЛпГА', 'МЖооШпЦмсуГцАвМЕ']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ЩГТРШКИОРБРеНЖПКиуМОкхЛугИе'), ['брЕнЖ', 'РбрЕНЖпКиУМокХЛу', 'ГТрШКИорБРеНЖпКиУМ', 'рШКиоРбрЕнЖпкИУМОК', 'ИорбрЕнЖПК', 'Окхл', 'шкИоРБРеНЖПк', 'ТРШкИоРБрЕнжПКИУМОкхл', 'КИОРБРЕнжпкиУм', 'Н', 'КиОРбРЕнЖпкИУмоКхл', 'к', 'ГтРшКИоРБРЕнЖпк', 'гтрШкиорбрЕНЖпк']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ШНвпкфЗвгДжУЙГлрТШаШЛгНЗг'), ['нЗБенВшщрЛАрблцщшБАдзччммсцКЖ', 'бЗЩхзЗЗбФЕйМоазщугБбмМ', 'рЙсВжВсхдйлЩгБтХлчсщФ', 'пиБшКРнбВБгЕуЖ', 'жПшнхпШзУБрУЛРНЩДиаГШщКдЕвшоуПС', 'чЕщкЗмДуузуСдддзгКлИнгРмЙщВКТчхзЗЛ', 'кЖУЗЖС', 'щххОВМшуажвН', 'фбцЖМ', 'ДШитЧЩДсйНбдШеООУдг', 'ЛХПфБВХЦТИаФПЕвгкпкпщлхмЙхГбц', 'чЦсщЗщрМ']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ФРХгаСлчЧОцкШгзмКЗшФфББвЧ'), ['кзШфФб', 'ГАслЧЧОцкшг', 'ФфббВЧ', 'ЦкШ', '', 'АслчЧОЦКШгзМкЗШффбБвч', 'РХгаслЧчОЦКШГз', 'РхгаслчЧОцКШгзМкзшФфБбВ', 'Шг', 'Ф', 'ХГАслчЧоцКШГзМкзш', 'ШгЗмКЗшфФб']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ЧдйШкхОлалщНйбССХКаФзОМрКЕЙР'), ['бссХкафзОм', 'ХОЛаЛщнйБссХкаФз', 'лаЛщнйБсСХ', 'ЩнЙбСсхКаФЗО', 'йБСсХКАФЗОмР', 'йшкХолаЛЩНйбсСхК', 'С', '', 'ЙшкхОлалщНЙБсСхКаФзом', 'Йр', 'щнЙБссхКАфзоМрК', 'рКе']) from system.numbers limit 10; - -select 1 = multiSearchFirstIndex(materialize('alhpvldsiwsydwhfdasqju'), ['sydwh', 'dwh', 'dwhfdasqj', 'w', 'briozrtpq', 'fdasq', 'lnuvpuxdhhuxjbolw', 'vldsiws', 'dasqju', 'uancllygwoifwnnp', 'wfxputfnen', 'hzaclvjumecnmweungz']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('kcwchxxyujbhrxkxgnomg'), ['jmvqipszutxfnhdfaxqwoxcz', 'nrgzkbsakdtdiiyphozjoauyughyvlz', 'qbszx', 'sllthykcnttqecpequommemygee', 'bvsbdiufrrrjxaxzxgbd', 'hdkpcmpdyjildw', 'frxkyukiywngfcxfzwkcun', 'dmvxf', 'esamivybor', 'eoggdynqwlnlxr']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('zggbeilrfpkleafjjldgyfgn'), ['rpypxkpgvljhqthneremvabcd', 'qchzlnsctuwkdxqcrjgihvtfxhqxfqsxm', 'vtozkivjyqcqetmqenuihq', 'fixcvjyzbzejmwdivjf', 'lydoolvnuuamwlnzbyuuwpqqjaxf', 'elkodwthxqpcybwezm', 'wpiju', 'wdzuuwumlqfvga', 'iokphkai', 'wkbwdstplhivjyk', 'wxfbhfturuqoymwklohawgwltptytc', 'jehprkzofqvurepbvuwdqj']) from system.numbers limit 10; -select 9 = multiSearchFirstIndex(materialize('bwhfigqufrbwsrnnkjdzjhplfck'), ['v', 'ovusuizkdn', 'ttnsliwvxbvck', 'uh', 'lfourtjqblwdtvbgtbejkygkdurerqqdwm', 'snmtctvqmyyqiz', 'ckpixecvternrg', 'gluetlfyforxcygqnj', 'igqufrbwsr', 'om', 'huwazltjsnohsrcbfttzwyvcrobdixsuerkle', 'gqufrbwsrnnkjdzj', 'hfigqufrbwsrn', 'lhhyosbtznyeqzsddnqkfxayiyyajggxb', 'igqufrbwsrnnkjdzjhplf', 'pl', 'jtbqaqakbkesnazbvlaaojppxlbxccs', 'gqufrbwsrnnkjdz']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('yevfiumtjatfdnqixatbprvzwqlfgu'), ['ozghvskaixje', 'vmdrvdjhwxdvajmkpcxigsjzmtuhdxgllhzrpqd', 'qfhnxpcmtzpociajidwlcvobjfyxfcugsxy', 'pgamvhedjibcghinjrnowqzkfzibmfmh', 'bcmrdzpcczhquy', 'czosacvwfsbdvwwyirpvbve', 'qu', 'fdkobwlnmxbpvjkapextlbcrny', 'bqutjqobkyobhtpevjvewyksnoqyjunnnmtocr', 'kjlgff', 'oitltmhdburybwfxrjtxdiry', 'kiokuquyllpeagxygqugfmtm', 'wlbkl', 'khubpmstqjzzjzmsvfmrbmknykszqvue', 'lqrbmyndsztyrkcgqxcsnsanqjigimaxce', 'nitnyonuzedorrtkxhhgedohqcojbvtvjx']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('wmvuoeqphsycrvtxghrcozortmdnh'), ['hv', 'ugcmpebvlzgdtcmgkbgzyfel', 'qvmofayljsvybupvvnbhhibsz', 'zvlihxmyxlxwbffwjzjrfjgimmltftqqre', 'mwassqvxptav', 'jrumvqzkiaewngoufhrleakcfrsaxhpxyg', 'sxlxwhvkpavgfhxrxcbnqbstyrejtosxwe', 'psnlqakyfhcupryqatrmwqlswwjylpaiqammx', 'ivozojwldsgtnxpvsi', 'epyzjs', 'legi', 'sdqxxahfbddhacqrglgdcmlslraxfaahhfyodon']) from system.numbers limit 10; -select 12 = multiSearchFirstIndex(materialize('lebwdwxfdzwquhqhbvmte'), ['mwhruilzxvlyrgxivavxbbsq', 'ubuiizuasp', 'xpkzcsf', 'qpeqitoqqqeivohajzhmjbo', 'kbftixqmqgonemmbfpazcvf', 'iyhluioqs', 'hws', 'tupfdksgc', 'ows', 'pngzkoedabstewcdtdc', 'zdmyczldeftgdlwedcjfcoqycjcivf', '', 'xt', 'syuojejhbblohzwvjzzedzgmwc']) from system.numbers limit 10; -select 7 = multiSearchFirstIndex(materialize('wcrqaoecjwkhnskrbahqxfqgf'), ['qegldkdmyaznlmlhzvxfgoukngzbatnuq', 'khgcvgrifwtc', 'hkwcpogbbdqulizrycmneqmqynvj', 'zkqjf', 'xfduxyy', 'ructdekcoywfxsvpumfefoglljptsuwd', 'wkhnskrbahq', 'crqaoecjwkh', 'ikmpbunpguleinptzfelysiqc', 'lhldcci', 'nooepfypkoxxbriztycqam', 'uxeroptbiqrjartlnxzhhnlvjp']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('psgkkcwttitgrjsobiofheyohadu'), ['achfrepey', 'minlzeiwgjfvvmhnevisky', 'oxfghfdthtyczzveppcoxrued', 'ydhaupodnezvxhcqahfkwtpvxnymriixf', 'slxsbxidylxyurq', 'socyyabwbjdabnuqswrtjtqogirctqsk', 'lvbnacirctyxxspjmispi', 'oj', 'ihmmuuqlosorrwhfxvpygfrzsqpmilcvjodmcz', 'idmtmemqfyrlbwhxz', 'hsqfsfdzvslwbtlwrfavez', 'gszl', 'ei', 'pnywjnezncpjtyazuudpaxulyv', 'iqgavdjfqmxufapuziwwzkdmovdprlhfpl', 'yigk', 'mjidozklrpedutllijluv', 'vixwko']) from system.numbers limit 10; -select 3 = multiSearchFirstIndex(materialize('xtjxvytsseiqrpkbspwipjns'), ['bwmoghrdbaeybrmsnucbd', 'zoslqabihtlcqatlczbf', 'sseiqrpkbspwipjn', 'mdnbzcvtayycqfbycwum', 'npueimpsprhfdfnbtyzcogqsb', 'ytsseiqrpkbspwipj', 'fzvhcobygkwqohwutfyauwocwid', 'naacyhhkirpqlywrrpforhkcjrjsnz', 'vezbzderculzpmsehxqrkoihfoziaxhghh', 'mvvdfqzskcyomjbaxjfrtmbduvm', 'pwipjns', 'tsseiqrpkbspwipjn', 'sseiqrpkbspwip', 'qgrtbcdqcbybzevizw', 'isjouwql', 'rlbeidykltcyopzsfstukduxabothywwbq']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('zxmeusmehplcgbqabjof'), ['hqxgrw', 'fydjyrr', 'cocwtbazwjrswygttvrna', 'wpkvowuq', 'mwnzdxihrxihzhqtl', 'ljkjtmrfbonhqkioyzotyeegrw', 'ofxo', 'rjubwtpbweratrelqlrqotl', 'wvxkcil', 'qvolxxgqs', 'afqlhjnlvxowtnuuzywxuob', 'slwbmq']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('tjcmtoisgbilkygushkpuxklis'), ['bkdohwx', 'dfohgzhcjqirlbrokwy', 'zaemgqgxltznvkccyumhgsftnfigbol', 'otgcaybejwe', 'qn', 'gvfzcyhvmsnbgkulsqrzeekmjkc', 'cajuyauvmhkrriehgwfmtqbkupysudle', 'pmcupysyllzpstolkfpdvieffxaupqtjty', 'elhlzvescbfpayngnnalzixxgunqdhx', 'cvxpgdnqcxeesk', 'etlewyipypeiiowuoewulkpalvcfe', 'ordhwrkwqq', 'wnroixlkrqnydblfrtlbywc', 'xshujuttvcdxzbetuvifiqi', 'meqqxqhntkvzwoptnwskdgsxsgjdawe', 'dnmicrfshqnzosxhnrftxxeifoqlnfdhheg']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('iepqqbvekaflprupsmnpoijrld'), ['kqomoeysekwcplpegdwcdoeh', 'mwdvr', 'aobviioktzwzmpilblbdwstndhimabfgct', 'vqustluciruiyfkoontehnwylnauwpol', 'utcqnitztcgr', 'ityszrqmlwzspnrwdcvdhtziob', 'hmll', 'ilfzvuxbkyppwejtp', 'euxdzqcqutnfeiivw', 'rbcjlmjniiznzaktsuawnfjzqjri', 'fzyxlzzretsshklrkwru', 'jrujmdevqqojloz']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('cufztqffwjhtlkysekklpaywemm'), ['cpawuauqodogaitybtvplknjrsb', 'ynsocxfnxshzwnhlrfilynvz', 'ylrpytgcvtiumdckm', 'mvgrkueaslpgnjvvhzairgldtl', 'iliorsjypskmxfuuplfagktoycywb', 'drvwngp', 'zviuhcxaspwmqqz', 'qfgmrmhycskus', 'szj', 'rooivliiqufztcqlhrqyqvp', 'tufdmsmwue', 'cssowtldgwksbzlqyfereodcpuedighwd', 'odcjdffchhabtaxjvnr', 'o']) from system.numbers limit 10; -select 7 = multiSearchFirstIndex(materialize('zqwvlarwmhhtjjgwrivwfpsjkvx'), ['zcwhagxehtswbdkey', 'okezglmrjoim', 'ilwdviqimijzgoopmxdswouh', 'aqztpsntwjqpluygrvwdyz', 'uzxhjuhiwpz', 'akgc', 'larwmhhtjjgwrivwfpsj', 'isqghxsmcrwlgyloslmlyeboywtttgejdyma', 'arwmhhtjjgwri', 'rwmhhtjj']) from system.numbers limit 10; -select 9 = multiSearchFirstIndex(materialize('fuddujwwcewlhthgwsrn'), ['shtzrrtukxmdovtixf', 'rkcnzzzojqvvysm', 'jlamctgphjqcxlvmpzyxtghnoaq', 'pthrwvbheydmrot', 'kpniaqbcrgtxdyxxdxonbbltbdo', 'igulngxgtauumhckvbdt', 'khgrmskijoxruzzzaigjxonsc', 'rxzeykfxwssltw', 'hthg', '']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('jtgvvkggpkqhbxptjgoy'), ['nplzawmacgtqfxsp', 'oosw', 'akw', 'hnsenqoqwiydiufozomkyirgjepeqw', 'fpafgahvfdxukzvskbuy', 'tqimmsqffiqfoni', 'rrxkjklmkdhxqwcpfyutqzxu', 'esfqeujcbqxwnvodkwwdbsyozptaf', 'rqnyguyz', 'fftl', 'ccfyavxtxrpi', 'wftpsblszgovfgf']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('steccxkwnptybaddcuau'), ['qagxfznhjaxtyclxdsi', 'rtxwptfyzgthkwrx', 'rmcoxxs', 'vlubx', 'siecygstzivz', 'tksiagm', 'kq', 'dgsqrobxegmdbjkanb', 'lxokyvhveklvdakrxyiqokr', 'tgpmehwdrirpfjonqzhqshbo', 'cqmkargvsfjoxrguymtzsfwkg', 'avkmufhoywprjw', 'xzywtvlpoozmgkrcavevwebv', 'hfiuwslapamiceaouznxm', 'tmfjhqddafhhjbybfphlbwu', 'mrigvhmjvdpny']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('ccbgxzoivbqtmyzqyooyepnmwufizz'), ['lcclseplkhxbrrzlnani', 'xggxivwqlpxmpypzovprdkmhrcgjkro', 'dbbmiegotfxjxybs', 'hqtcowpupsyqfx', 'znatfzjbeevbaqbmpofhywbyfxn', 'mnditiygex', 'lazqapwjswhkuimwmjoyseyucllnrfxrwnzj', 'jg', 'dmqwnuvsufgffuubhqeugwcanvflseorrydyyxvr', 'wpjfcfwfgjiybncrw', 'joucnvxxcyjyqlwhrzwnstyj', 'babtxkzasyaffxzd', 'wgcfdyhwxjoytbxffdxbdfinolbltnhqkvyzybc', 'yhrgwbdwopznltjtyidxawqg', 'bvrrt', 'bcwmsys', 'ijdjojhhzaiyjyai', 'eevxwppogogdbmqpbeqtembiqxeiwf']) from system.numbers limit 10; -select 2 = multiSearchFirstIndex(materialize('xrwjeznohtbdvijwsbdksf'), ['hwdfufmoemohatqafdrcvdk', 'tbdvijwsbdks', 'xzwjczbuteujfjifzkbxvezs', 'bdvijwsbd', 'eznohtbdvijwsbdks', 'xadezwhbbmlqz', 'b', 'socrdjxsibkb', 'dk', 'eznohtbdvijws', 'pavsosnncajr', 'jixlmxxmxnnbpebjhitvtsaiwzmtqq', 'yuxmmnrqz', 'mpzytweuycabvu', 'tbdvi', 'ip']) from system.numbers limit 10; - -select 0 = multiSearchFirstIndexUTF8(materialize('црвтгмсрооацволепкщкпнгшкамщ'), ['гйцбсханрейщжнфбхтщбйала', 'дирдфнжпнччхаоцшрийнйнечллтгцбфедгсш', 'жфйндбффаилбндмлточиирасдзйлжбдзег', 'жвоуйфсйойфцвгзшцитсчпкч', 'ршонтбгщжооилчхрзшгсдцпзчесххцп', 'пйучихссгнхщлутвменлмм', 'хишгешегдефесо', 'знупгж', 'щчфу', 'знвтжифбнщсибеноожжметачаохфхсжосдзйуп', 'ггтоцйпгхчсбохлрчлваисивжбшбохдурввагш', 'щлийбчштбсч']) from system.numbers limit 10; -select 5 = multiSearchFirstIndexUTF8(materialize('опднхссгртрхтотлпагхжипхпитраб'), ['шфршсцешушклудефцугщцмйщлошечедзг', 'нйумйхфщцгщклдожхвосочжжислцрц', 'згтпвзцбхйптцбагсвцгтнф', 'пшичси', 'ссгртрхтотлпа', 'апзазогвсбежзрйгщоитмдкн', 'непгайтзкгштглхифмзданоихц', 'пднхссгртрхтотлпагхжипхпитр', 'ждднфлрзалшптсбтущвошрйтхкцнегшхрсв', 'брп', 'сгртрхтотлпагхжипх', 'нхссгртрхтотлпагхжипхп', 'пагхж', 'мфкжм']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('овччцнтчайомсйййоуйуучщххиффсб'), ['жжрддцпнехйр', 'шзбвуооинпаххесйкпкошжмцзгхе', 'ррсннилщлщжгцтйрпхабкехахззнтщемагдйшпсч', 'пуфугнказепщ', 'гддхтплвд', 'сщсчи', 'бйрсахедщфкхиевкетнс', 'йфжцжшпхлййхачзхнфоц', 'цтмтжлщдщофисзрвтбо', 'кщсевбоуйб', 'щгаапзкн', 'осймщовшчозцййизм', 'фкмаат', 'бкзцсдонфгттнфтаглпрцтбхбсок', 'жлмичлйнйсжбгсейбсиезщдмутационжгмзп', 'нбищижнлпмтморлхцхвеибщщлкйкндлтпбд']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('фдситчщдвхмфйтхшдтуцтщжрочщо'), ['ейшфдннтйечгк', 'фуйщгбйшдцирзб', 'ехйцмчщрсртнк', 'увтцмдорщжфгцгзущпувтщкнрфсйбщрзй', 'хчщпхвуарнббпзсцшчщуносйгщпсбтх', 'жтдчрхфмхцххккзппзбнуббс', 'тчохнмбаваошернеймгготлузвсбрщезднеил', 'стссчкшрчррйбхдуефвеепщшзмербгц', 'жбезжпещ', 'вйтсрхптлкшвавдаакгохжцоощд', 'искеубочвчмдхе', 'щмлочпзбунщнхлрдлщтбеощчшчхцелшоп', 'екуийтсйукцн', 'дочахгжошвшйжцпчзвжйкис', 'лтеенешпсболгчиожпжобка', 'букзппщрчбпшвпопвйцач']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('гопвмрутфпфбхмидшлуб'), ['цнхшдойгщн', 'дкаежщрапщпщеа', 'фмогимдничрфтхмсцмчпдфтиофнтйц', 'фчмсщисхщуп', 'ощмвдчефозйжбеесбмещочевцчд', 'апкбцйщжщабвппофм', 'мтйоддлфцгдуммптднпщшрн', 'икхнсмжчбхнфхнссгл', 'ущмунинлбпрман', 'ллкнечрезп', 'ажтнвбиччджсзтйешйффдгдрувер', 'йрщ', 'чигдкйшфщжужзлвщулквдфщхубги', 'иккшсмаеодейнкмгхбдлоижххдан']) from system.numbers limit 10; -select 12 = multiSearchFirstIndexUTF8(materialize('срлцчуийдлрзтейоцгиз'), ['жщлнвбубжпф', 'оклвцедмиср', 'нлзхмчдзрззегщ', 'хоу', 'шайиуд', 'ерслщтзцфзвмйтжвфеблщдурстмйжо', 'жмгуйузнчгтт', 'стеглмрдмирйрумилвшнзззр', 'втедлчрчайвщнллнцдмурутш', 'цимхргмрвмщиогврнпиччубцйе', 'ктчтцбснзцйцймридвш', 'ейоц']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('лрицжленфилзсжпжйнцжжупупдфз'), ['чпбрмлрнцмвеуфу', 'рмпизмпжчшбхдудчшохтжш', 'гргцжчпгщищннусв', 'ийщтщвзчшпдзитщубакусхавслрсбткб', 'бйбакижцтибгбгхжцвйчжжщжсжкзф', 'чгрп', 'чуносжусжфчмфжхрщзлщрдвбашажаанча', 'чекршбш', 'лбцкхйсооцц', 'сгвнлегвфмпчтййлрмд', 'наатущркхйимхщщг', 'щпзоеимфощулбзхафпц', 'дцабцхлврк', 'умидмчуегтхпу', 'дщнаойрмчсуффиббдйопдииуефосжхнлржрйлз', 'щзжетезвндхптпфлк', 'бгчемкццдбжп', 'иихуеоцедгрсеужрииомкбззцнгфифоаневц']) from system.numbers limit 10; -select 3 = multiSearchFirstIndexUTF8(materialize('бхжвчашрощбмсбущлхевозожзуцгбе'), ['амидхмуеийхрнчйейтущлуегрртщрхвг', 'фнисцщггбщйа', 'хжвчашрощбмсбу', 'фщвщцнеспдддцчччекчвеещ', 'ущуджсшежчелмкдмщхашв', 'цкуфбиз', 'евозожз', 'ппт', 'лвцнелшхцш', 'ощбмсбущлхев', 'ефхсзишшвтмцжнвклцуо', 'цржсржмчвмфмнеещхмиркчмцойвйц', 'ашрощбмсбущлхевозожзу', 'гхщншфрщзтнтжкмлщанв', '', 'хевозо', 'ощбмсбущлхевозожзуц', 'возожзуц']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('мзчатйжщгтзлвефчшмлшт'), ['гхшфрунирйдзтеафщгк', 'ймхмфлц', 'звуумивмвштчтнтеобзщесакийгк', 'чщжетзнцишхрммтбцакиббчп', 'блмидикавущщдпгпчхйаатйанд', 'цмщшбклгцгмчредмущаофпткеф', 'бнетввйцзпдерхщ', 'ицйнцрввемсвтштчфрпжнатаихцклкц', 'дзлщсштофвздтмчвсефишс', 'пбзртдцвгкглцфесидлвваисщр', 'ммеилбзфнчищч', 'жш', 'лздиззтпемкх', 'байлужднфугмкшгвгулффмщзхомпав', 'рсзнббедсчзущафббзбйоелид', 'цфшйкцксйгуйо']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('жжмзмащйфжщлрффбпврзнидииейщ'), ['ржфзнлйщсздйткаоцруйцгцт', 'илинксщмгщшещееифвпданмйлж', 'кг', 'гпааймцщпмсочтеиффосицхйпруйшнццвс', 'кнзфгжйирблщлл', 'ищуушфчорзлкбцппидчннцвхщщжййнкфтлрдчм', 'тбтдчлвцилргоргжсфбоо', 'ехаех', 'нехщмдлйджждмрцпйкбрнщсифхфщ', 'тцжпснйофцжфивзфбхзузщтмдкцжплавозмше']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('биаризлрвххжкпщтккучфизуршткпн'), ['йбручвндбщвссаеха', 'ол', 'еузкмпогщзгзафшдшоплбфнфдккх', 'ибзихщйфбтаз', 'ибрчиейш', 'нафрпбснзрузнтмнйиомтечтшзбкпзутдилтф', 'тщтбапцчдий', 'щкнггмфцжрзщцзжвлкчбммхтхтуж', 'ваам', 'цкфиушзигбжтацнчдлжжзфшщммтнлж', 'туфовжтнкзщсщщизмрйкхкпц', 'пирзксзикфтшодожшчцг', 'жфчфцфвлйбмеглжйдазгптзщгж', 'тутириждкзчвтсоажп', 'мотзусбхту', 'слщкгхжщфщоцкцтрлгп', 'бругтбфесвсшцхнтулк', 'восур', 'ссежгнггщдтишхйнн', 'вгзосзгоукмтубахжнзгшн']) from system.numbers limit 10; -select 8 = multiSearchFirstIndexUTF8(materialize('мчслвбжвманджййсикнврцдчмш'), ['рлбмй', 'иб', 'жажлцсзхйфдцудппефвжфк', 'огггхзгтцфслхацбщ', 'дзтцкогаибевсйещпг', 'зпцтйзфмвгщшуоилл', 'етщзгцпдйчзмфнхпфцен', 'нджййсик', 'сикнврцдчмш', 'жййсикн', 'икнврцдч', 'паокаочввеулщв', '', '', 'кечзсшип', 'вбжвманджййсикнвр']) from system.numbers limit 10; -select 2 = multiSearchFirstIndexUTF8(materialize('нвррммппогдйншбшнехнвлхм'), ['нфошцншблеооту', 'лх', 'цртд', 'огдйншбшн', 'уулддйдщицчпшбоиоцшй', '', 'дрдужзжпцкслетгвп', 'й', 'мппогдйншбшнех', 'дйншб', 'лжвофчзвдд', 'рммппогдйншб', 'ехнв', 'втущсщзбчсжцмаанчлнасп']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('удехбкабиацхпгзнхжелшц'), ['фмнбтйезсфоахофофдблкжщжфмгхтзс', 'тщтамзафозхлз', 'цшжфсбл', 'йзгзилупшллвипучхавшнмщафзмнк', 'лу', 'гтебпднцчвмктщсзи', 'лпщлмцийгуеджекшд', 'пцдхфоецфрунзм', 'зис', 'хпж', 'цтцплхцжишфнплуеохн', 'впх', 'чцчдацлуецрчцжижфиквтйийкез', 'гчшмекотд', 'пйгкцчафеавзихзтххтсмкал', 'сжфхпцгдфицжслрдчлдхлсувчнрогнву']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('щщвфчгамтжашнуошбзшуйчтшх'), ['дийу', 'жеомлсжщймемрсччошдфажцтдп', 'нгопнцквбф', 'хопб', 'ив', 'чвфвшфрдфелрдбтатшвейтг', 'вхкцадмупдчбаушшлдксйв', 'жтжбсвмшшсйеуфдпбдлкквдиовж', 'гтсдолснхесйцкйкмищгсзедх', 'ошплп', 'ифпуррикбопйгиччи', 'чдфймудаибвфчжтзглс', 'зпцмвпнлтунвйж', 'еждрйитхччещлцч', 'вмофсужхгрнзехкх', 'щжгквкрфжмжжсефпахст']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('рфгигуужжцфмоаешщечувщгонт'), ['слащченщлуоцргврбаб', 'тцизут', 'лйрсцолзклжбчрзгббммоищщ', 'уицмлоилзф', 'зпхмшвфйккфщщп', 'ймижрпдщмшв', 'пуощжлрмжлщхмкйгщшщивдпчпжчл', 'ойахшафнж', 'гксомбвцрсбжепхкхжхнсббци', 'панлраптщмцмйфебцщемйахенг', 'сохлгожштлднчсзпгтифсйгфмфп', 'аждчвзну', 'дхшуфд', 'борзизцхнийбщгхепрнзшй', 'фщшздруггрке', 'оевупрйщктнолшбкунзжху']) from system.numbers limit 10; -select 8 = multiSearchFirstIndexUTF8(materialize('кщзпапйднучлктхжслмищ'), ['апмдйлсафхугшдезксш', 'кйрм', 'цйивайчшуалгащсхйш', 'злорнмхекг', 'сгщврурфопжнлхкбилдч', 'бнлпщшнвубддрлижпайм', 'нукдонццнрмовфнбгзщсшщшдичежффе', 'йднучлктхжс', 'зпапйднучлктхж', 'затйотдсмпбевлжаиутсуг']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('жцажссефррршнфмнупщаоафгкщваа'), ['жфпщкгзкрмтщчцтжйчпйдошбшоцд', 'бхгйлйдробптвущшппзуиидежнлтпбжащткцф', 'хлещазйцепдханпажчизнхгншйуазщхй', 'ашцк', 'фрбммхдднчзшс', 'нжцанилзжаречвучозрущцдщаон', 'длмчзцрмжщбневрхуонпйейм', 'шкбщттврлпреабпоиожнууупшмкере', 'вуцпщдиифпеоурчвибойбпкпбкйбшхдбхнаббж', 'нртжвкдйтнлншцанцпугтогщгчигзтоищпм', 'цкплнкщлкшемощмстздхпацефогтск', 'цвждйбсмпгацфн', 'шсжшрзрардтпщлгчфздумупд', 'цйииткглчжйвуейеиииинврщу', 'унлодтулшпймашоквббчйнибтвалалрвбцж', 'нбнфнвйишйжлзхкахчмнлшзуеенк', 'бшлпсщжквпцахигчдтибкййб', 'фчакпзовтрлкншзцулшщмпзж']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('иматеччдфлггшпучумджпфпзмвх'), ['дахахпчлцлаачгцгтфпнжлшчйуцбшсг', 'атжйувхец', 'грдсбвиднницдвшпйршгмегцаоопнжгй', 'чзлхречмктфащмтеечуиагоуб', 'савбхлпилийщтихутйчдгфсойй', 'вбгочбзистзщшденусцофит', 'мар', 'дфшажхдсри', 'тжлмщшж', 'птсрсщгшммв', 'ре', 'зратамкткфкинййй', 'гуцмсизулвазужфдмхнелфнжббдтрудчтнфцр', 'нйчинеучкхнпчгнйвчвсвлгминуцахгщввжц', 'ечагчнуулфббгбел', 'йшжуговрйкащцофдокфчушжктнптйеззушфо']) from system.numbers limit 10; -select 11 = multiSearchFirstIndexUTF8(materialize('азтммйтшхцхлгдрнтхфжбдрлцхщ'), ['нпучщфвспндщшспзмшочгсщжчйгжбжзжжтн', 'хккдйшабисдузфртнллщпбоуооврайцз', 'йпхрфжждгпнйаспйппвхбргшйвжччт', 'ффеее', 'кежцновв', 'еххрчштарзмкпйззсйлмплхбчбулзибвчбщ', 'шфжйдотрщттфхобббг', 'ожоцжущопгоцимсфчйщцддзнфи', 'цуимеимймкфччц', 'прммщмтбт', 'хцхлгдрнтхфж', 'лгд', 'цжбдаичхпщзцасбиршшикджцунйохдлхй', 'пидхцмхйнспйокнттмййвчщпхап', 'йтйзмеаизкшйошзвфучйирг', 'хцхлгдр']) from system.numbers limit 10; - -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('gyhTlBTDPlwbsznFtODVUzGJtq'), ['seSqNDSccPGLUJjb', 'xHvtZaHNEwtPVTRHuTPZDFERaTsDoSdX', 'QCeZOYqoYDU', 'bsybOMriWGxpwvJhbPfYR', 'FFHhlxfSLzMYwLPPz', 'tvDAJjaLNCCsLPbN', 'kOykGaSibakfHcr', 'mWAZaefkrIuYafkCDegF', 'ILrFDapnEDGCZWEQxSDHjWnjJmeMJlcMXh', 'zHvaaTgspUDUx', 'tss', 'laUe', 'euUKFLSUqGCjgj', 'Kd', 'MxyBG', 'qRXMsQbNsmFKbYSfEKieYGOxfVvSOuQZw', 'PdBrNIsprvTHfTuLgObTt', 'kMekbxI']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('ZxTznPEbfoBfLElYOrRiHrDLMmTpIh'), ['bJhYwKLeeLvLmXwWvQHWFkDQp', 'dLyZmUicTZmUfjfsFjxxgOiMJn', 'UCYbbGcY', 'kpPiwfWHEuh', 'jviwmHeiTQGxlTKGVEnse', 'cVnEyLFjKXiLebXjjVxvVeNzPPhizhAWnfCFr', 'gkcoAlFFA', 'ahZFvTJLErKpnnqesNYueUzI', 'VIJXPlFhp', 'rxWeMpmRFMZYwHnUP', 'iFwXBONeEUkQTxczRgm', 'ZnbOGKnoWh', 'SokGzZpkdaMe', 'EfKstISJNTmwrJAsxJoAqAzmZgGCzVRoC', 'HTmHWsY', 'CpRDbhLIroWakVkTQujcAJgrHHxc']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('VELfidaBvVtAghxjkrdZnG'), ['fvEFyRHvixuAYbuXygKeD', 'zFNHINreSOFksEGssBI', 'hcdWEcKDGWvfu', 'KczaFjvN', 'nZLTZAYSbfqcNWzWuGatDPUBYaRzuMBO', 'UdOdfdyPWPlUVeBzLRPMnqKLSuHvHgKX', 'DgVLuvxPhqRdSHVRSeoJwWeJQKQnKqFM', 'NNfgQylawNsoRJNpmFJVjAtoYy', 'tWFyALHEAyladtnPaTsmFJQfafkFjL', 'lYIXNiApypgtQuziDNKYfjwAqT', 'QjbTezRorweORubheFFrj', 'htIjVIFzLlMJDsPnBPF', 'ltDTemMQEgITf', 'fprmapUHaSQNLkRLWAfhOZNy', 'dOJMvPoNCUjEk', 'm', 'vEEXwfF', 'aVIsuUeKGAcmBcxOHubKuk']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('kOzLaInSCOFHikwfkXaBfkyjdQ'), ['t', 'emHGfAiZSkZaVTSfplxRiPoDZUTT', 'YHnGJDTzxsboDsLPGHChMHwrHHICBIs', 'gbcbVHSlVeVDOeILWtSLkKfVVjG', 'fPaJjbnNthEwWZyg', 'qS', 'PCQxoLaSdQOjioMKPglmoWR', 'KLMNszm', 'TCErEFyxOvqnHs', 'dRbGzEJqvIGAcilZoHlXtZpjmLLZfsYueKqo', 'iKHmNSbGgaJYJEdMkbobXTdlFgAGEJMQ', 'mUGB']) from system.numbers limit 10; -select 1 = multiSearchFirstIndexCaseInsensitive(materialize('JGcICnWOGwFmJzHjtGJM'), ['fmJzHj', 'LhGTreYju', 'yCELHyNLiAJENFOLKOeuvEPxDPUQj', 'kWqx', 'OBnNMuaeQWmZqjWvQI', 'ektduDXTNNeelv', 'J', 'iCNwoGwfMJzhjtGJ', 'uiIipgCRWeKm', 'bNIWEfWyZlLd']) from system.numbers limit 10; -select 7 = multiSearchFirstIndexCaseInsensitive(materialize('fsoSePRpplvNyBVQYjRFHHIh'), ['ZqGBzyQJYuhTupkOLLqgXdtIkhZx', 'pouH', 'mzCauXdgBdEpuzzFkfJ', 'uOrjMmsHkPpGAhjJwVOFw', 'KbKrrCJrTtiuu', 'jxbLtHIrwYXDERFHfMzVJxgUAofwUrB', 'PLvNyBVQYjRfhhi', 'wTPkeRGqqYiIxwExFu', 'PplvNybvqyJ', 'qOWuzwzvWrvzamVTPUZPMmZkIESq', 'ZDGM', 'nLyiGwqGIcr', 'GdaWtNcVvIYClQBiomWUrBNNKWV', 'QQxsPMoliytEtQ', 'TVarlkYnCsDWm', 'BvqYJr', 'YJr', 'sePrPPLVNYbvqYJRFhh', 'ybvq', 'VQYjrFHh']) from system.numbers limit 10; -select 3 = multiSearchFirstIndexCaseInsensitive(materialize('aliAsDgMSDPISdriLduBFnuWaaRej'), ['gWOFTxMrQGQaLrpJamvRhgeHwk', 'iWsBLzLycWvbJXBNlBazmJqxNlaPX', 'Ri', 'FPLRURSsjvsySncekcxaWQFGKn', 'wgXSTVzddtSGJQWxucYorRjnQQlJcd', 'wOLJWZcjHEatZWYfIwGIqnuzdcHKSFqfARfNLky', 'eEECZMNmWcoEnVeSrDNJxcOKDz', 'duBF', 'EhfLOjeEOQ', 'dUbFNUWA']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('EUzxPFYxMsJaTDzAKRXgZIVSFXU'), ['TDKAgICICjzBKHRqgFAuPCSODemldGGd', 'LvMluSJTIlgL', 'srbRhQKjPIchsipVHsjxwhK', 'vdurVsYkUWiFQVaDOnoNIJEX', 'UzZsZqAUNjMvWJaTqSWMHpzlDhVOaLzHPZfV', 'XcnnPXXEJJv', 'JSwFBNnYzNbIRZdeMfYiAfxzWfnCQFqoTUjns', 'HBMeqdLkrhebQeYfPzfJKAZgtuWHl', 'cMfSOnWgJvGhFPjgZdMBncnqdX', 'orDafpQXkrADEikyLVTHYmbVxtD', 'Vz', 'bfYwQkUC', 'q', 'YqomKpmYpHGv']) from system.numbers limit 10; -select 4 = multiSearchFirstIndexCaseInsensitive(materialize('mDFzyOuNsuOCSzyjWXxePRRIAHi'), ['TfejIlXcxqqoVmNHsOocEogH', 'clyblaTFmyY', 'JQfxMAWVnQDucIQ', 'jw', 'fGetlRA', 'uWwCOCd', 'rInhyxSIFiogdCCdTPqJNrqVaKIPWvLFI', 'mimSJjfCWI', 'jqnJvNZXMEPorpIxpWkhCoiGzlcfqRGyWxQL', 'bxCJeVlWhqGHoakarZcK', 'unsUOcSZyjwxxe', 'E', 'PR', 'nsUoCSZyjwxXEPr', 'sfotzRPMmalUSjHkZDDOzjens', 'zYJwxx', 'DFzyouNsUocsZ', 'QBaQfeznthSEMIPFwuvtolRzrXjjhpUY', 'sQPVBaoeYlUyZRHtapfGM', 'lPiZLi']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('VOAJfSkbDvNWAZNLIwqUgvBOddX'), ['pHrGGgJ', 'VohjtPdQZSNeKAlChDCnRTelroghFbZXVpnD', 'rnWebvdsmiFypMKL', 'NtKRiJOfAkWyKvubXrkOODgmZxvfOohsnHJEO', 'nxsDisKarasSZwESIInCJnYREUcoRUTXHBUH', 'mXYYr', 'jujScxeTBWujKhKyAswXPRszFcOKMSbk', 'INEegRWNgEoxqwNaGZV', 'VVyjMXVWVyuaOwiVnEsYN', 'mkLXSmXppxJhFsmH', 'pRVnBrWjqPeUDHvhVuDbzUgy', 'PzchFdPTkOCIVhCKml', 'KXaGWnzqoHBd', 'PhzQVqIOLleqDSYNHLjAceHLKYPhCVq', 'aixxTqAtOAOylYGSYwtMkZbrKGnQLVxnq', 'ruEiaxeRaOOXGggRSPlUOGWSjxh', 'prSULtHvDMw', 'vEpaIIDbGvIePYIHHZVNSPYJl']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('ZHcEinZEFtfmHBLuCHntUhbIgY'), ['GKElMPEtmkLl', 'mkrzzjSRfXThuCQHkbZxRbhcymzTxcn', 'PREwQjxBJkpkiyuYEvtMZNFELgbINWsgf', 'lFEGlPtaDJSyoXzwREiRfpzNpsaBYo', 'tmVTuLPhqhgnFNhHvqpmc', 'NtijVhVfAwpRsvkUTkhwxcHJ', 'O', 'FSweqlUXdDcrlT', 'uljEFtKVjIzAEUBUeKZXzCWmG', 'dBIsjfm', 'CNaZCAQdKGiRUDOGMtUvFigloLEUr', 'yWjizKZ', 'QqPVdyIFXcweHz', 'uPmgGWGjhzt']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('AYMpbVsUQqAfoaMiJcYsulujYoSIx'), ['aXECumHNmAEefHPJy', 'hTosrERBdVCIilCYcMdHwaRh', 'PVDBpwrc', 'uFvQRPePvmzmocOauvEjqoxMhytzOwPSOCjmtm', 'kQqIlSCHDmWXCKN', 'ybAHGYDEDvvOJsF', 'WpkANi', 'cFGuzEcdahZtTdLFNBrRW', 'EBaybUFxO', 'mRlZUzHzMsMAgvtRtATEDLQvXZnZHw', 'uqxckjqpCBHiLgSPRz', 'Lv', 'AJcRfAvBmQVMOjaFfMfNHJt', 'FYsPM', 'pkKXTPgijOHFclqgVq', 'Ck']) from system.numbers limit 10; -select 11 = multiSearchFirstIndexCaseInsensitive(materialize('gmKSXWkNhKckrVNgvwiP'), ['bdJMecfCwQlrsgxkqA', 'NTgcYkMNDnTiQj', 'fmRZvPRkvNFnamMxyseerPoNBa', 'rfcRLxKJIVkLaRiUSTqnKYUrH', 'YSUWAyEvbUHc', 'PridoKqGiaCKp', 'quwOidiRRFT', 'yHmxxUyeVwXKnuAofwYD', 'gichY', 'QlNKUQpsQPxAg', 'knhkCKRVNGvWIp', 'jAuJorWkuxaGcEvpkXpqetHnWToeEp', 'KnHKCKrvNgVW', 'tCvFhhhzqegmltWKea', 'luZUmrtKmmgasVXS', 'mageZacuFgxBOkBfHsfJVBeAFx', 'hKC', 'hkRCMCgJScJusY', 'MKSXWknHkckrVNgv', 'osbRPcYXDxgYjSodlMgV']) from system.numbers limit 10; -select 15 = multiSearchFirstIndexCaseInsensitive(materialize('lcXsRFUrGxroGIcpdeSJGiSseJldX'), ['pBYVjxNcQiyAFfzBvHYHhheAHZpeLcieaTu', 'SQSQp', 'OQePajOcTpkOhSKmoIKCAcUDRGsQFln', 'AYMDhpMbxWpBXytgWYXjq', 'gkUC', 'oWcNKfmSTwoWNxrfXjyMpst', 'fQSqkjRNiBGSfceVgJsxgZLSnUu', 'LRrhUjQstxBlmPWLGFMwbLCaBEkWdNJ', 'cZnaActZVoCZhffIMlkMbvbT', 'Uxg', 'vlKdriGMajSlGdmrwoAEBrdI', 'Fl', 'XzcNdlUJShjddbUQiRtR', 'AqowAuWqVQMppR', 'SRFUrGXrOgiCP', 'k']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('KhwhbOzWvobUwJcteCHguFCn'), ['LkDYrpvDfPL', 'CIaTaShobVIaWjdbsNsCMdZKlGdtWuJmn', 'zYcsxxFyfuGrPdTPgEvGbXoYy', 'vDIeYpJbLMGMuRkIrPkAnqDDkqXPzy', 'Ievib', 'CREiuEsErFgvGEkQzThHtYtPmcL', 'JjRWKyALtSkoGmRxh', 'JxPhpijkDOpncCKyDEyXvKNua', 'jo', 'mKpFscuBEABMAlQO', 'qiFTgJpcnUMRKzTEuKY', 'pXBtITxCPRaXijM', 'guYVLpIbu', 'tSKYIxv', 'oDnWaFAmsXGRdGvRPhbCIvFSFQNlSVYB', 'phdckINUiYL']) from system.numbers limit 10; -select 14 = multiSearchFirstIndexCaseInsensitive(materialize('pXFoUGwVTAItBqgbBaQwAqmeh'), ['LfBevBpGnaSlmGhbeZ', 'NtBYzEksiXvYI', 'jMeRw', 'omtaduY', 'BsWyvNdkfXsTBxf', 'CtoOIvaesuca', 'pgJcRIBVbyaPBgGsNKP', 'bAwdUMnwKvMXfFHQWrtfMeqcORIJH', 'GDxZblrqWSxUJFjEuXArPtfHPdwSNGGL', 'LLxcfp', 'NrLghkFpwCdvHJBfPBgiMatNRaDKjO', 'XCzr', 'cCojPpfLkGZnaWBGpaZvrGMwgHNF', 'BaQWAQmE', 'AQ', 'RtxxEZDfcEZAgURg']) from system.numbers limit 10; -select 5 = multiSearchFirstIndexCaseInsensitive(materialize('KoLaGGWMRbPbKNChdKPGuNCDKZtWRX'), ['FBmf', 'QJxevrlVWhTDAJetlGoEBZWYz', 'tKoWKKXBOATZukMuBEaYYBPHuyncskOZYD', 'kgjgTpaHXji', '', 'xOJWVRvQoAYNVSN', 'YApQjWJCFuusXpTLfmLPinKNEuqfYAz', 'GXGfZJxhHcChCaoLwNNocnCjtIuw', 'ZLBHIwyivzQDbGsmVNBFDpVaWkIDRqsl', 'Kp', 'EyrNtIFdsoUWqLcVOpuqJBdMQ', 'AggwmRBpbknCHdKPgun', 'xNlnPtyQsdqH', 'hDk']) from system.numbers limit 10; -select 6 = multiSearchFirstIndexCaseInsensitive(materialize('OlyNppgrtlubvhpJfxeWsRHpr'), ['slbiGvzIFnqPgKZbzuh', 'fakuDHZWkYbXycUwNWC', 'HnVViUypZxAsLJocdwFFPgTDIkI', 'bLx', 'fmXVYOINsdIMmTJAQYWbBAuX', 'pjFXews', 'BG', 'vrSQLb', 'ub', 'pREPyIjRhXGKZovTqlDyYIuoYHewBH', 'hnNQpJmOKnGMlVbkSOyJxoQMdbGhTAsQU', 'UwaNyOQuYpkE', 'yHNlFVnuOLUxqHyzAtNgNohLT', 'YJRazuUZkP', 'z', 'lUbVhpjFxEWsRhP']) from system.numbers limit 10; -select 6 = multiSearchFirstIndexCaseInsensitive(materialize('ryHzepjmzFdLkCcYqoFCgnJh'), ['cLwBRJmuspkoOgKwtLXLbKFsj', 'YSgEdzTdYTZAEtaoJpjyfwymbERCVvveR', 'RzdDRzKjPXQzberVJRry', 'HUitVdjGjxYwIaLozmnKcCpFOjotfpAy', 'LWqtEkIiSvufymDiYjwt', 'FDlKCCYqoFCGNj', 'jmZfdlKCcyQOFcGnJ', 'OZCPsxgxYHdhqlnPnfRVGOJRL', 'JfhoyhbUhmDrKtYjZDCDFDcdNs', 'KCCYqo', 'EPJMzFDLKcCYQ', 'zLQb', 'qsqFDGqVnDX', 'MzfdLkCCyQOFc']) from system.numbers limit 10; -select 5 = multiSearchFirstIndexCaseInsensitive(materialize('oQLuuhKsqjdTaZmMiThIJrtwSrFv'), ['MsfVCGMIlgwomkNhkKn', 'fBzcso', 'meOeEdkEbFjgyAaeQeuqZXFFXqIxBkLbYiPk', 'tNV', 'i', 'EwuTkQnYCWktMAIdZEeJkgl', '', 'hUo', 'dtAzmMITHijRtwsrFV', 'vhnipYCl', 'puor', 'TazMmiTh', 'ITHIJRTWSrf', 'luuHksqJDTaz', 'uHkSQjDtazMMiThIjrtwSRFV', 'gpWugfu', 'QjdtazmmIthIjRTWSRFV', 'ZdJpc']) from system.numbers limit 10; - -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ИпрУщйжббКВНИчйацпцоЛП'), ['цШСкЕвеГЕЗЦщруИБтЦсБГАу', 'Хнщта', 'БшА', 'СалШйР', 'ЩфДГРРчшБДММГЧоноЖСчдпВХшшгйН', 'бЕжПШЦддожнЧоЕишчшЕЙфСщиВПФМ', 'ТЗзГФх', 'Чфл', 'КнНкнЖЕкППварНрхдгЙкДешмСКИЛкеО', 'ЖИсЧПСФФМДиТШХЦфмЗУпфрУщСЛщсфмвШ', 'ллЙумпхчОсЦМщУ', 'ГМУНЦФшНУбРжоПвШШщлВФАтоРфИ', 'БХцжеНЗкжЗЗшЦзфгдЖОзЗЖщКМИШАтЦАп', 'мтСкЕнбХШнЛхХГР']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('цмйвГЖруДлдЦавхЖАлоЕхЗКд'), ['ХфБПМДВХЙ', 'саЗваАбднХбЦттмКсМбШбВМУйНКСЖжХЦНц', 'плиЩщШАцЖсхГ', 'ЗнУЕФЗВаНА', 'ЧДйСаЗГЕшойСжбсуЩуЩщбПР', 'ЧЕуЩкФБВвчмабШЦтЖбОрЗп', 'йХбМсрТАФм', 'РЖСЗвЦлНВПЧщГУцЖ', 'ГГлЩрОХКнШРТуДФ', 'шСабРжла', 'ЕчБвгаРЧифаЙщХПпГЦхчШ', 'дайшйцВНЩЧуцйдМХг', 'УнзНКЧххВрцЩМлАнЖСДОДцбИгЛЛР', 'сЛЗзПбиАгзК']) from system.numbers limit 10; -select 2 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('дфЧлзОжММФкЖгиЗЩлоШжФТкцк'), ['ЗРТцИрсФСбПрщГЗ', '', 'ЖГИЗщлОш', 'АДПН', '', 'чЛЗОЖмМфКжг', 'Мфкж', 'ндаовк', 'зГЛРГАНШмСмШМефазшеБкзДвЕШиЖСЗЧПИфо', 'ФЧЛзОЖммфКжгиЗЩ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ИИКДМЛхРчнвЙЕкВЧелТйЛВТ'), ['АчшОЛтНЙуЦЛЙфАКУйуТЗМеЗщОХТМЗеТА', 'НЦУДбчфРТОпЛкОгВпоО', 'неДавнНРеАУфТтфАнДчтнУМЛПШнроАчжш', 'бГржВПЧлЛтСВТтаМЦШШ', 'БщГщРнБхЕЛоЛсмЙцВЕГ', 'цбАжЦРеу', 'ХсЦРаНиН', 'нббДдВЗРС', 'змОПпеЛЖзушлнДЛфчЗлцЙЛфЖрЛКг', 'фШиЖСУоаНПйИВшшаоуЙУА', 'ЛктХиШРП', 'МапщВйцХч', 'жмУТкуГбУ', 'сйпзДЩоНдШЕТбПзФтсрмАФГСз', 'ЛБУвйладЕижрКзШУАгНЩчЕмАа', 'мЧпФлМчРбШРблмтмПМоС']) from system.numbers limit 10; -select 8 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ПоДУЗАтХншЦатИшХвмИЖчГнжчНцух'), ['жЛЧХзкжлиЛцЩбЧСнЛУжЖпКРвиСРН', 'шадмЩеУШБврУдЕБЗИгмЗЕФШчЦБСзПидтАлб', 'йпГмШСз', 'хЖФЙиПГЗЩавиЗЩйПнБЗЦЩмАЧ', 'ХесщтлбСИуЦ', 'вар', 'ЙкМаСхаЩаЗнФЩфКжПщб', 'ОдУзАТХншЦатИШхвМиЖчгнЖч', 'ЗВЗДБпФфцвжУКвНсбухссбЙКЙйккЛиим', 'гХхсГЛшдфЖЛбгчоЕмоЧр']) from system.numbers limit 10; -select 7 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ихзКЖЩсЧРСЖсЖжЛАшкТхИйТгМБпск'), ['ДРОБм', 'нз', 'тОЛ', 'щРзуЖрТ', 'Мдд', 'АЦГРК', 'Чрсжсжжл', 'чРсжсЖжл', 'ктхИйтГмБ', 'аАзЙддМДЦЩФкРТЧзЧПУойоТхБиЧПлХДв', 'иЙтгМбп', 'РицлПн', 'йДГнЧкЕв', 'ВМЩцАш', 'хКЩнДшуБЕЛТФГВгцБПРихШЙХгГД', 'иЙТГМ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('жггкщцзщшамдбРЗжйТзвхшАпХСбе'), ['лВТвтлРБжиЛЦвРЦкАЦаНБгуОН', 'рШаавцжзМрзВЧДРСузб', 'оемрЗМгФБНмжп', 'ЛбмХбФЧШГЛХИуТСрфхп', 'ЖшТдтЧйчМР', 'ЧнИМбфУпмЙлШЗТрТИкКИЩОЧеМщПЩлдБ', 'ГвРдПжГдБаснилз', 'уТнТчТРЗИЛ', 'ИТЕВ', 'дИСЖпПнПСНОвсЩЩшНтХЧшВ', 'штабтлМнсчРЗтфсТЩублЕЧйцеЦТтХ', 'ХбхгУШвАзкшЖ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('нсЩЙЕМмЧЛСйФцГВМиатГХш'), ['КсОПЧИкВсКшРхнкхБжду', 'мШмпТащжФ', 'ББЖнианЧЦпмрГЩГМаЛКжА', 'арИжзжфГТУДИРРРбцил', 'дфдмшМИщТиЗПруКфОнСЦ', 'Рцч', 'гмДгВДАтсщКЗлхвжЦУеФДАТГЙЦЧОЗвРш', 'чфХЩсДбУбВжАМшРлКРщв', 'нцБйсУ', 'фасДЕчвчДмбтЖХвоД', 'аБЧшЖшЖАКргОИшпШЧзТбтфйвкЕц', 'ЗжжсмкжЛд', 'щщлПзг', 'бП']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('сКиурчоиаЦйхгаУДПфчИтИК'), ['МЧПцУАМрХКЧмАЦннУшмРчкЖКХвху', 'РвДуВиашрРКкмжшЖНШБфлцжБЦР', 'йМУиУчНЧчРшДйБЗфЩЦйПсцгкДС', 'НсмаЛзЧвНЦШФуВРпзБГзйКцп', 'ЖлМЛУХОБллСЗСКвМКМдГчЩ', 'ЩХПШиобЛх', 'аФАЖВтРиЦнжбкСожУЖЙипм', 'аУГжУНуМУВФлж', 'ШБчтЗкЖНЙк', 'ЩоГПГчНП', 'мВЗйЛаХПоЕМХиИйДлшРгзугЙЖлнМппКЦ', 'вчмДФхНеЦйЗсЗйкфпОщПтШпспИМдГйВМх', 'ИЗИжЧжаГЩСуцСЩдкскздмЖЦ', 'дАмфЕбгс', 'ГМттнхчЩжМЧДфщШБкфчтЧ', 'ШЕииФБпщЙИДцРиЖжЩл', 'ОпуОлБ', 'хБ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('риШМбгиПЖннНоЧргзГзеДпЛиНт'), ['икДкбйдройВУсвФзрПСусДнАШо', 'чуУеТкУВФхз', 'ЕГпйчехЗвЛлБблЧПДм', 'зеоЩЧожКЛбШЩдАрКБНйшКВШаЗгПш', 'виФКуЗОтгВмТкБ', 'цДрЙгЗРаЧКаМДдБЕЧзСРщВФзПВЧГвЩрАУшс', 'мБЗИУдчХХжТж', 'ФТНМмгЖилуЛйМ', 'ЗегЩЦнЦщцИк', 'оГОусхФсДЖДЩИЕХЗпсПЩХБТГЕп', 'АУКНзАДНкусВЧХвАж', 'КвКрбсВлНАоЗсфХОйЦхТ', 'вФдеХацЧБкрхМЖЗЧчКшпфВчс', 'йХшиОвХЗжТпДТбвУрпшЕ']) from system.numbers limit 10; -select 11 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('МойрЙлтЖйБдББЛЕЕЦузЛфпИЕГт'), ['ПОжЦЩа', 'СШзЧФтСЗохЦЗдФтцНТу', 'вЕдТ', 'ечУФаМДнХщЕНУи', 'вмеосТзБАБуроЙУЛгФжДсЧщтчЕзлепгк', 'ИЧтБрцПмРаВрйИвНЛСйпЖжУВдНрурКшоКХП', 'ЕН', 'щКЦЩгФБСХпкпит', 'ей', 'ЕахшеОМРдЕГХуГЖчвКХМЕ', 'Гт', 'НужЛЛЙОАл']) from system.numbers limit 10; -select 11 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('еззЦАвУаДнзИКЙнЙдртРоП'), ['КгЩбшПЛКвтИРцйчккгЧчЧмтГ', 'кЛппСФщзМмТйВЕтбЩЦлО', 'ШпдзиЖх', 'иИХ', 'пУаАФгсмтофНФХиЦЕтТЗсОШЗЙ', 'фаКАБТцФМиКЖрИКшГБЗБ', 'идЖЙдЦММУнХЦЦфсФМ', 'МиЦечЖЦЙмРВЙОХсБРНнрлйЙшц', 'ТфдСтМгтмимТМАучтхПНЦлуф', 'бейККЛСггУЦБсокЕЙпнРЧ', 'цавУАДНЗИКЙнЙд', 'ЩйЕЖчЧщаПшжФсхХЛЕТчвмЙнуце', 'РТРОП', 'цАВуАДнзИкЙНЙдРтРо', 'аЩПИд', 'ОСчКшОАчВмр', '', 'уЙЛИуЕУвцДшНОгбТбИШв', 'АВУаднзИКЙНйдР', 'жТйоП']) from system.numbers limit 10; -select 12 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('шйМЦУщвфщшбмлТНВохСЖНУ'), ['хшТАпТоШхКНсДпвДЕчДМНбАНччд', 'ХКуПСтфСйРжмБглОШЙлйДкСФВйВ', 'хпмНЦМУШеАД', 'чзмЧВвлбЧкАщПкзТгеуГущб', 'шзжрДд', 'еЗГОЙНйИБЗДщИИНицмсЙЗгФУл', 'кнщЙхооДТООе', 'всзЙнТшжФЗДБДрщВДлбвулДИаз', 'мп', 'уБОйцзнМпИсксхефбдЕЛйгИмГШГЗЩ', 'ОМпзШШщчФФнвУЧгжчиндЧч', 'щВФЩШбмЛТн', 'бм', 'БпфнкнйЗцПдЧЩбВ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('НЗБлОбшмОПктткоччКиКрФсбкШАХ'), ['нффЕББУЖГшЖвГфЦФГЕСщсЩЧлфнАшшктизУ', 'нСмпцхшИои', 'ЧИчЗУтйЦхГезппФРХХШуцЗШВ', 'РИнщН', 'НЩдВТсЙсОдхРбМФнСпАбОПкудБФСчмб', 'йхглпдКтртгош', 'ибгУРАБцх', 'ИЕиЛрИДафмЗИкТвАуГчШугбЧмЛШщсОЧбБкП', 'ЩСМуХМ', 'АУсмдЗБвКфЩ', 'пгбТНОйц', 'МоИ', 'КОйкзОЕИЗМЩ', 'чщттЛРНнГхЗхХй', 'ЩшцЧРКмШЖЩЦемтЧУЛГкХтВНзОжУХТпН', 'ЕшбБНчрДпЩЧМлераУЖХйфйдчтсчПШ', 'дбФйтИАшДйЩтбФйШуПиРлГмВОШаСлШЧИвфЖщгж', 'ОДжТЦщпщИжфуеЩмн', 'ПЛНЕзжСчВКДттуФРУ', 'БбмеГЩХшжрцОжХНииВКВлдиХБДСмнНфХЛТХ']) from system.numbers limit 10; -select 4 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ЕКаЖСЗЗЕЗгПдШкфцЙТцл'), ['ЙКМИХРОХ', 'НвМУХзфчДбАРЙДу', 'чмщжФшшжсЗТв', 'жСЗзеЗг', 'ЛФсКзВСдЦД', 'АЖсЗЗЕЗГ', 'Пдшкфц', 'усйсКщшрДрвнФЛедуГХ', '', 'цйтЦ', 'Ощс', 'ЕЗГпдшКф', 'ззеЗгп', 'УгЛйхШТтшрЛ', 'ЗзЕЗгП', 'КЛмТЩРтрзБбЩРгФбиОБазУнтУЦ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('чЕжАфАрБпКбДмшАшТШККауЩИхНВО'), ['ЧЙпЗЧЧлйПЙЖЙшККг', 'зйхуМЩАИПГЗА', 'ЙцехноХниИбзБЧ', 'чВомЗОфУроС', 'дбРхХЗрзоДДШщЕДжиФаЙ', 'еЛзТцЩДиДГрдМОНЧУнеТуДЩЧЦпГЕщПОРсйпЧ', 'ФчнпМРЧцПЙЩЩвфДХПнУхцЩСИ', 'цлШеУкМБнжЧлУцСуСЙуотшМфйс', 'лугГлкщКщкзЛйпбдсишргДДшОувр', 'ЗРИаФЛЗФрСзм', 'аЗвжВгхЩоЦ', 'чГКлеБНДнИЖЧеШЧДнИвсГДЖЖфБМНсУЦосВс', 'щЦнПУзЧщнЩЕ', 'рВУв']) from system.numbers limit 10; -select 20 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('анктгЦВВкЧвЖиБпфТйлр'), ['НшДПчтсСЧпкидаХжаЙчаДчЦГшГ', 'ХнцЛШИрХВаРхнЧИЙрОЛЛИТпППфгЖЩФ', 'ФАЛущПупмдМБмтйзУшрВМзцзШжгД', 'ГчЛЧеЛДХеипдшЦЦмаШНаРшУТ', 'фОЕфжО', 'ТНсУАнчшУЛЦкцчЙ', 'ЛйЦКБЗГЦйКЩиОПуТЦкБкБувснЙи', 'Бунф', 'ИтХЛШСУНЦВйРСЙчДчНвйшЗЦй', 'АцСКнзБаЖУДЖегавйБгужШАДЙтжИВк', 'ЦцХщфирДПрСуХзхЖМЕщ', 'кфдБЖКншвУФкЗДКуЙ', 'СкиСЦЗЦРмгЦНпБхфХДЙщЛзХ', 'йУепВЖАПНбАЩуЛжвЧпхМ', 'БпЧшпДочУвибщерйхйтОБАСПнЧМИОЩ', 'чФгНЗщвхавбшсООоВштбЧ', 'уДиЕцнЙХВЕйИАГдЕ', 'тп', 'ЧЕРЖсгВ', 'вЖибПФТЙЛ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ипозйпхЛОЛТлСМХЩдМвМгШИвГиЛп'), ['ФСГзиГррБДНКГЛХбААФхИ', 'гегпАвхДЕ', 'ЦХжзщХИвхп', 'ЗЖ', 'ХОКцКзЩо', 'абИОрГПМТКшБ', 'кмХТмФихСЦсшУдхВбИШМНАНмпмХОЗйПЩч', 'еОжТСкфЕТУУжГ', 'НтщМЕПЧИКЙКйй', 'ежСикИвйЛж', 'ушЩФОтпБзЩЛЗЦЧЙиВгБЧоПХНгОуАДТЙж', 'фМЕРефнутпнцФРнрГЖ', 'хшДЧзнХпфорвЩжмГРЦуХГ', 'ЧЖн', 'вВзгОСхгНумм', 'ЗДоВлСжпфщСКсщХаолЛнЛЗбСхвЩвЩНоЩЩМ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('МрЗтВФуЖРеЕШЧхПФбжжхчД'), ['щжОожЦндцШйТАй', 'йуРСЦУЗФУЦПвРфевСлфдРещЦтИтЩЩТг', 'ЕГЧдмХмРАлнЧ', 'йнкФизГСЗнуКбЙВЙчАТТрСхаЙШтсдгХ', 'ЧПрнРЖЙцХИщ', 'зЕ', 'СжВЩчГзБХбйТиклкдШШИееАлЧЩН', 'МШщГйБХжЙпйЕЗТзКмпе', 'НКбНщОМДзлдЧОс', 'НчзВХОпХХШМОХФумБгсрРЧИчВтгутВЩо']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('упТУЖелФкЧЧУЦРжоБтХсжКщД'), ['щКшуОЖааЖйнЕбДИжМК', 'ЕкнШцХРВтНйШоНбЙйУоЧщУиРпШЧхмКЧжх', 'рвЩЗоЗхшЗвлизкСзебЩКМКжбша', 'ДииБНСШвцЦбаСсИжЕЗмхмВ', 'СЦоБЗПМтмшрУлрДТФГЖиувШЗууШзв', 'ЦЗБЕзВХЙбйВОмЗпхндЗ', 'ЗНизЧВШкГВтпсЖж', 'уШиБПЙЧтРаЕгИ', 'ЙшпПА', 'ЧоММаАйМСфбхуФкефФштгУА']) from system.numbers limit 10; - -select 0 = multiSearchFirstPosition(materialize('abcdefgh'), ['z', 'pq']) from system.numbers limit 10; -select 1 = multiSearchFirstPosition(materialize('abcdefgh'), ['a', 'b', 'c', 'd']) from system.numbers limit 10; -select 1 = multiSearchFirstPosition(materialize('abcdefgh'), ['defgh', 'bcd', 'abcd', 'c']) from system.numbers limit 10; -select 1 = multiSearchFirstPosition(materialize('abcdefgh'), ['', 'bcd', 'bcd', 'c']) from system.numbers limit 10; -select 2 = multiSearchFirstPosition(materialize('abcdefgh'), ['something', 'bcd', 'bcd', 'c']) from system.numbers limit 10; -select 6 = multiSearchFirstPosition(materialize('abcdefgh'), ['something', 'bcdz', 'fgh', 'f']) from system.numbers limit 10; - -select 0 = multiSearchFirstPositionCaseInsensitive(materialize('abcdefgh'), ['z', 'pq']) from system.numbers limit 10; -select 1 = multiSearchFirstPositionCaseInsensitive(materialize('aBcdefgh'), ['A', 'b', 'c', 'd']) from system.numbers limit 10; -select 1 = multiSearchFirstPositionCaseInsensitive(materialize('abCDefgh'), ['defgh', 'bcd', 'aBCd', 'c']) from system.numbers limit 10; -select 1 = multiSearchFirstPositionCaseInsensitive(materialize('abCdeFgH'), ['', 'bcd', 'bcd', 'c']) from system.numbers limit 10; -select 2 = multiSearchFirstPositionCaseInsensitive(materialize('ABCDEFGH'), ['something', 'bcd', 'bcd', 'c']) from system.numbers limit 10; -select 6 = multiSearchFirstPositionCaseInsensitive(materialize('abcdefgh'), ['sOmEthIng', 'bcdZ', 'fGh', 'F']) from system.numbers limit 10; - -select 0 = multiSearchFirstPositionUTF8(materialize('абвгдежз'), ['л', 'ъ']) from system.numbers limit 10; -select 1 = multiSearchFirstPositionUTF8(materialize('абвгдежз'), ['а', 'б', 'в', 'г']) from system.numbers limit 10; -select 1 = multiSearchFirstPositionUTF8(materialize('абвгдежз'), ['гдежз', 'бвг', 'абвг', 'вг']) from system.numbers limit 10; -select 1 = multiSearchFirstPositionUTF8(materialize('абвгдежз'), ['', 'бвг', 'бвг', 'в']) from system.numbers limit 10; -select 2 = multiSearchFirstPositionUTF8(materialize('абвгдежз'), ['что', 'в', 'гдз', 'бвг']) from system.numbers limit 10; -select 6 = multiSearchFirstPositionUTF8(materialize('абвгдежз'), ['з', 'бвгя', 'ежз', 'з']) from system.numbers limit 10; - -select 0 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДежз'), ['Л', 'Ъ']) from system.numbers limit 10; -select 1 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДежз'), ['А', 'б', 'в', 'г']) from system.numbers limit 10; -select 1 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДежз'), ['гДеЖз', 'бВг', 'АБВг', 'вг']) from system.numbers limit 10; -select 1 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДежз'), ['', 'бвг', 'Бвг', 'в']) from system.numbers limit 10; -select 2 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДежз'), ['что', 'в', 'гдз', 'бвг']) from system.numbers limit 10; -select 6 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДежЗ'), ['З', 'бвгЯ', 'ЕЖз', 'з']) from system.numbers limit 10; - -select 1 = multiSearchAny(materialize('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'), -['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaa']); - -select 1 = multiSearchFirstIndex(materialize('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'), -['aaaa', 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaab']); - -select 1 = multiSearchAny(materialize('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'), -['aaaa', 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaab']); - -select 1 = multiSearchFirstPosition(materialize('aaaabaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'), -['aaaa', 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaab']); - -select 1 = multiSearchFirstPosition(materialize('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'), -['aaab', 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaa']); - -select 0 = multiSearchAny(materialize('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'), -['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'b']); - --- 254 -select -[ -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1 -] = -multiSearchAllPositions(materialize('string'), -['o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'str']); - -select 254 = multiSearchFirstIndex(materialize('string'), -['o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'str']); - - -select -[ -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1 -] = -multiSearchAllPositions(materialize('string'), -['o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'str']); - -select 255 = multiSearchFirstIndex(materialize('string'), -['o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'str']); - -select multiSearchAllPositions(materialize('string'), -['o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'str']); -- { serverError 42 } - -select multiSearchFirstIndex(materialize('string'), -['o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', -'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'str']); -- { serverError 42 } - - -select [1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1]= multiSearchAllPositions(materialize('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'), -['aaaa', 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaa']); diff --git a/tests/queries/0_stateless/02364_multiSearch_function_family.reference b/tests/queries/0_stateless/02364_multiSearch_function_family.reference new file mode 100644 index 00000000000..4ddb6036240 --- /dev/null +++ b/tests/queries/0_stateless/02364_multiSearch_function_family.reference @@ -0,0 +1,16516 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02364_multiSearch_function_family.sql b/tests/queries/0_stateless/02364_multiSearch_function_family.sql new file mode 100644 index 00000000000..c814fdcfe04 --- /dev/null +++ b/tests/queries/0_stateless/02364_multiSearch_function_family.sql @@ -0,0 +1,1047 @@ +SET send_logs_level = 'fatal'; + +select 0 = multiSearchAny('\0', CAST([], 'Array(String)')); +select 0 = multiSearchAnyCaseInsensitive('\0', CAST([], 'Array(String)')); +select 0 = multiSearchAnyCaseInsensitiveUTF8('\0', CAST([], 'Array(String)')); +select 0 = multiSearchAnyUTF8('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstIndex('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstIndexCaseInsensitive('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstIndexUTF8('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstPosition('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstPositionCaseInsensitive('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstPositionCaseInsensitiveUTF8('\0', CAST([], 'Array(String)')); +select 0 = multiSearchFirstPositionUTF8('\0', CAST([], 'Array(String)')); +select [] = multiSearchAllPositions('\0', CAST([], 'Array(String)')); +select [] = multiSearchAllPositionsCaseInsensitive('\0', CAST([], 'Array(String)')); +select [] = multiSearchAllPositionsCaseInsensitiveUTF8('\0', CAST([], 'Array(String)')); +select [] = multiSearchAllPositionsUTF8('\0', CAST([], 'Array(String)')); + +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['b']); +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bc']); +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcd']); +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcde']); +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdef']); +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdefg']); +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdefgh']); + +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdefgh']); +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdefg']); +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdef']); +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcde']); +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcd']); +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abc']); +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['ab']); +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['a']); + +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['c']); +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cd']); +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cde']); +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdef']); +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdefg']); +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdefgh']); + +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['defgh']); +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['defg']); +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['def']); +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['de']); +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['d']); + +select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['e']); +select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['ef']); +select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['efg']); +select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['efgh']); + +select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['fgh']); +select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['fg']); +select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['f']); + +select [7] = multiSearchAllPositions(materialize('abcdefgh'), ['g']); +select [7] = multiSearchAllPositions(materialize('abcdefgh'), ['gh']); + +select [8] = multiSearchAllPositions(materialize('abcdefgh'), ['h']); + +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['b']) from system.numbers limit 10; +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bc']) from system.numbers limit 10; +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcd']) from system.numbers limit 10; +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcde']) from system.numbers limit 10; +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdef']) from system.numbers limit 10; +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdefg']) from system.numbers limit 10; +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdefgh']) from system.numbers limit 10; + +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdefgh']) from system.numbers limit 10; +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdefg']) from system.numbers limit 10; +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdef']) from system.numbers limit 10; +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcde']) from system.numbers limit 10; +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcd']) from system.numbers limit 10; +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abc']) from system.numbers limit 10; +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['ab']) from system.numbers limit 10; +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['a']) from system.numbers limit 10; + +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['c']) from system.numbers limit 10; +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cd']) from system.numbers limit 10; +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cde']) from system.numbers limit 10; +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdef']) from system.numbers limit 10; +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdefg']) from system.numbers limit 10; +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdefgh']) from system.numbers limit 10; + +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['defgh']) from system.numbers limit 10; +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['defg']) from system.numbers limit 10; +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['def']) from system.numbers limit 10; +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['de']) from system.numbers limit 10; +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['d']) from system.numbers limit 10; + +select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['e']) from system.numbers limit 10; +select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['ef']) from system.numbers limit 10; +select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['efg']) from system.numbers limit 10; +select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['efgh']) from system.numbers limit 10; + +select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['fgh']) from system.numbers limit 10; +select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['fg']) from system.numbers limit 10; +select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['f']) from system.numbers limit 10; + +select [7] = multiSearchAllPositions(materialize('abcdefgh'), ['g']) from system.numbers limit 10; +select [7] = multiSearchAllPositions(materialize('abcdefgh'), ['gh']) from system.numbers limit 10; + +select [8] = multiSearchAllPositions(materialize('abcdefgh'), ['h']) from system.numbers limit 10; + +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['b']) from system.numbers limit 129; +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bc']) from system.numbers limit 129; +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcd']) from system.numbers limit 10; +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcde']) from system.numbers limit 129; +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdef']) from system.numbers limit 129; +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdefg']) from system.numbers limit 129; +select [2] = multiSearchAllPositions(materialize('abcdefgh'), ['bcdefgh']) from system.numbers limit 129; + +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdefgh']) from system.numbers limit 129; +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdefg']) from system.numbers limit 129; +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcdef']) from system.numbers limit 129; +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcde']) from system.numbers limit 129; +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abcd']) from system.numbers limit 129; +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['abc']) from system.numbers limit 129; +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['ab']) from system.numbers limit 129; +select [1] = multiSearchAllPositions(materialize('abcdefgh'), ['a']) from system.numbers limit 129; + +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['c']) from system.numbers limit 129; +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cd']) from system.numbers limit 129; +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cde']) from system.numbers limit 129; +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdef']) from system.numbers limit 129; +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdefg']) from system.numbers limit 129; +select [3] = multiSearchAllPositions(materialize('abcdefgh'), ['cdefgh']) from system.numbers limit 129; + +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['defgh']) from system.numbers limit 129; +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['defg']) from system.numbers limit 129; +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['def']) from system.numbers limit 129; +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['de']) from system.numbers limit 129; +select [4] = multiSearchAllPositions(materialize('abcdefgh'), ['d']) from system.numbers limit 129; + +select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['e']) from system.numbers limit 129; +select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['ef']) from system.numbers limit 129; +select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['efg']) from system.numbers limit 129; +select [5] = multiSearchAllPositions(materialize('abcdefgh'), ['efgh']) from system.numbers limit 129; + +select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['fgh']) from system.numbers limit 129; +select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['fg']) from system.numbers limit 129; +select [6] = multiSearchAllPositions(materialize('abcdefgh'), ['f']) from system.numbers limit 129; + +select [7] = multiSearchAllPositions(materialize('abcdefgh'), ['g']) from system.numbers limit 129; +select [7] = multiSearchAllPositions(materialize('abcdefgh'), ['gh']) from system.numbers limit 129; + +select [8] = multiSearchAllPositions(materialize('abcdefgh'), ['h']) from system.numbers limit 129; + +select [2] = multiSearchAllPositions(materialize('abc'), ['b']); +select [2] = multiSearchAllPositions(materialize('abc'), ['bc']); +select [0] = multiSearchAllPositions(materialize('abc'), ['bcde']); +select [0] = multiSearchAllPositions(materialize('abc'), ['bcdef']); +select [0] = multiSearchAllPositions(materialize('abc'), ['bcdefg']); +select [0] = multiSearchAllPositions(materialize('abc'), ['bcdefgh']); + +select [0] = multiSearchAllPositions(materialize('abc'), ['abcdefg']); +select [0] = multiSearchAllPositions(materialize('abc'), ['abcdef']); +select [0] = multiSearchAllPositions(materialize('abc'), ['abcde']); +select [0] = multiSearchAllPositions(materialize('abc'), ['abcd']); +select [1] = multiSearchAllPositions(materialize('abc'), ['abc']); +select [1] = multiSearchAllPositions(materialize('abc'), ['ab']); +select [1] = multiSearchAllPositions(materialize('abc'), ['a']); + +select [3] = multiSearchAllPositions(materialize('abcd'), ['c']); +select [3] = multiSearchAllPositions(materialize('abcd'), ['cd']); +select [0] = multiSearchAllPositions(materialize('abcd'), ['cde']); +select [0] = multiSearchAllPositions(materialize('abcd'), ['cdef']); +select [0] = multiSearchAllPositions(materialize('abcd'), ['cdefg']); +select [0] = multiSearchAllPositions(materialize('abcd'), ['cdefgh']); + +select [0] = multiSearchAllPositions(materialize('abc'), ['defgh']); +select [0] = multiSearchAllPositions(materialize('abc'), ['defg']); +select [0] = multiSearchAllPositions(materialize('abc'), ['def']); +select [0] = multiSearchAllPositions(materialize('abc'), ['de']); +select [0] = multiSearchAllPositions(materialize('abc'), ['d']); + + +select [2] = multiSearchAllPositions(materialize('abc'), ['b']) from system.numbers limit 10; +select [2] = multiSearchAllPositions(materialize('abc'), ['bc']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abc'), ['bcde']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abc'), ['bcdef']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abc'), ['bcdefg']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abc'), ['bcdefgh']) from system.numbers limit 10; + + +select [0] = multiSearchAllPositions(materialize('abc'), ['abcdefg']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abc'), ['abcdef']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abc'), ['abcde']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abc'), ['abcd']) from system.numbers limit 10; +select [1] = multiSearchAllPositions(materialize('abc'), ['abc']) from system.numbers limit 10; +select [1] = multiSearchAllPositions(materialize('abc'), ['ab']) from system.numbers limit 10; +select [1] = multiSearchAllPositions(materialize('abc'), ['a']) from system.numbers limit 10; + +select [3] = multiSearchAllPositions(materialize('abcd'), ['c']) from system.numbers limit 10; +select [3] = multiSearchAllPositions(materialize('abcd'), ['cd']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abcd'), ['cde']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abcd'), ['cdef']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abcd'), ['cdefg']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abcd'), ['cdefgh']) from system.numbers limit 10; + +select [0] = multiSearchAllPositions(materialize('abc'), ['defgh']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abc'), ['defg']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abc'), ['def']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abc'), ['de']) from system.numbers limit 10; +select [0] = multiSearchAllPositions(materialize('abc'), ['d']) from system.numbers limit 10; + +select [1] = multiSearchAllPositions(materialize('abc'), ['']); +select [1] = multiSearchAllPositions(materialize('abc'), ['']) from system.numbers limit 10; +select [1] = multiSearchAllPositions(materialize('abc'), ['']) from system.numbers limit 100; +select [1] = multiSearchAllPositions(materialize('abc'), ['']) from system.numbers limit 1000; + +select [1] = multiSearchAllPositions(materialize('abab'), ['ab']); +select [1] = multiSearchAllPositions(materialize('abababababababababababab'), ['abab']); +select [1] = multiSearchAllPositions(materialize('abababababababababababab'), ['abababababababababa']); + +select [1] = multiSearchAllPositions(materialize('abc'), materialize([''])); +select [1] = multiSearchAllPositions(materialize('abc'), materialize([''])) from system.numbers limit 10; +select [1] = multiSearchAllPositions(materialize('abab'), materialize(['ab'])); +select [2] = multiSearchAllPositions(materialize('abab'), materialize(['ba'])); +select [1] = multiSearchAllPositionsCaseInsensitive(materialize('aBaB'), materialize(['abab'])); +select [3] = multiSearchAllPositionsUTF8(materialize('ab€ab'), materialize(['€'])); +select [3] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ab€AB'), materialize(['€ab'])); + +select 1 = multiSearchAny(materialize('abcdefgh'), ['b']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['bc']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcd']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcde']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdef']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdefg']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdefgh']); + +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdefgh']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdefg']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdef']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcde']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcd']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['abc']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['ab']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['a']); + +select 1 = multiSearchAny(materialize('abcdefgh'), ['c']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['cd']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['cde']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['cdef']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['cdefg']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['cdefgh']); + +select 1 = multiSearchAny(materialize('abcdefgh'), ['defgh']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['defg']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['def']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['de']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['d']); + +select 1 = multiSearchAny(materialize('abcdefgh'), ['e']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['ef']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['efg']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['efgh']); + +select 1 = multiSearchAny(materialize('abcdefgh'), ['fgh']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['fg']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['f']); + +select 1 = multiSearchAny(materialize('abcdefgh'), ['g']); +select 1 = multiSearchAny(materialize('abcdefgh'), ['gh']); + +select 1 = multiSearchAny(materialize('abcdefgh'), ['h']); + +select 1 = multiSearchAny(materialize('abcdefgh'), ['b']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['bc']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcd']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcde']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdef']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdefg']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdefgh']) from system.numbers limit 10; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdefgh']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdefg']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdef']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcde']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcd']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['abc']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['ab']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['a']) from system.numbers limit 10; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['c']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['cd']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['cde']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['cdef']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['cdefg']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['cdefgh']) from system.numbers limit 10; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['defgh']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['defg']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['def']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['de']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['d']) from system.numbers limit 10; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['e']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['ef']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['efg']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['efgh']) from system.numbers limit 10; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['fgh']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['fg']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['f']) from system.numbers limit 10; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['g']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['gh']) from system.numbers limit 10; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['h']) from system.numbers limit 10; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['b']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['bc']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcd']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcde']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdef']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdefg']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['bcdefgh']) from system.numbers limit 129; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdefgh']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdefg']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcdef']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcde']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['abcd']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['abc']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['ab']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['a']) from system.numbers limit 129; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['c']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['cd']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['cde']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['cdef']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['cdefg']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['cdefgh']) from system.numbers limit 129; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['defgh']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['defg']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['def']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['de']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['d']) from system.numbers limit 129; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['e']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['ef']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['efg']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['efgh']) from system.numbers limit 129; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['fgh']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['fg']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['f']) from system.numbers limit 129; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['g']) from system.numbers limit 129; +select 1 = multiSearchAny(materialize('abcdefgh'), ['gh']) from system.numbers limit 129; + +select 1 = multiSearchAny(materialize('abcdefgh'), ['h']) from system.numbers limit 129; + +select 1 = multiSearchAny(materialize('abc'), ['b']); +select 1 = multiSearchAny(materialize('abc'), ['bc']); +select 0 = multiSearchAny(materialize('abc'), ['bcde']); +select 0 = multiSearchAny(materialize('abc'), ['bcdef']); +select 0 = multiSearchAny(materialize('abc'), ['bcdefg']); +select 0 = multiSearchAny(materialize('abc'), ['bcdefgh']); + +select 0 = multiSearchAny(materialize('abc'), ['abcdefg']); +select 0 = multiSearchAny(materialize('abc'), ['abcdef']); +select 0 = multiSearchAny(materialize('abc'), ['abcde']); +select 0 = multiSearchAny(materialize('abc'), ['abcd']); +select 1 = multiSearchAny(materialize('abc'), ['abc']); +select 1 = multiSearchAny(materialize('abc'), ['ab']); +select 1 = multiSearchAny(materialize('abc'), ['a']); + +select 1 = multiSearchAny(materialize('abcd'), ['c']); +select 1 = multiSearchAny(materialize('abcd'), ['cd']); +select 0 = multiSearchAny(materialize('abcd'), ['cde']); +select 0 = multiSearchAny(materialize('abcd'), ['cdef']); +select 0 = multiSearchAny(materialize('abcd'), ['cdefg']); +select 0 = multiSearchAny(materialize('abcd'), ['cdefgh']); + +select 0 = multiSearchAny(materialize('abc'), ['defgh']); +select 0 = multiSearchAny(materialize('abc'), ['defg']); +select 0 = multiSearchAny(materialize('abc'), ['def']); +select 0 = multiSearchAny(materialize('abc'), ['de']); +select 0 = multiSearchAny(materialize('abc'), ['d']); + + +select 1 = multiSearchAny(materialize('abc'), ['b']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abc'), ['bc']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abc'), ['bcde']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abc'), ['bcdef']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abc'), ['bcdefg']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abc'), ['bcdefgh']) from system.numbers limit 10; + + +select 0 = multiSearchAny(materialize('abc'), ['abcdefg']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abc'), ['abcdef']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abc'), ['abcde']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abc'), ['abcd']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abc'), ['abc']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abc'), ['ab']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abc'), ['a']) from system.numbers limit 10; + +select 1 = multiSearchAny(materialize('abcd'), ['c']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abcd'), ['cd']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abcd'), ['cde']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abcd'), ['cdef']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abcd'), ['cdefg']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abcd'), ['cdefgh']) from system.numbers limit 10; + +select 0 = multiSearchAny(materialize('abc'), ['defgh']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abc'), ['defg']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abc'), ['def']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abc'), ['de']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('abc'), ['d']) from system.numbers limit 10; + +select 1 = multiSearchAny(materialize('abc'), ['']); +select 1 = multiSearchAny(materialize('abc'), ['']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('abc'), ['']) from system.numbers limit 100; +select 1 = multiSearchAny(materialize('abc'), ['']) from system.numbers limit 1000; + +select 1 = multiSearchAny(materialize('abab'), ['ab']); +select 1 = multiSearchAny(materialize('abababababababababababab'), ['abab']); +select 1 = multiSearchAny(materialize('abababababababababababab'), ['abababababababababa']); + +-- select 'some random tests'; + +select [4, 1, 1, 2, 6, 1, 1, 0, 4, 1, 14, 0, 10, 0, 16, 6] = multiSearchAllPositions(materialize('jmdqwjbrxlbatqeixknricfk'), ['qwjbrxlba', 'jmd', '', 'mdqwjbrxlbatqe', 'jbrxlbatqeixknric', 'jmdqwjbrxlbatqeixknri', '', 'fdtmnwtts', 'qwjbrxlba', '', 'qeixknricfk', 'hzjjgrnoilfkvzxaemzhf', 'lb', 'kamz', 'ixknr', 'jbrxlbatq']) from system.numbers limit 10; +select [0, 0, 0, 2, 3, 0, 1, 0, 5, 0, 0, 0, 11, 10, 6, 7] = multiSearchAllPositions(materialize('coxcctuehmzkbrsmodfvx'), ['bkhnp', 'nlypjvriuk', 'rkslxwfqjjivcwdexrdtvjdtvuu', 'oxcctuehm', 'xcctuehmzkbrsm', 'kfrieuocovykjmkwxbdlkgwctwvcuh', 'coxc', 'lbwvetgxyndxjqqwthtkgasbafii', 'ctuehmzkbrsmodfvx', 'obzldxjldxowk', 'ngfikgigeyll', 'wdaejjukowgvzijnw', 'zkbr', 'mzkb', 'tuehm', 'ue']) from system.numbers limit 10; +select [1, 1, 0, 0, 0, 1, 1, 1, 4, 0, 6, 6, 0, 10, 1, 5] = multiSearchAllPositions(materialize('mpswgtljbbrmivkcglamemayfn'), ['', 'm', 'saejhpnfgfq', 'rzanrkdssmmkanqjpfi', 'oputeneprgoowg', 'mp', '', '', 'wgtljbbrmivkcglamemay', 'cbpthtrgrmgfypizi', 'tl', 'tlj', 'xuhs', 'brmivkcglamemayfn', '', 'gtljb']) from system.numbers limit 10; +select [1, 0, 0, 8, 6, 0, 7, 1, 3, 0, 0, 0, 0, 12] = multiSearchAllPositions(materialize('arbphzbbecypbzsqsljurtddve'), ['arbphzb', 'mnrboimjfijnti', 'cikcrd', 'becypbz', 'z', 'uocmqgnczhdcrvtqrnaxdxjjlhakoszuwc', 'bbe', '', 'bp', 'yhltnexlpdijkdzt', 'jkwjmrckvgmccmmrolqvy', 'vdxmicjmfbtsbqqmqcgtnrvdgaucsgspwg', 'witlfqwvhmmyjrnrzttrikhhsrd', 'pbzsqsljurt']) from system.numbers limit 10; +select [7, 0, 0, 8, 0, 2, 0, 0, 6, 0, 2, 0, 3, 1] = multiSearchAllPositions(materialize('aizovxqpzcbbxuhwtiaaqhdqjdei'), ['qpzcbbxuhw', 'jugrpglqbm', 'dspwhzpyjohhtizegrnswhjfpdz', 'pzcbbxuh', 'vayzeszlycke', 'i', 'gvrontcpqavsjxtjwzgwxugiyhkhmhq', 'gyzmeroxztgaurmrqwtmsxcqnxaezuoapatvu', 'xqpzc', 'mjiswsvlvlpqrhhptqq', 'iz', 'hmzjxxfjsvcvdpqwtrdrp', 'zovxqpzcbbxuhwtia', 'ai']) from system.numbers limit 10; +select [0, 0, 0, 19, 14, 22, 10, 0, 0, 13, 0, 8] = multiSearchAllPositions(materialize('ydfgiluhyxwqdfiwtzobwzscyxhuov'), ['srsoubrgghleyheujsbwwwykerzlqphgejpxvog', 'axchkyleddjwkvbuyhmekpbbbztxdlm', 'zqodzvlkmfe', 'obwz', 'fi', 'zsc', 'xwq', 'pvmurvrd', 'uulcdtexckmrsokmgdpkstlkoavyrmxeaacvydxf', 'dfi', 'mxcngttujzgtlssrmluaflmjuv', 'hyxwqdfiwtzobwzscyxhu']) from system.numbers limit 10; +select [6, 1, 1, 0, 0, 5, 1, 0, 8, 0, 5, 0, 2, 12, 0, 15, 0, 0] = multiSearchAllPositions(materialize('pyepgwainvmwekwhhqxxvzdjw'), ['w', '', '', 'gvvkllofjnxvcu', 'kmwwhboplctvzazcyfpxhwtaddfnhekei', 'gwainv', 'pyepgwain', 'ekpnogkzzmbpfynsunwqp', 'invmwe', 'hrxpiplfplqjsstuybksuteoz', 'gwa', 'akfpyduqrwosxcbdemtxrxvundrgse', 'yepgwainvmw', 'wekwhhqxxvzdjw', 'fyimzvedmyriubgoznmcav', 'whhq', 'ozxowbwdqfisuupyzaqynoprgsjhkwlum', 'vpoufrofekajksdp']) from system.numbers limit 10; +select [0, 0, 5, 1, 1, 0, 15, 1, 5, 10, 4, 0, 1, 0, 3, 0, 0, 0] = multiSearchAllPositions(materialize('lqwahffxurkbhhzytequotkfk'), ['rwjqudpuaiufle', 'livwgbnflvy', 'hffxurkbhh', '', '', 'xcajwbqbttzfzfowjubmmgnmssat', 'zytequ', 'lq', 'h', 'rkbhh', 'a', 'immejthwgdr', '', 'llhhnlhcvnxxorzzjt', 'w', 'cvjynqxcivmmmvc', 'wexjomdcmursppjtsweybheyxzleuz', 'fzronsnddfxwlkkzidiknhpjipyrcrzel']) from system.numbers limit 10; +select [0, 1, 11, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4, 1] = multiSearchAllPositions(materialize('nkddriylnakicwgdwrfxpodqea'), ['izwdpgrgpmjlwkanjrffgela', '', 'kicw', 'hltmfymgmrjckdiylkzjlvvyuleksikdjrg', 'yigveskrbidknjxigwilmkgyizewikh', 'xyvzhsnqmuec', 'odcgzlavzrwesjks', 'oilvfgliktoujukpgzvhmokdgkssqgqot', 'llsfsurvimbahwqtbqbp', 'nxj', 'pimydixeobdxmdkvhcyzcgnbhzsydx', 'couzmvxedobuohibgxwoxvmpote', 'driylnakicwgdwrf', 'nkddr']) from system.numbers limit 10; +select [0, 0, 0, 3, 0, 15, 0, 0, 12, 7, 0, 0, 0, 0, 5, 0] = multiSearchAllPositions(materialize('jnckhtjqwycyihuejibqmddrdxe'), ['tajzx', 'vuddoylclxatcjvinusdwt', 'spxkhxvzsljkmnzpeubszjnhqczavgtqopxn', 'ckhtjqwycyi', 'xlbfzdxspldoes', 'u', 'czosfebeznt', 'gzhabdsuyreisxvyfrfrkq', 'yihuejibqmd', 'jqwycyihuejibqm', 'cfbvprgzx', 'hxu', 'vxbhrfpzacgd', 'afoaij', 'htjqwycyihu', 'httzbskqd']) from system.numbers limit 10; +select [0, 0, 12, 4, 4, 0, 13, 23, 0, 1, 0, 2, 0, 0, 0, 3, 0, 0] = multiSearchAllPositions(materialize('dzejajvpoojdkqbnayahygidyrjmb'), ['khwxxvtnqhobbvwgwkpusjlhlzifiuclycml', 'nzvuhtwdaivo', 'dkqbnayahygidyr', 'jajvpoo', 'j', 'wdtbvwmeqgyvetu', 'kqbn', 'idyrjmb', 'tsnxuxevsxrxpgpfdgrkhwqpkse', '', 'efsdgzuefhdzkmquxu', 'zejajvpoojdkqbnayahyg', 'ugwfuighbygrxyctop', 'fcbxzbdugc', 'dxmzzrcplob', 'ejaj', 'wmmupyxrylvawsyfccluiiene', 'ohzmsqhpzbafvbzqwzftbvftei']) from system.numbers limit 10; +select [6, 8, 1, 4, 0, 10, 0, 1, 14, 0, 1, 0, 5, 0, 0, 0, 0, 15, 0, 1] = multiSearchAllPositions(materialize('ffaujlverosspbzaqefjzql'), ['lvero', 'erossp', 'f', 'ujlverosspbz', 'btfimgklzzxlbkbuqyrmnud', 'osspb', 'muqexvtjuaar', 'f', 'bzaq', 'lprihswhwkdhqciqhfaowarn', 'ffaujlve', 'uhbbjrqjb', 'jlver', 'umucyhbbu', 'pjthtzmgxhvpbdphesnnztuu', 'xfqhfdfsbbazactpastzvzqudgk', 'lvovjfoatc', 'z', 'givejzhoqsd', '']) from system.numbers limit 10; +select [5, 7, 0, 1, 6, 0, 0, 1, 1, 2, 0, 1, 4, 2, 0, 6, 0, 0] = multiSearchAllPositions(materialize('hzftozkvquknsahhxefzg'), ['ozkvquknsahhxefzg', 'kv', 'lkdhmafrec', '', 'zkvquknsahh', 'xmjuizyconipirigdmhqclox', 'dqqwolnkkwbyyjicsoshidbay', '', '', 'zf', 'sonvmkapcjcakgpejvn', 'hzftoz', 't', 'zftozkvqukns', 'dyuqohvehxsvdzdlqzl', 'zkvquknsahhx', 'vueohmytvmglqwptfbhxffspf', 'ilkdurxg']) from system.numbers limit 10; +select [1, 7, 6, 4, 0, 1, 0, 0, 0, 9, 7, 1, 1, 0, 0, 0] = multiSearchAllPositions(materialize('aapdygjzrhskntrphianzjob'), ['', 'jz', 'gjzrh', 'dygjzrhskntrphia', 'qcnahphlxmdru', '', 'rnwvzdn', 'isbekwuivytqggsxniqojrvpwjdr', 'sstwvgyavbwxvjojrpg', 'rhskn', 'jzrhskntrp', '', '', 'toilvppgjizaxtidizgbgygubmob', 'vjwzwpvsklkxqgeqqmtssnhlmw', 'znvpjjlydvzhkt']) from system.numbers limit 10; +select [0, 1, 0, 1, 0, 0, 10, 0, 0, 0, 11, 0, 5, 0] = multiSearchAllPositions(materialize('blwpfdjjkxettfetdoxvxbyk'), ['wgylnwqcrojacofrcanjme', 'bl', 'qqcunzpvgi', '', 'ijemdmmdxkakrawwdqrjtrttig', 'qwkaifalc', 'xe', 'zqocnfuvzowuqkmwrfxw', 'xpaayeljvly', 'wvphqqhulpepjjjnxjfudfcomajc', 'ettfetdoxvx', 'ikablovwhnbohibbuhwjshhdemidgreqf', 'fdjjkxett', 'kiairehwbxveqkcfqhgopztgpatljgqp']) from system.numbers limit 10; +select [0, 0, 6, 1, 1, 0, 0, 1, 2, 0, 0, 0, 0, 0] = multiSearchAllPositions(materialize('vghzgedqpnqtvaoonwsz'), ['mfyndhucfpzjxzaezny', 'niejb', 'edqpnqt', '', 'v', 'kivdvealqadzdatziujdnvymmia', 'lvznmgwtlwevcxyfbkqc', 'vghzge', 'gh', 'tbzle', 'vjiqponbvgvguuhqdijbdeu', 'mshlyabasgukboknbqgmmmj', 'kjk', 'abkeftpnpvdkfyrxbrihyfxcfxablv']) from system.numbers limit 10; +select [0, 0, 0, 0, 9, 0, 7, 0, 9, 8, 0, 0] = multiSearchAllPositions(materialize('oaghnutqsqcnwvmzrnxgacsovxiko'), ['upien', 'moqszigvduvvwvmpemupvmmzctbrbtqggrk', 'igeiaccvxejtfvifrmimwpewllcggji', 'wnwjorpzgsqiociw', 'sq', 'rkysegpoej', 'tqsqcnwvmzrnxgacsovxiko', 'ioykypvfjufbicpyrpfuhugk', 's', 'qsqcnwvmzrnxgacsov', 'hhbeisvmpnkwmimgyfmybtljiu', 'kfozjowd']) from system.numbers limit 10; +select [0, 5, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 5, 0, 0, 14, 1, 1, 0, 0] = multiSearchAllPositions(materialize('wbjfsevqspsvbwlzrkhcfuhxddbq'), ['ltgjbz', 's', 'qdfnmggupdfxjfnmvwyrqopxtxf', 'sazlkmaikcltojbzbmdfddu', 'yzanifqxufyfwrxzkhngoxkrrph', 'iwskc', 'xkykshryphyfnwcnmjfqjrixykmzmwm', 'wwpenztbhkdbwidfkypqlxivsjs', 'rlkevy', 'qigywtkezwd', '', 'c', 'sevqspsvbwlzrk', 'gwg', 'iduhrjsrtodxdkjykjoghtjtvplrscitxnvt', 'wlzrkhcfuhxddb', '', 'wbjfsev', 'zytusrcvqbazb', 'tec']) from system.numbers limit 10; +select [0, 1, 5, 0, 6, 8, 0, 3, 2, 0, 0, 9, 0, 4, 0, 0] = multiSearchAllPositions(materialize('mxiifpzlovgfozpgirtio'), ['srullnscuzenzhp', '', 'f', 'apetxezid', 'pzlovgf', 'lo', 'ecbmso', 'i', 'xiifpzlovgfozpgir', 'bnefwypvctubvslsesnctqspdyctq', 'tdncmgbikboss', 'o', 'zmgobcarxlxaho', 'ifpzlovgfozpg', 'dwmjqyylvsxzfr', 'pxhrecconce']) from system.numbers limit 10; +select [0, 0, 0, 2, 0, 0, 2, 0, 8, 0, 0, 0, 7, 0, 0, 0, 21, 3, 1, 8] = multiSearchAllPositions(materialize('jtvnrdpdevgnzexqdrrxqgiujexhm'), ['ibkvzoqmiyfgfztupug', 'iqzeixfykxcghlbgsicxiywlurrgjsywwk', 'vzdffjzlqxgzdcrkgoro', 'tvnrdpdevgnzexqdr', 'nqywueahcmoojtyjlhfpysk', 'iqalixciiidvrtmpzozfb', 'tv', 'rxkfeasoff', 'devgnzexqdrrxqgiuj', 'kvvuvyplboowjrestyvdfrxdjjujvkxy', 'shkhpneekuyyqtxfxutvz', 'yy', 'pdevgnz', 'nplpydxiwnbvlhoorcmqkycqisi', 'jlkxplbftfkxqgnqnaw', 'qdggpjenbrwbjtorbi', 'qgiuje', 'vnrdpd', '', 'dev']) from system.numbers limit 10; +select [14, 0, 0, 7, 20, 6, 0, 13, 0, 0, 20, 0, 20, 2, 0, 8, 2, 11, 2, 0] = multiSearchAllPositions(materialize('asjwxabjrwgcdviokfaoqvqiafz'), ['v', 'zqngytligwwpzxhatyayvdnbbj', 'gjicovfzgbyagiirn', 'bjrwgcdviok', 'oqvqiafz', 'abjrwgc', 'wulrpfzh', 'dviokfao', 'esnchjuiufjadqmdtrpcd', 'tkodqzsjchpaftk', 'oqvq', 'eyoshlrlvmnqjmtmloryvg', 'oqv', 'sjwx', 'uokueelyytnoidplwmmox', 'jrwgcdviokfaoqvqiaf', 'sjwxabjrwgcdviokfaoqvqi', 'gcdviokfa', 'sjwxab', 'zneabsnfucjcwauxmudyxibnmxzfx']) from system.numbers limit 10; +select [0, 16, 8, 0, 10, 0, 0, 0, 0, 1, 0, 6, 0, 1, 0, 4, 0, 6, 0, 0] = multiSearchAllPositions(materialize('soxfqagiuhkaylzootfjy'), ['eveprzxphyenbrnnznpctvxn', 'oo', 'iuhka', 'ikutjhrnvzfb', 'h', 'duyvvjizristnkczgwj', 'ihfrp', 'afpyrlj', 'uonp', 'soxfqagiuhkaylzootfjy', 'qeckxkoxldpzzpmkbvcex', 'agiuhkaylzo', 'tckcumkbsgrgqjvtlijack', '', 'fnfweqlldcdnwfaohqohp', 'fqagiuhkayl', 'pqnvwprxwwrcjqvfsbfimwye', 'agi', 'ta', 'r']) from system.numbers limit 10; +select [3, 7, 1, 6, 0, 1, 0, 11, 0, 9, 17, 1, 18, 12] = multiSearchAllPositions(materialize('ladbcypcbcxahmujwezkvweud'), ['db', 'pcbcxahm', 'lad', 'ypcb', 'atevkzyyxhphtuekymhh', 'lad', 'mltjrwaibetrtwpfa', 'xahmujwezkvweud', 'dg', 'bcxahmujw', 'we', '', 'e', 'ahmujwezkvw']) from system.numbers limit 10; +select [6, 0, 11, 0, 7, 0, 0, 0, 6, 1, 0, 3, 0, 0, 0, 0] = multiSearchAllPositions(materialize('hhkscgmqzmuwltmrhtxnnzsxl'), ['gmqzmuwltmrh', 'qtescwjubeqhurqoqfjauwxdoc', 'uwltmrh', 'qlhyfuspwdtecdbrmrqcnxghhlnbmzs', 'm', 'kcsuocwokvohnqonnfzmeiqtomehksehwc', 'hoxocyilgrxxoek', 'nisnlmbdczjsiw', 'gmqz', '', 'cqzz', 'k', 'utxctwtzelxmtioyqshxedecih', 'ifsmsljxzkyuigdtunwk', 'ojxvxwdosaqjhrnjwisss', 'dz']) from system.numbers limit 10; +select [0, 0, 19, 7, 0, 0, 1, 0, 0, 12, 0, 0, 1, 0, 1, 1, 5, 0, 23, 8] = multiSearchAllPositions(materialize('raxgcqizulxfwivauupqnofbijxfr'), ['sxvhaxlrpviwuinrcebtfepxxkhxxgqu', 'cuodfevkpszuimhymxypktdvicmyxm', 'pqnof', 'i', 'ufpljiniflkctwkwcrsbdhvrvkizticpqkgvq', 'osojyhejhrlhjvqrtobwthjgw', '', 'anzlevtxre', 'ufnpkjvgidirrnpvbsndfnovebdily', 'fwivauupqnofbi', 'rywyadwcvk', 'ltnlhftdfefmkenadahcpxw', '', 'xryluzlhnsqk', 'r', '', 'cqizulxfwivauupqnofb', 'y', 'fb', 'zulxfwivauupqnofbijxf']) from system.numbers limit 10; +select [4, 0, 0, 0, 0, 24, 1, 2, 0, 2, 0, 0, 8, 0] = multiSearchAllPositions(materialize('cwcqyjjodlepauupgobsgrzdvii'), ['q', 'yjppewylsqbnjwnhokzqtauggsjhhhkkkqsy', 'uutltzhjtc', 'pkmuptmzzeqhichaikwbggronli', 'erzgcuxnec', 'dvii', '', 'w', 'fkmpha', 'wcqyjjodlepauupgobsgrz', 'cbnmwirigaf', 'fcumlot', 'odlepauu', 'lthautlklktfukpt']) from system.numbers limit 10; +select [1, 1, 1, 1, 22, 0, 0, 8, 18, 15] = multiSearchAllPositions(materialize('vpscxxibyhvtmrdzrocvdngpb'), ['', '', '', '', 'n', 'agrahemfuhmftacvpnaxkx', 'dqqwvfsrqv', 'byhvtmrdzrocv', 'ocvdn', 'dzrocvdngpb']) from system.numbers limit 10; +select [1, 1, 1, 15, 10, 0, 0, 0, 0, 2] = multiSearchAllPositions(materialize('nfoievsrpvheprosjdsoiz'), ['', 'nfo', '', 'osjd', 'vheprosjdsoiz', 'az', 'blhvdycvjnxaipvxybs', 'umgxmpkvuvuvdaczkz', 'gfspmnzidixcjgjw', 'f']) from system.numbers limit 10; +select [0, 0, 2, 2, 0, 0, 0, 11, 10, 4, 9, 1, 6, 4, 0, 0] = multiSearchAllPositions(materialize('bdmfwdisdlgbcidshnhautsye'), ['uxdceftnmnqpveljer', 'xdnh', 'dmf', 'dmfwdisdlgbc', 'cpwnaijpkpyjgaq', 'doquvlrzhusjbxyqcqxvwr', 'llppnnmtqggyfoxtawnngsiiunvjjxxsufh', 'gbcidshnhau', 'lgbcids', 'f', 'dlgbc', 'bdmfwdisdlgbcids', 'disdlgbcidshnhautsy', 'fwdisdlgbcidshn', 'zfpbfc', 'triqajlyfmxlredivqiambigmge']) from system.numbers limit 10; +select [0, 0, 16, 0, 0, 0, 14, 6, 2, 1, 0, 0, 1, 0, 10, 12, 0, 0, 0, 0] = multiSearchAllPositions(materialize('absimumlxdlxuzpyrunivcb'), ['jglfzroni', 'wzfmtbjlcdxlbpialqjafjwz', 'yrun', 'fgmljkkp', 'nniob', 'fdektoyhxrumiycvkwekphypgti', 'zp', 'um', 'bsimu', '', 'yslsnfisaebuujltpgcskhhqcucdhb', 'xlaphsqgqsfykhilddctrawerneqoigb', '', 'pdvcfxdlurmegspidojt', 'd', 'xu', 'fdp', 'xjrqmybmccjbjtvyvdh', 'nvhdfatqi', 'neubuiykajzcrzdbvpwjhlpdmd']) from system.numbers limit 10; +select [0, 0, 0, 9, 0, 0, 1, 1, 1, 1] = multiSearchAllPositions(materialize('lvyenvktdnylszlypuwqecohy'), ['ihlsiynj', 'ctcnhbkumvbgfdclwjhsswpqyfrx', 'rpgqwkydwlfclcuupoynwrfffogxesvmbj', 'dnyl', 'coeqgdtbemkhgplprfxgwpl', 'dkbshktectbduxlcaptlzspq', 'l', 'lvyenvktdnylszlypuw', 'lvyenvk', '']) from system.numbers limit 10; +select [1, 0, 0, 0, 0, 1, 2, 22, 8, 17, 1, 13, 0, 0, 0, 0, 0, 5] = multiSearchAllPositions(materialize('wphcobonpgaqwgfenotzadgqezx'), ['', 'qeuycfhkfjwokxgrkaodqioaotkepzlhnrv', 'taehtytq', 'gejlcipocalc', 'poyvvvntrvqazixkwigtairjvxkgouiuva', '', 'phc', 'dg', 'npgaqwg', 'notzadgqe', '', 'wgfe', 'smipuxgvntys', 'qhrfdytbfeujzievelffzrv', 'cfmzw', 'hcywnyguzjredwjbqtwyuhtewuhzkc', 'tssfeinoykdauderpjyxtmb', 'obonpgaqwgfen']) from system.numbers limit 10; +select [0, 0, 0, 0, 0, 6, 6, 0, 0, 2, 0, 5, 2, 0, 6, 3] = multiSearchAllPositions(materialize('qvslufpsddtfudzrzlvrzdra'), ['jxsgyzgnjwyd', 'hqhxzhskwivpuqkjheywwfhthm', 'kbwlwadilqhgwlcpxkadkamsnzngms', 'fxunda', 'nlltydufobnfxjyhch', 'fpsddtfudzrzl', 'fp', 'ykhxjyqtvjbykskbejpnmbxpumknqucu', 'iyecekjcbkowdothxc', 'vslufpsddtfu', 'mjgtofkjeknlikrugkfhxlioicevil', 'uf', 'vslufpsdd', 'cxizdzygyu', 'fpsddtfudzrz', 'slufp']) from system.numbers limit 10; +select [12, 0, 0, 0, 0, 1, 6, 0, 1, 2] = multiSearchAllPositions(materialize('ydsbycnifbcforymknzfi'), ['forymkn', 'vgxtcdkfmjhc', 'ymugjvtmtzvghmifolzdihutqoisl', 'fzooddrlhi', 'bdefmxxdepcqi', '', 'cnif', 'ilzbhegpcnkdkooopaguljlie', '', 'dsbycnifbcforym']) from system.numbers limit 10; +select [0, 2, 4, 1, 1, 3, 0, 0, 0, 7] = multiSearchAllPositions(materialize('sksoirfwdhpdyxrkklhc'), ['vuixtegnp', 'ks', 'oirfwdhpd', 'sksoirf', 'skso', 'soi', 'eoxpa', 'vpfmzovgatllf', 'txsezmqvduxbmwu', 'fw']) from system.numbers limit 10; +select [2, 21, 8, 10, 6, 0, 1, 11, 0, 0, 21, 4, 29, 0] = multiSearchAllPositions(materialize('wlkublfclrvgixpbvgliylzbuuoyai'), ['l', 'ylzbuu', 'clr', 'rvgi', 'lf', 'bqtzaqjdfhvgddyaywaiybk', '', 'vgixpbv', 'ponnohwdvrq', 'dqioxovlbvobwkgeghlqxtwre', 'y', 'ublfclrvgix', 'a', 'eoxxbkaawwsdgzfweci']) from system.numbers limit 10; +select [0, 0, 2, 1, 1, 9, 1, 0, 0, 1] = multiSearchAllPositions(materialize('llpbsbgmfiadwvvsciak'), ['knyjtntotuldifbndcpxzsdwdduv', 'lfhofdxavpsiporpdyfziqzcni', 'lpbsbgmf', 'llpbsbgmfi', 'llpbsbgmfiadwvv', 'fia', '', 'uomksovcuhfmztuqwzwchmwvonk', 'ujbasmokvghmredszgwe', '']) from system.numbers limit 10; +select [3, 0, 0, 0, 6, 1, 7, 0, 2, 1, 1, 0, 7, 0, 1, 0, 1, 1, 5, 11] = multiSearchAllPositions(materialize('hnmrouevovxrzrejesigfukkmbiid'), ['m', 'apqlvipphjbui', 'wkepvtnpu', 'amjvdpudkdsddjgsmzhzovnwjrzjirdoxk', 'ue', '', 'evov', 'qoplzddxjejvbmthnplyha', 'nmrouevovxrz', '', 'hnmrouev', 'hnzevrvlmxnjmvhitgdhgd', 'evovxrzrejesig', 'yvlxrjaqdaizishkftgcuikt', '', 'buyrmbkvqukochjteumqchrhxgtmuorsdgzlfn', '', 'hnmrouevov', 'ouevovx', 'xr']) from system.numbers limit 10; +select [0, 13, 0, 0, 0, 0, 0, 14, 0, 0, 1, 12, 0, 1] = multiSearchAllPositions(materialize('uwfgpemgdjimotxuxrxxoynxoaw'), ['uzcevfdfy', 'otxuxrxxoynxoa', 'xeduvwhrogxccwhnzkiolksry', 'pxdszcyzxlrvkymhomz', 'vhsacxoaymycvcevuujpvozsqklahstmvgt', 'zydsajykft', 'vdvqynfhlhoilkhjjkcehnpmwgdtfkspk', 'txuxrx', 'slcaryelankprkeyzaucfhe', 'iocwevqwpkbrbqvddaob', 'uwfg', 'motxuxrxx', 'kpzbg', '']) from system.numbers limit 10; +select [1, 1, 0, 6, 6, 0, 0, 0, 8, 0, 8, 14, 1, 5, 6, 0, 0, 1] = multiSearchAllPositions(materialize('epudevopgooprmhqzjdvjvqm'), ['ep', 'epudevopg', 'tlyinfnhputxggivtyxgtupzs', 'vopgoop', 'v', 'hjfcoemfk', 'zjyhmybeuzxkuwaxtcut', 'txrxzndoxyzgnzepjzagc', 'pgooprmhqzj', 'wmtqcbsofbe', 'pgo', 'm', '', 'evopgooprmhqzjdv', 'vopgooprmhqzjdv', 'gmvqubpsnvrabixk', 'wjevqrrywloomnpsjbuybhkhzdeamj', '']) from system.numbers limit 10; +select [15, 4, 4, 0, 0, 1, 1, 0, 0, 0, 0, 20, 0, 10, 1, 1, 0, 2, 4, 3] = multiSearchAllPositions(materialize('uogsfbdefogwnekfoeobtkrgiceksz'), ['kfoeobtkrgice', 'sfbd', 'sfbdefogwn', 'zwtenhiqavmqoolkvjiqjfb', 'vnjkshyvpwhrauackplqllakcjyamvsuokrxbfv', 'uog', '', 'qtzuhdcdymytgtscvzlzswdlrqidreuuuqk', 'vlridmjlbxyiljpgxsctzygzyawqqysf', 'xsnkwyrmjaaaryvrdgtoshdxpvgsjjrov', 'fanchgljgwosfamgscuuriwospheze', 'btkrgicek', 'ohsclekvizgfoatxybxbjoxpsd', 'ogwnekfoeobtkr', '', '', 'vtzcobbhadfwubkcd', 'og', 's', 'gs']) from system.numbers limit 10; +select [0, 0, 5, 1, 0, 5, 1, 6, 0, 1, 9, 0, 1, 1] = multiSearchAllPositions(materialize('aoiqztelubikzmxchloa'), ['blc', 'p', 'ztelubikzmxchlo', 'aoiqztelubi', 'uckqledkyfboolq', 'ztelubikzmxch', 'a', 'telubikzm', 'powokpdraslpadpwvrqpbb', 'aoiqztelu', 'u', 'kishbitagsxnhyyswn', '', '']) from system.numbers limit 10; +select [5, 11, 0, 0, 0, 5, 0, 0, 0, 1, 16, 0, 0, 0, 0, 0] = multiSearchAllPositions(materialize('egxmimubhidowgnfziwgnlqiw'), ['imubhidowgnfzi', 'dowgnf', 'yqpcpfvnfpxetozraxbmzxxcvtzm', 'xkbaqvzlqjyjoiqourezbzwaqkfyekcfie', 'jjctusdmxr', 'imubhi', 'zawnslbfrtqohnztmnssxscymonlhkitq', 'oxcitennfpuoptwrlmc', 'ac', 'egxmi', 'fziwgn', 'rt', 'fuxfuctdmawmhxxxg', 'suulqkrsfgynruygjckrmizsksjcfwath', 'slgsq', 'zcbqjpehilwyztumebmdrsl']) from system.numbers limit 10; +select [20, 0, 9, 0, 0, 14, 0, 5, 8, 3, 0, 0, 0, 4] = multiSearchAllPositions(materialize('zczprzdcvcqzqdnhubyoblg'), ['obl', 'lzrjyezgqqoiydn', 'vc', 'nbvwfpmqlziedob', 'pnezljnnujjbyviqsdpaqkkrlogeht', 'dn', 'irvgeaq', 'rzdcvcqzqdnh', 'cvcqzqdnh', 'zprzdcv', 'wvvgoexuevmqjeqavsianoviubfixdpe', 'aeavhqipsvfkcynyrtlxwpegwqmnd', 'blckyiacwgfaoarfkptwcei', 'prz']) from system.numbers limit 10; +select [2, 1, 1, 9, 10, 5, 0, 0, 0, 2, 9, 7, 9, 0, 1, 9, 7, 0] = multiSearchAllPositions(materialize('mvovpvuhjwdzjwojcxxrbxy'), ['vo', '', '', 'jwdz', 'wdzj', 'pvu', 'ocxprubxhjnji', 'phzfbtacrg', 'jguuqhhxbrwbo', 'vovpvuhjwd', 'jw', 'u', 'jwdzjwojcx', 'nlwfvolaklizslylbvcgicbjw', '', 'jwd', 'uhjwdz', 'bbcsuvtru']) from system.numbers limit 10; +select [2, 0, 21, 0, 0, 0, 3, 0, 0, 0, 0, 10, 1, 18] = multiSearchAllPositions(materialize('nmdkwvafhcbipwoqtsrzitwxsnabwf'), ['m', 'ohlfouwyucostahqlwlbkjgmdhdyagnihtmlt', 'itwx', 'jjkyhungzqqyzxrq', 'abkqvxxpu', 'lvzgnaxzctaarxuqowcski', 'dkwvafhcb', 'xuxjexmeeqvyjmpznpdmcn', 'vklvpoaakfnhtkprnijihxdbbhbllnz', 'fpcdgmcrwmdbflnijjmljlhtkszkocnafzaubtxp', 'hmysdmmhnebmhpjrrqpjdqsgeuutsj', 'cbipwoqtsrzitwxsna', 'nm', 'srzitwx']) from system.numbers limit 10; +select [17, 5, 0, 13, 0, 0, 10, 1, 0, 19, 10, 8, 0, 4] = multiSearchAllPositions(materialize('gfvndbztroigxfujasvcdgfbh'), ['asvcdgf', 'dbztroigxfujas', 'pr', 'xfujas', 'nxwdmqsobxgm', 'wdvoepclqfhy', 'oigxfu', '', 'flgcghcfeiqvhvqiriciywbkhrxraxvneu', 'vcd', 'oigxfu', 'troigxfuj', 'gbnyvjhptuehkefhwjo', 'ndbz']) from system.numbers limit 10; +select [0, 14, 1, 0, 0, 1, 1, 11, 0, 8, 6, 0, 3, 19, 7, 0] = multiSearchAllPositions(materialize('nofwsbvvzgijgskbqjwyjmtfdogzzo'), ['kthjocfzvys', 'skbqjwyjmtfdo', 'nof', 'mfapvffuhueofutby', 'vqmkgjldhqohipgecie', 'nofwsbv', '', 'ijgs', 'telzjcbsloysamquwsoaso', 'vzgijgskbqjwyjmt', 'bvvzgijgskbqjwyjmtfd', 'hdlvuoylcmoicsejofcgnvddx', 'fwsbvvzgijgskb', 'wyjm', 'vvzgijg', 'fwzysuvkjtdiufetvlfwf']) from system.numbers limit 10; +select [10, 2, 13, 0, 0, 0, 2, 0, 9, 2, 4, 1, 1, 0, 1, 6] = multiSearchAllPositions(materialize('litdbgdtgtbkyflsvpjbqwsg'), ['tbky', 'itdbgdtgtb', 'yflsvpjb', 'ikbylslpoqxeqoqurbdehlroympy', 'hxejlgsbthvjalqjybc', 'sontq', 'itdbgd', 'ozqwgcjqmqqlkiaqppitsvjztwkh', 'gtbkyf', 'itdbgdtgtbkyfls', 'dbg', 'litdb', '', 'qesbakrnkbtfvwu', 'litd', 'g']) from system.numbers limit 10; +select [0, 0, 1, 1, 5, 0, 8, 12, 0, 2, 0, 7, 0, 6] = multiSearchAllPositions(materialize('ijzojxumpvcxwgekqimrkomvuzl'), ['xirqhjqibnirldvbfsb', 'htckarpuctrasdxoosutyxqioizsnzi', '', '', 'jxu', 'dskssv', 'mpvcxwgekqi', 'xwgek', 'qsuexmzfcxlrhkvlzwceqxfkyzogpoku', 'jzojx', 'carjpqihtpjniqz', 'umpvcxwgekq', 'krpkzzrxxtvfhdopjpqcyxfnbas', 'xumpvcxwg']) from system.numbers limit 10; +select [0, 0, 0, 6, 0, 8, 0, 2, 0, 0, 0, 0, 14, 0, 0, 1, 1, 0, 0, 0] = multiSearchAllPositions(materialize('zpplelzzxsjwktedrrtqhfmoufv'), ['jzzlntsokwlm', 'cb', 'wuxotyiegupflu', 'lzzxsjwkte', 'owbxgndpcmfuizpcduvucnntgryn', 'zxsjwktedrrtqhf', 'kystlupelnmormqmqclgjakfwnyt', 'pple', 'lishqmxa', 'mulwlrbizkmtbved', 'uchtfzizjiooetgjfydhmzbtmqsyhayd', 'hrzgjifkinwyxnazokuhicvloaygeinpd', 'tedrrt', 'shntwxsuxux', 'evrjehtdzzoxkismtfnqp', 'z', '', 'nxtybut', 'vfdchgqclhxpqpmitppysbvxepzhxv', 'wxmvmvjlrrehwylgqhpehzotgrzkgi']) from system.numbers limit 10; + +select [15, 19, 0, 0, 15, 0, 0, 1, 2, 6] = multiSearchAllPositionsUTF8(materialize('зжерхмчсйирдчрришкраоддцфгх'), ['ришкра', 'раоддц', 'фттиалусгоцжлтщзвумрдчи', 'влййи', 'ришкра', 'цгфжуцгивй', 'ккгжхрггчфглх', 'з', 'жерхмчсйи', 'мчсйирдчрришкраоддц']) from system.numbers limit 10; +select [0, 0, 0, 1, 4, 0, 14, 0, 1, 8, 8, 9, 0, 0, 4, 0] = multiSearchAllPositionsUTF8(materialize('етвхйчдобкчукхпщлмжпфайтфдоизщ'), ['амфшужперосрфщфлижйййжжжй', 'ххкбщшзлмщггтшцпсдйкдшйвхскемц', 'ергйплгпнглккшкарещимгапхг', '', 'хйчдо', 'вввбжовшзйбгуоиждепйабаххеквщижтйиухос', 'хпщл', 'жфуомщуххнедзхищнгхрквлпмзауеегз', 'етвхй', 'о', 'о', 'бк', 'цфецккифж', 'аизлокл', 'х', 'слщгеивлевбчнчбтшгфмжрфка']) from system.numbers limit 10; +select [0, 0, 1, 2, 0, 0, 14, 0, 3, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('йбемооабурнирйофшдгпснж'), ['гпфцл', 'нчбперпмцкввдчсщвзйрдфнф', '', 'бем', 'ч', 'жгш', 'йофшдгпснж', 'шасгафчг', 'емооабур', 'пиохцжццгппщчопзйлмуотз', 'рпдомнфвопхкшешйишумбацтл', 'нисиийфррбдоц']) from system.numbers limit 10; +select [1, 18, 12, 0, 0, 1, 1, 3, 7, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('гсщнфийтфзжцйпфбйалущ'), ['', 'алущ', 'цйпфбйал', 'цвбфцйвсвлицсчнргпцнр', 'х', 'гс', '', 'щн', 'й', 'дгйрвцщтп', 'уитвквоффвцхфишрлерйцувф', 'кфтййлпнзжчижвглзкижн']) from system.numbers limit 10; +select [14, 0, 5, 5, 0, 6, 0, 16, 0, 0] = multiSearchAllPositionsUTF8(materialize('ефщнйнуйебнснлрцгкеитбг'), ['лрцгкеитб', 'епклжфцпнфопе', 'йнуйебн', 'й', 'тлт', 'нуйебнснлрцгкеит', 'глечршгвотумкимтлм', 'цгк', 'щгйчой', 'звкцкчк']) from system.numbers limit 10; +select [0, 1, 18, 6, 0, 3, 0, 0, 25, 0, 0, 1, 16, 5, 1, 7, 0, 0] = multiSearchAllPositionsUTF8(materialize('пумгмцшмжштсшлачсжарерфиозиг'), ['чсуубфийемквмоотванухмбрфхжоест', '', 'жар', 'цшмжш', 'жртещтинтвпочнкдткцза', 'м', 'адзгтбаскщгдшжл', 'штфжшллезпджигщфлезфгзчайанхктицштйй', 'о', 'етадаарйсцейдошшцечхзлшлрртсрггцртспд', 'зтвшалрпфлщбцд', 'пу', 'ч', 'мцшмжштсшлачсж', '', 'шмжшт', 'ещтжшйтчзчаноемрбц', 'тевбусешйрйчшзо']) from system.numbers limit 10; +select [7, 10, 0, 0, 0, 0, 1, 12, 9, 2, 0, 0, 0, 4, 1, 1, 0, 6] = multiSearchAllPositionsUTF8(materialize('дупгвндвйжмаузнллнзл'), ['двйжмаузн', 'жмаузнлл', 'емйжркоблновцгпезрдавкбелцщста', 'щзкгм', 'лебрпцрсутшриащгайвц', 'лзнмл', 'д', 'ауз', 'йжмау', 'упгвндвйж', 'жщсббфвихг', 'всигсеигцбгаелтчкирлнзшзцжещнс', 'рмшиеиесрлщципщхкхтоцщчйоо', 'гвн', '', '', 'йадеоцлпшпвщзещзкхйрейопмажбб', 'ндв']) from system.numbers limit 10; +select [0, 0, 0, 8, 3, 10, 22, 0, 13, 11, 0, 1, 18, 0, 1, 0] = multiSearchAllPositionsUTF8(materialize('жшзфппавввслфцлнщшопкдшку'), ['саоткнхфодзаа', 'кйхванкзаисйбврщве', 'бчоуучватхфукчф', 'вввслфц', 'з', 'вслфцлнщшопк', 'дшк', 'из', 'фцл', 'с', 'зртмцтпощпщхк', 'жшзфппавввслфц', 'шопк', 'збтхрсдтатхпрзлхдооощифачхчфн', '', 'жщшийугз']) from system.numbers limit 10; +select [2, 4, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 21, 0, 6, 0] = multiSearchAllPositionsUTF8(materialize('пчботухвгдчекмжндбоожш'), ['чботухвгдчекмжндб', 'от', 'гвсжжйлбтщчучнхсмдйни', 'жцжзмшлибшефуоуомпацбщщу', 'онхфлуцйлхтбмц', 'йтепжу', 'хтдрпвкщрли', 'аддайф', 'нхегщккбфедндоацкиз', 'йгкцзртфжгв', 'буелрщмхйохгибжндфшщвшрлдччрмфмс', 'цщцтзфнщ', 'уч', 'пчб', 'жш', 'пнфббтшйгхйрочнлксщпгвжтч', 'ухвг', 'лсцппузазщрйхймщбзоршощбзленхп']) from system.numbers limit 10; +select [0, 0, 4, 11, 0, 0, 0, 0, 0, 11, 2, 4, 6, 0, 0, 1, 2, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('тжрмчпваухрхуфбгнифгбопфт'), ['дпмгкекщлнемссаицщпащтиуцхкфчихтц', 'акйиуоатунтчф', 'мчпва', 'рхуфбгнифгб', 'кнаишж', 'пчвотенеафкухжцешбцхг', 'опеа', 'ушчадфтчхечеуркбтел', 'ашшптаударчжчмвалтдхкимищпф', 'рхуфбгниф', 'ж', 'мчпваухрхуфбгнифг', 'пваухрху', 'зргачбтцдахвймсбсврбндзтнущхвп', 'асбфцавбгуолг', 'тж', 'жрмчпваухрх', 'мрвзцгоб', 'чрцснчсдхтзжвнздзфцвхеилишдбж', 'кчт']) from system.numbers limit 10; +select [0, 2, 4, 0, 6, 0, 0, 0, 0, 19, 7, 1, 0, 1, 0, 0, 2, 10, 0, 1] = multiSearchAllPositionsUTF8(materialize('опрурпгабеарушиойцрхвбнсщ'), ['йошуоесдщеж', 'пр', 'урпгабеарушиой', 'хщиаршблашфажщметчзи', 'пгабеарушиойцрхвб', 'щцбдвц', 'еечрззвкожзсдурйщувмцйшихдц', 'офхачгсзашфзозрлба', 'айдфжджшжлрргмабапткбцпиизигдтс', 'рх', 'габ', '', 'цнкдбфчщшмчулврбцчакщвзхлазфа', '', 'екбтфпфилсаванхфкмчнпумехиищди', 'епвщхаклшомвцжбф', 'прурпгабе', 'еарушиойцрхв', 'црвтгрзтитц', 'опрурпг']) from system.numbers limit 10; +select [0, 10, 1, 0, 0, 0, 0, 0, 10, 0, 15, 2] = multiSearchAllPositionsUTF8(materialize('угпщлзчжшбзвууцшатпщцр'), ['цоуарцжсз', 'бз', '', 'пщфтзрч', 'лфуипмсдмнхнгйнтк', 'айжунцйбйцасчфдхй', 'щдфщлцптплсачв', 'грв', 'бзвууц', 'бумййшдшфашцгзфвчвзвтсувнжс', 'цшатпщ', 'гпщлзчжшб']) from system.numbers limit 10; +select [0, 15, 0, 1, 5, 0, 0, 5, 0, 0, 0, 1, 0, 0] = multiSearchAllPositionsUTF8(materialize('цнлеодлмдцдйснитвдчтхжизв'), ['ивкчсзшугоцжчохщцабл', 'итвдчт', 'кнх', '', 'одлм', 'ктшфзбщзцуймагсоукщщудвуфо', 'ххеаефудгчхр', 'одлмдцдйснитвдчт', 'умцлпкв', 'зщсокйтцзачщафвбповжгнлавсгйг', 'бкибм', '', 'охсоихнцчцшевчеележтука', 'фаийхгжнсгищгщц']) from system.numbers limit 10; +select [0, 0, 0, 2, 0, 0, 0, 0, 3, 2, 3, 6, 0, 0, 0, 12, 4, 1] = multiSearchAllPositionsUTF8(materialize('бгдбувдужщвоошлтчрбй'), ['щвбаиф', 'итчднесжкчжвпжйвл', 'мм', 'г', 'хктзгтзазфгщшфгбеулцмдмдбдпчзх', 'сфуак', 'злйфцощегзекщб', 'фшлдтолрщфзжчмих', 'дбувдужщ', 'гдб', 'дбувдужщ', 'в', 'лчищкечнжщисцичбнзшмулпмлп', 'чжцсгмгфвлиецахзнрбмщин', 'обпжвй', 'о', 'бувдужщвоош', '']) from system.numbers limit 10; +select [0, 2, 5, 3, 2, 0, 1, 0, 0, 4, 2, 0, 0, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('шсушлорзфжзудбсейенм'), ['чнзпбновтршеумбвщчлх', 'су', 'лорзфж', 'ушлорзфжзудб', 'сушлорзфжзудбсейенм', 'ткдрхфнб', '', 'пщд', 'чбдцмщ', 'шлорзфж', 'су', 'сккигркедчожжемгнайвйчтдмхлтти', 'мц', 'пхнхрхйцйсйбхчлктз', 'иафжстлйфцр', 'алщщлангнбнйхлшлфшйонщек']) from system.numbers limit 10; +select [12, 1, 0, 5, 0, 10, 1, 0, 7, 4, 0, 1, 12, 1, 1, 1, 0, 1, 15, 0] = multiSearchAllPositionsUTF8(materialize('ощзллчубоггцвжриуардрулащйпу'), ['цвжр', '', 'нмзкаиудзтиффззшзканжвулт', 'лчубоггцвжриуардрулащйпу', 'чтцлзшуижолибаоххвшихбфжйхетивп', 'ггцвжри', '', 'йдгнвс', 'у', 'л', 'зпщнжуойдлдвхокцжнзйсйзе', '', 'цв', '', '', '', 'ехлцзгвф', '', 'риу', 'уйжгтжноомонгщ']) from system.numbers limit 10; +select [0, 12, 13, 20, 0, 1, 0, 0, 3, 4] = multiSearchAllPositionsUTF8(materialize('цбкифйтшузажопнжщарбштвдерзтдш'), ['щлмлижтншчсмксгтнсврро', 'жопнжщарбштвд', 'опнжщарб', 'бштвдерзтд', 'пуфслейщбкжмпнш', 'ц', 'маве', 'кмйхойрдлшцхишдтищвйбцкщуигваещгтнхйц', 'кифй', 'и']) from system.numbers limit 10; +select [0, 6, 0, 0, 0, 8, 0, 3, 6, 0] = multiSearchAllPositionsUTF8(materialize('еачачгбмомоххкгнвштггпчудл'), ['ндзчфчвжтцщпхщуккбеф', 'г', 'рткнфвчтфннхлжфцкгштймгмейжй', 'йчннбщфкщф', 'лсртщиндшшкичзррущвдйвнаркмешерв', 'момоххк', 'рфафчмсизлрхзуа', 'ч', 'гбмомоххкгнвштг', 'валжпошзбгзлвевчнтз']) from system.numbers limit 10; +select [0, 0, 10, 0, 8, 13, 0, 0, 19, 15, 3, 1] = multiSearchAllPositionsUTF8(materialize('зокимчгхухшкшмтшцчффвззкалпва'), ['цалфжажщщширнрвтпвмщжннрагвойм', 'оукзрдцсадешжмз', 'хшкшмтшцч', 'ауилтсаомуркпаркбцркугм', 'хухшкшмтшцчффв', 'шмтшцч', 'зщгшпцхзгцншднпеусмтжбцшч', 'щлраащсйлщрд', 'ффвзз', 'тшцчффвззкалпв', 'кимчгхухшкш', '']) from system.numbers limit 10; +select [0, 0, 1, 0, 6, 0, 6, 0, 5, 0, 13, 0, 0, 6] = multiSearchAllPositionsUTF8(materialize('йдйндиибщекгтчбфйдредпхв'), ['тдршвтцихцичощнцницшдхйбогбчубие', 'акппакуцйсхцдххнотлгирввоу', '', 'улщвзхохблтксчтб', 'и', 'ибейзчшклепзриж', 'иибщекгт', 'шидббеухчпшусцнрз', 'диибщекгтчбфйд', 'дейуонечзйзлдкшщрцйбйклччсцуй', 'тч', 'лшицлшме', 'чйнжчоейасмрщегтхвйвеевбма', 'ии']) from system.numbers limit 10; +select [15, 3, 3, 2, 0, 11, 0, 0, 0, 2, 0, 4, 0, 1, 1, 3, 0, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('нхгбфчшджсвхлкхфвтдтлж'), ['хфвтдтлж', 'гбфчшд', 'гбфчш', 'х', 'ачдгбккжра', 'вхлк', 'мщчвещлвшдщпдиимлшрвнщнфсзгщм', 'жчоббгшзщлгеепщжкчецумегпйчт', 'жжд', 'хг', 'мтсааролшгмоуйфйгщгтрв', 'бфчшд', 'чейрбтофпшишгуасоодлакчдф', 'н', 'нхгбфч', 'гбф', 'гдежсх', 'йифжацзгжбклх', 'ещпзущпбаолплвевфиаибшйубйцсзгт', 'жезгчжатзтучжб']) from system.numbers limit 10; +select [0, 10, 1, 0, 0, 0, 4, 0, 13, 1, 12, 1, 0, 6] = multiSearchAllPositionsUTF8(materialize('акбдестрдшерунпвойзв'), ['нркчх', 'шерунп', '', 'зжвахслфббтоиоцрзаззасгнфчх', 'шлжмдг', 'тлйайвцжчсфтцйрчосмижт', 'дестрдшерунп', 'мвамйшцбдщпчлрщд', 'у', 'акбдестрд', 'рунпвойз', '', 'айздцоилсйшцфнчтхбн', 'с']) from system.numbers limit 10; +select [1, 0, 0, 3, 2, 1, 0, 0, 1, 10, 7, 0, 5, 0, 8, 4, 1, 0, 8, 1] = multiSearchAllPositionsUTF8(materialize('кйхпукаеуддтйччхлнпсуклрф'), ['кйхпукаеуддтйччхл', 'йатлрйкстлхфхз', 'фгихслшкж', 'хпу', 'йхпукаеу', '', 'сруакбфоа', 'оажуз', 'кйхпукаеуддтйччх', 'ддтйччхлн', 'аеуддтйччхл', 'тмажиойщтпуцглхфишеиф', 'укаеуддтйччхлнпс', 'ретифе', 'еуддтйччхлнпсуклр', 'пукаеуд', 'кйхпу', 'таппфггвджлцпжшпишбпциуохсцх', 'еуд', '']) from system.numbers limit 10; +select [2, 3, 3, 16, 5, 13, 0, 0, 0, 18, 0, 6, 0, 16, 0, 10, 3, 0] = multiSearchAllPositionsUTF8(materialize('плврйщовкзнбзлбжнсатрцщщучтйач'), ['лврйщовкзнбзлбж', 'врйщовкзнбзлбжнса', 'врйщовкзнбз', 'жнсатрцщщучтйач', 'йщовкзнбзлбжнсатрцщщуч', 'злбжнсатрцщ', 'ввтбрдт', 'нжйапойг', 'ннцппгперхйвдхоеожупйебочуежбвб', 'сатрцщщу', 'деваийтна', 'щ', 'вкжйгкужжгтевлцм', 'жнс', 'датг', 'знбзлбжнсатрцщщучтйа', 'врйщовк', 'оашмкгчдзщефм']) from system.numbers limit 10; +select [3, 1, 19, 1, 0, 0, 0, 0, 11, 3, 0, 0] = multiSearchAllPositionsUTF8(materialize('фчдеахвщжхутхрккхасвсхепщ'), ['деах', '', 'свсхепщ', '', 'анчнсржйоарвтщмрж', 'нечбтшщвркгд', 'вштчцгшж', 'з', 'у', 'деахвщ', 'ххкцжрвзкжзжчугнфцшуиаклтмц', 'фцкжшо']) from system.numbers limit 10; +select [16, 0, 0, 1, 8, 14, 0, 12, 12, 5, 0, 0, 16, 0, 11, 0] = multiSearchAllPositionsUTF8(materialize('щмнжчввбжцчммчшсрхйшбктш'), ['срхйшбк', 'йлзцнржчууочвселцхоучмщфчмнфос', 'еижлафатшхщгшейххжтубзвшпгзмзцод', '', 'бжцчммчшсрхй', 'чшсрхй', 'влемчммйтителщвзган', 'ммч', 'ммчшсрх', 'чввбж', 'нобзжучшошмбщешлхжфгдхлпнгпопип', 'цгт', 'срхйш', 'лкклмйжтеа', 'чммчшсрхйшбктш', 'йежффзнфтнжхфедгбоахпг']) from system.numbers limit 10; +select [1, 12, 9, 5, 1, 0, 6, 3, 0, 1] = multiSearchAllPositionsUTF8(materialize('кжнщсашдзитдмщцхуоебтфжл'), ['', 'дмщцхуоебт', 'зитдмщцхуоебт', 'сашдзитдмщцхуое', 'кжнщ', 'тхкйтшебчигбтмглшеужззоббдилмдм', 'ашдзитдмщцхуоебтф', 'нщсашдз', 'аузщшр', 'кжнщсашдз']) from system.numbers limit 10; +select [2, 0, 0, 0, 1, 0, 2, 0, 0, 17, 0, 8, 7, 14, 0, 0, 0, 7, 9, 23] = multiSearchAllPositionsUTF8(materialize('закуфгхчтшивзчжаппбжнтслщввущ'), ['а', 'днойвхфрммтж', 'внтлжрхзрпчбтуркшдатннглечг', 'ахиеушжтфкгцщтзхмжнрхдшт', '', 'тцчгрззржмдшйщфдцрбшжеичч', 'а', 'ктиечцпршнфнбчуолипацчдсосцнлфаццм', 'аусрлхдцегферуо', 'ппбжнт', 'жкццуосгвп', 'чтшивзчжаппб', 'хчтшивзчжаппб', 'чжаппбжнтслщ', 'ччрлфдмлу', 'щзршффбфчзо', 'ущуймшддннрхзийлваежщухч', 'хчтши', 'тшивзчжаппбжнтсл', 'слщв']) from system.numbers limit 10; +select [1, 1, 9, 2, 0, 3, 7, 0, 0, 19, 2, 2, 0, 8] = multiSearchAllPositionsUTF8(materialize('мвкзккупнокченйнзкшбдрай'), ['м', '', 'н', 'вкз', 'гдпертшйбртотунур', 'к', 'упнокченйнзкшбдр', 'нфшрг', 'нмждрйббдцлйемжпулдвкещхтжч', 'ш', 'вкзккупнокченйнзкшбдр', 'вкзккупнокченйнзкшбдрай', 'адииксвеавогтйторчтцвемвойшпгбнз', 'пнокченй']) from system.numbers limit 10; +select [15, 0, 0, 1, 12, 1, 0, 0, 1, 11, 0, 4, 0, 2] = multiSearchAllPositionsUTF8(materialize('отарлшпсабждфалпшножид'), ['лпшно', 'вт', 'лпжшосндутхорлиифжаакш', 'отарлшпсабждфалпшнож', 'дфал', '', 'бкцжучншжбгзжхщпзхирртнбийбтж', 'уцвцкшдзревпршурбсвйнемоетчс', '', 'ждфал', 'тлскхрнпмойчбцпфущфгф', 'рлшпсабж', 'нхнмк', 'тарлшпса']) from system.numbers limit 10; +select [0, 2, 0, 20, 0, 17, 18, 0, 1, 1, 21, 1, 0, 1, 6, 26] = multiSearchAllPositionsUTF8(materialize('ачйвцштвобижнзжнчбппйеабтцнйн'), ['сзхшзпетншйисщкшрвйшжуогцвбл', 'чйвцштво', 'евз', 'пй', 'хуждапрахитйажрищуллйзвчт', 'чбппйе', 'бппйеабтцнйн', 'схш', 'а', 'ачйвцштвобижнзжнчбпп', 'йеабтцнй', '', 'ег', '', 'штвобижнзжнчбпп', 'цн']) from system.numbers limit 10; +select [1, 0, 0, 3, 4, 12, 0, 9, 0, 12, 0, 0, 8, 0, 10, 3, 4, 1, 1, 9] = multiSearchAllPositionsUTF8(materialize('жмхоужежйуфцзеусеоднчкечфмемба'), ['', 'идосйксзнщйервосогф', 'тхмсйлвкул', 'хоужежйуф', 'оужежйуфцзеусеоднчкечфм', 'цзеусеоднчкеч', 'бецвдиубххвхйкажуурщщшщфбзххт', 'йуфцзеусеодн', 'мглкфтуеайсржисстнпкгебфцпа', 'цзеусео', 'уехцфучецчгшйиржтсмгхакчшввохочжпухс', 'дчвмсбткзталшбу', 'жйуфцзеусеоднчке', 'ччшщтдбпвчд', 'уфцзеусеоднчкечфмем', 'хоужежйуфцзеусеоднчкечф', 'оуже', '', 'жмхоужежйуфцзеу', 'й']) from system.numbers limit 10; +select [0, 0, 0, 3, 0, 0, 0, 0, 1, 0, 1, 0, 1, 2, 0, 0, 0, 6] = multiSearchAllPositionsUTF8(materialize('лшпцхкмтресзпзйвцфрз'), ['енрнцепацлщлблкццжсч', 'ецжужлуфаееоггрчохпчн', 'зхзнгасхебнаейбддсфб', 'пцхкмтресзпзйв', 'фчетгеодщтавиииухцундпнхлчте', 'шшгсдошкфлгдвкурбуохзчзучбжйк', 'мцщщцп', 'рх', '', 'зйошвщцгхбж', '', 'ввлпнамуцвлпзеух', '', 'шпцхкмтре', 'маабтруздрфйпзшлсжшгож', 'фдчптишмштссщшдшгх', 'оллохфпкаем', 'кмтресзпз']) from system.numbers limit 10; +select [2, 5, 0, 0, 6, 0, 0, 0, 0, 0, 0, 0, 1, 1, 12, 0, 0, 0, 4, 8] = multiSearchAllPositionsUTF8(materialize('есипзсвшемлхчзмйрсфз'), ['с', 'з', 'пщчсмаиахппферзжбпвиибаачй', 'гтщкзоиежав', 'свшемлхчзм', 'шийанбке', 'зхе', 'авркудфаусзквкфффйцпзжщввенттб', 'ножцваушапиж', 'иизкежлщиафицкчщмалнпсащсднкс', 'вчмв', 'кщеурмуужжлшррце', '', '', 'х', 'алзебзпчеложихашжвхмйхрицн', 'тпзмумчшдпицпдшиаог', 'сулксфчоштаййзбзшкджббщшсей', 'пзсвшемлхчзм', 'ш']) from system.numbers limit 10; +select [0, 1, 2, 4, 0, 0, 14, 1, 13, 4, 0, 0, 1, 1] = multiSearchAllPositionsUTF8(materialize('сзиимонзффичвфжоеулсадону'), ['зфтшебтршхддмеесчд', '', 'зиимонзф', 'имон', 'езбдйшжичценлгршщшаумайаицй', 'птпщемтбмднацлг', 'фжоеулса', '', 'вфжоеулсадону', 'имонзфф', 'йщвдфдиркважгйджгжашарчучйххйднпт', 'дй', '', '']) from system.numbers limit 10; +select [12, 0, 24, 0, 9, 0, 1, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('ижсщщрзжфнгццпзкфбвезгбохлж'), ['ццпзкфбвез', 'ацррвхоптаоснулнжкщжел', 'охлж', 'тнсхбпшщнб', 'фнг', 'урйвг', '', 'цохс', 'щбйрйкжчмйзачуефч', 'афа']) from system.numbers limit 10; +select [9, 0, 0, 0, 1, 0, 7, 7, 0, 0, 1, 0, 7, 0, 0, 8, 0, 3, 0, 0] = multiSearchAllPositionsUTF8(materialize('рерфвирачйнашхрмцебфдйааеммд'), ['чйнашхрмцебфдйааеммд', 'сжщзснвкущлжплцзлизаомдизцнжлмййбохрцч', 'еппбжджмримфчйеаолидпцруоовх', 'едтжкоийггснехшсчйлвфбкцжжрчтш', '', 'пжахфднхсотй', 'ра', 'рач', 'вчримуцнхбкуйжрвфиугзфсзг', 'кщфехрххциаашщсифвашгйцвхевцщнйахтбпжщ', '', 'ртщиобчжстовйчфабалзц', 'рачйнашхрмцебфдйаае', 'ощгжосччфкуг', 'гехвжнщжссидмрфчйтнепдсртбажм', 'а', 'ицжлсрсиатевбвнжрдмзцувввтзцфтвгвш', 'рф', 'прсмлча', 'ндлхшцааурмзфгверуфниац']) from system.numbers limit 10; +select [2, 14, 10, 0, 6, 15, 1, 0, 0, 4, 5, 17, 0, 0, 3, 0, 3, 0, 9, 0] = multiSearchAllPositionsUTF8(materialize('влфощсшкщумчллфшшвбшинфппкчуи'), ['лфощ', 'лфшшвбшинфпп', 'умчллфшшвбшинф', 'слмтнг', 'сшкщумчллфшшвбшинф', 'фшшвб', '', 'рчфбчййсффнодцтнтнбцмолф', 'щфнщокхжккшкудлцжрлжкнп', 'ощ', 'щсшкщумчлл', 'швбшинфппкч', 'септзкщотишсехийлоцчапщжшжсфмщхсацг', 'нт', 'фощсшкщумчллфшшвбшинфп', 'нщпдш', 'фощс', 'мивсмча', 'щумч', 'щчйнткжпмгавфтйтибпхх']) from system.numbers limit 10; +select [0, 10, 0, 0, 0, 0, 0, 3, 0, 0, 0, 2, 0, 11, 0, 0] = multiSearchAllPositionsUTF8(materialize('еаиалмзхцгфунфеагшчцд'), ['йнш', 'гфун', 'жлйудмхнсвфхсуедспщбтутс', 'елмуийгдйучшфлтхцппамфклйг', 'евйдецц', 'пчтфцоучфбсйщпвдацмчриуцжлтжк', 'нстмпумчспцвцмахб', 'иалмз', 'зифчп', 'чогфщимоопт', 'фдйблзеп', 'аиа', 'щугмзужзлйдктш', 'фунфеагшч', 'нйхшмсгцфжчхжвхгдхцуппдц', 'асмвмтнрейшгардллмсрзгзфйи']) from system.numbers limit 10; +select [23, 0, 8, 0, 0, 0, 0, 0, 0, 4, 0, 5, 7, 1, 9, 4] = multiSearchAllPositionsUTF8(materialize('зузйфзлхходфрхгтбпржшрктпйхеоп'), ['ктпйхео', 'лжитуддикчсмкглдфнзцроцбзтсугпвмхзллжж', 'х', 'меуфтено', 'фтдшбшрпоцедктсийка', 'кхтоомтбчвеонксабшйптаихжбтирпзшймчемжим', 'чиаущлрдкухцрдумсвивпафгмр', 'фрнпродв', 'тдгтишхйсашвмдгкчбмшн', 'йфзлхходфрхгтбпржшр', 'бежшлрйврзмумеуооплкицхлйажвцчнчсеакм', 'ф', 'лхходфрхгтб', '', 'ходфрхгтбпржшр', 'й']) from system.numbers limit 10; +select [0, 0, 0, 1, 0, 1, 22, 1, 0, 0, 0, 0, 18, 1, 0, 0, 0, 1] = multiSearchAllPositionsUTF8(materialize('чфгвчхчпщазтгмбнплдгщикойчднж'), ['мштцгтмблаезочкхзвхгрбпкбмзмтбе', 'канбжгсшхшз', 'кзинвщйччажацзйнсанкнщ', 'чфгвчхчпщазтгмбнп', 'етйцгтбнщзнржнйхж', '', 'ик', '', 'еизщвпрохдгхир', 'псумйгшфбвгщдмхжтц', 'слмжопинйхнштх', 'йшралцицммбщлквмгхцввизопнт', 'л', 'чфгвчхчпщазтгмбнплдгщ', 'пбзмхжнпгикиищжтшботкцеолчцгхпбвхи', 'хзкцгрмшгхпхуоцгоудойнжлсоййосссмрткцес', 'ажуофйпщратдйцбржжлжнжащцикжиа', '']) from system.numbers limit 10; +select [6, 0, 2, 5, 2, 9, 10, 0, 0, 4, 0, 6, 3, 2] = multiSearchAllPositionsUTF8(materialize('ишогпсисжашфшлйичлба'), ['сисжашфшлй', 'пднещбгзпмшепкфосовбеге', 'шогп', 'пс', 'шогпси', 'жаш', 'аш', 'деисмжатуклдшфлщчубфс', 'грмквкщзур', 'гпсис', 'кйпкбцмисчхдмшбу', 'сисжашф', 'о', 'шо']) from system.numbers limit 10; +select [8, 15, 13, 0, 1, 2, 5, 2, 9, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('нсчщчвсанпрлисблснокзагансхм'), ['анпрлисблснокзагансхм', 'блснокз', 'исб', 'дрмгвснпл', '', 'счщчвса', 'чвсанпрлисблснокзагансх', 'счщчвсанпрлис', 'нпрли', 'пциишуецнймуодасмжсойглретиефо', 'фхимщвкехшлг', 'слщмаимшжчфхзпрцмхшуниврлуйлжмфжц']) from system.numbers limit 10; +select [0, 5, 0, 0, 14, 0, 12, 0, 2, 3, 0, 3, 21, 5] = multiSearchAllPositionsUTF8(materialize('хажуижанндвблищдтлорпзчфзк'), ['щуфхл', 'и', 'фцежлакчннуувпаму', 'щесщжрчиктфсмтжнхекзфс', 'ищдтлорпзчф', 'дееичч', 'блищ', 'гиефгйзбдвишхбкбнфпкддмбтзиутч', 'ажуижа', 'жуижанндвблищдтлорпзчфзк', 'чщщдзетвщтччмудвзчгг', 'ж', 'пзчфз', 'ижанн']) from system.numbers limit 10; +select [0, 0, 0, 9, 15, 0, 0, 0, 1, 3, 0, 0, 1, 0, 10, 0, 4, 0, 0, 7] = multiSearchAllPositionsUTF8(materialize('россроапцмцагвиигнозхзчотус'), ['ошажбчвхсншсвйршсашкм', 'пфдчпдчдмауцгкйдажрйефапввшжлшгд', 'иеаочутввжмемчушлуч', 'цмцагвиигно', 'ииг', 'ммпжщожфйкакбущчирзоммагеиучнщмтвгихк', 'укррхбпезбжууеипрзжсло', 'ншопзжфзббилйбувгпшшиохврнфчч', '', 'ссроап', 'лийщфшдн', 'йчкбцциснгначдцйчпа', 'россроапцмцагвииг', 'кштндцтсшорввжсфщчмщчжфжквзралнивчзт', 'мца', 'нбтзетфтздцао', 'сроа', 'мщсфие', 'дткодбошенищйтрподублжскенлдик', 'апцмцагвиигноз']) from system.numbers limit 10; +select [16, 0, 0, 2, 1, 1, 0, 1, 9, 0, 0, 3] = multiSearchAllPositionsUTF8(materialize('тйсдйилфзчфплсджбарйиолцус'), ['жбарйиолцу', 'цназщжждефлбрджктеглщпунйжддгпммк', 'хгжоашцшсзкеазуцесудифчнощр', 'йс', '', 'тйсдйилфзчфп', 'ивфсплшвслфмлтххжчсстзл', '', 'зчфплсдж', 'йртопзлодбехрфижчдцйс', 'цлащцкенмшеоерееиуноп', 'с']) from system.numbers limit 10; +select [3, 2, 1, 1, 0, 0, 0, 14, 6, 0] = multiSearchAllPositionsUTF8(materialize('нсцннйрмщфбшщховвццбдеишиохл'), ['цннйр', 'сцннйрм', 'н', 'нс', 'двтфхйзгеиеиауимбчхмщрцутф', 'пчтмшйцзсфщзшгнхщсутфжтлпаввфгххв', 'лшмусе', 'ховвццбд', 'йрмщфбшщховвццбдеи', 'гндруущрфзсфжикшзцжбил']) from system.numbers limit 10; +select [0, 18, 0, 1, 2, 0, 0, 0, 1, 7, 10, 0, 1, 0, 2, 0, 0, 18] = multiSearchAllPositionsUTF8(materialize('щидмфрсготсгхбомлмущлаф'), ['тлтфхпмфдлуоцгчскусфжчкфцхдухм', 'мущла', 'емлвзузхгндгафги', '', 'идмфрсготсгхбомлмущла', 'зфаргзлщолисцфдщсеайапибд', 'кдхоорхзжтсйимкггйлжни', 'лчгупсзждплаблаеклсвчвгвдмхклщк', 'щидмфр', 'сготсгхбомлму', 'тсгхбомлмущла', 'хсзафйлкчлди', '', 'й', 'ид', 'щлйпмздйхфзайсщсасейлфцгхфк', 'шдщчбшжбмййзеормнрноейй', 'мущ']) from system.numbers limit 10; +select [0, 13, 0, 0, 1, 0, 7, 7, 8, 0, 2, 0, 3, 0, 0, 13] = multiSearchAllPositionsUTF8(materialize('трцмлщввадлжввзчфипп'), ['хшзйийфжмдпуигсбтглй', 'ввзчфи', 'нсцчцгзегммтсшбатщзузпкшрг', 'гувйддежзфилйтш', '', 'хгзечиа', 'ввадлжввз', 'ввадлжввзчфи', 'ва', 'щтшсамклегш', 'рцмлщ', 'учзмиерфбтцучйдглбщсз', 'цмлщввадлжввзчфи', 'орйжччцнаррбоабцжзйлл', 'квпжматпцсхзузхвмйч', 'ввзчфип']) from system.numbers limit 10; +select [0, 1, 1, 0, 11, 4, 1, 2, 0, 0] = multiSearchAllPositionsUTF8(materialize('инкщблбвнскцдндбмсщщш'), ['жхрбсусахрфкафоилмецчебржкписуз', 'инкщблбвнс', '', 'зисгжфлашймлджинаоджруй', 'кцднд', 'щблбвнскцдндбмсщщ', 'инкщблбвнс', 'н', 'зб', 'фчпупшйфшбдфенгитатхч']) from system.numbers limit 10; +select [6, 0, 4, 20, 1, 0, 5, 0, 1, 0] = multiSearchAllPositionsUTF8(materialize('рзтецуйхлоорйхдбжашнларнцт'), ['у', 'бпгййекцчглпдвсцсещщкакцзтцбччввл', 'ецуйхлоо', 'нлар', 'рз', 'ккнжзшекфирфгсгбрнвжчл', 'цуйхлоорйхдбжашн', 'йнучгрчдлйвводт', 'рзте', 'нткрввтубчлщк']) from system.numbers limit 10; + +select [1, 1, 0, 0, 1, 0, 0, 3, 3, 3, 1, 0, 8, 0, 8, 1, 0, 1] = multiSearchAllPositionsCaseInsensitive(materialize('OTMMDcziXMLglehgkklbcGeAZkkdh'), ['', 'OTmmDCZiX', 'SfwUmhcGTvdYgxlzsBJpikOxVrg', 'ngqLQNIkqwguAHyqA', '', 'VVZPhzGizPnKJAkRPbosoNGJTeO', 'YHpLYTVkHnhTxMODfABor', 'mMdcZi', 'MmdCZI', 'MMdCZixmlg', '', 'hgaQHHHkIQRpPjv', 'ixMLgLeHgkkL', 'uKozJxZBorYWjrx', 'i', '', 'WSOYdEKatHkWiCtlwsCbKRnXuKcLggbkBxoq', '']) from system.numbers limit 10; +select [4, 15, 0, 0, 0, 0, 5, 0, 5, 1, 0, 1, 13, 0, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('VcrBhHvWSFXnSEdYCYpU'), ['bhhVwSfXnSEd', 'DycyP', 'kEbKocUxLxmIAFQDiUNoAmJd', 'bsOjljbyCEcedqL', 'uJZxIXwICFBPDlUPRyDHMmTxv', 'BCIPfyArrdtv', 'hHv', 'eEMkLteHsuwsxkJKG', 'hHVWsFxNseDy', '', 'HsFlleAQfyVVCoOSLQqTNTaA', '', 'sEDY', 'UMCKQJY', 'j', 'rBhHvw']) from system.numbers limit 10; +select [1, 1, 0, 0, 1, 0, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('wZyCLyiWnNNdNAPWeGSQZcdqk'), ['w', '', 'vlgiXgFTplwqRbnwBumAjHvQuM', 'QoIRVKDHMlapLNiIZXvwYxluUivjY', 'WZY', 'gAFpUfPDAwgzARCIMrtbZUsNcR', 'egkLWqqdNiETeETsMG', 'dzSlJaoHKlQmENIboow', 'vPNBhcaIfsgLH', 'mlWPTCBDVTdKHxlvIUVcJXBrmTcJokAls']) from system.numbers limit 10; +select [0, 10, 0, 1, 7, 1, 6, 1, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('pqliUxqpRcOOKMjtrZSEsdW'), ['YhskuppNFdWaTaZo', 'Coo', 'mTEADzHXPeSMCQaYbKpikXBqcfIGKs', 'PQLiUxq', 'qpRCoOK', 'PQLIu', 'XQPrcoOK', '', 'pR', 'cTmgRtcSdRIklNQVcGZthwfarLtAYh']) from system.numbers limit 10; +select [16, 1, 1, 1, 1, 4, 17, 0, 0, 0, 1, 0, 0, 0, 20, 0] = multiSearchAllPositionsCaseInsensitive(materialize('kJyseeDFCeUWoqMfubYqJqWA'), ['fub', 'kJY', '', '', 'Kj', 's', 'uBYQJq', 'sUqCmHUZIBtZPswObXSrYCwrdxdznM', 'mtZDCJENYuikJnCcJfRcSCDYDPXU', 'IDXjRjHhmjqXmCOlQ', '', 'jiEwAxIsJDu', 'YXqcEKbHxlgUliIALorSKDMlGGWeCO', 'OstKrLpYuASEUrIlIuHIRdwLr', 'qJq', 'tnmvMTFvjsW']) from system.numbers limit 10; +select [11, 3, 1, 0, 9, 0, 0, 0, 0, 8, 3, 0] = multiSearchAllPositionsCaseInsensitive(materialize('EBSPtFpDaCIydASuyreS'), ['iyD', 'sptfpdAciyDAsuyR', 'EbS', 'IJlqfAcPMTUsTFXkvmtsma', 'AcIYda', 'fbWuKoCaCpRMddUr', 'srlRzZKeOQGGLtTLOwylLNpVM', 'ZeIgfTFxUyNwDkbnpeiPxQumD', 'j', 'daciydA', 'sp', 'dyGFtyfnngIIbcCRQzphoqIgIMt']) from system.numbers limit 10; +select [6, 0, 0, 0, 10, 0, 1, 4, 0, 15, 0, 2, 2, 6] = multiSearchAllPositionsCaseInsensitive(materialize('QvlLEEsgpydemRZAZcYbqPZHx'), ['eSgpYDEMRzAzcyBQPzH', 'NUabuIKDlDxoPXoZOKbUMdioqwQjQAiArv', 'pRFrIAGTrggEOBBxFmnZKRPtsUHEMUEg', 'CDvyjef', 'YdEMrzaZc', 'BO', '', 'leEsgPyDEmRzaZCYBqPz', 'EzcTkEbqVXaVKXNuoxqNWHM', 'Z', 'cuuHNcHCcLGb', 'V', 'vllEes', 'eS']) from system.numbers limit 10; +select [0, 0, 0, 0, 0, 3, 0, 0, 0, 0, 5, 7, 5, 0, 11, 1] = multiSearchAllPositionsCaseInsensitive(materialize('eiCZvPdGJSmwxMIrZvEzfYFOFJmV'), ['lSydrmJDeXDYHGFFiFOOJGyCbCCDbLzbSbub', 'ewsAVflvcTBQFtvWBwuZOJKkrUArIg', 'fpEkBWaBkRWypFWtMz', 'YatSURyNtcSuerWWlTBSdBNClO', 'YO', 'CZvpdg', 'uoH', 'gtGwQSVqSJDVROmsBIxjuVNfrQnxDhWGXLBH', 'IKNs', 'HElLuRMlsRgINaNp', 'V', 'DGjsMW', 'vPDgJSmW', 'SGCwNiAmNfHSwLGZkRYEqrxBTaDRAWcyHZYzn', 'mWXMiRZvezfYf', '']) from system.numbers limit 10; +select [23, 1, 0, 17, 0, 0, 9, 3, 0, 2] = multiSearchAllPositionsCaseInsensitive(materialize('BizUwoENfLxIIYVDflhOaxyPJw'), ['yPJ', '', 'gExRSJWtZwOptFTkNlBGuxyQrAu', 'FLH', 'hCqo', 'oVGcArersxMUCNewhTMmjpyZYAIU', 'FlXIiYVdflHoAX', 'ZuWOe', 'bhfAfNdgEAtGdHylxkjgvU', 'IZUWo']) from system.numbers limit 10; +select [0, 9, 0, 0, 0, 0, 1, 0, 0, 1, 3, 0, 13, 0, 3, 5] = multiSearchAllPositionsCaseInsensitive(materialize('loKxfFSIAjbRcguvSnCdTdyk'), ['UWLIDIermdFaQVqEsdpPpAJ', 'ajBrcg', 'xmDmuYoRpGu', 'wlNjlKhVzpC', 'MxIjTspHAQCDbGrIdepFmLHgQzfO', 'FybQUvFFJwMxpVQRrsKSNHfKyyf', '', 'vBWzlOChNgEf', 'DiCssjczvdDYZVXdCfdSDrWaxmgpPXDiD', '', 'kxFFSIAjBRCGUVSNcD', 'LrPRUqeehMZapsyNJdu', 'cGuVSNcdTdy', 'NmZpHGkBIHVSoOcj', 'KxffSIAjBr', 'ffsIaJB']) from system.numbers limit 10; +select [14, 0, 11, 0, 10, 0, 0, 0, 13, 1, 2, 11, 5, 0] = multiSearchAllPositionsCaseInsensitive(materialize('uijOrdZfWXamCseueEbq'), ['sE', 'VV', 'AmcsEu', 'fUNjxmUKgnDLHbbezdTOzyLaknQ', 'XAmCsE', 'HqprIpxIcOTkDIKcVK', 'NbmirQlNsTHnAVKlF', 'VVDNOxFKSnQGKPsTqgtwLhZnIPkL', 'c', '', 'IJ', 'aM', 'rDzF', 'YFwP']) from system.numbers limit 10; +select [0, 8, 17, 0, 1, 0, 0, 0, 0, 0, 5, 0] = multiSearchAllPositionsCaseInsensitive(materialize('PzIxktujxHZsaDlwSGQPgvA'), ['zrYlZdnUxlPrVJJeZEASwdCHlNEm', 'jxhZS', 'sGQPgV', 'MZMChmRBgsxhdgspUhALoxmrkZVp', 'pzIxktuJxHzsADlw', 'xavwOAibQuoKg', 'vuuETOrWLBNLhrMeWLgGQpeFPdcWmWu', 'TZrAgmdorqZIdudhyCMypHYKFO', 'ztcCyGxRKrcUTv', 'OUvwdMZrcZuwGtjuEBeGU', 'k', 'rFTpnfGIOCfwktWnyOMeXQZelkYwqZ']) from system.numbers limit 10; +select [3, 1, 4, 1, 0, 17, 13, 0, 0, 0, 0, 0, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('pUOaQLUvgmqvxaMsfJpud'), ['OaqLUvGm', '', 'aQ', '', 'VajqJSlkmQTOYcedjiwZwqNH', 'f', 'xaMsfj', 'CirvGMezpiIoacBGAGQhTJyr', 'vucKngiFjTlzltKHexFVFuUlVbey', 'ppalHtIYycBCEjsgsXbFeecpkQMNr', 'nEgIYVoGkhTsFgBUSHJvIcYCYbuOBP', 'efjBVRVzknGrikGHxExlFEtYf', 'v', 'QgRBCaGlwNYWRslDylOrfPxZxAOF']) from system.numbers limit 10; +select [14, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 1, 0, 20, 5, 0, 4, 0] = multiSearchAllPositionsCaseInsensitive(materialize('WZNWOCjFkCAAzIptkUtyPCyC'), ['iPTkuT', 'BngeNlFbKymzMYmNPfV', 'XKEjbLtADFMqS', 'dbRQKJGSFhzljAiZV', 'wZnwoCjFKCAAzIPTKuTYpc', 'yBaUvSSGOEL', 'iEYopROOYKxBwPdCgbPNPAsMwVksHgagnO', 'TljXPJVebHqrnhSiTGwpMaNeKy', 'wzNWocjF', 'bLxLrZnOCeIfxkfZEOcqDteUvc', 'CtHYpAZDANEv', '', 'XMAMpGYMiOb', 'y', 'o', 'floswnnFjXDTxantSvDYPSnaORL', 'WOcjFkcAaZIp', 'buqBHbZsLDnCUDhLdgd']) from system.numbers limit 10; +select [0, 20, 14, 0, 2, 0, 1, 14, 0, 0, 0, 1, 0, 26, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('XJMggEHaxfddDadtwKMCcPsMlSFVJ'), ['NzbUAZvCsnRnuzTglTsoT', 'ccP', 'ADTwKmc', 'JaUzcvWHMotuEMUtjsTfJzrsXqKf', 'jMGgEHaXfdddAdTWKMCcpsM', 'SMnb', '', 'AdTWkMccPSMlsfv', 'fVjPVafkp', 'goqsYAFqhhnCkGwhg', 'CNHNPZHZreFwhRMr', '', 'vcimNhmdbtoiCgVzNuvdgZG', 'sfvJ', 'AqKmroxmRMSFAKjfhwrzxmNSSjMHxKow', 'Xhub']) from system.numbers limit 10; +select [0, 0, 7, 0, 1, 1, 0, 0, 13, 0, 1, 1, 5, 0] = multiSearchAllPositionsCaseInsensitive(materialize('VQuEWycGbGcTcCCvWkujgdoWjKgVYy'), ['UevGaXmEAtBdWsPhBfqp', 'aQOrNMPmoVGSu', 'c', 'TMhzvbNJCaxtGNUgRBmTFEqgNBIBpSJ', '', 'vq', 'pVNUTCqXr', 'QSvkansbdPbvVmQpcQXDk', 'cCCvwkUjgdOWjKgVYy', 'EtCGaEzsSbJ', 'V', '', 'WycgBgCTCcCvwkujgdoWJKgv', 'xPBJqKrZbZHJawYvPxgqrgxPN']) from system.numbers limit 10; +select [4, 1, 0, 0, 0, 0, 0, 0, 0, 18] = multiSearchAllPositionsCaseInsensitive(materialize('LODBfQsqxfeNuoGtzvrUMRVWNKUKKs'), ['Bf', 'lOdbfQs', 'ZDSDfKXABsFiZRwsebyU', 'DT', 'GEUukPEwWZ', 'GNSbrGYqEDWNNCFRYokZbZEzGzc', 'kYCF', 'Kh', 'jRMxqdmGYpTkePeReXJNdnxagceitMJlmbbro', 'VrumrvWnKU']) from system.numbers limit 10; +select [1, 1, 3, 1, 10, 0, 9, 2, 2, 0, 0, 0, 0, 0, 8, 0, 1, 11, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('lStPVtsQypFlZQoQhCuP'), ['', '', 'tpV', 'L', 'PF', 'pGPggwbkQMZandXugTpUorlPOubk', 'yPFlz', 'sTPVTsQyPfLzQOqhCU', 'StPVtSq', 'cbCxBjAfJXYgueqMFNIoSguFm', 'AosIZKMPduRfumDZ', 'AGcNTHObH', 'oPaGpsQ', 'kwQCczyY', 'q', 'HHUYdzGAzVJyn', '', 'fLZQoqHcUp', 'q', 'SSonzfqLVwIGzdHtj']) from system.numbers limit 10; +select [0, 1, 2, 0, 0, 0, 13, 1, 27, 1, 0, 1, 3, 1, 0, 1, 3, 0] = multiSearchAllPositionsCaseInsensitive(materialize('NhKJtvBUddKWpseWwRiMyBsTWmlk'), ['toBjODDZoRAjFeppAdsne', '', 'HKjTvBu', 'QpFOZJzUHHQAExAqkdoBpSbXzPnTzuPd', 'gE', 'hLmXhcEOwCkatUrLGuEIJRkjATPlqBjKPOV', 'Ps', 'NH', 'l', '', 'aSZiWpmNKfglqAbMZpEwZKmIVNjyJTtDianY', 'NhKJTvBUDDkwpS', 'KJtvbUDDKWPSewwrimYbstwm', 'NHKJTvbudDKwpSEwwR', 'hmMeWEpksVAaXd', 'NHkJTvBUDd', 'kjTvbudd', 'kmwUzfEpWSIWkEylDeRPpJDGb']) from system.numbers limit 10; +select [0, 5, 0, 0, 0, 1, 1, 15, 2, 3, 4, 5] = multiSearchAllPositionsCaseInsensitive(materialize('NAfMyPcNINKcgsShJMascJunjJva'), ['ftHhHaJoHcALmFYVvNaazowvQlgxwqdTBkIF', 'yp', 'zDEdjPPkAdtkBqgLpBfCtsepRZScuQKbyxeYP', 'yPPTvdFcwNsUSeqdAUGySOGVIhxsJhMkZRGI', 'JQEqJOlnSSam', 'nAFmy', '', 'sHJmaScjUnJj', 'afmY', 'FmYpcnINKCg', 'MYPCniNkcgSS', 'YPCNiNkCgSsHjmasCJuNjJ']) from system.numbers limit 10; +select [0, 0, 6, 3, 2, 0, 8, 2, 2, 10, 0, 0, 14, 0, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('hgpZVERvggiLOpjMJhgUhpBKaN'), ['Nr', 'jMcd', 'e', 'PZVeRvggiLOPjmjh', 'GpZVe', 'cVbWQeTQGhYcWEANtAiihYzVGUoHKH', 'VGgilOPj', 'GPZVervgGiLopjmjHGuHp', 'GP', 'gil', 'fzwDPTewvwuCvpxNZDi', 'gLLycXDitSXUZTgwyeQgMSyC', 'PJmjh', 'bTQdrFiMiBtYBcEnYbKlqpTvGLmo', 'ggHxiDatVcGTiMogkIWDxmNnKyVDJth', 'pzv']) from system.numbers limit 10; +select [7, 1, 9, 3, 0, 0, 2, 0, 1, 11] = multiSearchAllPositionsCaseInsensitive(materialize('xUHVawrEvgeYyUZGmGZejClfinvNS'), ['RevGeYyuz', 'XUHvAWrev', 'Vg', 'hvawR', 'eRQbWyincvqjohEcYHMwmDbjU', 'nuQCxaoxEdadhptAhZMxkZl', 'UhVAwREvGEy', 'lHtwTFqlcQcoOAkujHSaj', '', 'eYYUzgMgzEjCLfIn']) from system.numbers limit 10; +select [0, 0, 8, 5, 9, 1, 0, 4, 12, 6, 4, 0, 0, 12] = multiSearchAllPositionsCaseInsensitive(materialize('DbtStWzfvScJMGVPQEGkGFoS'), ['CSjYiEgihaqQDxZsOiSDCWXPrBdiVg', 'aQukOYRCSLiildgifpuUXvepbXuAXnYMyk', 'fvsCjmgv', 'TWZFV', 'VscjMgVpQ', 'dBtSTwZfVsCjmGVP', 'wqpMklzJiEvqRFnZYMfd', 'StwZfVScJ', 'j', 'wzfVsCjmGV', 'STWZfVS', 'kdrDcqSnKFvKGAcsjcAPEwUUGWxh', 'UtrcmrgonvUlLnzWXvZI', 'jMgvP']) from system.numbers limit 10; +select [0, 0, 0, 0, 7, 3, 0, 11, 1, 10, 0, 0, 7, 1, 4, 0, 17, 3, 15, 0] = multiSearchAllPositionsCaseInsensitive(materialize('YSBdcQkWhYJMtqdEXFoLfDmSFeQrf'), ['TnclcrBJjLBtkdVtecaZQTUZjkXBC', 'SPwzygXYMrxKzdmBRTbppBQSvDADMUIWSEpVI', 'QnMXyFwUouXBoCGLtbBPDSxyaLTcjLcf', 'dOwcYyLWtJEhlXxiQLRYQBcU', 'KWhYjMtqdEXFo', 'BD', 'nnPsgvdYUIhjaMRVcbpPGWOgVjJxoUsliZi', 'j', '', 'YjmtQdeXF', 'peeOAjH', 'agVscUvPQNDwxyFfXpuUVPJZOjpSBv', 'kWh', '', 'dcQKWHYjmTQD', 'qjWSZOgiTCJyEvXYqaPFqbwvrwadJsGVTOhD', 'xfoL', 'b', 'DeXf', 'HyBR']) from system.numbers limit 10; +select [4, 0, 0, 13, 1, 0, 3, 13, 16, 1, 0, 1, 16, 1, 12, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('SoVPMQNqmaTGuzYxDvZvapSuPiaP'), ['pMqNQMAtGuzYxDVz', 'TEJtgLhyredMnIpoZfmWvNwpkxnm', 'XRWmsfWVOCHhk', 'u', '', 'HvkXtxFdhVIyccpzFFSL', 'VPM', 'uZyXDVzvAPsUpIaP', 'xDvzV', 'sovpmqNQmATguZYx', 'wEG', 'soVPmQnQ', 'XDVzV', '', 'GUZyXdvzva', 'FetUahWwGtwEpVdlJCJntL', 'B', 'lSCUttZM']) from system.numbers limit 10; +select [1, 0, 1, 2, 15, 0, 0, 0, 1, 0] = multiSearchAllPositionsCaseInsensitive(materialize('zFWmqRMtsDjSeWBSFoqvWsrV'), ['', 'GItrPyYRBwNUqwSaUBpbHJ', '', 'f', 'BsfOQvWsR', 'JgvsMUZzWaddD', 'wxRECkgoCBPjSMRorZpBwuOQL', 'xHKLLxUoWexAM', '', 'YlckoSedfStmFOumjm']) from system.numbers limit 10; +select [11, 1, 1, 1, 0, 0, 1, 0, 4, 0, 0, 0, 1, 0, 5, 8] = multiSearchAllPositionsCaseInsensitive(materialize('THBuPkHbMokPQgchYfBFFXme'), ['KpqGchyfBF', '', '', 'TH', 'NjnC', 'ssbzgYTybNDbtuwJnvCCM', 'tHbupKHBMOkPQgcHy', 'RpOBhT', 'uPKHbMoKpq', 'oNQLkpSKwocBuPglKvciSjttK', 'TaCqLisKvOjznOxnTuZe', 'HmQJhFyZrcfeWbXVXsnqpcgRlg', 'tHB', 'gkFGbYje', 'pkhbMokPq', 'Bm']) from system.numbers limit 10; +select [7, 10, 0, 0, 9, 0, 0, 3, 0, 10] = multiSearchAllPositionsCaseInsensitive(materialize('ESKeuHuVsDbiNtvxUrfPFjxblv'), ['uvsDBiNtV', 'DbInTvxu', 'YcLzbvwQghvrCtCGTWVuosE', 'cGMNo', 'SDb', 'nFIRTLImfrLpxsVFMBJKHBKdSeBy', 'EUSiPjqCXVOFOJkGnKYdrpuxzlbKizCURgQ', 'KeUHU', 'gStFdxQlrDcUEbOlhLjdtQlddJ', 'DBInTVx']) from system.numbers limit 10; +select [1, 0, 2, 18, 1, 3, 15, 8, 0, 0, 1, 3, 0, 23, 2, 0, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('TzczIDSFtrkjCmDQyHxSlvYTNVKjMT'), ['', 'AmIFsYdYFaIYObkyiXtxgvnwMVZxLNlmytkSqAyb', 'ZcZI', 'HXsLVYTnvKjm', '', 'CZiDsFtRKJ', 'DQYhxSl', 'fTRKjCmdqYHxsLvYtNvk', 'hxVpKFQojYDnGjPaTNPhGkRFzkNhnMUeDLKnd', 'RBVNIxIvzjGYmQBNFhubBMOMvInMQMqXQnjnzyw', '', 'c', 'vcvyskDmNYOobeNSfmlWcpfpXHfdAdgZNXzNm', 'ytnvKJM', 'ZcZidsFtRKjcmdqy', 'IRNETsfz', 'fTR', 'POwVxuBifnvZmtBICqOWhbOmrcU']) from system.numbers limit 10; +select [14, 16, 10, 2, 6, 1, 0, 8, 0, 0, 12, 1, 0, 1, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('tejdZOLhjpFLkGBWTGPfmk'), ['GBWtgPF', 'Wt', 'PflkgBWTgpFmK', 'ejdZOLhJPFlKgb', 'o', 'TejDZ', 'HlQfCP', 'hJP', 'ydiyWEfPGyRwcKGfGVdYxAXmkY', 'QsOyrgkTGMpVUAmLjtnWEIW', 'LKGBw', 'tejDzolHJpFLKgbWT', 'IK', '', 'WrzLpcmudcIJEBapkToDbYSazKTwilW', 'DmEWOxoieDsQHYsLNelMc']) from system.numbers limit 10; +select [9, 0, 1, 4, 13, 0, 0, 1, 3, 7, 9, 0, 1, 1, 0, 7] = multiSearchAllPositionsCaseInsensitive(materialize('ZWHpzwUiXxltWPAIGGxIcJB'), ['XxLTWpA', 'YOv', '', 'pzwUIXXl', 'wp', 'lpMMLDAuflLnWMFrETXRethzCUZOWfQ', 'la', '', 'HPZ', 'UixxlTw', 'xXLTWP', 'YlfpbSBqkbddrVwTEmXxgymedH', '', '', 'QZWlplahlCRTMjmNBeoSlcBoKBTnNZAS', 'UiXxlTwPAiGG']) from system.numbers limit 10; +select [0, 9, 6, 0, 4, 0, 3, 0, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('NytxaLUvmiojEepjuCzwUYPoWL'), ['LcOnnmjbZSifx', 'm', 'lUvMIOjeE', 'vuZsNMSsutiLCDbClPUSsrziohmoZaQeXtKG', 'XaLuvm', 'hlUevDfTSEGOjvLNdRTYjJQvMvwrMpwy', 'TXALuVmioJeePjUczw', 'pKaQKZg', 'PAdX', 'FKLMfNAwNqeZeWplTLjd', 'DODpbzUmMCzfGZwfkjH', 'HMcEGRHLspYdJIiJXqwjDUBp']) from system.numbers limit 10; +select [2, 1, 0, 16, 8, 1, 6, 0, 0, 1, 8, 0, 7, 0, 9, 1, 1, 0, 0, 1] = multiSearchAllPositionsCaseInsensitive(materialize('WGVvkXuhsbzkLqiIEOuyiRfomy'), ['GVv', '', 'VbldWXHWzdziNcJKqIkDWrO', 'iEOUyIRFomy', 'hsBZklqiieOuy', '', 'X', 'emXjmIqLvXsNz', 'rxhVkujX', 'wgvvK', 'HsBzKLQiie', 'wVzJBMSdKOqjiNrXrfLEjjXozolCgYv', 'UHsbzklQiiEouyirf', 'UOvUsiKtUnwIt', 'SBZKLqiIEoUYIrfom', 'wg', '', 'BefhETEirL', 'WyTCSmbKLbkQ', '']) from system.numbers limit 10; +select [8, 1, 2, 8, 1, 0, 5, 0, 0, 4, 0, 1, 14, 0, 0, 7, 0, 1] = multiSearchAllPositionsCaseInsensitive(materialize('uyWhVSwxUFitYoVQqUaCVlsZN'), ['XufitYOVqqUACVlszn', '', 'ywH', 'XUFIT', 'uywHvSWXuFIt', 'dGhpjGRnQlrZhzGeInmOj', 'vswXuFitYovqQuA', 'dHCfJRAAQJUZeMJNXLqrqYCygdozjAC', 'rojpIwYfNLECl', 'hVswxufiTYov', 'bgJdgRoye', '', 'ovQ', 'AdVrJlq', 'krJFOKilvBTGZ', 'WxuFITYOV', 'AsskQjNPViwyTF', 'u']) from system.numbers limit 10; +select [0, 2, 0, 0, 0, 6, 0, 5, 0, 15, 0, 0, 3, 0] = multiSearchAllPositionsCaseInsensitive(materialize('BEKRRKLkptaZQvBxKoBL'), ['HTwmOxzMykTOkDVKjSbOqaAbg', 'eKrRKl', 'UrLKPVVwK', 'TyuqYmTlQDMXJUfbiTCr', 'fyHrUaoMGdq', 'KLkPtaZq', 'cPUJp', 'RKLk', 'yMnNgUOpDdP', 'BX', 'tXZScAuxcwYEfSKXzyfioYPWsrpuZz', 'dsiqhlAKbCXkyTjBbXGxOENd', 'k', 'juPjORNFlAoEeMAUVH']) from system.numbers limit 10; +select [9, 0, 0, 0, 1, 4, 2, 0, 0, 0, 0, 8, 0, 2, 0, 3, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('PFkLcrbouhBTisTkuUcO'), ['UhBtistKU', 'ioQunYMFWHD', 'VgYHTKZazRtfgRtvywtIgVoBqNBwVn', 'ijSNLKch', 'pFKlcrBOuhbtIsTku', 'lCRboUHBtI', 'fKLCRBOu', 'XTeBYUCBQVFwqRkElrvDOpZiZYmh', 'KzXfBUupnT', 'OgIjgQO', 'icmYVdmekJlUGSmPLXHc', 'OuH', 'BWDGzBZFhTKQErIRCbtUDIIjzw', 'F', 'LuWyPfSdNHIAOYwRMFhP', 'kL', 'PQmvXDCkEhrlFBkUmRqqWBxYi', 'kLcrbo']) from system.numbers limit 10; +select [0, 1, 1, 6, 14, 3, 0, 1, 9, 1, 9, 0, 1, 10, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('pfynpJvgIjSqXWlZzqSGPTTW'), ['ZzeqsJPmHmpoYyTnKcWJGReOSUCITAX', '', 'P', 'jvGIj', 'wLZzQsgP', 'YnPjVGij', 'DmpcmWsyilwHwAFcKpLhkiV', '', 'I', 'pFy', 'IjsqxwLZzqSgpT', 'pKpe', 'PfynpJvgiJSqXwlzZ', 'jsQXwLZZqs', 'onQyQzglEOJwMCO', 'GV']) from system.numbers limit 10; +select [1, 17, 1, 20, 0, 0, 5, 0, 0, 0, 24, 0] = multiSearchAllPositionsCaseInsensitive(materialize('BLNRADHLMQstZkAlKJVylmBUDHqEVa'), ['bLnRaDhLm', 'kJVYlmbuD', 'bLnr', 'yLMbU', 'eAZtcqAMoqPEgwtcrHTgooQcOOCmn', 'jPmVwqZfp', 'aDHlmqS', 'fmaauDbUAQsTeijxJFhpRFjkbYPX', 'aqIXStybzbcMjyDKRUFBrhfRcNjauljlqolfDX', 'WPIuzORuNbTGTNb', 'uDhqeVa', 'fQRglSARIviYABcjGeLK']) from system.numbers limit 10; +select [2, 0, 4, 5, 1, 15, 1, 9, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('BEwjMzphoTMoGikbrjTVyqDq'), ['E', 'sClgniMsZoGTEuLO', 'jmzphotmoGIKBRjtv', 'MZPhOtmo', '', 'Kb', '', 'otm', 'tVpxYRttoVpRLencV', 'SJAhAuMttGaeMsalRjeelAGG']) from system.numbers limit 10; +select [1, 0, 0, 0, 0, 0, 4, 0, 0, 19, 0, 7] = multiSearchAllPositionsCaseInsensitive(materialize('yNnYRQfcyemQdxUEPOiwRn'), ['', 'SJteoGNeIAMPWWBltkNKMrWDiVfR', 'kKnnKQhIPiekpnqTXJuyHfvWL', 'GPDUQEMWKzEEpvjLaIRYiuNfpzxsnSBX', 'oPrngRKwruyH', 'ukTSzFePSeVoeZeLQlAaOUe', 'yRqfcyemQDXUepo', 'CwmxidvpPHIbkJnVfSpbiZY', 'FUxmQdFVISApa', 'iwr', 'ciGHzDpMGNQbytsKpRP', 'Fcy']) from system.numbers limit 10; +select [0, 1, 0, 11, 2, 0, 1, 3, 0, 0, 0, 21] = multiSearchAllPositionsCaseInsensitive(materialize('EgGWQFaRsjTzAzejYhVrboju'), ['DVnaLFtCeuFJsFMLsfk', '', 'thaqudWdT', 'Tzazejy', 'GGW', 'RolbbeLLHOJpzmUgCN', '', 'gwqfarsjtzaZeJYHvR', 'KkaoIcijmfILoe', 'UofWvICTEbwVgISstVjIzkdrrGryxNB', 'UJEvDeESWShjvsJeioXMddXDkaWkOiCV', 'B']) from system.numbers limit 10; +select [0, 5, 2, 0, 0, 7, 0, 0, 0, 11, 0, 12, 22, 10, 0, 12] = multiSearchAllPositionsCaseInsensitive(materialize('ONgpDBjfRUCmkAOabDkgHXICkKuuL'), ['XiMhnzJKAulYUCAUkHa', 'dbj', 'nGpDbJFRU', 'xwbyFAiJjkohARSeXmaU', 'QgsJHnGqKZOsFCfxXEBexQHrNpewEBFgme', 'JFruCM', 'DLiobjNSVmQk', 'vx', 'HYQYzwiCArqkVOwnjoVNZxhbjFaMK', 'Cm', 'ckHlrEXBPMrVIlyD', 'M', 'xI', 'UcmkAOabdKg', 'jursqSsWYOLbXMLQAEhvnuHclcrNcKqB', 'mKaoaBdKghxiCkkUUL']) from system.numbers limit 10; +select [0, 1, 0, 1, 0, 0, 0, 0, 7, 21] = multiSearchAllPositionsCaseInsensitive(materialize('WhdlibCbKUmdiGbJRshgdOWe'), ['kDPiHmzbHUZB', '', 'CukBhVOzElTdbEBHyrspj', '', 'QOmMle', 'wiRqgNwjpdfgyQabxzksjg', 'RgilTJqakLrXnlWMn', 'bSPXSjkbypwqyazFLQ', 'CBkuMDiGbJRShGdOWe', 'dow']) from system.numbers limit 10; +select [0, 8, 0, 1, 1, 0, 1, 7, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('oOccAoDDoPzHUyRqdWhJxNmATEqtE'), ['LFuvoQkVx', 'DoPzh', 'YaBSTdWvmUzlgRloppaShkRmLC', 'oO', '', 'eeEpOSLSXbyaOxTscOPoaTcKcchPmSGThk', '', 'dDO', 'oFXmyIJtmcSnebywDlKruvPUgmPFzEnMvA', 'vCs', 'MsxHLTgQcaQYZdPWJshIMWbk', 'yqrjIzvrxd']) from system.numbers limit 10; +select [0, 16, 0, 0, 0, 0, 7, 1, 0, 0, 1, 2, 1, 4, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('FtjOSBIjcnZecmFEoECoep'), ['FQQwzxsyauVUBufEBdLTKKSdxSxoMFpL', 'EOecoEP', 'HGWzNTDfHxLtKrIODGnDehl', 'ZxirLbookpoHaxvASAMfiZUhYlfuJJN', 'mKh', 'GZaxbwVOEEsApJgkLFBRXvmrymSp', 'Ij', '', 'X', 'AnCEVAe', 'fTj', 'tjOSbIjcNZECMfeoEC', '', 'OsBIjcN', 'LtdJpFximOmwYmawvlAIadIstt', 'JOsBiJCNzEc']) from system.numbers limit 10; +select [0, 2, 0, 0, 19, 0, 0, 12, 1, 0, 3, 1, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('ugpnWWncvqSLsYUCVXRZk'), ['yOWnQmZuhppRVZamgmRIXXMDQdeUich', 'gPNww', 'jlyFSbvmjaYPsMe', 'fQUeGVxgQdmPbVH', 'rZk', 'ariCX', 'grAffMPlefMQvugtAzN', 'LsYuCVX', '', 'jZFoQdWEWJFfSmNDqxIyNjvxnZJ', 'P', 'UgPN', 'JmKMsbegxNvusaiGGAZKglq', 'qArXLxzdYvabPv']) from system.numbers limit 10; +select [0, 0, 0, 0, 0, 0, 8, 0, 0, 1, 1, 15, 0, 1, 7, 0] = multiSearchAllPositionsCaseInsensitive(materialize('nxwotjpplUAXvoQaHgQzr'), ['ABiEhaADbBLzPwhSfhu', 'TbIqtlkCnFdPgvXAYpUuLjqnnDjDD', 'oPszWpzxuhcyuWxiOyfMBi', 'fLkacEEeHXCYuGYQXbDHKTBntqCQOnD', 'GHGZkWVqyooxtKtFTh', 'CvHcLTbMOQBKNCizyEXIZSgFxJY', 'PlUAxVoQah', 'zrhYwNUzoYjUSswEFEQKvkI', 'c', 'NXWOt', '', 'qAhG', 'JNqCpsMJfOcDxWLVhSSqyNauaRxC', '', 'PpLuaxV', 'DLITYGE']) from system.numbers limit 10; +select [2, 0, 0, 1, 0, 0, 28, 1, 16, 1] = multiSearchAllPositionsCaseInsensitive(materialize('undxzJRxBhUkJpInxxJZvcUkINlya'), ['ndxzjRxbhuKjP', 'QdJVLzIyWazIfRcXU', 'oiXcYEsTIKdDZSyQ', 'U', 'dRLPRY', 'jTQRHyW', 'Y', '', 'nxxJZVcU', '']) from system.numbers limit 10; +select [1, 4, 1, 0, 4, 1, 0, 1, 16, 1, 0, 0, 0, 8, 12, 14, 0, 2] = multiSearchAllPositionsCaseInsensitive(materialize('lrDgweYHmpzOASVeiFcrDQUsv'), ['', 'gwEYhMP', 'LrDGwEyHmPzOaSVEifC', 'oMN', 'gwEYhMpZO', 'lrdGWEy', 'pOKrxN', 'lrDgwEyhmpZoaSv', 'eifcrdqU', 'LrDgw', 'dUvarZ', 'giYIvswNbNaBWprMd', 'pPPqKPhVaBhNdmZqrBmb', 'hmPzoASVEiF', 'O', 'SVEi', 'gIGLmHnctIkFsDFfeJWahtjDzjPXwY', 'rDGweyHmP']) from system.numbers limit 10; +select [0, 0, 11, 1, 1, 1, 0, 16, 0, 1, 5, 0, 0, 0, 2, 0, 2, 0] = multiSearchAllPositionsCaseInsensitive(materialize('XAtDvcDVPxZSQsnmVSXMvHcKVab'), ['bFLmyGwEdXiyNfnzjKxUlhweubGMeuHxaL', 'IhXOeTDqcamcAHzSh', 'ZSQsNMvsxmVHcK', '', '', '', 'dbrLiMzYMQotrvgwjh', 'MvsxMV', 'zMp', 'XaTDvCdvpXzsqSNMVSxm', 'v', 'LkUkcjfrhyFmgPXPmXNkuDjGYlSfzPi', 'ULpAlGowytswrAqYdaufOyWybVOhWMQrvxqMs', 'wGdptUwQtNaS', 'ATdVcdVPXzSqsnmVSXMvHcKVab', 'JnhhGhONmMlUvrKGjQcsWbQGgDCYSDOlor', 'atdvCdvpXzsqSnMVSxMVhCkvAb', 'ybNczkKjdlMoOavqBaouwI']) from system.numbers limit 10; +select [8, 0, 0, 0, 4, 0, 0, 5, 5, 2] = multiSearchAllPositionsCaseInsensitive(materialize('XPquCTjqgYymRuwolcgmcIqS'), ['qgyYMruW', 'tPWiStuETZYRkfjfqBeTfYlhmsjRjMVLJZ', 'PkTdqDkRpPpQAMksmkRNXydKBmrlOAzIKe', 'wDUMtn', 'UcTJQgYYMRuWoLCgMcI', 'PieFD', 'kCBaCC', 'Ct', 'C', 'pQuctjqgyymRuwOLCgmc']) from system.numbers limit 10; + +select [1, 0, 7, 1, 0, 24, 17, 0, 0, 0, 2, 0, 1, 7, 4, 1, 12, 8] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('гГБаДнФбпнЩврЩшЩЩМщЕБшЩПЖПчдт'), ['', 'таОХхрзИДжЛСдЖКЧжБВЩжЛкКХУКждАКРеаЗТгч', 'Ф', '', 'ЙЩИФМфАГщХзКЩЧТЙжмуГшСЛ', 'ПЖпчдТ', 'ЩМщЕбшЩПжПч', 'ФгА', 'гУД', 'зУцкжРоППЖчиШйЗЕшаНаЧаЦх', 'гбаДНФбПНЩВРЩШЩщМЩеБшЩпжПЧд', 'РДЧЖАбрФЦ', 'гГ', 'ФбпНщвр', 'адНфБПнщвРщШщщМщЕбШщ', 'ггб', 'ВРЩ', 'бПНщврЩш']) from system.numbers limit 10; +select [0, 12, 8, 0, 12, 0, 0, 10, 0, 8, 4, 6] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('айРВбЧБжКВИхБкчФЖЖНВнпФйФБДфЗ'), ['ЛрЦфуУДВК', 'хБкчфЖжНвнпфйфБдФ', 'жКВИХБкчФЖжНвнПф', 'кЖчвУцВСфЗБТИфбСжТИдРкшгзХвщ', 'хбк', 'штДезйААУЛчнЖофМисНЗо', 'нлнШЧВЙхОПежкцевчлКрайдХНчНб', 'вИХбкчфжжНВН', 'ЩдзЦТуоЛДСеШГфЦ', 'ЖКВихбКЧфжЖ', 'вбЧбЖкВихБкЧфЖжНВ', 'Чб']) from system.numbers limit 10; +select [18, 15, 0, 0, 0, 0, 5, 0, 14, 1, 0, 0, 0, 0, 0, 15] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('пМИОкоЗжГйНТПЙацччЧАЩгЕВБбЕ'), ['ЧЧАЩгЕВБ', 'а', 'ФбРВщшййпХдфаЗЖлЛСЗПРШПпАОинЧКзЩхждН', 'ЛфРКДЙВСУСЙОчтнИкРЗбСГфкЩреИхЛлчХчШСч', 'ШйвБПАдФдФепЗТкНУрААйеЧПВйТоЧмБГДгс', 'ФтЙлЖЕсИАХИФЗаЕМшсшуцлцАМФМгбО', 'КО', 'лиШБнлпОХИнБаФЩдмцпжЗИЛнвсЩЙ', 'йацччЧАщгевбБЕ', 'ПмИоКозжГйНТП', 'ИГНннСчКАИСБщцП', 'ПнжмЙЛвШтЩейХЛутОРЩжифбЗчгМУЛруГпх', 'ХжЗПлГЖЛйсбпрЩОТИеБвулДСиГзлЛНГ', 'учклЦНЕгжмщлжАшщжМд', 'ЩеПОЙтЖзСифОУ', 'АЦЧ']) from system.numbers limit 10; +select [10, 0, 1, 1, 6, 1, 7, 6, 0, 0, 0, 2, 12, 0, 6, 0, 4, 8, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('квхБнцхйзЕпйИмтЙхфзвгдФ'), ['еПйИМт', 'хгкиМжСБИТНщенЩИщНСкй', '', 'Квхб', 'цхЙЗЕПйИмТйХФЗ', 'к', 'хйЗЕПЙИмтй', 'Цх', 'нКлШбМЖГйШкРзадрЛ', 'ДштШвБШТг', 'СЦКйЕамЦщПглдСзМлоНШарУтМднЕтв', 'ВхБнцхйЗЕПйимТ', 'йимтЙХФЗВГД', 'жчссунЙаРцМкЖУЦщнцОЕхнРж', 'цХЙЗЕП', 'ОгНФдМЛПТИдшцмХИеКйРЛД', 'бнЦхЙ', 'ЙЗе', 'згЩищШ', 'фХлФчлХ']) from system.numbers limit 10; +select [0, 0, 0, 12, 0, 0, 27, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('хНпсРТХВдтоЦчдлеФПвнЛгЗКлПйнМВ'), ['ШиБфЗШПДЧхОЩшхфщЗЩ', 'иГйСЧЗтШЛуч', 'АЗХЦхедхОуРАСВЙС', 'цчдЛЕфП', 'СДбйГйВЕРмЙЩЛщнжен', 'НДлцСфТшАщижгфмуЖицжчзегЕСЕНп', 'й', '', 'йлчМкРИЙиМКЙжссЦТцРГзщнхТмОР', 'ПРцГувЧкйУХггОгЖНРРсшГДрлЧНжГМчрХЗфЧЕ']) from system.numbers limit 10; +select [0, 0, 2, 0, 10, 7, 1, 1, 0, 9, 0, 2, 0, 17, 0, 0, 0, 6, 5, 2] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЙзЗжпжДЕСУхчйдттСЙзоЗо'), ['щОЙУшееЧщкхГККреБкВ', 'жВ', 'ззЖпждЕсУХчЙДТТсЙ', 'ЙЦШЦЙЖзХШРвнкЕд', 'УхчйДтТсйЗОз', 'дЕСу', '', '', 'дсцеррищндЗдНкжаНЦ', 'сУхчЙдттсйзОзО', 'ЦЖРжмц', 'ЗЗ', 'СгЛГАГЕЖНгщОеЖЦДмБССцЩафзЗ', 'Сйзоз', 'ЦГХТЕвЕЗБМА', 'пмВоиеХГжВшдфАЖАшТйуСщШчИДРЙБнФц', 'Оа', 'ждЕ', 'ПжДесу', 'ЗзЖПждЕСУ']) from system.numbers limit 10; +select [0, 0, 0, 0, 5, 1, 0, 6, 0, 1, 17, 15, 1, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('уФШЙбШТоХВбзЦцЖОЕКТлщхнЖГ'), ['цЛ', 'ууМ', 'ТИгЙолМФсибтЕМнетквЦИЩИБккйн', 'оФОаМогсХЧЦооДТПхб', 'бШтОХВбЗцЦЖоЕКтЛ', 'уфШйбШтоХ', 'фдтщрФОЦсшигдПУхЛцнХрЦл', 'ШтО', 'НИкИТрбФБГИДКфшзЕмЙнДЖОсЙпЩцщкеЖхкР', 'уфШЙБш', 'екТлщ', 'ЖоекТл', 'уфШйБшТоХвбз', 'ТуОхдЗмгФеТаафЙм']) from system.numbers limit 10; +select [0, 1, 6, 1, 0, 1, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чМЩБЛЛПРлщкВУбПефХВФлАЗШао'), ['гаТкЛВнрвдПМоеКПОйр', 'ч', 'ЛпрЛЩКвуБпе', 'ЧмЩб', 'ц', '', 'жгаччЖйГЧацмдсИИВЩЩжВЛо', 'йГеЙнБзгнкЦЛБКдОЕЧ', 'ПоЦРвпЕЗСАШж', 'ЙОНЦОбиееО']) from system.numbers limit 10; +select [2, 0, 17, 1, 0, 0, 0, 5, 0, 4, 0, 0, 0, 0, 0, 2] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЕаЩичщМЦЖиЗБЛЧжуНМЧК'), ['АЩиЧЩ', 'ИлУсшДБнжщаатуРТтраПОЙКЩйТГ', 'НМЧк', 'Еа', 'зАВФЛЩбФрМВШбПФГгВЕвЖббИТйе', 'РЗНРБЩ', 'ЦдЙНГпефзЛчпУ', 'ч', 'НШШчПЗР', 'ИчЩмЦжИЗБлЧЖУНМч', 'аннвГДлмОнТЖЗЙ', 'ШдчЩшЕБвхПУсШпг', 'гФИШНфЖПжймРчхАБШкЖ', 'ЖзгЖАБлШЗДпд', 'Д', 'ащиЧ']) from system.numbers limit 10; +select [4, 1, 0, 7, 0, 7, 1, 1, 0, 3, 7, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('иОцХКЙвувМИжШдУМУЕйНсБ'), ['ХкйвуВмИжШдУм', '', 'звМАОМЩщЙПшкиТчЩдгТЦмфзеИ', 'вуВМиж', 'КДщчшЙВЕ', 'в', '', 'ИоЦхКЙВувМижШ', 'ЕвТАРи', 'цхКЙвувмИЖШДумуе', 'вУвМи', 'зПШИХчУщШХУвврХйсуЙЗеВЧКНмКШ']) from system.numbers limit 10; +select [0, 5, 0, 0, 0, 0, 0, 12, 0, 11] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЦОфбчУФсвТймЦчдщгЩжИАБ'), ['йлрк', 'ЧуФсвтйМцчдЩгщ', 'МНлЕжорв', 'иНзТЖМсмх', 'шЕМЖжпИчсБжмтЧЙчщФХб', 'жШХДнФКАЩГсОЩвЕаам', 'НпКЦХулЛвФчШЕЗкхХо', 'мЦчДЩгЩжиАб', 'мпцгВАЕ', 'Й']) from system.numbers limit 10; +select [1, 0, 0, 0, 8, 0, 2, 0, 0, 7] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чТХЙНщФфцИНБаеЖкОвлиУДР'), ['', 'рВХмжКцНцИЙраштМппсодЛнЧАКуЩ', 'ИХфХЖЧХВкзЩВЙхчфМрчдтКздиОфЙжУ', 'Гзлр', 'фЦи', 'абПф', 'тХЙНщффЦИн', 'нссГбВеЖх', 'амлЗщрсУ', 'фФ']) from system.numbers limit 10; +select [0, 9, 11, 0, 11, 1, 0, 0, 0, 1, 6, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('зДЗпщАцвТгРдврщхЩфЖл'), ['йХЛ', 'Т', 'рд', 'АИЦщгниДфВОе', 'Р', 'здзпщ', 'вКТвВШмгч', 'ввирАйбЗЕЕНПс', 'тХиХоОтхПК', '', 'аЦВТгРДврщ', '', 'уЗЗЖвУЕйтчудноЕКМЖцВРаНТЙЗСОиЕ', 'оЕфПхЕДжАаНхЕцЖжжофЦхкШоБЙр']) from system.numbers limit 10; +select [1, 1, 0, 0, 1, 7, 0, 0, 0, 2] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('йЛПЛшмЦШНЖРрЧрМцкЖзЕНжЧДелФжАн'), ['', 'йЛПлшМЦшНЖррч', 'ПНКдфтДейуиШзЗХАРУХизВ', 'ПценмщЧОФУСЙЖв', '', 'ЦшнжрРчрМЦКЖЗе', 'МрПзЕАгжРбТЧ', 'ЕДФмаФНвТЦгКТЧЦжцЛбещЛ', 'УтПУвЛкТасдЦкеИмОещНИАоИжЖдЛРгБЩнвЖКЛЕП', 'Л']) from system.numbers limit 10; +select [1, 5, 1, 1, 0, 0, 1, 1, 0, 2, 19, 0, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('сйДпмжнДРщКБгфцЖОчтГНБ'), ['', 'МЖнДРщ', 'Сй', '', 'пУщ', 'йгВИАЦнозаемТиХВвожКАПТдкПИаж', 'Сйд', 'СЙДпмжНдРщ', 'ФПщБцАпетаЙФГ', 'ЙдпМжНдрЩКбГфЦжОЧТГНб', 'т', 'гллрБВМнвУБгНаЙцМцТйЙФпзЧОЙЛвчЙ', 'йДПМжндРЩкБ', 'ЗмфОмГСНПщшЧкиССдГБУсчМ']) from system.numbers limit 10; +select [0, 18, 10, 5, 0, 2, 8, 1, 4, 11] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ХпИРддХрмВНйфчвгШиЧМКП'), ['хЗФДлДУБЙаЦтжРБЗсуйнЦпш', 'иЧмК', 'внЙ', 'д', 'зиМУЩГиГ', 'ПИр', 'РМвнЙфчвгШич', '', 'РдДхРМ', 'нЙфчВГШИ']) from system.numbers limit 10; +select [18, 0, 0, 1, 0, 0, 6, 0, 0, 9] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('нГгФкдуФШуИТбшпХфтаГт'), ['Таг', 'рРпшУйчГд', 'гК', '', 'лаВНбездпШШ', 'ЕБРйаНрОБожкКИсв', 'ДУфШУитБ', 'ГРиГШфШтйфЖлРФзфбащМЗ', 'мхЩжЛнК', 'ШуИтБШ']) from system.numbers limit 10; +select [13, 0, 0, 7, 0, 15, 0, 0, 15, 0, 0, 5, 6, 0, 18, 21, 11, 1] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('рлобшдПЦИхжФуХщжгПФукшзт'), ['УхщжГ', 'ТВщЦфФсчЩГ', 'ЕжФШойжуЛРМчУвк', 'пцИХжфуХЩж', 'бР', 'щЖГПфуКШЗТ', 'йжРГгЛуШКдлил', 'ТщЖГкбШНИщЩеЩлаАГхрАфЙНцЦгВкб', 'щжГПфУ', 'бкаДБЛХ', 'АЗ', 'шДПЦихжфух', 'дП', 'вфнЙобСцвЩмКОбЦсИббФКзЩ', 'пФУкшзТ', 'К', 'жфу', '']) from system.numbers limit 10; +select [12, 19, 8, 1, 0, 0, 0, 15, 0, 0, 12, 2, 0, 4, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЦкЛЗепкЕХЩГлКФрБдТрлвйАхдООШ'), ['лК', 'рЛв', 'Ехщ', '', 'еаПКБгЦЩАоЗВонйТЗгМхццСАаодМЕЩГ', 'ишОНиеБидфбФБЖриУЩЩ', 'дуж', 'РбДТ', 'пЗсГХКсгРущкЙРФкАНЩОржФвбЦнЩНЖЩ', 'щрОУАГФащзхффКвЕйизцсйВТШКбнБПеОГ', 'лкФрБдТРлвЙа', 'КЛзеп', 'УЛФЗРшкРщзеФуМвгПасШЧЛАЦр', 'зеПКеХщглкфР', 'ЦЖЗдХеМЕ', 'зЖжрт', 'уЩФрйрЖдЦз', 'МфцУГЩтвПАЦжтМТоеищЕфнЖй']) from system.numbers limit 10; +select [0, 0, 1, 0, 1, 0, 0, 7, 0, 5, 1, 6, 1, 1, 1, 5, 6, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('РННЕШвжМКФтшДЙлфЛИзЙ'), ['ГаМРош', 'Дтфс', '', 'еБбиаКщГхххШвхМЖКзЛАезФУчХо', 'РНн', 'сВбТМ', 'ЖЗЦПБчиСйе', 'жМкфтШДЙл', 'нЖХуеДзтЧтулиСХпТпеМлИа', 'ШВжМкФТШдЙлфл', '', 'вЖМКфТ', '', '', '', 'швЖМКфтШДЙЛфлИЗй', 'вЖмКФТ', 'еМ']) from system.numbers limit 10; +select [0, 0, 15, 1, 0, 0, 8, 1, 0, 0, 0, 4, 8, 10] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('РиучГийдХутДЕЙДпфиуд'), ['ЩмгцлЖрц', 'ЕСжСлЩЧИЖгЗЛлф', 'дП', '', 'щГЦаБтПШВзЦСрриСЙбД', 'тдРгОЛТШ', 'д', '', 'КЕбЗКСХЦТщЦДЖХпфаЧйоХАл', 'мТвзелНКрЖЧЦПпЕЙвдШтеШйБ', 'ЙОТКрБСШпШд', 'ЧГ', 'ДХУТДЕЙд', 'УТд']) from system.numbers limit 10; +select [0, 0, 0, 0, 15, 0, 0, 0, 11, 0, 0, 5, 1, 1, 0, 2, 3, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('МшазшргхОПивОлбмДоебАшцН'), ['ЦИшштН', 'еМСЗкФЕКДйОНМ', 'ЛСчГрбеРЕбЩМПМЗЦИп', 'ХнИПЧжЗдзФщЗ', 'бмдоЕ', 'гМОдйсбТСЦЩбФВЗШзшщбчегаЕмЕБаХаРР', 'фщнР', 'щмТчФчсМАОгчБщшг', 'иВ', 'УщцГОшТзпУХКоКЖБеМШ', 'мйаАЛцАегСмПОаСТИСфбЧДБКоИВчбЦЙ', 'шРгхоп', '', '', 'еИпАЩпнЛцФжЩХИрЧаИИТЛвшиСНЩ', 'шаЗ', 'АЗ', 'ФгдтфвКЩБреногуир', 'ДБжШгщШБЩпЖИЛК', 'ЧдРЩрбфЛзЙклхдМСФУЙЛн']) from system.numbers limit 10; +select [5, 0, 0, 18, 13, 0, 2, 7, 0, 0, 1, 15, 1, 0, 0, 0, 3, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('хщеКЗПчуНЙтрЧЩгфСбоКЕАДТййАрр'), ['зп', 'хчПЦшпДбзСфНВЧзНжЕМФОП', 'ЧЖхЕУк', 'БОКеАдтЙЙа', 'чЩГфС', 'шллддЩщеМжШйкЩн', 'щЕкзпЧуНЙТ', 'ЧунйтРЧщгФс', 'ввНздЙуоТЖРаВЙчМИчхРвфЛЖБН', 'ЗХМХПщПкктцАзщЙкдпжф', '', 'ГФСбОкеАДтйЙа', '', 'МБХВЕчпБМчуххРбнИМЛТшЩИщЙгаДцзЛАМвйаО', 'ЛкОзц', 'ЕцпАДЗСРрсЕвтВщДвцбЗузУннТИгХжхрцПДРДПм', 'екЗПЧунЙТРчщгФсбоК', 'шпИфЕчгШжцГВСйм', 'ЛхйЧбЧД', 'ВзЗоМцкЩНХГж']) from system.numbers limit 10; +select [0, 0, 6, 20, 0, 10, 0, 0, 0, 9, 10, 3, 23, 1, 0, 0, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('лцапШиХчЛДшдксСНИбшгикзчЙанми'), ['ХууатТдтбодМГЧгщЧнклШтЗПНчкЦОаЙг', 'МЦЧчпИхКЛаФхщХдРУДщжУчфлжахц', 'иХЧлдшдкСсНИбШГикзЧЙ', 'гикЗчйА', 'ГсТзЛОфИББлекЩАсЛвмБ', 'Д', 'ЦХрТЖощНрУШфнужзжецсНХВфЩБбДУоМШШиГйж', 'йуВдЕзоггПВДЖб', 'ЙфБГйХМбжоакЖЛфБаГИаБФСнБЖсТшбмЗЙТГОДКИ', 'ЛДШдКССНИБшГикзч', 'ДШдКССниБ', 'аПШИХчЛДШДКсс', 'з', '', 'ФоохПЩОГЖоУШлКшзЙДоуп', 'хАДХЩхлвУИсшчрбРШУдФА', 'ЦА', 'гвптУФлчУуРхпрмЖКИрБеЩКчН']) from system.numbers limit 10; +select [0, 4, 5, 7, 15, 3, 3, 17, 7, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('зЗАЩлЕЕЕПИохЧчШвКЧйрсКХдд'), ['пКРбуШОНТЙБГНзИРвЖБсхрЛщчИрлЧУ', 'ЩЛЕЕЕПиоХЧ', 'ЛеЕеп', 'Еепио', 'швкЧйрС', 'ащЛеееПИох', 'АЩлеЕЕпиОхЧЧШвкЧЙРсК', 'КчйРскхД', 'ЕЕПИохччшВКчй', 'у']) from system.numbers limit 10; +select [1, 12, 0, 8, 1, 1, 0, 1, 5, 0, 1, 0, 0, 0, 0, 3, 1, 0, 4, 5] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ПмКСйСКЖККмШеоигЙчПфжТ'), ['', 'Шео', 'РчвлдЙЙлПщуКмтН', 'жкКмшЕоИГЙЧ', '', '', 'йРмМЖнПиЙ', '', 'йс', 'тфФРСцл', '', 'щлЩХиКсС', 'кпнТЖпФЩиЙЛ', 'абкКптбИВгмЧкцфЦртЛДЦФФВоУхЗБн', 'чНшоВСГДМйДлтвфмхХВВуеЩЦВтЖтв', 'кС', '', 'фидБлйеЙЧШРЗЗОулщеЕЩщЙсЙшА', 'СЙс', 'йсКжкКМшЕо']) from system.numbers limit 10; +select [0, 0, 1, 0, 2, 2, 1, 2, 7, 0, 1, 2, 1, 0, 6, 8] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('УгЖЕугАЩХйчидаррлжНпфФГшр'), ['утвШ', 'кЕвФч', 'угжеУг', 'тШлТвЕШЗчЖеЛНджЦазЩХцж', 'гЖеугаЩхй', 'ГжЕугаЩХйЧидАР', 'УгжЕУГаЩХЙЧИда', 'гЖеу', 'ащхЙчИ', 'мЧлщгкЛдмЙЩРЧДИу', '', 'ГжеугАщХйЧиДаРРЛЖНП', '', 'зЕМвИКбУГКЩФшоГЧГ', 'ГАЩХйчИДАррлЖНпФфг', 'ЩХЙчИдАррЛЖНпфФгш']) from system.numbers limit 10; +select [1, 0, 0, 7, 0, 6, 0, 11, 0, 0, 0, 2, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЗЕГЛЩПцГНтзЕЦШЧхНКГТХЙЙФШ'), ['', 'шзкиЗсаИщАБмаз', 'Ж', 'ц', 'гШуЕжЛСПодРнхе', 'пцГНтЗЕЦ', 'щРкЩАеНржЙПМАизшщКвЗщглТкКИф', 'ЗеЦшчхнКГтхЙЙ', 'пелгЩКкцвтфнжЖУуКосЙлкЛ', 'рф', 'хНШчНрАХМШщфЧкЩБНзХУкилЙмП', 'ЕгЛЩПЦгнтзецШЧ', 'ЩУчБчРнЖугабУоиХоИККтО', 'СГмЦШтФШЛмЙЩ', 'ауТПЛШВадоХМПиБу', 'ЩЩйр']) from system.numbers limit 10; +select [2, 2, 1, 0, 0, 0, 0, 0, 1, 0, 7, 9, 0, 15, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('гЙЧЙФХнЖБвомгАШГбОВГксИйцз'), ['ЙЧйфхНЖбвО', 'Й', 'гЙЧйфхнЖбв', 'хсЩмШЙЙММВЦмУБТчгзУЛР', 'зктшп', 'дЕоиЖлгШж', 'хКкаНЛБ', 'ЗКйСчсоЗшскГЩбИта', '', 'у', 'НжбВОмгашГ', 'БВо', 'ещфРШлчСчмаЖШПЧфоК', 'шгбо', 'ЙСтШШДЩшзМмдпЧдЙЖевТвоУСЕп', 'Л']) from system.numbers limit 10; +select [0, 9, 0, 0, 18, 13, 13, 11, 0, 0, 4, 1] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЙЛмоЦСдТаоФчШКЖЦСНРаРЦзоС'), ['ДфгЗАасВфаМмшхчлмР', 'аоФчШкЖцСнРАРЦзОС', 'зЩзнйтФРТЙжУлхФВт', 'чЦкШВчЕщДУМкхЛУЩФшА', 'н', 'Шк', 'шКЖцсНРаРцЗос', 'фчшкЖцснрАРЦз', 'лку', 'пЧШМЦквоемЕщ', 'о', 'йЛМоцСДТАофЧшкжЦСнРаРЦзос']) from system.numbers limit 10; +select [21, 0, 0, 17, 1, 11, 0, 2, 0, 7] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('кЧЖнЕбМЛпШЗХиЙжиМщлнСФрПЧЖВН'), ['сФ', 'гцХаШЛсаШЛкшфЧОКЛцзешХСиЩоаЕОш', 'Г', 'МщЛНСФРпч', '', 'зХ', 'ОАДепНпСГшгФАЦмлуНуШШЗфдЧРШфрБЛчРМ', 'чЖне', 'СфЕАбФн', 'М']) from system.numbers limit 10; +select [4, 0, 1, 1, 0, 2, 4, 16, 3, 6, 5, 0, 0, 6, 1, 0, 5, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('кдАпЩСШИСцРхтеСиФЖЧСсОоц'), ['пщСшиСцрХТЕсифЖчССоОц', 'рхнкикДТКДВШчиЖЦнВм', '', '', 'жПЛСнЦцн', 'дА', 'ПщсШИсцрХтЕс', 'иФжЧсСоОЦ', 'ап', 'с', 'щсШИ', 'МАзашДРПЩПзРТЛАсБцкСШнЕРЙцИЩлТЛеУ', 'ичцпДбАК', 'сшИСЦрхтЕсифжчСсООц', 'КдАПЩСшИСЦРХТЕсИфЖЧСсо', 'ЛнБсИПоМЩвЛпиЩЗЖСд', 'щс', 'шщДНБаСщЗАхкизжнЛАХЙ']) from system.numbers limit 10; +select [0, 13, 0, 2, 16, 1, 3, 0, 9, 0, 2, 0, 1, 4, 0, 0, 0, 1] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('иНхеЕкхЩщмгзМГхсгРБсхОКцУгуНБ'), ['ДиоУлФЖЛисУСЕтсЕалщн', 'МгХсгрБСХО', 'ЖХНцршПшгйО', 'нХЕЕкхЩ', 'сГРбсхОКцУг', '', 'х', 'Ж', 'щМгЗмгхСг', 'СрпхДГОУ', 'НхеЕкХщ', 'ПМтБцЦЙЖАЙКВБпФ', 'ИнхеЕ', 'еЕКхЩ', 'мМГлРзш', 'гтдоЙБСВещкЩАЩЦйТВИгоАЦлчКнНРНПДЖшСЧа', 'ЖшеН', '']) from system.numbers limit 10; +select [1, 5, 0, 0, 3, 0, 2, 0, 14, 14, 1, 0, 17, 13, 3, 25] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('айлзсЗБоГйтГжЙРККФхКшлНРОрЦкфо'), ['', 'с', 'Д', 'шиБраНИЦЧуИжп', 'Лз', 'ДРБСУфКСшцГДц', 'йЛЗСЗбОгЙтГЖйРК', 'ЕЙЦсвРЕШшщЕЗб', 'ЙркКфхкшЛнРОР', 'ЙРкКФхкШ', 'а', 'ГдоДКшСудНл', 'КФхКшлНРоР', 'ж', 'лзСзБогйТГЖйрККф', 'оР']) from system.numbers limit 10; +select [6, 0, 8, 10, 1, 0, 1, 13, 0, 0, 0, 2, 2, 0, 4, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('РучУлрХчЗУпИчДТЕфщИЙщрНлн'), ['РХЧ', 'оДсГСЛЙшйиЧРСКзчХВоХарцНШ', 'ЧЗУпИ', 'УПичдТе', 'Р', 'ВЙЩхжАутПСНЦфхКщеЩИуЧдчусцАесзМпмУв', '', 'ЧдТ', 'ООсШИ', 'ФШсВжХтБУШз', 'ЕЩуДдшкМУРЕБшщпДОСАцйауи', 'УЧ', 'УЧУЛрХчзуПИчдТеФщий', 'йнЦцДСхйШВЛнШКМСфмдЩВйлнеЖуВдС', 'улрхчзупиЧдтефщИ', 'СХТЧШшГТВвлЕИчНОВи']) from system.numbers limit 10; +select [0, 0, 0, 2, 1, 1, 0, 1, 19, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('УецжлЦЦщМшРГгЩЩдБмхЖЗЧзШЙб'), ['НзИуАузуРЗРуКфоТМмлПкрсмЕЕЕнТ', 'ЕЩГХхЧш', 'ХоЙпООчфЖввИжЙшЖжЕФОтБхлВен', 'ЕЦЖЛЦцщ', '', '', 'ухогСИФвемдпаШЗуЛтлизОЧ', 'УецЖ', 'ХЖзЧЗ', 'П', 'мБкзХ', 'уБуОБхШ']) from system.numbers limit 10; +select [6, 1, 15, 5, 0, 0, 0, 3, 2, 4, 0, 12, 0, 2, 0, 3, 1, 6, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ГЖФеачМаКчПСпкВкхсПтг'), ['чмАкЧ', '', 'ВкХс', 'ачМА', 'КлтжУлОЛршБЕблФЩ', 'тцуМфж', 'л', 'фе', 'Жф', 'ЕАЧМак', 'лЖЕРТнФбЧЙТййвзШМСплИхбЙЛЖзДпм', 'СпкВК', 'ЩзчжИш', 'жФеАчМ', 'КбЦбйЕШмКтЩЕКдуЩтмпИЕВТЖл', 'ФЕаЧмАКчПСПквкхспТ', 'гжФеАЧмаКчпСп', 'ЧмАК', 'дцкДННМБцйЕгайхшжПГх', 'ТЩбвЦЖАНшрАШФДчОщй']) from system.numbers limit 10; +select [1, 6, 0, 1, 0, 0, 3, 1, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('МФННЧйОнцЛИЧЕПШПЧйоГФО'), ['', 'йОн', 'шУлгИЛЛРЙАсфЗоИЙЗРхуПбОЙсшдхо', 'МФННчЙоНц', 'лзВжбЦзфкзтуОйзуЗ', 'ЖГДщшЦзсжщцЦЖеЧвРфНИНОСАОщг', 'ННчйОНЦлИчЕПШ', '', 'Ф', 'ЩрИдНСлЙуАНЗвЕчмчАКмФУипндиП']) from system.numbers limit 10; +select [5, 0, 8, 13, 0, 0, 0, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('зВйймХЩМзЦГЕкЕКфоСтхПблуКМхц'), ['МХщмз', 'НАНрШоНДмурМлО', 'мзцгЕкек', 'кеКфоСтХПбЛУК', 'СУУксО', 'ЦоШжЧфйШЦаГЧйбЛШГЙггцРРчт', 'НбтвВбМ', '', 'тЩФкСтоСЧЦЦЙаСДЩСГЙГРИФЗОЗфбТДЙИб', 'ВГж']) from system.numbers limit 10; +select [0, 0, 0, 8, 19, 0, 3, 12, 1, 4] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ДпбЙЖНЗбПнЛбахБаХТуабШ'), ['цИаЩвгеИР', 'Ф', 'РЖиА', 'БпнЛб', 'У', 'Тфн', 'Б', 'БА', '', 'ЙЖНзБПнлбАхбаХ']) from system.numbers limit 10; +select [0, 0, 0, 0, 0, 1, 0, 17, 1, 0, 1, 1, 1, 11, 0, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ТЦмЩОинХзоДДпПНЩигрРщОзКц'), ['ЕжЙВпПл', 'ВКфКТ', 'ШкДсЖхшфоПИадУбхФЩБчОАкпУеБхи', 'НТЕЙОШЦЖоЩбзВзшс', 'учГгуКФзлУдНУУуПУлкаЦЕ', '', 'фАПМКуЧйБЧзСоЗргШДб', 'ИГРрщОзк', '', 'йупОМшУйзВиВрЛЩЕеЩмп', '', '', '', 'дДППнщИгРР', 'ШФвИЧакеЦвШ', 'ТцМЩоинхЗОДдппнЩ', 'мрОгЩшЩеЧ', 'еЖРиркуаОТсАолЩДББВАМБфРфпШшРРРм']) from system.numbers limit 10; +select [3, 0, 0, 0, 0, 0, 1, 0, 0, 14, 0, 1, 0, 1, 1, 1, 0, 7] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('аОкиЛгКйхаОГОУзЦЛрбцш'), ['кИЛГкйхАогоУЗЦл', 'щЧДпХИхбпсГвфДФХкчХ', 'ШвАмБЗлДОИПткжхФТФН', 'щфсхФмЦсЛеувЙО', 'лВУЖц', 'еИщРшозЖАдцтКииДУлДОУФв', 'а', 'ХгЦРШ', 'ФзрЖкРЗЩЧИеЧцКФИфЧЧжаооИФк', 'уЗ', 'фЦФдцжжМчЗЖлиСЧзлщжжЦт', '', 'МдхжизИХфвбМААрйФНХдЕжп', 'аОкиЛг', 'АОКИЛгкйХАОГОУЗЦ', '', 'МбЖйрсумщиеОЩк', 'КйХАоГоУЗцлРБЦШ']) from system.numbers limit 10; +select [0, 0, 2, 1, 0, 0, 12, 0, 17, 0, 0, 0, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('КУчЛХФчЛХшвбМЦинРвНрФМРкмиеЕп'), ['ТБЩБзхАмщПщЧПИФПашгЕТиКЦМБМпСЩСуЩМчтшеш', 'йлВЕЙшфшаШЗШЩВХЦчЛБс', 'УЧл', '', 'ЛДсЖщмНЦсКуфЗуГиука', 'РТТОТфГЕлЩЕгЛтДфлВЖШГзЦЖвнЗ', 'БМцИНРвнРф', 'ОЕИЕдИсАНаифТПмузЧчЖфШЕуеЩсслСШМоЖуЩЛМп', 'рвНРфМркМи', 'ЦзБМСиКчУжКУЩИИПУДвлбдБИОЙКТЛвтз', 'злСГе', 'ВдтцвОИРМЕжХО', 'учЛХфЧл', 'БшччШбУзЕТзфКпиШжнезвоеК']) from system.numbers limit 10; +select [0, 7, 0, 0, 0, 0, 7, 6, 0, 16, 12, 12, 15, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('оЖиогсфклШМСДрбхРбМбрЕщНЙЗйод'), ['иПмДКейууОклНХГЗсбаЙдШ', 'ФКлШмсДрБХРбМбрещНЙЗЙОд', 'арчжтСТнк', 'чбТНЛЕжооЗшзОУ', 'ощАЩучРСУгауДхГКлмОхЙцЕо', 'аЛбкиЦаКМбКхБМДнмФМкйРвРр', 'ФКлШмСДрбХРбм', 'СфклШ', 'еДйилкУлиИчХЙШтхцЗБУ', 'хрБ', 'СДрбХрбМБР', 'СдрбхРБ', 'бхрБМБРЕщНйз', 'КИб']) from system.numbers limit 10; +select [22, 1, 8, 0, 0, 1, 0, 3, 0, 6, 20, 0, 0, 0, 4, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЕЖДФбКужЙЦЦмсЖГГжБзеЙнПйЙри'), ['НПййР', '', 'Жй', 'Щ', 'ФхУО', 'ЕЖДфБКУЖйЦЦмСжГГ', 'НФЙзщЩГЧпфсфЦШОМЕЗгцрс', 'д', 'ЦтщДДЖтбвкгКонСк', 'кУЖЙЦЦм', 'ЕйНПййРИ', 'РчеЙйичФбдЦОтпчлТЖИлДучЙПгЗр', 'внчзшЗзОнФфхДгфзХТеНПШРшфБТЖДйф', 'кНснгмулМуГНурщЕББСузВмбнЧаХ', 'фбКУЖйЦцМсЖГгЖб', 'ЩСЕ']) from system.numbers limit 10; +select [0, 0, 0, 1, 10, 4, 0, 0, 5, 0, 1, 0, 7, 0, 3, 7, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чБхлжгКЖХлЙнкКЦфжЕгЖАндЧ'), ['ПдмРрЖАтВнСдСБШпПЗГгшИ', 'цшцг', 'тчАЙЧОеЕАвГпЗцЖЧгдХуЛСЛНрвАЖщ', '', 'Лй', 'Л', 'ОйррцУжчуЦБАжтшл', 'вХУКк', 'жгКжхЛЙН', 'уцбЕЕОЧГКУПуШХВЕчГБнт', '', 'ПсАжБИКштЕаН', 'КжхлЙН', 'ЩгШухЦПАТКежхгХксгокбщФЙПсдТНШФЦ', 'Х', 'кЖХЛйНккЦФжЕГЖ', 'ЙзРДСПднаСтбЧЖхощ', 'пАПОУЧмИпслБЗПфУ']) from system.numbers limit 10; +select [0, 0, 0, 5, 2, 16, 4, 4, 11, 0, 0, 3, 3, 0, 0, 6] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('кпМаоуГГфвощолЦЩщЧПРОКепеА'), ['ЗзуФжНшщПТнЧЦКВОиАУсЧХОШбк', 'тмПкАпеайзуХсурШй', 'АЕЦавбШиСДвВДумВкиИУБШЕ', 'о', 'ПМаОУггФВощоЛЦЩЩЧПрокЕПеа', 'щЩ', 'аоУг', 'аОуГгФВ', 'оЩоЛЦЩщчПРОК', 'виХЛшчБсщ', 'УчАМаЦкйДЦфКСмГУЧт', 'мАоУ', 'МАО', 'щФФА', 'Н', 'У']) from system.numbers limit 10; +select [0, 3, 10, 8, 3, 0, 4, 0, 9, 4, 1, 9] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('щЙЧРпшИцхпргЦНуДййусЧЧнЖ'), ['ДлУцтееЖБКХгМзСВжА', 'чРпШИЦ', 'пргЦнУДЙЙУ', 'Ц', 'ЧРПш', 'нЩрЕвмрМеРйхтшЩче', 'РпШИЦхПРГцнУд', 'ПНоЙтПкоаОКгПОМЦпДЛФЩДНКПбСгЗНЗ', 'ХПРГцНудЙЙ', 'рПши', '', 'ХПРГ']) from system.numbers limit 10; +select [11, 4, 1, 0, 1, 0, 0, 0, 0, 12, 0, 9, 5, 0, 16, 0, 12, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('пкзщщЛНОНбфЦноИЧфхбФ'), ['ф', 'щщл', 'ПКзЩщЛНОн', 'ЩшФйЧБНДОИзМхеЖНЦцеЛлУЧ', '', 'сЗоЙТклйДШкДИЗгЖ', 'орЛФХПвБбУхНс', 'доЗмЩВу', 'ШиЕ', 'ЦНО', 'ндЩдРУЖШМпнзНссЖШДЦФвпТмуМЙйцН', 'НбФЦнОИч', 'ЩлНонБФ', 'ЛдРжКММЙм', 'чфх', 'ЦматДйиСфЦфааЦо', 'ЦНОИчФх', 'иржЦщн']) from system.numbers limit 10; +select [0, 0, 0, 0, 0, 1, 0, 0, 0, 1, 0, 1, 0, 3, 0, 5] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чЖажцВбшЛттзДааАугШщАйПгщП'), ['ШгУТсчГОВЦЦеЛАСфдЗоЗЦВЛйлТДзчвЛва', 'УшЕшищЖткрвРСйиФЗйТФТЛЗаЗ', 'ВдикЙббщузоФХщХХГтЗоДпхбЕкМщц', 'срйеХ', 'рАшуПсЙоДнхчВкПЖ', '', 'гНЗбКРНСБВрАВФлнДШг', 'фХЧгмКнлПШлЩР', 'мкйЗбИФрЗахжгАдвЕ', 'чжаЖцВБШлТ', 'лХЕСрлПрОс', '', 'ЗЧПтчЙОцвОФУФО', 'ажцвБшЛТт', 'уНчЖШчМЕА', 'ц']) from system.numbers limit 10; +select [7, 1, 0, 7, 1, 19, 8, 6, 3, 0, 2, 13, 6, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('НТКПпмБжДцбАКПНСЖоиТФД'), ['б', '', 'аУщЛМХЖбвИтНчГБМГдДнч', 'Б', 'НТкппм', 'и', 'Жд', 'МБждЦбАкП', 'кппмБждцБа', 'ПЕрнЦпМЦВгЧЧгГ', 'ткПпМБЖДцбаКпнСжО', 'кПнСЖоИ', 'МБжДцБакпН', 'гхОХжГуОвШШАкфКМщсшФДШеИжоАйг']) from system.numbers limit 10; + +select 0 = multiSearchAny(materialize('mpnsguhwsitzvuleiwebwjfitmsg'), ['wbirxqoabpblrnvvmjizj', 'cfcxhuvrexyzyjsh', 'oldhtubemyuqlqbwvwwkwin', 'bumoozxdkjglzu', 'intxlfohlxmajjomw', 'dxkeghohv', 'arsvmwwkjeopnlwnan', 'ouugllgowpqtaxslcopkytbfhifaxbgt', 'hkedmjlbcrzvryaopjqdjjc', 'tbqkljywstuahzh', 'o', 'wowoclosyfcuwotmvjygzuzhrery', 'vpefjiffkhlggntcu', 'ytdixvasrorhripzfhjdmlhqksmctyycwp']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('qjjzqexjpgkglgxpzrbqbnskq'), ['vaiatcjacmlffdzsejpdareqzy', 'xspcfzdufkmecud', 'bcvtbuqtctq', 'nkcopwbfytgemkqcfnnno', 'dylxnzuyhq', 'tno', 'scukuhufly', 'cdyquzuqlptv', 'ohluyfeksyxepezdhqmtfmgkvzsyph', 'ualzwtahvqvtijwp', 'jg', 'gwbawqlngzcknzgtmlj', 'qimvjcgbkkp', 'eaedbcgyrdvv', 'qcwrncjoewwedyyewcdkh', 'uqcvhngoqngmitjfxpznqomertqnqcveoqk', 'ydrgjiankgygpm', 'axepgap']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('fdkmtqmxnegwvnjhghjq'), ['vynkybvdmhgeezybbdqfrukibisj', 'knazzamgjjpavwhvdkwigykh', 'peumnifrmdhhmrqqnemw', 'lmsnyvqoisinlaqobxojlwfbi', 'oqwfzs', 'dymudxxeodwjpgbibnkvr', 'vomtfsnizkplgzktqyoiw', 'yoyfuhlpgrzds', 'cefao', 'gi', 'srpgxfjwl', 'etsjusdeiwbfe', 'ikvtzdopxo', 'ljfkavrau', 'soqdhxtenfrkmeic', 'ktprjwfcelzbup', 'pcvuoddqwsaurcqdtjfnczekwni', 'agkqkqxkfbkfgyqliahsljim']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('khljxzxlpcrxpkrfybbfk'), ['', 'lpc', 'rxpkrfybb', 'crxp', '', 'pkr', 'jxzxlpcrxpkrf', '', 'xzxlpcr', 'xpk', 'fyb', 'xzxlpcrxpkrfybbfk', 'k', 'lpcrxp', 'ljxzxlpcr', 'r', 'pkr', 'fk']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('rbrizgjbigvzfnpgmpkqxoqxvdj'), ['ee', 'cohqnb', 'msol', 'yhlujcvhklnhuomy', 'ietn', 'vgmnlkcsybtokrepzrm', 'wspiryefojxysgrzsxyrluykxfnnbzdstcel', 'mxisnsivndbefqxwznimwgazuulupbaihavg', 'vpzdjvqqeizascxmzdhuq', 'pgvncohlxcqjhfkm', 'mbaypcnfapltsegquurahlsruqvipfhrhq', 'ioxjbcyyqujfveujfhnfdfokfcrlsincjbdt', 'cnvlujyowompdrqjwjx', 'wobwed', 'kdfhaoxiuifotmptcmdbk', 'leoamsnorcvtlmokdomkzuo', 'jjw', 'ogugysetxuqmvggneosbsfbonszepsatq']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('uymwxzyjbfegbhgswiqhinf'), ['lizxzbzlwljkr', 'ukxygktlpzuyijcqeqktxenlaqi', 'onperabgbdiafsxwbvpjtyt', 'xfqgoqvhqph', 'aflmcwabtwgmajmmqelxwkaolyyhmdlc', 'yfz', 'meffuiaicvwed', 'hhzvgmifzamgftkifaeowayjrnnzw', 'nwewybtajv', 'ectiye', 'epjeiljegmqqjncubj', 'zsjgftqjrn', 'pssng', 'raqoarfhdoeujulvqmdo']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('omgghgnzjmecpzqmtcvw'), ['fjhlzbszodmzavzg', 'gfofrnwrxprkfiokv', 'jmjiiqpgznlmyrxwewzqzbe', 'pkyrsqkltlmxr', 'crqgkgqkkyujcyoc', 'endagbcxwqhueczuasykmajfsvtcmh', 'xytmxtrnkdysuwltqomehddp', 'etmdxyyfotfyifwvbykghijvwv', 'mwqtgrncyhkfhjdg', 'iuvymofrqpp', 'pgllsdanlhzqhkstwsmzzftp', 'disjylcceufxtjdvhy']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('mznihnmshftvnmmhnrulizzpslq'), ['nrul', 'mshftvnmmhnr', 'z', 'mhnrulizzps', 'hftvnmmhnrul', 'ihnmshftvnmmhnrulizzp', 'izz', '', 'uli', 'nihnmshftvnmmhnru', 'hnrulizzp', 'nrulizz']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('ruqmqrsxrbftvruvahonradau'), ['uqmqrsxrbft', 'ftv', 'tvruvahonrad', 'mqrsxrbftvruvahon', 'rbftvruvah', 'qrsxrbftvru', 'o', 'ahonradau', 'a', 'ft', '', 'u', 'rsxrbftvruvahonradau', 'ruvahon', 'bftvruvahonradau', 'qrsxrbftvru', 't', 'vahonrada', 'vruvahonradau', 'onra']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('gpsevxtcoeexrltyzduyidmtzxf'), ['exrltyzduyid', 'vxtcoeexrltyz', 'xr', 'ltyzduyidmt', 'yzduy', 'exr', 'coeexrltyzduy', 'coeexrltyzduy', 'rlty', 'rltyzduyidm', 'exrltyz', 'xtcoeexrlty', 'vxtcoeexrltyzduyidm', '', 'coeexrl', 'sevxtcoeexrltyzdu', 'dmt', '']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('dyhycfhzyewaikgursyxfkuv'), ['sktnofpugrmyxmbizzrivmhn', 'fhlgadpoqcvktbfzncxbllvwutdawmw', 'eewzjpcgzrqmltbgmhafwlwqb', 'tpogbkyj', 'rtllntxjgkzs', 'mirbvsqexscnzglogigbujgdwjvcv', 'iktwpgjsakemewmahgqza', 'xgfvzkvqgiuoihjjnxwwpznxhz', 'nxaumpaknreklbwynvxdsmatjekdlxvklh', 'zadzwqhgfxqllihuudozxeixyokhny', 'tdqpgfpzexlkslodps', 'slztannufxaabqfcjyfquafgfhfb', 'xvjldhfuwurvkb', 'aecv', 'uycfsughpikqsbcmwvqygdyexkcykhbnau', 'jr']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('vbcsettndwuntnruiyclvvwoo'), ['dwuntnru', '', 'ttndwuntnruiyclvv', 'ntnr', 'nruiyclvvw', 'wo', '', 'bcsettndwuntnruiycl', 'yc', 'untnruiyclvvw', 'csettndwuntnr', 'ntnruiyclvvwo']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('pqqnugshlczcuxhpjxjbcnro'), ['dpeedqy', 'rtsc', 'jdgla', 'qkgudqjiyzvlvsj', 'xmfxawhijgxxtydbd', 'ebgzazqthb', 'wyrjhvhwzhmpybnylirrn', 'iviqbyuclayqketooztwegtkgwnsezfl', 'bhvidy', 'hijctxxweboq', 't', 'osnzfbziidteiaifgaanm']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('loqchlxspwuvvccucskuytr'), ['', 'k', 'qchlxspwu', 'u', 'hlxspwuvv', 'wuvvccucsku', 'vcc', 'uyt', 'uvv', 'spwu', 'ytr', 'wuvvccucs', 'xspwuv', 'lxspwuvvccuc', 'spwuvvccu', 'oqchlxspwuvvccucskuy']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('pjjyzupzwllshlnatiujmwvaofr'), ['lnatiujmwvao', '', 'zupzwllsh', 'nati', 'wllshl', 'hlnatiujmwv', 'mwvao', 'shlnat', 'ati', 'wllshlnatiujmwvao', 'wllshlnatiujmwvaofr', 'nat']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('iketunkleyaqaxdlocci'), ['nkleyaqaxd', 'etunkleyaq', 'yaqaxdlocci', 'tunkleyaq', 'eyaqaxdlocc', 'leyaq', 'nkleyaqaxdl', 'tunkleya', 'kleyaqa', 'etunkleya', 'leyaqa', 'dlo', 'yaqa', 'leyaqaxd', 'etunkleyaq', '']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('drqianqtangmgbdwruvblkqd'), ['wusajejyucamkyl', 'wsgibljugzrpkniliy', 'lhwqqiuafwffyersqjgjvvvfurx', 'jfokpzzxfdonelorqu', 'ccwkpcgac', 'jmyulqpndkmzbfztobwtm', 'rwrgfkccgxht', 'ggldjecrgbngkonphtcxrkcviujihidjx', 'spwweavbiokizv', 'lv', 'krb', 'vstnhvkbwlqbconaxgbfobqky', 'pvxwdc', 'thrl', 'ahsblffdveamceonqwrbeyxzccmux', 'yozji', 'oejtaxwmeovtqtz', 'zsnzznvqpxdvdxhznxrjn', 'hse', 'kcmkrccxmljzizracxwmpoaggywhdfpxkq']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('yasnpckniistxcejowfijjsvkdajz'), ['slkpxhtsmrtvtm', 'crsbq', 'rdeshtxbfrlfwpsqojassxmvlfbzefldavmgme', 'ipetilcbpsfroefkjirquciwtxhrimbmwnlyv', 'knjpwkmdwbvdbapuyqbtsw', 'horueidziztxovqhsicnklmharuxhtgrsr', 'ofohrgpz', 'oneqnwyevbaqsonrcpmxcynflojmsnix', 'shg', 'nglqzczevgevwawdfperpeytuodjlf']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('ueptpscfgxhplwsueckkxs'), ['ohhygchclbpcdwmftperprn', 'dvpjdqmqckekndvcerqrpkxen', 'lohhvarnmyi', 'zppd', 'qmqxgfewitsunbuhffozcpjtc', 'hsjbioisycsrawktqssjovkmltxodjgv', 'dbzuunwbkrtosyvctdujqtvaawfnvuq', 'gupbvpqthqxae', 'abjdmijaaiasnccgxttmqdsz', 'uccyumqoyqe', 'kxxliepyzlc', 'wbqcqtbyyjbqcgdbpkmzugksmcxhvr', 'piedxm', 'uncpphzoif', 'exkdankwck', 'qeitzozdrqopsergzr', 'hesgrhaftgesnzflrrtjdobxhbepjoas', 'wfpexx']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('ldrzgttlqaphekkkdukgngl'), ['gttlqaphekkkdukgn', 'ekkkd', 'gttlqaphe', 'qaphek', 'h', 'kdu', 'he', 'phek', '', 'drzgttlqaphekkkd']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('ololo'), ['ololo', 'ololo', 'ololo']); + +select 1 = multiSearchAnyUTF8(materialize('иечбпрхгебилцмпфвжцс'), ['лцмпфвж', 'ечбпрхгебилц', 'фвж', 'мпфвж', 'вжцс', 'пфвжцс', 'ц', 'чбпрхгебил', 'илцмп', 'фвж', 'ечбпрхгеби', '', 'б', 'хгеб', '', '', 'ил', 'ебилцмпфвжцс']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('змейдмоодкшуищвеишчддуцпх'), ['здсщесгдкзмчбжчщчиоо', 'чфззцмудщхтфрмсзрвшйщ', 'рлунбнзрфубуббдочтвлзмпгскузохк', 'ктзлебцам', 'вчспмж', 'нгкк', 'гпзйа', 'щпйкччнабакцтлапсбваихншхфридб', 'афсузжнайхфи', 'йрздеучфдбсвпжохрз', 'ошбечпзлг', 'полшхидфр']) from system.numbers limit 10; +select 1 = multiSearchAnyUTF8(materialize('лшнуухевгплвйужчошгнкнгбпщф'), ['гбпщф', 'б', 'ф', 'чошгнкнг', 'йужчо', 'гплвйужчошгнкн', 'бпщф', 'плвйужч', 'шгнкнг', 'хевгплвй', 'плвйужчошгн', 'вй', 'лвйужчошгнкнгбпщф', 'лвйужчошгнкн']) from system.numbers limit 10; +select 1 = multiSearchAnyUTF8(materialize('кцпгуоойвщталпобщафибирад'), ['ойвщталпобща', 'щта', 'пгуоойвщтал', 'ф', 'общ', 'цпгуоойвщталпобща', 'побщ', 'ф', 'цпгуоойвщталпобщафиб', 'побщаф', 'лпобщафи', 'цпгуоойвщталпобщафи', 'пгуоойвщталпобщаф', 'талпоб', 'уоойвщталпо', 'гуоойвщтал', 'уоойвщталп', 'щ', '', 'цпгуоойвщталпобщафибирад']) from system.numbers limit 10; +select 1 = multiSearchAnyUTF8(materialize('фвгйсеккзбщвфтмблщходео'), ['еккзбщвфтмблщходе', 'йсеккзбщвфтм', 'вфтмблщходео', 'вгйсеккзбщ', '', 'йсеккзбщвфт', 'бщвфтмблщход', 'ккзбщвфтмблщход', 'ккзбщвфтм', 'еккзбщвфтмблщходе', 'еккзбщвфтмблщх', 'вгйсеккзбщвф', 'оде', 'оде', '', 'бщвфтмблщх', 'б', 'йсеккзбщвфтмблщходео', 'вфтмблщ', 'кзбщ']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('хбаипфшнкнлтбшрскшщдувчтг'), ['хгшгднфуркшщвфгдглххс', 'цогчщки', 'тдмщшйзйхиквмб', 'етелфмшвмтзгеурнтбгчнщпмйпйжжциш', 'чсбк', 'ибащлшздеуревжйфуепфхкузбзао', 'дкмбщдсбжййсвгкхбхпшноншлщ', 'щхбеехнцегрфжжу', 'збфлпгсмащр', 'скчдигцнсзфрещйлвзнбнл', 'освзелагррдоортлрз', 'утхрч', 'йкбрвруенчччпшрнгмхобщимантешищщбж', 'жгивтеншхкцаргдасгирфанебкзаспбдшж', 'ййекжшщцщ', 'ефдсфбунйчдбуй', 'бвжцирзшмзщ', 'випжцщйзхнгахчсцвфгщзкдтвчйцемшлй', 'лдрфгвднеиопннтчсйффвлхемввег', 'бмтцжжеоебщупфчазпгхггцегнрутр']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('фбуоойпцщишщлбхчрсллзвг'), ['уччхщ', 'вщчсарфмйшгшпйфгмжугмщжкцщгйжзфл', 'кклл', 'лпнжирпсиуо', 'нчипзфщхнтштхйхщрпзитко', 'вйпсдергвцзсцсгмхпбз', 'чфщдфоилгцевпц', 'чааиае', 'чгингршжтчпу', 'щетбнгутшйсгмвмучдхстнбрптничихб']) from system.numbers limit 10; +select 1 = multiSearchAnyUTF8(materialize('лйвзжфснтлгбгцерлзсжфещ'), ['зсжф', '', 'бгц', 'зжфснтлгбгц', 'л', 'цер', 'жфснтлгбгц', 'тлгбг', 'це', 'гбгцерл', 'нтлгбгцерлзсж', 'жфещ', 'взжфснтлг', 'фснтлгбгцерлзсжфещ', 'нтлгбгцерлзсж', 'зжфснтлгбг', 'взжфснтлгбгцерлз', 'взжфснтлгбгце']) from system.numbers limit 10; +select 1 = multiSearchAnyUTF8(materialize('нфдцжбхуучеинивсжуеблмйрзцршз'), ['чеинивсжуеблм', 'жуебл', 'блмйрзцрш', 'цр', 'м', 'фдцжбхуучеинивсжуеблмйрзцр', 'нивсж', 'ивсжуеблмй', 'й', 'всжуеблмйрзцршз']) from system.numbers limit 10; +select 1 = multiSearchAnyUTF8(materialize('всщромуцйсхрпчщрхгбцмхшуиоб'), ['муцйсхрп', '', 'уцйсхрп', 'сщромуцйсхрпчщ', 'схрпчщр', 'сщромуцйсхрп', '', 'уцйсхрпчщрхгбцмх', '', 'цмхшуиоб', 'гбц', 'пчщр', 'цйсхрпчщр', 'омуцйсхрпч', 'схрпчщрхгбцм', 'йсхрпчщрхгбцм', '', 'пчщрхгбцм', 'уцйсхрпчщрхгбцмх', 'омуцйсхрпчщ']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('уузшсржоцчтсачтедебозцвчвс'), ['бомбсзхйхкх', 'отвгстзихфойукарацуздшгбщеховпзкй', 'мфнев', 'вйийшшггилцохнзбхрлхи', 'втинбтпсщрбевзуокб', 'оиойвулхкзлифкзиххт', 'зацччзвибшицщрзиптвицзхщхкбйгшфи', 'кнузршшднмвтощрцвтрулхцх', 'рчбкагчкпзжвтбажиабиркдсройцл', 'щргчкзожийтпдзфч', 'щбошгщзсжтнжцтлкщитеееигзцлцсмч', 'сцкк']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('щчбслгзвйдйжрнщчвфшй'), ['пдашзбалйнзвузкдвймц', 'щхтшйоч', 'фднвфигозржаз', 'рйфопхкшщвщдвл', 'цдкйхтусожпешпджпатфуиткп', 'щпбчсслгщййлвскшц', 'жпснс', 'уиицуувешвмчмиеднлекшснчлйц', 'пххаедштхмчщчбч', 'ичтмжз', 'лсбкчу', 'бгфдвпзрл', 'йицц', 'цфйвфлнвопкмщк', 'бгщцвбелхефв', 'мймсвзаелхнжйчохомлизенфш', 'трйднхндшсщмпвщомашчнгхд', 'жфцнифлгдзйе', 'зспкшщщенбцжгл', 'рщтб']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('шщпееасбтхогвгвцниуевисгшгбч'), ['гпа', 'стимсркзебхрвфпиемзчзу', 'нзгофухвекудблкадбшшусбеулрлмгфнйгиух', 'кфиашфобакщворувгвкчавфзшх', 'гфпгщгедкмтгрдодфпуйддхзчждихгрчтб', 'тцтжр', 'рцйна', 'йцбпбдрреаолг', 'житсфосшлтгсщдцидгсгфтвлз', 'жвтнжедцфцтхжчщч']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('вхкшгфпфмнщаохтмизпврйопцуйзмк'), ['дтчбкхащаткифружжейабфйкйтрскбощиеч', 'фтоуабхмдааиснрбраттклмйонлфна', 'цадзиднщймшкщолттпгщбх', 'кштбчжтждпкцнтщвмухнлби', 'микудпдпумцдцгфахгб', 'ирик', 'емлжухвмк', 'чгуросфйдцшигцхжрухжпшдкфгдклмдцнмодкп', 'ттбнллквдувтфжвчттжщажзчлнбждчщцонцлуж', 'елцофйамкхзегхклйгглаувфтуувее', 'двкзчсифвтекб', 'шсус']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('йхцглкцвзтшщочпзмнчтуеао'), ['йечдай', 'дащжщзлосмй', 'афуккгугаазшрчпцнхщцтмлфф', 'чфтфскрфйщк', 'жлччкцшнфижтехппафхвщфс', 'бзжчв', 'щкщймнкщлпедидсу', 'оцбажцзшзйпптгщтфекртдпдзшодвойвох', 'йжддбссерхичгнчлкидвгбдзуфембрц', 'ктщвшкрщмдшчогхфхусдотсщтцхтищ', 'пшстккамнбнардпзчлшечхундргтоегцзр', 'нсрнфузгжррчнжначучиелебрб', 'шгжмквршжтккднгаткзтпвкгзхшйр', 'змквцефтулхфохбнхбакдичудфмйчп']) from system.numbers limit 10; +select 1 = multiSearchAnyUTF8(materialize('шждйрчйавщбйфвмнжоржмвдфжх'), ['ор', '', 'йрчйавщбйфвмнжо', 'вщбйфвмнжорж', 'ждйрчйавщбйфвмнжорж', 'йавщбйф', 'дф', 'вщбйф', 'бйфвмнжорж', 'мнж']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('кдшнсйршгвлицбенйбцфрсаччетфм'), ['асмун', 'йогкдчодиф', 'лштйбжнзфкикмпбитжшгкбоослщгзнщо', 'улштжцисцажзчштгжтфффабйлофедуфме', 'дрпгкчджихшзммймиамзфнуиорлищзгйвху', 'йиоршнйоввквбдвдзасма', 'члмвасмфрхжсхрбцро', 'лшкизщушборшчшастйсцкжцбонсшейрщ', 'масдфкршлупасвйфщфважсуфсейшзлащхрж', 'дгхшщферодщцнйна', 'цзфзждбавкжрткст', 'рфбожзееаце', 'кошомвгпрщсдквазчавожпечдиуйлщадфкгфи', 'бшпхнхсгшикеавааизцсйажсдийаачбхч']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('хтиелйтарквурйлжпеегфш'), ['зпмйвзуднцпвжкбмйрпушдуавднвцх', 'фбссчгчвжакуагдвижйтщтшоабпхабжш', 'щхшибаскрщбшрндххщт', 'сммрсцзмптисвим', 'цсргщфж', 'восжбшйштезвлкммвдхд', 'вбсапкефецщжквплуо', 'даеуфчвеби', 'бтптлжпин', 'шчддтнсйкщйщ', 'фжхщецпзчбйкц', 'цсвфпздхрщхцбуцвтег']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('апрчвзфжмбутццрйщкар'), ['индхжз', 'жилцовщччгстби', 'ажс', 'фктйамйтаг', 'шммнзачггоннксцушпчн', 'чдлйтзтоцдгзццисц', 'пнбтувщцдсчнщмсакрлгфмгрй', 'овмсйнщзушвщгуитщрхвйодф', 'бзлштезвлаижхбмигйзалчолшеунлц', 'фкжпеввгшгащз', 'тменбщжмсхщсогттршгек', 'чап', 'х', 'шкомегурлнйпшбщглав']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('двхопооллаеийтпцчфжштнргкк'), ['йймчнздешхбццбжибопгктрнркевпиз', 'фйрохсамщцнмф', 'ййхфдпецжзгнуорвбплоахрфиле', 'пкллкацнвдббогг', 'йщдезамтжйзихщжмцлх', 'гдзувмщиеулиддердшпитвд', 'фхтунйшзхтщжтзхгцорошднпбс', 'фнситбеелцдкйщойлатиуухгффдвищсше', 'нзщщщндцрнищпхйвтбвмцтнуадцбву', 'вбщкапшнв', 'зйлмуимчскщнивтшлчмуузщепшйр', 'шжбвйдр', 'гддждбкначдттфшжшхпфиклртпгм', 'еншащцфафчнгбнщххнзочбтпушщорегшцзб', 'уунеущкззоетбучкц', 'щасифзоажребийещ', 'пщбххсдгйтт', 'хшсчуотрт', 'жкднйрозбцшужчшбкккагрщчхат', 'шачефцгч']) from system.numbers limit 10; + +select 0 = multiSearchAnyCaseInsensitive(materialize('QWyWngrQGrDmZxgRnlOMYHBtuMW'), ['ZnvckNbkeVHnIBwAwpPZIr', 'NCzFhWQmOqIGQzMORw', 'tDYaxfQXWpKNLsawBUUOmik', 'IMveCViyAvmoTEQqmbcTbdfjULnnl', 'NRvsdotmmfwumsDpDtZU', 'mnqVnwWOvMiD', 'HXpHrMvGQpbuhVgnUkfFPqjpoRdhXBrFB', 'awtr', 'IMIdOmMHZccbOZHhWOKcKjkwwgkJSfxHDCzR', 'jPLISbIwWJEKPwgvajTxVLws', 'HBfRrzEC', 'VXsysGnAsFbqNOvIaR', 'upCaeaIOK', 'GUDFkrzBiqrbZVnS', 'MoCOePXRlVqCQpSCaIKpEXkH', 'rfF', 'fjhMEpySIpevBVWLOpqi', 'KdeskLSktU', 'vjUuNUlBEGkQyRuojZLyrmf', 'SvSxotkTKCeVzNICcSZLsScKsf']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('gcDqqBCNqhQgVVgsxMXkevYIAxNl'), ['BHnoKRqOoKgmOVkjtehGSsInDvavDWOhkKAUL', 'nYqpmKPTWGdnyMcg', 'TIplHzsSXUz', 'SiQwpQgEdZ', 'YoJTWBJgsbJvq', 'CwyazvXERUFMCJWhTjvltxFBkkvMwAysRLe', 'tXUxqmPbYFeLUlNrNlvKFKAwLhCXg', 'vUbNusJGlwsOyAqxPS', 'ME', 'ASUzpELipnYwAknh', 'VtTdMpsQALpibryKQfPBzDFNLz', 'KmujbORrULAYfSBDyYvA', 'BaLGNBliWdgmqnzUx', 'IzwKIbbSUiwhFQrujMgRcigX', 'pnS', 'UKSZbRGwGtFyLMSxcinKvBvaX']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('HCPOPUUEVVsuZDbyRnbowGuOMhQ'), ['UzDbYrNBoWgUo', '', 'pUUEVVsUzdByrNB', 'nBO', 'SUZdbYrNbOWgUoMH', 'pOpuUevVSUZDbYRnb', 'bowGUoMh', 'VsUZDbyrNbo', 'suzdBYrN', 'uueVvsUZDBYRnBoW', 'gUom', 'eVvsuzDBYRNBoWgUOM']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('RIDPJWYYSGBFWyXikHofbTcZAnj'), ['aFxQyVe', 'OcnZBgPsA', 'iBQaH', 'oesSvsWtgQprSSIPaDHdW', 'EfytiMfW', 'qHiFjeUvQRm', 'LfQkfmhTMUfoTOmGJUnJpevIoPpfpzMuKKjv', 'scYbCYNzJhEMMg', 'yTLwClSbqklywqDiSKmEdyfU', 'HYlGFMM', 'TMQhjOMTImXbCv', 'AVtzpxurFkmpVkddQANedlyVlQsCXWcRjEr']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('GEsmYgXgMWWYsdhZaVvikXZiN'), ['wySd', 'smYgxGMWWYsDHZ', 'vIk', 'smyGxgmwWysDHzAvvikxZi', 'WYsdHZAvVI', 'YGxGmwWYSDhzavvI', 'XzI', 'ySDhZAvvIK', '', 'myGXgmwWySdHz', 'MYGxgmwWySdHZaVvik', 'wYsDhzAvvikXz', 'wwYsdHzav', 'Z']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('XKCeCpxYeaYOWzIDcreyPWJWdrck'), ['tTRLUYJTkSWOabLJlIBshARIkwVRKemt', 'jQgn', 'wdNRsKIVunGlvwqkwn', 'BsbKGBJlkWQDBwqqeIjENvtkQue', 'yLuUru', 'zoLGzThznNmsitmJFIjQ', 'WFKnfdrnoxOWcXBqxkvqrFbahQx', 'QHbgRXcfuESPcMkwGJuDN', 'NPqfqLS', 'bi', 'HnccYFPObXjeGYtrmAEHDZQiXTvbNcOiesqRPS', 'KobVCJewfUsjBXDfgSnPxzeJhz', 'AqYNUPOYDZjwXx', 'xbZydBGZFFYFsFHwm']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('AnIhBNnXKYQwRSuSqrDCnI'), ['', 'HBNNxkyqWRS', 'xKyqwrSUSQR', 'yQwr', 'ihbnnxKYQWrsUS', 'bnnXkYqwrSuS', 'qWRs', 'nXKyqWRSUS', 'qrdcN', 'NiHBnNXkYQWrS', 'NnXkYQwRSUsqRDCn', 'rSusqRd']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('OySHBUpomaqcWHcHgyufm'), ['lihJlyBiOyyqzeveErImIJuJlfl', 'WyfAXSwZPcxOEDtiCGBJvkCHNnYfA', 'hZ', 'fDQzngAutwHSVeoGVihUyvHXmAE', 'aCpcZqWKdNqTdLwBnQENgQptIyRuOT', 'PFQVrlctEwb', 'ggpNUNnWqoubvmAFdjhLXzohmT', 'VFsfaLwcwNME', 'nHuIzNMciJjmK', 'OryyjtFfIaxViPXRyzKiMu', 'XufDMKXzqKjYynmmZzZHcDm', 'xWbDgq', 'ArElRZqdLQmN', 'obzvBzKQuJXZHMVmEBgFdnnQvtZSV', 'ZEHSnSmlbfsjc', 'gjmWPiLylEkYMTFCOVFB']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('NwMuwbdjhSYlzKoAZIceDx'), ['ZKOaZ', 'wBDJhsYlZKo', 'hSy', 'MUwbDjHsyl', 'sYlzK', 'ylZKOAZ', 'y', 'lZKoaZICEdX', 'azIce', 'djHSylZkoAzice', 'djHsYLZKoAzi', 'dJHSYlZK', 'muWbDJHsYLzKOaziC', 'zi']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('gtBXzVqRbepHJVsMocOxn'), ['DidFXiqhRVBCHBVklLHudA', 'yEhumIpaYXlj', 'iaEmViTRLPM', 'vTwKBlbpaJZGYGdMifOVd', 'zvgfzWeLsMQNLutdAdCeuAgEBhy', 'Ca', 'iHabiaRoIeiJgSx', 'EBfgrJnzHbuinysDBKc', 'kT', 'SGIT', 'BTRuKgHDuXMzxwwEgvE', 'OWJIeTLqLfaPT', 'BQM', 'yMimBqutKovoBIvMBok', 'zIBCYVNYAwu', 'EFDEFWGqvuxygsLszSwSiWYEqJu', 'QJDIXvPOYtvhPyfIKqebhTfL', 'ssALaXRxjguUIVKMCdWRPkivww']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('MowjvqBkjnVTelCcXpoSuUowuzF'), ['Su', 'vqBkJNvTelC', 'Elccxp', 'vtElc', 'JVqBkJnVTELCcxpOsU', 'OsUuOWUz', 'ElccxPoSU', 'wJVQbkJNVtElCC', 'xpOSUUo', 'VQbkJnvTELCCXp', '', 'TeLcCxPOsuuO']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('VfVQmlYIDdGBpRyfoeuLffUUpMordC'), ['vqMLyIddgBPrYFoEulFFu', 'lyIDdgBPrYFOeul', 'dGBPRYFOeUlffUupmOrD', 'OEulffU', 'pMordc', 'FVqmlyiDdgBpRyFoeUlFfuUpMOrD', 'PmO', 'o', 'YiDDgbPRYFOe', 'DGBPryfoeU', 'yIDdgbpRyFOeULfFU', 'lyIddgBPryfoeulfFuU', 'gbPrYfOeUlFfuupmO', 'yFoeULF']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('CdnrzjzmwtMMPLjgcXWsbtrBs'), ['RfgIUeerlPIozKpRQR', 'QRoYzjZlgngJxX', 'mEbqlBIzTQH', 'UmrfJxKyTllktPfyHA', 'ukoZeOPA', 'pbbRaUcJijcxt', 'Rg', 'lSBG', 'HvuwuiqVy', 'Fo', 'aGpUVjaFCrOwFCvjc', 'zKhfkgymcWmXdsSrqAHBnxJhvcpplgUecg', 'ioTdwUnrJBGUEESnxKuaRM', 'QciYRCjRDUxPkafN']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('miTQkQcxbKMwGOyzzRJpfXLyGx'), ['yMwgQQJkeshUugm', 'wGVe', 'XncShWqjp', 'KWjGQCOsfMKWRcgCfebkXZwZ', 'SFWbU', 'WdFDMIcfWeApTteNfcDsHIjEB', 'XRuUJznPOCQbK', 'tibBMGZHiIKVAKuUAIwuRAAfG', 'VVCqVGGObZLQsuqUjrXrsBSQJKChGpZxb', 'bWYAOLuwMcwWYeECkpVYLGeWHRrIp', 'SLzCgfkRWmZQQcQzP', 'VvfOhFBhfiVezUSPdIbr']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('KXoTIgVktxiXoEwfoLCENiEhz'), ['oLCENie', 'xix', 'en', 'IgvktxIXoEWFOLCEnieHz', 'xOEWFoL', 'LC', 'ktxIxoEwfolCenie', 'ce', 'oTIGvktXIXOE', 'eW', 'otigVKTXIXOEwFolC', 'E', 'CEni', 'gVKtxIxoEwfOLCENieh']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('DXKzSivrdLuBdCrEYfMEgPhOZ'), ['', 'sIVRDlUBdcr', 'luBDcrE', 'rDLUbDCreY', 'KzSiVRdLuBDCr', 'dcREYFme', 'lUbdCReyFMEgph', 'sivrDlubdCr', 'BdcreYfMEgP', 'ZSiVrdluBDCrEYfmegpHOZ']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('lTvINMXVojkokvNBXPZOm'), ['ZQOJMEJfrjm', 'vIpmXnGlmWze', 'wbdDKcjrrIzBHypzJU', 'omotHOYbZjWfyVNeNtyOsfXPALJG', 'SXxu', 'yZPDFsZq', 'OVYVWUjQDSQTKRgKoHSovXbROLRQ', 'RnXWZfZwHipewOJimTeRoNRYIdcZGzv', 'sizoEJibbfzwqFb', 'vgFmePQYlajiqSyBpvaKdmMYZohM', 'ENsFoFCxDQofsBSkLZRtOcJNU', 'nG']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('LsTqxiGRdvQClVNBCGMOUHOAmOqPEC'), ['NdFuUQEUWaxS', 'fdOHzUzineBDnWJJvhPNZgB', 'rYAWGIBPxOLrjuquqGjLLoIHrHqSFmjh', 'IVgYBJARY', 'ToivVgUJAxRJoCIFo', 'yQXGrRjhIqFtC', 'PNYdEPsWVqjZOhanGNAq', 'nrQIDDOfETr', 'usJcPtiHKhgKtYO', 'vPKqumGhPbmAJGAoiyZHJvNBd', 'eXINlP', 'WQeESQJcJJV']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('gRzzQYOwLNiDcMFjXzSFleV'), ['XZSfLe', 'wLnIdcMFjxZSf', 'F', 'm', 'Le', 'qYoWLNidcMFjXzsf', 'zqyoWlNIdcMFj', '', 'oWlnIDCMfJxzsfL', 'wlNIdCmfjXzS']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('cYnMXJMJCdibMXoUQHEw'), ['BFrGFZRgzwHGkUVbBiZMe', 'piORdVIWHMBsBDeJRLbGZAHGBrzNg', 'bmDePbTPnFQiCFfBJUxAEYNSbgrOoM', 'gtzeAGwqjFrasTQUgAscfcangexE', 'okLG', 'l', 'EBkkGYNZZURgFgJPlb', 'HDQVngp', 'vEHhtBqWhZHCOrqEKO', 'fgqdFc', 'COig', 'VftTpSXAmTmvnShHJqJTdEFcyKPUN', 'WDI', 'knBm']) from system.numbers limit 10; + +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('мтдчЛВЖАгвзщущвкфИКмТбжВ'), ['щУщвкФИкМ', 'чЛвжАГвЗЩуЩвКФикм', 'ДчлвЖАГвзЩУЩвКфИКМтБЖВ', 'ЖагвзщуЩВКФикМТБжВ', 'ВжагВзЩУ', 'гВЗщущвкфИКмТБж', 'ГвЗщ', 'щВкФикМТБЖВ', 'вЖАГВзщущ', 'взЩуЩвКФИкМТ', 'ЧЛВЖагвЗщуЩВк', 'тДчлвЖагвзЩуЩвкфИк', 'ТДЧлвжаГВзЩущВ', 'тДчлВжАГВЗЩУ']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('дтрцФхИнпиОШфдАгзктвбУвсб'), ['чТрВиУРФсРпДЩОащчзЦНцхИДА', 'ЗжмПВтмиойУГхАЦПиДУЦноНуййЩХаФТофшЩ', 'уБшлОЙцМПгетЖЧетШжу', 'ЧзИАУХобФрачТеХОШбМщЖСамиМВАКРщАЦ', 'ВйвТзхЙФЧоАЖвщиушАз', 'ЦшИфххкжиФйСЛЛНЛчВоЙВПпхиИ', 'ОатЕтщкЦпбСБйцОшГШРОшхБцщЙЧиУЩЕеФлщ', 'цСПпЧА', 'ШЧНфПмФсКМКДВЦАоФчОУеТЦИзЦ', 'зАбдЛНДГИ', 'фхЩлЗДНСсКЖИФлУАбЛеТФЕпЖлпПхЙиТЕ', 'иВшкНслТКМШЗиДПйфвйНкМЛхеФДзИм', 'лпушПБванпцев', 'ЧОшЧЧмшЦЛЙйГСДФйЛАв']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('нщМаНдЧЛмиВврПокПШмКБичкхвРГ'), ['АЙбаЙйШЛЙРЦмЗчВеИЕощсЦ', 'щЦФдВжчТСЩВКЦСпачЙсумщАтЩувеиниХПДоМС', 'иоАкДРршуойиЩищпрфВаЦПж', 'еЖПйШкГжЧтоГЙМВ', 'ЩПалиБ', 'ТвВлт', 'оХжйЛФеКчхЗВвЕ', 'ерцЩ', 'ШХЖОАрзеп', 'ККМрфктКГишпГЩхаллхДиВИИЛЗДеКйХмжШ']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('вШЙчоМгОттЧАЕнЧаВеЦщчЧошМУ'), ['ЧОмГотТчАЕН', 'ОмГотТчАЕнчАвецЩчч', 'ЧАВецЩч', 'ТЧАеНЧаВ', 'ттчаЕнча', 'ТчАЕ', 'мготтЧАенчавЕЦЩ', 'НЧаВец', 'тТЧаенчАвецщчЧошм', 'Ав', 'ТЧаЕнчавецщчЧоШму', 'аЕнЧав', 'АеНЧав', 'шйЧомГОТТчаЕнчАВЕ', 'шйчоМгОтТЧаЕНчаВеЦщЧчош', 'МУ', 'ошМ', 'гОТтЧаеНЧА']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('фйадзЧмщЖШйЖЛшцГигцШ'), ['НТХеМРшДНУЗгадцуЧИ', 'жпСИКЩМлНлиоктлЦИвНЛ', 'КхшКРчХ', 'кгТЗаШИарХЧЛЖмСЖм', 'ОмиЛй', 'жЕРбФЩНуЕКЕАВоБМОнАЕнКщшзйПкОЗ', 'гиЗдадкбжХМЗслшВИШай', 'двтЗйЙНгПуТзД', 'ТНкмаВЕФ', 'Шеа']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ШЕшхмеЦХеАСКощеКИфлсТЧИЗЛ'), ['КифЛсТ', 'ХеаСКощЕк', 'КифлсТЧ', 'шХМеЦхЕаскОЩеКИ', 'ЕшхмЕцХеаСК', 'ХЕасКоЩ', 'чИ', 'ЕцхеАсКОЩек', 'ЩЕкИфлс', 'асКощЕкифЛсТ']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('шоКнВЕрОЖЛпУйХзСугКПВжиРсЙпо'), ['игВербфНахчжЙггч', 'лтимрдфЕг', 'нкеаЖАшНБвйСдКИВГДшАГиАТнФШ', 'МжсТЙМГОииУКВГнцткДнцсоАд', 'ХтпгУСдБдцАЖЛАННоЕцзЕшштккз', 'ншУЦгФСЖшмс', 'нЩшМ', 'гоЖхМшаЕмаДРЧБЛИТпмЗОоД', 'фГКШхчФбЕГЛйкчПИЙххуМГНШзхг', 'ХпХщПЦАзщтг']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('кЧбоЗХфвБХхусмШгНаШШаГзш'), ['Ури', 'лММшткфНзцЦСВАдЩПМШфйОМшефигЖлуЕП', 'сМтЕдчЦафйСТЖЗфлРЙПЦдипжШскцВКХЦЖ', 'АУкжИФцшЛБЦЧм', 'ФПлнАаДСХзфоХПСБоСгМТОкЗЧйЛ', 'ЦшСГЛрцДмнНнХщивППттжв', 'жзЕгнциФ', 'МШЛсЙЧтЛАГжд', 'уИиЕжцоРНх', 'ЧбйГуХтшОНкрЧИеПД', 'ЦдЩЕкКвРЦжщЧциекЗРйхрббЖуЧ', 'иВжен', 'ГчОржвБГсжштРЕБ', 'ШоЖдуЙфчсЧегумщс', 'йчЙГ', 'РДедвТ']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ткРНбЩаРкгГчХшецИкНЕнСЖкйзАуУЖ'), ['ХлЖхУИллрРННйЗйсРуШЧвМбЧЧщфФЦц', 'СЛчКБцСФДшлфщаФлЙСзШабмбхуБжТСТ', 'УКУиввЗЩуВМцпчбпнДГбпЕЖрПбИДркМРОеЧмЧдГ', 'ПчщвШЩвГсЛмММГБ', 'хКЦЧсчжХЩИЖХеНнтоФЦлнмЛЧРФКпмСшгСЧДБ', 'удсЗйУДНЧУнтЕйЦЗЖзВСх', 'хПЖЙИрцхмУкКоСмГсвПаДОаЦНЖПп', 'сВОей', 'ЩЦжщоабнСгдчрХнЩиМХзжЩмФцррвД', 'ЦИсйнЦДоЕДглЕЦД', 'жзйПфБфУФоцзмКЩГПЧХхщщПТпдодмап', 'ДНХГНипжШлСхХхСнШЩЛИснУйЧЩЖДССФфиС', 'ОйЩНнйЕшцФчБГЛвхЖ', 'КЧРВшИуШйВфрпБНМсУмнСЦРпхЗАщЗУСвЧйБХтшХЧ', 'зЛбНу', 'ЗСрзпшЕйРржПСсФсШиМдйМЦГхдйтРКЩКНцкбмгС', 'СУццБуКнчОищГ', 'уЕГЧлЗБНпУисЕЛ']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ВЦХсЖЗЧЙБЗНбРитщстеМНжвВ'), ['итщст', 'ЧйБЗНбрИтщстЕМнЖ', 'ХСЖЗЧйбзНБриТщ', 'Темнж', 'сЖзЧЙБзнб', 'хСжЗчйБзнБрИтЩстЕм', 'БзнБРиТщ', 'ЗчЙбзНбрИТщ', 'чйбЗНбри', 'зЧйбзНБРИ', 'нБРитщсТе', 'зНб', 'цхСжзчйБЗнБРИТЩСтЕм', 'жЗЧЙБЗнбрит']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ХцМШКАБАОххЕижгГХЩГиНциД'), ['ОРАБЕРВомЛфГНМИКупбхЛаАкЗдМзтш', 'лЗУЩнлбмиЛАфсгМРкцВтлснййишИНАС', 'ТлжлУоУгжукФжЖва', 'жоСШПоУНЩшРМГшОЛзЦБЛиЛдТхПДнфжн', 'чнСУЗбДаГогжДфвШКеЙПБПутрРпсалцоБ', 'ЙозоПщчакщаАлРХбЦгац', 'иаИСсчЙЧБШорлгЧТнчцйзоВБХбхЙФтоЩ', 'ПСзсБЗЕщурфДЛХйГИеПНрмииаРнвСФч', 'ЦйЖЕуТфЖбхЩМтйсЙОгЛбхгтКЕЩСАЩ', 'гтЗуЩлужДУцФВПЛмрБТсСНпА', 'тГвлбчЗМасМЖхдЕгхмЩксоЩдрквук', 'ВРаг']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('тУйВЖдНнщцЗЖфКгфжГфиХСБЕЩ'), ['КгФЖГФи', 'сБе', 'ЖФ', 'гфжгФИхсбе', 'ВЖДНнщЦзжфКГфЖгфИхсбещ', 'ВЖДНнЩЦзжфкГ', 'вЖДННЩЦзжФКГфЖгФ', 'ф', 'НщЦЗж', 'нщЦЗЖФк', 'Их', 'дННщцзЖФКгф', '', 'нщцзжФкг']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ШкКРаоПеЗалРсТОиовРжгЙЧМКЛШ'), ['рчсажЕК', 'пЧТМфУрУММждЛйжзУрбкмам', 'бАШеНмВШзлзтушШШсхОсцрчЙПКИБнКжфЧЕХ', 'ЖМЛшбсУМкшфзочщАЖцМбмШСбВб', 'гтРХсщхАИОащчлИЧуйиСпСДФПбРл', 'ЧуОРУаоойГбУппМйЩФДКПВ', 'уУпугйРЕетвцБес', 'ЙЖЦТбСЖж', 'ИБКЛ', 'ТДтвОШСХГКУИПСмФМтНМзвбЦрднлхвДРсРФ', 'вВгНЙХИрвйЕЗпчРГЩ', 'ПчмТуивШб']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('РлчгхзуВШежХЦуМмнВЙщдцО'), ['ХшвМЦДШпЩОСшЦПдруа', 'ФИЦчУвРкпнПшИЕСЧАувиХд', 'фшвбЦОИЗфпИУМщзОЧЗфВцЙПнмтаТгг', 'мЖЩйавтнМСЛ', 'НВбШ', 'ааФДДрВвЙТдПд', 'ЗнчЧущшхЙС', 'рзуСзнеДфЩПуХЙЕл', 'ШСЩсАгдЦбНиШмшКрКс', 'ггнЕфБГзрОнАГЙзЧеИП', 'вшТИпЧдЖРкМНшзпиоиЩчзДмлШКТдпЦчж', 'фЦТЙц', 'ОтУшмбптТКЗеПлЧцЛОкЩБпккфгИн', 'ЩпвхпЗлШБЦ']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ЙбйнхНщЧЖщчГОАпчФнЛШФбгЛа'), ['щчг', '', 'апЧфНЛШфб', 'ЙнхНЩЧЖщчгОАПЧф', 'ХНщЧжЩЧгоАпч', 'ХНщЧжщчГо', 'нщЧжщчГОа', 'чЖЩЧГоапЧФНл', 'оапчФ', 'щЧГОАпЧФНлшФ', 'ЩЧГОАпЧФНЛшфБг', 'БЙНхнщчЖщчГоаПЧФНЛШФБгЛ', 'ОапЧфн', 'ф', 'БглА', 'ш', 'шфбГ', 'ХнЩЧЖщчГоА', 'ХНщчжщЧгоапч', 'хНЩчжщЧГоапчфнлшФбгЛ']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('кдЙДТЩеВЕфйКЩЦДиКБМф'), ['щфЛ', 'фЧЩЩичрКйЦКхеИИАпоБВЙЗбДАФио', 'мИтиЦРоВЙсБбСлНзиЛЧОфФевТмижщК', 'тЙгнКШфНТЕБЛцтГШЦхШхБ', 'уаабРГрМЙпМаБуЗпБЙчНивЦеДК', 'мпВЛНДеКПУгРЛЛинзуЕщиВШ', 'ЩжКйШшпгллщУ', 'пршЙПцхХЗжБС', 'нбЗНЙШБш', 'йцхИщиоцаМРсвнНфКБекзЛкчТ', 'хсмЦмнТрЩкДТЖиХщцкЦМх', 'ГмЛАбМщЗцЦйаОНвзуЗмЕКПБЙмАЕЛГ', 'ОЦХРЗРмкжмРИЖИЙ', 'з', 'лЕТкпкдЗчЗшжНфо', 'ИТПфйгЖЛзУТсЩ', 'ОфрбЛпГА', 'МЖооШпЦмсуГцАвМЕ']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ЩГТРШКИОРБРеНЖПКиуМОкхЛугИе'), ['брЕнЖ', 'РбрЕНЖпКиУМокХЛу', 'ГТрШКИорБРеНЖпКиУМ', 'рШКиоРбрЕнЖпкИУМОК', 'ИорбрЕнЖПК', 'Окхл', 'шкИоРБРеНЖПк', 'ТРШкИоРБрЕнжПКИУМОкхл', 'КИОРБРЕнжпкиУм', 'Н', 'КиОРбРЕнЖпкИУмоКхл', 'к', 'ГтРшКИоРБРЕнЖпк', 'гтрШкиорбрЕНЖпк']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ШНвпкфЗвгДжУЙГлрТШаШЛгНЗг'), ['нЗБенВшщрЛАрблцщшБАдзччммсцКЖ', 'бЗЩхзЗЗбФЕйМоазщугБбмМ', 'рЙсВжВсхдйлЩгБтХлчсщФ', 'пиБшКРнбВБгЕуЖ', 'жПшнхпШзУБрУЛРНЩДиаГШщКдЕвшоуПС', 'чЕщкЗмДуузуСдддзгКлИнгРмЙщВКТчхзЗЛ', 'кЖУЗЖС', 'щххОВМшуажвН', 'фбцЖМ', 'ДШитЧЩДсйНбдШеООУдг', 'ЛХПфБВХЦТИаФПЕвгкпкпщлхмЙхГбц', 'чЦсщЗщрМ']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ФРХгаСлчЧОцкШгзмКЗшФфББвЧ'), ['кзШфФб', 'ГАслЧЧОцкшг', 'ФфббВЧ', 'ЦкШ', '', 'АслчЧОЦКШгзМкЗШффбБвч', 'РХгаслЧчОЦКШГз', 'РхгаслчЧОцКШгзМкзшФфБбВ', 'Шг', 'Ф', 'ХГАслчЧоцКШГзМкзш', 'ШгЗмКЗшфФб']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ЧдйШкхОлалщНйбССХКаФзОМрКЕЙР'), ['бссХкафзОм', 'ХОЛаЛщнйБссХкаФз', 'лаЛщнйБсСХ', 'ЩнЙбСсхКаФЗО', 'йБСсХКАФЗОмР', 'йшкХолаЛЩНйбсСхК', 'С', '', 'ЙшкхОлалщНЙБсСхКаФзом', 'Йр', 'щнЙБссхКАфзоМрК', 'рКе']) from system.numbers limit 10; + +select 1 = multiSearchFirstIndex(materialize('alhpvldsiwsydwhfdasqju'), ['sydwh', 'dwh', 'dwhfdasqj', 'w', 'briozrtpq', 'fdasq', 'lnuvpuxdhhuxjbolw', 'vldsiws', 'dasqju', 'uancllygwoifwnnp', 'wfxputfnen', 'hzaclvjumecnmweungz']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('kcwchxxyujbhrxkxgnomg'), ['jmvqipszutxfnhdfaxqwoxcz', 'nrgzkbsakdtdiiyphozjoauyughyvlz', 'qbszx', 'sllthykcnttqecpequommemygee', 'bvsbdiufrrrjxaxzxgbd', 'hdkpcmpdyjildw', 'frxkyukiywngfcxfzwkcun', 'dmvxf', 'esamivybor', 'eoggdynqwlnlxr']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('zggbeilrfpkleafjjldgyfgn'), ['rpypxkpgvljhqthneremvabcd', 'qchzlnsctuwkdxqcrjgihvtfxhqxfqsxm', 'vtozkivjyqcqetmqenuihq', 'fixcvjyzbzejmwdivjf', 'lydoolvnuuamwlnzbyuuwpqqjaxf', 'elkodwthxqpcybwezm', 'wpiju', 'wdzuuwumlqfvga', 'iokphkai', 'wkbwdstplhivjyk', 'wxfbhfturuqoymwklohawgwltptytc', 'jehprkzofqvurepbvuwdqj']) from system.numbers limit 10; +select 9 = multiSearchFirstIndex(materialize('bwhfigqufrbwsrnnkjdzjhplfck'), ['v', 'ovusuizkdn', 'ttnsliwvxbvck', 'uh', 'lfourtjqblwdtvbgtbejkygkdurerqqdwm', 'snmtctvqmyyqiz', 'ckpixecvternrg', 'gluetlfyforxcygqnj', 'igqufrbwsr', 'om', 'huwazltjsnohsrcbfttzwyvcrobdixsuerkle', 'gqufrbwsrnnkjdzj', 'hfigqufrbwsrn', 'lhhyosbtznyeqzsddnqkfxayiyyajggxb', 'igqufrbwsrnnkjdzjhplf', 'pl', 'jtbqaqakbkesnazbvlaaojppxlbxccs', 'gqufrbwsrnnkjdz']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('yevfiumtjatfdnqixatbprvzwqlfgu'), ['ozghvskaixje', 'vmdrvdjhwxdvajmkpcxigsjzmtuhdxgllhzrpqd', 'qfhnxpcmtzpociajidwlcvobjfyxfcugsxy', 'pgamvhedjibcghinjrnowqzkfzibmfmh', 'bcmrdzpcczhquy', 'czosacvwfsbdvwwyirpvbve', 'qu', 'fdkobwlnmxbpvjkapextlbcrny', 'bqutjqobkyobhtpevjvewyksnoqyjunnnmtocr', 'kjlgff', 'oitltmhdburybwfxrjtxdiry', 'kiokuquyllpeagxygqugfmtm', 'wlbkl', 'khubpmstqjzzjzmsvfmrbmknykszqvue', 'lqrbmyndsztyrkcgqxcsnsanqjigimaxce', 'nitnyonuzedorrtkxhhgedohqcojbvtvjx']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('wmvuoeqphsycrvtxghrcozortmdnh'), ['hv', 'ugcmpebvlzgdtcmgkbgzyfel', 'qvmofayljsvybupvvnbhhibsz', 'zvlihxmyxlxwbffwjzjrfjgimmltftqqre', 'mwassqvxptav', 'jrumvqzkiaewngoufhrleakcfrsaxhpxyg', 'sxlxwhvkpavgfhxrxcbnqbstyrejtosxwe', 'psnlqakyfhcupryqatrmwqlswwjylpaiqammx', 'ivozojwldsgtnxpvsi', 'epyzjs', 'legi', 'sdqxxahfbddhacqrglgdcmlslraxfaahhfyodon']) from system.numbers limit 10; +select 12 = multiSearchFirstIndex(materialize('lebwdwxfdzwquhqhbvmte'), ['mwhruilzxvlyrgxivavxbbsq', 'ubuiizuasp', 'xpkzcsf', 'qpeqitoqqqeivohajzhmjbo', 'kbftixqmqgonemmbfpazcvf', 'iyhluioqs', 'hws', 'tupfdksgc', 'ows', 'pngzkoedabstewcdtdc', 'zdmyczldeftgdlwedcjfcoqycjcivf', '', 'xt', 'syuojejhbblohzwvjzzedzgmwc']) from system.numbers limit 10; +select 7 = multiSearchFirstIndex(materialize('wcrqaoecjwkhnskrbahqxfqgf'), ['qegldkdmyaznlmlhzvxfgoukngzbatnuq', 'khgcvgrifwtc', 'hkwcpogbbdqulizrycmneqmqynvj', 'zkqjf', 'xfduxyy', 'ructdekcoywfxsvpumfefoglljptsuwd', 'wkhnskrbahq', 'crqaoecjwkh', 'ikmpbunpguleinptzfelysiqc', 'lhldcci', 'nooepfypkoxxbriztycqam', 'uxeroptbiqrjartlnxzhhnlvjp']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('psgkkcwttitgrjsobiofheyohadu'), ['achfrepey', 'minlzeiwgjfvvmhnevisky', 'oxfghfdthtyczzveppcoxrued', 'ydhaupodnezvxhcqahfkwtpvxnymriixf', 'slxsbxidylxyurq', 'socyyabwbjdabnuqswrtjtqogirctqsk', 'lvbnacirctyxxspjmispi', 'oj', 'ihmmuuqlosorrwhfxvpygfrzsqpmilcvjodmcz', 'idmtmemqfyrlbwhxz', 'hsqfsfdzvslwbtlwrfavez', 'gszl', 'ei', 'pnywjnezncpjtyazuudpaxulyv', 'iqgavdjfqmxufapuziwwzkdmovdprlhfpl', 'yigk', 'mjidozklrpedutllijluv', 'vixwko']) from system.numbers limit 10; +select 3 = multiSearchFirstIndex(materialize('xtjxvytsseiqrpkbspwipjns'), ['bwmoghrdbaeybrmsnucbd', 'zoslqabihtlcqatlczbf', 'sseiqrpkbspwipjn', 'mdnbzcvtayycqfbycwum', 'npueimpsprhfdfnbtyzcogqsb', 'ytsseiqrpkbspwipj', 'fzvhcobygkwqohwutfyauwocwid', 'naacyhhkirpqlywrrpforhkcjrjsnz', 'vezbzderculzpmsehxqrkoihfoziaxhghh', 'mvvdfqzskcyomjbaxjfrtmbduvm', 'pwipjns', 'tsseiqrpkbspwipjn', 'sseiqrpkbspwip', 'qgrtbcdqcbybzevizw', 'isjouwql', 'rlbeidykltcyopzsfstukduxabothywwbq']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('zxmeusmehplcgbqabjof'), ['hqxgrw', 'fydjyrr', 'cocwtbazwjrswygttvrna', 'wpkvowuq', 'mwnzdxihrxihzhqtl', 'ljkjtmrfbonhqkioyzotyeegrw', 'ofxo', 'rjubwtpbweratrelqlrqotl', 'wvxkcil', 'qvolxxgqs', 'afqlhjnlvxowtnuuzywxuob', 'slwbmq']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('tjcmtoisgbilkygushkpuxklis'), ['bkdohwx', 'dfohgzhcjqirlbrokwy', 'zaemgqgxltznvkccyumhgsftnfigbol', 'otgcaybejwe', 'qn', 'gvfzcyhvmsnbgkulsqrzeekmjkc', 'cajuyauvmhkrriehgwfmtqbkupysudle', 'pmcupysyllzpstolkfpdvieffxaupqtjty', 'elhlzvescbfpayngnnalzixxgunqdhx', 'cvxpgdnqcxeesk', 'etlewyipypeiiowuoewulkpalvcfe', 'ordhwrkwqq', 'wnroixlkrqnydblfrtlbywc', 'xshujuttvcdxzbetuvifiqi', 'meqqxqhntkvzwoptnwskdgsxsgjdawe', 'dnmicrfshqnzosxhnrftxxeifoqlnfdhheg']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('iepqqbvekaflprupsmnpoijrld'), ['kqomoeysekwcplpegdwcdoeh', 'mwdvr', 'aobviioktzwzmpilblbdwstndhimabfgct', 'vqustluciruiyfkoontehnwylnauwpol', 'utcqnitztcgr', 'ityszrqmlwzspnrwdcvdhtziob', 'hmll', 'ilfzvuxbkyppwejtp', 'euxdzqcqutnfeiivw', 'rbcjlmjniiznzaktsuawnfjzqjri', 'fzyxlzzretsshklrkwru', 'jrujmdevqqojloz']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('cufztqffwjhtlkysekklpaywemm'), ['cpawuauqodogaitybtvplknjrsb', 'ynsocxfnxshzwnhlrfilynvz', 'ylrpytgcvtiumdckm', 'mvgrkueaslpgnjvvhzairgldtl', 'iliorsjypskmxfuuplfagktoycywb', 'drvwngp', 'zviuhcxaspwmqqz', 'qfgmrmhycskus', 'szj', 'rooivliiqufztcqlhrqyqvp', 'tufdmsmwue', 'cssowtldgwksbzlqyfereodcpuedighwd', 'odcjdffchhabtaxjvnr', 'o']) from system.numbers limit 10; +select 7 = multiSearchFirstIndex(materialize('zqwvlarwmhhtjjgwrivwfpsjkvx'), ['zcwhagxehtswbdkey', 'okezglmrjoim', 'ilwdviqimijzgoopmxdswouh', 'aqztpsntwjqpluygrvwdyz', 'uzxhjuhiwpz', 'akgc', 'larwmhhtjjgwrivwfpsj', 'isqghxsmcrwlgyloslmlyeboywtttgejdyma', 'arwmhhtjjgwri', 'rwmhhtjj']) from system.numbers limit 10; +select 9 = multiSearchFirstIndex(materialize('fuddujwwcewlhthgwsrn'), ['shtzrrtukxmdovtixf', 'rkcnzzzojqvvysm', 'jlamctgphjqcxlvmpzyxtghnoaq', 'pthrwvbheydmrot', 'kpniaqbcrgtxdyxxdxonbbltbdo', 'igulngxgtauumhckvbdt', 'khgrmskijoxruzzzaigjxonsc', 'rxzeykfxwssltw', 'hthg', '']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('jtgvvkggpkqhbxptjgoy'), ['nplzawmacgtqfxsp', 'oosw', 'akw', 'hnsenqoqwiydiufozomkyirgjepeqw', 'fpafgahvfdxukzvskbuy', 'tqimmsqffiqfoni', 'rrxkjklmkdhxqwcpfyutqzxu', 'esfqeujcbqxwnvodkwwdbsyozptaf', 'rqnyguyz', 'fftl', 'ccfyavxtxrpi', 'wftpsblszgovfgf']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('steccxkwnptybaddcuau'), ['qagxfznhjaxtyclxdsi', 'rtxwptfyzgthkwrx', 'rmcoxxs', 'vlubx', 'siecygstzivz', 'tksiagm', 'kq', 'dgsqrobxegmdbjkanb', 'lxokyvhveklvdakrxyiqokr', 'tgpmehwdrirpfjonqzhqshbo', 'cqmkargvsfjoxrguymtzsfwkg', 'avkmufhoywprjw', 'xzywtvlpoozmgkrcavevwebv', 'hfiuwslapamiceaouznxm', 'tmfjhqddafhhjbybfphlbwu', 'mrigvhmjvdpny']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('ccbgxzoivbqtmyzqyooyepnmwufizz'), ['lcclseplkhxbrrzlnani', 'xggxivwqlpxmpypzovprdkmhrcgjkro', 'dbbmiegotfxjxybs', 'hqtcowpupsyqfx', 'znatfzjbeevbaqbmpofhywbyfxn', 'mnditiygex', 'lazqapwjswhkuimwmjoyseyucllnrfxrwnzj', 'jg', 'dmqwnuvsufgffuubhqeugwcanvflseorrydyyxvr', 'wpjfcfwfgjiybncrw', 'joucnvxxcyjyqlwhrzwnstyj', 'babtxkzasyaffxzd', 'wgcfdyhwxjoytbxffdxbdfinolbltnhqkvyzybc', 'yhrgwbdwopznltjtyidxawqg', 'bvrrt', 'bcwmsys', 'ijdjojhhzaiyjyai', 'eevxwppogogdbmqpbeqtembiqxeiwf']) from system.numbers limit 10; +select 2 = multiSearchFirstIndex(materialize('xrwjeznohtbdvijwsbdksf'), ['hwdfufmoemohatqafdrcvdk', 'tbdvijwsbdks', 'xzwjczbuteujfjifzkbxvezs', 'bdvijwsbd', 'eznohtbdvijwsbdks', 'xadezwhbbmlqz', 'b', 'socrdjxsibkb', 'dk', 'eznohtbdvijws', 'pavsosnncajr', 'jixlmxxmxnnbpebjhitvtsaiwzmtqq', 'yuxmmnrqz', 'mpzytweuycabvu', 'tbdvi', 'ip']) from system.numbers limit 10; + +select 0 = multiSearchFirstIndexUTF8(materialize('црвтгмсрооацволепкщкпнгшкамщ'), ['гйцбсханрейщжнфбхтщбйала', 'дирдфнжпнччхаоцшрийнйнечллтгцбфедгсш', 'жфйндбффаилбндмлточиирасдзйлжбдзег', 'жвоуйфсйойфцвгзшцитсчпкч', 'ршонтбгщжооилчхрзшгсдцпзчесххцп', 'пйучихссгнхщлутвменлмм', 'хишгешегдефесо', 'знупгж', 'щчфу', 'знвтжифбнщсибеноожжметачаохфхсжосдзйуп', 'ггтоцйпгхчсбохлрчлваисивжбшбохдурввагш', 'щлийбчштбсч']) from system.numbers limit 10; +select 5 = multiSearchFirstIndexUTF8(materialize('опднхссгртрхтотлпагхжипхпитраб'), ['шфршсцешушклудефцугщцмйщлошечедзг', 'нйумйхфщцгщклдожхвосочжжислцрц', 'згтпвзцбхйптцбагсвцгтнф', 'пшичси', 'ссгртрхтотлпа', 'апзазогвсбежзрйгщоитмдкн', 'непгайтзкгштглхифмзданоихц', 'пднхссгртрхтотлпагхжипхпитр', 'ждднфлрзалшптсбтущвошрйтхкцнегшхрсв', 'брп', 'сгртрхтотлпагхжипх', 'нхссгртрхтотлпагхжипхп', 'пагхж', 'мфкжм']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('овччцнтчайомсйййоуйуучщххиффсб'), ['жжрддцпнехйр', 'шзбвуооинпаххесйкпкошжмцзгхе', 'ррсннилщлщжгцтйрпхабкехахззнтщемагдйшпсч', 'пуфугнказепщ', 'гддхтплвд', 'сщсчи', 'бйрсахедщфкхиевкетнс', 'йфжцжшпхлййхачзхнфоц', 'цтмтжлщдщофисзрвтбо', 'кщсевбоуйб', 'щгаапзкн', 'осймщовшчозцййизм', 'фкмаат', 'бкзцсдонфгттнфтаглпрцтбхбсок', 'жлмичлйнйсжбгсейбсиезщдмутационжгмзп', 'нбищижнлпмтморлхцхвеибщщлкйкндлтпбд']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('фдситчщдвхмфйтхшдтуцтщжрочщо'), ['ейшфдннтйечгк', 'фуйщгбйшдцирзб', 'ехйцмчщрсртнк', 'увтцмдорщжфгцгзущпувтщкнрфсйбщрзй', 'хчщпхвуарнббпзсцшчщуносйгщпсбтх', 'жтдчрхфмхцххккзппзбнуббс', 'тчохнмбаваошернеймгготлузвсбрщезднеил', 'стссчкшрчррйбхдуефвеепщшзмербгц', 'жбезжпещ', 'вйтсрхптлкшвавдаакгохжцоощд', 'искеубочвчмдхе', 'щмлочпзбунщнхлрдлщтбеощчшчхцелшоп', 'екуийтсйукцн', 'дочахгжошвшйжцпчзвжйкис', 'лтеенешпсболгчиожпжобка', 'букзппщрчбпшвпопвйцач']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('гопвмрутфпфбхмидшлуб'), ['цнхшдойгщн', 'дкаежщрапщпщеа', 'фмогимдничрфтхмсцмчпдфтиофнтйц', 'фчмсщисхщуп', 'ощмвдчефозйжбеесбмещочевцчд', 'апкбцйщжщабвппофм', 'мтйоддлфцгдуммптднпщшрн', 'икхнсмжчбхнфхнссгл', 'ущмунинлбпрман', 'ллкнечрезп', 'ажтнвбиччджсзтйешйффдгдрувер', 'йрщ', 'чигдкйшфщжужзлвщулквдфщхубги', 'иккшсмаеодейнкмгхбдлоижххдан']) from system.numbers limit 10; +select 12 = multiSearchFirstIndexUTF8(materialize('срлцчуийдлрзтейоцгиз'), ['жщлнвбубжпф', 'оклвцедмиср', 'нлзхмчдзрззегщ', 'хоу', 'шайиуд', 'ерслщтзцфзвмйтжвфеблщдурстмйжо', 'жмгуйузнчгтт', 'стеглмрдмирйрумилвшнзззр', 'втедлчрчайвщнллнцдмурутш', 'цимхргмрвмщиогврнпиччубцйе', 'ктчтцбснзцйцймридвш', 'ейоц']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('лрицжленфилзсжпжйнцжжупупдфз'), ['чпбрмлрнцмвеуфу', 'рмпизмпжчшбхдудчшохтжш', 'гргцжчпгщищннусв', 'ийщтщвзчшпдзитщубакусхавслрсбткб', 'бйбакижцтибгбгхжцвйчжжщжсжкзф', 'чгрп', 'чуносжусжфчмфжхрщзлщрдвбашажаанча', 'чекршбш', 'лбцкхйсооцц', 'сгвнлегвфмпчтййлрмд', 'наатущркхйимхщщг', 'щпзоеимфощулбзхафпц', 'дцабцхлврк', 'умидмчуегтхпу', 'дщнаойрмчсуффиббдйопдииуефосжхнлржрйлз', 'щзжетезвндхптпфлк', 'бгчемкццдбжп', 'иихуеоцедгрсеужрииомкбззцнгфифоаневц']) from system.numbers limit 10; +select 3 = multiSearchFirstIndexUTF8(materialize('бхжвчашрощбмсбущлхевозожзуцгбе'), ['амидхмуеийхрнчйейтущлуегрртщрхвг', 'фнисцщггбщйа', 'хжвчашрощбмсбу', 'фщвщцнеспдддцчччекчвеещ', 'ущуджсшежчелмкдмщхашв', 'цкуфбиз', 'евозожз', 'ппт', 'лвцнелшхцш', 'ощбмсбущлхев', 'ефхсзишшвтмцжнвклцуо', 'цржсржмчвмфмнеещхмиркчмцойвйц', 'ашрощбмсбущлхевозожзу', 'гхщншфрщзтнтжкмлщанв', '', 'хевозо', 'ощбмсбущлхевозожзуц', 'возожзуц']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('мзчатйжщгтзлвефчшмлшт'), ['гхшфрунирйдзтеафщгк', 'ймхмфлц', 'звуумивмвштчтнтеобзщесакийгк', 'чщжетзнцишхрммтбцакиббчп', 'блмидикавущщдпгпчхйаатйанд', 'цмщшбклгцгмчредмущаофпткеф', 'бнетввйцзпдерхщ', 'ицйнцрввемсвтштчфрпжнатаихцклкц', 'дзлщсштофвздтмчвсефишс', 'пбзртдцвгкглцфесидлвваисщр', 'ммеилбзфнчищч', 'жш', 'лздиззтпемкх', 'байлужднфугмкшгвгулффмщзхомпав', 'рсзнббедсчзущафббзбйоелид', 'цфшйкцксйгуйо']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('жжмзмащйфжщлрффбпврзнидииейщ'), ['ржфзнлйщсздйткаоцруйцгцт', 'илинксщмгщшещееифвпданмйлж', 'кг', 'гпааймцщпмсочтеиффосицхйпруйшнццвс', 'кнзфгжйирблщлл', 'ищуушфчорзлкбцппидчннцвхщщжййнкфтлрдчм', 'тбтдчлвцилргоргжсфбоо', 'ехаех', 'нехщмдлйджждмрцпйкбрнщсифхфщ', 'тцжпснйофцжфивзфбхзузщтмдкцжплавозмше']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('биаризлрвххжкпщтккучфизуршткпн'), ['йбручвндбщвссаеха', 'ол', 'еузкмпогщзгзафшдшоплбфнфдккх', 'ибзихщйфбтаз', 'ибрчиейш', 'нафрпбснзрузнтмнйиомтечтшзбкпзутдилтф', 'тщтбапцчдий', 'щкнггмфцжрзщцзжвлкчбммхтхтуж', 'ваам', 'цкфиушзигбжтацнчдлжжзфшщммтнлж', 'туфовжтнкзщсщщизмрйкхкпц', 'пирзксзикфтшодожшчцг', 'жфчфцфвлйбмеглжйдазгптзщгж', 'тутириждкзчвтсоажп', 'мотзусбхту', 'слщкгхжщфщоцкцтрлгп', 'бругтбфесвсшцхнтулк', 'восур', 'ссежгнггщдтишхйнн', 'вгзосзгоукмтубахжнзгшн']) from system.numbers limit 10; +select 8 = multiSearchFirstIndexUTF8(materialize('мчслвбжвманджййсикнврцдчмш'), ['рлбмй', 'иб', 'жажлцсзхйфдцудппефвжфк', 'огггхзгтцфслхацбщ', 'дзтцкогаибевсйещпг', 'зпцтйзфмвгщшуоилл', 'етщзгцпдйчзмфнхпфцен', 'нджййсик', 'сикнврцдчмш', 'жййсикн', 'икнврцдч', 'паокаочввеулщв', '', '', 'кечзсшип', 'вбжвманджййсикнвр']) from system.numbers limit 10; +select 2 = multiSearchFirstIndexUTF8(materialize('нвррммппогдйншбшнехнвлхм'), ['нфошцншблеооту', 'лх', 'цртд', 'огдйншбшн', 'уулддйдщицчпшбоиоцшй', '', 'дрдужзжпцкслетгвп', 'й', 'мппогдйншбшнех', 'дйншб', 'лжвофчзвдд', 'рммппогдйншб', 'ехнв', 'втущсщзбчсжцмаанчлнасп']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('удехбкабиацхпгзнхжелшц'), ['фмнбтйезсфоахофофдблкжщжфмгхтзс', 'тщтамзафозхлз', 'цшжфсбл', 'йзгзилупшллвипучхавшнмщафзмнк', 'лу', 'гтебпднцчвмктщсзи', 'лпщлмцийгуеджекшд', 'пцдхфоецфрунзм', 'зис', 'хпж', 'цтцплхцжишфнплуеохн', 'впх', 'чцчдацлуецрчцжижфиквтйийкез', 'гчшмекотд', 'пйгкцчафеавзихзтххтсмкал', 'сжфхпцгдфицжслрдчлдхлсувчнрогнву']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('щщвфчгамтжашнуошбзшуйчтшх'), ['дийу', 'жеомлсжщймемрсччошдфажцтдп', 'нгопнцквбф', 'хопб', 'ив', 'чвфвшфрдфелрдбтатшвейтг', 'вхкцадмупдчбаушшлдксйв', 'жтжбсвмшшсйеуфдпбдлкквдиовж', 'гтсдолснхесйцкйкмищгсзедх', 'ошплп', 'ифпуррикбопйгиччи', 'чдфймудаибвфчжтзглс', 'зпцмвпнлтунвйж', 'еждрйитхччещлцч', 'вмофсужхгрнзехкх', 'щжгквкрфжмжжсефпахст']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('рфгигуужжцфмоаешщечувщгонт'), ['слащченщлуоцргврбаб', 'тцизут', 'лйрсцолзклжбчрзгббммоищщ', 'уицмлоилзф', 'зпхмшвфйккфщщп', 'ймижрпдщмшв', 'пуощжлрмжлщхмкйгщшщивдпчпжчл', 'ойахшафнж', 'гксомбвцрсбжепхкхжхнсббци', 'панлраптщмцмйфебцщемйахенг', 'сохлгожштлднчсзпгтифсйгфмфп', 'аждчвзну', 'дхшуфд', 'борзизцхнийбщгхепрнзшй', 'фщшздруггрке', 'оевупрйщктнолшбкунзжху']) from system.numbers limit 10; +select 8 = multiSearchFirstIndexUTF8(materialize('кщзпапйднучлктхжслмищ'), ['апмдйлсафхугшдезксш', 'кйрм', 'цйивайчшуалгащсхйш', 'злорнмхекг', 'сгщврурфопжнлхкбилдч', 'бнлпщшнвубддрлижпайм', 'нукдонццнрмовфнбгзщсшщшдичежффе', 'йднучлктхжс', 'зпапйднучлктхж', 'затйотдсмпбевлжаиутсуг']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('жцажссефррршнфмнупщаоафгкщваа'), ['жфпщкгзкрмтщчцтжйчпйдошбшоцд', 'бхгйлйдробптвущшппзуиидежнлтпбжащткцф', 'хлещазйцепдханпажчизнхгншйуазщхй', 'ашцк', 'фрбммхдднчзшс', 'нжцанилзжаречвучозрущцдщаон', 'длмчзцрмжщбневрхуонпйейм', 'шкбщттврлпреабпоиожнууупшмкере', 'вуцпщдиифпеоурчвибойбпкпбкйбшхдбхнаббж', 'нртжвкдйтнлншцанцпугтогщгчигзтоищпм', 'цкплнкщлкшемощмстздхпацефогтск', 'цвждйбсмпгацфн', 'шсжшрзрардтпщлгчфздумупд', 'цйииткглчжйвуейеиииинврщу', 'унлодтулшпймашоквббчйнибтвалалрвбцж', 'нбнфнвйишйжлзхкахчмнлшзуеенк', 'бшлпсщжквпцахигчдтибкййб', 'фчакпзовтрлкншзцулшщмпзж']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('иматеччдфлггшпучумджпфпзмвх'), ['дахахпчлцлаачгцгтфпнжлшчйуцбшсг', 'атжйувхец', 'грдсбвиднницдвшпйршгмегцаоопнжгй', 'чзлхречмктфащмтеечуиагоуб', 'савбхлпилийщтихутйчдгфсойй', 'вбгочбзистзщшденусцофит', 'мар', 'дфшажхдсри', 'тжлмщшж', 'птсрсщгшммв', 'ре', 'зратамкткфкинййй', 'гуцмсизулвазужфдмхнелфнжббдтрудчтнфцр', 'нйчинеучкхнпчгнйвчвсвлгминуцахгщввжц', 'ечагчнуулфббгбел', 'йшжуговрйкащцофдокфчушжктнптйеззушфо']) from system.numbers limit 10; +select 11 = multiSearchFirstIndexUTF8(materialize('азтммйтшхцхлгдрнтхфжбдрлцхщ'), ['нпучщфвспндщшспзмшочгсщжчйгжбжзжжтн', 'хккдйшабисдузфртнллщпбоуооврайцз', 'йпхрфжждгпнйаспйппвхбргшйвжччт', 'ффеее', 'кежцновв', 'еххрчштарзмкпйззсйлмплхбчбулзибвчбщ', 'шфжйдотрщттфхобббг', 'ожоцжущопгоцимсфчйщцддзнфи', 'цуимеимймкфччц', 'прммщмтбт', 'хцхлгдрнтхфж', 'лгд', 'цжбдаичхпщзцасбиршшикджцунйохдлхй', 'пидхцмхйнспйокнттмййвчщпхап', 'йтйзмеаизкшйошзвфучйирг', 'хцхлгдр']) from system.numbers limit 10; + +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('gyhTlBTDPlwbsznFtODVUzGJtq'), ['seSqNDSccPGLUJjb', 'xHvtZaHNEwtPVTRHuTPZDFERaTsDoSdX', 'QCeZOYqoYDU', 'bsybOMriWGxpwvJhbPfYR', 'FFHhlxfSLzMYwLPPz', 'tvDAJjaLNCCsLPbN', 'kOykGaSibakfHcr', 'mWAZaefkrIuYafkCDegF', 'ILrFDapnEDGCZWEQxSDHjWnjJmeMJlcMXh', 'zHvaaTgspUDUx', 'tss', 'laUe', 'euUKFLSUqGCjgj', 'Kd', 'MxyBG', 'qRXMsQbNsmFKbYSfEKieYGOxfVvSOuQZw', 'PdBrNIsprvTHfTuLgObTt', 'kMekbxI']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('ZxTznPEbfoBfLElYOrRiHrDLMmTpIh'), ['bJhYwKLeeLvLmXwWvQHWFkDQp', 'dLyZmUicTZmUfjfsFjxxgOiMJn', 'UCYbbGcY', 'kpPiwfWHEuh', 'jviwmHeiTQGxlTKGVEnse', 'cVnEyLFjKXiLebXjjVxvVeNzPPhizhAWnfCFr', 'gkcoAlFFA', 'ahZFvTJLErKpnnqesNYueUzI', 'VIJXPlFhp', 'rxWeMpmRFMZYwHnUP', 'iFwXBONeEUkQTxczRgm', 'ZnbOGKnoWh', 'SokGzZpkdaMe', 'EfKstISJNTmwrJAsxJoAqAzmZgGCzVRoC', 'HTmHWsY', 'CpRDbhLIroWakVkTQujcAJgrHHxc']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('VELfidaBvVtAghxjkrdZnG'), ['fvEFyRHvixuAYbuXygKeD', 'zFNHINreSOFksEGssBI', 'hcdWEcKDGWvfu', 'KczaFjvN', 'nZLTZAYSbfqcNWzWuGatDPUBYaRzuMBO', 'UdOdfdyPWPlUVeBzLRPMnqKLSuHvHgKX', 'DgVLuvxPhqRdSHVRSeoJwWeJQKQnKqFM', 'NNfgQylawNsoRJNpmFJVjAtoYy', 'tWFyALHEAyladtnPaTsmFJQfafkFjL', 'lYIXNiApypgtQuziDNKYfjwAqT', 'QjbTezRorweORubheFFrj', 'htIjVIFzLlMJDsPnBPF', 'ltDTemMQEgITf', 'fprmapUHaSQNLkRLWAfhOZNy', 'dOJMvPoNCUjEk', 'm', 'vEEXwfF', 'aVIsuUeKGAcmBcxOHubKuk']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('kOzLaInSCOFHikwfkXaBfkyjdQ'), ['t', 'emHGfAiZSkZaVTSfplxRiPoDZUTT', 'YHnGJDTzxsboDsLPGHChMHwrHHICBIs', 'gbcbVHSlVeVDOeILWtSLkKfVVjG', 'fPaJjbnNthEwWZyg', 'qS', 'PCQxoLaSdQOjioMKPglmoWR', 'KLMNszm', 'TCErEFyxOvqnHs', 'dRbGzEJqvIGAcilZoHlXtZpjmLLZfsYueKqo', 'iKHmNSbGgaJYJEdMkbobXTdlFgAGEJMQ', 'mUGB']) from system.numbers limit 10; +select 1 = multiSearchFirstIndexCaseInsensitive(materialize('JGcICnWOGwFmJzHjtGJM'), ['fmJzHj', 'LhGTreYju', 'yCELHyNLiAJENFOLKOeuvEPxDPUQj', 'kWqx', 'OBnNMuaeQWmZqjWvQI', 'ektduDXTNNeelv', 'J', 'iCNwoGwfMJzhjtGJ', 'uiIipgCRWeKm', 'bNIWEfWyZlLd']) from system.numbers limit 10; +select 7 = multiSearchFirstIndexCaseInsensitive(materialize('fsoSePRpplvNyBVQYjRFHHIh'), ['ZqGBzyQJYuhTupkOLLqgXdtIkhZx', 'pouH', 'mzCauXdgBdEpuzzFkfJ', 'uOrjMmsHkPpGAhjJwVOFw', 'KbKrrCJrTtiuu', 'jxbLtHIrwYXDERFHfMzVJxgUAofwUrB', 'PLvNyBVQYjRfhhi', 'wTPkeRGqqYiIxwExFu', 'PplvNybvqyJ', 'qOWuzwzvWrvzamVTPUZPMmZkIESq', 'ZDGM', 'nLyiGwqGIcr', 'GdaWtNcVvIYClQBiomWUrBNNKWV', 'QQxsPMoliytEtQ', 'TVarlkYnCsDWm', 'BvqYJr', 'YJr', 'sePrPPLVNYbvqYJRFhh', 'ybvq', 'VQYjrFHh']) from system.numbers limit 10; +select 3 = multiSearchFirstIndexCaseInsensitive(materialize('aliAsDgMSDPISdriLduBFnuWaaRej'), ['gWOFTxMrQGQaLrpJamvRhgeHwk', 'iWsBLzLycWvbJXBNlBazmJqxNlaPX', 'Ri', 'FPLRURSsjvsySncekcxaWQFGKn', 'wgXSTVzddtSGJQWxucYorRjnQQlJcd', 'wOLJWZcjHEatZWYfIwGIqnuzdcHKSFqfARfNLky', 'eEECZMNmWcoEnVeSrDNJxcOKDz', 'duBF', 'EhfLOjeEOQ', 'dUbFNUWA']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('EUzxPFYxMsJaTDzAKRXgZIVSFXU'), ['TDKAgICICjzBKHRqgFAuPCSODemldGGd', 'LvMluSJTIlgL', 'srbRhQKjPIchsipVHsjxwhK', 'vdurVsYkUWiFQVaDOnoNIJEX', 'UzZsZqAUNjMvWJaTqSWMHpzlDhVOaLzHPZfV', 'XcnnPXXEJJv', 'JSwFBNnYzNbIRZdeMfYiAfxzWfnCQFqoTUjns', 'HBMeqdLkrhebQeYfPzfJKAZgtuWHl', 'cMfSOnWgJvGhFPjgZdMBncnqdX', 'orDafpQXkrADEikyLVTHYmbVxtD', 'Vz', 'bfYwQkUC', 'q', 'YqomKpmYpHGv']) from system.numbers limit 10; +select 4 = multiSearchFirstIndexCaseInsensitive(materialize('mDFzyOuNsuOCSzyjWXxePRRIAHi'), ['TfejIlXcxqqoVmNHsOocEogH', 'clyblaTFmyY', 'JQfxMAWVnQDucIQ', 'jw', 'fGetlRA', 'uWwCOCd', 'rInhyxSIFiogdCCdTPqJNrqVaKIPWvLFI', 'mimSJjfCWI', 'jqnJvNZXMEPorpIxpWkhCoiGzlcfqRGyWxQL', 'bxCJeVlWhqGHoakarZcK', 'unsUOcSZyjwxxe', 'E', 'PR', 'nsUoCSZyjwxXEPr', 'sfotzRPMmalUSjHkZDDOzjens', 'zYJwxx', 'DFzyouNsUocsZ', 'QBaQfeznthSEMIPFwuvtolRzrXjjhpUY', 'sQPVBaoeYlUyZRHtapfGM', 'lPiZLi']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('VOAJfSkbDvNWAZNLIwqUgvBOddX'), ['pHrGGgJ', 'VohjtPdQZSNeKAlChDCnRTelroghFbZXVpnD', 'rnWebvdsmiFypMKL', 'NtKRiJOfAkWyKvubXrkOODgmZxvfOohsnHJEO', 'nxsDisKarasSZwESIInCJnYREUcoRUTXHBUH', 'mXYYr', 'jujScxeTBWujKhKyAswXPRszFcOKMSbk', 'INEegRWNgEoxqwNaGZV', 'VVyjMXVWVyuaOwiVnEsYN', 'mkLXSmXppxJhFsmH', 'pRVnBrWjqPeUDHvhVuDbzUgy', 'PzchFdPTkOCIVhCKml', 'KXaGWnzqoHBd', 'PhzQVqIOLleqDSYNHLjAceHLKYPhCVq', 'aixxTqAtOAOylYGSYwtMkZbrKGnQLVxnq', 'ruEiaxeRaOOXGggRSPlUOGWSjxh', 'prSULtHvDMw', 'vEpaIIDbGvIePYIHHZVNSPYJl']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('ZHcEinZEFtfmHBLuCHntUhbIgY'), ['GKElMPEtmkLl', 'mkrzzjSRfXThuCQHkbZxRbhcymzTxcn', 'PREwQjxBJkpkiyuYEvtMZNFELgbINWsgf', 'lFEGlPtaDJSyoXzwREiRfpzNpsaBYo', 'tmVTuLPhqhgnFNhHvqpmc', 'NtijVhVfAwpRsvkUTkhwxcHJ', 'O', 'FSweqlUXdDcrlT', 'uljEFtKVjIzAEUBUeKZXzCWmG', 'dBIsjfm', 'CNaZCAQdKGiRUDOGMtUvFigloLEUr', 'yWjizKZ', 'QqPVdyIFXcweHz', 'uPmgGWGjhzt']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('AYMpbVsUQqAfoaMiJcYsulujYoSIx'), ['aXECumHNmAEefHPJy', 'hTosrERBdVCIilCYcMdHwaRh', 'PVDBpwrc', 'uFvQRPePvmzmocOauvEjqoxMhytzOwPSOCjmtm', 'kQqIlSCHDmWXCKN', 'ybAHGYDEDvvOJsF', 'WpkANi', 'cFGuzEcdahZtTdLFNBrRW', 'EBaybUFxO', 'mRlZUzHzMsMAgvtRtATEDLQvXZnZHw', 'uqxckjqpCBHiLgSPRz', 'Lv', 'AJcRfAvBmQVMOjaFfMfNHJt', 'FYsPM', 'pkKXTPgijOHFclqgVq', 'Ck']) from system.numbers limit 10; +select 11 = multiSearchFirstIndexCaseInsensitive(materialize('gmKSXWkNhKckrVNgvwiP'), ['bdJMecfCwQlrsgxkqA', 'NTgcYkMNDnTiQj', 'fmRZvPRkvNFnamMxyseerPoNBa', 'rfcRLxKJIVkLaRiUSTqnKYUrH', 'YSUWAyEvbUHc', 'PridoKqGiaCKp', 'quwOidiRRFT', 'yHmxxUyeVwXKnuAofwYD', 'gichY', 'QlNKUQpsQPxAg', 'knhkCKRVNGvWIp', 'jAuJorWkuxaGcEvpkXpqetHnWToeEp', 'KnHKCKrvNgVW', 'tCvFhhhzqegmltWKea', 'luZUmrtKmmgasVXS', 'mageZacuFgxBOkBfHsfJVBeAFx', 'hKC', 'hkRCMCgJScJusY', 'MKSXWknHkckrVNgv', 'osbRPcYXDxgYjSodlMgV']) from system.numbers limit 10; +select 15 = multiSearchFirstIndexCaseInsensitive(materialize('lcXsRFUrGxroGIcpdeSJGiSseJldX'), ['pBYVjxNcQiyAFfzBvHYHhheAHZpeLcieaTu', 'SQSQp', 'OQePajOcTpkOhSKmoIKCAcUDRGsQFln', 'AYMDhpMbxWpBXytgWYXjq', 'gkUC', 'oWcNKfmSTwoWNxrfXjyMpst', 'fQSqkjRNiBGSfceVgJsxgZLSnUu', 'LRrhUjQstxBlmPWLGFMwbLCaBEkWdNJ', 'cZnaActZVoCZhffIMlkMbvbT', 'Uxg', 'vlKdriGMajSlGdmrwoAEBrdI', 'Fl', 'XzcNdlUJShjddbUQiRtR', 'AqowAuWqVQMppR', 'SRFUrGXrOgiCP', 'k']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('KhwhbOzWvobUwJcteCHguFCn'), ['LkDYrpvDfPL', 'CIaTaShobVIaWjdbsNsCMdZKlGdtWuJmn', 'zYcsxxFyfuGrPdTPgEvGbXoYy', 'vDIeYpJbLMGMuRkIrPkAnqDDkqXPzy', 'Ievib', 'CREiuEsErFgvGEkQzThHtYtPmcL', 'JjRWKyALtSkoGmRxh', 'JxPhpijkDOpncCKyDEyXvKNua', 'jo', 'mKpFscuBEABMAlQO', 'qiFTgJpcnUMRKzTEuKY', 'pXBtITxCPRaXijM', 'guYVLpIbu', 'tSKYIxv', 'oDnWaFAmsXGRdGvRPhbCIvFSFQNlSVYB', 'phdckINUiYL']) from system.numbers limit 10; +select 14 = multiSearchFirstIndexCaseInsensitive(materialize('pXFoUGwVTAItBqgbBaQwAqmeh'), ['LfBevBpGnaSlmGhbeZ', 'NtBYzEksiXvYI', 'jMeRw', 'omtaduY', 'BsWyvNdkfXsTBxf', 'CtoOIvaesuca', 'pgJcRIBVbyaPBgGsNKP', 'bAwdUMnwKvMXfFHQWrtfMeqcORIJH', 'GDxZblrqWSxUJFjEuXArPtfHPdwSNGGL', 'LLxcfp', 'NrLghkFpwCdvHJBfPBgiMatNRaDKjO', 'XCzr', 'cCojPpfLkGZnaWBGpaZvrGMwgHNF', 'BaQWAQmE', 'AQ', 'RtxxEZDfcEZAgURg']) from system.numbers limit 10; +select 5 = multiSearchFirstIndexCaseInsensitive(materialize('KoLaGGWMRbPbKNChdKPGuNCDKZtWRX'), ['FBmf', 'QJxevrlVWhTDAJetlGoEBZWYz', 'tKoWKKXBOATZukMuBEaYYBPHuyncskOZYD', 'kgjgTpaHXji', '', 'xOJWVRvQoAYNVSN', 'YApQjWJCFuusXpTLfmLPinKNEuqfYAz', 'GXGfZJxhHcChCaoLwNNocnCjtIuw', 'ZLBHIwyivzQDbGsmVNBFDpVaWkIDRqsl', 'Kp', 'EyrNtIFdsoUWqLcVOpuqJBdMQ', 'AggwmRBpbknCHdKPgun', 'xNlnPtyQsdqH', 'hDk']) from system.numbers limit 10; +select 6 = multiSearchFirstIndexCaseInsensitive(materialize('OlyNppgrtlubvhpJfxeWsRHpr'), ['slbiGvzIFnqPgKZbzuh', 'fakuDHZWkYbXycUwNWC', 'HnVViUypZxAsLJocdwFFPgTDIkI', 'bLx', 'fmXVYOINsdIMmTJAQYWbBAuX', 'pjFXews', 'BG', 'vrSQLb', 'ub', 'pREPyIjRhXGKZovTqlDyYIuoYHewBH', 'hnNQpJmOKnGMlVbkSOyJxoQMdbGhTAsQU', 'UwaNyOQuYpkE', 'yHNlFVnuOLUxqHyzAtNgNohLT', 'YJRazuUZkP', 'z', 'lUbVhpjFxEWsRhP']) from system.numbers limit 10; +select 6 = multiSearchFirstIndexCaseInsensitive(materialize('ryHzepjmzFdLkCcYqoFCgnJh'), ['cLwBRJmuspkoOgKwtLXLbKFsj', 'YSgEdzTdYTZAEtaoJpjyfwymbERCVvveR', 'RzdDRzKjPXQzberVJRry', 'HUitVdjGjxYwIaLozmnKcCpFOjotfpAy', 'LWqtEkIiSvufymDiYjwt', 'FDlKCCYqoFCGNj', 'jmZfdlKCcyQOFcGnJ', 'OZCPsxgxYHdhqlnPnfRVGOJRL', 'JfhoyhbUhmDrKtYjZDCDFDcdNs', 'KCCYqo', 'EPJMzFDLKcCYQ', 'zLQb', 'qsqFDGqVnDX', 'MzfdLkCCyQOFc']) from system.numbers limit 10; +select 5 = multiSearchFirstIndexCaseInsensitive(materialize('oQLuuhKsqjdTaZmMiThIJrtwSrFv'), ['MsfVCGMIlgwomkNhkKn', 'fBzcso', 'meOeEdkEbFjgyAaeQeuqZXFFXqIxBkLbYiPk', 'tNV', 'i', 'EwuTkQnYCWktMAIdZEeJkgl', '', 'hUo', 'dtAzmMITHijRtwsrFV', 'vhnipYCl', 'puor', 'TazMmiTh', 'ITHIJRTWSrf', 'luuHksqJDTaz', 'uHkSQjDtazMMiThIjrtwSRFV', 'gpWugfu', 'QjdtazmmIthIjRTWSRFV', 'ZdJpc']) from system.numbers limit 10; + +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ИпрУщйжббКВНИчйацпцоЛП'), ['цШСкЕвеГЕЗЦщруИБтЦсБГАу', 'Хнщта', 'БшА', 'СалШйР', 'ЩфДГРРчшБДММГЧоноЖСчдпВХшшгйН', 'бЕжПШЦддожнЧоЕишчшЕЙфСщиВПФМ', 'ТЗзГФх', 'Чфл', 'КнНкнЖЕкППварНрхдгЙкДешмСКИЛкеО', 'ЖИсЧПСФФМДиТШХЦфмЗУпфрУщСЛщсфмвШ', 'ллЙумпхчОсЦМщУ', 'ГМУНЦФшНУбРжоПвШШщлВФАтоРфИ', 'БХцжеНЗкжЗЗшЦзфгдЖОзЗЖщКМИШАтЦАп', 'мтСкЕнбХШнЛхХГР']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('цмйвГЖруДлдЦавхЖАлоЕхЗКд'), ['ХфБПМДВХЙ', 'саЗваАбднХбЦттмКсМбШбВМУйНКСЖжХЦНц', 'плиЩщШАцЖсхГ', 'ЗнУЕФЗВаНА', 'ЧДйСаЗГЕшойСжбсуЩуЩщбПР', 'ЧЕуЩкФБВвчмабШЦтЖбОрЗп', 'йХбМсрТАФм', 'РЖСЗвЦлНВПЧщГУцЖ', 'ГГлЩрОХКнШРТуДФ', 'шСабРжла', 'ЕчБвгаРЧифаЙщХПпГЦхчШ', 'дайшйцВНЩЧуцйдМХг', 'УнзНКЧххВрцЩМлАнЖСДОДцбИгЛЛР', 'сЛЗзПбиАгзК']) from system.numbers limit 10; +select 2 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('дфЧлзОжММФкЖгиЗЩлоШжФТкцк'), ['ЗРТцИрсФСбПрщГЗ', '', 'ЖГИЗщлОш', 'АДПН', '', 'чЛЗОЖмМфКжг', 'Мфкж', 'ндаовк', 'зГЛРГАНШмСмШМефазшеБкзДвЕШиЖСЗЧПИфо', 'ФЧЛзОЖммфКжгиЗЩ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ИИКДМЛхРчнвЙЕкВЧелТйЛВТ'), ['АчшОЛтНЙуЦЛЙфАКУйуТЗМеЗщОХТМЗеТА', 'НЦУДбчфРТОпЛкОгВпоО', 'неДавнНРеАУфТтфАнДчтнУМЛПШнроАчжш', 'бГржВПЧлЛтСВТтаМЦШШ', 'БщГщРнБхЕЛоЛсмЙцВЕГ', 'цбАжЦРеу', 'ХсЦРаНиН', 'нббДдВЗРС', 'змОПпеЛЖзушлнДЛфчЗлцЙЛфЖрЛКг', 'фШиЖСУоаНПйИВшшаоуЙУА', 'ЛктХиШРП', 'МапщВйцХч', 'жмУТкуГбУ', 'сйпзДЩоНдШЕТбПзФтсрмАФГСз', 'ЛБУвйладЕижрКзШУАгНЩчЕмАа', 'мЧпФлМчРбШРблмтмПМоС']) from system.numbers limit 10; +select 8 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ПоДУЗАтХншЦатИшХвмИЖчГнжчНцух'), ['жЛЧХзкжлиЛцЩбЧСнЛУжЖпКРвиСРН', 'шадмЩеУШБврУдЕБЗИгмЗЕФШчЦБСзПидтАлб', 'йпГмШСз', 'хЖФЙиПГЗЩавиЗЩйПнБЗЦЩмАЧ', 'ХесщтлбСИуЦ', 'вар', 'ЙкМаСхаЩаЗнФЩфКжПщб', 'ОдУзАТХншЦатИШхвМиЖчгнЖч', 'ЗВЗДБпФфцвжУКвНсбухссбЙКЙйккЛиим', 'гХхсГЛшдфЖЛбгчоЕмоЧр']) from system.numbers limit 10; +select 7 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ихзКЖЩсЧРСЖсЖжЛАшкТхИйТгМБпск'), ['ДРОБм', 'нз', 'тОЛ', 'щРзуЖрТ', 'Мдд', 'АЦГРК', 'Чрсжсжжл', 'чРсжсЖжл', 'ктхИйтГмБ', 'аАзЙддМДЦЩФкРТЧзЧПУойоТхБиЧПлХДв', 'иЙтгМбп', 'РицлПн', 'йДГнЧкЕв', 'ВМЩцАш', 'хКЩнДшуБЕЛТФГВгцБПРихШЙХгГД', 'иЙТГМ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('жггкщцзщшамдбРЗжйТзвхшАпХСбе'), ['лВТвтлРБжиЛЦвРЦкАЦаНБгуОН', 'рШаавцжзМрзВЧДРСузб', 'оемрЗМгФБНмжп', 'ЛбмХбФЧШГЛХИуТСрфхп', 'ЖшТдтЧйчМР', 'ЧнИМбфУпмЙлШЗТрТИкКИЩОЧеМщПЩлдБ', 'ГвРдПжГдБаснилз', 'уТнТчТРЗИЛ', 'ИТЕВ', 'дИСЖпПнПСНОвсЩЩшНтХЧшВ', 'штабтлМнсчРЗтфсТЩублЕЧйцеЦТтХ', 'ХбхгУШвАзкшЖ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('нсЩЙЕМмЧЛСйФцГВМиатГХш'), ['КсОПЧИкВсКшРхнкхБжду', 'мШмпТащжФ', 'ББЖнианЧЦпмрГЩГМаЛКжА', 'арИжзжфГТУДИРРРбцил', 'дфдмшМИщТиЗПруКфОнСЦ', 'Рцч', 'гмДгВДАтсщКЗлхвжЦУеФДАТГЙЦЧОЗвРш', 'чфХЩсДбУбВжАМшРлКРщв', 'нцБйсУ', 'фасДЕчвчДмбтЖХвоД', 'аБЧшЖшЖАКргОИшпШЧзТбтфйвкЕц', 'ЗжжсмкжЛд', 'щщлПзг', 'бП']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('сКиурчоиаЦйхгаУДПфчИтИК'), ['МЧПцУАМрХКЧмАЦннУшмРчкЖКХвху', 'РвДуВиашрРКкмжшЖНШБфлцжБЦР', 'йМУиУчНЧчРшДйБЗфЩЦйПсцгкДС', 'НсмаЛзЧвНЦШФуВРпзБГзйКцп', 'ЖлМЛУХОБллСЗСКвМКМдГчЩ', 'ЩХПШиобЛх', 'аФАЖВтРиЦнжбкСожУЖЙипм', 'аУГжУНуМУВФлж', 'ШБчтЗкЖНЙк', 'ЩоГПГчНП', 'мВЗйЛаХПоЕМХиИйДлшРгзугЙЖлнМппКЦ', 'вчмДФхНеЦйЗсЗйкфпОщПтШпспИМдГйВМх', 'ИЗИжЧжаГЩСуцСЩдкскздмЖЦ', 'дАмфЕбгс', 'ГМттнхчЩжМЧДфщШБкфчтЧ', 'ШЕииФБпщЙИДцРиЖжЩл', 'ОпуОлБ', 'хБ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('риШМбгиПЖннНоЧргзГзеДпЛиНт'), ['икДкбйдройВУсвФзрПСусДнАШо', 'чуУеТкУВФхз', 'ЕГпйчехЗвЛлБблЧПДм', 'зеоЩЧожКЛбШЩдАрКБНйшКВШаЗгПш', 'виФКуЗОтгВмТкБ', 'цДрЙгЗРаЧКаМДдБЕЧзСРщВФзПВЧГвЩрАУшс', 'мБЗИУдчХХжТж', 'ФТНМмгЖилуЛйМ', 'ЗегЩЦнЦщцИк', 'оГОусхФсДЖДЩИЕХЗпсПЩХБТГЕп', 'АУКНзАДНкусВЧХвАж', 'КвКрбсВлНАоЗсфХОйЦхТ', 'вФдеХацЧБкрхМЖЗЧчКшпфВчс', 'йХшиОвХЗжТпДТбвУрпшЕ']) from system.numbers limit 10; +select 11 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('МойрЙлтЖйБдББЛЕЕЦузЛфпИЕГт'), ['ПОжЦЩа', 'СШзЧФтСЗохЦЗдФтцНТу', 'вЕдТ', 'ечУФаМДнХщЕНУи', 'вмеосТзБАБуроЙУЛгФжДсЧщтчЕзлепгк', 'ИЧтБрцПмРаВрйИвНЛСйпЖжУВдНрурКшоКХП', 'ЕН', 'щКЦЩгФБСХпкпит', 'ей', 'ЕахшеОМРдЕГХуГЖчвКХМЕ', 'Гт', 'НужЛЛЙОАл']) from system.numbers limit 10; +select 11 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('еззЦАвУаДнзИКЙнЙдртРоП'), ['КгЩбшПЛКвтИРцйчккгЧчЧмтГ', 'кЛппСФщзМмТйВЕтбЩЦлО', 'ШпдзиЖх', 'иИХ', 'пУаАФгсмтофНФХиЦЕтТЗсОШЗЙ', 'фаКАБТцФМиКЖрИКшГБЗБ', 'идЖЙдЦММУнХЦЦфсФМ', 'МиЦечЖЦЙмРВЙОХсБРНнрлйЙшц', 'ТфдСтМгтмимТМАучтхПНЦлуф', 'бейККЛСггУЦБсокЕЙпнРЧ', 'цавУАДНЗИКЙнЙд', 'ЩйЕЖчЧщаПшжФсхХЛЕТчвмЙнуце', 'РТРОП', 'цАВуАДнзИкЙНЙдРтРо', 'аЩПИд', 'ОСчКшОАчВмр', '', 'уЙЛИуЕУвцДшНОгбТбИШв', 'АВУаднзИКЙНйдР', 'жТйоП']) from system.numbers limit 10; +select 12 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('шйМЦУщвфщшбмлТНВохСЖНУ'), ['хшТАпТоШхКНсДпвДЕчДМНбАНччд', 'ХКуПСтфСйРжмБглОШЙлйДкСФВйВ', 'хпмНЦМУШеАД', 'чзмЧВвлбЧкАщПкзТгеуГущб', 'шзжрДд', 'еЗГОЙНйИБЗДщИИНицмсЙЗгФУл', 'кнщЙхооДТООе', 'всзЙнТшжФЗДБДрщВДлбвулДИаз', 'мп', 'уБОйцзнМпИсксхефбдЕЛйгИмГШГЗЩ', 'ОМпзШШщчФФнвУЧгжчиндЧч', 'щВФЩШбмЛТн', 'бм', 'БпфнкнйЗцПдЧЩбВ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('НЗБлОбшмОПктткоччКиКрФсбкШАХ'), ['нффЕББУЖГшЖвГфЦФГЕСщсЩЧлфнАшшктизУ', 'нСмпцхшИои', 'ЧИчЗУтйЦхГезппФРХХШуцЗШВ', 'РИнщН', 'НЩдВТсЙсОдхРбМФнСпАбОПкудБФСчмб', 'йхглпдКтртгош', 'ибгУРАБцх', 'ИЕиЛрИДафмЗИкТвАуГчШугбЧмЛШщсОЧбБкП', 'ЩСМуХМ', 'АУсмдЗБвКфЩ', 'пгбТНОйц', 'МоИ', 'КОйкзОЕИЗМЩ', 'чщттЛРНнГхЗхХй', 'ЩшцЧРКмШЖЩЦемтЧУЛГкХтВНзОжУХТпН', 'ЕшбБНчрДпЩЧМлераУЖХйфйдчтсчПШ', 'дбФйтИАшДйЩтбФйШуПиРлГмВОШаСлШЧИвфЖщгж', 'ОДжТЦщпщИжфуеЩмн', 'ПЛНЕзжСчВКДттуФРУ', 'БбмеГЩХшжрцОжХНииВКВлдиХБДСмнНфХЛТХ']) from system.numbers limit 10; +select 4 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ЕКаЖСЗЗЕЗгПдШкфцЙТцл'), ['ЙКМИХРОХ', 'НвМУХзфчДбАРЙДу', 'чмщжФшшжсЗТв', 'жСЗзеЗг', 'ЛФсКзВСдЦД', 'АЖсЗЗЕЗГ', 'Пдшкфц', 'усйсКщшрДрвнФЛедуГХ', '', 'цйтЦ', 'Ощс', 'ЕЗГпдшКф', 'ззеЗгп', 'УгЛйхШТтшрЛ', 'ЗзЕЗгП', 'КЛмТЩРтрзБбЩРгФбиОБазУнтУЦ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('чЕжАфАрБпКбДмшАшТШККауЩИхНВО'), ['ЧЙпЗЧЧлйПЙЖЙшККг', 'зйхуМЩАИПГЗА', 'ЙцехноХниИбзБЧ', 'чВомЗОфУроС', 'дбРхХЗрзоДДШщЕДжиФаЙ', 'еЛзТцЩДиДГрдМОНЧУнеТуДЩЧЦпГЕщПОРсйпЧ', 'ФчнпМРЧцПЙЩЩвфДХПнУхцЩСИ', 'цлШеУкМБнжЧлУцСуСЙуотшМфйс', 'лугГлкщКщкзЛйпбдсишргДДшОувр', 'ЗРИаФЛЗФрСзм', 'аЗвжВгхЩоЦ', 'чГКлеБНДнИЖЧеШЧДнИвсГДЖЖфБМНсУЦосВс', 'щЦнПУзЧщнЩЕ', 'рВУв']) from system.numbers limit 10; +select 20 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('анктгЦВВкЧвЖиБпфТйлр'), ['НшДПчтсСЧпкидаХжаЙчаДчЦГшГ', 'ХнцЛШИрХВаРхнЧИЙрОЛЛИТпППфгЖЩФ', 'ФАЛущПупмдМБмтйзУшрВМзцзШжгД', 'ГчЛЧеЛДХеипдшЦЦмаШНаРшУТ', 'фОЕфжО', 'ТНсУАнчшУЛЦкцчЙ', 'ЛйЦКБЗГЦйКЩиОПуТЦкБкБувснЙи', 'Бунф', 'ИтХЛШСУНЦВйРСЙчДчНвйшЗЦй', 'АцСКнзБаЖУДЖегавйБгужШАДЙтжИВк', 'ЦцХщфирДПрСуХзхЖМЕщ', 'кфдБЖКншвУФкЗДКуЙ', 'СкиСЦЗЦРмгЦНпБхфХДЙщЛзХ', 'йУепВЖАПНбАЩуЛжвЧпхМ', 'БпЧшпДочУвибщерйхйтОБАСПнЧМИОЩ', 'чФгНЗщвхавбшсООоВштбЧ', 'уДиЕцнЙХВЕйИАГдЕ', 'тп', 'ЧЕРЖсгВ', 'вЖибПФТЙЛ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ипозйпхЛОЛТлСМХЩдМвМгШИвГиЛп'), ['ФСГзиГррБДНКГЛХбААФхИ', 'гегпАвхДЕ', 'ЦХжзщХИвхп', 'ЗЖ', 'ХОКцКзЩо', 'абИОрГПМТКшБ', 'кмХТмФихСЦсшУдхВбИШМНАНмпмХОЗйПЩч', 'еОжТСкфЕТУУжГ', 'НтщМЕПЧИКЙКйй', 'ежСикИвйЛж', 'ушЩФОтпБзЩЛЗЦЧЙиВгБЧоПХНгОуАДТЙж', 'фМЕРефнутпнцФРнрГЖ', 'хшДЧзнХпфорвЩжмГРЦуХГ', 'ЧЖн', 'вВзгОСхгНумм', 'ЗДоВлСжпфщСКсщХаолЛнЛЗбСхвЩвЩНоЩЩМ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('МрЗтВФуЖРеЕШЧхПФбжжхчД'), ['щжОожЦндцШйТАй', 'йуРСЦУЗФУЦПвРфевСлфдРещЦтИтЩЩТг', 'ЕГЧдмХмРАлнЧ', 'йнкФизГСЗнуКбЙВЙчАТТрСхаЙШтсдгХ', 'ЧПрнРЖЙцХИщ', 'зЕ', 'СжВЩчГзБХбйТиклкдШШИееАлЧЩН', 'МШщГйБХжЙпйЕЗТзКмпе', 'НКбНщОМДзлдЧОс', 'НчзВХОпХХШМОХФумБгсрРЧИчВтгутВЩо']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('упТУЖелФкЧЧУЦРжоБтХсжКщД'), ['щКшуОЖааЖйнЕбДИжМК', 'ЕкнШцХРВтНйШоНбЙйУоЧщУиРпШЧхмКЧжх', 'рвЩЗоЗхшЗвлизкСзебЩКМКжбша', 'ДииБНСШвцЦбаСсИжЕЗмхмВ', 'СЦоБЗПМтмшрУлрДТФГЖиувШЗууШзв', 'ЦЗБЕзВХЙбйВОмЗпхндЗ', 'ЗНизЧВШкГВтпсЖж', 'уШиБПЙЧтРаЕгИ', 'ЙшпПА', 'ЧоММаАйМСфбхуФкефФштгУА']) from system.numbers limit 10; + +select 0 = multiSearchFirstPosition(materialize('abcdefgh'), ['z', 'pq']) from system.numbers limit 10; +select 1 = multiSearchFirstPosition(materialize('abcdefgh'), ['a', 'b', 'c', 'd']) from system.numbers limit 10; +select 1 = multiSearchFirstPosition(materialize('abcdefgh'), ['defgh', 'bcd', 'abcd', 'c']) from system.numbers limit 10; +select 1 = multiSearchFirstPosition(materialize('abcdefgh'), ['', 'bcd', 'bcd', 'c']) from system.numbers limit 10; +select 2 = multiSearchFirstPosition(materialize('abcdefgh'), ['something', 'bcd', 'bcd', 'c']) from system.numbers limit 10; +select 6 = multiSearchFirstPosition(materialize('abcdefgh'), ['something', 'bcdz', 'fgh', 'f']) from system.numbers limit 10; + +select 0 = multiSearchFirstPositionCaseInsensitive(materialize('abcdefgh'), ['z', 'pq']) from system.numbers limit 10; +select 1 = multiSearchFirstPositionCaseInsensitive(materialize('aBcdefgh'), ['A', 'b', 'c', 'd']) from system.numbers limit 10; +select 1 = multiSearchFirstPositionCaseInsensitive(materialize('abCDefgh'), ['defgh', 'bcd', 'aBCd', 'c']) from system.numbers limit 10; +select 1 = multiSearchFirstPositionCaseInsensitive(materialize('abCdeFgH'), ['', 'bcd', 'bcd', 'c']) from system.numbers limit 10; +select 2 = multiSearchFirstPositionCaseInsensitive(materialize('ABCDEFGH'), ['something', 'bcd', 'bcd', 'c']) from system.numbers limit 10; +select 6 = multiSearchFirstPositionCaseInsensitive(materialize('abcdefgh'), ['sOmEthIng', 'bcdZ', 'fGh', 'F']) from system.numbers limit 10; + +select 0 = multiSearchFirstPositionUTF8(materialize('абвгдежз'), ['л', 'ъ']) from system.numbers limit 10; +select 1 = multiSearchFirstPositionUTF8(materialize('абвгдежз'), ['а', 'б', 'в', 'г']) from system.numbers limit 10; +select 1 = multiSearchFirstPositionUTF8(materialize('абвгдежз'), ['гдежз', 'бвг', 'абвг', 'вг']) from system.numbers limit 10; +select 1 = multiSearchFirstPositionUTF8(materialize('абвгдежз'), ['', 'бвг', 'бвг', 'в']) from system.numbers limit 10; +select 2 = multiSearchFirstPositionUTF8(materialize('абвгдежз'), ['что', 'в', 'гдз', 'бвг']) from system.numbers limit 10; +select 6 = multiSearchFirstPositionUTF8(materialize('абвгдежз'), ['з', 'бвгя', 'ежз', 'з']) from system.numbers limit 10; + +select 0 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДежз'), ['Л', 'Ъ']) from system.numbers limit 10; +select 1 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДежз'), ['А', 'б', 'в', 'г']) from system.numbers limit 10; +select 1 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДежз'), ['гДеЖз', 'бВг', 'АБВг', 'вг']) from system.numbers limit 10; +select 1 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДежз'), ['', 'бвг', 'Бвг', 'в']) from system.numbers limit 10; +select 2 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДежз'), ['что', 'в', 'гдз', 'бвг']) from system.numbers limit 10; +select 6 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДежЗ'), ['З', 'бвгЯ', 'ЕЖз', 'з']) from system.numbers limit 10; + +-- 254 +select +[ +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1 +] = +multiSearchAllPositions(materialize('string'), +['o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'str']); + +select 254 = multiSearchFirstIndex(materialize('string'), +['o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'str']); + + +select +[ +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, +0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1 +] = +multiSearchAllPositions(materialize('string'), +['o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'str']); + +select 255 = multiSearchFirstIndex(materialize('string'), +['o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'str']); + +select multiSearchAllPositions(materialize('string'), +['o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'str']); -- { serverError 42 } + +select multiSearchFirstIndex(materialize('string'), +['o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', +'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'str']); -- { serverError 42 } + + +select [1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1]= multiSearchAllPositions(materialize('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'), +['aaaa', 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', +'aaaa']); From 5f18640215159ed1ad50be1efce2cb996a49fd73 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Jul 2022 23:16:54 +0200 Subject: [PATCH 320/659] Fix universal installation script --- docs/_includes/install/universal.sh | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/_includes/install/universal.sh b/docs/_includes/install/universal.sh index 59b814abf32..c2970924fb0 100755 --- a/docs/_includes/install/universal.sh +++ b/docs/_includes/install/universal.sh @@ -7,31 +7,31 @@ DIR= if [ "${OS}" = "Linux" ] then - if [ "${ARCH}" = "x86_64" ] + if [ "${ARCH}" = "x86_64" -o "${ARCH}" = "amd64" ] then DIR="amd64" - elif [ "${ARCH}" = "aarch64" ] + elif [ "${ARCH}" = "aarch64" -o "${ARCH}" = "arm64" ] then DIR="aarch64" - elif [ "${ARCH}" = "powerpc64le" ] || [ "${ARCH}" = "ppc64le" ] + elif [ "${ARCH}" = "powerpc64le" -o "${ARCH}" = "ppc64le" ] then DIR="powerpc64le" fi elif [ "${OS}" = "FreeBSD" ] then - if [ "${ARCH}" = "x86_64" ] + if [ "${ARCH}" = "x86_64" -o "${ARCH}" = "amd64" ] then DIR="freebsd" - elif [ "${ARCH}" = "aarch64" ] + elif [ "${ARCH}" = "aarch64" -o "${ARCH}" = "arm64" ] then DIR="freebsd-aarch64" - elif [ "${ARCH}" = "powerpc64le" ] || [ "${ARCH}" = "ppc64le" ] + elif [ "${ARCH}" = "powerpc64le" -o "${ARCH}" = "ppc64le" ] then DIR="freebsd-powerpc64le" fi elif [ "${OS}" = "Darwin" ] then - if [ "${ARCH}" = "x86_64" ] + if [ "${ARCH}" = "x86_64" -o "${ARCH}" = "amd64" ] then DIR="macos" elif [ "${ARCH}" = "aarch64" -o "${ARCH}" = "arm64" ] From 03aec06da1cef6886b9209c5bc6af93983ac8221 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Jul 2022 23:20:05 +0200 Subject: [PATCH 321/659] Whitespaces --- src/Functions/registerFunctions.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Functions/registerFunctions.cpp b/src/Functions/registerFunctions.cpp index bafaf61c2f5..f578bfc9d68 100644 --- a/src/Functions/registerFunctions.cpp +++ b/src/Functions/registerFunctions.cpp @@ -60,7 +60,7 @@ void registerFunctionsTimeWindow(FunctionFactory &); void registerFunctionToBool(FunctionFactory &); void registerFunctionMinSampleSize(FunctionFactory &); -// meilisearch +/// For Meilisearch void registerFunctionMeiliMatch(FunctionFactory & factory); #if USE_SSL @@ -126,8 +126,11 @@ void registerFunctions() registerFunctionsTimeWindow(factory); registerFunctionToBool(factory); registerFunctionMinSampleSize(factory); + registerFunctionTid(factory); + registerFunctionLogTrace(factory); + registerFunctionHashID(factory); - //meilisearch + /// For Meilisearch registerFunctionMeiliMatch(factory); #if USE_SSL @@ -137,9 +140,6 @@ void registerFunctions() registerFunctionAESDecryptMysql(factory); registerFunctionShowCertificate(factory); #endif - registerFunctionTid(factory); - registerFunctionLogTrace(factory); - registerFunctionHashID(factory); } } From 4e9adc5211806eb272a20ad5c9a207b72c7b806d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Sun, 17 Jul 2022 23:12:37 +0000 Subject: [PATCH 322/659] Add test --- tests/queries/0_stateless/02364_window_case.reference | 10 ++++++++++ tests/queries/0_stateless/02364_window_case.sql | 4 ++++ 2 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/02364_window_case.reference create mode 100644 tests/queries/0_stateless/02364_window_case.sql diff --git a/tests/queries/0_stateless/02364_window_case.reference b/tests/queries/0_stateless/02364_window_case.reference new file mode 100644 index 00000000000..f00c965d830 --- /dev/null +++ b/tests/queries/0_stateless/02364_window_case.reference @@ -0,0 +1,10 @@ +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 diff --git a/tests/queries/0_stateless/02364_window_case.sql b/tests/queries/0_stateless/02364_window_case.sql new file mode 100644 index 00000000000..b34686c3c9a --- /dev/null +++ b/tests/queries/0_stateless/02364_window_case.sql @@ -0,0 +1,4 @@ +SELECT CASE + WHEN sum(number) over () > 0 THEN number + 1 + ELSE 0 END +FROM numbers(10) From 577978026aa9cd5681f85bcceb3d3548c44e2e21 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 17 Jul 2022 21:09:01 -0300 Subject: [PATCH 323/659] Update partition.md --- docs/en/sql-reference/statements/alter/partition.md | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 079d462a536..b7787fbef92 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -108,7 +108,8 @@ Note that data will be deleted neither from `table1` nor from `table2`. 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. +- Both tables must have the same partition key, the same order by key and the same primary key. +- Both tables must have the same storage policy (a disk where the partition is stored should be available for both tables). ## REPLACE PARTITION @@ -121,7 +122,8 @@ This query copies the data partition from the `table1` to `table2` and replaces 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. +- Both tables must have the same partition key, the same order by key and the same primary key. +- Both tables must have the same storage policy (a disk where the partition is stored should be available for both tables). ## MOVE PARTITION TO TABLE @@ -134,9 +136,9 @@ This query moves the data partition from the `table_source` to `table_dest` with 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. +- Both tables must have the same partition key, the same order by key and the same primary key. +- Both tables must have the same storage policy (a disk where the partition is stored should be available for both tables). - Both tables must be the same engine family (replicated or non-replicated). -- Both tables must have the same storage policy. ## CLEAR COLUMN IN PARTITION From 27326da00b80ff63155fdd5f9ca7fa7582613605 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 17 Jul 2022 21:12:53 -0300 Subject: [PATCH 324/659] Update partition.md --- docs/ru/sql-reference/statements/alter/partition.md | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/ru/sql-reference/statements/alter/partition.md b/docs/ru/sql-reference/statements/alter/partition.md index 036f72fc951..75fcedf501f 100644 --- a/docs/ru/sql-reference/statements/alter/partition.md +++ b/docs/ru/sql-reference/statements/alter/partition.md @@ -106,7 +106,8 @@ ALTER TABLE table2 [ON CLUSTER cluster] ATTACH PARTITION partition_expr FROM tab Следует иметь в виду: - Таблицы должны иметь одинаковую структуру. -- Для таблиц должен быть задан одинаковый ключ партиционирования. +- Для таблиц должна быть задана одинаковая политика хранения (диск, на котором хранится партиция, должен быть доступен для обеих таблиц). +- Для таблиц должен быть задан одинаковый ключ партиционирования, одинаковый ключ сортировки и одинаковый первичный ключ. Подробнее о том, как корректно задать имя партиции, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr). @@ -121,7 +122,8 @@ ALTER TABLE table2 [ON CLUSTER cluster] REPLACE PARTITION partition_expr FROM ta Следует иметь в виду: - Таблицы должны иметь одинаковую структуру. -- Для таблиц должен быть задан одинаковый ключ партиционирования. +- Для таблиц должна быть задана одинаковая политика хранения (диск, на котором хранится партиция, должен быть доступен для обеих таблиц). +- Для таблиц должен быть задан одинаковый ключ партиционирования, одинаковый ключ сортировки и одинаковый первичный ключ. Подробнее о том, как корректно задать имя партиции, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr). @@ -136,9 +138,9 @@ ALTER TABLE table_source [ON CLUSTER cluster] MOVE PARTITION partition_expr TO T Следует иметь в виду: - Таблицы должны иметь одинаковую структуру. -- Для таблиц должен быть задан одинаковый ключ партиционирования. +- Для таблиц должен быть задан одинаковый ключ партиционирования, одинаковый ключ сортировки и одинаковый первичный ключ. +- Для таблиц должна быть задана одинаковая политика хранения (диск, на котором хранится партиция, должен быть доступен для обеих таблиц). - Движки таблиц должны быть одинакового семейства (реплицированные или нереплицированные). -- Для таблиц должна быть задана одинаковая политика хранения. ## CLEAR COLUMN IN PARTITION {#alter_clear-column-partition} From 429b19da8b7663f3b942b869a4111266e6831a11 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 17 Jul 2022 21:16:40 -0300 Subject: [PATCH 325/659] Update partition.md --- docs/ru/sql-reference/statements/alter/partition.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/statements/alter/partition.md b/docs/ru/sql-reference/statements/alter/partition.md index 75fcedf501f..aecf954a45a 100644 --- a/docs/ru/sql-reference/statements/alter/partition.md +++ b/docs/ru/sql-reference/statements/alter/partition.md @@ -106,8 +106,8 @@ ALTER TABLE table2 [ON CLUSTER cluster] ATTACH PARTITION partition_expr FROM tab Следует иметь в виду: - Таблицы должны иметь одинаковую структуру. -- Для таблиц должна быть задана одинаковая политика хранения (диск, на котором хранится партиция, должен быть доступен для обеих таблиц). - Для таблиц должен быть задан одинаковый ключ партиционирования, одинаковый ключ сортировки и одинаковый первичный ключ. +- Для таблиц должна быть задана одинаковая политика хранения (диск, на котором хранится партиция, должен быть доступен для обеих таблиц). Подробнее о том, как корректно задать имя партиции, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr). @@ -122,8 +122,8 @@ ALTER TABLE table2 [ON CLUSTER cluster] REPLACE PARTITION partition_expr FROM ta Следует иметь в виду: - Таблицы должны иметь одинаковую структуру. -- Для таблиц должна быть задана одинаковая политика хранения (диск, на котором хранится партиция, должен быть доступен для обеих таблиц). - Для таблиц должен быть задан одинаковый ключ партиционирования, одинаковый ключ сортировки и одинаковый первичный ключ. +- Для таблиц должна быть задана одинаковая политика хранения (диск, на котором хранится партиция, должен быть доступен для обеих таблиц). Подробнее о том, как корректно задать имя партиции, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr). From 10d0a5e95b626f4c3fd8927479c08e0fb549f775 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Jul 2022 07:05:34 +0000 Subject: [PATCH 326/659] More split up of tests to reduce runtimes --- ...2364_multiSearch_function_family.reference | 3642 ----------------- .../02364_multiSearch_function_family.sql | 511 --- .../02365_multisearch_random_tests.reference | 3641 ++++++++++++++++ .../02365_multisearch_random_tests.sql | 379 ++ 4 files changed, 4020 insertions(+), 4153 deletions(-) create mode 100644 tests/queries/0_stateless/02365_multisearch_random_tests.reference create mode 100644 tests/queries/0_stateless/02365_multisearch_random_tests.sql diff --git a/tests/queries/0_stateless/02364_multiSearch_function_family.reference b/tests/queries/0_stateless/02364_multiSearch_function_family.reference index 4ddb6036240..eb93a2509b6 100644 --- a/tests/queries/0_stateless/02364_multiSearch_function_family.reference +++ b/tests/queries/0_stateless/02364_multiSearch_function_family.reference @@ -12872,3645 +12872,3 @@ 1 1 1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/02364_multiSearch_function_family.sql b/tests/queries/0_stateless/02364_multiSearch_function_family.sql index c814fdcfe04..4ad1a68eeb7 100644 --- a/tests/queries/0_stateless/02364_multiSearch_function_family.sql +++ b/tests/queries/0_stateless/02364_multiSearch_function_family.sql @@ -423,384 +423,6 @@ select 1 = multiSearchAny(materialize('abab'), ['ab']); select 1 = multiSearchAny(materialize('abababababababababababab'), ['abab']); select 1 = multiSearchAny(materialize('abababababababababababab'), ['abababababababababa']); --- select 'some random tests'; - -select [4, 1, 1, 2, 6, 1, 1, 0, 4, 1, 14, 0, 10, 0, 16, 6] = multiSearchAllPositions(materialize('jmdqwjbrxlbatqeixknricfk'), ['qwjbrxlba', 'jmd', '', 'mdqwjbrxlbatqe', 'jbrxlbatqeixknric', 'jmdqwjbrxlbatqeixknri', '', 'fdtmnwtts', 'qwjbrxlba', '', 'qeixknricfk', 'hzjjgrnoilfkvzxaemzhf', 'lb', 'kamz', 'ixknr', 'jbrxlbatq']) from system.numbers limit 10; -select [0, 0, 0, 2, 3, 0, 1, 0, 5, 0, 0, 0, 11, 10, 6, 7] = multiSearchAllPositions(materialize('coxcctuehmzkbrsmodfvx'), ['bkhnp', 'nlypjvriuk', 'rkslxwfqjjivcwdexrdtvjdtvuu', 'oxcctuehm', 'xcctuehmzkbrsm', 'kfrieuocovykjmkwxbdlkgwctwvcuh', 'coxc', 'lbwvetgxyndxjqqwthtkgasbafii', 'ctuehmzkbrsmodfvx', 'obzldxjldxowk', 'ngfikgigeyll', 'wdaejjukowgvzijnw', 'zkbr', 'mzkb', 'tuehm', 'ue']) from system.numbers limit 10; -select [1, 1, 0, 0, 0, 1, 1, 1, 4, 0, 6, 6, 0, 10, 1, 5] = multiSearchAllPositions(materialize('mpswgtljbbrmivkcglamemayfn'), ['', 'm', 'saejhpnfgfq', 'rzanrkdssmmkanqjpfi', 'oputeneprgoowg', 'mp', '', '', 'wgtljbbrmivkcglamemay', 'cbpthtrgrmgfypizi', 'tl', 'tlj', 'xuhs', 'brmivkcglamemayfn', '', 'gtljb']) from system.numbers limit 10; -select [1, 0, 0, 8, 6, 0, 7, 1, 3, 0, 0, 0, 0, 12] = multiSearchAllPositions(materialize('arbphzbbecypbzsqsljurtddve'), ['arbphzb', 'mnrboimjfijnti', 'cikcrd', 'becypbz', 'z', 'uocmqgnczhdcrvtqrnaxdxjjlhakoszuwc', 'bbe', '', 'bp', 'yhltnexlpdijkdzt', 'jkwjmrckvgmccmmrolqvy', 'vdxmicjmfbtsbqqmqcgtnrvdgaucsgspwg', 'witlfqwvhmmyjrnrzttrikhhsrd', 'pbzsqsljurt']) from system.numbers limit 10; -select [7, 0, 0, 8, 0, 2, 0, 0, 6, 0, 2, 0, 3, 1] = multiSearchAllPositions(materialize('aizovxqpzcbbxuhwtiaaqhdqjdei'), ['qpzcbbxuhw', 'jugrpglqbm', 'dspwhzpyjohhtizegrnswhjfpdz', 'pzcbbxuh', 'vayzeszlycke', 'i', 'gvrontcpqavsjxtjwzgwxugiyhkhmhq', 'gyzmeroxztgaurmrqwtmsxcqnxaezuoapatvu', 'xqpzc', 'mjiswsvlvlpqrhhptqq', 'iz', 'hmzjxxfjsvcvdpqwtrdrp', 'zovxqpzcbbxuhwtia', 'ai']) from system.numbers limit 10; -select [0, 0, 0, 19, 14, 22, 10, 0, 0, 13, 0, 8] = multiSearchAllPositions(materialize('ydfgiluhyxwqdfiwtzobwzscyxhuov'), ['srsoubrgghleyheujsbwwwykerzlqphgejpxvog', 'axchkyleddjwkvbuyhmekpbbbztxdlm', 'zqodzvlkmfe', 'obwz', 'fi', 'zsc', 'xwq', 'pvmurvrd', 'uulcdtexckmrsokmgdpkstlkoavyrmxeaacvydxf', 'dfi', 'mxcngttujzgtlssrmluaflmjuv', 'hyxwqdfiwtzobwzscyxhu']) from system.numbers limit 10; -select [6, 1, 1, 0, 0, 5, 1, 0, 8, 0, 5, 0, 2, 12, 0, 15, 0, 0] = multiSearchAllPositions(materialize('pyepgwainvmwekwhhqxxvzdjw'), ['w', '', '', 'gvvkllofjnxvcu', 'kmwwhboplctvzazcyfpxhwtaddfnhekei', 'gwainv', 'pyepgwain', 'ekpnogkzzmbpfynsunwqp', 'invmwe', 'hrxpiplfplqjsstuybksuteoz', 'gwa', 'akfpyduqrwosxcbdemtxrxvundrgse', 'yepgwainvmw', 'wekwhhqxxvzdjw', 'fyimzvedmyriubgoznmcav', 'whhq', 'ozxowbwdqfisuupyzaqynoprgsjhkwlum', 'vpoufrofekajksdp']) from system.numbers limit 10; -select [0, 0, 5, 1, 1, 0, 15, 1, 5, 10, 4, 0, 1, 0, 3, 0, 0, 0] = multiSearchAllPositions(materialize('lqwahffxurkbhhzytequotkfk'), ['rwjqudpuaiufle', 'livwgbnflvy', 'hffxurkbhh', '', '', 'xcajwbqbttzfzfowjubmmgnmssat', 'zytequ', 'lq', 'h', 'rkbhh', 'a', 'immejthwgdr', '', 'llhhnlhcvnxxorzzjt', 'w', 'cvjynqxcivmmmvc', 'wexjomdcmursppjtsweybheyxzleuz', 'fzronsnddfxwlkkzidiknhpjipyrcrzel']) from system.numbers limit 10; -select [0, 1, 11, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4, 1] = multiSearchAllPositions(materialize('nkddriylnakicwgdwrfxpodqea'), ['izwdpgrgpmjlwkanjrffgela', '', 'kicw', 'hltmfymgmrjckdiylkzjlvvyuleksikdjrg', 'yigveskrbidknjxigwilmkgyizewikh', 'xyvzhsnqmuec', 'odcgzlavzrwesjks', 'oilvfgliktoujukpgzvhmokdgkssqgqot', 'llsfsurvimbahwqtbqbp', 'nxj', 'pimydixeobdxmdkvhcyzcgnbhzsydx', 'couzmvxedobuohibgxwoxvmpote', 'driylnakicwgdwrf', 'nkddr']) from system.numbers limit 10; -select [0, 0, 0, 3, 0, 15, 0, 0, 12, 7, 0, 0, 0, 0, 5, 0] = multiSearchAllPositions(materialize('jnckhtjqwycyihuejibqmddrdxe'), ['tajzx', 'vuddoylclxatcjvinusdwt', 'spxkhxvzsljkmnzpeubszjnhqczavgtqopxn', 'ckhtjqwycyi', 'xlbfzdxspldoes', 'u', 'czosfebeznt', 'gzhabdsuyreisxvyfrfrkq', 'yihuejibqmd', 'jqwycyihuejibqm', 'cfbvprgzx', 'hxu', 'vxbhrfpzacgd', 'afoaij', 'htjqwycyihu', 'httzbskqd']) from system.numbers limit 10; -select [0, 0, 12, 4, 4, 0, 13, 23, 0, 1, 0, 2, 0, 0, 0, 3, 0, 0] = multiSearchAllPositions(materialize('dzejajvpoojdkqbnayahygidyrjmb'), ['khwxxvtnqhobbvwgwkpusjlhlzifiuclycml', 'nzvuhtwdaivo', 'dkqbnayahygidyr', 'jajvpoo', 'j', 'wdtbvwmeqgyvetu', 'kqbn', 'idyrjmb', 'tsnxuxevsxrxpgpfdgrkhwqpkse', '', 'efsdgzuefhdzkmquxu', 'zejajvpoojdkqbnayahyg', 'ugwfuighbygrxyctop', 'fcbxzbdugc', 'dxmzzrcplob', 'ejaj', 'wmmupyxrylvawsyfccluiiene', 'ohzmsqhpzbafvbzqwzftbvftei']) from system.numbers limit 10; -select [6, 8, 1, 4, 0, 10, 0, 1, 14, 0, 1, 0, 5, 0, 0, 0, 0, 15, 0, 1] = multiSearchAllPositions(materialize('ffaujlverosspbzaqefjzql'), ['lvero', 'erossp', 'f', 'ujlverosspbz', 'btfimgklzzxlbkbuqyrmnud', 'osspb', 'muqexvtjuaar', 'f', 'bzaq', 'lprihswhwkdhqciqhfaowarn', 'ffaujlve', 'uhbbjrqjb', 'jlver', 'umucyhbbu', 'pjthtzmgxhvpbdphesnnztuu', 'xfqhfdfsbbazactpastzvzqudgk', 'lvovjfoatc', 'z', 'givejzhoqsd', '']) from system.numbers limit 10; -select [5, 7, 0, 1, 6, 0, 0, 1, 1, 2, 0, 1, 4, 2, 0, 6, 0, 0] = multiSearchAllPositions(materialize('hzftozkvquknsahhxefzg'), ['ozkvquknsahhxefzg', 'kv', 'lkdhmafrec', '', 'zkvquknsahh', 'xmjuizyconipirigdmhqclox', 'dqqwolnkkwbyyjicsoshidbay', '', '', 'zf', 'sonvmkapcjcakgpejvn', 'hzftoz', 't', 'zftozkvqukns', 'dyuqohvehxsvdzdlqzl', 'zkvquknsahhx', 'vueohmytvmglqwptfbhxffspf', 'ilkdurxg']) from system.numbers limit 10; -select [1, 7, 6, 4, 0, 1, 0, 0, 0, 9, 7, 1, 1, 0, 0, 0] = multiSearchAllPositions(materialize('aapdygjzrhskntrphianzjob'), ['', 'jz', 'gjzrh', 'dygjzrhskntrphia', 'qcnahphlxmdru', '', 'rnwvzdn', 'isbekwuivytqggsxniqojrvpwjdr', 'sstwvgyavbwxvjojrpg', 'rhskn', 'jzrhskntrp', '', '', 'toilvppgjizaxtidizgbgygubmob', 'vjwzwpvsklkxqgeqqmtssnhlmw', 'znvpjjlydvzhkt']) from system.numbers limit 10; -select [0, 1, 0, 1, 0, 0, 10, 0, 0, 0, 11, 0, 5, 0] = multiSearchAllPositions(materialize('blwpfdjjkxettfetdoxvxbyk'), ['wgylnwqcrojacofrcanjme', 'bl', 'qqcunzpvgi', '', 'ijemdmmdxkakrawwdqrjtrttig', 'qwkaifalc', 'xe', 'zqocnfuvzowuqkmwrfxw', 'xpaayeljvly', 'wvphqqhulpepjjjnxjfudfcomajc', 'ettfetdoxvx', 'ikablovwhnbohibbuhwjshhdemidgreqf', 'fdjjkxett', 'kiairehwbxveqkcfqhgopztgpatljgqp']) from system.numbers limit 10; -select [0, 0, 6, 1, 1, 0, 0, 1, 2, 0, 0, 0, 0, 0] = multiSearchAllPositions(materialize('vghzgedqpnqtvaoonwsz'), ['mfyndhucfpzjxzaezny', 'niejb', 'edqpnqt', '', 'v', 'kivdvealqadzdatziujdnvymmia', 'lvznmgwtlwevcxyfbkqc', 'vghzge', 'gh', 'tbzle', 'vjiqponbvgvguuhqdijbdeu', 'mshlyabasgukboknbqgmmmj', 'kjk', 'abkeftpnpvdkfyrxbrihyfxcfxablv']) from system.numbers limit 10; -select [0, 0, 0, 0, 9, 0, 7, 0, 9, 8, 0, 0] = multiSearchAllPositions(materialize('oaghnutqsqcnwvmzrnxgacsovxiko'), ['upien', 'moqszigvduvvwvmpemupvmmzctbrbtqggrk', 'igeiaccvxejtfvifrmimwpewllcggji', 'wnwjorpzgsqiociw', 'sq', 'rkysegpoej', 'tqsqcnwvmzrnxgacsovxiko', 'ioykypvfjufbicpyrpfuhugk', 's', 'qsqcnwvmzrnxgacsov', 'hhbeisvmpnkwmimgyfmybtljiu', 'kfozjowd']) from system.numbers limit 10; -select [0, 5, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 5, 0, 0, 14, 1, 1, 0, 0] = multiSearchAllPositions(materialize('wbjfsevqspsvbwlzrkhcfuhxddbq'), ['ltgjbz', 's', 'qdfnmggupdfxjfnmvwyrqopxtxf', 'sazlkmaikcltojbzbmdfddu', 'yzanifqxufyfwrxzkhngoxkrrph', 'iwskc', 'xkykshryphyfnwcnmjfqjrixykmzmwm', 'wwpenztbhkdbwidfkypqlxivsjs', 'rlkevy', 'qigywtkezwd', '', 'c', 'sevqspsvbwlzrk', 'gwg', 'iduhrjsrtodxdkjykjoghtjtvplrscitxnvt', 'wlzrkhcfuhxddb', '', 'wbjfsev', 'zytusrcvqbazb', 'tec']) from system.numbers limit 10; -select [0, 1, 5, 0, 6, 8, 0, 3, 2, 0, 0, 9, 0, 4, 0, 0] = multiSearchAllPositions(materialize('mxiifpzlovgfozpgirtio'), ['srullnscuzenzhp', '', 'f', 'apetxezid', 'pzlovgf', 'lo', 'ecbmso', 'i', 'xiifpzlovgfozpgir', 'bnefwypvctubvslsesnctqspdyctq', 'tdncmgbikboss', 'o', 'zmgobcarxlxaho', 'ifpzlovgfozpg', 'dwmjqyylvsxzfr', 'pxhrecconce']) from system.numbers limit 10; -select [0, 0, 0, 2, 0, 0, 2, 0, 8, 0, 0, 0, 7, 0, 0, 0, 21, 3, 1, 8] = multiSearchAllPositions(materialize('jtvnrdpdevgnzexqdrrxqgiujexhm'), ['ibkvzoqmiyfgfztupug', 'iqzeixfykxcghlbgsicxiywlurrgjsywwk', 'vzdffjzlqxgzdcrkgoro', 'tvnrdpdevgnzexqdr', 'nqywueahcmoojtyjlhfpysk', 'iqalixciiidvrtmpzozfb', 'tv', 'rxkfeasoff', 'devgnzexqdrrxqgiuj', 'kvvuvyplboowjrestyvdfrxdjjujvkxy', 'shkhpneekuyyqtxfxutvz', 'yy', 'pdevgnz', 'nplpydxiwnbvlhoorcmqkycqisi', 'jlkxplbftfkxqgnqnaw', 'qdggpjenbrwbjtorbi', 'qgiuje', 'vnrdpd', '', 'dev']) from system.numbers limit 10; -select [14, 0, 0, 7, 20, 6, 0, 13, 0, 0, 20, 0, 20, 2, 0, 8, 2, 11, 2, 0] = multiSearchAllPositions(materialize('asjwxabjrwgcdviokfaoqvqiafz'), ['v', 'zqngytligwwpzxhatyayvdnbbj', 'gjicovfzgbyagiirn', 'bjrwgcdviok', 'oqvqiafz', 'abjrwgc', 'wulrpfzh', 'dviokfao', 'esnchjuiufjadqmdtrpcd', 'tkodqzsjchpaftk', 'oqvq', 'eyoshlrlvmnqjmtmloryvg', 'oqv', 'sjwx', 'uokueelyytnoidplwmmox', 'jrwgcdviokfaoqvqiaf', 'sjwxabjrwgcdviokfaoqvqi', 'gcdviokfa', 'sjwxab', 'zneabsnfucjcwauxmudyxibnmxzfx']) from system.numbers limit 10; -select [0, 16, 8, 0, 10, 0, 0, 0, 0, 1, 0, 6, 0, 1, 0, 4, 0, 6, 0, 0] = multiSearchAllPositions(materialize('soxfqagiuhkaylzootfjy'), ['eveprzxphyenbrnnznpctvxn', 'oo', 'iuhka', 'ikutjhrnvzfb', 'h', 'duyvvjizristnkczgwj', 'ihfrp', 'afpyrlj', 'uonp', 'soxfqagiuhkaylzootfjy', 'qeckxkoxldpzzpmkbvcex', 'agiuhkaylzo', 'tckcumkbsgrgqjvtlijack', '', 'fnfweqlldcdnwfaohqohp', 'fqagiuhkayl', 'pqnvwprxwwrcjqvfsbfimwye', 'agi', 'ta', 'r']) from system.numbers limit 10; -select [3, 7, 1, 6, 0, 1, 0, 11, 0, 9, 17, 1, 18, 12] = multiSearchAllPositions(materialize('ladbcypcbcxahmujwezkvweud'), ['db', 'pcbcxahm', 'lad', 'ypcb', 'atevkzyyxhphtuekymhh', 'lad', 'mltjrwaibetrtwpfa', 'xahmujwezkvweud', 'dg', 'bcxahmujw', 'we', '', 'e', 'ahmujwezkvw']) from system.numbers limit 10; -select [6, 0, 11, 0, 7, 0, 0, 0, 6, 1, 0, 3, 0, 0, 0, 0] = multiSearchAllPositions(materialize('hhkscgmqzmuwltmrhtxnnzsxl'), ['gmqzmuwltmrh', 'qtescwjubeqhurqoqfjauwxdoc', 'uwltmrh', 'qlhyfuspwdtecdbrmrqcnxghhlnbmzs', 'm', 'kcsuocwokvohnqonnfzmeiqtomehksehwc', 'hoxocyilgrxxoek', 'nisnlmbdczjsiw', 'gmqz', '', 'cqzz', 'k', 'utxctwtzelxmtioyqshxedecih', 'ifsmsljxzkyuigdtunwk', 'ojxvxwdosaqjhrnjwisss', 'dz']) from system.numbers limit 10; -select [0, 0, 19, 7, 0, 0, 1, 0, 0, 12, 0, 0, 1, 0, 1, 1, 5, 0, 23, 8] = multiSearchAllPositions(materialize('raxgcqizulxfwivauupqnofbijxfr'), ['sxvhaxlrpviwuinrcebtfepxxkhxxgqu', 'cuodfevkpszuimhymxypktdvicmyxm', 'pqnof', 'i', 'ufpljiniflkctwkwcrsbdhvrvkizticpqkgvq', 'osojyhejhrlhjvqrtobwthjgw', '', 'anzlevtxre', 'ufnpkjvgidirrnpvbsndfnovebdily', 'fwivauupqnofbi', 'rywyadwcvk', 'ltnlhftdfefmkenadahcpxw', '', 'xryluzlhnsqk', 'r', '', 'cqizulxfwivauupqnofb', 'y', 'fb', 'zulxfwivauupqnofbijxf']) from system.numbers limit 10; -select [4, 0, 0, 0, 0, 24, 1, 2, 0, 2, 0, 0, 8, 0] = multiSearchAllPositions(materialize('cwcqyjjodlepauupgobsgrzdvii'), ['q', 'yjppewylsqbnjwnhokzqtauggsjhhhkkkqsy', 'uutltzhjtc', 'pkmuptmzzeqhichaikwbggronli', 'erzgcuxnec', 'dvii', '', 'w', 'fkmpha', 'wcqyjjodlepauupgobsgrz', 'cbnmwirigaf', 'fcumlot', 'odlepauu', 'lthautlklktfukpt']) from system.numbers limit 10; -select [1, 1, 1, 1, 22, 0, 0, 8, 18, 15] = multiSearchAllPositions(materialize('vpscxxibyhvtmrdzrocvdngpb'), ['', '', '', '', 'n', 'agrahemfuhmftacvpnaxkx', 'dqqwvfsrqv', 'byhvtmrdzrocv', 'ocvdn', 'dzrocvdngpb']) from system.numbers limit 10; -select [1, 1, 1, 15, 10, 0, 0, 0, 0, 2] = multiSearchAllPositions(materialize('nfoievsrpvheprosjdsoiz'), ['', 'nfo', '', 'osjd', 'vheprosjdsoiz', 'az', 'blhvdycvjnxaipvxybs', 'umgxmpkvuvuvdaczkz', 'gfspmnzidixcjgjw', 'f']) from system.numbers limit 10; -select [0, 0, 2, 2, 0, 0, 0, 11, 10, 4, 9, 1, 6, 4, 0, 0] = multiSearchAllPositions(materialize('bdmfwdisdlgbcidshnhautsye'), ['uxdceftnmnqpveljer', 'xdnh', 'dmf', 'dmfwdisdlgbc', 'cpwnaijpkpyjgaq', 'doquvlrzhusjbxyqcqxvwr', 'llppnnmtqggyfoxtawnngsiiunvjjxxsufh', 'gbcidshnhau', 'lgbcids', 'f', 'dlgbc', 'bdmfwdisdlgbcids', 'disdlgbcidshnhautsy', 'fwdisdlgbcidshn', 'zfpbfc', 'triqajlyfmxlredivqiambigmge']) from system.numbers limit 10; -select [0, 0, 16, 0, 0, 0, 14, 6, 2, 1, 0, 0, 1, 0, 10, 12, 0, 0, 0, 0] = multiSearchAllPositions(materialize('absimumlxdlxuzpyrunivcb'), ['jglfzroni', 'wzfmtbjlcdxlbpialqjafjwz', 'yrun', 'fgmljkkp', 'nniob', 'fdektoyhxrumiycvkwekphypgti', 'zp', 'um', 'bsimu', '', 'yslsnfisaebuujltpgcskhhqcucdhb', 'xlaphsqgqsfykhilddctrawerneqoigb', '', 'pdvcfxdlurmegspidojt', 'd', 'xu', 'fdp', 'xjrqmybmccjbjtvyvdh', 'nvhdfatqi', 'neubuiykajzcrzdbvpwjhlpdmd']) from system.numbers limit 10; -select [0, 0, 0, 9, 0, 0, 1, 1, 1, 1] = multiSearchAllPositions(materialize('lvyenvktdnylszlypuwqecohy'), ['ihlsiynj', 'ctcnhbkumvbgfdclwjhsswpqyfrx', 'rpgqwkydwlfclcuupoynwrfffogxesvmbj', 'dnyl', 'coeqgdtbemkhgplprfxgwpl', 'dkbshktectbduxlcaptlzspq', 'l', 'lvyenvktdnylszlypuw', 'lvyenvk', '']) from system.numbers limit 10; -select [1, 0, 0, 0, 0, 1, 2, 22, 8, 17, 1, 13, 0, 0, 0, 0, 0, 5] = multiSearchAllPositions(materialize('wphcobonpgaqwgfenotzadgqezx'), ['', 'qeuycfhkfjwokxgrkaodqioaotkepzlhnrv', 'taehtytq', 'gejlcipocalc', 'poyvvvntrvqazixkwigtairjvxkgouiuva', '', 'phc', 'dg', 'npgaqwg', 'notzadgqe', '', 'wgfe', 'smipuxgvntys', 'qhrfdytbfeujzievelffzrv', 'cfmzw', 'hcywnyguzjredwjbqtwyuhtewuhzkc', 'tssfeinoykdauderpjyxtmb', 'obonpgaqwgfen']) from system.numbers limit 10; -select [0, 0, 0, 0, 0, 6, 6, 0, 0, 2, 0, 5, 2, 0, 6, 3] = multiSearchAllPositions(materialize('qvslufpsddtfudzrzlvrzdra'), ['jxsgyzgnjwyd', 'hqhxzhskwivpuqkjheywwfhthm', 'kbwlwadilqhgwlcpxkadkamsnzngms', 'fxunda', 'nlltydufobnfxjyhch', 'fpsddtfudzrzl', 'fp', 'ykhxjyqtvjbykskbejpnmbxpumknqucu', 'iyecekjcbkowdothxc', 'vslufpsddtfu', 'mjgtofkjeknlikrugkfhxlioicevil', 'uf', 'vslufpsdd', 'cxizdzygyu', 'fpsddtfudzrz', 'slufp']) from system.numbers limit 10; -select [12, 0, 0, 0, 0, 1, 6, 0, 1, 2] = multiSearchAllPositions(materialize('ydsbycnifbcforymknzfi'), ['forymkn', 'vgxtcdkfmjhc', 'ymugjvtmtzvghmifolzdihutqoisl', 'fzooddrlhi', 'bdefmxxdepcqi', '', 'cnif', 'ilzbhegpcnkdkooopaguljlie', '', 'dsbycnifbcforym']) from system.numbers limit 10; -select [0, 2, 4, 1, 1, 3, 0, 0, 0, 7] = multiSearchAllPositions(materialize('sksoirfwdhpdyxrkklhc'), ['vuixtegnp', 'ks', 'oirfwdhpd', 'sksoirf', 'skso', 'soi', 'eoxpa', 'vpfmzovgatllf', 'txsezmqvduxbmwu', 'fw']) from system.numbers limit 10; -select [2, 21, 8, 10, 6, 0, 1, 11, 0, 0, 21, 4, 29, 0] = multiSearchAllPositions(materialize('wlkublfclrvgixpbvgliylzbuuoyai'), ['l', 'ylzbuu', 'clr', 'rvgi', 'lf', 'bqtzaqjdfhvgddyaywaiybk', '', 'vgixpbv', 'ponnohwdvrq', 'dqioxovlbvobwkgeghlqxtwre', 'y', 'ublfclrvgix', 'a', 'eoxxbkaawwsdgzfweci']) from system.numbers limit 10; -select [0, 0, 2, 1, 1, 9, 1, 0, 0, 1] = multiSearchAllPositions(materialize('llpbsbgmfiadwvvsciak'), ['knyjtntotuldifbndcpxzsdwdduv', 'lfhofdxavpsiporpdyfziqzcni', 'lpbsbgmf', 'llpbsbgmfi', 'llpbsbgmfiadwvv', 'fia', '', 'uomksovcuhfmztuqwzwchmwvonk', 'ujbasmokvghmredszgwe', '']) from system.numbers limit 10; -select [3, 0, 0, 0, 6, 1, 7, 0, 2, 1, 1, 0, 7, 0, 1, 0, 1, 1, 5, 11] = multiSearchAllPositions(materialize('hnmrouevovxrzrejesigfukkmbiid'), ['m', 'apqlvipphjbui', 'wkepvtnpu', 'amjvdpudkdsddjgsmzhzovnwjrzjirdoxk', 'ue', '', 'evov', 'qoplzddxjejvbmthnplyha', 'nmrouevovxrz', '', 'hnmrouev', 'hnzevrvlmxnjmvhitgdhgd', 'evovxrzrejesig', 'yvlxrjaqdaizishkftgcuikt', '', 'buyrmbkvqukochjteumqchrhxgtmuorsdgzlfn', '', 'hnmrouevov', 'ouevovx', 'xr']) from system.numbers limit 10; -select [0, 13, 0, 0, 0, 0, 0, 14, 0, 0, 1, 12, 0, 1] = multiSearchAllPositions(materialize('uwfgpemgdjimotxuxrxxoynxoaw'), ['uzcevfdfy', 'otxuxrxxoynxoa', 'xeduvwhrogxccwhnzkiolksry', 'pxdszcyzxlrvkymhomz', 'vhsacxoaymycvcevuujpvozsqklahstmvgt', 'zydsajykft', 'vdvqynfhlhoilkhjjkcehnpmwgdtfkspk', 'txuxrx', 'slcaryelankprkeyzaucfhe', 'iocwevqwpkbrbqvddaob', 'uwfg', 'motxuxrxx', 'kpzbg', '']) from system.numbers limit 10; -select [1, 1, 0, 6, 6, 0, 0, 0, 8, 0, 8, 14, 1, 5, 6, 0, 0, 1] = multiSearchAllPositions(materialize('epudevopgooprmhqzjdvjvqm'), ['ep', 'epudevopg', 'tlyinfnhputxggivtyxgtupzs', 'vopgoop', 'v', 'hjfcoemfk', 'zjyhmybeuzxkuwaxtcut', 'txrxzndoxyzgnzepjzagc', 'pgooprmhqzj', 'wmtqcbsofbe', 'pgo', 'm', '', 'evopgooprmhqzjdv', 'vopgooprmhqzjdv', 'gmvqubpsnvrabixk', 'wjevqrrywloomnpsjbuybhkhzdeamj', '']) from system.numbers limit 10; -select [15, 4, 4, 0, 0, 1, 1, 0, 0, 0, 0, 20, 0, 10, 1, 1, 0, 2, 4, 3] = multiSearchAllPositions(materialize('uogsfbdefogwnekfoeobtkrgiceksz'), ['kfoeobtkrgice', 'sfbd', 'sfbdefogwn', 'zwtenhiqavmqoolkvjiqjfb', 'vnjkshyvpwhrauackplqllakcjyamvsuokrxbfv', 'uog', '', 'qtzuhdcdymytgtscvzlzswdlrqidreuuuqk', 'vlridmjlbxyiljpgxsctzygzyawqqysf', 'xsnkwyrmjaaaryvrdgtoshdxpvgsjjrov', 'fanchgljgwosfamgscuuriwospheze', 'btkrgicek', 'ohsclekvizgfoatxybxbjoxpsd', 'ogwnekfoeobtkr', '', '', 'vtzcobbhadfwubkcd', 'og', 's', 'gs']) from system.numbers limit 10; -select [0, 0, 5, 1, 0, 5, 1, 6, 0, 1, 9, 0, 1, 1] = multiSearchAllPositions(materialize('aoiqztelubikzmxchloa'), ['blc', 'p', 'ztelubikzmxchlo', 'aoiqztelubi', 'uckqledkyfboolq', 'ztelubikzmxch', 'a', 'telubikzm', 'powokpdraslpadpwvrqpbb', 'aoiqztelu', 'u', 'kishbitagsxnhyyswn', '', '']) from system.numbers limit 10; -select [5, 11, 0, 0, 0, 5, 0, 0, 0, 1, 16, 0, 0, 0, 0, 0] = multiSearchAllPositions(materialize('egxmimubhidowgnfziwgnlqiw'), ['imubhidowgnfzi', 'dowgnf', 'yqpcpfvnfpxetozraxbmzxxcvtzm', 'xkbaqvzlqjyjoiqourezbzwaqkfyekcfie', 'jjctusdmxr', 'imubhi', 'zawnslbfrtqohnztmnssxscymonlhkitq', 'oxcitennfpuoptwrlmc', 'ac', 'egxmi', 'fziwgn', 'rt', 'fuxfuctdmawmhxxxg', 'suulqkrsfgynruygjckrmizsksjcfwath', 'slgsq', 'zcbqjpehilwyztumebmdrsl']) from system.numbers limit 10; -select [20, 0, 9, 0, 0, 14, 0, 5, 8, 3, 0, 0, 0, 4] = multiSearchAllPositions(materialize('zczprzdcvcqzqdnhubyoblg'), ['obl', 'lzrjyezgqqoiydn', 'vc', 'nbvwfpmqlziedob', 'pnezljnnujjbyviqsdpaqkkrlogeht', 'dn', 'irvgeaq', 'rzdcvcqzqdnh', 'cvcqzqdnh', 'zprzdcv', 'wvvgoexuevmqjeqavsianoviubfixdpe', 'aeavhqipsvfkcynyrtlxwpegwqmnd', 'blckyiacwgfaoarfkptwcei', 'prz']) from system.numbers limit 10; -select [2, 1, 1, 9, 10, 5, 0, 0, 0, 2, 9, 7, 9, 0, 1, 9, 7, 0] = multiSearchAllPositions(materialize('mvovpvuhjwdzjwojcxxrbxy'), ['vo', '', '', 'jwdz', 'wdzj', 'pvu', 'ocxprubxhjnji', 'phzfbtacrg', 'jguuqhhxbrwbo', 'vovpvuhjwd', 'jw', 'u', 'jwdzjwojcx', 'nlwfvolaklizslylbvcgicbjw', '', 'jwd', 'uhjwdz', 'bbcsuvtru']) from system.numbers limit 10; -select [2, 0, 21, 0, 0, 0, 3, 0, 0, 0, 0, 10, 1, 18] = multiSearchAllPositions(materialize('nmdkwvafhcbipwoqtsrzitwxsnabwf'), ['m', 'ohlfouwyucostahqlwlbkjgmdhdyagnihtmlt', 'itwx', 'jjkyhungzqqyzxrq', 'abkqvxxpu', 'lvzgnaxzctaarxuqowcski', 'dkwvafhcb', 'xuxjexmeeqvyjmpznpdmcn', 'vklvpoaakfnhtkprnijihxdbbhbllnz', 'fpcdgmcrwmdbflnijjmljlhtkszkocnafzaubtxp', 'hmysdmmhnebmhpjrrqpjdqsgeuutsj', 'cbipwoqtsrzitwxsna', 'nm', 'srzitwx']) from system.numbers limit 10; -select [17, 5, 0, 13, 0, 0, 10, 1, 0, 19, 10, 8, 0, 4] = multiSearchAllPositions(materialize('gfvndbztroigxfujasvcdgfbh'), ['asvcdgf', 'dbztroigxfujas', 'pr', 'xfujas', 'nxwdmqsobxgm', 'wdvoepclqfhy', 'oigxfu', '', 'flgcghcfeiqvhvqiriciywbkhrxraxvneu', 'vcd', 'oigxfu', 'troigxfuj', 'gbnyvjhptuehkefhwjo', 'ndbz']) from system.numbers limit 10; -select [0, 14, 1, 0, 0, 1, 1, 11, 0, 8, 6, 0, 3, 19, 7, 0] = multiSearchAllPositions(materialize('nofwsbvvzgijgskbqjwyjmtfdogzzo'), ['kthjocfzvys', 'skbqjwyjmtfdo', 'nof', 'mfapvffuhueofutby', 'vqmkgjldhqohipgecie', 'nofwsbv', '', 'ijgs', 'telzjcbsloysamquwsoaso', 'vzgijgskbqjwyjmt', 'bvvzgijgskbqjwyjmtfd', 'hdlvuoylcmoicsejofcgnvddx', 'fwsbvvzgijgskb', 'wyjm', 'vvzgijg', 'fwzysuvkjtdiufetvlfwf']) from system.numbers limit 10; -select [10, 2, 13, 0, 0, 0, 2, 0, 9, 2, 4, 1, 1, 0, 1, 6] = multiSearchAllPositions(materialize('litdbgdtgtbkyflsvpjbqwsg'), ['tbky', 'itdbgdtgtb', 'yflsvpjb', 'ikbylslpoqxeqoqurbdehlroympy', 'hxejlgsbthvjalqjybc', 'sontq', 'itdbgd', 'ozqwgcjqmqqlkiaqppitsvjztwkh', 'gtbkyf', 'itdbgdtgtbkyfls', 'dbg', 'litdb', '', 'qesbakrnkbtfvwu', 'litd', 'g']) from system.numbers limit 10; -select [0, 0, 1, 1, 5, 0, 8, 12, 0, 2, 0, 7, 0, 6] = multiSearchAllPositions(materialize('ijzojxumpvcxwgekqimrkomvuzl'), ['xirqhjqibnirldvbfsb', 'htckarpuctrasdxoosutyxqioizsnzi', '', '', 'jxu', 'dskssv', 'mpvcxwgekqi', 'xwgek', 'qsuexmzfcxlrhkvlzwceqxfkyzogpoku', 'jzojx', 'carjpqihtpjniqz', 'umpvcxwgekq', 'krpkzzrxxtvfhdopjpqcyxfnbas', 'xumpvcxwg']) from system.numbers limit 10; -select [0, 0, 0, 6, 0, 8, 0, 2, 0, 0, 0, 0, 14, 0, 0, 1, 1, 0, 0, 0] = multiSearchAllPositions(materialize('zpplelzzxsjwktedrrtqhfmoufv'), ['jzzlntsokwlm', 'cb', 'wuxotyiegupflu', 'lzzxsjwkte', 'owbxgndpcmfuizpcduvucnntgryn', 'zxsjwktedrrtqhf', 'kystlupelnmormqmqclgjakfwnyt', 'pple', 'lishqmxa', 'mulwlrbizkmtbved', 'uchtfzizjiooetgjfydhmzbtmqsyhayd', 'hrzgjifkinwyxnazokuhicvloaygeinpd', 'tedrrt', 'shntwxsuxux', 'evrjehtdzzoxkismtfnqp', 'z', '', 'nxtybut', 'vfdchgqclhxpqpmitppysbvxepzhxv', 'wxmvmvjlrrehwylgqhpehzotgrzkgi']) from system.numbers limit 10; - -select [15, 19, 0, 0, 15, 0, 0, 1, 2, 6] = multiSearchAllPositionsUTF8(materialize('зжерхмчсйирдчрришкраоддцфгх'), ['ришкра', 'раоддц', 'фттиалусгоцжлтщзвумрдчи', 'влййи', 'ришкра', 'цгфжуцгивй', 'ккгжхрггчфглх', 'з', 'жерхмчсйи', 'мчсйирдчрришкраоддц']) from system.numbers limit 10; -select [0, 0, 0, 1, 4, 0, 14, 0, 1, 8, 8, 9, 0, 0, 4, 0] = multiSearchAllPositionsUTF8(materialize('етвхйчдобкчукхпщлмжпфайтфдоизщ'), ['амфшужперосрфщфлижйййжжжй', 'ххкбщшзлмщггтшцпсдйкдшйвхскемц', 'ергйплгпнглккшкарещимгапхг', '', 'хйчдо', 'вввбжовшзйбгуоиждепйабаххеквщижтйиухос', 'хпщл', 'жфуомщуххнедзхищнгхрквлпмзауеегз', 'етвхй', 'о', 'о', 'бк', 'цфецккифж', 'аизлокл', 'х', 'слщгеивлевбчнчбтшгфмжрфка']) from system.numbers limit 10; -select [0, 0, 1, 2, 0, 0, 14, 0, 3, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('йбемооабурнирйофшдгпснж'), ['гпфцл', 'нчбперпмцкввдчсщвзйрдфнф', '', 'бем', 'ч', 'жгш', 'йофшдгпснж', 'шасгафчг', 'емооабур', 'пиохцжццгппщчопзйлмуотз', 'рпдомнфвопхкшешйишумбацтл', 'нисиийфррбдоц']) from system.numbers limit 10; -select [1, 18, 12, 0, 0, 1, 1, 3, 7, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('гсщнфийтфзжцйпфбйалущ'), ['', 'алущ', 'цйпфбйал', 'цвбфцйвсвлицсчнргпцнр', 'х', 'гс', '', 'щн', 'й', 'дгйрвцщтп', 'уитвквоффвцхфишрлерйцувф', 'кфтййлпнзжчижвглзкижн']) from system.numbers limit 10; -select [14, 0, 5, 5, 0, 6, 0, 16, 0, 0] = multiSearchAllPositionsUTF8(materialize('ефщнйнуйебнснлрцгкеитбг'), ['лрцгкеитб', 'епклжфцпнфопе', 'йнуйебн', 'й', 'тлт', 'нуйебнснлрцгкеит', 'глечршгвотумкимтлм', 'цгк', 'щгйчой', 'звкцкчк']) from system.numbers limit 10; -select [0, 1, 18, 6, 0, 3, 0, 0, 25, 0, 0, 1, 16, 5, 1, 7, 0, 0] = multiSearchAllPositionsUTF8(materialize('пумгмцшмжштсшлачсжарерфиозиг'), ['чсуубфийемквмоотванухмбрфхжоест', '', 'жар', 'цшмжш', 'жртещтинтвпочнкдткцза', 'м', 'адзгтбаскщгдшжл', 'штфжшллезпджигщфлезфгзчайанхктицштйй', 'о', 'етадаарйсцейдошшцечхзлшлрртсрггцртспд', 'зтвшалрпфлщбцд', 'пу', 'ч', 'мцшмжштсшлачсж', '', 'шмжшт', 'ещтжшйтчзчаноемрбц', 'тевбусешйрйчшзо']) from system.numbers limit 10; -select [7, 10, 0, 0, 0, 0, 1, 12, 9, 2, 0, 0, 0, 4, 1, 1, 0, 6] = multiSearchAllPositionsUTF8(materialize('дупгвндвйжмаузнллнзл'), ['двйжмаузн', 'жмаузнлл', 'емйжркоблновцгпезрдавкбелцщста', 'щзкгм', 'лебрпцрсутшриащгайвц', 'лзнмл', 'д', 'ауз', 'йжмау', 'упгвндвйж', 'жщсббфвихг', 'всигсеигцбгаелтчкирлнзшзцжещнс', 'рмшиеиесрлщципщхкхтоцщчйоо', 'гвн', '', '', 'йадеоцлпшпвщзещзкхйрейопмажбб', 'ндв']) from system.numbers limit 10; -select [0, 0, 0, 8, 3, 10, 22, 0, 13, 11, 0, 1, 18, 0, 1, 0] = multiSearchAllPositionsUTF8(materialize('жшзфппавввслфцлнщшопкдшку'), ['саоткнхфодзаа', 'кйхванкзаисйбврщве', 'бчоуучватхфукчф', 'вввслфц', 'з', 'вслфцлнщшопк', 'дшк', 'из', 'фцл', 'с', 'зртмцтпощпщхк', 'жшзфппавввслфц', 'шопк', 'збтхрсдтатхпрзлхдооощифачхчфн', '', 'жщшийугз']) from system.numbers limit 10; -select [2, 4, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 21, 0, 6, 0] = multiSearchAllPositionsUTF8(materialize('пчботухвгдчекмжндбоожш'), ['чботухвгдчекмжндб', 'от', 'гвсжжйлбтщчучнхсмдйни', 'жцжзмшлибшефуоуомпацбщщу', 'онхфлуцйлхтбмц', 'йтепжу', 'хтдрпвкщрли', 'аддайф', 'нхегщккбфедндоацкиз', 'йгкцзртфжгв', 'буелрщмхйохгибжндфшщвшрлдччрмфмс', 'цщцтзфнщ', 'уч', 'пчб', 'жш', 'пнфббтшйгхйрочнлксщпгвжтч', 'ухвг', 'лсцппузазщрйхймщбзоршощбзленхп']) from system.numbers limit 10; -select [0, 0, 4, 11, 0, 0, 0, 0, 0, 11, 2, 4, 6, 0, 0, 1, 2, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('тжрмчпваухрхуфбгнифгбопфт'), ['дпмгкекщлнемссаицщпащтиуцхкфчихтц', 'акйиуоатунтчф', 'мчпва', 'рхуфбгнифгб', 'кнаишж', 'пчвотенеафкухжцешбцхг', 'опеа', 'ушчадфтчхечеуркбтел', 'ашшптаударчжчмвалтдхкимищпф', 'рхуфбгниф', 'ж', 'мчпваухрхуфбгнифг', 'пваухрху', 'зргачбтцдахвймсбсврбндзтнущхвп', 'асбфцавбгуолг', 'тж', 'жрмчпваухрх', 'мрвзцгоб', 'чрцснчсдхтзжвнздзфцвхеилишдбж', 'кчт']) from system.numbers limit 10; -select [0, 2, 4, 0, 6, 0, 0, 0, 0, 19, 7, 1, 0, 1, 0, 0, 2, 10, 0, 1] = multiSearchAllPositionsUTF8(materialize('опрурпгабеарушиойцрхвбнсщ'), ['йошуоесдщеж', 'пр', 'урпгабеарушиой', 'хщиаршблашфажщметчзи', 'пгабеарушиойцрхвб', 'щцбдвц', 'еечрззвкожзсдурйщувмцйшихдц', 'офхачгсзашфзозрлба', 'айдфжджшжлрргмабапткбцпиизигдтс', 'рх', 'габ', '', 'цнкдбфчщшмчулврбцчакщвзхлазфа', '', 'екбтфпфилсаванхфкмчнпумехиищди', 'епвщхаклшомвцжбф', 'прурпгабе', 'еарушиойцрхв', 'црвтгрзтитц', 'опрурпг']) from system.numbers limit 10; -select [0, 10, 1, 0, 0, 0, 0, 0, 10, 0, 15, 2] = multiSearchAllPositionsUTF8(materialize('угпщлзчжшбзвууцшатпщцр'), ['цоуарцжсз', 'бз', '', 'пщфтзрч', 'лфуипмсдмнхнгйнтк', 'айжунцйбйцасчфдхй', 'щдфщлцптплсачв', 'грв', 'бзвууц', 'бумййшдшфашцгзфвчвзвтсувнжс', 'цшатпщ', 'гпщлзчжшб']) from system.numbers limit 10; -select [0, 15, 0, 1, 5, 0, 0, 5, 0, 0, 0, 1, 0, 0] = multiSearchAllPositionsUTF8(materialize('цнлеодлмдцдйснитвдчтхжизв'), ['ивкчсзшугоцжчохщцабл', 'итвдчт', 'кнх', '', 'одлм', 'ктшфзбщзцуймагсоукщщудвуфо', 'ххеаефудгчхр', 'одлмдцдйснитвдчт', 'умцлпкв', 'зщсокйтцзачщафвбповжгнлавсгйг', 'бкибм', '', 'охсоихнцчцшевчеележтука', 'фаийхгжнсгищгщц']) from system.numbers limit 10; -select [0, 0, 0, 2, 0, 0, 0, 0, 3, 2, 3, 6, 0, 0, 0, 12, 4, 1] = multiSearchAllPositionsUTF8(materialize('бгдбувдужщвоошлтчрбй'), ['щвбаиф', 'итчднесжкчжвпжйвл', 'мм', 'г', 'хктзгтзазфгщшфгбеулцмдмдбдпчзх', 'сфуак', 'злйфцощегзекщб', 'фшлдтолрщфзжчмих', 'дбувдужщ', 'гдб', 'дбувдужщ', 'в', 'лчищкечнжщисцичбнзшмулпмлп', 'чжцсгмгфвлиецахзнрбмщин', 'обпжвй', 'о', 'бувдужщвоош', '']) from system.numbers limit 10; -select [0, 2, 5, 3, 2, 0, 1, 0, 0, 4, 2, 0, 0, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('шсушлорзфжзудбсейенм'), ['чнзпбновтршеумбвщчлх', 'су', 'лорзфж', 'ушлорзфжзудб', 'сушлорзфжзудбсейенм', 'ткдрхфнб', '', 'пщд', 'чбдцмщ', 'шлорзфж', 'су', 'сккигркедчожжемгнайвйчтдмхлтти', 'мц', 'пхнхрхйцйсйбхчлктз', 'иафжстлйфцр', 'алщщлангнбнйхлшлфшйонщек']) from system.numbers limit 10; -select [12, 1, 0, 5, 0, 10, 1, 0, 7, 4, 0, 1, 12, 1, 1, 1, 0, 1, 15, 0] = multiSearchAllPositionsUTF8(materialize('ощзллчубоггцвжриуардрулащйпу'), ['цвжр', '', 'нмзкаиудзтиффззшзканжвулт', 'лчубоггцвжриуардрулащйпу', 'чтцлзшуижолибаоххвшихбфжйхетивп', 'ггцвжри', '', 'йдгнвс', 'у', 'л', 'зпщнжуойдлдвхокцжнзйсйзе', '', 'цв', '', '', '', 'ехлцзгвф', '', 'риу', 'уйжгтжноомонгщ']) from system.numbers limit 10; -select [0, 12, 13, 20, 0, 1, 0, 0, 3, 4] = multiSearchAllPositionsUTF8(materialize('цбкифйтшузажопнжщарбштвдерзтдш'), ['щлмлижтншчсмксгтнсврро', 'жопнжщарбштвд', 'опнжщарб', 'бштвдерзтд', 'пуфслейщбкжмпнш', 'ц', 'маве', 'кмйхойрдлшцхишдтищвйбцкщуигваещгтнхйц', 'кифй', 'и']) from system.numbers limit 10; -select [0, 6, 0, 0, 0, 8, 0, 3, 6, 0] = multiSearchAllPositionsUTF8(materialize('еачачгбмомоххкгнвштггпчудл'), ['ндзчфчвжтцщпхщуккбеф', 'г', 'рткнфвчтфннхлжфцкгштймгмейжй', 'йчннбщфкщф', 'лсртщиндшшкичзррущвдйвнаркмешерв', 'момоххк', 'рфафчмсизлрхзуа', 'ч', 'гбмомоххкгнвштг', 'валжпошзбгзлвевчнтз']) from system.numbers limit 10; -select [0, 0, 10, 0, 8, 13, 0, 0, 19, 15, 3, 1] = multiSearchAllPositionsUTF8(materialize('зокимчгхухшкшмтшцчффвззкалпва'), ['цалфжажщщширнрвтпвмщжннрагвойм', 'оукзрдцсадешжмз', 'хшкшмтшцч', 'ауилтсаомуркпаркбцркугм', 'хухшкшмтшцчффв', 'шмтшцч', 'зщгшпцхзгцншднпеусмтжбцшч', 'щлраащсйлщрд', 'ффвзз', 'тшцчффвззкалпв', 'кимчгхухшкш', '']) from system.numbers limit 10; -select [0, 0, 1, 0, 6, 0, 6, 0, 5, 0, 13, 0, 0, 6] = multiSearchAllPositionsUTF8(materialize('йдйндиибщекгтчбфйдредпхв'), ['тдршвтцихцичощнцницшдхйбогбчубие', 'акппакуцйсхцдххнотлгирввоу', '', 'улщвзхохблтксчтб', 'и', 'ибейзчшклепзриж', 'иибщекгт', 'шидббеухчпшусцнрз', 'диибщекгтчбфйд', 'дейуонечзйзлдкшщрцйбйклччсцуй', 'тч', 'лшицлшме', 'чйнжчоейасмрщегтхвйвеевбма', 'ии']) from system.numbers limit 10; -select [15, 3, 3, 2, 0, 11, 0, 0, 0, 2, 0, 4, 0, 1, 1, 3, 0, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('нхгбфчшджсвхлкхфвтдтлж'), ['хфвтдтлж', 'гбфчшд', 'гбфчш', 'х', 'ачдгбккжра', 'вхлк', 'мщчвещлвшдщпдиимлшрвнщнфсзгщм', 'жчоббгшзщлгеепщжкчецумегпйчт', 'жжд', 'хг', 'мтсааролшгмоуйфйгщгтрв', 'бфчшд', 'чейрбтофпшишгуасоодлакчдф', 'н', 'нхгбфч', 'гбф', 'гдежсх', 'йифжацзгжбклх', 'ещпзущпбаолплвевфиаибшйубйцсзгт', 'жезгчжатзтучжб']) from system.numbers limit 10; -select [0, 10, 1, 0, 0, 0, 4, 0, 13, 1, 12, 1, 0, 6] = multiSearchAllPositionsUTF8(materialize('акбдестрдшерунпвойзв'), ['нркчх', 'шерунп', '', 'зжвахслфббтоиоцрзаззасгнфчх', 'шлжмдг', 'тлйайвцжчсфтцйрчосмижт', 'дестрдшерунп', 'мвамйшцбдщпчлрщд', 'у', 'акбдестрд', 'рунпвойз', '', 'айздцоилсйшцфнчтхбн', 'с']) from system.numbers limit 10; -select [1, 0, 0, 3, 2, 1, 0, 0, 1, 10, 7, 0, 5, 0, 8, 4, 1, 0, 8, 1] = multiSearchAllPositionsUTF8(materialize('кйхпукаеуддтйччхлнпсуклрф'), ['кйхпукаеуддтйччхл', 'йатлрйкстлхфхз', 'фгихслшкж', 'хпу', 'йхпукаеу', '', 'сруакбфоа', 'оажуз', 'кйхпукаеуддтйччх', 'ддтйччхлн', 'аеуддтйччхл', 'тмажиойщтпуцглхфишеиф', 'укаеуддтйччхлнпс', 'ретифе', 'еуддтйччхлнпсуклр', 'пукаеуд', 'кйхпу', 'таппфггвджлцпжшпишбпциуохсцх', 'еуд', '']) from system.numbers limit 10; -select [2, 3, 3, 16, 5, 13, 0, 0, 0, 18, 0, 6, 0, 16, 0, 10, 3, 0] = multiSearchAllPositionsUTF8(materialize('плврйщовкзнбзлбжнсатрцщщучтйач'), ['лврйщовкзнбзлбж', 'врйщовкзнбзлбжнса', 'врйщовкзнбз', 'жнсатрцщщучтйач', 'йщовкзнбзлбжнсатрцщщуч', 'злбжнсатрцщ', 'ввтбрдт', 'нжйапойг', 'ннцппгперхйвдхоеожупйебочуежбвб', 'сатрцщщу', 'деваийтна', 'щ', 'вкжйгкужжгтевлцм', 'жнс', 'датг', 'знбзлбжнсатрцщщучтйа', 'врйщовк', 'оашмкгчдзщефм']) from system.numbers limit 10; -select [3, 1, 19, 1, 0, 0, 0, 0, 11, 3, 0, 0] = multiSearchAllPositionsUTF8(materialize('фчдеахвщжхутхрккхасвсхепщ'), ['деах', '', 'свсхепщ', '', 'анчнсржйоарвтщмрж', 'нечбтшщвркгд', 'вштчцгшж', 'з', 'у', 'деахвщ', 'ххкцжрвзкжзжчугнфцшуиаклтмц', 'фцкжшо']) from system.numbers limit 10; -select [16, 0, 0, 1, 8, 14, 0, 12, 12, 5, 0, 0, 16, 0, 11, 0] = multiSearchAllPositionsUTF8(materialize('щмнжчввбжцчммчшсрхйшбктш'), ['срхйшбк', 'йлзцнржчууочвселцхоучмщфчмнфос', 'еижлафатшхщгшейххжтубзвшпгзмзцод', '', 'бжцчммчшсрхй', 'чшсрхй', 'влемчммйтителщвзган', 'ммч', 'ммчшсрх', 'чввбж', 'нобзжучшошмбщешлхжфгдхлпнгпопип', 'цгт', 'срхйш', 'лкклмйжтеа', 'чммчшсрхйшбктш', 'йежффзнфтнжхфедгбоахпг']) from system.numbers limit 10; -select [1, 12, 9, 5, 1, 0, 6, 3, 0, 1] = multiSearchAllPositionsUTF8(materialize('кжнщсашдзитдмщцхуоебтфжл'), ['', 'дмщцхуоебт', 'зитдмщцхуоебт', 'сашдзитдмщцхуое', 'кжнщ', 'тхкйтшебчигбтмглшеужззоббдилмдм', 'ашдзитдмщцхуоебтф', 'нщсашдз', 'аузщшр', 'кжнщсашдз']) from system.numbers limit 10; -select [2, 0, 0, 0, 1, 0, 2, 0, 0, 17, 0, 8, 7, 14, 0, 0, 0, 7, 9, 23] = multiSearchAllPositionsUTF8(materialize('закуфгхчтшивзчжаппбжнтслщввущ'), ['а', 'днойвхфрммтж', 'внтлжрхзрпчбтуркшдатннглечг', 'ахиеушжтфкгцщтзхмжнрхдшт', '', 'тцчгрззржмдшйщфдцрбшжеичч', 'а', 'ктиечцпршнфнбчуолипацчдсосцнлфаццм', 'аусрлхдцегферуо', 'ппбжнт', 'жкццуосгвп', 'чтшивзчжаппб', 'хчтшивзчжаппб', 'чжаппбжнтслщ', 'ччрлфдмлу', 'щзршффбфчзо', 'ущуймшддннрхзийлваежщухч', 'хчтши', 'тшивзчжаппбжнтсл', 'слщв']) from system.numbers limit 10; -select [1, 1, 9, 2, 0, 3, 7, 0, 0, 19, 2, 2, 0, 8] = multiSearchAllPositionsUTF8(materialize('мвкзккупнокченйнзкшбдрай'), ['м', '', 'н', 'вкз', 'гдпертшйбртотунур', 'к', 'упнокченйнзкшбдр', 'нфшрг', 'нмждрйббдцлйемжпулдвкещхтжч', 'ш', 'вкзккупнокченйнзкшбдр', 'вкзккупнокченйнзкшбдрай', 'адииксвеавогтйторчтцвемвойшпгбнз', 'пнокченй']) from system.numbers limit 10; -select [15, 0, 0, 1, 12, 1, 0, 0, 1, 11, 0, 4, 0, 2] = multiSearchAllPositionsUTF8(materialize('отарлшпсабждфалпшножид'), ['лпшно', 'вт', 'лпжшосндутхорлиифжаакш', 'отарлшпсабждфалпшнож', 'дфал', '', 'бкцжучншжбгзжхщпзхирртнбийбтж', 'уцвцкшдзревпршурбсвйнемоетчс', '', 'ждфал', 'тлскхрнпмойчбцпфущфгф', 'рлшпсабж', 'нхнмк', 'тарлшпса']) from system.numbers limit 10; -select [0, 2, 0, 20, 0, 17, 18, 0, 1, 1, 21, 1, 0, 1, 6, 26] = multiSearchAllPositionsUTF8(materialize('ачйвцштвобижнзжнчбппйеабтцнйн'), ['сзхшзпетншйисщкшрвйшжуогцвбл', 'чйвцштво', 'евз', 'пй', 'хуждапрахитйажрищуллйзвчт', 'чбппйе', 'бппйеабтцнйн', 'схш', 'а', 'ачйвцштвобижнзжнчбпп', 'йеабтцнй', '', 'ег', '', 'штвобижнзжнчбпп', 'цн']) from system.numbers limit 10; -select [1, 0, 0, 3, 4, 12, 0, 9, 0, 12, 0, 0, 8, 0, 10, 3, 4, 1, 1, 9] = multiSearchAllPositionsUTF8(materialize('жмхоужежйуфцзеусеоднчкечфмемба'), ['', 'идосйксзнщйервосогф', 'тхмсйлвкул', 'хоужежйуф', 'оужежйуфцзеусеоднчкечфм', 'цзеусеоднчкеч', 'бецвдиубххвхйкажуурщщшщфбзххт', 'йуфцзеусеодн', 'мглкфтуеайсржисстнпкгебфцпа', 'цзеусео', 'уехцфучецчгшйиржтсмгхакчшввохочжпухс', 'дчвмсбткзталшбу', 'жйуфцзеусеоднчке', 'ччшщтдбпвчд', 'уфцзеусеоднчкечфмем', 'хоужежйуфцзеусеоднчкечф', 'оуже', '', 'жмхоужежйуфцзеу', 'й']) from system.numbers limit 10; -select [0, 0, 0, 3, 0, 0, 0, 0, 1, 0, 1, 0, 1, 2, 0, 0, 0, 6] = multiSearchAllPositionsUTF8(materialize('лшпцхкмтресзпзйвцфрз'), ['енрнцепацлщлблкццжсч', 'ецжужлуфаееоггрчохпчн', 'зхзнгасхебнаейбддсфб', 'пцхкмтресзпзйв', 'фчетгеодщтавиииухцундпнхлчте', 'шшгсдошкфлгдвкурбуохзчзучбжйк', 'мцщщцп', 'рх', '', 'зйошвщцгхбж', '', 'ввлпнамуцвлпзеух', '', 'шпцхкмтре', 'маабтруздрфйпзшлсжшгож', 'фдчптишмштссщшдшгх', 'оллохфпкаем', 'кмтресзпз']) from system.numbers limit 10; -select [2, 5, 0, 0, 6, 0, 0, 0, 0, 0, 0, 0, 1, 1, 12, 0, 0, 0, 4, 8] = multiSearchAllPositionsUTF8(materialize('есипзсвшемлхчзмйрсфз'), ['с', 'з', 'пщчсмаиахппферзжбпвиибаачй', 'гтщкзоиежав', 'свшемлхчзм', 'шийанбке', 'зхе', 'авркудфаусзквкфффйцпзжщввенттб', 'ножцваушапиж', 'иизкежлщиафицкчщмалнпсащсднкс', 'вчмв', 'кщеурмуужжлшррце', '', '', 'х', 'алзебзпчеложихашжвхмйхрицн', 'тпзмумчшдпицпдшиаог', 'сулксфчоштаййзбзшкджббщшсей', 'пзсвшемлхчзм', 'ш']) from system.numbers limit 10; -select [0, 1, 2, 4, 0, 0, 14, 1, 13, 4, 0, 0, 1, 1] = multiSearchAllPositionsUTF8(materialize('сзиимонзффичвфжоеулсадону'), ['зфтшебтршхддмеесчд', '', 'зиимонзф', 'имон', 'езбдйшжичценлгршщшаумайаицй', 'птпщемтбмднацлг', 'фжоеулса', '', 'вфжоеулсадону', 'имонзфф', 'йщвдфдиркважгйджгжашарчучйххйднпт', 'дй', '', '']) from system.numbers limit 10; -select [12, 0, 24, 0, 9, 0, 1, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('ижсщщрзжфнгццпзкфбвезгбохлж'), ['ццпзкфбвез', 'ацррвхоптаоснулнжкщжел', 'охлж', 'тнсхбпшщнб', 'фнг', 'урйвг', '', 'цохс', 'щбйрйкжчмйзачуефч', 'афа']) from system.numbers limit 10; -select [9, 0, 0, 0, 1, 0, 7, 7, 0, 0, 1, 0, 7, 0, 0, 8, 0, 3, 0, 0] = multiSearchAllPositionsUTF8(materialize('рерфвирачйнашхрмцебфдйааеммд'), ['чйнашхрмцебфдйааеммд', 'сжщзснвкущлжплцзлизаомдизцнжлмййбохрцч', 'еппбжджмримфчйеаолидпцруоовх', 'едтжкоийггснехшсчйлвфбкцжжрчтш', '', 'пжахфднхсотй', 'ра', 'рач', 'вчримуцнхбкуйжрвфиугзфсзг', 'кщфехрххциаашщсифвашгйцвхевцщнйахтбпжщ', '', 'ртщиобчжстовйчфабалзц', 'рачйнашхрмцебфдйаае', 'ощгжосччфкуг', 'гехвжнщжссидмрфчйтнепдсртбажм', 'а', 'ицжлсрсиатевбвнжрдмзцувввтзцфтвгвш', 'рф', 'прсмлча', 'ндлхшцааурмзфгверуфниац']) from system.numbers limit 10; -select [2, 14, 10, 0, 6, 15, 1, 0, 0, 4, 5, 17, 0, 0, 3, 0, 3, 0, 9, 0] = multiSearchAllPositionsUTF8(materialize('влфощсшкщумчллфшшвбшинфппкчуи'), ['лфощ', 'лфшшвбшинфпп', 'умчллфшшвбшинф', 'слмтнг', 'сшкщумчллфшшвбшинф', 'фшшвб', '', 'рчфбчййсффнодцтнтнбцмолф', 'щфнщокхжккшкудлцжрлжкнп', 'ощ', 'щсшкщумчлл', 'швбшинфппкч', 'септзкщотишсехийлоцчапщжшжсфмщхсацг', 'нт', 'фощсшкщумчллфшшвбшинфп', 'нщпдш', 'фощс', 'мивсмча', 'щумч', 'щчйнткжпмгавфтйтибпхх']) from system.numbers limit 10; -select [0, 10, 0, 0, 0, 0, 0, 3, 0, 0, 0, 2, 0, 11, 0, 0] = multiSearchAllPositionsUTF8(materialize('еаиалмзхцгфунфеагшчцд'), ['йнш', 'гфун', 'жлйудмхнсвфхсуедспщбтутс', 'елмуийгдйучшфлтхцппамфклйг', 'евйдецц', 'пчтфцоучфбсйщпвдацмчриуцжлтжк', 'нстмпумчспцвцмахб', 'иалмз', 'зифчп', 'чогфщимоопт', 'фдйблзеп', 'аиа', 'щугмзужзлйдктш', 'фунфеагшч', 'нйхшмсгцфжчхжвхгдхцуппдц', 'асмвмтнрейшгардллмсрзгзфйи']) from system.numbers limit 10; -select [23, 0, 8, 0, 0, 0, 0, 0, 0, 4, 0, 5, 7, 1, 9, 4] = multiSearchAllPositionsUTF8(materialize('зузйфзлхходфрхгтбпржшрктпйхеоп'), ['ктпйхео', 'лжитуддикчсмкглдфнзцроцбзтсугпвмхзллжж', 'х', 'меуфтено', 'фтдшбшрпоцедктсийка', 'кхтоомтбчвеонксабшйптаихжбтирпзшймчемжим', 'чиаущлрдкухцрдумсвивпафгмр', 'фрнпродв', 'тдгтишхйсашвмдгкчбмшн', 'йфзлхходфрхгтбпржшр', 'бежшлрйврзмумеуооплкицхлйажвцчнчсеакм', 'ф', 'лхходфрхгтб', '', 'ходфрхгтбпржшр', 'й']) from system.numbers limit 10; -select [0, 0, 0, 1, 0, 1, 22, 1, 0, 0, 0, 0, 18, 1, 0, 0, 0, 1] = multiSearchAllPositionsUTF8(materialize('чфгвчхчпщазтгмбнплдгщикойчднж'), ['мштцгтмблаезочкхзвхгрбпкбмзмтбе', 'канбжгсшхшз', 'кзинвщйччажацзйнсанкнщ', 'чфгвчхчпщазтгмбнп', 'етйцгтбнщзнржнйхж', '', 'ик', '', 'еизщвпрохдгхир', 'псумйгшфбвгщдмхжтц', 'слмжопинйхнштх', 'йшралцицммбщлквмгхцввизопнт', 'л', 'чфгвчхчпщазтгмбнплдгщ', 'пбзмхжнпгикиищжтшботкцеолчцгхпбвхи', 'хзкцгрмшгхпхуоцгоудойнжлсоййосссмрткцес', 'ажуофйпщратдйцбржжлжнжащцикжиа', '']) from system.numbers limit 10; -select [6, 0, 2, 5, 2, 9, 10, 0, 0, 4, 0, 6, 3, 2] = multiSearchAllPositionsUTF8(materialize('ишогпсисжашфшлйичлба'), ['сисжашфшлй', 'пднещбгзпмшепкфосовбеге', 'шогп', 'пс', 'шогпси', 'жаш', 'аш', 'деисмжатуклдшфлщчубфс', 'грмквкщзур', 'гпсис', 'кйпкбцмисчхдмшбу', 'сисжашф', 'о', 'шо']) from system.numbers limit 10; -select [8, 15, 13, 0, 1, 2, 5, 2, 9, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('нсчщчвсанпрлисблснокзагансхм'), ['анпрлисблснокзагансхм', 'блснокз', 'исб', 'дрмгвснпл', '', 'счщчвса', 'чвсанпрлисблснокзагансх', 'счщчвсанпрлис', 'нпрли', 'пциишуецнймуодасмжсойглретиефо', 'фхимщвкехшлг', 'слщмаимшжчфхзпрцмхшуниврлуйлжмфжц']) from system.numbers limit 10; -select [0, 5, 0, 0, 14, 0, 12, 0, 2, 3, 0, 3, 21, 5] = multiSearchAllPositionsUTF8(materialize('хажуижанндвблищдтлорпзчфзк'), ['щуфхл', 'и', 'фцежлакчннуувпаму', 'щесщжрчиктфсмтжнхекзфс', 'ищдтлорпзчф', 'дееичч', 'блищ', 'гиефгйзбдвишхбкбнфпкддмбтзиутч', 'ажуижа', 'жуижанндвблищдтлорпзчфзк', 'чщщдзетвщтччмудвзчгг', 'ж', 'пзчфз', 'ижанн']) from system.numbers limit 10; -select [0, 0, 0, 9, 15, 0, 0, 0, 1, 3, 0, 0, 1, 0, 10, 0, 4, 0, 0, 7] = multiSearchAllPositionsUTF8(materialize('россроапцмцагвиигнозхзчотус'), ['ошажбчвхсншсвйршсашкм', 'пфдчпдчдмауцгкйдажрйефапввшжлшгд', 'иеаочутввжмемчушлуч', 'цмцагвиигно', 'ииг', 'ммпжщожфйкакбущчирзоммагеиучнщмтвгихк', 'укррхбпезбжууеипрзжсло', 'ншопзжфзббилйбувгпшшиохврнфчч', '', 'ссроап', 'лийщфшдн', 'йчкбцциснгначдцйчпа', 'россроапцмцагвииг', 'кштндцтсшорввжсфщчмщчжфжквзралнивчзт', 'мца', 'нбтзетфтздцао', 'сроа', 'мщсфие', 'дткодбошенищйтрподублжскенлдик', 'апцмцагвиигноз']) from system.numbers limit 10; -select [16, 0, 0, 2, 1, 1, 0, 1, 9, 0, 0, 3] = multiSearchAllPositionsUTF8(materialize('тйсдйилфзчфплсджбарйиолцус'), ['жбарйиолцу', 'цназщжждефлбрджктеглщпунйжддгпммк', 'хгжоашцшсзкеазуцесудифчнощр', 'йс', '', 'тйсдйилфзчфп', 'ивфсплшвслфмлтххжчсстзл', '', 'зчфплсдж', 'йртопзлодбехрфижчдцйс', 'цлащцкенмшеоерееиуноп', 'с']) from system.numbers limit 10; -select [3, 2, 1, 1, 0, 0, 0, 14, 6, 0] = multiSearchAllPositionsUTF8(materialize('нсцннйрмщфбшщховвццбдеишиохл'), ['цннйр', 'сцннйрм', 'н', 'нс', 'двтфхйзгеиеиауимбчхмщрцутф', 'пчтмшйцзсфщзшгнхщсутфжтлпаввфгххв', 'лшмусе', 'ховвццбд', 'йрмщфбшщховвццбдеи', 'гндруущрфзсфжикшзцжбил']) from system.numbers limit 10; -select [0, 18, 0, 1, 2, 0, 0, 0, 1, 7, 10, 0, 1, 0, 2, 0, 0, 18] = multiSearchAllPositionsUTF8(materialize('щидмфрсготсгхбомлмущлаф'), ['тлтфхпмфдлуоцгчскусфжчкфцхдухм', 'мущла', 'емлвзузхгндгафги', '', 'идмфрсготсгхбомлмущла', 'зфаргзлщолисцфдщсеайапибд', 'кдхоорхзжтсйимкггйлжни', 'лчгупсзждплаблаеклсвчвгвдмхклщк', 'щидмфр', 'сготсгхбомлму', 'тсгхбомлмущла', 'хсзафйлкчлди', '', 'й', 'ид', 'щлйпмздйхфзайсщсасейлфцгхфк', 'шдщчбшжбмййзеормнрноейй', 'мущ']) from system.numbers limit 10; -select [0, 13, 0, 0, 1, 0, 7, 7, 8, 0, 2, 0, 3, 0, 0, 13] = multiSearchAllPositionsUTF8(materialize('трцмлщввадлжввзчфипп'), ['хшзйийфжмдпуигсбтглй', 'ввзчфи', 'нсцчцгзегммтсшбатщзузпкшрг', 'гувйддежзфилйтш', '', 'хгзечиа', 'ввадлжввз', 'ввадлжввзчфи', 'ва', 'щтшсамклегш', 'рцмлщ', 'учзмиерфбтцучйдглбщсз', 'цмлщввадлжввзчфи', 'орйжччцнаррбоабцжзйлл', 'квпжматпцсхзузхвмйч', 'ввзчфип']) from system.numbers limit 10; -select [0, 1, 1, 0, 11, 4, 1, 2, 0, 0] = multiSearchAllPositionsUTF8(materialize('инкщблбвнскцдндбмсщщш'), ['жхрбсусахрфкафоилмецчебржкписуз', 'инкщблбвнс', '', 'зисгжфлашймлджинаоджруй', 'кцднд', 'щблбвнскцдндбмсщщ', 'инкщблбвнс', 'н', 'зб', 'фчпупшйфшбдфенгитатхч']) from system.numbers limit 10; -select [6, 0, 4, 20, 1, 0, 5, 0, 1, 0] = multiSearchAllPositionsUTF8(materialize('рзтецуйхлоорйхдбжашнларнцт'), ['у', 'бпгййекцчглпдвсцсещщкакцзтцбччввл', 'ецуйхлоо', 'нлар', 'рз', 'ккнжзшекфирфгсгбрнвжчл', 'цуйхлоорйхдбжашн', 'йнучгрчдлйвводт', 'рзте', 'нткрввтубчлщк']) from system.numbers limit 10; - -select [1, 1, 0, 0, 1, 0, 0, 3, 3, 3, 1, 0, 8, 0, 8, 1, 0, 1] = multiSearchAllPositionsCaseInsensitive(materialize('OTMMDcziXMLglehgkklbcGeAZkkdh'), ['', 'OTmmDCZiX', 'SfwUmhcGTvdYgxlzsBJpikOxVrg', 'ngqLQNIkqwguAHyqA', '', 'VVZPhzGizPnKJAkRPbosoNGJTeO', 'YHpLYTVkHnhTxMODfABor', 'mMdcZi', 'MmdCZI', 'MMdCZixmlg', '', 'hgaQHHHkIQRpPjv', 'ixMLgLeHgkkL', 'uKozJxZBorYWjrx', 'i', '', 'WSOYdEKatHkWiCtlwsCbKRnXuKcLggbkBxoq', '']) from system.numbers limit 10; -select [4, 15, 0, 0, 0, 0, 5, 0, 5, 1, 0, 1, 13, 0, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('VcrBhHvWSFXnSEdYCYpU'), ['bhhVwSfXnSEd', 'DycyP', 'kEbKocUxLxmIAFQDiUNoAmJd', 'bsOjljbyCEcedqL', 'uJZxIXwICFBPDlUPRyDHMmTxv', 'BCIPfyArrdtv', 'hHv', 'eEMkLteHsuwsxkJKG', 'hHVWsFxNseDy', '', 'HsFlleAQfyVVCoOSLQqTNTaA', '', 'sEDY', 'UMCKQJY', 'j', 'rBhHvw']) from system.numbers limit 10; -select [1, 1, 0, 0, 1, 0, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('wZyCLyiWnNNdNAPWeGSQZcdqk'), ['w', '', 'vlgiXgFTplwqRbnwBumAjHvQuM', 'QoIRVKDHMlapLNiIZXvwYxluUivjY', 'WZY', 'gAFpUfPDAwgzARCIMrtbZUsNcR', 'egkLWqqdNiETeETsMG', 'dzSlJaoHKlQmENIboow', 'vPNBhcaIfsgLH', 'mlWPTCBDVTdKHxlvIUVcJXBrmTcJokAls']) from system.numbers limit 10; -select [0, 10, 0, 1, 7, 1, 6, 1, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('pqliUxqpRcOOKMjtrZSEsdW'), ['YhskuppNFdWaTaZo', 'Coo', 'mTEADzHXPeSMCQaYbKpikXBqcfIGKs', 'PQLiUxq', 'qpRCoOK', 'PQLIu', 'XQPrcoOK', '', 'pR', 'cTmgRtcSdRIklNQVcGZthwfarLtAYh']) from system.numbers limit 10; -select [16, 1, 1, 1, 1, 4, 17, 0, 0, 0, 1, 0, 0, 0, 20, 0] = multiSearchAllPositionsCaseInsensitive(materialize('kJyseeDFCeUWoqMfubYqJqWA'), ['fub', 'kJY', '', '', 'Kj', 's', 'uBYQJq', 'sUqCmHUZIBtZPswObXSrYCwrdxdznM', 'mtZDCJENYuikJnCcJfRcSCDYDPXU', 'IDXjRjHhmjqXmCOlQ', '', 'jiEwAxIsJDu', 'YXqcEKbHxlgUliIALorSKDMlGGWeCO', 'OstKrLpYuASEUrIlIuHIRdwLr', 'qJq', 'tnmvMTFvjsW']) from system.numbers limit 10; -select [11, 3, 1, 0, 9, 0, 0, 0, 0, 8, 3, 0] = multiSearchAllPositionsCaseInsensitive(materialize('EBSPtFpDaCIydASuyreS'), ['iyD', 'sptfpdAciyDAsuyR', 'EbS', 'IJlqfAcPMTUsTFXkvmtsma', 'AcIYda', 'fbWuKoCaCpRMddUr', 'srlRzZKeOQGGLtTLOwylLNpVM', 'ZeIgfTFxUyNwDkbnpeiPxQumD', 'j', 'daciydA', 'sp', 'dyGFtyfnngIIbcCRQzphoqIgIMt']) from system.numbers limit 10; -select [6, 0, 0, 0, 10, 0, 1, 4, 0, 15, 0, 2, 2, 6] = multiSearchAllPositionsCaseInsensitive(materialize('QvlLEEsgpydemRZAZcYbqPZHx'), ['eSgpYDEMRzAzcyBQPzH', 'NUabuIKDlDxoPXoZOKbUMdioqwQjQAiArv', 'pRFrIAGTrggEOBBxFmnZKRPtsUHEMUEg', 'CDvyjef', 'YdEMrzaZc', 'BO', '', 'leEsgPyDEmRzaZCYBqPz', 'EzcTkEbqVXaVKXNuoxqNWHM', 'Z', 'cuuHNcHCcLGb', 'V', 'vllEes', 'eS']) from system.numbers limit 10; -select [0, 0, 0, 0, 0, 3, 0, 0, 0, 0, 5, 7, 5, 0, 11, 1] = multiSearchAllPositionsCaseInsensitive(materialize('eiCZvPdGJSmwxMIrZvEzfYFOFJmV'), ['lSydrmJDeXDYHGFFiFOOJGyCbCCDbLzbSbub', 'ewsAVflvcTBQFtvWBwuZOJKkrUArIg', 'fpEkBWaBkRWypFWtMz', 'YatSURyNtcSuerWWlTBSdBNClO', 'YO', 'CZvpdg', 'uoH', 'gtGwQSVqSJDVROmsBIxjuVNfrQnxDhWGXLBH', 'IKNs', 'HElLuRMlsRgINaNp', 'V', 'DGjsMW', 'vPDgJSmW', 'SGCwNiAmNfHSwLGZkRYEqrxBTaDRAWcyHZYzn', 'mWXMiRZvezfYf', '']) from system.numbers limit 10; -select [23, 1, 0, 17, 0, 0, 9, 3, 0, 2] = multiSearchAllPositionsCaseInsensitive(materialize('BizUwoENfLxIIYVDflhOaxyPJw'), ['yPJ', '', 'gExRSJWtZwOptFTkNlBGuxyQrAu', 'FLH', 'hCqo', 'oVGcArersxMUCNewhTMmjpyZYAIU', 'FlXIiYVdflHoAX', 'ZuWOe', 'bhfAfNdgEAtGdHylxkjgvU', 'IZUWo']) from system.numbers limit 10; -select [0, 9, 0, 0, 0, 0, 1, 0, 0, 1, 3, 0, 13, 0, 3, 5] = multiSearchAllPositionsCaseInsensitive(materialize('loKxfFSIAjbRcguvSnCdTdyk'), ['UWLIDIermdFaQVqEsdpPpAJ', 'ajBrcg', 'xmDmuYoRpGu', 'wlNjlKhVzpC', 'MxIjTspHAQCDbGrIdepFmLHgQzfO', 'FybQUvFFJwMxpVQRrsKSNHfKyyf', '', 'vBWzlOChNgEf', 'DiCssjczvdDYZVXdCfdSDrWaxmgpPXDiD', '', 'kxFFSIAjBRCGUVSNcD', 'LrPRUqeehMZapsyNJdu', 'cGuVSNcdTdy', 'NmZpHGkBIHVSoOcj', 'KxffSIAjBr', 'ffsIaJB']) from system.numbers limit 10; -select [14, 0, 11, 0, 10, 0, 0, 0, 13, 1, 2, 11, 5, 0] = multiSearchAllPositionsCaseInsensitive(materialize('uijOrdZfWXamCseueEbq'), ['sE', 'VV', 'AmcsEu', 'fUNjxmUKgnDLHbbezdTOzyLaknQ', 'XAmCsE', 'HqprIpxIcOTkDIKcVK', 'NbmirQlNsTHnAVKlF', 'VVDNOxFKSnQGKPsTqgtwLhZnIPkL', 'c', '', 'IJ', 'aM', 'rDzF', 'YFwP']) from system.numbers limit 10; -select [0, 8, 17, 0, 1, 0, 0, 0, 0, 0, 5, 0] = multiSearchAllPositionsCaseInsensitive(materialize('PzIxktujxHZsaDlwSGQPgvA'), ['zrYlZdnUxlPrVJJeZEASwdCHlNEm', 'jxhZS', 'sGQPgV', 'MZMChmRBgsxhdgspUhALoxmrkZVp', 'pzIxktuJxHzsADlw', 'xavwOAibQuoKg', 'vuuETOrWLBNLhrMeWLgGQpeFPdcWmWu', 'TZrAgmdorqZIdudhyCMypHYKFO', 'ztcCyGxRKrcUTv', 'OUvwdMZrcZuwGtjuEBeGU', 'k', 'rFTpnfGIOCfwktWnyOMeXQZelkYwqZ']) from system.numbers limit 10; -select [3, 1, 4, 1, 0, 17, 13, 0, 0, 0, 0, 0, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('pUOaQLUvgmqvxaMsfJpud'), ['OaqLUvGm', '', 'aQ', '', 'VajqJSlkmQTOYcedjiwZwqNH', 'f', 'xaMsfj', 'CirvGMezpiIoacBGAGQhTJyr', 'vucKngiFjTlzltKHexFVFuUlVbey', 'ppalHtIYycBCEjsgsXbFeecpkQMNr', 'nEgIYVoGkhTsFgBUSHJvIcYCYbuOBP', 'efjBVRVzknGrikGHxExlFEtYf', 'v', 'QgRBCaGlwNYWRslDylOrfPxZxAOF']) from system.numbers limit 10; -select [14, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 1, 0, 20, 5, 0, 4, 0] = multiSearchAllPositionsCaseInsensitive(materialize('WZNWOCjFkCAAzIptkUtyPCyC'), ['iPTkuT', 'BngeNlFbKymzMYmNPfV', 'XKEjbLtADFMqS', 'dbRQKJGSFhzljAiZV', 'wZnwoCjFKCAAzIPTKuTYpc', 'yBaUvSSGOEL', 'iEYopROOYKxBwPdCgbPNPAsMwVksHgagnO', 'TljXPJVebHqrnhSiTGwpMaNeKy', 'wzNWocjF', 'bLxLrZnOCeIfxkfZEOcqDteUvc', 'CtHYpAZDANEv', '', 'XMAMpGYMiOb', 'y', 'o', 'floswnnFjXDTxantSvDYPSnaORL', 'WOcjFkcAaZIp', 'buqBHbZsLDnCUDhLdgd']) from system.numbers limit 10; -select [0, 20, 14, 0, 2, 0, 1, 14, 0, 0, 0, 1, 0, 26, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('XJMggEHaxfddDadtwKMCcPsMlSFVJ'), ['NzbUAZvCsnRnuzTglTsoT', 'ccP', 'ADTwKmc', 'JaUzcvWHMotuEMUtjsTfJzrsXqKf', 'jMGgEHaXfdddAdTWKMCcpsM', 'SMnb', '', 'AdTWkMccPSMlsfv', 'fVjPVafkp', 'goqsYAFqhhnCkGwhg', 'CNHNPZHZreFwhRMr', '', 'vcimNhmdbtoiCgVzNuvdgZG', 'sfvJ', 'AqKmroxmRMSFAKjfhwrzxmNSSjMHxKow', 'Xhub']) from system.numbers limit 10; -select [0, 0, 7, 0, 1, 1, 0, 0, 13, 0, 1, 1, 5, 0] = multiSearchAllPositionsCaseInsensitive(materialize('VQuEWycGbGcTcCCvWkujgdoWjKgVYy'), ['UevGaXmEAtBdWsPhBfqp', 'aQOrNMPmoVGSu', 'c', 'TMhzvbNJCaxtGNUgRBmTFEqgNBIBpSJ', '', 'vq', 'pVNUTCqXr', 'QSvkansbdPbvVmQpcQXDk', 'cCCvwkUjgdOWjKgVYy', 'EtCGaEzsSbJ', 'V', '', 'WycgBgCTCcCvwkujgdoWJKgv', 'xPBJqKrZbZHJawYvPxgqrgxPN']) from system.numbers limit 10; -select [4, 1, 0, 0, 0, 0, 0, 0, 0, 18] = multiSearchAllPositionsCaseInsensitive(materialize('LODBfQsqxfeNuoGtzvrUMRVWNKUKKs'), ['Bf', 'lOdbfQs', 'ZDSDfKXABsFiZRwsebyU', 'DT', 'GEUukPEwWZ', 'GNSbrGYqEDWNNCFRYokZbZEzGzc', 'kYCF', 'Kh', 'jRMxqdmGYpTkePeReXJNdnxagceitMJlmbbro', 'VrumrvWnKU']) from system.numbers limit 10; -select [1, 1, 3, 1, 10, 0, 9, 2, 2, 0, 0, 0, 0, 0, 8, 0, 1, 11, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('lStPVtsQypFlZQoQhCuP'), ['', '', 'tpV', 'L', 'PF', 'pGPggwbkQMZandXugTpUorlPOubk', 'yPFlz', 'sTPVTsQyPfLzQOqhCU', 'StPVtSq', 'cbCxBjAfJXYgueqMFNIoSguFm', 'AosIZKMPduRfumDZ', 'AGcNTHObH', 'oPaGpsQ', 'kwQCczyY', 'q', 'HHUYdzGAzVJyn', '', 'fLZQoqHcUp', 'q', 'SSonzfqLVwIGzdHtj']) from system.numbers limit 10; -select [0, 1, 2, 0, 0, 0, 13, 1, 27, 1, 0, 1, 3, 1, 0, 1, 3, 0] = multiSearchAllPositionsCaseInsensitive(materialize('NhKJtvBUddKWpseWwRiMyBsTWmlk'), ['toBjODDZoRAjFeppAdsne', '', 'HKjTvBu', 'QpFOZJzUHHQAExAqkdoBpSbXzPnTzuPd', 'gE', 'hLmXhcEOwCkatUrLGuEIJRkjATPlqBjKPOV', 'Ps', 'NH', 'l', '', 'aSZiWpmNKfglqAbMZpEwZKmIVNjyJTtDianY', 'NhKJTvBUDDkwpS', 'KJtvbUDDKWPSewwrimYbstwm', 'NHKJTvbudDKwpSEwwR', 'hmMeWEpksVAaXd', 'NHkJTvBUDd', 'kjTvbudd', 'kmwUzfEpWSIWkEylDeRPpJDGb']) from system.numbers limit 10; -select [0, 5, 0, 0, 0, 1, 1, 15, 2, 3, 4, 5] = multiSearchAllPositionsCaseInsensitive(materialize('NAfMyPcNINKcgsShJMascJunjJva'), ['ftHhHaJoHcALmFYVvNaazowvQlgxwqdTBkIF', 'yp', 'zDEdjPPkAdtkBqgLpBfCtsepRZScuQKbyxeYP', 'yPPTvdFcwNsUSeqdAUGySOGVIhxsJhMkZRGI', 'JQEqJOlnSSam', 'nAFmy', '', 'sHJmaScjUnJj', 'afmY', 'FmYpcnINKCg', 'MYPCniNkcgSS', 'YPCNiNkCgSsHjmasCJuNjJ']) from system.numbers limit 10; -select [0, 0, 6, 3, 2, 0, 8, 2, 2, 10, 0, 0, 14, 0, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('hgpZVERvggiLOpjMJhgUhpBKaN'), ['Nr', 'jMcd', 'e', 'PZVeRvggiLOPjmjh', 'GpZVe', 'cVbWQeTQGhYcWEANtAiihYzVGUoHKH', 'VGgilOPj', 'GPZVervgGiLopjmjHGuHp', 'GP', 'gil', 'fzwDPTewvwuCvpxNZDi', 'gLLycXDitSXUZTgwyeQgMSyC', 'PJmjh', 'bTQdrFiMiBtYBcEnYbKlqpTvGLmo', 'ggHxiDatVcGTiMogkIWDxmNnKyVDJth', 'pzv']) from system.numbers limit 10; -select [7, 1, 9, 3, 0, 0, 2, 0, 1, 11] = multiSearchAllPositionsCaseInsensitive(materialize('xUHVawrEvgeYyUZGmGZejClfinvNS'), ['RevGeYyuz', 'XUHvAWrev', 'Vg', 'hvawR', 'eRQbWyincvqjohEcYHMwmDbjU', 'nuQCxaoxEdadhptAhZMxkZl', 'UhVAwREvGEy', 'lHtwTFqlcQcoOAkujHSaj', '', 'eYYUzgMgzEjCLfIn']) from system.numbers limit 10; -select [0, 0, 8, 5, 9, 1, 0, 4, 12, 6, 4, 0, 0, 12] = multiSearchAllPositionsCaseInsensitive(materialize('DbtStWzfvScJMGVPQEGkGFoS'), ['CSjYiEgihaqQDxZsOiSDCWXPrBdiVg', 'aQukOYRCSLiildgifpuUXvepbXuAXnYMyk', 'fvsCjmgv', 'TWZFV', 'VscjMgVpQ', 'dBtSTwZfVsCjmGVP', 'wqpMklzJiEvqRFnZYMfd', 'StwZfVScJ', 'j', 'wzfVsCjmGV', 'STWZfVS', 'kdrDcqSnKFvKGAcsjcAPEwUUGWxh', 'UtrcmrgonvUlLnzWXvZI', 'jMgvP']) from system.numbers limit 10; -select [0, 0, 0, 0, 7, 3, 0, 11, 1, 10, 0, 0, 7, 1, 4, 0, 17, 3, 15, 0] = multiSearchAllPositionsCaseInsensitive(materialize('YSBdcQkWhYJMtqdEXFoLfDmSFeQrf'), ['TnclcrBJjLBtkdVtecaZQTUZjkXBC', 'SPwzygXYMrxKzdmBRTbppBQSvDADMUIWSEpVI', 'QnMXyFwUouXBoCGLtbBPDSxyaLTcjLcf', 'dOwcYyLWtJEhlXxiQLRYQBcU', 'KWhYjMtqdEXFo', 'BD', 'nnPsgvdYUIhjaMRVcbpPGWOgVjJxoUsliZi', 'j', '', 'YjmtQdeXF', 'peeOAjH', 'agVscUvPQNDwxyFfXpuUVPJZOjpSBv', 'kWh', '', 'dcQKWHYjmTQD', 'qjWSZOgiTCJyEvXYqaPFqbwvrwadJsGVTOhD', 'xfoL', 'b', 'DeXf', 'HyBR']) from system.numbers limit 10; -select [4, 0, 0, 13, 1, 0, 3, 13, 16, 1, 0, 1, 16, 1, 12, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('SoVPMQNqmaTGuzYxDvZvapSuPiaP'), ['pMqNQMAtGuzYxDVz', 'TEJtgLhyredMnIpoZfmWvNwpkxnm', 'XRWmsfWVOCHhk', 'u', '', 'HvkXtxFdhVIyccpzFFSL', 'VPM', 'uZyXDVzvAPsUpIaP', 'xDvzV', 'sovpmqNQmATguZYx', 'wEG', 'soVPmQnQ', 'XDVzV', '', 'GUZyXdvzva', 'FetUahWwGtwEpVdlJCJntL', 'B', 'lSCUttZM']) from system.numbers limit 10; -select [1, 0, 1, 2, 15, 0, 0, 0, 1, 0] = multiSearchAllPositionsCaseInsensitive(materialize('zFWmqRMtsDjSeWBSFoqvWsrV'), ['', 'GItrPyYRBwNUqwSaUBpbHJ', '', 'f', 'BsfOQvWsR', 'JgvsMUZzWaddD', 'wxRECkgoCBPjSMRorZpBwuOQL', 'xHKLLxUoWexAM', '', 'YlckoSedfStmFOumjm']) from system.numbers limit 10; -select [11, 1, 1, 1, 0, 0, 1, 0, 4, 0, 0, 0, 1, 0, 5, 8] = multiSearchAllPositionsCaseInsensitive(materialize('THBuPkHbMokPQgchYfBFFXme'), ['KpqGchyfBF', '', '', 'TH', 'NjnC', 'ssbzgYTybNDbtuwJnvCCM', 'tHbupKHBMOkPQgcHy', 'RpOBhT', 'uPKHbMoKpq', 'oNQLkpSKwocBuPglKvciSjttK', 'TaCqLisKvOjznOxnTuZe', 'HmQJhFyZrcfeWbXVXsnqpcgRlg', 'tHB', 'gkFGbYje', 'pkhbMokPq', 'Bm']) from system.numbers limit 10; -select [7, 10, 0, 0, 9, 0, 0, 3, 0, 10] = multiSearchAllPositionsCaseInsensitive(materialize('ESKeuHuVsDbiNtvxUrfPFjxblv'), ['uvsDBiNtV', 'DbInTvxu', 'YcLzbvwQghvrCtCGTWVuosE', 'cGMNo', 'SDb', 'nFIRTLImfrLpxsVFMBJKHBKdSeBy', 'EUSiPjqCXVOFOJkGnKYdrpuxzlbKizCURgQ', 'KeUHU', 'gStFdxQlrDcUEbOlhLjdtQlddJ', 'DBInTVx']) from system.numbers limit 10; -select [1, 0, 2, 18, 1, 3, 15, 8, 0, 0, 1, 3, 0, 23, 2, 0, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('TzczIDSFtrkjCmDQyHxSlvYTNVKjMT'), ['', 'AmIFsYdYFaIYObkyiXtxgvnwMVZxLNlmytkSqAyb', 'ZcZI', 'HXsLVYTnvKjm', '', 'CZiDsFtRKJ', 'DQYhxSl', 'fTRKjCmdqYHxsLvYtNvk', 'hxVpKFQojYDnGjPaTNPhGkRFzkNhnMUeDLKnd', 'RBVNIxIvzjGYmQBNFhubBMOMvInMQMqXQnjnzyw', '', 'c', 'vcvyskDmNYOobeNSfmlWcpfpXHfdAdgZNXzNm', 'ytnvKJM', 'ZcZidsFtRKjcmdqy', 'IRNETsfz', 'fTR', 'POwVxuBifnvZmtBICqOWhbOmrcU']) from system.numbers limit 10; -select [14, 16, 10, 2, 6, 1, 0, 8, 0, 0, 12, 1, 0, 1, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('tejdZOLhjpFLkGBWTGPfmk'), ['GBWtgPF', 'Wt', 'PflkgBWTgpFmK', 'ejdZOLhJPFlKgb', 'o', 'TejDZ', 'HlQfCP', 'hJP', 'ydiyWEfPGyRwcKGfGVdYxAXmkY', 'QsOyrgkTGMpVUAmLjtnWEIW', 'LKGBw', 'tejDzolHJpFLKgbWT', 'IK', '', 'WrzLpcmudcIJEBapkToDbYSazKTwilW', 'DmEWOxoieDsQHYsLNelMc']) from system.numbers limit 10; -select [9, 0, 1, 4, 13, 0, 0, 1, 3, 7, 9, 0, 1, 1, 0, 7] = multiSearchAllPositionsCaseInsensitive(materialize('ZWHpzwUiXxltWPAIGGxIcJB'), ['XxLTWpA', 'YOv', '', 'pzwUIXXl', 'wp', 'lpMMLDAuflLnWMFrETXRethzCUZOWfQ', 'la', '', 'HPZ', 'UixxlTw', 'xXLTWP', 'YlfpbSBqkbddrVwTEmXxgymedH', '', '', 'QZWlplahlCRTMjmNBeoSlcBoKBTnNZAS', 'UiXxlTwPAiGG']) from system.numbers limit 10; -select [0, 9, 6, 0, 4, 0, 3, 0, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('NytxaLUvmiojEepjuCzwUYPoWL'), ['LcOnnmjbZSifx', 'm', 'lUvMIOjeE', 'vuZsNMSsutiLCDbClPUSsrziohmoZaQeXtKG', 'XaLuvm', 'hlUevDfTSEGOjvLNdRTYjJQvMvwrMpwy', 'TXALuVmioJeePjUczw', 'pKaQKZg', 'PAdX', 'FKLMfNAwNqeZeWplTLjd', 'DODpbzUmMCzfGZwfkjH', 'HMcEGRHLspYdJIiJXqwjDUBp']) from system.numbers limit 10; -select [2, 1, 0, 16, 8, 1, 6, 0, 0, 1, 8, 0, 7, 0, 9, 1, 1, 0, 0, 1] = multiSearchAllPositionsCaseInsensitive(materialize('WGVvkXuhsbzkLqiIEOuyiRfomy'), ['GVv', '', 'VbldWXHWzdziNcJKqIkDWrO', 'iEOUyIRFomy', 'hsBZklqiieOuy', '', 'X', 'emXjmIqLvXsNz', 'rxhVkujX', 'wgvvK', 'HsBzKLQiie', 'wVzJBMSdKOqjiNrXrfLEjjXozolCgYv', 'UHsbzklQiiEouyirf', 'UOvUsiKtUnwIt', 'SBZKLqiIEoUYIrfom', 'wg', '', 'BefhETEirL', 'WyTCSmbKLbkQ', '']) from system.numbers limit 10; -select [8, 1, 2, 8, 1, 0, 5, 0, 0, 4, 0, 1, 14, 0, 0, 7, 0, 1] = multiSearchAllPositionsCaseInsensitive(materialize('uyWhVSwxUFitYoVQqUaCVlsZN'), ['XufitYOVqqUACVlszn', '', 'ywH', 'XUFIT', 'uywHvSWXuFIt', 'dGhpjGRnQlrZhzGeInmOj', 'vswXuFitYovqQuA', 'dHCfJRAAQJUZeMJNXLqrqYCygdozjAC', 'rojpIwYfNLECl', 'hVswxufiTYov', 'bgJdgRoye', '', 'ovQ', 'AdVrJlq', 'krJFOKilvBTGZ', 'WxuFITYOV', 'AsskQjNPViwyTF', 'u']) from system.numbers limit 10; -select [0, 2, 0, 0, 0, 6, 0, 5, 0, 15, 0, 0, 3, 0] = multiSearchAllPositionsCaseInsensitive(materialize('BEKRRKLkptaZQvBxKoBL'), ['HTwmOxzMykTOkDVKjSbOqaAbg', 'eKrRKl', 'UrLKPVVwK', 'TyuqYmTlQDMXJUfbiTCr', 'fyHrUaoMGdq', 'KLkPtaZq', 'cPUJp', 'RKLk', 'yMnNgUOpDdP', 'BX', 'tXZScAuxcwYEfSKXzyfioYPWsrpuZz', 'dsiqhlAKbCXkyTjBbXGxOENd', 'k', 'juPjORNFlAoEeMAUVH']) from system.numbers limit 10; -select [9, 0, 0, 0, 1, 4, 2, 0, 0, 0, 0, 8, 0, 2, 0, 3, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('PFkLcrbouhBTisTkuUcO'), ['UhBtistKU', 'ioQunYMFWHD', 'VgYHTKZazRtfgRtvywtIgVoBqNBwVn', 'ijSNLKch', 'pFKlcrBOuhbtIsTku', 'lCRboUHBtI', 'fKLCRBOu', 'XTeBYUCBQVFwqRkElrvDOpZiZYmh', 'KzXfBUupnT', 'OgIjgQO', 'icmYVdmekJlUGSmPLXHc', 'OuH', 'BWDGzBZFhTKQErIRCbtUDIIjzw', 'F', 'LuWyPfSdNHIAOYwRMFhP', 'kL', 'PQmvXDCkEhrlFBkUmRqqWBxYi', 'kLcrbo']) from system.numbers limit 10; -select [0, 1, 1, 6, 14, 3, 0, 1, 9, 1, 9, 0, 1, 10, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('pfynpJvgIjSqXWlZzqSGPTTW'), ['ZzeqsJPmHmpoYyTnKcWJGReOSUCITAX', '', 'P', 'jvGIj', 'wLZzQsgP', 'YnPjVGij', 'DmpcmWsyilwHwAFcKpLhkiV', '', 'I', 'pFy', 'IjsqxwLZzqSgpT', 'pKpe', 'PfynpJvgiJSqXwlzZ', 'jsQXwLZZqs', 'onQyQzglEOJwMCO', 'GV']) from system.numbers limit 10; -select [1, 17, 1, 20, 0, 0, 5, 0, 0, 0, 24, 0] = multiSearchAllPositionsCaseInsensitive(materialize('BLNRADHLMQstZkAlKJVylmBUDHqEVa'), ['bLnRaDhLm', 'kJVYlmbuD', 'bLnr', 'yLMbU', 'eAZtcqAMoqPEgwtcrHTgooQcOOCmn', 'jPmVwqZfp', 'aDHlmqS', 'fmaauDbUAQsTeijxJFhpRFjkbYPX', 'aqIXStybzbcMjyDKRUFBrhfRcNjauljlqolfDX', 'WPIuzORuNbTGTNb', 'uDhqeVa', 'fQRglSARIviYABcjGeLK']) from system.numbers limit 10; -select [2, 0, 4, 5, 1, 15, 1, 9, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('BEwjMzphoTMoGikbrjTVyqDq'), ['E', 'sClgniMsZoGTEuLO', 'jmzphotmoGIKBRjtv', 'MZPhOtmo', '', 'Kb', '', 'otm', 'tVpxYRttoVpRLencV', 'SJAhAuMttGaeMsalRjeelAGG']) from system.numbers limit 10; -select [1, 0, 0, 0, 0, 0, 4, 0, 0, 19, 0, 7] = multiSearchAllPositionsCaseInsensitive(materialize('yNnYRQfcyemQdxUEPOiwRn'), ['', 'SJteoGNeIAMPWWBltkNKMrWDiVfR', 'kKnnKQhIPiekpnqTXJuyHfvWL', 'GPDUQEMWKzEEpvjLaIRYiuNfpzxsnSBX', 'oPrngRKwruyH', 'ukTSzFePSeVoeZeLQlAaOUe', 'yRqfcyemQDXUepo', 'CwmxidvpPHIbkJnVfSpbiZY', 'FUxmQdFVISApa', 'iwr', 'ciGHzDpMGNQbytsKpRP', 'Fcy']) from system.numbers limit 10; -select [0, 1, 0, 11, 2, 0, 1, 3, 0, 0, 0, 21] = multiSearchAllPositionsCaseInsensitive(materialize('EgGWQFaRsjTzAzejYhVrboju'), ['DVnaLFtCeuFJsFMLsfk', '', 'thaqudWdT', 'Tzazejy', 'GGW', 'RolbbeLLHOJpzmUgCN', '', 'gwqfarsjtzaZeJYHvR', 'KkaoIcijmfILoe', 'UofWvICTEbwVgISstVjIzkdrrGryxNB', 'UJEvDeESWShjvsJeioXMddXDkaWkOiCV', 'B']) from system.numbers limit 10; -select [0, 5, 2, 0, 0, 7, 0, 0, 0, 11, 0, 12, 22, 10, 0, 12] = multiSearchAllPositionsCaseInsensitive(materialize('ONgpDBjfRUCmkAOabDkgHXICkKuuL'), ['XiMhnzJKAulYUCAUkHa', 'dbj', 'nGpDbJFRU', 'xwbyFAiJjkohARSeXmaU', 'QgsJHnGqKZOsFCfxXEBexQHrNpewEBFgme', 'JFruCM', 'DLiobjNSVmQk', 'vx', 'HYQYzwiCArqkVOwnjoVNZxhbjFaMK', 'Cm', 'ckHlrEXBPMrVIlyD', 'M', 'xI', 'UcmkAOabdKg', 'jursqSsWYOLbXMLQAEhvnuHclcrNcKqB', 'mKaoaBdKghxiCkkUUL']) from system.numbers limit 10; -select [0, 1, 0, 1, 0, 0, 0, 0, 7, 21] = multiSearchAllPositionsCaseInsensitive(materialize('WhdlibCbKUmdiGbJRshgdOWe'), ['kDPiHmzbHUZB', '', 'CukBhVOzElTdbEBHyrspj', '', 'QOmMle', 'wiRqgNwjpdfgyQabxzksjg', 'RgilTJqakLrXnlWMn', 'bSPXSjkbypwqyazFLQ', 'CBkuMDiGbJRShGdOWe', 'dow']) from system.numbers limit 10; -select [0, 8, 0, 1, 1, 0, 1, 7, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('oOccAoDDoPzHUyRqdWhJxNmATEqtE'), ['LFuvoQkVx', 'DoPzh', 'YaBSTdWvmUzlgRloppaShkRmLC', 'oO', '', 'eeEpOSLSXbyaOxTscOPoaTcKcchPmSGThk', '', 'dDO', 'oFXmyIJtmcSnebywDlKruvPUgmPFzEnMvA', 'vCs', 'MsxHLTgQcaQYZdPWJshIMWbk', 'yqrjIzvrxd']) from system.numbers limit 10; -select [0, 16, 0, 0, 0, 0, 7, 1, 0, 0, 1, 2, 1, 4, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('FtjOSBIjcnZecmFEoECoep'), ['FQQwzxsyauVUBufEBdLTKKSdxSxoMFpL', 'EOecoEP', 'HGWzNTDfHxLtKrIODGnDehl', 'ZxirLbookpoHaxvASAMfiZUhYlfuJJN', 'mKh', 'GZaxbwVOEEsApJgkLFBRXvmrymSp', 'Ij', '', 'X', 'AnCEVAe', 'fTj', 'tjOSbIjcNZECMfeoEC', '', 'OsBIjcN', 'LtdJpFximOmwYmawvlAIadIstt', 'JOsBiJCNzEc']) from system.numbers limit 10; -select [0, 2, 0, 0, 19, 0, 0, 12, 1, 0, 3, 1, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('ugpnWWncvqSLsYUCVXRZk'), ['yOWnQmZuhppRVZamgmRIXXMDQdeUich', 'gPNww', 'jlyFSbvmjaYPsMe', 'fQUeGVxgQdmPbVH', 'rZk', 'ariCX', 'grAffMPlefMQvugtAzN', 'LsYuCVX', '', 'jZFoQdWEWJFfSmNDqxIyNjvxnZJ', 'P', 'UgPN', 'JmKMsbegxNvusaiGGAZKglq', 'qArXLxzdYvabPv']) from system.numbers limit 10; -select [0, 0, 0, 0, 0, 0, 8, 0, 0, 1, 1, 15, 0, 1, 7, 0] = multiSearchAllPositionsCaseInsensitive(materialize('nxwotjpplUAXvoQaHgQzr'), ['ABiEhaADbBLzPwhSfhu', 'TbIqtlkCnFdPgvXAYpUuLjqnnDjDD', 'oPszWpzxuhcyuWxiOyfMBi', 'fLkacEEeHXCYuGYQXbDHKTBntqCQOnD', 'GHGZkWVqyooxtKtFTh', 'CvHcLTbMOQBKNCizyEXIZSgFxJY', 'PlUAxVoQah', 'zrhYwNUzoYjUSswEFEQKvkI', 'c', 'NXWOt', '', 'qAhG', 'JNqCpsMJfOcDxWLVhSSqyNauaRxC', '', 'PpLuaxV', 'DLITYGE']) from system.numbers limit 10; -select [2, 0, 0, 1, 0, 0, 28, 1, 16, 1] = multiSearchAllPositionsCaseInsensitive(materialize('undxzJRxBhUkJpInxxJZvcUkINlya'), ['ndxzjRxbhuKjP', 'QdJVLzIyWazIfRcXU', 'oiXcYEsTIKdDZSyQ', 'U', 'dRLPRY', 'jTQRHyW', 'Y', '', 'nxxJZVcU', '']) from system.numbers limit 10; -select [1, 4, 1, 0, 4, 1, 0, 1, 16, 1, 0, 0, 0, 8, 12, 14, 0, 2] = multiSearchAllPositionsCaseInsensitive(materialize('lrDgweYHmpzOASVeiFcrDQUsv'), ['', 'gwEYhMP', 'LrDGwEyHmPzOaSVEifC', 'oMN', 'gwEYhMpZO', 'lrdGWEy', 'pOKrxN', 'lrDgwEyhmpZoaSv', 'eifcrdqU', 'LrDgw', 'dUvarZ', 'giYIvswNbNaBWprMd', 'pPPqKPhVaBhNdmZqrBmb', 'hmPzoASVEiF', 'O', 'SVEi', 'gIGLmHnctIkFsDFfeJWahtjDzjPXwY', 'rDGweyHmP']) from system.numbers limit 10; -select [0, 0, 11, 1, 1, 1, 0, 16, 0, 1, 5, 0, 0, 0, 2, 0, 2, 0] = multiSearchAllPositionsCaseInsensitive(materialize('XAtDvcDVPxZSQsnmVSXMvHcKVab'), ['bFLmyGwEdXiyNfnzjKxUlhweubGMeuHxaL', 'IhXOeTDqcamcAHzSh', 'ZSQsNMvsxmVHcK', '', '', '', 'dbrLiMzYMQotrvgwjh', 'MvsxMV', 'zMp', 'XaTDvCdvpXzsqSNMVSxm', 'v', 'LkUkcjfrhyFmgPXPmXNkuDjGYlSfzPi', 'ULpAlGowytswrAqYdaufOyWybVOhWMQrvxqMs', 'wGdptUwQtNaS', 'ATdVcdVPXzSqsnmVSXMvHcKVab', 'JnhhGhONmMlUvrKGjQcsWbQGgDCYSDOlor', 'atdvCdvpXzsqSnMVSxMVhCkvAb', 'ybNczkKjdlMoOavqBaouwI']) from system.numbers limit 10; -select [8, 0, 0, 0, 4, 0, 0, 5, 5, 2] = multiSearchAllPositionsCaseInsensitive(materialize('XPquCTjqgYymRuwolcgmcIqS'), ['qgyYMruW', 'tPWiStuETZYRkfjfqBeTfYlhmsjRjMVLJZ', 'PkTdqDkRpPpQAMksmkRNXydKBmrlOAzIKe', 'wDUMtn', 'UcTJQgYYMRuWoLCgMcI', 'PieFD', 'kCBaCC', 'Ct', 'C', 'pQuctjqgyymRuwOLCgmc']) from system.numbers limit 10; - -select [1, 0, 7, 1, 0, 24, 17, 0, 0, 0, 2, 0, 1, 7, 4, 1, 12, 8] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('гГБаДнФбпнЩврЩшЩЩМщЕБшЩПЖПчдт'), ['', 'таОХхрзИДжЛСдЖКЧжБВЩжЛкКХУКждАКРеаЗТгч', 'Ф', '', 'ЙЩИФМфАГщХзКЩЧТЙжмуГшСЛ', 'ПЖпчдТ', 'ЩМщЕбшЩПжПч', 'ФгА', 'гУД', 'зУцкжРоППЖчиШйЗЕшаНаЧаЦх', 'гбаДНФбПНЩВРЩШЩщМЩеБшЩпжПЧд', 'РДЧЖАбрФЦ', 'гГ', 'ФбпНщвр', 'адНфБПнщвРщШщщМщЕбШщ', 'ггб', 'ВРЩ', 'бПНщврЩш']) from system.numbers limit 10; -select [0, 12, 8, 0, 12, 0, 0, 10, 0, 8, 4, 6] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('айРВбЧБжКВИхБкчФЖЖНВнпФйФБДфЗ'), ['ЛрЦфуУДВК', 'хБкчфЖжНвнпфйфБдФ', 'жКВИХБкчФЖжНвнПф', 'кЖчвУцВСфЗБТИфбСжТИдРкшгзХвщ', 'хбк', 'штДезйААУЛчнЖофМисНЗо', 'нлнШЧВЙхОПежкцевчлКрайдХНчНб', 'вИХбкчфжжНВН', 'ЩдзЦТуоЛДСеШГфЦ', 'ЖКВихбКЧфжЖ', 'вбЧбЖкВихБкЧфЖжНВ', 'Чб']) from system.numbers limit 10; -select [18, 15, 0, 0, 0, 0, 5, 0, 14, 1, 0, 0, 0, 0, 0, 15] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('пМИОкоЗжГйНТПЙацччЧАЩгЕВБбЕ'), ['ЧЧАЩгЕВБ', 'а', 'ФбРВщшййпХдфаЗЖлЛСЗПРШПпАОинЧКзЩхждН', 'ЛфРКДЙВСУСЙОчтнИкРЗбСГфкЩреИхЛлчХчШСч', 'ШйвБПАдФдФепЗТкНУрААйеЧПВйТоЧмБГДгс', 'ФтЙлЖЕсИАХИФЗаЕМшсшуцлцАМФМгбО', 'КО', 'лиШБнлпОХИнБаФЩдмцпжЗИЛнвсЩЙ', 'йацччЧАщгевбБЕ', 'ПмИоКозжГйНТП', 'ИГНннСчКАИСБщцП', 'ПнжмЙЛвШтЩейХЛутОРЩжифбЗчгМУЛруГпх', 'ХжЗПлГЖЛйсбпрЩОТИеБвулДСиГзлЛНГ', 'учклЦНЕгжмщлжАшщжМд', 'ЩеПОЙтЖзСифОУ', 'АЦЧ']) from system.numbers limit 10; -select [10, 0, 1, 1, 6, 1, 7, 6, 0, 0, 0, 2, 12, 0, 6, 0, 4, 8, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('квхБнцхйзЕпйИмтЙхфзвгдФ'), ['еПйИМт', 'хгкиМжСБИТНщенЩИщНСкй', '', 'Квхб', 'цхЙЗЕПйИмТйХФЗ', 'к', 'хйЗЕПЙИмтй', 'Цх', 'нКлШбМЖГйШкРзадрЛ', 'ДштШвБШТг', 'СЦКйЕамЦщПглдСзМлоНШарУтМднЕтв', 'ВхБнцхйЗЕПйимТ', 'йимтЙХФЗВГД', 'жчссунЙаРцМкЖУЦщнцОЕхнРж', 'цХЙЗЕП', 'ОгНФдМЛПТИдшцмХИеКйРЛД', 'бнЦхЙ', 'ЙЗе', 'згЩищШ', 'фХлФчлХ']) from system.numbers limit 10; -select [0, 0, 0, 12, 0, 0, 27, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('хНпсРТХВдтоЦчдлеФПвнЛгЗКлПйнМВ'), ['ШиБфЗШПДЧхОЩшхфщЗЩ', 'иГйСЧЗтШЛуч', 'АЗХЦхедхОуРАСВЙС', 'цчдЛЕфП', 'СДбйГйВЕРмЙЩЛщнжен', 'НДлцСфТшАщижгфмуЖицжчзегЕСЕНп', 'й', '', 'йлчМкРИЙиМКЙжссЦТцРГзщнхТмОР', 'ПРцГувЧкйУХггОгЖНРРсшГДрлЧНжГМчрХЗфЧЕ']) from system.numbers limit 10; -select [0, 0, 2, 0, 10, 7, 1, 1, 0, 9, 0, 2, 0, 17, 0, 0, 0, 6, 5, 2] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЙзЗжпжДЕСУхчйдттСЙзоЗо'), ['щОЙУшееЧщкхГККреБкВ', 'жВ', 'ззЖпждЕсУХчЙДТТсЙ', 'ЙЦШЦЙЖзХШРвнкЕд', 'УхчйДтТсйЗОз', 'дЕСу', '', '', 'дсцеррищндЗдНкжаНЦ', 'сУхчЙдттсйзОзО', 'ЦЖРжмц', 'ЗЗ', 'СгЛГАГЕЖНгщОеЖЦДмБССцЩафзЗ', 'Сйзоз', 'ЦГХТЕвЕЗБМА', 'пмВоиеХГжВшдфАЖАшТйуСщШчИДРЙБнФц', 'Оа', 'ждЕ', 'ПжДесу', 'ЗзЖПждЕСУ']) from system.numbers limit 10; -select [0, 0, 0, 0, 5, 1, 0, 6, 0, 1, 17, 15, 1, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('уФШЙбШТоХВбзЦцЖОЕКТлщхнЖГ'), ['цЛ', 'ууМ', 'ТИгЙолМФсибтЕМнетквЦИЩИБккйн', 'оФОаМогсХЧЦооДТПхб', 'бШтОХВбЗцЦЖоЕКтЛ', 'уфШйбШтоХ', 'фдтщрФОЦсшигдПУхЛцнХрЦл', 'ШтО', 'НИкИТрбФБГИДКфшзЕмЙнДЖОсЙпЩцщкеЖхкР', 'уфШЙБш', 'екТлщ', 'ЖоекТл', 'уфШйБшТоХвбз', 'ТуОхдЗмгФеТаафЙм']) from system.numbers limit 10; -select [0, 1, 6, 1, 0, 1, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чМЩБЛЛПРлщкВУбПефХВФлАЗШао'), ['гаТкЛВнрвдПМоеКПОйр', 'ч', 'ЛпрЛЩКвуБпе', 'ЧмЩб', 'ц', '', 'жгаччЖйГЧацмдсИИВЩЩжВЛо', 'йГеЙнБзгнкЦЛБКдОЕЧ', 'ПоЦРвпЕЗСАШж', 'ЙОНЦОбиееО']) from system.numbers limit 10; -select [2, 0, 17, 1, 0, 0, 0, 5, 0, 4, 0, 0, 0, 0, 0, 2] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЕаЩичщМЦЖиЗБЛЧжуНМЧК'), ['АЩиЧЩ', 'ИлУсшДБнжщаатуРТтраПОЙКЩйТГ', 'НМЧк', 'Еа', 'зАВФЛЩбФрМВШбПФГгВЕвЖббИТйе', 'РЗНРБЩ', 'ЦдЙНГпефзЛчпУ', 'ч', 'НШШчПЗР', 'ИчЩмЦжИЗБлЧЖУНМч', 'аннвГДлмОнТЖЗЙ', 'ШдчЩшЕБвхПУсШпг', 'гФИШНфЖПжймРчхАБШкЖ', 'ЖзгЖАБлШЗДпд', 'Д', 'ащиЧ']) from system.numbers limit 10; -select [4, 1, 0, 7, 0, 7, 1, 1, 0, 3, 7, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('иОцХКЙвувМИжШдУМУЕйНсБ'), ['ХкйвуВмИжШдУм', '', 'звМАОМЩщЙПшкиТчЩдгТЦмфзеИ', 'вуВМиж', 'КДщчшЙВЕ', 'в', '', 'ИоЦхКЙВувМижШ', 'ЕвТАРи', 'цхКЙвувмИЖШДумуе', 'вУвМи', 'зПШИХчУщШХУвврХйсуЙЗеВЧКНмКШ']) from system.numbers limit 10; -select [0, 5, 0, 0, 0, 0, 0, 12, 0, 11] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЦОфбчУФсвТймЦчдщгЩжИАБ'), ['йлрк', 'ЧуФсвтйМцчдЩгщ', 'МНлЕжорв', 'иНзТЖМсмх', 'шЕМЖжпИчсБжмтЧЙчщФХб', 'жШХДнФКАЩГсОЩвЕаам', 'НпКЦХулЛвФчШЕЗкхХо', 'мЦчДЩгЩжиАб', 'мпцгВАЕ', 'Й']) from system.numbers limit 10; -select [1, 0, 0, 0, 8, 0, 2, 0, 0, 7] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чТХЙНщФфцИНБаеЖкОвлиУДР'), ['', 'рВХмжКцНцИЙраштМппсодЛнЧАКуЩ', 'ИХфХЖЧХВкзЩВЙхчфМрчдтКздиОфЙжУ', 'Гзлр', 'фЦи', 'абПф', 'тХЙНщффЦИн', 'нссГбВеЖх', 'амлЗщрсУ', 'фФ']) from system.numbers limit 10; -select [0, 9, 11, 0, 11, 1, 0, 0, 0, 1, 6, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('зДЗпщАцвТгРдврщхЩфЖл'), ['йХЛ', 'Т', 'рд', 'АИЦщгниДфВОе', 'Р', 'здзпщ', 'вКТвВШмгч', 'ввирАйбЗЕЕНПс', 'тХиХоОтхПК', '', 'аЦВТгРДврщ', '', 'уЗЗЖвУЕйтчудноЕКМЖцВРаНТЙЗСОиЕ', 'оЕфПхЕДжАаНхЕцЖжжофЦхкШоБЙр']) from system.numbers limit 10; -select [1, 1, 0, 0, 1, 7, 0, 0, 0, 2] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('йЛПЛшмЦШНЖРрЧрМцкЖзЕНжЧДелФжАн'), ['', 'йЛПлшМЦшНЖррч', 'ПНКдфтДейуиШзЗХАРУХизВ', 'ПценмщЧОФУСЙЖв', '', 'ЦшнжрРчрМЦКЖЗе', 'МрПзЕАгжРбТЧ', 'ЕДФмаФНвТЦгКТЧЦжцЛбещЛ', 'УтПУвЛкТасдЦкеИмОещНИАоИжЖдЛРгБЩнвЖКЛЕП', 'Л']) from system.numbers limit 10; -select [1, 5, 1, 1, 0, 0, 1, 1, 0, 2, 19, 0, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('сйДпмжнДРщКБгфцЖОчтГНБ'), ['', 'МЖнДРщ', 'Сй', '', 'пУщ', 'йгВИАЦнозаемТиХВвожКАПТдкПИаж', 'Сйд', 'СЙДпмжНдРщ', 'ФПщБцАпетаЙФГ', 'ЙдпМжНдрЩКбГфЦжОЧТГНб', 'т', 'гллрБВМнвУБгНаЙцМцТйЙФпзЧОЙЛвчЙ', 'йДПМжндРЩкБ', 'ЗмфОмГСНПщшЧкиССдГБУсчМ']) from system.numbers limit 10; -select [0, 18, 10, 5, 0, 2, 8, 1, 4, 11] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ХпИРддХрмВНйфчвгШиЧМКП'), ['хЗФДлДУБЙаЦтжРБЗсуйнЦпш', 'иЧмК', 'внЙ', 'д', 'зиМУЩГиГ', 'ПИр', 'РМвнЙфчвгШич', '', 'РдДхРМ', 'нЙфчВГШИ']) from system.numbers limit 10; -select [18, 0, 0, 1, 0, 0, 6, 0, 0, 9] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('нГгФкдуФШуИТбшпХфтаГт'), ['Таг', 'рРпшУйчГд', 'гК', '', 'лаВНбездпШШ', 'ЕБРйаНрОБожкКИсв', 'ДУфШУитБ', 'ГРиГШфШтйфЖлРФзфбащМЗ', 'мхЩжЛнК', 'ШуИтБШ']) from system.numbers limit 10; -select [13, 0, 0, 7, 0, 15, 0, 0, 15, 0, 0, 5, 6, 0, 18, 21, 11, 1] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('рлобшдПЦИхжФуХщжгПФукшзт'), ['УхщжГ', 'ТВщЦфФсчЩГ', 'ЕжФШойжуЛРМчУвк', 'пцИХжфуХЩж', 'бР', 'щЖГПфуКШЗТ', 'йжРГгЛуШКдлил', 'ТщЖГкбШНИщЩеЩлаАГхрАфЙНцЦгВкб', 'щжГПфУ', 'бкаДБЛХ', 'АЗ', 'шДПЦихжфух', 'дП', 'вфнЙобСцвЩмКОбЦсИббФКзЩ', 'пФУкшзТ', 'К', 'жфу', '']) from system.numbers limit 10; -select [12, 19, 8, 1, 0, 0, 0, 15, 0, 0, 12, 2, 0, 4, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЦкЛЗепкЕХЩГлКФрБдТрлвйАхдООШ'), ['лК', 'рЛв', 'Ехщ', '', 'еаПКБгЦЩАоЗВонйТЗгМхццСАаодМЕЩГ', 'ишОНиеБидфбФБЖриУЩЩ', 'дуж', 'РбДТ', 'пЗсГХКсгРущкЙРФкАНЩОржФвбЦнЩНЖЩ', 'щрОУАГФащзхффКвЕйизцсйВТШКбнБПеОГ', 'лкФрБдТРлвЙа', 'КЛзеп', 'УЛФЗРшкРщзеФуМвгПасШЧЛАЦр', 'зеПКеХщглкфР', 'ЦЖЗдХеМЕ', 'зЖжрт', 'уЩФрйрЖдЦз', 'МфцУГЩтвПАЦжтМТоеищЕфнЖй']) from system.numbers limit 10; -select [0, 0, 1, 0, 1, 0, 0, 7, 0, 5, 1, 6, 1, 1, 1, 5, 6, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('РННЕШвжМКФтшДЙлфЛИзЙ'), ['ГаМРош', 'Дтфс', '', 'еБбиаКщГхххШвхМЖКзЛАезФУчХо', 'РНн', 'сВбТМ', 'ЖЗЦПБчиСйе', 'жМкфтШДЙл', 'нЖХуеДзтЧтулиСХпТпеМлИа', 'ШВжМкФТШдЙлфл', '', 'вЖМКфТ', '', '', '', 'швЖМКфтШДЙЛфлИЗй', 'вЖмКФТ', 'еМ']) from system.numbers limit 10; -select [0, 0, 15, 1, 0, 0, 8, 1, 0, 0, 0, 4, 8, 10] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('РиучГийдХутДЕЙДпфиуд'), ['ЩмгцлЖрц', 'ЕСжСлЩЧИЖгЗЛлф', 'дП', '', 'щГЦаБтПШВзЦСрриСЙбД', 'тдРгОЛТШ', 'д', '', 'КЕбЗКСХЦТщЦДЖХпфаЧйоХАл', 'мТвзелНКрЖЧЦПпЕЙвдШтеШйБ', 'ЙОТКрБСШпШд', 'ЧГ', 'ДХУТДЕЙд', 'УТд']) from system.numbers limit 10; -select [0, 0, 0, 0, 15, 0, 0, 0, 11, 0, 0, 5, 1, 1, 0, 2, 3, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('МшазшргхОПивОлбмДоебАшцН'), ['ЦИшштН', 'еМСЗкФЕКДйОНМ', 'ЛСчГрбеРЕбЩМПМЗЦИп', 'ХнИПЧжЗдзФщЗ', 'бмдоЕ', 'гМОдйсбТСЦЩбФВЗШзшщбчегаЕмЕБаХаРР', 'фщнР', 'щмТчФчсМАОгчБщшг', 'иВ', 'УщцГОшТзпУХКоКЖБеМШ', 'мйаАЛцАегСмПОаСТИСфбЧДБКоИВчбЦЙ', 'шРгхоп', '', '', 'еИпАЩпнЛцФжЩХИрЧаИИТЛвшиСНЩ', 'шаЗ', 'АЗ', 'ФгдтфвКЩБреногуир', 'ДБжШгщШБЩпЖИЛК', 'ЧдРЩрбфЛзЙклхдМСФУЙЛн']) from system.numbers limit 10; -select [5, 0, 0, 18, 13, 0, 2, 7, 0, 0, 1, 15, 1, 0, 0, 0, 3, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('хщеКЗПчуНЙтрЧЩгфСбоКЕАДТййАрр'), ['зп', 'хчПЦшпДбзСфНВЧзНжЕМФОП', 'ЧЖхЕУк', 'БОКеАдтЙЙа', 'чЩГфС', 'шллддЩщеМжШйкЩн', 'щЕкзпЧуНЙТ', 'ЧунйтРЧщгФс', 'ввНздЙуоТЖРаВЙчМИчхРвфЛЖБН', 'ЗХМХПщПкктцАзщЙкдпжф', '', 'ГФСбОкеАДтйЙа', '', 'МБХВЕчпБМчуххРбнИМЛТшЩИщЙгаДцзЛАМвйаО', 'ЛкОзц', 'ЕцпАДЗСРрсЕвтВщДвцбЗузУннТИгХжхрцПДРДПм', 'екЗПЧунЙТРчщгФсбоК', 'шпИфЕчгШжцГВСйм', 'ЛхйЧбЧД', 'ВзЗоМцкЩНХГж']) from system.numbers limit 10; -select [0, 0, 6, 20, 0, 10, 0, 0, 0, 9, 10, 3, 23, 1, 0, 0, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('лцапШиХчЛДшдксСНИбшгикзчЙанми'), ['ХууатТдтбодМГЧгщЧнклШтЗПНчкЦОаЙг', 'МЦЧчпИхКЛаФхщХдРУДщжУчфлжахц', 'иХЧлдшдкСсНИбШГикзЧЙ', 'гикЗчйА', 'ГсТзЛОфИББлекЩАсЛвмБ', 'Д', 'ЦХрТЖощНрУШфнужзжецсНХВфЩБбДУоМШШиГйж', 'йуВдЕзоггПВДЖб', 'ЙфБГйХМбжоакЖЛфБаГИаБФСнБЖсТшбмЗЙТГОДКИ', 'ЛДШдКССНИБшГикзч', 'ДШдКССниБ', 'аПШИХчЛДШДКсс', 'з', '', 'ФоохПЩОГЖоУШлКшзЙДоуп', 'хАДХЩхлвУИсшчрбРШУдФА', 'ЦА', 'гвптУФлчУуРхпрмЖКИрБеЩКчН']) from system.numbers limit 10; -select [0, 4, 5, 7, 15, 3, 3, 17, 7, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('зЗАЩлЕЕЕПИохЧчШвКЧйрсКХдд'), ['пКРбуШОНТЙБГНзИРвЖБсхрЛщчИрлЧУ', 'ЩЛЕЕЕПиоХЧ', 'ЛеЕеп', 'Еепио', 'швкЧйрС', 'ащЛеееПИох', 'АЩлеЕЕпиОхЧЧШвкЧЙРсК', 'КчйРскхД', 'ЕЕПИохччшВКчй', 'у']) from system.numbers limit 10; -select [1, 12, 0, 8, 1, 1, 0, 1, 5, 0, 1, 0, 0, 0, 0, 3, 1, 0, 4, 5] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ПмКСйСКЖККмШеоигЙчПфжТ'), ['', 'Шео', 'РчвлдЙЙлПщуКмтН', 'жкКмшЕоИГЙЧ', '', '', 'йРмМЖнПиЙ', '', 'йс', 'тфФРСцл', '', 'щлЩХиКсС', 'кпнТЖпФЩиЙЛ', 'абкКптбИВгмЧкцфЦртЛДЦФФВоУхЗБн', 'чНшоВСГДМйДлтвфмхХВВуеЩЦВтЖтв', 'кС', '', 'фидБлйеЙЧШРЗЗОулщеЕЩщЙсЙшА', 'СЙс', 'йсКжкКМшЕо']) from system.numbers limit 10; -select [0, 0, 1, 0, 2, 2, 1, 2, 7, 0, 1, 2, 1, 0, 6, 8] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('УгЖЕугАЩХйчидаррлжНпфФГшр'), ['утвШ', 'кЕвФч', 'угжеУг', 'тШлТвЕШЗчЖеЛНджЦазЩХцж', 'гЖеугаЩхй', 'ГжЕугаЩХйЧидАР', 'УгжЕУГаЩХЙЧИда', 'гЖеу', 'ащхЙчИ', 'мЧлщгкЛдмЙЩРЧДИу', '', 'ГжеугАщХйЧиДаРРЛЖНП', '', 'зЕМвИКбУГКЩФшоГЧГ', 'ГАЩХйчИДАррлЖНпФфг', 'ЩХЙчИдАррЛЖНпфФгш']) from system.numbers limit 10; -select [1, 0, 0, 7, 0, 6, 0, 11, 0, 0, 0, 2, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЗЕГЛЩПцГНтзЕЦШЧхНКГТХЙЙФШ'), ['', 'шзкиЗсаИщАБмаз', 'Ж', 'ц', 'гШуЕжЛСПодРнхе', 'пцГНтЗЕЦ', 'щРкЩАеНржЙПМАизшщКвЗщглТкКИф', 'ЗеЦшчхнКГтхЙЙ', 'пелгЩКкцвтфнжЖУуКосЙлкЛ', 'рф', 'хНШчНрАХМШщфЧкЩБНзХУкилЙмП', 'ЕгЛЩПЦгнтзецШЧ', 'ЩУчБчРнЖугабУоиХоИККтО', 'СГмЦШтФШЛмЙЩ', 'ауТПЛШВадоХМПиБу', 'ЩЩйр']) from system.numbers limit 10; -select [2, 2, 1, 0, 0, 0, 0, 0, 1, 0, 7, 9, 0, 15, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('гЙЧЙФХнЖБвомгАШГбОВГксИйцз'), ['ЙЧйфхНЖбвО', 'Й', 'гЙЧйфхнЖбв', 'хсЩмШЙЙММВЦмУБТчгзУЛР', 'зктшп', 'дЕоиЖлгШж', 'хКкаНЛБ', 'ЗКйСчсоЗшскГЩбИта', '', 'у', 'НжбВОмгашГ', 'БВо', 'ещфРШлчСчмаЖШПЧфоК', 'шгбо', 'ЙСтШШДЩшзМмдпЧдЙЖевТвоУСЕп', 'Л']) from system.numbers limit 10; -select [0, 9, 0, 0, 18, 13, 13, 11, 0, 0, 4, 1] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЙЛмоЦСдТаоФчШКЖЦСНРаРЦзоС'), ['ДфгЗАасВфаМмшхчлмР', 'аоФчШкЖцСнРАРЦзОС', 'зЩзнйтФРТЙжУлхФВт', 'чЦкШВчЕщДУМкхЛУЩФшА', 'н', 'Шк', 'шКЖцсНРаРцЗос', 'фчшкЖцснрАРЦз', 'лку', 'пЧШМЦквоемЕщ', 'о', 'йЛМоцСДТАофЧшкжЦСнРаРЦзос']) from system.numbers limit 10; -select [21, 0, 0, 17, 1, 11, 0, 2, 0, 7] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('кЧЖнЕбМЛпШЗХиЙжиМщлнСФрПЧЖВН'), ['сФ', 'гцХаШЛсаШЛкшфЧОКЛцзешХСиЩоаЕОш', 'Г', 'МщЛНСФРпч', '', 'зХ', 'ОАДепНпСГшгФАЦмлуНуШШЗфдЧРШфрБЛчРМ', 'чЖне', 'СфЕАбФн', 'М']) from system.numbers limit 10; -select [4, 0, 1, 1, 0, 2, 4, 16, 3, 6, 5, 0, 0, 6, 1, 0, 5, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('кдАпЩСШИСцРхтеСиФЖЧСсОоц'), ['пщСшиСцрХТЕсифЖчССоОц', 'рхнкикДТКДВШчиЖЦнВм', '', '', 'жПЛСнЦцн', 'дА', 'ПщсШИсцрХтЕс', 'иФжЧсСоОЦ', 'ап', 'с', 'щсШИ', 'МАзашДРПЩПзРТЛАсБцкСШнЕРЙцИЩлТЛеУ', 'ичцпДбАК', 'сшИСЦрхтЕсифжчСсООц', 'КдАПЩСшИСЦРХТЕсИфЖЧСсо', 'ЛнБсИПоМЩвЛпиЩЗЖСд', 'щс', 'шщДНБаСщЗАхкизжнЛАХЙ']) from system.numbers limit 10; -select [0, 13, 0, 2, 16, 1, 3, 0, 9, 0, 2, 0, 1, 4, 0, 0, 0, 1] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('иНхеЕкхЩщмгзМГхсгРБсхОКцУгуНБ'), ['ДиоУлФЖЛисУСЕтсЕалщн', 'МгХсгрБСХО', 'ЖХНцршПшгйО', 'нХЕЕкхЩ', 'сГРбсхОКцУг', '', 'х', 'Ж', 'щМгЗмгхСг', 'СрпхДГОУ', 'НхеЕкХщ', 'ПМтБцЦЙЖАЙКВБпФ', 'ИнхеЕ', 'еЕКхЩ', 'мМГлРзш', 'гтдоЙБСВещкЩАЩЦйТВИгоАЦлчКнНРНПДЖшСЧа', 'ЖшеН', '']) from system.numbers limit 10; -select [1, 5, 0, 0, 3, 0, 2, 0, 14, 14, 1, 0, 17, 13, 3, 25] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('айлзсЗБоГйтГжЙРККФхКшлНРОрЦкфо'), ['', 'с', 'Д', 'шиБраНИЦЧуИжп', 'Лз', 'ДРБСУфКСшцГДц', 'йЛЗСЗбОгЙтГЖйРК', 'ЕЙЦсвРЕШшщЕЗб', 'ЙркКфхкшЛнРОР', 'ЙРкКФхкШ', 'а', 'ГдоДКшСудНл', 'КФхКшлНРоР', 'ж', 'лзСзБогйТГЖйрККф', 'оР']) from system.numbers limit 10; -select [6, 0, 8, 10, 1, 0, 1, 13, 0, 0, 0, 2, 2, 0, 4, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('РучУлрХчЗУпИчДТЕфщИЙщрНлн'), ['РХЧ', 'оДсГСЛЙшйиЧРСКзчХВоХарцНШ', 'ЧЗУпИ', 'УПичдТе', 'Р', 'ВЙЩхжАутПСНЦфхКщеЩИуЧдчусцАесзМпмУв', '', 'ЧдТ', 'ООсШИ', 'ФШсВжХтБУШз', 'ЕЩуДдшкМУРЕБшщпДОСАцйауи', 'УЧ', 'УЧУЛрХчзуПИчдТеФщий', 'йнЦцДСхйШВЛнШКМСфмдЩВйлнеЖуВдС', 'улрхчзупиЧдтефщИ', 'СХТЧШшГТВвлЕИчНОВи']) from system.numbers limit 10; -select [0, 0, 0, 2, 1, 1, 0, 1, 19, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('УецжлЦЦщМшРГгЩЩдБмхЖЗЧзШЙб'), ['НзИуАузуРЗРуКфоТМмлПкрсмЕЕЕнТ', 'ЕЩГХхЧш', 'ХоЙпООчфЖввИжЙшЖжЕФОтБхлВен', 'ЕЦЖЛЦцщ', '', '', 'ухогСИФвемдпаШЗуЛтлизОЧ', 'УецЖ', 'ХЖзЧЗ', 'П', 'мБкзХ', 'уБуОБхШ']) from system.numbers limit 10; -select [6, 1, 15, 5, 0, 0, 0, 3, 2, 4, 0, 12, 0, 2, 0, 3, 1, 6, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ГЖФеачМаКчПСпкВкхсПтг'), ['чмАкЧ', '', 'ВкХс', 'ачМА', 'КлтжУлОЛршБЕблФЩ', 'тцуМфж', 'л', 'фе', 'Жф', 'ЕАЧМак', 'лЖЕРТнФбЧЙТййвзШМСплИхбЙЛЖзДпм', 'СпкВК', 'ЩзчжИш', 'жФеАчМ', 'КбЦбйЕШмКтЩЕКдуЩтмпИЕВТЖл', 'ФЕаЧмАКчПСПквкхспТ', 'гжФеАЧмаКчпСп', 'ЧмАК', 'дцкДННМБцйЕгайхшжПГх', 'ТЩбвЦЖАНшрАШФДчОщй']) from system.numbers limit 10; -select [1, 6, 0, 1, 0, 0, 3, 1, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('МФННЧйОнцЛИЧЕПШПЧйоГФО'), ['', 'йОн', 'шУлгИЛЛРЙАсфЗоИЙЗРхуПбОЙсшдхо', 'МФННчЙоНц', 'лзВжбЦзфкзтуОйзуЗ', 'ЖГДщшЦзсжщцЦЖеЧвРфНИНОСАОщг', 'ННчйОНЦлИчЕПШ', '', 'Ф', 'ЩрИдНСлЙуАНЗвЕчмчАКмФУипндиП']) from system.numbers limit 10; -select [5, 0, 8, 13, 0, 0, 0, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('зВйймХЩМзЦГЕкЕКфоСтхПблуКМхц'), ['МХщмз', 'НАНрШоНДмурМлО', 'мзцгЕкек', 'кеКфоСтХПбЛУК', 'СУУксО', 'ЦоШжЧфйШЦаГЧйбЛШГЙггцРРчт', 'НбтвВбМ', '', 'тЩФкСтоСЧЦЦЙаСДЩСГЙГРИФЗОЗфбТДЙИб', 'ВГж']) from system.numbers limit 10; -select [0, 0, 0, 8, 19, 0, 3, 12, 1, 4] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ДпбЙЖНЗбПнЛбахБаХТуабШ'), ['цИаЩвгеИР', 'Ф', 'РЖиА', 'БпнЛб', 'У', 'Тфн', 'Б', 'БА', '', 'ЙЖНзБПнлбАхбаХ']) from system.numbers limit 10; -select [0, 0, 0, 0, 0, 1, 0, 17, 1, 0, 1, 1, 1, 11, 0, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ТЦмЩОинХзоДДпПНЩигрРщОзКц'), ['ЕжЙВпПл', 'ВКфКТ', 'ШкДсЖхшфоПИадУбхФЩБчОАкпУеБхи', 'НТЕЙОШЦЖоЩбзВзшс', 'учГгуКФзлУдНУУуПУлкаЦЕ', '', 'фАПМКуЧйБЧзСоЗргШДб', 'ИГРрщОзк', '', 'йупОМшУйзВиВрЛЩЕеЩмп', '', '', '', 'дДППнщИгРР', 'ШФвИЧакеЦвШ', 'ТцМЩоинхЗОДдппнЩ', 'мрОгЩшЩеЧ', 'еЖРиркуаОТсАолЩДББВАМБфРфпШшРРРм']) from system.numbers limit 10; -select [3, 0, 0, 0, 0, 0, 1, 0, 0, 14, 0, 1, 0, 1, 1, 1, 0, 7] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('аОкиЛгКйхаОГОУзЦЛрбцш'), ['кИЛГкйхАогоУЗЦл', 'щЧДпХИхбпсГвфДФХкчХ', 'ШвАмБЗлДОИПткжхФТФН', 'щфсхФмЦсЛеувЙО', 'лВУЖц', 'еИщРшозЖАдцтКииДУлДОУФв', 'а', 'ХгЦРШ', 'ФзрЖкРЗЩЧИеЧцКФИфЧЧжаооИФк', 'уЗ', 'фЦФдцжжМчЗЖлиСЧзлщжжЦт', '', 'МдхжизИХфвбМААрйФНХдЕжп', 'аОкиЛг', 'АОКИЛгкйХАОГОУЗЦ', '', 'МбЖйрсумщиеОЩк', 'КйХАоГоУЗцлРБЦШ']) from system.numbers limit 10; -select [0, 0, 2, 1, 0, 0, 12, 0, 17, 0, 0, 0, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('КУчЛХФчЛХшвбМЦинРвНрФМРкмиеЕп'), ['ТБЩБзхАмщПщЧПИФПашгЕТиКЦМБМпСЩСуЩМчтшеш', 'йлВЕЙшфшаШЗШЩВХЦчЛБс', 'УЧл', '', 'ЛДсЖщмНЦсКуфЗуГиука', 'РТТОТфГЕлЩЕгЛтДфлВЖШГзЦЖвнЗ', 'БМцИНРвнРф', 'ОЕИЕдИсАНаифТПмузЧчЖфШЕуеЩсслСШМоЖуЩЛМп', 'рвНРфМркМи', 'ЦзБМСиКчУжКУЩИИПУДвлбдБИОЙКТЛвтз', 'злСГе', 'ВдтцвОИРМЕжХО', 'учЛХфЧл', 'БшччШбУзЕТзфКпиШжнезвоеК']) from system.numbers limit 10; -select [0, 7, 0, 0, 0, 0, 7, 6, 0, 16, 12, 12, 15, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('оЖиогсфклШМСДрбхРбМбрЕщНЙЗйод'), ['иПмДКейууОклНХГЗсбаЙдШ', 'ФКлШмсДрБХРбМбрещНЙЗЙОд', 'арчжтСТнк', 'чбТНЛЕжооЗшзОУ', 'ощАЩучРСУгауДхГКлмОхЙцЕо', 'аЛбкиЦаКМбКхБМДнмФМкйРвРр', 'ФКлШмСДрбХРбм', 'СфклШ', 'еДйилкУлиИчХЙШтхцЗБУ', 'хрБ', 'СДрбХрбМБР', 'СдрбхРБ', 'бхрБМБРЕщНйз', 'КИб']) from system.numbers limit 10; -select [22, 1, 8, 0, 0, 1, 0, 3, 0, 6, 20, 0, 0, 0, 4, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЕЖДФбКужЙЦЦмсЖГГжБзеЙнПйЙри'), ['НПййР', '', 'Жй', 'Щ', 'ФхУО', 'ЕЖДфБКУЖйЦЦмСжГГ', 'НФЙзщЩГЧпфсфЦШОМЕЗгцрс', 'д', 'ЦтщДДЖтбвкгКонСк', 'кУЖЙЦЦм', 'ЕйНПййРИ', 'РчеЙйичФбдЦОтпчлТЖИлДучЙПгЗр', 'внчзшЗзОнФфхДгфзХТеНПШРшфБТЖДйф', 'кНснгмулМуГНурщЕББСузВмбнЧаХ', 'фбКУЖйЦцМсЖГгЖб', 'ЩСЕ']) from system.numbers limit 10; -select [0, 0, 0, 1, 10, 4, 0, 0, 5, 0, 1, 0, 7, 0, 3, 7, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чБхлжгКЖХлЙнкКЦфжЕгЖАндЧ'), ['ПдмРрЖАтВнСдСБШпПЗГгшИ', 'цшцг', 'тчАЙЧОеЕАвГпЗцЖЧгдХуЛСЛНрвАЖщ', '', 'Лй', 'Л', 'ОйррцУжчуЦБАжтшл', 'вХУКк', 'жгКжхЛЙН', 'уцбЕЕОЧГКУПуШХВЕчГБнт', '', 'ПсАжБИКштЕаН', 'КжхлЙН', 'ЩгШухЦПАТКежхгХксгокбщФЙПсдТНШФЦ', 'Х', 'кЖХЛйНккЦФжЕГЖ', 'ЙзРДСПднаСтбЧЖхощ', 'пАПОУЧмИпслБЗПфУ']) from system.numbers limit 10; -select [0, 0, 0, 5, 2, 16, 4, 4, 11, 0, 0, 3, 3, 0, 0, 6] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('кпМаоуГГфвощолЦЩщЧПРОКепеА'), ['ЗзуФжНшщПТнЧЦКВОиАУсЧХОШбк', 'тмПкАпеайзуХсурШй', 'АЕЦавбШиСДвВДумВкиИУБШЕ', 'о', 'ПМаОУггФВощоЛЦЩЩЧПрокЕПеа', 'щЩ', 'аоУг', 'аОуГгФВ', 'оЩоЛЦЩщчПРОК', 'виХЛшчБсщ', 'УчАМаЦкйДЦфКСмГУЧт', 'мАоУ', 'МАО', 'щФФА', 'Н', 'У']) from system.numbers limit 10; -select [0, 3, 10, 8, 3, 0, 4, 0, 9, 4, 1, 9] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('щЙЧРпшИцхпргЦНуДййусЧЧнЖ'), ['ДлУцтееЖБКХгМзСВжА', 'чРпШИЦ', 'пргЦнУДЙЙУ', 'Ц', 'ЧРПш', 'нЩрЕвмрМеРйхтшЩче', 'РпШИЦхПРГцнУд', 'ПНоЙтПкоаОКгПОМЦпДЛФЩДНКПбСгЗНЗ', 'ХПРГцНудЙЙ', 'рПши', '', 'ХПРГ']) from system.numbers limit 10; -select [11, 4, 1, 0, 1, 0, 0, 0, 0, 12, 0, 9, 5, 0, 16, 0, 12, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('пкзщщЛНОНбфЦноИЧфхбФ'), ['ф', 'щщл', 'ПКзЩщЛНОн', 'ЩшФйЧБНДОИзМхеЖНЦцеЛлУЧ', '', 'сЗоЙТклйДШкДИЗгЖ', 'орЛФХПвБбУхНс', 'доЗмЩВу', 'ШиЕ', 'ЦНО', 'ндЩдРУЖШМпнзНссЖШДЦФвпТмуМЙйцН', 'НбФЦнОИч', 'ЩлНонБФ', 'ЛдРжКММЙм', 'чфх', 'ЦматДйиСфЦфааЦо', 'ЦНОИчФх', 'иржЦщн']) from system.numbers limit 10; -select [0, 0, 0, 0, 0, 1, 0, 0, 0, 1, 0, 1, 0, 3, 0, 5] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чЖажцВбшЛттзДааАугШщАйПгщП'), ['ШгУТсчГОВЦЦеЛАСфдЗоЗЦВЛйлТДзчвЛва', 'УшЕшищЖткрвРСйиФЗйТФТЛЗаЗ', 'ВдикЙббщузоФХщХХГтЗоДпхбЕкМщц', 'срйеХ', 'рАшуПсЙоДнхчВкПЖ', '', 'гНЗбКРНСБВрАВФлнДШг', 'фХЧгмКнлПШлЩР', 'мкйЗбИФрЗахжгАдвЕ', 'чжаЖцВБШлТ', 'лХЕСрлПрОс', '', 'ЗЧПтчЙОцвОФУФО', 'ажцвБшЛТт', 'уНчЖШчМЕА', 'ц']) from system.numbers limit 10; -select [7, 1, 0, 7, 1, 19, 8, 6, 3, 0, 2, 13, 6, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('НТКПпмБжДцбАКПНСЖоиТФД'), ['б', '', 'аУщЛМХЖбвИтНчГБМГдДнч', 'Б', 'НТкппм', 'и', 'Жд', 'МБждЦбАкП', 'кппмБждцБа', 'ПЕрнЦпМЦВгЧЧгГ', 'ткПпМБЖДцбаКпнСжО', 'кПнСЖоИ', 'МБжДцБакпН', 'гхОХжГуОвШШАкфКМщсшФДШеИжоАйг']) from system.numbers limit 10; - -select 0 = multiSearchAny(materialize('mpnsguhwsitzvuleiwebwjfitmsg'), ['wbirxqoabpblrnvvmjizj', 'cfcxhuvrexyzyjsh', 'oldhtubemyuqlqbwvwwkwin', 'bumoozxdkjglzu', 'intxlfohlxmajjomw', 'dxkeghohv', 'arsvmwwkjeopnlwnan', 'ouugllgowpqtaxslcopkytbfhifaxbgt', 'hkedmjlbcrzvryaopjqdjjc', 'tbqkljywstuahzh', 'o', 'wowoclosyfcuwotmvjygzuzhrery', 'vpefjiffkhlggntcu', 'ytdixvasrorhripzfhjdmlhqksmctyycwp']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('qjjzqexjpgkglgxpzrbqbnskq'), ['vaiatcjacmlffdzsejpdareqzy', 'xspcfzdufkmecud', 'bcvtbuqtctq', 'nkcopwbfytgemkqcfnnno', 'dylxnzuyhq', 'tno', 'scukuhufly', 'cdyquzuqlptv', 'ohluyfeksyxepezdhqmtfmgkvzsyph', 'ualzwtahvqvtijwp', 'jg', 'gwbawqlngzcknzgtmlj', 'qimvjcgbkkp', 'eaedbcgyrdvv', 'qcwrncjoewwedyyewcdkh', 'uqcvhngoqngmitjfxpznqomertqnqcveoqk', 'ydrgjiankgygpm', 'axepgap']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('fdkmtqmxnegwvnjhghjq'), ['vynkybvdmhgeezybbdqfrukibisj', 'knazzamgjjpavwhvdkwigykh', 'peumnifrmdhhmrqqnemw', 'lmsnyvqoisinlaqobxojlwfbi', 'oqwfzs', 'dymudxxeodwjpgbibnkvr', 'vomtfsnizkplgzktqyoiw', 'yoyfuhlpgrzds', 'cefao', 'gi', 'srpgxfjwl', 'etsjusdeiwbfe', 'ikvtzdopxo', 'ljfkavrau', 'soqdhxtenfrkmeic', 'ktprjwfcelzbup', 'pcvuoddqwsaurcqdtjfnczekwni', 'agkqkqxkfbkfgyqliahsljim']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('khljxzxlpcrxpkrfybbfk'), ['', 'lpc', 'rxpkrfybb', 'crxp', '', 'pkr', 'jxzxlpcrxpkrf', '', 'xzxlpcr', 'xpk', 'fyb', 'xzxlpcrxpkrfybbfk', 'k', 'lpcrxp', 'ljxzxlpcr', 'r', 'pkr', 'fk']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('rbrizgjbigvzfnpgmpkqxoqxvdj'), ['ee', 'cohqnb', 'msol', 'yhlujcvhklnhuomy', 'ietn', 'vgmnlkcsybtokrepzrm', 'wspiryefojxysgrzsxyrluykxfnnbzdstcel', 'mxisnsivndbefqxwznimwgazuulupbaihavg', 'vpzdjvqqeizascxmzdhuq', 'pgvncohlxcqjhfkm', 'mbaypcnfapltsegquurahlsruqvipfhrhq', 'ioxjbcyyqujfveujfhnfdfokfcrlsincjbdt', 'cnvlujyowompdrqjwjx', 'wobwed', 'kdfhaoxiuifotmptcmdbk', 'leoamsnorcvtlmokdomkzuo', 'jjw', 'ogugysetxuqmvggneosbsfbonszepsatq']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('uymwxzyjbfegbhgswiqhinf'), ['lizxzbzlwljkr', 'ukxygktlpzuyijcqeqktxenlaqi', 'onperabgbdiafsxwbvpjtyt', 'xfqgoqvhqph', 'aflmcwabtwgmajmmqelxwkaolyyhmdlc', 'yfz', 'meffuiaicvwed', 'hhzvgmifzamgftkifaeowayjrnnzw', 'nwewybtajv', 'ectiye', 'epjeiljegmqqjncubj', 'zsjgftqjrn', 'pssng', 'raqoarfhdoeujulvqmdo']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('omgghgnzjmecpzqmtcvw'), ['fjhlzbszodmzavzg', 'gfofrnwrxprkfiokv', 'jmjiiqpgznlmyrxwewzqzbe', 'pkyrsqkltlmxr', 'crqgkgqkkyujcyoc', 'endagbcxwqhueczuasykmajfsvtcmh', 'xytmxtrnkdysuwltqomehddp', 'etmdxyyfotfyifwvbykghijvwv', 'mwqtgrncyhkfhjdg', 'iuvymofrqpp', 'pgllsdanlhzqhkstwsmzzftp', 'disjylcceufxtjdvhy']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('mznihnmshftvnmmhnrulizzpslq'), ['nrul', 'mshftvnmmhnr', 'z', 'mhnrulizzps', 'hftvnmmhnrul', 'ihnmshftvnmmhnrulizzp', 'izz', '', 'uli', 'nihnmshftvnmmhnru', 'hnrulizzp', 'nrulizz']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('ruqmqrsxrbftvruvahonradau'), ['uqmqrsxrbft', 'ftv', 'tvruvahonrad', 'mqrsxrbftvruvahon', 'rbftvruvah', 'qrsxrbftvru', 'o', 'ahonradau', 'a', 'ft', '', 'u', 'rsxrbftvruvahonradau', 'ruvahon', 'bftvruvahonradau', 'qrsxrbftvru', 't', 'vahonrada', 'vruvahonradau', 'onra']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('gpsevxtcoeexrltyzduyidmtzxf'), ['exrltyzduyid', 'vxtcoeexrltyz', 'xr', 'ltyzduyidmt', 'yzduy', 'exr', 'coeexrltyzduy', 'coeexrltyzduy', 'rlty', 'rltyzduyidm', 'exrltyz', 'xtcoeexrlty', 'vxtcoeexrltyzduyidm', '', 'coeexrl', 'sevxtcoeexrltyzdu', 'dmt', '']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('dyhycfhzyewaikgursyxfkuv'), ['sktnofpugrmyxmbizzrivmhn', 'fhlgadpoqcvktbfzncxbllvwutdawmw', 'eewzjpcgzrqmltbgmhafwlwqb', 'tpogbkyj', 'rtllntxjgkzs', 'mirbvsqexscnzglogigbujgdwjvcv', 'iktwpgjsakemewmahgqza', 'xgfvzkvqgiuoihjjnxwwpznxhz', 'nxaumpaknreklbwynvxdsmatjekdlxvklh', 'zadzwqhgfxqllihuudozxeixyokhny', 'tdqpgfpzexlkslodps', 'slztannufxaabqfcjyfquafgfhfb', 'xvjldhfuwurvkb', 'aecv', 'uycfsughpikqsbcmwvqygdyexkcykhbnau', 'jr']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('vbcsettndwuntnruiyclvvwoo'), ['dwuntnru', '', 'ttndwuntnruiyclvv', 'ntnr', 'nruiyclvvw', 'wo', '', 'bcsettndwuntnruiycl', 'yc', 'untnruiyclvvw', 'csettndwuntnr', 'ntnruiyclvvwo']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('pqqnugshlczcuxhpjxjbcnro'), ['dpeedqy', 'rtsc', 'jdgla', 'qkgudqjiyzvlvsj', 'xmfxawhijgxxtydbd', 'ebgzazqthb', 'wyrjhvhwzhmpybnylirrn', 'iviqbyuclayqketooztwegtkgwnsezfl', 'bhvidy', 'hijctxxweboq', 't', 'osnzfbziidteiaifgaanm']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('loqchlxspwuvvccucskuytr'), ['', 'k', 'qchlxspwu', 'u', 'hlxspwuvv', 'wuvvccucsku', 'vcc', 'uyt', 'uvv', 'spwu', 'ytr', 'wuvvccucs', 'xspwuv', 'lxspwuvvccuc', 'spwuvvccu', 'oqchlxspwuvvccucskuy']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('pjjyzupzwllshlnatiujmwvaofr'), ['lnatiujmwvao', '', 'zupzwllsh', 'nati', 'wllshl', 'hlnatiujmwv', 'mwvao', 'shlnat', 'ati', 'wllshlnatiujmwvao', 'wllshlnatiujmwvaofr', 'nat']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('iketunkleyaqaxdlocci'), ['nkleyaqaxd', 'etunkleyaq', 'yaqaxdlocci', 'tunkleyaq', 'eyaqaxdlocc', 'leyaq', 'nkleyaqaxdl', 'tunkleya', 'kleyaqa', 'etunkleya', 'leyaqa', 'dlo', 'yaqa', 'leyaqaxd', 'etunkleyaq', '']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('drqianqtangmgbdwruvblkqd'), ['wusajejyucamkyl', 'wsgibljugzrpkniliy', 'lhwqqiuafwffyersqjgjvvvfurx', 'jfokpzzxfdonelorqu', 'ccwkpcgac', 'jmyulqpndkmzbfztobwtm', 'rwrgfkccgxht', 'ggldjecrgbngkonphtcxrkcviujihidjx', 'spwweavbiokizv', 'lv', 'krb', 'vstnhvkbwlqbconaxgbfobqky', 'pvxwdc', 'thrl', 'ahsblffdveamceonqwrbeyxzccmux', 'yozji', 'oejtaxwmeovtqtz', 'zsnzznvqpxdvdxhznxrjn', 'hse', 'kcmkrccxmljzizracxwmpoaggywhdfpxkq']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('yasnpckniistxcejowfijjsvkdajz'), ['slkpxhtsmrtvtm', 'crsbq', 'rdeshtxbfrlfwpsqojassxmvlfbzefldavmgme', 'ipetilcbpsfroefkjirquciwtxhrimbmwnlyv', 'knjpwkmdwbvdbapuyqbtsw', 'horueidziztxovqhsicnklmharuxhtgrsr', 'ofohrgpz', 'oneqnwyevbaqsonrcpmxcynflojmsnix', 'shg', 'nglqzczevgevwawdfperpeytuodjlf']) from system.numbers limit 10; -select 0 = multiSearchAny(materialize('ueptpscfgxhplwsueckkxs'), ['ohhygchclbpcdwmftperprn', 'dvpjdqmqckekndvcerqrpkxen', 'lohhvarnmyi', 'zppd', 'qmqxgfewitsunbuhffozcpjtc', 'hsjbioisycsrawktqssjovkmltxodjgv', 'dbzuunwbkrtosyvctdujqtvaawfnvuq', 'gupbvpqthqxae', 'abjdmijaaiasnccgxttmqdsz', 'uccyumqoyqe', 'kxxliepyzlc', 'wbqcqtbyyjbqcgdbpkmzugksmcxhvr', 'piedxm', 'uncpphzoif', 'exkdankwck', 'qeitzozdrqopsergzr', 'hesgrhaftgesnzflrrtjdobxhbepjoas', 'wfpexx']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('ldrzgttlqaphekkkdukgngl'), ['gttlqaphekkkdukgn', 'ekkkd', 'gttlqaphe', 'qaphek', 'h', 'kdu', 'he', 'phek', '', 'drzgttlqaphekkkd']) from system.numbers limit 10; -select 1 = multiSearchAny(materialize('ololo'), ['ololo', 'ololo', 'ololo']); - -select 1 = multiSearchAnyUTF8(materialize('иечбпрхгебилцмпфвжцс'), ['лцмпфвж', 'ечбпрхгебилц', 'фвж', 'мпфвж', 'вжцс', 'пфвжцс', 'ц', 'чбпрхгебил', 'илцмп', 'фвж', 'ечбпрхгеби', '', 'б', 'хгеб', '', '', 'ил', 'ебилцмпфвжцс']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('змейдмоодкшуищвеишчддуцпх'), ['здсщесгдкзмчбжчщчиоо', 'чфззцмудщхтфрмсзрвшйщ', 'рлунбнзрфубуббдочтвлзмпгскузохк', 'ктзлебцам', 'вчспмж', 'нгкк', 'гпзйа', 'щпйкччнабакцтлапсбваихншхфридб', 'афсузжнайхфи', 'йрздеучфдбсвпжохрз', 'ошбечпзлг', 'полшхидфр']) from system.numbers limit 10; -select 1 = multiSearchAnyUTF8(materialize('лшнуухевгплвйужчошгнкнгбпщф'), ['гбпщф', 'б', 'ф', 'чошгнкнг', 'йужчо', 'гплвйужчошгнкн', 'бпщф', 'плвйужч', 'шгнкнг', 'хевгплвй', 'плвйужчошгн', 'вй', 'лвйужчошгнкнгбпщф', 'лвйужчошгнкн']) from system.numbers limit 10; -select 1 = multiSearchAnyUTF8(materialize('кцпгуоойвщталпобщафибирад'), ['ойвщталпобща', 'щта', 'пгуоойвщтал', 'ф', 'общ', 'цпгуоойвщталпобща', 'побщ', 'ф', 'цпгуоойвщталпобщафиб', 'побщаф', 'лпобщафи', 'цпгуоойвщталпобщафи', 'пгуоойвщталпобщаф', 'талпоб', 'уоойвщталпо', 'гуоойвщтал', 'уоойвщталп', 'щ', '', 'цпгуоойвщталпобщафибирад']) from system.numbers limit 10; -select 1 = multiSearchAnyUTF8(materialize('фвгйсеккзбщвфтмблщходео'), ['еккзбщвфтмблщходе', 'йсеккзбщвфтм', 'вфтмблщходео', 'вгйсеккзбщ', '', 'йсеккзбщвфт', 'бщвфтмблщход', 'ккзбщвфтмблщход', 'ккзбщвфтм', 'еккзбщвфтмблщходе', 'еккзбщвфтмблщх', 'вгйсеккзбщвф', 'оде', 'оде', '', 'бщвфтмблщх', 'б', 'йсеккзбщвфтмблщходео', 'вфтмблщ', 'кзбщ']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('хбаипфшнкнлтбшрскшщдувчтг'), ['хгшгднфуркшщвфгдглххс', 'цогчщки', 'тдмщшйзйхиквмб', 'етелфмшвмтзгеурнтбгчнщпмйпйжжциш', 'чсбк', 'ибащлшздеуревжйфуепфхкузбзао', 'дкмбщдсбжййсвгкхбхпшноншлщ', 'щхбеехнцегрфжжу', 'збфлпгсмащр', 'скчдигцнсзфрещйлвзнбнл', 'освзелагррдоортлрз', 'утхрч', 'йкбрвруенчччпшрнгмхобщимантешищщбж', 'жгивтеншхкцаргдасгирфанебкзаспбдшж', 'ййекжшщцщ', 'ефдсфбунйчдбуй', 'бвжцирзшмзщ', 'випжцщйзхнгахчсцвфгщзкдтвчйцемшлй', 'лдрфгвднеиопннтчсйффвлхемввег', 'бмтцжжеоебщупфчазпгхггцегнрутр']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('фбуоойпцщишщлбхчрсллзвг'), ['уччхщ', 'вщчсарфмйшгшпйфгмжугмщжкцщгйжзфл', 'кклл', 'лпнжирпсиуо', 'нчипзфщхнтштхйхщрпзитко', 'вйпсдергвцзсцсгмхпбз', 'чфщдфоилгцевпц', 'чааиае', 'чгингршжтчпу', 'щетбнгутшйсгмвмучдхстнбрптничихб']) from system.numbers limit 10; -select 1 = multiSearchAnyUTF8(materialize('лйвзжфснтлгбгцерлзсжфещ'), ['зсжф', '', 'бгц', 'зжфснтлгбгц', 'л', 'цер', 'жфснтлгбгц', 'тлгбг', 'це', 'гбгцерл', 'нтлгбгцерлзсж', 'жфещ', 'взжфснтлг', 'фснтлгбгцерлзсжфещ', 'нтлгбгцерлзсж', 'зжфснтлгбг', 'взжфснтлгбгцерлз', 'взжфснтлгбгце']) from system.numbers limit 10; -select 1 = multiSearchAnyUTF8(materialize('нфдцжбхуучеинивсжуеблмйрзцршз'), ['чеинивсжуеблм', 'жуебл', 'блмйрзцрш', 'цр', 'м', 'фдцжбхуучеинивсжуеблмйрзцр', 'нивсж', 'ивсжуеблмй', 'й', 'всжуеблмйрзцршз']) from system.numbers limit 10; -select 1 = multiSearchAnyUTF8(materialize('всщромуцйсхрпчщрхгбцмхшуиоб'), ['муцйсхрп', '', 'уцйсхрп', 'сщромуцйсхрпчщ', 'схрпчщр', 'сщромуцйсхрп', '', 'уцйсхрпчщрхгбцмх', '', 'цмхшуиоб', 'гбц', 'пчщр', 'цйсхрпчщр', 'омуцйсхрпч', 'схрпчщрхгбцм', 'йсхрпчщрхгбцм', '', 'пчщрхгбцм', 'уцйсхрпчщрхгбцмх', 'омуцйсхрпчщ']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('уузшсржоцчтсачтедебозцвчвс'), ['бомбсзхйхкх', 'отвгстзихфойукарацуздшгбщеховпзкй', 'мфнев', 'вйийшшггилцохнзбхрлхи', 'втинбтпсщрбевзуокб', 'оиойвулхкзлифкзиххт', 'зацччзвибшицщрзиптвицзхщхкбйгшфи', 'кнузршшднмвтощрцвтрулхцх', 'рчбкагчкпзжвтбажиабиркдсройцл', 'щргчкзожийтпдзфч', 'щбошгщзсжтнжцтлкщитеееигзцлцсмч', 'сцкк']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('щчбслгзвйдйжрнщчвфшй'), ['пдашзбалйнзвузкдвймц', 'щхтшйоч', 'фднвфигозржаз', 'рйфопхкшщвщдвл', 'цдкйхтусожпешпджпатфуиткп', 'щпбчсслгщййлвскшц', 'жпснс', 'уиицуувешвмчмиеднлекшснчлйц', 'пххаедштхмчщчбч', 'ичтмжз', 'лсбкчу', 'бгфдвпзрл', 'йицц', 'цфйвфлнвопкмщк', 'бгщцвбелхефв', 'мймсвзаелхнжйчохомлизенфш', 'трйднхндшсщмпвщомашчнгхд', 'жфцнифлгдзйе', 'зспкшщщенбцжгл', 'рщтб']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('шщпееасбтхогвгвцниуевисгшгбч'), ['гпа', 'стимсркзебхрвфпиемзчзу', 'нзгофухвекудблкадбшшусбеулрлмгфнйгиух', 'кфиашфобакщворувгвкчавфзшх', 'гфпгщгедкмтгрдодфпуйддхзчждихгрчтб', 'тцтжр', 'рцйна', 'йцбпбдрреаолг', 'житсфосшлтгсщдцидгсгфтвлз', 'жвтнжедцфцтхжчщч']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('вхкшгфпфмнщаохтмизпврйопцуйзмк'), ['дтчбкхащаткифружжейабфйкйтрскбощиеч', 'фтоуабхмдааиснрбраттклмйонлфна', 'цадзиднщймшкщолттпгщбх', 'кштбчжтждпкцнтщвмухнлби', 'микудпдпумцдцгфахгб', 'ирик', 'емлжухвмк', 'чгуросфйдцшигцхжрухжпшдкфгдклмдцнмодкп', 'ттбнллквдувтфжвчттжщажзчлнбждчщцонцлуж', 'елцофйамкхзегхклйгглаувфтуувее', 'двкзчсифвтекб', 'шсус']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('йхцглкцвзтшщочпзмнчтуеао'), ['йечдай', 'дащжщзлосмй', 'афуккгугаазшрчпцнхщцтмлфф', 'чфтфскрфйщк', 'жлччкцшнфижтехппафхвщфс', 'бзжчв', 'щкщймнкщлпедидсу', 'оцбажцзшзйпптгщтфекртдпдзшодвойвох', 'йжддбссерхичгнчлкидвгбдзуфембрц', 'ктщвшкрщмдшчогхфхусдотсщтцхтищ', 'пшстккамнбнардпзчлшечхундргтоегцзр', 'нсрнфузгжррчнжначучиелебрб', 'шгжмквршжтккднгаткзтпвкгзхшйр', 'змквцефтулхфохбнхбакдичудфмйчп']) from system.numbers limit 10; -select 1 = multiSearchAnyUTF8(materialize('шждйрчйавщбйфвмнжоржмвдфжх'), ['ор', '', 'йрчйавщбйфвмнжо', 'вщбйфвмнжорж', 'ждйрчйавщбйфвмнжорж', 'йавщбйф', 'дф', 'вщбйф', 'бйфвмнжорж', 'мнж']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('кдшнсйршгвлицбенйбцфрсаччетфм'), ['асмун', 'йогкдчодиф', 'лштйбжнзфкикмпбитжшгкбоослщгзнщо', 'улштжцисцажзчштгжтфффабйлофедуфме', 'дрпгкчджихшзммймиамзфнуиорлищзгйвху', 'йиоршнйоввквбдвдзасма', 'члмвасмфрхжсхрбцро', 'лшкизщушборшчшастйсцкжцбонсшейрщ', 'масдфкршлупасвйфщфважсуфсейшзлащхрж', 'дгхшщферодщцнйна', 'цзфзждбавкжрткст', 'рфбожзееаце', 'кошомвгпрщсдквазчавожпечдиуйлщадфкгфи', 'бшпхнхсгшикеавааизцсйажсдийаачбхч']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('хтиелйтарквурйлжпеегфш'), ['зпмйвзуднцпвжкбмйрпушдуавднвцх', 'фбссчгчвжакуагдвижйтщтшоабпхабжш', 'щхшибаскрщбшрндххщт', 'сммрсцзмптисвим', 'цсргщфж', 'восжбшйштезвлкммвдхд', 'вбсапкефецщжквплуо', 'даеуфчвеби', 'бтптлжпин', 'шчддтнсйкщйщ', 'фжхщецпзчбйкц', 'цсвфпздхрщхцбуцвтег']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('апрчвзфжмбутццрйщкар'), ['индхжз', 'жилцовщччгстби', 'ажс', 'фктйамйтаг', 'шммнзачггоннксцушпчн', 'чдлйтзтоцдгзццисц', 'пнбтувщцдсчнщмсакрлгфмгрй', 'овмсйнщзушвщгуитщрхвйодф', 'бзлштезвлаижхбмигйзалчолшеунлц', 'фкжпеввгшгащз', 'тменбщжмсхщсогттршгек', 'чап', 'х', 'шкомегурлнйпшбщглав']) from system.numbers limit 10; -select 0 = multiSearchAnyUTF8(materialize('двхопооллаеийтпцчфжштнргкк'), ['йймчнздешхбццбжибопгктрнркевпиз', 'фйрохсамщцнмф', 'ййхфдпецжзгнуорвбплоахрфиле', 'пкллкацнвдббогг', 'йщдезамтжйзихщжмцлх', 'гдзувмщиеулиддердшпитвд', 'фхтунйшзхтщжтзхгцорошднпбс', 'фнситбеелцдкйщойлатиуухгффдвищсше', 'нзщщщндцрнищпхйвтбвмцтнуадцбву', 'вбщкапшнв', 'зйлмуимчскщнивтшлчмуузщепшйр', 'шжбвйдр', 'гддждбкначдттфшжшхпфиклртпгм', 'еншащцфафчнгбнщххнзочбтпушщорегшцзб', 'уунеущкззоетбучкц', 'щасифзоажребийещ', 'пщбххсдгйтт', 'хшсчуотрт', 'жкднйрозбцшужчшбкккагрщчхат', 'шачефцгч']) from system.numbers limit 10; - -select 0 = multiSearchAnyCaseInsensitive(materialize('QWyWngrQGrDmZxgRnlOMYHBtuMW'), ['ZnvckNbkeVHnIBwAwpPZIr', 'NCzFhWQmOqIGQzMORw', 'tDYaxfQXWpKNLsawBUUOmik', 'IMveCViyAvmoTEQqmbcTbdfjULnnl', 'NRvsdotmmfwumsDpDtZU', 'mnqVnwWOvMiD', 'HXpHrMvGQpbuhVgnUkfFPqjpoRdhXBrFB', 'awtr', 'IMIdOmMHZccbOZHhWOKcKjkwwgkJSfxHDCzR', 'jPLISbIwWJEKPwgvajTxVLws', 'HBfRrzEC', 'VXsysGnAsFbqNOvIaR', 'upCaeaIOK', 'GUDFkrzBiqrbZVnS', 'MoCOePXRlVqCQpSCaIKpEXkH', 'rfF', 'fjhMEpySIpevBVWLOpqi', 'KdeskLSktU', 'vjUuNUlBEGkQyRuojZLyrmf', 'SvSxotkTKCeVzNICcSZLsScKsf']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('gcDqqBCNqhQgVVgsxMXkevYIAxNl'), ['BHnoKRqOoKgmOVkjtehGSsInDvavDWOhkKAUL', 'nYqpmKPTWGdnyMcg', 'TIplHzsSXUz', 'SiQwpQgEdZ', 'YoJTWBJgsbJvq', 'CwyazvXERUFMCJWhTjvltxFBkkvMwAysRLe', 'tXUxqmPbYFeLUlNrNlvKFKAwLhCXg', 'vUbNusJGlwsOyAqxPS', 'ME', 'ASUzpELipnYwAknh', 'VtTdMpsQALpibryKQfPBzDFNLz', 'KmujbORrULAYfSBDyYvA', 'BaLGNBliWdgmqnzUx', 'IzwKIbbSUiwhFQrujMgRcigX', 'pnS', 'UKSZbRGwGtFyLMSxcinKvBvaX']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('HCPOPUUEVVsuZDbyRnbowGuOMhQ'), ['UzDbYrNBoWgUo', '', 'pUUEVVsUzdByrNB', 'nBO', 'SUZdbYrNbOWgUoMH', 'pOpuUevVSUZDbYRnb', 'bowGUoMh', 'VsUZDbyrNbo', 'suzdBYrN', 'uueVvsUZDBYRnBoW', 'gUom', 'eVvsuzDBYRNBoWgUOM']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('RIDPJWYYSGBFWyXikHofbTcZAnj'), ['aFxQyVe', 'OcnZBgPsA', 'iBQaH', 'oesSvsWtgQprSSIPaDHdW', 'EfytiMfW', 'qHiFjeUvQRm', 'LfQkfmhTMUfoTOmGJUnJpevIoPpfpzMuKKjv', 'scYbCYNzJhEMMg', 'yTLwClSbqklywqDiSKmEdyfU', 'HYlGFMM', 'TMQhjOMTImXbCv', 'AVtzpxurFkmpVkddQANedlyVlQsCXWcRjEr']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('GEsmYgXgMWWYsdhZaVvikXZiN'), ['wySd', 'smYgxGMWWYsDHZ', 'vIk', 'smyGxgmwWysDHzAvvikxZi', 'WYsdHZAvVI', 'YGxGmwWYSDhzavvI', 'XzI', 'ySDhZAvvIK', '', 'myGXgmwWySdHz', 'MYGxgmwWySdHZaVvik', 'wYsDhzAvvikXz', 'wwYsdHzav', 'Z']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('XKCeCpxYeaYOWzIDcreyPWJWdrck'), ['tTRLUYJTkSWOabLJlIBshARIkwVRKemt', 'jQgn', 'wdNRsKIVunGlvwqkwn', 'BsbKGBJlkWQDBwqqeIjENvtkQue', 'yLuUru', 'zoLGzThznNmsitmJFIjQ', 'WFKnfdrnoxOWcXBqxkvqrFbahQx', 'QHbgRXcfuESPcMkwGJuDN', 'NPqfqLS', 'bi', 'HnccYFPObXjeGYtrmAEHDZQiXTvbNcOiesqRPS', 'KobVCJewfUsjBXDfgSnPxzeJhz', 'AqYNUPOYDZjwXx', 'xbZydBGZFFYFsFHwm']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('AnIhBNnXKYQwRSuSqrDCnI'), ['', 'HBNNxkyqWRS', 'xKyqwrSUSQR', 'yQwr', 'ihbnnxKYQWrsUS', 'bnnXkYqwrSuS', 'qWRs', 'nXKyqWRSUS', 'qrdcN', 'NiHBnNXkYQWrS', 'NnXkYQwRSUsqRDCn', 'rSusqRd']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('OySHBUpomaqcWHcHgyufm'), ['lihJlyBiOyyqzeveErImIJuJlfl', 'WyfAXSwZPcxOEDtiCGBJvkCHNnYfA', 'hZ', 'fDQzngAutwHSVeoGVihUyvHXmAE', 'aCpcZqWKdNqTdLwBnQENgQptIyRuOT', 'PFQVrlctEwb', 'ggpNUNnWqoubvmAFdjhLXzohmT', 'VFsfaLwcwNME', 'nHuIzNMciJjmK', 'OryyjtFfIaxViPXRyzKiMu', 'XufDMKXzqKjYynmmZzZHcDm', 'xWbDgq', 'ArElRZqdLQmN', 'obzvBzKQuJXZHMVmEBgFdnnQvtZSV', 'ZEHSnSmlbfsjc', 'gjmWPiLylEkYMTFCOVFB']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('NwMuwbdjhSYlzKoAZIceDx'), ['ZKOaZ', 'wBDJhsYlZKo', 'hSy', 'MUwbDjHsyl', 'sYlzK', 'ylZKOAZ', 'y', 'lZKoaZICEdX', 'azIce', 'djHSylZkoAzice', 'djHsYLZKoAzi', 'dJHSYlZK', 'muWbDJHsYLzKOaziC', 'zi']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('gtBXzVqRbepHJVsMocOxn'), ['DidFXiqhRVBCHBVklLHudA', 'yEhumIpaYXlj', 'iaEmViTRLPM', 'vTwKBlbpaJZGYGdMifOVd', 'zvgfzWeLsMQNLutdAdCeuAgEBhy', 'Ca', 'iHabiaRoIeiJgSx', 'EBfgrJnzHbuinysDBKc', 'kT', 'SGIT', 'BTRuKgHDuXMzxwwEgvE', 'OWJIeTLqLfaPT', 'BQM', 'yMimBqutKovoBIvMBok', 'zIBCYVNYAwu', 'EFDEFWGqvuxygsLszSwSiWYEqJu', 'QJDIXvPOYtvhPyfIKqebhTfL', 'ssALaXRxjguUIVKMCdWRPkivww']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('MowjvqBkjnVTelCcXpoSuUowuzF'), ['Su', 'vqBkJNvTelC', 'Elccxp', 'vtElc', 'JVqBkJnVTELCcxpOsU', 'OsUuOWUz', 'ElccxPoSU', 'wJVQbkJNVtElCC', 'xpOSUUo', 'VQbkJnvTELCCXp', '', 'TeLcCxPOsuuO']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('VfVQmlYIDdGBpRyfoeuLffUUpMordC'), ['vqMLyIddgBPrYFoEulFFu', 'lyIDdgBPrYFOeul', 'dGBPRYFOeUlffUupmOrD', 'OEulffU', 'pMordc', 'FVqmlyiDdgBpRyFoeUlFfuUpMOrD', 'PmO', 'o', 'YiDDgbPRYFOe', 'DGBPryfoeU', 'yIDdgbpRyFOeULfFU', 'lyIddgBPryfoeulfFuU', 'gbPrYfOeUlFfuupmO', 'yFoeULF']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('CdnrzjzmwtMMPLjgcXWsbtrBs'), ['RfgIUeerlPIozKpRQR', 'QRoYzjZlgngJxX', 'mEbqlBIzTQH', 'UmrfJxKyTllktPfyHA', 'ukoZeOPA', 'pbbRaUcJijcxt', 'Rg', 'lSBG', 'HvuwuiqVy', 'Fo', 'aGpUVjaFCrOwFCvjc', 'zKhfkgymcWmXdsSrqAHBnxJhvcpplgUecg', 'ioTdwUnrJBGUEESnxKuaRM', 'QciYRCjRDUxPkafN']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('miTQkQcxbKMwGOyzzRJpfXLyGx'), ['yMwgQQJkeshUugm', 'wGVe', 'XncShWqjp', 'KWjGQCOsfMKWRcgCfebkXZwZ', 'SFWbU', 'WdFDMIcfWeApTteNfcDsHIjEB', 'XRuUJznPOCQbK', 'tibBMGZHiIKVAKuUAIwuRAAfG', 'VVCqVGGObZLQsuqUjrXrsBSQJKChGpZxb', 'bWYAOLuwMcwWYeECkpVYLGeWHRrIp', 'SLzCgfkRWmZQQcQzP', 'VvfOhFBhfiVezUSPdIbr']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('KXoTIgVktxiXoEwfoLCENiEhz'), ['oLCENie', 'xix', 'en', 'IgvktxIXoEWFOLCEnieHz', 'xOEWFoL', 'LC', 'ktxIxoEwfolCenie', 'ce', 'oTIGvktXIXOE', 'eW', 'otigVKTXIXOEwFolC', 'E', 'CEni', 'gVKtxIxoEwfOLCENieh']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('DXKzSivrdLuBdCrEYfMEgPhOZ'), ['', 'sIVRDlUBdcr', 'luBDcrE', 'rDLUbDCreY', 'KzSiVRdLuBDCr', 'dcREYFme', 'lUbdCReyFMEgph', 'sivrDlubdCr', 'BdcreYfMEgP', 'ZSiVrdluBDCrEYfmegpHOZ']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('lTvINMXVojkokvNBXPZOm'), ['ZQOJMEJfrjm', 'vIpmXnGlmWze', 'wbdDKcjrrIzBHypzJU', 'omotHOYbZjWfyVNeNtyOsfXPALJG', 'SXxu', 'yZPDFsZq', 'OVYVWUjQDSQTKRgKoHSovXbROLRQ', 'RnXWZfZwHipewOJimTeRoNRYIdcZGzv', 'sizoEJibbfzwqFb', 'vgFmePQYlajiqSyBpvaKdmMYZohM', 'ENsFoFCxDQofsBSkLZRtOcJNU', 'nG']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('LsTqxiGRdvQClVNBCGMOUHOAmOqPEC'), ['NdFuUQEUWaxS', 'fdOHzUzineBDnWJJvhPNZgB', 'rYAWGIBPxOLrjuquqGjLLoIHrHqSFmjh', 'IVgYBJARY', 'ToivVgUJAxRJoCIFo', 'yQXGrRjhIqFtC', 'PNYdEPsWVqjZOhanGNAq', 'nrQIDDOfETr', 'usJcPtiHKhgKtYO', 'vPKqumGhPbmAJGAoiyZHJvNBd', 'eXINlP', 'WQeESQJcJJV']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitive(materialize('gRzzQYOwLNiDcMFjXzSFleV'), ['XZSfLe', 'wLnIdcMFjxZSf', 'F', 'm', 'Le', 'qYoWLNidcMFjXzsf', 'zqyoWlNIdcMFj', '', 'oWlnIDCMfJxzsfL', 'wlNIdCmfjXzS']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitive(materialize('cYnMXJMJCdibMXoUQHEw'), ['BFrGFZRgzwHGkUVbBiZMe', 'piORdVIWHMBsBDeJRLbGZAHGBrzNg', 'bmDePbTPnFQiCFfBJUxAEYNSbgrOoM', 'gtzeAGwqjFrasTQUgAscfcangexE', 'okLG', 'l', 'EBkkGYNZZURgFgJPlb', 'HDQVngp', 'vEHhtBqWhZHCOrqEKO', 'fgqdFc', 'COig', 'VftTpSXAmTmvnShHJqJTdEFcyKPUN', 'WDI', 'knBm']) from system.numbers limit 10; - -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('мтдчЛВЖАгвзщущвкфИКмТбжВ'), ['щУщвкФИкМ', 'чЛвжАГвЗЩуЩвКФикм', 'ДчлвЖАГвзЩУЩвКфИКМтБЖВ', 'ЖагвзщуЩВКФикМТБжВ', 'ВжагВзЩУ', 'гВЗщущвкфИКмТБж', 'ГвЗщ', 'щВкФикМТБЖВ', 'вЖАГВзщущ', 'взЩуЩвКФИкМТ', 'ЧЛВЖагвЗщуЩВк', 'тДчлвЖагвзЩуЩвкфИк', 'ТДЧлвжаГВзЩущВ', 'тДчлВжАГВЗЩУ']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('дтрцФхИнпиОШфдАгзктвбУвсб'), ['чТрВиУРФсРпДЩОащчзЦНцхИДА', 'ЗжмПВтмиойУГхАЦПиДУЦноНуййЩХаФТофшЩ', 'уБшлОЙцМПгетЖЧетШжу', 'ЧзИАУХобФрачТеХОШбМщЖСамиМВАКРщАЦ', 'ВйвТзхЙФЧоАЖвщиушАз', 'ЦшИфххкжиФйСЛЛНЛчВоЙВПпхиИ', 'ОатЕтщкЦпбСБйцОшГШРОшхБцщЙЧиУЩЕеФлщ', 'цСПпЧА', 'ШЧНфПмФсКМКДВЦАоФчОУеТЦИзЦ', 'зАбдЛНДГИ', 'фхЩлЗДНСсКЖИФлУАбЛеТФЕпЖлпПхЙиТЕ', 'иВшкНслТКМШЗиДПйфвйНкМЛхеФДзИм', 'лпушПБванпцев', 'ЧОшЧЧмшЦЛЙйГСДФйЛАв']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('нщМаНдЧЛмиВврПокПШмКБичкхвРГ'), ['АЙбаЙйШЛЙРЦмЗчВеИЕощсЦ', 'щЦФдВжчТСЩВКЦСпачЙсумщАтЩувеиниХПДоМС', 'иоАкДРршуойиЩищпрфВаЦПж', 'еЖПйШкГжЧтоГЙМВ', 'ЩПалиБ', 'ТвВлт', 'оХжйЛФеКчхЗВвЕ', 'ерцЩ', 'ШХЖОАрзеп', 'ККМрфктКГишпГЩхаллхДиВИИЛЗДеКйХмжШ']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('вШЙчоМгОттЧАЕнЧаВеЦщчЧошМУ'), ['ЧОмГотТчАЕН', 'ОмГотТчАЕнчАвецЩчч', 'ЧАВецЩч', 'ТЧАеНЧаВ', 'ттчаЕнча', 'ТчАЕ', 'мготтЧАенчавЕЦЩ', 'НЧаВец', 'тТЧаенчАвецщчЧошм', 'Ав', 'ТЧаЕнчавецщчЧоШму', 'аЕнЧав', 'АеНЧав', 'шйЧомГОТТчаЕнчАВЕ', 'шйчоМгОтТЧаЕНчаВеЦщЧчош', 'МУ', 'ошМ', 'гОТтЧаеНЧА']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('фйадзЧмщЖШйЖЛшцГигцШ'), ['НТХеМРшДНУЗгадцуЧИ', 'жпСИКЩМлНлиоктлЦИвНЛ', 'КхшКРчХ', 'кгТЗаШИарХЧЛЖмСЖм', 'ОмиЛй', 'жЕРбФЩНуЕКЕАВоБМОнАЕнКщшзйПкОЗ', 'гиЗдадкбжХМЗслшВИШай', 'двтЗйЙНгПуТзД', 'ТНкмаВЕФ', 'Шеа']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ШЕшхмеЦХеАСКощеКИфлсТЧИЗЛ'), ['КифЛсТ', 'ХеаСКощЕк', 'КифлсТЧ', 'шХМеЦхЕаскОЩеКИ', 'ЕшхмЕцХеаСК', 'ХЕасКоЩ', 'чИ', 'ЕцхеАсКОЩек', 'ЩЕкИфлс', 'асКощЕкифЛсТ']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('шоКнВЕрОЖЛпУйХзСугКПВжиРсЙпо'), ['игВербфНахчжЙггч', 'лтимрдфЕг', 'нкеаЖАшНБвйСдКИВГДшАГиАТнФШ', 'МжсТЙМГОииУКВГнцткДнцсоАд', 'ХтпгУСдБдцАЖЛАННоЕцзЕшштккз', 'ншУЦгФСЖшмс', 'нЩшМ', 'гоЖхМшаЕмаДРЧБЛИТпмЗОоД', 'фГКШхчФбЕГЛйкчПИЙххуМГНШзхг', 'ХпХщПЦАзщтг']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('кЧбоЗХфвБХхусмШгНаШШаГзш'), ['Ури', 'лММшткфНзцЦСВАдЩПМШфйОМшефигЖлуЕП', 'сМтЕдчЦафйСТЖЗфлРЙПЦдипжШскцВКХЦЖ', 'АУкжИФцшЛБЦЧм', 'ФПлнАаДСХзфоХПСБоСгМТОкЗЧйЛ', 'ЦшСГЛрцДмнНнХщивППттжв', 'жзЕгнциФ', 'МШЛсЙЧтЛАГжд', 'уИиЕжцоРНх', 'ЧбйГуХтшОНкрЧИеПД', 'ЦдЩЕкКвРЦжщЧциекЗРйхрббЖуЧ', 'иВжен', 'ГчОржвБГсжштРЕБ', 'ШоЖдуЙфчсЧегумщс', 'йчЙГ', 'РДедвТ']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ткРНбЩаРкгГчХшецИкНЕнСЖкйзАуУЖ'), ['ХлЖхУИллрРННйЗйсРуШЧвМбЧЧщфФЦц', 'СЛчКБцСФДшлфщаФлЙСзШабмбхуБжТСТ', 'УКУиввЗЩуВМцпчбпнДГбпЕЖрПбИДркМРОеЧмЧдГ', 'ПчщвШЩвГсЛмММГБ', 'хКЦЧсчжХЩИЖХеНнтоФЦлнмЛЧРФКпмСшгСЧДБ', 'удсЗйУДНЧУнтЕйЦЗЖзВСх', 'хПЖЙИрцхмУкКоСмГсвПаДОаЦНЖПп', 'сВОей', 'ЩЦжщоабнСгдчрХнЩиМХзжЩмФцррвД', 'ЦИсйнЦДоЕДглЕЦД', 'жзйПфБфУФоцзмКЩГПЧХхщщПТпдодмап', 'ДНХГНипжШлСхХхСнШЩЛИснУйЧЩЖДССФфиС', 'ОйЩНнйЕшцФчБГЛвхЖ', 'КЧРВшИуШйВфрпБНМсУмнСЦРпхЗАщЗУСвЧйБХтшХЧ', 'зЛбНу', 'ЗСрзпшЕйРржПСсФсШиМдйМЦГхдйтРКЩКНцкбмгС', 'СУццБуКнчОищГ', 'уЕГЧлЗБНпУисЕЛ']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ВЦХсЖЗЧЙБЗНбРитщстеМНжвВ'), ['итщст', 'ЧйБЗНбрИтщстЕМнЖ', 'ХСЖЗЧйбзНБриТщ', 'Темнж', 'сЖзЧЙБзнб', 'хСжЗчйБзнБрИтЩстЕм', 'БзнБРиТщ', 'ЗчЙбзНбрИТщ', 'чйбЗНбри', 'зЧйбзНБРИ', 'нБРитщсТе', 'зНб', 'цхСжзчйБЗнБРИТЩСтЕм', 'жЗЧЙБЗнбрит']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ХцМШКАБАОххЕижгГХЩГиНциД'), ['ОРАБЕРВомЛфГНМИКупбхЛаАкЗдМзтш', 'лЗУЩнлбмиЛАфсгМРкцВтлснййишИНАС', 'ТлжлУоУгжукФжЖва', 'жоСШПоУНЩшРМГшОЛзЦБЛиЛдТхПДнфжн', 'чнСУЗбДаГогжДфвШКеЙПБПутрРпсалцоБ', 'ЙозоПщчакщаАлРХбЦгац', 'иаИСсчЙЧБШорлгЧТнчцйзоВБХбхЙФтоЩ', 'ПСзсБЗЕщурфДЛХйГИеПНрмииаРнвСФч', 'ЦйЖЕуТфЖбхЩМтйсЙОгЛбхгтКЕЩСАЩ', 'гтЗуЩлужДУцФВПЛмрБТсСНпА', 'тГвлбчЗМасМЖхдЕгхмЩксоЩдрквук', 'ВРаг']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('тУйВЖдНнщцЗЖфКгфжГфиХСБЕЩ'), ['КгФЖГФи', 'сБе', 'ЖФ', 'гфжгФИхсбе', 'ВЖДНнщЦзжфКГфЖгфИхсбещ', 'ВЖДНнЩЦзжфкГ', 'вЖДННЩЦзжФКГфЖгФ', 'ф', 'НщЦЗж', 'нщЦЗЖФк', 'Их', 'дННщцзЖФКгф', '', 'нщцзжФкг']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ШкКРаоПеЗалРсТОиовРжгЙЧМКЛШ'), ['рчсажЕК', 'пЧТМфУрУММждЛйжзУрбкмам', 'бАШеНмВШзлзтушШШсхОсцрчЙПКИБнКжфЧЕХ', 'ЖМЛшбсУМкшфзочщАЖцМбмШСбВб', 'гтРХсщхАИОащчлИЧуйиСпСДФПбРл', 'ЧуОРУаоойГбУппМйЩФДКПВ', 'уУпугйРЕетвцБес', 'ЙЖЦТбСЖж', 'ИБКЛ', 'ТДтвОШСХГКУИПСмФМтНМзвбЦрднлхвДРсРФ', 'вВгНЙХИрвйЕЗпчРГЩ', 'ПчмТуивШб']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('РлчгхзуВШежХЦуМмнВЙщдцО'), ['ХшвМЦДШпЩОСшЦПдруа', 'ФИЦчУвРкпнПшИЕСЧАувиХд', 'фшвбЦОИЗфпИУМщзОЧЗфВцЙПнмтаТгг', 'мЖЩйавтнМСЛ', 'НВбШ', 'ааФДДрВвЙТдПд', 'ЗнчЧущшхЙС', 'рзуСзнеДфЩПуХЙЕл', 'ШСЩсАгдЦбНиШмшКрКс', 'ггнЕфБГзрОнАГЙзЧеИП', 'вшТИпЧдЖРкМНшзпиоиЩчзДмлШКТдпЦчж', 'фЦТЙц', 'ОтУшмбптТКЗеПлЧцЛОкЩБпккфгИн', 'ЩпвхпЗлШБЦ']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ЙбйнхНщЧЖщчГОАпчФнЛШФбгЛа'), ['щчг', '', 'апЧфНЛШфб', 'ЙнхНЩЧЖщчгОАПЧф', 'ХНщЧжЩЧгоАпч', 'ХНщЧжщчГо', 'нщЧжщчГОа', 'чЖЩЧГоапЧФНл', 'оапчФ', 'щЧГОАпЧФНлшФ', 'ЩЧГОАпЧФНЛшфБг', 'БЙНхнщчЖщчГоаПЧФНЛШФБгЛ', 'ОапЧфн', 'ф', 'БглА', 'ш', 'шфбГ', 'ХнЩЧЖщчГоА', 'ХНщчжщЧгоапч', 'хНЩчжщЧГоапчфнлшФбгЛ']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('кдЙДТЩеВЕфйКЩЦДиКБМф'), ['щфЛ', 'фЧЩЩичрКйЦКхеИИАпоБВЙЗбДАФио', 'мИтиЦРоВЙсБбСлНзиЛЧОфФевТмижщК', 'тЙгнКШфНТЕБЛцтГШЦхШхБ', 'уаабРГрМЙпМаБуЗпБЙчНивЦеДК', 'мпВЛНДеКПУгРЛЛинзуЕщиВШ', 'ЩжКйШшпгллщУ', 'пршЙПцхХЗжБС', 'нбЗНЙШБш', 'йцхИщиоцаМРсвнНфКБекзЛкчТ', 'хсмЦмнТрЩкДТЖиХщцкЦМх', 'ГмЛАбМщЗцЦйаОНвзуЗмЕКПБЙмАЕЛГ', 'ОЦХРЗРмкжмРИЖИЙ', 'з', 'лЕТкпкдЗчЗшжНфо', 'ИТПфйгЖЛзУТсЩ', 'ОфрбЛпГА', 'МЖооШпЦмсуГцАвМЕ']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ЩГТРШКИОРБРеНЖПКиуМОкхЛугИе'), ['брЕнЖ', 'РбрЕНЖпКиУМокХЛу', 'ГТрШКИорБРеНЖпКиУМ', 'рШКиоРбрЕнЖпкИУМОК', 'ИорбрЕнЖПК', 'Окхл', 'шкИоРБРеНЖПк', 'ТРШкИоРБрЕнжПКИУМОкхл', 'КИОРБРЕнжпкиУм', 'Н', 'КиОРбРЕнЖпкИУмоКхл', 'к', 'ГтРшКИоРБРЕнЖпк', 'гтрШкиорбрЕНЖпк']) from system.numbers limit 10; -select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ШНвпкфЗвгДжУЙГлрТШаШЛгНЗг'), ['нЗБенВшщрЛАрблцщшБАдзччммсцКЖ', 'бЗЩхзЗЗбФЕйМоазщугБбмМ', 'рЙсВжВсхдйлЩгБтХлчсщФ', 'пиБшКРнбВБгЕуЖ', 'жПшнхпШзУБрУЛРНЩДиаГШщКдЕвшоуПС', 'чЕщкЗмДуузуСдддзгКлИнгРмЙщВКТчхзЗЛ', 'кЖУЗЖС', 'щххОВМшуажвН', 'фбцЖМ', 'ДШитЧЩДсйНбдШеООУдг', 'ЛХПфБВХЦТИаФПЕвгкпкпщлхмЙхГбц', 'чЦсщЗщрМ']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ФРХгаСлчЧОцкШгзмКЗшФфББвЧ'), ['кзШфФб', 'ГАслЧЧОцкшг', 'ФфббВЧ', 'ЦкШ', '', 'АслчЧОЦКШгзМкЗШффбБвч', 'РХгаслЧчОЦКШГз', 'РхгаслчЧОцКШгзМкзшФфБбВ', 'Шг', 'Ф', 'ХГАслчЧоцКШГзМкзш', 'ШгЗмКЗшфФб']) from system.numbers limit 10; -select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ЧдйШкхОлалщНйбССХКаФзОМрКЕЙР'), ['бссХкафзОм', 'ХОЛаЛщнйБссХкаФз', 'лаЛщнйБсСХ', 'ЩнЙбСсхКаФЗО', 'йБСсХКАФЗОмР', 'йшкХолаЛЩНйбсСхК', 'С', '', 'ЙшкхОлалщНЙБсСхКаФзом', 'Йр', 'щнЙБссхКАфзоМрК', 'рКе']) from system.numbers limit 10; - -select 1 = multiSearchFirstIndex(materialize('alhpvldsiwsydwhfdasqju'), ['sydwh', 'dwh', 'dwhfdasqj', 'w', 'briozrtpq', 'fdasq', 'lnuvpuxdhhuxjbolw', 'vldsiws', 'dasqju', 'uancllygwoifwnnp', 'wfxputfnen', 'hzaclvjumecnmweungz']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('kcwchxxyujbhrxkxgnomg'), ['jmvqipszutxfnhdfaxqwoxcz', 'nrgzkbsakdtdiiyphozjoauyughyvlz', 'qbszx', 'sllthykcnttqecpequommemygee', 'bvsbdiufrrrjxaxzxgbd', 'hdkpcmpdyjildw', 'frxkyukiywngfcxfzwkcun', 'dmvxf', 'esamivybor', 'eoggdynqwlnlxr']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('zggbeilrfpkleafjjldgyfgn'), ['rpypxkpgvljhqthneremvabcd', 'qchzlnsctuwkdxqcrjgihvtfxhqxfqsxm', 'vtozkivjyqcqetmqenuihq', 'fixcvjyzbzejmwdivjf', 'lydoolvnuuamwlnzbyuuwpqqjaxf', 'elkodwthxqpcybwezm', 'wpiju', 'wdzuuwumlqfvga', 'iokphkai', 'wkbwdstplhivjyk', 'wxfbhfturuqoymwklohawgwltptytc', 'jehprkzofqvurepbvuwdqj']) from system.numbers limit 10; -select 9 = multiSearchFirstIndex(materialize('bwhfigqufrbwsrnnkjdzjhplfck'), ['v', 'ovusuizkdn', 'ttnsliwvxbvck', 'uh', 'lfourtjqblwdtvbgtbejkygkdurerqqdwm', 'snmtctvqmyyqiz', 'ckpixecvternrg', 'gluetlfyforxcygqnj', 'igqufrbwsr', 'om', 'huwazltjsnohsrcbfttzwyvcrobdixsuerkle', 'gqufrbwsrnnkjdzj', 'hfigqufrbwsrn', 'lhhyosbtznyeqzsddnqkfxayiyyajggxb', 'igqufrbwsrnnkjdzjhplf', 'pl', 'jtbqaqakbkesnazbvlaaojppxlbxccs', 'gqufrbwsrnnkjdz']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('yevfiumtjatfdnqixatbprvzwqlfgu'), ['ozghvskaixje', 'vmdrvdjhwxdvajmkpcxigsjzmtuhdxgllhzrpqd', 'qfhnxpcmtzpociajidwlcvobjfyxfcugsxy', 'pgamvhedjibcghinjrnowqzkfzibmfmh', 'bcmrdzpcczhquy', 'czosacvwfsbdvwwyirpvbve', 'qu', 'fdkobwlnmxbpvjkapextlbcrny', 'bqutjqobkyobhtpevjvewyksnoqyjunnnmtocr', 'kjlgff', 'oitltmhdburybwfxrjtxdiry', 'kiokuquyllpeagxygqugfmtm', 'wlbkl', 'khubpmstqjzzjzmsvfmrbmknykszqvue', 'lqrbmyndsztyrkcgqxcsnsanqjigimaxce', 'nitnyonuzedorrtkxhhgedohqcojbvtvjx']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('wmvuoeqphsycrvtxghrcozortmdnh'), ['hv', 'ugcmpebvlzgdtcmgkbgzyfel', 'qvmofayljsvybupvvnbhhibsz', 'zvlihxmyxlxwbffwjzjrfjgimmltftqqre', 'mwassqvxptav', 'jrumvqzkiaewngoufhrleakcfrsaxhpxyg', 'sxlxwhvkpavgfhxrxcbnqbstyrejtosxwe', 'psnlqakyfhcupryqatrmwqlswwjylpaiqammx', 'ivozojwldsgtnxpvsi', 'epyzjs', 'legi', 'sdqxxahfbddhacqrglgdcmlslraxfaahhfyodon']) from system.numbers limit 10; -select 12 = multiSearchFirstIndex(materialize('lebwdwxfdzwquhqhbvmte'), ['mwhruilzxvlyrgxivavxbbsq', 'ubuiizuasp', 'xpkzcsf', 'qpeqitoqqqeivohajzhmjbo', 'kbftixqmqgonemmbfpazcvf', 'iyhluioqs', 'hws', 'tupfdksgc', 'ows', 'pngzkoedabstewcdtdc', 'zdmyczldeftgdlwedcjfcoqycjcivf', '', 'xt', 'syuojejhbblohzwvjzzedzgmwc']) from system.numbers limit 10; -select 7 = multiSearchFirstIndex(materialize('wcrqaoecjwkhnskrbahqxfqgf'), ['qegldkdmyaznlmlhzvxfgoukngzbatnuq', 'khgcvgrifwtc', 'hkwcpogbbdqulizrycmneqmqynvj', 'zkqjf', 'xfduxyy', 'ructdekcoywfxsvpumfefoglljptsuwd', 'wkhnskrbahq', 'crqaoecjwkh', 'ikmpbunpguleinptzfelysiqc', 'lhldcci', 'nooepfypkoxxbriztycqam', 'uxeroptbiqrjartlnxzhhnlvjp']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('psgkkcwttitgrjsobiofheyohadu'), ['achfrepey', 'minlzeiwgjfvvmhnevisky', 'oxfghfdthtyczzveppcoxrued', 'ydhaupodnezvxhcqahfkwtpvxnymriixf', 'slxsbxidylxyurq', 'socyyabwbjdabnuqswrtjtqogirctqsk', 'lvbnacirctyxxspjmispi', 'oj', 'ihmmuuqlosorrwhfxvpygfrzsqpmilcvjodmcz', 'idmtmemqfyrlbwhxz', 'hsqfsfdzvslwbtlwrfavez', 'gszl', 'ei', 'pnywjnezncpjtyazuudpaxulyv', 'iqgavdjfqmxufapuziwwzkdmovdprlhfpl', 'yigk', 'mjidozklrpedutllijluv', 'vixwko']) from system.numbers limit 10; -select 3 = multiSearchFirstIndex(materialize('xtjxvytsseiqrpkbspwipjns'), ['bwmoghrdbaeybrmsnucbd', 'zoslqabihtlcqatlczbf', 'sseiqrpkbspwipjn', 'mdnbzcvtayycqfbycwum', 'npueimpsprhfdfnbtyzcogqsb', 'ytsseiqrpkbspwipj', 'fzvhcobygkwqohwutfyauwocwid', 'naacyhhkirpqlywrrpforhkcjrjsnz', 'vezbzderculzpmsehxqrkoihfoziaxhghh', 'mvvdfqzskcyomjbaxjfrtmbduvm', 'pwipjns', 'tsseiqrpkbspwipjn', 'sseiqrpkbspwip', 'qgrtbcdqcbybzevizw', 'isjouwql', 'rlbeidykltcyopzsfstukduxabothywwbq']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('zxmeusmehplcgbqabjof'), ['hqxgrw', 'fydjyrr', 'cocwtbazwjrswygttvrna', 'wpkvowuq', 'mwnzdxihrxihzhqtl', 'ljkjtmrfbonhqkioyzotyeegrw', 'ofxo', 'rjubwtpbweratrelqlrqotl', 'wvxkcil', 'qvolxxgqs', 'afqlhjnlvxowtnuuzywxuob', 'slwbmq']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('tjcmtoisgbilkygushkpuxklis'), ['bkdohwx', 'dfohgzhcjqirlbrokwy', 'zaemgqgxltznvkccyumhgsftnfigbol', 'otgcaybejwe', 'qn', 'gvfzcyhvmsnbgkulsqrzeekmjkc', 'cajuyauvmhkrriehgwfmtqbkupysudle', 'pmcupysyllzpstolkfpdvieffxaupqtjty', 'elhlzvescbfpayngnnalzixxgunqdhx', 'cvxpgdnqcxeesk', 'etlewyipypeiiowuoewulkpalvcfe', 'ordhwrkwqq', 'wnroixlkrqnydblfrtlbywc', 'xshujuttvcdxzbetuvifiqi', 'meqqxqhntkvzwoptnwskdgsxsgjdawe', 'dnmicrfshqnzosxhnrftxxeifoqlnfdhheg']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('iepqqbvekaflprupsmnpoijrld'), ['kqomoeysekwcplpegdwcdoeh', 'mwdvr', 'aobviioktzwzmpilblbdwstndhimabfgct', 'vqustluciruiyfkoontehnwylnauwpol', 'utcqnitztcgr', 'ityszrqmlwzspnrwdcvdhtziob', 'hmll', 'ilfzvuxbkyppwejtp', 'euxdzqcqutnfeiivw', 'rbcjlmjniiznzaktsuawnfjzqjri', 'fzyxlzzretsshklrkwru', 'jrujmdevqqojloz']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('cufztqffwjhtlkysekklpaywemm'), ['cpawuauqodogaitybtvplknjrsb', 'ynsocxfnxshzwnhlrfilynvz', 'ylrpytgcvtiumdckm', 'mvgrkueaslpgnjvvhzairgldtl', 'iliorsjypskmxfuuplfagktoycywb', 'drvwngp', 'zviuhcxaspwmqqz', 'qfgmrmhycskus', 'szj', 'rooivliiqufztcqlhrqyqvp', 'tufdmsmwue', 'cssowtldgwksbzlqyfereodcpuedighwd', 'odcjdffchhabtaxjvnr', 'o']) from system.numbers limit 10; -select 7 = multiSearchFirstIndex(materialize('zqwvlarwmhhtjjgwrivwfpsjkvx'), ['zcwhagxehtswbdkey', 'okezglmrjoim', 'ilwdviqimijzgoopmxdswouh', 'aqztpsntwjqpluygrvwdyz', 'uzxhjuhiwpz', 'akgc', 'larwmhhtjjgwrivwfpsj', 'isqghxsmcrwlgyloslmlyeboywtttgejdyma', 'arwmhhtjjgwri', 'rwmhhtjj']) from system.numbers limit 10; -select 9 = multiSearchFirstIndex(materialize('fuddujwwcewlhthgwsrn'), ['shtzrrtukxmdovtixf', 'rkcnzzzojqvvysm', 'jlamctgphjqcxlvmpzyxtghnoaq', 'pthrwvbheydmrot', 'kpniaqbcrgtxdyxxdxonbbltbdo', 'igulngxgtauumhckvbdt', 'khgrmskijoxruzzzaigjxonsc', 'rxzeykfxwssltw', 'hthg', '']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('jtgvvkggpkqhbxptjgoy'), ['nplzawmacgtqfxsp', 'oosw', 'akw', 'hnsenqoqwiydiufozomkyirgjepeqw', 'fpafgahvfdxukzvskbuy', 'tqimmsqffiqfoni', 'rrxkjklmkdhxqwcpfyutqzxu', 'esfqeujcbqxwnvodkwwdbsyozptaf', 'rqnyguyz', 'fftl', 'ccfyavxtxrpi', 'wftpsblszgovfgf']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('steccxkwnptybaddcuau'), ['qagxfznhjaxtyclxdsi', 'rtxwptfyzgthkwrx', 'rmcoxxs', 'vlubx', 'siecygstzivz', 'tksiagm', 'kq', 'dgsqrobxegmdbjkanb', 'lxokyvhveklvdakrxyiqokr', 'tgpmehwdrirpfjonqzhqshbo', 'cqmkargvsfjoxrguymtzsfwkg', 'avkmufhoywprjw', 'xzywtvlpoozmgkrcavevwebv', 'hfiuwslapamiceaouznxm', 'tmfjhqddafhhjbybfphlbwu', 'mrigvhmjvdpny']) from system.numbers limit 10; -select 0 = multiSearchFirstIndex(materialize('ccbgxzoivbqtmyzqyooyepnmwufizz'), ['lcclseplkhxbrrzlnani', 'xggxivwqlpxmpypzovprdkmhrcgjkro', 'dbbmiegotfxjxybs', 'hqtcowpupsyqfx', 'znatfzjbeevbaqbmpofhywbyfxn', 'mnditiygex', 'lazqapwjswhkuimwmjoyseyucllnrfxrwnzj', 'jg', 'dmqwnuvsufgffuubhqeugwcanvflseorrydyyxvr', 'wpjfcfwfgjiybncrw', 'joucnvxxcyjyqlwhrzwnstyj', 'babtxkzasyaffxzd', 'wgcfdyhwxjoytbxffdxbdfinolbltnhqkvyzybc', 'yhrgwbdwopznltjtyidxawqg', 'bvrrt', 'bcwmsys', 'ijdjojhhzaiyjyai', 'eevxwppogogdbmqpbeqtembiqxeiwf']) from system.numbers limit 10; -select 2 = multiSearchFirstIndex(materialize('xrwjeznohtbdvijwsbdksf'), ['hwdfufmoemohatqafdrcvdk', 'tbdvijwsbdks', 'xzwjczbuteujfjifzkbxvezs', 'bdvijwsbd', 'eznohtbdvijwsbdks', 'xadezwhbbmlqz', 'b', 'socrdjxsibkb', 'dk', 'eznohtbdvijws', 'pavsosnncajr', 'jixlmxxmxnnbpebjhitvtsaiwzmtqq', 'yuxmmnrqz', 'mpzytweuycabvu', 'tbdvi', 'ip']) from system.numbers limit 10; - -select 0 = multiSearchFirstIndexUTF8(materialize('црвтгмсрооацволепкщкпнгшкамщ'), ['гйцбсханрейщжнфбхтщбйала', 'дирдфнжпнччхаоцшрийнйнечллтгцбфедгсш', 'жфйндбффаилбндмлточиирасдзйлжбдзег', 'жвоуйфсйойфцвгзшцитсчпкч', 'ршонтбгщжооилчхрзшгсдцпзчесххцп', 'пйучихссгнхщлутвменлмм', 'хишгешегдефесо', 'знупгж', 'щчфу', 'знвтжифбнщсибеноожжметачаохфхсжосдзйуп', 'ггтоцйпгхчсбохлрчлваисивжбшбохдурввагш', 'щлийбчштбсч']) from system.numbers limit 10; -select 5 = multiSearchFirstIndexUTF8(materialize('опднхссгртрхтотлпагхжипхпитраб'), ['шфршсцешушклудефцугщцмйщлошечедзг', 'нйумйхфщцгщклдожхвосочжжислцрц', 'згтпвзцбхйптцбагсвцгтнф', 'пшичси', 'ссгртрхтотлпа', 'апзазогвсбежзрйгщоитмдкн', 'непгайтзкгштглхифмзданоихц', 'пднхссгртрхтотлпагхжипхпитр', 'ждднфлрзалшптсбтущвошрйтхкцнегшхрсв', 'брп', 'сгртрхтотлпагхжипх', 'нхссгртрхтотлпагхжипхп', 'пагхж', 'мфкжм']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('овччцнтчайомсйййоуйуучщххиффсб'), ['жжрддцпнехйр', 'шзбвуооинпаххесйкпкошжмцзгхе', 'ррсннилщлщжгцтйрпхабкехахззнтщемагдйшпсч', 'пуфугнказепщ', 'гддхтплвд', 'сщсчи', 'бйрсахедщфкхиевкетнс', 'йфжцжшпхлййхачзхнфоц', 'цтмтжлщдщофисзрвтбо', 'кщсевбоуйб', 'щгаапзкн', 'осймщовшчозцййизм', 'фкмаат', 'бкзцсдонфгттнфтаглпрцтбхбсок', 'жлмичлйнйсжбгсейбсиезщдмутационжгмзп', 'нбищижнлпмтморлхцхвеибщщлкйкндлтпбд']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('фдситчщдвхмфйтхшдтуцтщжрочщо'), ['ейшфдннтйечгк', 'фуйщгбйшдцирзб', 'ехйцмчщрсртнк', 'увтцмдорщжфгцгзущпувтщкнрфсйбщрзй', 'хчщпхвуарнббпзсцшчщуносйгщпсбтх', 'жтдчрхфмхцххккзппзбнуббс', 'тчохнмбаваошернеймгготлузвсбрщезднеил', 'стссчкшрчррйбхдуефвеепщшзмербгц', 'жбезжпещ', 'вйтсрхптлкшвавдаакгохжцоощд', 'искеубочвчмдхе', 'щмлочпзбунщнхлрдлщтбеощчшчхцелшоп', 'екуийтсйукцн', 'дочахгжошвшйжцпчзвжйкис', 'лтеенешпсболгчиожпжобка', 'букзппщрчбпшвпопвйцач']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('гопвмрутфпфбхмидшлуб'), ['цнхшдойгщн', 'дкаежщрапщпщеа', 'фмогимдничрфтхмсцмчпдфтиофнтйц', 'фчмсщисхщуп', 'ощмвдчефозйжбеесбмещочевцчд', 'апкбцйщжщабвппофм', 'мтйоддлфцгдуммптднпщшрн', 'икхнсмжчбхнфхнссгл', 'ущмунинлбпрман', 'ллкнечрезп', 'ажтнвбиччджсзтйешйффдгдрувер', 'йрщ', 'чигдкйшфщжужзлвщулквдфщхубги', 'иккшсмаеодейнкмгхбдлоижххдан']) from system.numbers limit 10; -select 12 = multiSearchFirstIndexUTF8(materialize('срлцчуийдлрзтейоцгиз'), ['жщлнвбубжпф', 'оклвцедмиср', 'нлзхмчдзрззегщ', 'хоу', 'шайиуд', 'ерслщтзцфзвмйтжвфеблщдурстмйжо', 'жмгуйузнчгтт', 'стеглмрдмирйрумилвшнзззр', 'втедлчрчайвщнллнцдмурутш', 'цимхргмрвмщиогврнпиччубцйе', 'ктчтцбснзцйцймридвш', 'ейоц']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('лрицжленфилзсжпжйнцжжупупдфз'), ['чпбрмлрнцмвеуфу', 'рмпизмпжчшбхдудчшохтжш', 'гргцжчпгщищннусв', 'ийщтщвзчшпдзитщубакусхавслрсбткб', 'бйбакижцтибгбгхжцвйчжжщжсжкзф', 'чгрп', 'чуносжусжфчмфжхрщзлщрдвбашажаанча', 'чекршбш', 'лбцкхйсооцц', 'сгвнлегвфмпчтййлрмд', 'наатущркхйимхщщг', 'щпзоеимфощулбзхафпц', 'дцабцхлврк', 'умидмчуегтхпу', 'дщнаойрмчсуффиббдйопдииуефосжхнлржрйлз', 'щзжетезвндхптпфлк', 'бгчемкццдбжп', 'иихуеоцедгрсеужрииомкбззцнгфифоаневц']) from system.numbers limit 10; -select 3 = multiSearchFirstIndexUTF8(materialize('бхжвчашрощбмсбущлхевозожзуцгбе'), ['амидхмуеийхрнчйейтущлуегрртщрхвг', 'фнисцщггбщйа', 'хжвчашрощбмсбу', 'фщвщцнеспдддцчччекчвеещ', 'ущуджсшежчелмкдмщхашв', 'цкуфбиз', 'евозожз', 'ппт', 'лвцнелшхцш', 'ощбмсбущлхев', 'ефхсзишшвтмцжнвклцуо', 'цржсржмчвмфмнеещхмиркчмцойвйц', 'ашрощбмсбущлхевозожзу', 'гхщншфрщзтнтжкмлщанв', '', 'хевозо', 'ощбмсбущлхевозожзуц', 'возожзуц']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('мзчатйжщгтзлвефчшмлшт'), ['гхшфрунирйдзтеафщгк', 'ймхмфлц', 'звуумивмвштчтнтеобзщесакийгк', 'чщжетзнцишхрммтбцакиббчп', 'блмидикавущщдпгпчхйаатйанд', 'цмщшбклгцгмчредмущаофпткеф', 'бнетввйцзпдерхщ', 'ицйнцрввемсвтштчфрпжнатаихцклкц', 'дзлщсштофвздтмчвсефишс', 'пбзртдцвгкглцфесидлвваисщр', 'ммеилбзфнчищч', 'жш', 'лздиззтпемкх', 'байлужднфугмкшгвгулффмщзхомпав', 'рсзнббедсчзущафббзбйоелид', 'цфшйкцксйгуйо']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('жжмзмащйфжщлрффбпврзнидииейщ'), ['ржфзнлйщсздйткаоцруйцгцт', 'илинксщмгщшещееифвпданмйлж', 'кг', 'гпааймцщпмсочтеиффосицхйпруйшнццвс', 'кнзфгжйирблщлл', 'ищуушфчорзлкбцппидчннцвхщщжййнкфтлрдчм', 'тбтдчлвцилргоргжсфбоо', 'ехаех', 'нехщмдлйджждмрцпйкбрнщсифхфщ', 'тцжпснйофцжфивзфбхзузщтмдкцжплавозмше']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('биаризлрвххжкпщтккучфизуршткпн'), ['йбручвндбщвссаеха', 'ол', 'еузкмпогщзгзафшдшоплбфнфдккх', 'ибзихщйфбтаз', 'ибрчиейш', 'нафрпбснзрузнтмнйиомтечтшзбкпзутдилтф', 'тщтбапцчдий', 'щкнггмфцжрзщцзжвлкчбммхтхтуж', 'ваам', 'цкфиушзигбжтацнчдлжжзфшщммтнлж', 'туфовжтнкзщсщщизмрйкхкпц', 'пирзксзикфтшодожшчцг', 'жфчфцфвлйбмеглжйдазгптзщгж', 'тутириждкзчвтсоажп', 'мотзусбхту', 'слщкгхжщфщоцкцтрлгп', 'бругтбфесвсшцхнтулк', 'восур', 'ссежгнггщдтишхйнн', 'вгзосзгоукмтубахжнзгшн']) from system.numbers limit 10; -select 8 = multiSearchFirstIndexUTF8(materialize('мчслвбжвманджййсикнврцдчмш'), ['рлбмй', 'иб', 'жажлцсзхйфдцудппефвжфк', 'огггхзгтцфслхацбщ', 'дзтцкогаибевсйещпг', 'зпцтйзфмвгщшуоилл', 'етщзгцпдйчзмфнхпфцен', 'нджййсик', 'сикнврцдчмш', 'жййсикн', 'икнврцдч', 'паокаочввеулщв', '', '', 'кечзсшип', 'вбжвманджййсикнвр']) from system.numbers limit 10; -select 2 = multiSearchFirstIndexUTF8(materialize('нвррммппогдйншбшнехнвлхм'), ['нфошцншблеооту', 'лх', 'цртд', 'огдйншбшн', 'уулддйдщицчпшбоиоцшй', '', 'дрдужзжпцкслетгвп', 'й', 'мппогдйншбшнех', 'дйншб', 'лжвофчзвдд', 'рммппогдйншб', 'ехнв', 'втущсщзбчсжцмаанчлнасп']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('удехбкабиацхпгзнхжелшц'), ['фмнбтйезсфоахофофдблкжщжфмгхтзс', 'тщтамзафозхлз', 'цшжфсбл', 'йзгзилупшллвипучхавшнмщафзмнк', 'лу', 'гтебпднцчвмктщсзи', 'лпщлмцийгуеджекшд', 'пцдхфоецфрунзм', 'зис', 'хпж', 'цтцплхцжишфнплуеохн', 'впх', 'чцчдацлуецрчцжижфиквтйийкез', 'гчшмекотд', 'пйгкцчафеавзихзтххтсмкал', 'сжфхпцгдфицжслрдчлдхлсувчнрогнву']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('щщвфчгамтжашнуошбзшуйчтшх'), ['дийу', 'жеомлсжщймемрсччошдфажцтдп', 'нгопнцквбф', 'хопб', 'ив', 'чвфвшфрдфелрдбтатшвейтг', 'вхкцадмупдчбаушшлдксйв', 'жтжбсвмшшсйеуфдпбдлкквдиовж', 'гтсдолснхесйцкйкмищгсзедх', 'ошплп', 'ифпуррикбопйгиччи', 'чдфймудаибвфчжтзглс', 'зпцмвпнлтунвйж', 'еждрйитхччещлцч', 'вмофсужхгрнзехкх', 'щжгквкрфжмжжсефпахст']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('рфгигуужжцфмоаешщечувщгонт'), ['слащченщлуоцргврбаб', 'тцизут', 'лйрсцолзклжбчрзгббммоищщ', 'уицмлоилзф', 'зпхмшвфйккфщщп', 'ймижрпдщмшв', 'пуощжлрмжлщхмкйгщшщивдпчпжчл', 'ойахшафнж', 'гксомбвцрсбжепхкхжхнсббци', 'панлраптщмцмйфебцщемйахенг', 'сохлгожштлднчсзпгтифсйгфмфп', 'аждчвзну', 'дхшуфд', 'борзизцхнийбщгхепрнзшй', 'фщшздруггрке', 'оевупрйщктнолшбкунзжху']) from system.numbers limit 10; -select 8 = multiSearchFirstIndexUTF8(materialize('кщзпапйднучлктхжслмищ'), ['апмдйлсафхугшдезксш', 'кйрм', 'цйивайчшуалгащсхйш', 'злорнмхекг', 'сгщврурфопжнлхкбилдч', 'бнлпщшнвубддрлижпайм', 'нукдонццнрмовфнбгзщсшщшдичежффе', 'йднучлктхжс', 'зпапйднучлктхж', 'затйотдсмпбевлжаиутсуг']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('жцажссефррршнфмнупщаоафгкщваа'), ['жфпщкгзкрмтщчцтжйчпйдошбшоцд', 'бхгйлйдробптвущшппзуиидежнлтпбжащткцф', 'хлещазйцепдханпажчизнхгншйуазщхй', 'ашцк', 'фрбммхдднчзшс', 'нжцанилзжаречвучозрущцдщаон', 'длмчзцрмжщбневрхуонпйейм', 'шкбщттврлпреабпоиожнууупшмкере', 'вуцпщдиифпеоурчвибойбпкпбкйбшхдбхнаббж', 'нртжвкдйтнлншцанцпугтогщгчигзтоищпм', 'цкплнкщлкшемощмстздхпацефогтск', 'цвждйбсмпгацфн', 'шсжшрзрардтпщлгчфздумупд', 'цйииткглчжйвуейеиииинврщу', 'унлодтулшпймашоквббчйнибтвалалрвбцж', 'нбнфнвйишйжлзхкахчмнлшзуеенк', 'бшлпсщжквпцахигчдтибкййб', 'фчакпзовтрлкншзцулшщмпзж']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexUTF8(materialize('иматеччдфлггшпучумджпфпзмвх'), ['дахахпчлцлаачгцгтфпнжлшчйуцбшсг', 'атжйувхец', 'грдсбвиднницдвшпйршгмегцаоопнжгй', 'чзлхречмктфащмтеечуиагоуб', 'савбхлпилийщтихутйчдгфсойй', 'вбгочбзистзщшденусцофит', 'мар', 'дфшажхдсри', 'тжлмщшж', 'птсрсщгшммв', 'ре', 'зратамкткфкинййй', 'гуцмсизулвазужфдмхнелфнжббдтрудчтнфцр', 'нйчинеучкхнпчгнйвчвсвлгминуцахгщввжц', 'ечагчнуулфббгбел', 'йшжуговрйкащцофдокфчушжктнптйеззушфо']) from system.numbers limit 10; -select 11 = multiSearchFirstIndexUTF8(materialize('азтммйтшхцхлгдрнтхфжбдрлцхщ'), ['нпучщфвспндщшспзмшочгсщжчйгжбжзжжтн', 'хккдйшабисдузфртнллщпбоуооврайцз', 'йпхрфжждгпнйаспйппвхбргшйвжччт', 'ффеее', 'кежцновв', 'еххрчштарзмкпйззсйлмплхбчбулзибвчбщ', 'шфжйдотрщттфхобббг', 'ожоцжущопгоцимсфчйщцддзнфи', 'цуимеимймкфччц', 'прммщмтбт', 'хцхлгдрнтхфж', 'лгд', 'цжбдаичхпщзцасбиршшикджцунйохдлхй', 'пидхцмхйнспйокнттмййвчщпхап', 'йтйзмеаизкшйошзвфучйирг', 'хцхлгдр']) from system.numbers limit 10; - -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('gyhTlBTDPlwbsznFtODVUzGJtq'), ['seSqNDSccPGLUJjb', 'xHvtZaHNEwtPVTRHuTPZDFERaTsDoSdX', 'QCeZOYqoYDU', 'bsybOMriWGxpwvJhbPfYR', 'FFHhlxfSLzMYwLPPz', 'tvDAJjaLNCCsLPbN', 'kOykGaSibakfHcr', 'mWAZaefkrIuYafkCDegF', 'ILrFDapnEDGCZWEQxSDHjWnjJmeMJlcMXh', 'zHvaaTgspUDUx', 'tss', 'laUe', 'euUKFLSUqGCjgj', 'Kd', 'MxyBG', 'qRXMsQbNsmFKbYSfEKieYGOxfVvSOuQZw', 'PdBrNIsprvTHfTuLgObTt', 'kMekbxI']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('ZxTznPEbfoBfLElYOrRiHrDLMmTpIh'), ['bJhYwKLeeLvLmXwWvQHWFkDQp', 'dLyZmUicTZmUfjfsFjxxgOiMJn', 'UCYbbGcY', 'kpPiwfWHEuh', 'jviwmHeiTQGxlTKGVEnse', 'cVnEyLFjKXiLebXjjVxvVeNzPPhizhAWnfCFr', 'gkcoAlFFA', 'ahZFvTJLErKpnnqesNYueUzI', 'VIJXPlFhp', 'rxWeMpmRFMZYwHnUP', 'iFwXBONeEUkQTxczRgm', 'ZnbOGKnoWh', 'SokGzZpkdaMe', 'EfKstISJNTmwrJAsxJoAqAzmZgGCzVRoC', 'HTmHWsY', 'CpRDbhLIroWakVkTQujcAJgrHHxc']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('VELfidaBvVtAghxjkrdZnG'), ['fvEFyRHvixuAYbuXygKeD', 'zFNHINreSOFksEGssBI', 'hcdWEcKDGWvfu', 'KczaFjvN', 'nZLTZAYSbfqcNWzWuGatDPUBYaRzuMBO', 'UdOdfdyPWPlUVeBzLRPMnqKLSuHvHgKX', 'DgVLuvxPhqRdSHVRSeoJwWeJQKQnKqFM', 'NNfgQylawNsoRJNpmFJVjAtoYy', 'tWFyALHEAyladtnPaTsmFJQfafkFjL', 'lYIXNiApypgtQuziDNKYfjwAqT', 'QjbTezRorweORubheFFrj', 'htIjVIFzLlMJDsPnBPF', 'ltDTemMQEgITf', 'fprmapUHaSQNLkRLWAfhOZNy', 'dOJMvPoNCUjEk', 'm', 'vEEXwfF', 'aVIsuUeKGAcmBcxOHubKuk']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('kOzLaInSCOFHikwfkXaBfkyjdQ'), ['t', 'emHGfAiZSkZaVTSfplxRiPoDZUTT', 'YHnGJDTzxsboDsLPGHChMHwrHHICBIs', 'gbcbVHSlVeVDOeILWtSLkKfVVjG', 'fPaJjbnNthEwWZyg', 'qS', 'PCQxoLaSdQOjioMKPglmoWR', 'KLMNszm', 'TCErEFyxOvqnHs', 'dRbGzEJqvIGAcilZoHlXtZpjmLLZfsYueKqo', 'iKHmNSbGgaJYJEdMkbobXTdlFgAGEJMQ', 'mUGB']) from system.numbers limit 10; -select 1 = multiSearchFirstIndexCaseInsensitive(materialize('JGcICnWOGwFmJzHjtGJM'), ['fmJzHj', 'LhGTreYju', 'yCELHyNLiAJENFOLKOeuvEPxDPUQj', 'kWqx', 'OBnNMuaeQWmZqjWvQI', 'ektduDXTNNeelv', 'J', 'iCNwoGwfMJzhjtGJ', 'uiIipgCRWeKm', 'bNIWEfWyZlLd']) from system.numbers limit 10; -select 7 = multiSearchFirstIndexCaseInsensitive(materialize('fsoSePRpplvNyBVQYjRFHHIh'), ['ZqGBzyQJYuhTupkOLLqgXdtIkhZx', 'pouH', 'mzCauXdgBdEpuzzFkfJ', 'uOrjMmsHkPpGAhjJwVOFw', 'KbKrrCJrTtiuu', 'jxbLtHIrwYXDERFHfMzVJxgUAofwUrB', 'PLvNyBVQYjRfhhi', 'wTPkeRGqqYiIxwExFu', 'PplvNybvqyJ', 'qOWuzwzvWrvzamVTPUZPMmZkIESq', 'ZDGM', 'nLyiGwqGIcr', 'GdaWtNcVvIYClQBiomWUrBNNKWV', 'QQxsPMoliytEtQ', 'TVarlkYnCsDWm', 'BvqYJr', 'YJr', 'sePrPPLVNYbvqYJRFhh', 'ybvq', 'VQYjrFHh']) from system.numbers limit 10; -select 3 = multiSearchFirstIndexCaseInsensitive(materialize('aliAsDgMSDPISdriLduBFnuWaaRej'), ['gWOFTxMrQGQaLrpJamvRhgeHwk', 'iWsBLzLycWvbJXBNlBazmJqxNlaPX', 'Ri', 'FPLRURSsjvsySncekcxaWQFGKn', 'wgXSTVzddtSGJQWxucYorRjnQQlJcd', 'wOLJWZcjHEatZWYfIwGIqnuzdcHKSFqfARfNLky', 'eEECZMNmWcoEnVeSrDNJxcOKDz', 'duBF', 'EhfLOjeEOQ', 'dUbFNUWA']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('EUzxPFYxMsJaTDzAKRXgZIVSFXU'), ['TDKAgICICjzBKHRqgFAuPCSODemldGGd', 'LvMluSJTIlgL', 'srbRhQKjPIchsipVHsjxwhK', 'vdurVsYkUWiFQVaDOnoNIJEX', 'UzZsZqAUNjMvWJaTqSWMHpzlDhVOaLzHPZfV', 'XcnnPXXEJJv', 'JSwFBNnYzNbIRZdeMfYiAfxzWfnCQFqoTUjns', 'HBMeqdLkrhebQeYfPzfJKAZgtuWHl', 'cMfSOnWgJvGhFPjgZdMBncnqdX', 'orDafpQXkrADEikyLVTHYmbVxtD', 'Vz', 'bfYwQkUC', 'q', 'YqomKpmYpHGv']) from system.numbers limit 10; -select 4 = multiSearchFirstIndexCaseInsensitive(materialize('mDFzyOuNsuOCSzyjWXxePRRIAHi'), ['TfejIlXcxqqoVmNHsOocEogH', 'clyblaTFmyY', 'JQfxMAWVnQDucIQ', 'jw', 'fGetlRA', 'uWwCOCd', 'rInhyxSIFiogdCCdTPqJNrqVaKIPWvLFI', 'mimSJjfCWI', 'jqnJvNZXMEPorpIxpWkhCoiGzlcfqRGyWxQL', 'bxCJeVlWhqGHoakarZcK', 'unsUOcSZyjwxxe', 'E', 'PR', 'nsUoCSZyjwxXEPr', 'sfotzRPMmalUSjHkZDDOzjens', 'zYJwxx', 'DFzyouNsUocsZ', 'QBaQfeznthSEMIPFwuvtolRzrXjjhpUY', 'sQPVBaoeYlUyZRHtapfGM', 'lPiZLi']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('VOAJfSkbDvNWAZNLIwqUgvBOddX'), ['pHrGGgJ', 'VohjtPdQZSNeKAlChDCnRTelroghFbZXVpnD', 'rnWebvdsmiFypMKL', 'NtKRiJOfAkWyKvubXrkOODgmZxvfOohsnHJEO', 'nxsDisKarasSZwESIInCJnYREUcoRUTXHBUH', 'mXYYr', 'jujScxeTBWujKhKyAswXPRszFcOKMSbk', 'INEegRWNgEoxqwNaGZV', 'VVyjMXVWVyuaOwiVnEsYN', 'mkLXSmXppxJhFsmH', 'pRVnBrWjqPeUDHvhVuDbzUgy', 'PzchFdPTkOCIVhCKml', 'KXaGWnzqoHBd', 'PhzQVqIOLleqDSYNHLjAceHLKYPhCVq', 'aixxTqAtOAOylYGSYwtMkZbrKGnQLVxnq', 'ruEiaxeRaOOXGggRSPlUOGWSjxh', 'prSULtHvDMw', 'vEpaIIDbGvIePYIHHZVNSPYJl']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('ZHcEinZEFtfmHBLuCHntUhbIgY'), ['GKElMPEtmkLl', 'mkrzzjSRfXThuCQHkbZxRbhcymzTxcn', 'PREwQjxBJkpkiyuYEvtMZNFELgbINWsgf', 'lFEGlPtaDJSyoXzwREiRfpzNpsaBYo', 'tmVTuLPhqhgnFNhHvqpmc', 'NtijVhVfAwpRsvkUTkhwxcHJ', 'O', 'FSweqlUXdDcrlT', 'uljEFtKVjIzAEUBUeKZXzCWmG', 'dBIsjfm', 'CNaZCAQdKGiRUDOGMtUvFigloLEUr', 'yWjizKZ', 'QqPVdyIFXcweHz', 'uPmgGWGjhzt']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('AYMpbVsUQqAfoaMiJcYsulujYoSIx'), ['aXECumHNmAEefHPJy', 'hTosrERBdVCIilCYcMdHwaRh', 'PVDBpwrc', 'uFvQRPePvmzmocOauvEjqoxMhytzOwPSOCjmtm', 'kQqIlSCHDmWXCKN', 'ybAHGYDEDvvOJsF', 'WpkANi', 'cFGuzEcdahZtTdLFNBrRW', 'EBaybUFxO', 'mRlZUzHzMsMAgvtRtATEDLQvXZnZHw', 'uqxckjqpCBHiLgSPRz', 'Lv', 'AJcRfAvBmQVMOjaFfMfNHJt', 'FYsPM', 'pkKXTPgijOHFclqgVq', 'Ck']) from system.numbers limit 10; -select 11 = multiSearchFirstIndexCaseInsensitive(materialize('gmKSXWkNhKckrVNgvwiP'), ['bdJMecfCwQlrsgxkqA', 'NTgcYkMNDnTiQj', 'fmRZvPRkvNFnamMxyseerPoNBa', 'rfcRLxKJIVkLaRiUSTqnKYUrH', 'YSUWAyEvbUHc', 'PridoKqGiaCKp', 'quwOidiRRFT', 'yHmxxUyeVwXKnuAofwYD', 'gichY', 'QlNKUQpsQPxAg', 'knhkCKRVNGvWIp', 'jAuJorWkuxaGcEvpkXpqetHnWToeEp', 'KnHKCKrvNgVW', 'tCvFhhhzqegmltWKea', 'luZUmrtKmmgasVXS', 'mageZacuFgxBOkBfHsfJVBeAFx', 'hKC', 'hkRCMCgJScJusY', 'MKSXWknHkckrVNgv', 'osbRPcYXDxgYjSodlMgV']) from system.numbers limit 10; -select 15 = multiSearchFirstIndexCaseInsensitive(materialize('lcXsRFUrGxroGIcpdeSJGiSseJldX'), ['pBYVjxNcQiyAFfzBvHYHhheAHZpeLcieaTu', 'SQSQp', 'OQePajOcTpkOhSKmoIKCAcUDRGsQFln', 'AYMDhpMbxWpBXytgWYXjq', 'gkUC', 'oWcNKfmSTwoWNxrfXjyMpst', 'fQSqkjRNiBGSfceVgJsxgZLSnUu', 'LRrhUjQstxBlmPWLGFMwbLCaBEkWdNJ', 'cZnaActZVoCZhffIMlkMbvbT', 'Uxg', 'vlKdriGMajSlGdmrwoAEBrdI', 'Fl', 'XzcNdlUJShjddbUQiRtR', 'AqowAuWqVQMppR', 'SRFUrGXrOgiCP', 'k']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitive(materialize('KhwhbOzWvobUwJcteCHguFCn'), ['LkDYrpvDfPL', 'CIaTaShobVIaWjdbsNsCMdZKlGdtWuJmn', 'zYcsxxFyfuGrPdTPgEvGbXoYy', 'vDIeYpJbLMGMuRkIrPkAnqDDkqXPzy', 'Ievib', 'CREiuEsErFgvGEkQzThHtYtPmcL', 'JjRWKyALtSkoGmRxh', 'JxPhpijkDOpncCKyDEyXvKNua', 'jo', 'mKpFscuBEABMAlQO', 'qiFTgJpcnUMRKzTEuKY', 'pXBtITxCPRaXijM', 'guYVLpIbu', 'tSKYIxv', 'oDnWaFAmsXGRdGvRPhbCIvFSFQNlSVYB', 'phdckINUiYL']) from system.numbers limit 10; -select 14 = multiSearchFirstIndexCaseInsensitive(materialize('pXFoUGwVTAItBqgbBaQwAqmeh'), ['LfBevBpGnaSlmGhbeZ', 'NtBYzEksiXvYI', 'jMeRw', 'omtaduY', 'BsWyvNdkfXsTBxf', 'CtoOIvaesuca', 'pgJcRIBVbyaPBgGsNKP', 'bAwdUMnwKvMXfFHQWrtfMeqcORIJH', 'GDxZblrqWSxUJFjEuXArPtfHPdwSNGGL', 'LLxcfp', 'NrLghkFpwCdvHJBfPBgiMatNRaDKjO', 'XCzr', 'cCojPpfLkGZnaWBGpaZvrGMwgHNF', 'BaQWAQmE', 'AQ', 'RtxxEZDfcEZAgURg']) from system.numbers limit 10; -select 5 = multiSearchFirstIndexCaseInsensitive(materialize('KoLaGGWMRbPbKNChdKPGuNCDKZtWRX'), ['FBmf', 'QJxevrlVWhTDAJetlGoEBZWYz', 'tKoWKKXBOATZukMuBEaYYBPHuyncskOZYD', 'kgjgTpaHXji', '', 'xOJWVRvQoAYNVSN', 'YApQjWJCFuusXpTLfmLPinKNEuqfYAz', 'GXGfZJxhHcChCaoLwNNocnCjtIuw', 'ZLBHIwyivzQDbGsmVNBFDpVaWkIDRqsl', 'Kp', 'EyrNtIFdsoUWqLcVOpuqJBdMQ', 'AggwmRBpbknCHdKPgun', 'xNlnPtyQsdqH', 'hDk']) from system.numbers limit 10; -select 6 = multiSearchFirstIndexCaseInsensitive(materialize('OlyNppgrtlubvhpJfxeWsRHpr'), ['slbiGvzIFnqPgKZbzuh', 'fakuDHZWkYbXycUwNWC', 'HnVViUypZxAsLJocdwFFPgTDIkI', 'bLx', 'fmXVYOINsdIMmTJAQYWbBAuX', 'pjFXews', 'BG', 'vrSQLb', 'ub', 'pREPyIjRhXGKZovTqlDyYIuoYHewBH', 'hnNQpJmOKnGMlVbkSOyJxoQMdbGhTAsQU', 'UwaNyOQuYpkE', 'yHNlFVnuOLUxqHyzAtNgNohLT', 'YJRazuUZkP', 'z', 'lUbVhpjFxEWsRhP']) from system.numbers limit 10; -select 6 = multiSearchFirstIndexCaseInsensitive(materialize('ryHzepjmzFdLkCcYqoFCgnJh'), ['cLwBRJmuspkoOgKwtLXLbKFsj', 'YSgEdzTdYTZAEtaoJpjyfwymbERCVvveR', 'RzdDRzKjPXQzberVJRry', 'HUitVdjGjxYwIaLozmnKcCpFOjotfpAy', 'LWqtEkIiSvufymDiYjwt', 'FDlKCCYqoFCGNj', 'jmZfdlKCcyQOFcGnJ', 'OZCPsxgxYHdhqlnPnfRVGOJRL', 'JfhoyhbUhmDrKtYjZDCDFDcdNs', 'KCCYqo', 'EPJMzFDLKcCYQ', 'zLQb', 'qsqFDGqVnDX', 'MzfdLkCCyQOFc']) from system.numbers limit 10; -select 5 = multiSearchFirstIndexCaseInsensitive(materialize('oQLuuhKsqjdTaZmMiThIJrtwSrFv'), ['MsfVCGMIlgwomkNhkKn', 'fBzcso', 'meOeEdkEbFjgyAaeQeuqZXFFXqIxBkLbYiPk', 'tNV', 'i', 'EwuTkQnYCWktMAIdZEeJkgl', '', 'hUo', 'dtAzmMITHijRtwsrFV', 'vhnipYCl', 'puor', 'TazMmiTh', 'ITHIJRTWSrf', 'luuHksqJDTaz', 'uHkSQjDtazMMiThIjrtwSRFV', 'gpWugfu', 'QjdtazmmIthIjRTWSRFV', 'ZdJpc']) from system.numbers limit 10; - -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ИпрУщйжббКВНИчйацпцоЛП'), ['цШСкЕвеГЕЗЦщруИБтЦсБГАу', 'Хнщта', 'БшА', 'СалШйР', 'ЩфДГРРчшБДММГЧоноЖСчдпВХшшгйН', 'бЕжПШЦддожнЧоЕишчшЕЙфСщиВПФМ', 'ТЗзГФх', 'Чфл', 'КнНкнЖЕкППварНрхдгЙкДешмСКИЛкеО', 'ЖИсЧПСФФМДиТШХЦфмЗУпфрУщСЛщсфмвШ', 'ллЙумпхчОсЦМщУ', 'ГМУНЦФшНУбРжоПвШШщлВФАтоРфИ', 'БХцжеНЗкжЗЗшЦзфгдЖОзЗЖщКМИШАтЦАп', 'мтСкЕнбХШнЛхХГР']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('цмйвГЖруДлдЦавхЖАлоЕхЗКд'), ['ХфБПМДВХЙ', 'саЗваАбднХбЦттмКсМбШбВМУйНКСЖжХЦНц', 'плиЩщШАцЖсхГ', 'ЗнУЕФЗВаНА', 'ЧДйСаЗГЕшойСжбсуЩуЩщбПР', 'ЧЕуЩкФБВвчмабШЦтЖбОрЗп', 'йХбМсрТАФм', 'РЖСЗвЦлНВПЧщГУцЖ', 'ГГлЩрОХКнШРТуДФ', 'шСабРжла', 'ЕчБвгаРЧифаЙщХПпГЦхчШ', 'дайшйцВНЩЧуцйдМХг', 'УнзНКЧххВрцЩМлАнЖСДОДцбИгЛЛР', 'сЛЗзПбиАгзК']) from system.numbers limit 10; -select 2 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('дфЧлзОжММФкЖгиЗЩлоШжФТкцк'), ['ЗРТцИрсФСбПрщГЗ', '', 'ЖГИЗщлОш', 'АДПН', '', 'чЛЗОЖмМфКжг', 'Мфкж', 'ндаовк', 'зГЛРГАНШмСмШМефазшеБкзДвЕШиЖСЗЧПИфо', 'ФЧЛзОЖммфКжгиЗЩ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ИИКДМЛхРчнвЙЕкВЧелТйЛВТ'), ['АчшОЛтНЙуЦЛЙфАКУйуТЗМеЗщОХТМЗеТА', 'НЦУДбчфРТОпЛкОгВпоО', 'неДавнНРеАУфТтфАнДчтнУМЛПШнроАчжш', 'бГржВПЧлЛтСВТтаМЦШШ', 'БщГщРнБхЕЛоЛсмЙцВЕГ', 'цбАжЦРеу', 'ХсЦРаНиН', 'нббДдВЗРС', 'змОПпеЛЖзушлнДЛфчЗлцЙЛфЖрЛКг', 'фШиЖСУоаНПйИВшшаоуЙУА', 'ЛктХиШРП', 'МапщВйцХч', 'жмУТкуГбУ', 'сйпзДЩоНдШЕТбПзФтсрмАФГСз', 'ЛБУвйладЕижрКзШУАгНЩчЕмАа', 'мЧпФлМчРбШРблмтмПМоС']) from system.numbers limit 10; -select 8 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ПоДУЗАтХншЦатИшХвмИЖчГнжчНцух'), ['жЛЧХзкжлиЛцЩбЧСнЛУжЖпКРвиСРН', 'шадмЩеУШБврУдЕБЗИгмЗЕФШчЦБСзПидтАлб', 'йпГмШСз', 'хЖФЙиПГЗЩавиЗЩйПнБЗЦЩмАЧ', 'ХесщтлбСИуЦ', 'вар', 'ЙкМаСхаЩаЗнФЩфКжПщб', 'ОдУзАТХншЦатИШхвМиЖчгнЖч', 'ЗВЗДБпФфцвжУКвНсбухссбЙКЙйккЛиим', 'гХхсГЛшдфЖЛбгчоЕмоЧр']) from system.numbers limit 10; -select 7 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ихзКЖЩсЧРСЖсЖжЛАшкТхИйТгМБпск'), ['ДРОБм', 'нз', 'тОЛ', 'щРзуЖрТ', 'Мдд', 'АЦГРК', 'Чрсжсжжл', 'чРсжсЖжл', 'ктхИйтГмБ', 'аАзЙддМДЦЩФкРТЧзЧПУойоТхБиЧПлХДв', 'иЙтгМбп', 'РицлПн', 'йДГнЧкЕв', 'ВМЩцАш', 'хКЩнДшуБЕЛТФГВгцБПРихШЙХгГД', 'иЙТГМ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('жггкщцзщшамдбРЗжйТзвхшАпХСбе'), ['лВТвтлРБжиЛЦвРЦкАЦаНБгуОН', 'рШаавцжзМрзВЧДРСузб', 'оемрЗМгФБНмжп', 'ЛбмХбФЧШГЛХИуТСрфхп', 'ЖшТдтЧйчМР', 'ЧнИМбфУпмЙлШЗТрТИкКИЩОЧеМщПЩлдБ', 'ГвРдПжГдБаснилз', 'уТнТчТРЗИЛ', 'ИТЕВ', 'дИСЖпПнПСНОвсЩЩшНтХЧшВ', 'штабтлМнсчРЗтфсТЩублЕЧйцеЦТтХ', 'ХбхгУШвАзкшЖ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('нсЩЙЕМмЧЛСйФцГВМиатГХш'), ['КсОПЧИкВсКшРхнкхБжду', 'мШмпТащжФ', 'ББЖнианЧЦпмрГЩГМаЛКжА', 'арИжзжфГТУДИРРРбцил', 'дфдмшМИщТиЗПруКфОнСЦ', 'Рцч', 'гмДгВДАтсщКЗлхвжЦУеФДАТГЙЦЧОЗвРш', 'чфХЩсДбУбВжАМшРлКРщв', 'нцБйсУ', 'фасДЕчвчДмбтЖХвоД', 'аБЧшЖшЖАКргОИшпШЧзТбтфйвкЕц', 'ЗжжсмкжЛд', 'щщлПзг', 'бП']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('сКиурчоиаЦйхгаУДПфчИтИК'), ['МЧПцУАМрХКЧмАЦннУшмРчкЖКХвху', 'РвДуВиашрРКкмжшЖНШБфлцжБЦР', 'йМУиУчНЧчРшДйБЗфЩЦйПсцгкДС', 'НсмаЛзЧвНЦШФуВРпзБГзйКцп', 'ЖлМЛУХОБллСЗСКвМКМдГчЩ', 'ЩХПШиобЛх', 'аФАЖВтРиЦнжбкСожУЖЙипм', 'аУГжУНуМУВФлж', 'ШБчтЗкЖНЙк', 'ЩоГПГчНП', 'мВЗйЛаХПоЕМХиИйДлшРгзугЙЖлнМппКЦ', 'вчмДФхНеЦйЗсЗйкфпОщПтШпспИМдГйВМх', 'ИЗИжЧжаГЩСуцСЩдкскздмЖЦ', 'дАмфЕбгс', 'ГМттнхчЩжМЧДфщШБкфчтЧ', 'ШЕииФБпщЙИДцРиЖжЩл', 'ОпуОлБ', 'хБ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('риШМбгиПЖннНоЧргзГзеДпЛиНт'), ['икДкбйдройВУсвФзрПСусДнАШо', 'чуУеТкУВФхз', 'ЕГпйчехЗвЛлБблЧПДм', 'зеоЩЧожКЛбШЩдАрКБНйшКВШаЗгПш', 'виФКуЗОтгВмТкБ', 'цДрЙгЗРаЧКаМДдБЕЧзСРщВФзПВЧГвЩрАУшс', 'мБЗИУдчХХжТж', 'ФТНМмгЖилуЛйМ', 'ЗегЩЦнЦщцИк', 'оГОусхФсДЖДЩИЕХЗпсПЩХБТГЕп', 'АУКНзАДНкусВЧХвАж', 'КвКрбсВлНАоЗсфХОйЦхТ', 'вФдеХацЧБкрхМЖЗЧчКшпфВчс', 'йХшиОвХЗжТпДТбвУрпшЕ']) from system.numbers limit 10; -select 11 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('МойрЙлтЖйБдББЛЕЕЦузЛфпИЕГт'), ['ПОжЦЩа', 'СШзЧФтСЗохЦЗдФтцНТу', 'вЕдТ', 'ечУФаМДнХщЕНУи', 'вмеосТзБАБуроЙУЛгФжДсЧщтчЕзлепгк', 'ИЧтБрцПмРаВрйИвНЛСйпЖжУВдНрурКшоКХП', 'ЕН', 'щКЦЩгФБСХпкпит', 'ей', 'ЕахшеОМРдЕГХуГЖчвКХМЕ', 'Гт', 'НужЛЛЙОАл']) from system.numbers limit 10; -select 11 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('еззЦАвУаДнзИКЙнЙдртРоП'), ['КгЩбшПЛКвтИРцйчккгЧчЧмтГ', 'кЛппСФщзМмТйВЕтбЩЦлО', 'ШпдзиЖх', 'иИХ', 'пУаАФгсмтофНФХиЦЕтТЗсОШЗЙ', 'фаКАБТцФМиКЖрИКшГБЗБ', 'идЖЙдЦММУнХЦЦфсФМ', 'МиЦечЖЦЙмРВЙОХсБРНнрлйЙшц', 'ТфдСтМгтмимТМАучтхПНЦлуф', 'бейККЛСггУЦБсокЕЙпнРЧ', 'цавУАДНЗИКЙнЙд', 'ЩйЕЖчЧщаПшжФсхХЛЕТчвмЙнуце', 'РТРОП', 'цАВуАДнзИкЙНЙдРтРо', 'аЩПИд', 'ОСчКшОАчВмр', '', 'уЙЛИуЕУвцДшНОгбТбИШв', 'АВУаднзИКЙНйдР', 'жТйоП']) from system.numbers limit 10; -select 12 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('шйМЦУщвфщшбмлТНВохСЖНУ'), ['хшТАпТоШхКНсДпвДЕчДМНбАНччд', 'ХКуПСтфСйРжмБглОШЙлйДкСФВйВ', 'хпмНЦМУШеАД', 'чзмЧВвлбЧкАщПкзТгеуГущб', 'шзжрДд', 'еЗГОЙНйИБЗДщИИНицмсЙЗгФУл', 'кнщЙхооДТООе', 'всзЙнТшжФЗДБДрщВДлбвулДИаз', 'мп', 'уБОйцзнМпИсксхефбдЕЛйгИмГШГЗЩ', 'ОМпзШШщчФФнвУЧгжчиндЧч', 'щВФЩШбмЛТн', 'бм', 'БпфнкнйЗцПдЧЩбВ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('НЗБлОбшмОПктткоччКиКрФсбкШАХ'), ['нффЕББУЖГшЖвГфЦФГЕСщсЩЧлфнАшшктизУ', 'нСмпцхшИои', 'ЧИчЗУтйЦхГезппФРХХШуцЗШВ', 'РИнщН', 'НЩдВТсЙсОдхРбМФнСпАбОПкудБФСчмб', 'йхглпдКтртгош', 'ибгУРАБцх', 'ИЕиЛрИДафмЗИкТвАуГчШугбЧмЛШщсОЧбБкП', 'ЩСМуХМ', 'АУсмдЗБвКфЩ', 'пгбТНОйц', 'МоИ', 'КОйкзОЕИЗМЩ', 'чщттЛРНнГхЗхХй', 'ЩшцЧРКмШЖЩЦемтЧУЛГкХтВНзОжУХТпН', 'ЕшбБНчрДпЩЧМлераУЖХйфйдчтсчПШ', 'дбФйтИАшДйЩтбФйШуПиРлГмВОШаСлШЧИвфЖщгж', 'ОДжТЦщпщИжфуеЩмн', 'ПЛНЕзжСчВКДттуФРУ', 'БбмеГЩХшжрцОжХНииВКВлдиХБДСмнНфХЛТХ']) from system.numbers limit 10; -select 4 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ЕКаЖСЗЗЕЗгПдШкфцЙТцл'), ['ЙКМИХРОХ', 'НвМУХзфчДбАРЙДу', 'чмщжФшшжсЗТв', 'жСЗзеЗг', 'ЛФсКзВСдЦД', 'АЖсЗЗЕЗГ', 'Пдшкфц', 'усйсКщшрДрвнФЛедуГХ', '', 'цйтЦ', 'Ощс', 'ЕЗГпдшКф', 'ззеЗгп', 'УгЛйхШТтшрЛ', 'ЗзЕЗгП', 'КЛмТЩРтрзБбЩРгФбиОБазУнтУЦ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('чЕжАфАрБпКбДмшАшТШККауЩИхНВО'), ['ЧЙпЗЧЧлйПЙЖЙшККг', 'зйхуМЩАИПГЗА', 'ЙцехноХниИбзБЧ', 'чВомЗОфУроС', 'дбРхХЗрзоДДШщЕДжиФаЙ', 'еЛзТцЩДиДГрдМОНЧУнеТуДЩЧЦпГЕщПОРсйпЧ', 'ФчнпМРЧцПЙЩЩвфДХПнУхцЩСИ', 'цлШеУкМБнжЧлУцСуСЙуотшМфйс', 'лугГлкщКщкзЛйпбдсишргДДшОувр', 'ЗРИаФЛЗФрСзм', 'аЗвжВгхЩоЦ', 'чГКлеБНДнИЖЧеШЧДнИвсГДЖЖфБМНсУЦосВс', 'щЦнПУзЧщнЩЕ', 'рВУв']) from system.numbers limit 10; -select 20 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('анктгЦВВкЧвЖиБпфТйлр'), ['НшДПчтсСЧпкидаХжаЙчаДчЦГшГ', 'ХнцЛШИрХВаРхнЧИЙрОЛЛИТпППфгЖЩФ', 'ФАЛущПупмдМБмтйзУшрВМзцзШжгД', 'ГчЛЧеЛДХеипдшЦЦмаШНаРшУТ', 'фОЕфжО', 'ТНсУАнчшУЛЦкцчЙ', 'ЛйЦКБЗГЦйКЩиОПуТЦкБкБувснЙи', 'Бунф', 'ИтХЛШСУНЦВйРСЙчДчНвйшЗЦй', 'АцСКнзБаЖУДЖегавйБгужШАДЙтжИВк', 'ЦцХщфирДПрСуХзхЖМЕщ', 'кфдБЖКншвУФкЗДКуЙ', 'СкиСЦЗЦРмгЦНпБхфХДЙщЛзХ', 'йУепВЖАПНбАЩуЛжвЧпхМ', 'БпЧшпДочУвибщерйхйтОБАСПнЧМИОЩ', 'чФгНЗщвхавбшсООоВштбЧ', 'уДиЕцнЙХВЕйИАГдЕ', 'тп', 'ЧЕРЖсгВ', 'вЖибПФТЙЛ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ипозйпхЛОЛТлСМХЩдМвМгШИвГиЛп'), ['ФСГзиГррБДНКГЛХбААФхИ', 'гегпАвхДЕ', 'ЦХжзщХИвхп', 'ЗЖ', 'ХОКцКзЩо', 'абИОрГПМТКшБ', 'кмХТмФихСЦсшУдхВбИШМНАНмпмХОЗйПЩч', 'еОжТСкфЕТУУжГ', 'НтщМЕПЧИКЙКйй', 'ежСикИвйЛж', 'ушЩФОтпБзЩЛЗЦЧЙиВгБЧоПХНгОуАДТЙж', 'фМЕРефнутпнцФРнрГЖ', 'хшДЧзнХпфорвЩжмГРЦуХГ', 'ЧЖн', 'вВзгОСхгНумм', 'ЗДоВлСжпфщСКсщХаолЛнЛЗбСхвЩвЩНоЩЩМ']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('МрЗтВФуЖРеЕШЧхПФбжжхчД'), ['щжОожЦндцШйТАй', 'йуРСЦУЗФУЦПвРфевСлфдРещЦтИтЩЩТг', 'ЕГЧдмХмРАлнЧ', 'йнкФизГСЗнуКбЙВЙчАТТрСхаЙШтсдгХ', 'ЧПрнРЖЙцХИщ', 'зЕ', 'СжВЩчГзБХбйТиклкдШШИееАлЧЩН', 'МШщГйБХжЙпйЕЗТзКмпе', 'НКбНщОМДзлдЧОс', 'НчзВХОпХХШМОХФумБгсрРЧИчВтгутВЩо']) from system.numbers limit 10; -select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('упТУЖелФкЧЧУЦРжоБтХсжКщД'), ['щКшуОЖааЖйнЕбДИжМК', 'ЕкнШцХРВтНйШоНбЙйУоЧщУиРпШЧхмКЧжх', 'рвЩЗоЗхшЗвлизкСзебЩКМКжбша', 'ДииБНСШвцЦбаСсИжЕЗмхмВ', 'СЦоБЗПМтмшрУлрДТФГЖиувШЗууШзв', 'ЦЗБЕзВХЙбйВОмЗпхндЗ', 'ЗНизЧВШкГВтпсЖж', 'уШиБПЙЧтРаЕгИ', 'ЙшпПА', 'ЧоММаАйМСфбхуФкефФштгУА']) from system.numbers limit 10; select 0 = multiSearchFirstPosition(materialize('abcdefgh'), ['z', 'pq']) from system.numbers limit 10; select 1 = multiSearchFirstPosition(materialize('abcdefgh'), ['a', 'b', 'c', 'd']) from system.numbers limit 10; @@ -830,7 +452,6 @@ select 1 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДе select 2 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДежз'), ['что', 'в', 'гдз', 'бвг']) from system.numbers limit 10; select 6 = multiSearchFirstPositionCaseInsensitiveUTF8(materialize('аБвгДежЗ'), ['З', 'бвгЯ', 'ЕЖз', 'з']) from system.numbers limit 10; --- 254 select [ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -913,135 +534,3 @@ select multiSearchFirstIndex(materialize('string'), 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'o', 'str']); -- { serverError 42 } - - -select [1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1]= multiSearchAllPositions(materialize('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'), -['aaaa', 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaab', -'aaaa']); diff --git a/tests/queries/0_stateless/02365_multisearch_random_tests.reference b/tests/queries/0_stateless/02365_multisearch_random_tests.reference new file mode 100644 index 00000000000..394c420ae8c --- /dev/null +++ b/tests/queries/0_stateless/02365_multisearch_random_tests.reference @@ -0,0 +1,3641 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02365_multisearch_random_tests.sql b/tests/queries/0_stateless/02365_multisearch_random_tests.sql new file mode 100644 index 00000000000..3243dd47689 --- /dev/null +++ b/tests/queries/0_stateless/02365_multisearch_random_tests.sql @@ -0,0 +1,379 @@ +SET send_logs_level = 'fatal'; + +select [4, 1, 1, 2, 6, 1, 1, 0, 4, 1, 14, 0, 10, 0, 16, 6] = multiSearchAllPositions(materialize('jmdqwjbrxlbatqeixknricfk'), ['qwjbrxlba', 'jmd', '', 'mdqwjbrxlbatqe', 'jbrxlbatqeixknric', 'jmdqwjbrxlbatqeixknri', '', 'fdtmnwtts', 'qwjbrxlba', '', 'qeixknricfk', 'hzjjgrnoilfkvzxaemzhf', 'lb', 'kamz', 'ixknr', 'jbrxlbatq']) from system.numbers limit 10; +select [0, 0, 0, 2, 3, 0, 1, 0, 5, 0, 0, 0, 11, 10, 6, 7] = multiSearchAllPositions(materialize('coxcctuehmzkbrsmodfvx'), ['bkhnp', 'nlypjvriuk', 'rkslxwfqjjivcwdexrdtvjdtvuu', 'oxcctuehm', 'xcctuehmzkbrsm', 'kfrieuocovykjmkwxbdlkgwctwvcuh', 'coxc', 'lbwvetgxyndxjqqwthtkgasbafii', 'ctuehmzkbrsmodfvx', 'obzldxjldxowk', 'ngfikgigeyll', 'wdaejjukowgvzijnw', 'zkbr', 'mzkb', 'tuehm', 'ue']) from system.numbers limit 10; +select [1, 1, 0, 0, 0, 1, 1, 1, 4, 0, 6, 6, 0, 10, 1, 5] = multiSearchAllPositions(materialize('mpswgtljbbrmivkcglamemayfn'), ['', 'm', 'saejhpnfgfq', 'rzanrkdssmmkanqjpfi', 'oputeneprgoowg', 'mp', '', '', 'wgtljbbrmivkcglamemay', 'cbpthtrgrmgfypizi', 'tl', 'tlj', 'xuhs', 'brmivkcglamemayfn', '', 'gtljb']) from system.numbers limit 10; +select [1, 0, 0, 8, 6, 0, 7, 1, 3, 0, 0, 0, 0, 12] = multiSearchAllPositions(materialize('arbphzbbecypbzsqsljurtddve'), ['arbphzb', 'mnrboimjfijnti', 'cikcrd', 'becypbz', 'z', 'uocmqgnczhdcrvtqrnaxdxjjlhakoszuwc', 'bbe', '', 'bp', 'yhltnexlpdijkdzt', 'jkwjmrckvgmccmmrolqvy', 'vdxmicjmfbtsbqqmqcgtnrvdgaucsgspwg', 'witlfqwvhmmyjrnrzttrikhhsrd', 'pbzsqsljurt']) from system.numbers limit 10; +select [7, 0, 0, 8, 0, 2, 0, 0, 6, 0, 2, 0, 3, 1] = multiSearchAllPositions(materialize('aizovxqpzcbbxuhwtiaaqhdqjdei'), ['qpzcbbxuhw', 'jugrpglqbm', 'dspwhzpyjohhtizegrnswhjfpdz', 'pzcbbxuh', 'vayzeszlycke', 'i', 'gvrontcpqavsjxtjwzgwxugiyhkhmhq', 'gyzmeroxztgaurmrqwtmsxcqnxaezuoapatvu', 'xqpzc', 'mjiswsvlvlpqrhhptqq', 'iz', 'hmzjxxfjsvcvdpqwtrdrp', 'zovxqpzcbbxuhwtia', 'ai']) from system.numbers limit 10; +select [0, 0, 0, 19, 14, 22, 10, 0, 0, 13, 0, 8] = multiSearchAllPositions(materialize('ydfgiluhyxwqdfiwtzobwzscyxhuov'), ['srsoubrgghleyheujsbwwwykerzlqphgejpxvog', 'axchkyleddjwkvbuyhmekpbbbztxdlm', 'zqodzvlkmfe', 'obwz', 'fi', 'zsc', 'xwq', 'pvmurvrd', 'uulcdtexckmrsokmgdpkstlkoavyrmxeaacvydxf', 'dfi', 'mxcngttujzgtlssrmluaflmjuv', 'hyxwqdfiwtzobwzscyxhu']) from system.numbers limit 10; +select [6, 1, 1, 0, 0, 5, 1, 0, 8, 0, 5, 0, 2, 12, 0, 15, 0, 0] = multiSearchAllPositions(materialize('pyepgwainvmwekwhhqxxvzdjw'), ['w', '', '', 'gvvkllofjnxvcu', 'kmwwhboplctvzazcyfpxhwtaddfnhekei', 'gwainv', 'pyepgwain', 'ekpnogkzzmbpfynsunwqp', 'invmwe', 'hrxpiplfplqjsstuybksuteoz', 'gwa', 'akfpyduqrwosxcbdemtxrxvundrgse', 'yepgwainvmw', 'wekwhhqxxvzdjw', 'fyimzvedmyriubgoznmcav', 'whhq', 'ozxowbwdqfisuupyzaqynoprgsjhkwlum', 'vpoufrofekajksdp']) from system.numbers limit 10; +select [0, 0, 5, 1, 1, 0, 15, 1, 5, 10, 4, 0, 1, 0, 3, 0, 0, 0] = multiSearchAllPositions(materialize('lqwahffxurkbhhzytequotkfk'), ['rwjqudpuaiufle', 'livwgbnflvy', 'hffxurkbhh', '', '', 'xcajwbqbttzfzfowjubmmgnmssat', 'zytequ', 'lq', 'h', 'rkbhh', 'a', 'immejthwgdr', '', 'llhhnlhcvnxxorzzjt', 'w', 'cvjynqxcivmmmvc', 'wexjomdcmursppjtsweybheyxzleuz', 'fzronsnddfxwlkkzidiknhpjipyrcrzel']) from system.numbers limit 10; +select [0, 1, 11, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4, 1] = multiSearchAllPositions(materialize('nkddriylnakicwgdwrfxpodqea'), ['izwdpgrgpmjlwkanjrffgela', '', 'kicw', 'hltmfymgmrjckdiylkzjlvvyuleksikdjrg', 'yigveskrbidknjxigwilmkgyizewikh', 'xyvzhsnqmuec', 'odcgzlavzrwesjks', 'oilvfgliktoujukpgzvhmokdgkssqgqot', 'llsfsurvimbahwqtbqbp', 'nxj', 'pimydixeobdxmdkvhcyzcgnbhzsydx', 'couzmvxedobuohibgxwoxvmpote', 'driylnakicwgdwrf', 'nkddr']) from system.numbers limit 10; +select [0, 0, 0, 3, 0, 15, 0, 0, 12, 7, 0, 0, 0, 0, 5, 0] = multiSearchAllPositions(materialize('jnckhtjqwycyihuejibqmddrdxe'), ['tajzx', 'vuddoylclxatcjvinusdwt', 'spxkhxvzsljkmnzpeubszjnhqczavgtqopxn', 'ckhtjqwycyi', 'xlbfzdxspldoes', 'u', 'czosfebeznt', 'gzhabdsuyreisxvyfrfrkq', 'yihuejibqmd', 'jqwycyihuejibqm', 'cfbvprgzx', 'hxu', 'vxbhrfpzacgd', 'afoaij', 'htjqwycyihu', 'httzbskqd']) from system.numbers limit 10; +select [0, 0, 12, 4, 4, 0, 13, 23, 0, 1, 0, 2, 0, 0, 0, 3, 0, 0] = multiSearchAllPositions(materialize('dzejajvpoojdkqbnayahygidyrjmb'), ['khwxxvtnqhobbvwgwkpusjlhlzifiuclycml', 'nzvuhtwdaivo', 'dkqbnayahygidyr', 'jajvpoo', 'j', 'wdtbvwmeqgyvetu', 'kqbn', 'idyrjmb', 'tsnxuxevsxrxpgpfdgrkhwqpkse', '', 'efsdgzuefhdzkmquxu', 'zejajvpoojdkqbnayahyg', 'ugwfuighbygrxyctop', 'fcbxzbdugc', 'dxmzzrcplob', 'ejaj', 'wmmupyxrylvawsyfccluiiene', 'ohzmsqhpzbafvbzqwzftbvftei']) from system.numbers limit 10; +select [6, 8, 1, 4, 0, 10, 0, 1, 14, 0, 1, 0, 5, 0, 0, 0, 0, 15, 0, 1] = multiSearchAllPositions(materialize('ffaujlverosspbzaqefjzql'), ['lvero', 'erossp', 'f', 'ujlverosspbz', 'btfimgklzzxlbkbuqyrmnud', 'osspb', 'muqexvtjuaar', 'f', 'bzaq', 'lprihswhwkdhqciqhfaowarn', 'ffaujlve', 'uhbbjrqjb', 'jlver', 'umucyhbbu', 'pjthtzmgxhvpbdphesnnztuu', 'xfqhfdfsbbazactpastzvzqudgk', 'lvovjfoatc', 'z', 'givejzhoqsd', '']) from system.numbers limit 10; +select [5, 7, 0, 1, 6, 0, 0, 1, 1, 2, 0, 1, 4, 2, 0, 6, 0, 0] = multiSearchAllPositions(materialize('hzftozkvquknsahhxefzg'), ['ozkvquknsahhxefzg', 'kv', 'lkdhmafrec', '', 'zkvquknsahh', 'xmjuizyconipirigdmhqclox', 'dqqwolnkkwbyyjicsoshidbay', '', '', 'zf', 'sonvmkapcjcakgpejvn', 'hzftoz', 't', 'zftozkvqukns', 'dyuqohvehxsvdzdlqzl', 'zkvquknsahhx', 'vueohmytvmglqwptfbhxffspf', 'ilkdurxg']) from system.numbers limit 10; +select [1, 7, 6, 4, 0, 1, 0, 0, 0, 9, 7, 1, 1, 0, 0, 0] = multiSearchAllPositions(materialize('aapdygjzrhskntrphianzjob'), ['', 'jz', 'gjzrh', 'dygjzrhskntrphia', 'qcnahphlxmdru', '', 'rnwvzdn', 'isbekwuivytqggsxniqojrvpwjdr', 'sstwvgyavbwxvjojrpg', 'rhskn', 'jzrhskntrp', '', '', 'toilvppgjizaxtidizgbgygubmob', 'vjwzwpvsklkxqgeqqmtssnhlmw', 'znvpjjlydvzhkt']) from system.numbers limit 10; +select [0, 1, 0, 1, 0, 0, 10, 0, 0, 0, 11, 0, 5, 0] = multiSearchAllPositions(materialize('blwpfdjjkxettfetdoxvxbyk'), ['wgylnwqcrojacofrcanjme', 'bl', 'qqcunzpvgi', '', 'ijemdmmdxkakrawwdqrjtrttig', 'qwkaifalc', 'xe', 'zqocnfuvzowuqkmwrfxw', 'xpaayeljvly', 'wvphqqhulpepjjjnxjfudfcomajc', 'ettfetdoxvx', 'ikablovwhnbohibbuhwjshhdemidgreqf', 'fdjjkxett', 'kiairehwbxveqkcfqhgopztgpatljgqp']) from system.numbers limit 10; +select [0, 0, 6, 1, 1, 0, 0, 1, 2, 0, 0, 0, 0, 0] = multiSearchAllPositions(materialize('vghzgedqpnqtvaoonwsz'), ['mfyndhucfpzjxzaezny', 'niejb', 'edqpnqt', '', 'v', 'kivdvealqadzdatziujdnvymmia', 'lvznmgwtlwevcxyfbkqc', 'vghzge', 'gh', 'tbzle', 'vjiqponbvgvguuhqdijbdeu', 'mshlyabasgukboknbqgmmmj', 'kjk', 'abkeftpnpvdkfyrxbrihyfxcfxablv']) from system.numbers limit 10; +select [0, 0, 0, 0, 9, 0, 7, 0, 9, 8, 0, 0] = multiSearchAllPositions(materialize('oaghnutqsqcnwvmzrnxgacsovxiko'), ['upien', 'moqszigvduvvwvmpemupvmmzctbrbtqggrk', 'igeiaccvxejtfvifrmimwpewllcggji', 'wnwjorpzgsqiociw', 'sq', 'rkysegpoej', 'tqsqcnwvmzrnxgacsovxiko', 'ioykypvfjufbicpyrpfuhugk', 's', 'qsqcnwvmzrnxgacsov', 'hhbeisvmpnkwmimgyfmybtljiu', 'kfozjowd']) from system.numbers limit 10; +select [0, 5, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 5, 0, 0, 14, 1, 1, 0, 0] = multiSearchAllPositions(materialize('wbjfsevqspsvbwlzrkhcfuhxddbq'), ['ltgjbz', 's', 'qdfnmggupdfxjfnmvwyrqopxtxf', 'sazlkmaikcltojbzbmdfddu', 'yzanifqxufyfwrxzkhngoxkrrph', 'iwskc', 'xkykshryphyfnwcnmjfqjrixykmzmwm', 'wwpenztbhkdbwidfkypqlxivsjs', 'rlkevy', 'qigywtkezwd', '', 'c', 'sevqspsvbwlzrk', 'gwg', 'iduhrjsrtodxdkjykjoghtjtvplrscitxnvt', 'wlzrkhcfuhxddb', '', 'wbjfsev', 'zytusrcvqbazb', 'tec']) from system.numbers limit 10; +select [0, 1, 5, 0, 6, 8, 0, 3, 2, 0, 0, 9, 0, 4, 0, 0] = multiSearchAllPositions(materialize('mxiifpzlovgfozpgirtio'), ['srullnscuzenzhp', '', 'f', 'apetxezid', 'pzlovgf', 'lo', 'ecbmso', 'i', 'xiifpzlovgfozpgir', 'bnefwypvctubvslsesnctqspdyctq', 'tdncmgbikboss', 'o', 'zmgobcarxlxaho', 'ifpzlovgfozpg', 'dwmjqyylvsxzfr', 'pxhrecconce']) from system.numbers limit 10; +select [0, 0, 0, 2, 0, 0, 2, 0, 8, 0, 0, 0, 7, 0, 0, 0, 21, 3, 1, 8] = multiSearchAllPositions(materialize('jtvnrdpdevgnzexqdrrxqgiujexhm'), ['ibkvzoqmiyfgfztupug', 'iqzeixfykxcghlbgsicxiywlurrgjsywwk', 'vzdffjzlqxgzdcrkgoro', 'tvnrdpdevgnzexqdr', 'nqywueahcmoojtyjlhfpysk', 'iqalixciiidvrtmpzozfb', 'tv', 'rxkfeasoff', 'devgnzexqdrrxqgiuj', 'kvvuvyplboowjrestyvdfrxdjjujvkxy', 'shkhpneekuyyqtxfxutvz', 'yy', 'pdevgnz', 'nplpydxiwnbvlhoorcmqkycqisi', 'jlkxplbftfkxqgnqnaw', 'qdggpjenbrwbjtorbi', 'qgiuje', 'vnrdpd', '', 'dev']) from system.numbers limit 10; +select [14, 0, 0, 7, 20, 6, 0, 13, 0, 0, 20, 0, 20, 2, 0, 8, 2, 11, 2, 0] = multiSearchAllPositions(materialize('asjwxabjrwgcdviokfaoqvqiafz'), ['v', 'zqngytligwwpzxhatyayvdnbbj', 'gjicovfzgbyagiirn', 'bjrwgcdviok', 'oqvqiafz', 'abjrwgc', 'wulrpfzh', 'dviokfao', 'esnchjuiufjadqmdtrpcd', 'tkodqzsjchpaftk', 'oqvq', 'eyoshlrlvmnqjmtmloryvg', 'oqv', 'sjwx', 'uokueelyytnoidplwmmox', 'jrwgcdviokfaoqvqiaf', 'sjwxabjrwgcdviokfaoqvqi', 'gcdviokfa', 'sjwxab', 'zneabsnfucjcwauxmudyxibnmxzfx']) from system.numbers limit 10; +select [0, 16, 8, 0, 10, 0, 0, 0, 0, 1, 0, 6, 0, 1, 0, 4, 0, 6, 0, 0] = multiSearchAllPositions(materialize('soxfqagiuhkaylzootfjy'), ['eveprzxphyenbrnnznpctvxn', 'oo', 'iuhka', 'ikutjhrnvzfb', 'h', 'duyvvjizristnkczgwj', 'ihfrp', 'afpyrlj', 'uonp', 'soxfqagiuhkaylzootfjy', 'qeckxkoxldpzzpmkbvcex', 'agiuhkaylzo', 'tckcumkbsgrgqjvtlijack', '', 'fnfweqlldcdnwfaohqohp', 'fqagiuhkayl', 'pqnvwprxwwrcjqvfsbfimwye', 'agi', 'ta', 'r']) from system.numbers limit 10; +select [3, 7, 1, 6, 0, 1, 0, 11, 0, 9, 17, 1, 18, 12] = multiSearchAllPositions(materialize('ladbcypcbcxahmujwezkvweud'), ['db', 'pcbcxahm', 'lad', 'ypcb', 'atevkzyyxhphtuekymhh', 'lad', 'mltjrwaibetrtwpfa', 'xahmujwezkvweud', 'dg', 'bcxahmujw', 'we', '', 'e', 'ahmujwezkvw']) from system.numbers limit 10; +select [6, 0, 11, 0, 7, 0, 0, 0, 6, 1, 0, 3, 0, 0, 0, 0] = multiSearchAllPositions(materialize('hhkscgmqzmuwltmrhtxnnzsxl'), ['gmqzmuwltmrh', 'qtescwjubeqhurqoqfjauwxdoc', 'uwltmrh', 'qlhyfuspwdtecdbrmrqcnxghhlnbmzs', 'm', 'kcsuocwokvohnqonnfzmeiqtomehksehwc', 'hoxocyilgrxxoek', 'nisnlmbdczjsiw', 'gmqz', '', 'cqzz', 'k', 'utxctwtzelxmtioyqshxedecih', 'ifsmsljxzkyuigdtunwk', 'ojxvxwdosaqjhrnjwisss', 'dz']) from system.numbers limit 10; +select [0, 0, 19, 7, 0, 0, 1, 0, 0, 12, 0, 0, 1, 0, 1, 1, 5, 0, 23, 8] = multiSearchAllPositions(materialize('raxgcqizulxfwivauupqnofbijxfr'), ['sxvhaxlrpviwuinrcebtfepxxkhxxgqu', 'cuodfevkpszuimhymxypktdvicmyxm', 'pqnof', 'i', 'ufpljiniflkctwkwcrsbdhvrvkizticpqkgvq', 'osojyhejhrlhjvqrtobwthjgw', '', 'anzlevtxre', 'ufnpkjvgidirrnpvbsndfnovebdily', 'fwivauupqnofbi', 'rywyadwcvk', 'ltnlhftdfefmkenadahcpxw', '', 'xryluzlhnsqk', 'r', '', 'cqizulxfwivauupqnofb', 'y', 'fb', 'zulxfwivauupqnofbijxf']) from system.numbers limit 10; +select [4, 0, 0, 0, 0, 24, 1, 2, 0, 2, 0, 0, 8, 0] = multiSearchAllPositions(materialize('cwcqyjjodlepauupgobsgrzdvii'), ['q', 'yjppewylsqbnjwnhokzqtauggsjhhhkkkqsy', 'uutltzhjtc', 'pkmuptmzzeqhichaikwbggronli', 'erzgcuxnec', 'dvii', '', 'w', 'fkmpha', 'wcqyjjodlepauupgobsgrz', 'cbnmwirigaf', 'fcumlot', 'odlepauu', 'lthautlklktfukpt']) from system.numbers limit 10; +select [1, 1, 1, 1, 22, 0, 0, 8, 18, 15] = multiSearchAllPositions(materialize('vpscxxibyhvtmrdzrocvdngpb'), ['', '', '', '', 'n', 'agrahemfuhmftacvpnaxkx', 'dqqwvfsrqv', 'byhvtmrdzrocv', 'ocvdn', 'dzrocvdngpb']) from system.numbers limit 10; +select [1, 1, 1, 15, 10, 0, 0, 0, 0, 2] = multiSearchAllPositions(materialize('nfoievsrpvheprosjdsoiz'), ['', 'nfo', '', 'osjd', 'vheprosjdsoiz', 'az', 'blhvdycvjnxaipvxybs', 'umgxmpkvuvuvdaczkz', 'gfspmnzidixcjgjw', 'f']) from system.numbers limit 10; +select [0, 0, 2, 2, 0, 0, 0, 11, 10, 4, 9, 1, 6, 4, 0, 0] = multiSearchAllPositions(materialize('bdmfwdisdlgbcidshnhautsye'), ['uxdceftnmnqpveljer', 'xdnh', 'dmf', 'dmfwdisdlgbc', 'cpwnaijpkpyjgaq', 'doquvlrzhusjbxyqcqxvwr', 'llppnnmtqggyfoxtawnngsiiunvjjxxsufh', 'gbcidshnhau', 'lgbcids', 'f', 'dlgbc', 'bdmfwdisdlgbcids', 'disdlgbcidshnhautsy', 'fwdisdlgbcidshn', 'zfpbfc', 'triqajlyfmxlredivqiambigmge']) from system.numbers limit 10; +select [0, 0, 16, 0, 0, 0, 14, 6, 2, 1, 0, 0, 1, 0, 10, 12, 0, 0, 0, 0] = multiSearchAllPositions(materialize('absimumlxdlxuzpyrunivcb'), ['jglfzroni', 'wzfmtbjlcdxlbpialqjafjwz', 'yrun', 'fgmljkkp', 'nniob', 'fdektoyhxrumiycvkwekphypgti', 'zp', 'um', 'bsimu', '', 'yslsnfisaebuujltpgcskhhqcucdhb', 'xlaphsqgqsfykhilddctrawerneqoigb', '', 'pdvcfxdlurmegspidojt', 'd', 'xu', 'fdp', 'xjrqmybmccjbjtvyvdh', 'nvhdfatqi', 'neubuiykajzcrzdbvpwjhlpdmd']) from system.numbers limit 10; +select [0, 0, 0, 9, 0, 0, 1, 1, 1, 1] = multiSearchAllPositions(materialize('lvyenvktdnylszlypuwqecohy'), ['ihlsiynj', 'ctcnhbkumvbgfdclwjhsswpqyfrx', 'rpgqwkydwlfclcuupoynwrfffogxesvmbj', 'dnyl', 'coeqgdtbemkhgplprfxgwpl', 'dkbshktectbduxlcaptlzspq', 'l', 'lvyenvktdnylszlypuw', 'lvyenvk', '']) from system.numbers limit 10; +select [1, 0, 0, 0, 0, 1, 2, 22, 8, 17, 1, 13, 0, 0, 0, 0, 0, 5] = multiSearchAllPositions(materialize('wphcobonpgaqwgfenotzadgqezx'), ['', 'qeuycfhkfjwokxgrkaodqioaotkepzlhnrv', 'taehtytq', 'gejlcipocalc', 'poyvvvntrvqazixkwigtairjvxkgouiuva', '', 'phc', 'dg', 'npgaqwg', 'notzadgqe', '', 'wgfe', 'smipuxgvntys', 'qhrfdytbfeujzievelffzrv', 'cfmzw', 'hcywnyguzjredwjbqtwyuhtewuhzkc', 'tssfeinoykdauderpjyxtmb', 'obonpgaqwgfen']) from system.numbers limit 10; +select [0, 0, 0, 0, 0, 6, 6, 0, 0, 2, 0, 5, 2, 0, 6, 3] = multiSearchAllPositions(materialize('qvslufpsddtfudzrzlvrzdra'), ['jxsgyzgnjwyd', 'hqhxzhskwivpuqkjheywwfhthm', 'kbwlwadilqhgwlcpxkadkamsnzngms', 'fxunda', 'nlltydufobnfxjyhch', 'fpsddtfudzrzl', 'fp', 'ykhxjyqtvjbykskbejpnmbxpumknqucu', 'iyecekjcbkowdothxc', 'vslufpsddtfu', 'mjgtofkjeknlikrugkfhxlioicevil', 'uf', 'vslufpsdd', 'cxizdzygyu', 'fpsddtfudzrz', 'slufp']) from system.numbers limit 10; +select [12, 0, 0, 0, 0, 1, 6, 0, 1, 2] = multiSearchAllPositions(materialize('ydsbycnifbcforymknzfi'), ['forymkn', 'vgxtcdkfmjhc', 'ymugjvtmtzvghmifolzdihutqoisl', 'fzooddrlhi', 'bdefmxxdepcqi', '', 'cnif', 'ilzbhegpcnkdkooopaguljlie', '', 'dsbycnifbcforym']) from system.numbers limit 10; +select [0, 2, 4, 1, 1, 3, 0, 0, 0, 7] = multiSearchAllPositions(materialize('sksoirfwdhpdyxrkklhc'), ['vuixtegnp', 'ks', 'oirfwdhpd', 'sksoirf', 'skso', 'soi', 'eoxpa', 'vpfmzovgatllf', 'txsezmqvduxbmwu', 'fw']) from system.numbers limit 10; +select [2, 21, 8, 10, 6, 0, 1, 11, 0, 0, 21, 4, 29, 0] = multiSearchAllPositions(materialize('wlkublfclrvgixpbvgliylzbuuoyai'), ['l', 'ylzbuu', 'clr', 'rvgi', 'lf', 'bqtzaqjdfhvgddyaywaiybk', '', 'vgixpbv', 'ponnohwdvrq', 'dqioxovlbvobwkgeghlqxtwre', 'y', 'ublfclrvgix', 'a', 'eoxxbkaawwsdgzfweci']) from system.numbers limit 10; +select [0, 0, 2, 1, 1, 9, 1, 0, 0, 1] = multiSearchAllPositions(materialize('llpbsbgmfiadwvvsciak'), ['knyjtntotuldifbndcpxzsdwdduv', 'lfhofdxavpsiporpdyfziqzcni', 'lpbsbgmf', 'llpbsbgmfi', 'llpbsbgmfiadwvv', 'fia', '', 'uomksovcuhfmztuqwzwchmwvonk', 'ujbasmokvghmredszgwe', '']) from system.numbers limit 10; +select [3, 0, 0, 0, 6, 1, 7, 0, 2, 1, 1, 0, 7, 0, 1, 0, 1, 1, 5, 11] = multiSearchAllPositions(materialize('hnmrouevovxrzrejesigfukkmbiid'), ['m', 'apqlvipphjbui', 'wkepvtnpu', 'amjvdpudkdsddjgsmzhzovnwjrzjirdoxk', 'ue', '', 'evov', 'qoplzddxjejvbmthnplyha', 'nmrouevovxrz', '', 'hnmrouev', 'hnzevrvlmxnjmvhitgdhgd', 'evovxrzrejesig', 'yvlxrjaqdaizishkftgcuikt', '', 'buyrmbkvqukochjteumqchrhxgtmuorsdgzlfn', '', 'hnmrouevov', 'ouevovx', 'xr']) from system.numbers limit 10; +select [0, 13, 0, 0, 0, 0, 0, 14, 0, 0, 1, 12, 0, 1] = multiSearchAllPositions(materialize('uwfgpemgdjimotxuxrxxoynxoaw'), ['uzcevfdfy', 'otxuxrxxoynxoa', 'xeduvwhrogxccwhnzkiolksry', 'pxdszcyzxlrvkymhomz', 'vhsacxoaymycvcevuujpvozsqklahstmvgt', 'zydsajykft', 'vdvqynfhlhoilkhjjkcehnpmwgdtfkspk', 'txuxrx', 'slcaryelankprkeyzaucfhe', 'iocwevqwpkbrbqvddaob', 'uwfg', 'motxuxrxx', 'kpzbg', '']) from system.numbers limit 10; +select [1, 1, 0, 6, 6, 0, 0, 0, 8, 0, 8, 14, 1, 5, 6, 0, 0, 1] = multiSearchAllPositions(materialize('epudevopgooprmhqzjdvjvqm'), ['ep', 'epudevopg', 'tlyinfnhputxggivtyxgtupzs', 'vopgoop', 'v', 'hjfcoemfk', 'zjyhmybeuzxkuwaxtcut', 'txrxzndoxyzgnzepjzagc', 'pgooprmhqzj', 'wmtqcbsofbe', 'pgo', 'm', '', 'evopgooprmhqzjdv', 'vopgooprmhqzjdv', 'gmvqubpsnvrabixk', 'wjevqrrywloomnpsjbuybhkhzdeamj', '']) from system.numbers limit 10; +select [15, 4, 4, 0, 0, 1, 1, 0, 0, 0, 0, 20, 0, 10, 1, 1, 0, 2, 4, 3] = multiSearchAllPositions(materialize('uogsfbdefogwnekfoeobtkrgiceksz'), ['kfoeobtkrgice', 'sfbd', 'sfbdefogwn', 'zwtenhiqavmqoolkvjiqjfb', 'vnjkshyvpwhrauackplqllakcjyamvsuokrxbfv', 'uog', '', 'qtzuhdcdymytgtscvzlzswdlrqidreuuuqk', 'vlridmjlbxyiljpgxsctzygzyawqqysf', 'xsnkwyrmjaaaryvrdgtoshdxpvgsjjrov', 'fanchgljgwosfamgscuuriwospheze', 'btkrgicek', 'ohsclekvizgfoatxybxbjoxpsd', 'ogwnekfoeobtkr', '', '', 'vtzcobbhadfwubkcd', 'og', 's', 'gs']) from system.numbers limit 10; +select [0, 0, 5, 1, 0, 5, 1, 6, 0, 1, 9, 0, 1, 1] = multiSearchAllPositions(materialize('aoiqztelubikzmxchloa'), ['blc', 'p', 'ztelubikzmxchlo', 'aoiqztelubi', 'uckqledkyfboolq', 'ztelubikzmxch', 'a', 'telubikzm', 'powokpdraslpadpwvrqpbb', 'aoiqztelu', 'u', 'kishbitagsxnhyyswn', '', '']) from system.numbers limit 10; +select [5, 11, 0, 0, 0, 5, 0, 0, 0, 1, 16, 0, 0, 0, 0, 0] = multiSearchAllPositions(materialize('egxmimubhidowgnfziwgnlqiw'), ['imubhidowgnfzi', 'dowgnf', 'yqpcpfvnfpxetozraxbmzxxcvtzm', 'xkbaqvzlqjyjoiqourezbzwaqkfyekcfie', 'jjctusdmxr', 'imubhi', 'zawnslbfrtqohnztmnssxscymonlhkitq', 'oxcitennfpuoptwrlmc', 'ac', 'egxmi', 'fziwgn', 'rt', 'fuxfuctdmawmhxxxg', 'suulqkrsfgynruygjckrmizsksjcfwath', 'slgsq', 'zcbqjpehilwyztumebmdrsl']) from system.numbers limit 10; +select [20, 0, 9, 0, 0, 14, 0, 5, 8, 3, 0, 0, 0, 4] = multiSearchAllPositions(materialize('zczprzdcvcqzqdnhubyoblg'), ['obl', 'lzrjyezgqqoiydn', 'vc', 'nbvwfpmqlziedob', 'pnezljnnujjbyviqsdpaqkkrlogeht', 'dn', 'irvgeaq', 'rzdcvcqzqdnh', 'cvcqzqdnh', 'zprzdcv', 'wvvgoexuevmqjeqavsianoviubfixdpe', 'aeavhqipsvfkcynyrtlxwpegwqmnd', 'blckyiacwgfaoarfkptwcei', 'prz']) from system.numbers limit 10; +select [2, 1, 1, 9, 10, 5, 0, 0, 0, 2, 9, 7, 9, 0, 1, 9, 7, 0] = multiSearchAllPositions(materialize('mvovpvuhjwdzjwojcxxrbxy'), ['vo', '', '', 'jwdz', 'wdzj', 'pvu', 'ocxprubxhjnji', 'phzfbtacrg', 'jguuqhhxbrwbo', 'vovpvuhjwd', 'jw', 'u', 'jwdzjwojcx', 'nlwfvolaklizslylbvcgicbjw', '', 'jwd', 'uhjwdz', 'bbcsuvtru']) from system.numbers limit 10; +select [2, 0, 21, 0, 0, 0, 3, 0, 0, 0, 0, 10, 1, 18] = multiSearchAllPositions(materialize('nmdkwvafhcbipwoqtsrzitwxsnabwf'), ['m', 'ohlfouwyucostahqlwlbkjgmdhdyagnihtmlt', 'itwx', 'jjkyhungzqqyzxrq', 'abkqvxxpu', 'lvzgnaxzctaarxuqowcski', 'dkwvafhcb', 'xuxjexmeeqvyjmpznpdmcn', 'vklvpoaakfnhtkprnijihxdbbhbllnz', 'fpcdgmcrwmdbflnijjmljlhtkszkocnafzaubtxp', 'hmysdmmhnebmhpjrrqpjdqsgeuutsj', 'cbipwoqtsrzitwxsna', 'nm', 'srzitwx']) from system.numbers limit 10; +select [17, 5, 0, 13, 0, 0, 10, 1, 0, 19, 10, 8, 0, 4] = multiSearchAllPositions(materialize('gfvndbztroigxfujasvcdgfbh'), ['asvcdgf', 'dbztroigxfujas', 'pr', 'xfujas', 'nxwdmqsobxgm', 'wdvoepclqfhy', 'oigxfu', '', 'flgcghcfeiqvhvqiriciywbkhrxraxvneu', 'vcd', 'oigxfu', 'troigxfuj', 'gbnyvjhptuehkefhwjo', 'ndbz']) from system.numbers limit 10; +select [0, 14, 1, 0, 0, 1, 1, 11, 0, 8, 6, 0, 3, 19, 7, 0] = multiSearchAllPositions(materialize('nofwsbvvzgijgskbqjwyjmtfdogzzo'), ['kthjocfzvys', 'skbqjwyjmtfdo', 'nof', 'mfapvffuhueofutby', 'vqmkgjldhqohipgecie', 'nofwsbv', '', 'ijgs', 'telzjcbsloysamquwsoaso', 'vzgijgskbqjwyjmt', 'bvvzgijgskbqjwyjmtfd', 'hdlvuoylcmoicsejofcgnvddx', 'fwsbvvzgijgskb', 'wyjm', 'vvzgijg', 'fwzysuvkjtdiufetvlfwf']) from system.numbers limit 10; +select [10, 2, 13, 0, 0, 0, 2, 0, 9, 2, 4, 1, 1, 0, 1, 6] = multiSearchAllPositions(materialize('litdbgdtgtbkyflsvpjbqwsg'), ['tbky', 'itdbgdtgtb', 'yflsvpjb', 'ikbylslpoqxeqoqurbdehlroympy', 'hxejlgsbthvjalqjybc', 'sontq', 'itdbgd', 'ozqwgcjqmqqlkiaqppitsvjztwkh', 'gtbkyf', 'itdbgdtgtbkyfls', 'dbg', 'litdb', '', 'qesbakrnkbtfvwu', 'litd', 'g']) from system.numbers limit 10; +select [0, 0, 1, 1, 5, 0, 8, 12, 0, 2, 0, 7, 0, 6] = multiSearchAllPositions(materialize('ijzojxumpvcxwgekqimrkomvuzl'), ['xirqhjqibnirldvbfsb', 'htckarpuctrasdxoosutyxqioizsnzi', '', '', 'jxu', 'dskssv', 'mpvcxwgekqi', 'xwgek', 'qsuexmzfcxlrhkvlzwceqxfkyzogpoku', 'jzojx', 'carjpqihtpjniqz', 'umpvcxwgekq', 'krpkzzrxxtvfhdopjpqcyxfnbas', 'xumpvcxwg']) from system.numbers limit 10; +select [0, 0, 0, 6, 0, 8, 0, 2, 0, 0, 0, 0, 14, 0, 0, 1, 1, 0, 0, 0] = multiSearchAllPositions(materialize('zpplelzzxsjwktedrrtqhfmoufv'), ['jzzlntsokwlm', 'cb', 'wuxotyiegupflu', 'lzzxsjwkte', 'owbxgndpcmfuizpcduvucnntgryn', 'zxsjwktedrrtqhf', 'kystlupelnmormqmqclgjakfwnyt', 'pple', 'lishqmxa', 'mulwlrbizkmtbved', 'uchtfzizjiooetgjfydhmzbtmqsyhayd', 'hrzgjifkinwyxnazokuhicvloaygeinpd', 'tedrrt', 'shntwxsuxux', 'evrjehtdzzoxkismtfnqp', 'z', '', 'nxtybut', 'vfdchgqclhxpqpmitppysbvxepzhxv', 'wxmvmvjlrrehwylgqhpehzotgrzkgi']) from system.numbers limit 10; + +select [15, 19, 0, 0, 15, 0, 0, 1, 2, 6] = multiSearchAllPositionsUTF8(materialize('зжерхмчсйирдчрришкраоддцфгх'), ['ришкра', 'раоддц', 'фттиалусгоцжлтщзвумрдчи', 'влййи', 'ришкра', 'цгфжуцгивй', 'ккгжхрггчфглх', 'з', 'жерхмчсйи', 'мчсйирдчрришкраоддц']) from system.numbers limit 10; +select [0, 0, 0, 1, 4, 0, 14, 0, 1, 8, 8, 9, 0, 0, 4, 0] = multiSearchAllPositionsUTF8(materialize('етвхйчдобкчукхпщлмжпфайтфдоизщ'), ['амфшужперосрфщфлижйййжжжй', 'ххкбщшзлмщггтшцпсдйкдшйвхскемц', 'ергйплгпнглккшкарещимгапхг', '', 'хйчдо', 'вввбжовшзйбгуоиждепйабаххеквщижтйиухос', 'хпщл', 'жфуомщуххнедзхищнгхрквлпмзауеегз', 'етвхй', 'о', 'о', 'бк', 'цфецккифж', 'аизлокл', 'х', 'слщгеивлевбчнчбтшгфмжрфка']) from system.numbers limit 10; +select [0, 0, 1, 2, 0, 0, 14, 0, 3, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('йбемооабурнирйофшдгпснж'), ['гпфцл', 'нчбперпмцкввдчсщвзйрдфнф', '', 'бем', 'ч', 'жгш', 'йофшдгпснж', 'шасгафчг', 'емооабур', 'пиохцжццгппщчопзйлмуотз', 'рпдомнфвопхкшешйишумбацтл', 'нисиийфррбдоц']) from system.numbers limit 10; +select [1, 18, 12, 0, 0, 1, 1, 3, 7, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('гсщнфийтфзжцйпфбйалущ'), ['', 'алущ', 'цйпфбйал', 'цвбфцйвсвлицсчнргпцнр', 'х', 'гс', '', 'щн', 'й', 'дгйрвцщтп', 'уитвквоффвцхфишрлерйцувф', 'кфтййлпнзжчижвглзкижн']) from system.numbers limit 10; +select [14, 0, 5, 5, 0, 6, 0, 16, 0, 0] = multiSearchAllPositionsUTF8(materialize('ефщнйнуйебнснлрцгкеитбг'), ['лрцгкеитб', 'епклжфцпнфопе', 'йнуйебн', 'й', 'тлт', 'нуйебнснлрцгкеит', 'глечршгвотумкимтлм', 'цгк', 'щгйчой', 'звкцкчк']) from system.numbers limit 10; +select [0, 1, 18, 6, 0, 3, 0, 0, 25, 0, 0, 1, 16, 5, 1, 7, 0, 0] = multiSearchAllPositionsUTF8(materialize('пумгмцшмжштсшлачсжарерфиозиг'), ['чсуубфийемквмоотванухмбрфхжоест', '', 'жар', 'цшмжш', 'жртещтинтвпочнкдткцза', 'м', 'адзгтбаскщгдшжл', 'штфжшллезпджигщфлезфгзчайанхктицштйй', 'о', 'етадаарйсцейдошшцечхзлшлрртсрггцртспд', 'зтвшалрпфлщбцд', 'пу', 'ч', 'мцшмжштсшлачсж', '', 'шмжшт', 'ещтжшйтчзчаноемрбц', 'тевбусешйрйчшзо']) from system.numbers limit 10; +select [7, 10, 0, 0, 0, 0, 1, 12, 9, 2, 0, 0, 0, 4, 1, 1, 0, 6] = multiSearchAllPositionsUTF8(materialize('дупгвндвйжмаузнллнзл'), ['двйжмаузн', 'жмаузнлл', 'емйжркоблновцгпезрдавкбелцщста', 'щзкгм', 'лебрпцрсутшриащгайвц', 'лзнмл', 'д', 'ауз', 'йжмау', 'упгвндвйж', 'жщсббфвихг', 'всигсеигцбгаелтчкирлнзшзцжещнс', 'рмшиеиесрлщципщхкхтоцщчйоо', 'гвн', '', '', 'йадеоцлпшпвщзещзкхйрейопмажбб', 'ндв']) from system.numbers limit 10; +select [0, 0, 0, 8, 3, 10, 22, 0, 13, 11, 0, 1, 18, 0, 1, 0] = multiSearchAllPositionsUTF8(materialize('жшзфппавввслфцлнщшопкдшку'), ['саоткнхфодзаа', 'кйхванкзаисйбврщве', 'бчоуучватхфукчф', 'вввслфц', 'з', 'вслфцлнщшопк', 'дшк', 'из', 'фцл', 'с', 'зртмцтпощпщхк', 'жшзфппавввслфц', 'шопк', 'збтхрсдтатхпрзлхдооощифачхчфн', '', 'жщшийугз']) from system.numbers limit 10; +select [2, 4, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 21, 0, 6, 0] = multiSearchAllPositionsUTF8(materialize('пчботухвгдчекмжндбоожш'), ['чботухвгдчекмжндб', 'от', 'гвсжжйлбтщчучнхсмдйни', 'жцжзмшлибшефуоуомпацбщщу', 'онхфлуцйлхтбмц', 'йтепжу', 'хтдрпвкщрли', 'аддайф', 'нхегщккбфедндоацкиз', 'йгкцзртфжгв', 'буелрщмхйохгибжндфшщвшрлдччрмфмс', 'цщцтзфнщ', 'уч', 'пчб', 'жш', 'пнфббтшйгхйрочнлксщпгвжтч', 'ухвг', 'лсцппузазщрйхймщбзоршощбзленхп']) from system.numbers limit 10; +select [0, 0, 4, 11, 0, 0, 0, 0, 0, 11, 2, 4, 6, 0, 0, 1, 2, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('тжрмчпваухрхуфбгнифгбопфт'), ['дпмгкекщлнемссаицщпащтиуцхкфчихтц', 'акйиуоатунтчф', 'мчпва', 'рхуфбгнифгб', 'кнаишж', 'пчвотенеафкухжцешбцхг', 'опеа', 'ушчадфтчхечеуркбтел', 'ашшптаударчжчмвалтдхкимищпф', 'рхуфбгниф', 'ж', 'мчпваухрхуфбгнифг', 'пваухрху', 'зргачбтцдахвймсбсврбндзтнущхвп', 'асбфцавбгуолг', 'тж', 'жрмчпваухрх', 'мрвзцгоб', 'чрцснчсдхтзжвнздзфцвхеилишдбж', 'кчт']) from system.numbers limit 10; +select [0, 2, 4, 0, 6, 0, 0, 0, 0, 19, 7, 1, 0, 1, 0, 0, 2, 10, 0, 1] = multiSearchAllPositionsUTF8(materialize('опрурпгабеарушиойцрхвбнсщ'), ['йошуоесдщеж', 'пр', 'урпгабеарушиой', 'хщиаршблашфажщметчзи', 'пгабеарушиойцрхвб', 'щцбдвц', 'еечрззвкожзсдурйщувмцйшихдц', 'офхачгсзашфзозрлба', 'айдфжджшжлрргмабапткбцпиизигдтс', 'рх', 'габ', '', 'цнкдбфчщшмчулврбцчакщвзхлазфа', '', 'екбтфпфилсаванхфкмчнпумехиищди', 'епвщхаклшомвцжбф', 'прурпгабе', 'еарушиойцрхв', 'црвтгрзтитц', 'опрурпг']) from system.numbers limit 10; +select [0, 10, 1, 0, 0, 0, 0, 0, 10, 0, 15, 2] = multiSearchAllPositionsUTF8(materialize('угпщлзчжшбзвууцшатпщцр'), ['цоуарцжсз', 'бз', '', 'пщфтзрч', 'лфуипмсдмнхнгйнтк', 'айжунцйбйцасчфдхй', 'щдфщлцптплсачв', 'грв', 'бзвууц', 'бумййшдшфашцгзфвчвзвтсувнжс', 'цшатпщ', 'гпщлзчжшб']) from system.numbers limit 10; +select [0, 15, 0, 1, 5, 0, 0, 5, 0, 0, 0, 1, 0, 0] = multiSearchAllPositionsUTF8(materialize('цнлеодлмдцдйснитвдчтхжизв'), ['ивкчсзшугоцжчохщцабл', 'итвдчт', 'кнх', '', 'одлм', 'ктшфзбщзцуймагсоукщщудвуфо', 'ххеаефудгчхр', 'одлмдцдйснитвдчт', 'умцлпкв', 'зщсокйтцзачщафвбповжгнлавсгйг', 'бкибм', '', 'охсоихнцчцшевчеележтука', 'фаийхгжнсгищгщц']) from system.numbers limit 10; +select [0, 0, 0, 2, 0, 0, 0, 0, 3, 2, 3, 6, 0, 0, 0, 12, 4, 1] = multiSearchAllPositionsUTF8(materialize('бгдбувдужщвоошлтчрбй'), ['щвбаиф', 'итчднесжкчжвпжйвл', 'мм', 'г', 'хктзгтзазфгщшфгбеулцмдмдбдпчзх', 'сфуак', 'злйфцощегзекщб', 'фшлдтолрщфзжчмих', 'дбувдужщ', 'гдб', 'дбувдужщ', 'в', 'лчищкечнжщисцичбнзшмулпмлп', 'чжцсгмгфвлиецахзнрбмщин', 'обпжвй', 'о', 'бувдужщвоош', '']) from system.numbers limit 10; +select [0, 2, 5, 3, 2, 0, 1, 0, 0, 4, 2, 0, 0, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('шсушлорзфжзудбсейенм'), ['чнзпбновтршеумбвщчлх', 'су', 'лорзфж', 'ушлорзфжзудб', 'сушлорзфжзудбсейенм', 'ткдрхфнб', '', 'пщд', 'чбдцмщ', 'шлорзфж', 'су', 'сккигркедчожжемгнайвйчтдмхлтти', 'мц', 'пхнхрхйцйсйбхчлктз', 'иафжстлйфцр', 'алщщлангнбнйхлшлфшйонщек']) from system.numbers limit 10; +select [12, 1, 0, 5, 0, 10, 1, 0, 7, 4, 0, 1, 12, 1, 1, 1, 0, 1, 15, 0] = multiSearchAllPositionsUTF8(materialize('ощзллчубоггцвжриуардрулащйпу'), ['цвжр', '', 'нмзкаиудзтиффззшзканжвулт', 'лчубоггцвжриуардрулащйпу', 'чтцлзшуижолибаоххвшихбфжйхетивп', 'ггцвжри', '', 'йдгнвс', 'у', 'л', 'зпщнжуойдлдвхокцжнзйсйзе', '', 'цв', '', '', '', 'ехлцзгвф', '', 'риу', 'уйжгтжноомонгщ']) from system.numbers limit 10; +select [0, 12, 13, 20, 0, 1, 0, 0, 3, 4] = multiSearchAllPositionsUTF8(materialize('цбкифйтшузажопнжщарбштвдерзтдш'), ['щлмлижтншчсмксгтнсврро', 'жопнжщарбштвд', 'опнжщарб', 'бштвдерзтд', 'пуфслейщбкжмпнш', 'ц', 'маве', 'кмйхойрдлшцхишдтищвйбцкщуигваещгтнхйц', 'кифй', 'и']) from system.numbers limit 10; +select [0, 6, 0, 0, 0, 8, 0, 3, 6, 0] = multiSearchAllPositionsUTF8(materialize('еачачгбмомоххкгнвштггпчудл'), ['ндзчфчвжтцщпхщуккбеф', 'г', 'рткнфвчтфннхлжфцкгштймгмейжй', 'йчннбщфкщф', 'лсртщиндшшкичзррущвдйвнаркмешерв', 'момоххк', 'рфафчмсизлрхзуа', 'ч', 'гбмомоххкгнвштг', 'валжпошзбгзлвевчнтз']) from system.numbers limit 10; +select [0, 0, 10, 0, 8, 13, 0, 0, 19, 15, 3, 1] = multiSearchAllPositionsUTF8(materialize('зокимчгхухшкшмтшцчффвззкалпва'), ['цалфжажщщширнрвтпвмщжннрагвойм', 'оукзрдцсадешжмз', 'хшкшмтшцч', 'ауилтсаомуркпаркбцркугм', 'хухшкшмтшцчффв', 'шмтшцч', 'зщгшпцхзгцншднпеусмтжбцшч', 'щлраащсйлщрд', 'ффвзз', 'тшцчффвззкалпв', 'кимчгхухшкш', '']) from system.numbers limit 10; +select [0, 0, 1, 0, 6, 0, 6, 0, 5, 0, 13, 0, 0, 6] = multiSearchAllPositionsUTF8(materialize('йдйндиибщекгтчбфйдредпхв'), ['тдршвтцихцичощнцницшдхйбогбчубие', 'акппакуцйсхцдххнотлгирввоу', '', 'улщвзхохблтксчтб', 'и', 'ибейзчшклепзриж', 'иибщекгт', 'шидббеухчпшусцнрз', 'диибщекгтчбфйд', 'дейуонечзйзлдкшщрцйбйклччсцуй', 'тч', 'лшицлшме', 'чйнжчоейасмрщегтхвйвеевбма', 'ии']) from system.numbers limit 10; +select [15, 3, 3, 2, 0, 11, 0, 0, 0, 2, 0, 4, 0, 1, 1, 3, 0, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('нхгбфчшджсвхлкхфвтдтлж'), ['хфвтдтлж', 'гбфчшд', 'гбфчш', 'х', 'ачдгбккжра', 'вхлк', 'мщчвещлвшдщпдиимлшрвнщнфсзгщм', 'жчоббгшзщлгеепщжкчецумегпйчт', 'жжд', 'хг', 'мтсааролшгмоуйфйгщгтрв', 'бфчшд', 'чейрбтофпшишгуасоодлакчдф', 'н', 'нхгбфч', 'гбф', 'гдежсх', 'йифжацзгжбклх', 'ещпзущпбаолплвевфиаибшйубйцсзгт', 'жезгчжатзтучжб']) from system.numbers limit 10; +select [0, 10, 1, 0, 0, 0, 4, 0, 13, 1, 12, 1, 0, 6] = multiSearchAllPositionsUTF8(materialize('акбдестрдшерунпвойзв'), ['нркчх', 'шерунп', '', 'зжвахслфббтоиоцрзаззасгнфчх', 'шлжмдг', 'тлйайвцжчсфтцйрчосмижт', 'дестрдшерунп', 'мвамйшцбдщпчлрщд', 'у', 'акбдестрд', 'рунпвойз', '', 'айздцоилсйшцфнчтхбн', 'с']) from system.numbers limit 10; +select [1, 0, 0, 3, 2, 1, 0, 0, 1, 10, 7, 0, 5, 0, 8, 4, 1, 0, 8, 1] = multiSearchAllPositionsUTF8(materialize('кйхпукаеуддтйччхлнпсуклрф'), ['кйхпукаеуддтйччхл', 'йатлрйкстлхфхз', 'фгихслшкж', 'хпу', 'йхпукаеу', '', 'сруакбфоа', 'оажуз', 'кйхпукаеуддтйччх', 'ддтйччхлн', 'аеуддтйччхл', 'тмажиойщтпуцглхфишеиф', 'укаеуддтйччхлнпс', 'ретифе', 'еуддтйччхлнпсуклр', 'пукаеуд', 'кйхпу', 'таппфггвджлцпжшпишбпциуохсцх', 'еуд', '']) from system.numbers limit 10; +select [2, 3, 3, 16, 5, 13, 0, 0, 0, 18, 0, 6, 0, 16, 0, 10, 3, 0] = multiSearchAllPositionsUTF8(materialize('плврйщовкзнбзлбжнсатрцщщучтйач'), ['лврйщовкзнбзлбж', 'врйщовкзнбзлбжнса', 'врйщовкзнбз', 'жнсатрцщщучтйач', 'йщовкзнбзлбжнсатрцщщуч', 'злбжнсатрцщ', 'ввтбрдт', 'нжйапойг', 'ннцппгперхйвдхоеожупйебочуежбвб', 'сатрцщщу', 'деваийтна', 'щ', 'вкжйгкужжгтевлцм', 'жнс', 'датг', 'знбзлбжнсатрцщщучтйа', 'врйщовк', 'оашмкгчдзщефм']) from system.numbers limit 10; +select [3, 1, 19, 1, 0, 0, 0, 0, 11, 3, 0, 0] = multiSearchAllPositionsUTF8(materialize('фчдеахвщжхутхрккхасвсхепщ'), ['деах', '', 'свсхепщ', '', 'анчнсржйоарвтщмрж', 'нечбтшщвркгд', 'вштчцгшж', 'з', 'у', 'деахвщ', 'ххкцжрвзкжзжчугнфцшуиаклтмц', 'фцкжшо']) from system.numbers limit 10; +select [16, 0, 0, 1, 8, 14, 0, 12, 12, 5, 0, 0, 16, 0, 11, 0] = multiSearchAllPositionsUTF8(materialize('щмнжчввбжцчммчшсрхйшбктш'), ['срхйшбк', 'йлзцнржчууочвселцхоучмщфчмнфос', 'еижлафатшхщгшейххжтубзвшпгзмзцод', '', 'бжцчммчшсрхй', 'чшсрхй', 'влемчммйтителщвзган', 'ммч', 'ммчшсрх', 'чввбж', 'нобзжучшошмбщешлхжфгдхлпнгпопип', 'цгт', 'срхйш', 'лкклмйжтеа', 'чммчшсрхйшбктш', 'йежффзнфтнжхфедгбоахпг']) from system.numbers limit 10; +select [1, 12, 9, 5, 1, 0, 6, 3, 0, 1] = multiSearchAllPositionsUTF8(materialize('кжнщсашдзитдмщцхуоебтфжл'), ['', 'дмщцхуоебт', 'зитдмщцхуоебт', 'сашдзитдмщцхуое', 'кжнщ', 'тхкйтшебчигбтмглшеужззоббдилмдм', 'ашдзитдмщцхуоебтф', 'нщсашдз', 'аузщшр', 'кжнщсашдз']) from system.numbers limit 10; +select [2, 0, 0, 0, 1, 0, 2, 0, 0, 17, 0, 8, 7, 14, 0, 0, 0, 7, 9, 23] = multiSearchAllPositionsUTF8(materialize('закуфгхчтшивзчжаппбжнтслщввущ'), ['а', 'днойвхфрммтж', 'внтлжрхзрпчбтуркшдатннглечг', 'ахиеушжтфкгцщтзхмжнрхдшт', '', 'тцчгрззржмдшйщфдцрбшжеичч', 'а', 'ктиечцпршнфнбчуолипацчдсосцнлфаццм', 'аусрлхдцегферуо', 'ппбжнт', 'жкццуосгвп', 'чтшивзчжаппб', 'хчтшивзчжаппб', 'чжаппбжнтслщ', 'ччрлфдмлу', 'щзршффбфчзо', 'ущуймшддннрхзийлваежщухч', 'хчтши', 'тшивзчжаппбжнтсл', 'слщв']) from system.numbers limit 10; +select [1, 1, 9, 2, 0, 3, 7, 0, 0, 19, 2, 2, 0, 8] = multiSearchAllPositionsUTF8(materialize('мвкзккупнокченйнзкшбдрай'), ['м', '', 'н', 'вкз', 'гдпертшйбртотунур', 'к', 'упнокченйнзкшбдр', 'нфшрг', 'нмждрйббдцлйемжпулдвкещхтжч', 'ш', 'вкзккупнокченйнзкшбдр', 'вкзккупнокченйнзкшбдрай', 'адииксвеавогтйторчтцвемвойшпгбнз', 'пнокченй']) from system.numbers limit 10; +select [15, 0, 0, 1, 12, 1, 0, 0, 1, 11, 0, 4, 0, 2] = multiSearchAllPositionsUTF8(materialize('отарлшпсабждфалпшножид'), ['лпшно', 'вт', 'лпжшосндутхорлиифжаакш', 'отарлшпсабждфалпшнож', 'дфал', '', 'бкцжучншжбгзжхщпзхирртнбийбтж', 'уцвцкшдзревпршурбсвйнемоетчс', '', 'ждфал', 'тлскхрнпмойчбцпфущфгф', 'рлшпсабж', 'нхнмк', 'тарлшпса']) from system.numbers limit 10; +select [0, 2, 0, 20, 0, 17, 18, 0, 1, 1, 21, 1, 0, 1, 6, 26] = multiSearchAllPositionsUTF8(materialize('ачйвцштвобижнзжнчбппйеабтцнйн'), ['сзхшзпетншйисщкшрвйшжуогцвбл', 'чйвцштво', 'евз', 'пй', 'хуждапрахитйажрищуллйзвчт', 'чбппйе', 'бппйеабтцнйн', 'схш', 'а', 'ачйвцштвобижнзжнчбпп', 'йеабтцнй', '', 'ег', '', 'штвобижнзжнчбпп', 'цн']) from system.numbers limit 10; +select [1, 0, 0, 3, 4, 12, 0, 9, 0, 12, 0, 0, 8, 0, 10, 3, 4, 1, 1, 9] = multiSearchAllPositionsUTF8(materialize('жмхоужежйуфцзеусеоднчкечфмемба'), ['', 'идосйксзнщйервосогф', 'тхмсйлвкул', 'хоужежйуф', 'оужежйуфцзеусеоднчкечфм', 'цзеусеоднчкеч', 'бецвдиубххвхйкажуурщщшщфбзххт', 'йуфцзеусеодн', 'мглкфтуеайсржисстнпкгебфцпа', 'цзеусео', 'уехцфучецчгшйиржтсмгхакчшввохочжпухс', 'дчвмсбткзталшбу', 'жйуфцзеусеоднчке', 'ччшщтдбпвчд', 'уфцзеусеоднчкечфмем', 'хоужежйуфцзеусеоднчкечф', 'оуже', '', 'жмхоужежйуфцзеу', 'й']) from system.numbers limit 10; +select [0, 0, 0, 3, 0, 0, 0, 0, 1, 0, 1, 0, 1, 2, 0, 0, 0, 6] = multiSearchAllPositionsUTF8(materialize('лшпцхкмтресзпзйвцфрз'), ['енрнцепацлщлблкццжсч', 'ецжужлуфаееоггрчохпчн', 'зхзнгасхебнаейбддсфб', 'пцхкмтресзпзйв', 'фчетгеодщтавиииухцундпнхлчте', 'шшгсдошкфлгдвкурбуохзчзучбжйк', 'мцщщцп', 'рх', '', 'зйошвщцгхбж', '', 'ввлпнамуцвлпзеух', '', 'шпцхкмтре', 'маабтруздрфйпзшлсжшгож', 'фдчптишмштссщшдшгх', 'оллохфпкаем', 'кмтресзпз']) from system.numbers limit 10; +select [2, 5, 0, 0, 6, 0, 0, 0, 0, 0, 0, 0, 1, 1, 12, 0, 0, 0, 4, 8] = multiSearchAllPositionsUTF8(materialize('есипзсвшемлхчзмйрсфз'), ['с', 'з', 'пщчсмаиахппферзжбпвиибаачй', 'гтщкзоиежав', 'свшемлхчзм', 'шийанбке', 'зхе', 'авркудфаусзквкфффйцпзжщввенттб', 'ножцваушапиж', 'иизкежлщиафицкчщмалнпсащсднкс', 'вчмв', 'кщеурмуужжлшррце', '', '', 'х', 'алзебзпчеложихашжвхмйхрицн', 'тпзмумчшдпицпдшиаог', 'сулксфчоштаййзбзшкджббщшсей', 'пзсвшемлхчзм', 'ш']) from system.numbers limit 10; +select [0, 1, 2, 4, 0, 0, 14, 1, 13, 4, 0, 0, 1, 1] = multiSearchAllPositionsUTF8(materialize('сзиимонзффичвфжоеулсадону'), ['зфтшебтршхддмеесчд', '', 'зиимонзф', 'имон', 'езбдйшжичценлгршщшаумайаицй', 'птпщемтбмднацлг', 'фжоеулса', '', 'вфжоеулсадону', 'имонзфф', 'йщвдфдиркважгйджгжашарчучйххйднпт', 'дй', '', '']) from system.numbers limit 10; +select [12, 0, 24, 0, 9, 0, 1, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('ижсщщрзжфнгццпзкфбвезгбохлж'), ['ццпзкфбвез', 'ацррвхоптаоснулнжкщжел', 'охлж', 'тнсхбпшщнб', 'фнг', 'урйвг', '', 'цохс', 'щбйрйкжчмйзачуефч', 'афа']) from system.numbers limit 10; +select [9, 0, 0, 0, 1, 0, 7, 7, 0, 0, 1, 0, 7, 0, 0, 8, 0, 3, 0, 0] = multiSearchAllPositionsUTF8(materialize('рерфвирачйнашхрмцебфдйааеммд'), ['чйнашхрмцебфдйааеммд', 'сжщзснвкущлжплцзлизаомдизцнжлмййбохрцч', 'еппбжджмримфчйеаолидпцруоовх', 'едтжкоийггснехшсчйлвфбкцжжрчтш', '', 'пжахфднхсотй', 'ра', 'рач', 'вчримуцнхбкуйжрвфиугзфсзг', 'кщфехрххциаашщсифвашгйцвхевцщнйахтбпжщ', '', 'ртщиобчжстовйчфабалзц', 'рачйнашхрмцебфдйаае', 'ощгжосччфкуг', 'гехвжнщжссидмрфчйтнепдсртбажм', 'а', 'ицжлсрсиатевбвнжрдмзцувввтзцфтвгвш', 'рф', 'прсмлча', 'ндлхшцааурмзфгверуфниац']) from system.numbers limit 10; +select [2, 14, 10, 0, 6, 15, 1, 0, 0, 4, 5, 17, 0, 0, 3, 0, 3, 0, 9, 0] = multiSearchAllPositionsUTF8(materialize('влфощсшкщумчллфшшвбшинфппкчуи'), ['лфощ', 'лфшшвбшинфпп', 'умчллфшшвбшинф', 'слмтнг', 'сшкщумчллфшшвбшинф', 'фшшвб', '', 'рчфбчййсффнодцтнтнбцмолф', 'щфнщокхжккшкудлцжрлжкнп', 'ощ', 'щсшкщумчлл', 'швбшинфппкч', 'септзкщотишсехийлоцчапщжшжсфмщхсацг', 'нт', 'фощсшкщумчллфшшвбшинфп', 'нщпдш', 'фощс', 'мивсмча', 'щумч', 'щчйнткжпмгавфтйтибпхх']) from system.numbers limit 10; +select [0, 10, 0, 0, 0, 0, 0, 3, 0, 0, 0, 2, 0, 11, 0, 0] = multiSearchAllPositionsUTF8(materialize('еаиалмзхцгфунфеагшчцд'), ['йнш', 'гфун', 'жлйудмхнсвфхсуедспщбтутс', 'елмуийгдйучшфлтхцппамфклйг', 'евйдецц', 'пчтфцоучфбсйщпвдацмчриуцжлтжк', 'нстмпумчспцвцмахб', 'иалмз', 'зифчп', 'чогфщимоопт', 'фдйблзеп', 'аиа', 'щугмзужзлйдктш', 'фунфеагшч', 'нйхшмсгцфжчхжвхгдхцуппдц', 'асмвмтнрейшгардллмсрзгзфйи']) from system.numbers limit 10; +select [23, 0, 8, 0, 0, 0, 0, 0, 0, 4, 0, 5, 7, 1, 9, 4] = multiSearchAllPositionsUTF8(materialize('зузйфзлхходфрхгтбпржшрктпйхеоп'), ['ктпйхео', 'лжитуддикчсмкглдфнзцроцбзтсугпвмхзллжж', 'х', 'меуфтено', 'фтдшбшрпоцедктсийка', 'кхтоомтбчвеонксабшйптаихжбтирпзшймчемжим', 'чиаущлрдкухцрдумсвивпафгмр', 'фрнпродв', 'тдгтишхйсашвмдгкчбмшн', 'йфзлхходфрхгтбпржшр', 'бежшлрйврзмумеуооплкицхлйажвцчнчсеакм', 'ф', 'лхходфрхгтб', '', 'ходфрхгтбпржшр', 'й']) from system.numbers limit 10; +select [0, 0, 0, 1, 0, 1, 22, 1, 0, 0, 0, 0, 18, 1, 0, 0, 0, 1] = multiSearchAllPositionsUTF8(materialize('чфгвчхчпщазтгмбнплдгщикойчднж'), ['мштцгтмблаезочкхзвхгрбпкбмзмтбе', 'канбжгсшхшз', 'кзинвщйччажацзйнсанкнщ', 'чфгвчхчпщазтгмбнп', 'етйцгтбнщзнржнйхж', '', 'ик', '', 'еизщвпрохдгхир', 'псумйгшфбвгщдмхжтц', 'слмжопинйхнштх', 'йшралцицммбщлквмгхцввизопнт', 'л', 'чфгвчхчпщазтгмбнплдгщ', 'пбзмхжнпгикиищжтшботкцеолчцгхпбвхи', 'хзкцгрмшгхпхуоцгоудойнжлсоййосссмрткцес', 'ажуофйпщратдйцбржжлжнжащцикжиа', '']) from system.numbers limit 10; +select [6, 0, 2, 5, 2, 9, 10, 0, 0, 4, 0, 6, 3, 2] = multiSearchAllPositionsUTF8(materialize('ишогпсисжашфшлйичлба'), ['сисжашфшлй', 'пднещбгзпмшепкфосовбеге', 'шогп', 'пс', 'шогпси', 'жаш', 'аш', 'деисмжатуклдшфлщчубфс', 'грмквкщзур', 'гпсис', 'кйпкбцмисчхдмшбу', 'сисжашф', 'о', 'шо']) from system.numbers limit 10; +select [8, 15, 13, 0, 1, 2, 5, 2, 9, 0, 0, 0] = multiSearchAllPositionsUTF8(materialize('нсчщчвсанпрлисблснокзагансхм'), ['анпрлисблснокзагансхм', 'блснокз', 'исб', 'дрмгвснпл', '', 'счщчвса', 'чвсанпрлисблснокзагансх', 'счщчвсанпрлис', 'нпрли', 'пциишуецнймуодасмжсойглретиефо', 'фхимщвкехшлг', 'слщмаимшжчфхзпрцмхшуниврлуйлжмфжц']) from system.numbers limit 10; +select [0, 5, 0, 0, 14, 0, 12, 0, 2, 3, 0, 3, 21, 5] = multiSearchAllPositionsUTF8(materialize('хажуижанндвблищдтлорпзчфзк'), ['щуфхл', 'и', 'фцежлакчннуувпаму', 'щесщжрчиктфсмтжнхекзфс', 'ищдтлорпзчф', 'дееичч', 'блищ', 'гиефгйзбдвишхбкбнфпкддмбтзиутч', 'ажуижа', 'жуижанндвблищдтлорпзчфзк', 'чщщдзетвщтччмудвзчгг', 'ж', 'пзчфз', 'ижанн']) from system.numbers limit 10; +select [0, 0, 0, 9, 15, 0, 0, 0, 1, 3, 0, 0, 1, 0, 10, 0, 4, 0, 0, 7] = multiSearchAllPositionsUTF8(materialize('россроапцмцагвиигнозхзчотус'), ['ошажбчвхсншсвйршсашкм', 'пфдчпдчдмауцгкйдажрйефапввшжлшгд', 'иеаочутввжмемчушлуч', 'цмцагвиигно', 'ииг', 'ммпжщожфйкакбущчирзоммагеиучнщмтвгихк', 'укррхбпезбжууеипрзжсло', 'ншопзжфзббилйбувгпшшиохврнфчч', '', 'ссроап', 'лийщфшдн', 'йчкбцциснгначдцйчпа', 'россроапцмцагвииг', 'кштндцтсшорввжсфщчмщчжфжквзралнивчзт', 'мца', 'нбтзетфтздцао', 'сроа', 'мщсфие', 'дткодбошенищйтрподублжскенлдик', 'апцмцагвиигноз']) from system.numbers limit 10; +select [16, 0, 0, 2, 1, 1, 0, 1, 9, 0, 0, 3] = multiSearchAllPositionsUTF8(materialize('тйсдйилфзчфплсджбарйиолцус'), ['жбарйиолцу', 'цназщжждефлбрджктеглщпунйжддгпммк', 'хгжоашцшсзкеазуцесудифчнощр', 'йс', '', 'тйсдйилфзчфп', 'ивфсплшвслфмлтххжчсстзл', '', 'зчфплсдж', 'йртопзлодбехрфижчдцйс', 'цлащцкенмшеоерееиуноп', 'с']) from system.numbers limit 10; +select [3, 2, 1, 1, 0, 0, 0, 14, 6, 0] = multiSearchAllPositionsUTF8(materialize('нсцннйрмщфбшщховвццбдеишиохл'), ['цннйр', 'сцннйрм', 'н', 'нс', 'двтфхйзгеиеиауимбчхмщрцутф', 'пчтмшйцзсфщзшгнхщсутфжтлпаввфгххв', 'лшмусе', 'ховвццбд', 'йрмщфбшщховвццбдеи', 'гндруущрфзсфжикшзцжбил']) from system.numbers limit 10; +select [0, 18, 0, 1, 2, 0, 0, 0, 1, 7, 10, 0, 1, 0, 2, 0, 0, 18] = multiSearchAllPositionsUTF8(materialize('щидмфрсготсгхбомлмущлаф'), ['тлтфхпмфдлуоцгчскусфжчкфцхдухм', 'мущла', 'емлвзузхгндгафги', '', 'идмфрсготсгхбомлмущла', 'зфаргзлщолисцфдщсеайапибд', 'кдхоорхзжтсйимкггйлжни', 'лчгупсзждплаблаеклсвчвгвдмхклщк', 'щидмфр', 'сготсгхбомлму', 'тсгхбомлмущла', 'хсзафйлкчлди', '', 'й', 'ид', 'щлйпмздйхфзайсщсасейлфцгхфк', 'шдщчбшжбмййзеормнрноейй', 'мущ']) from system.numbers limit 10; +select [0, 13, 0, 0, 1, 0, 7, 7, 8, 0, 2, 0, 3, 0, 0, 13] = multiSearchAllPositionsUTF8(materialize('трцмлщввадлжввзчфипп'), ['хшзйийфжмдпуигсбтглй', 'ввзчфи', 'нсцчцгзегммтсшбатщзузпкшрг', 'гувйддежзфилйтш', '', 'хгзечиа', 'ввадлжввз', 'ввадлжввзчфи', 'ва', 'щтшсамклегш', 'рцмлщ', 'учзмиерфбтцучйдглбщсз', 'цмлщввадлжввзчфи', 'орйжччцнаррбоабцжзйлл', 'квпжматпцсхзузхвмйч', 'ввзчфип']) from system.numbers limit 10; +select [0, 1, 1, 0, 11, 4, 1, 2, 0, 0] = multiSearchAllPositionsUTF8(materialize('инкщблбвнскцдндбмсщщш'), ['жхрбсусахрфкафоилмецчебржкписуз', 'инкщблбвнс', '', 'зисгжфлашймлджинаоджруй', 'кцднд', 'щблбвнскцдндбмсщщ', 'инкщблбвнс', 'н', 'зб', 'фчпупшйфшбдфенгитатхч']) from system.numbers limit 10; +select [6, 0, 4, 20, 1, 0, 5, 0, 1, 0] = multiSearchAllPositionsUTF8(materialize('рзтецуйхлоорйхдбжашнларнцт'), ['у', 'бпгййекцчглпдвсцсещщкакцзтцбччввл', 'ецуйхлоо', 'нлар', 'рз', 'ккнжзшекфирфгсгбрнвжчл', 'цуйхлоорйхдбжашн', 'йнучгрчдлйвводт', 'рзте', 'нткрввтубчлщк']) from system.numbers limit 10; + +select [1, 1, 0, 0, 1, 0, 0, 3, 3, 3, 1, 0, 8, 0, 8, 1, 0, 1] = multiSearchAllPositionsCaseInsensitive(materialize('OTMMDcziXMLglehgkklbcGeAZkkdh'), ['', 'OTmmDCZiX', 'SfwUmhcGTvdYgxlzsBJpikOxVrg', 'ngqLQNIkqwguAHyqA', '', 'VVZPhzGizPnKJAkRPbosoNGJTeO', 'YHpLYTVkHnhTxMODfABor', 'mMdcZi', 'MmdCZI', 'MMdCZixmlg', '', 'hgaQHHHkIQRpPjv', 'ixMLgLeHgkkL', 'uKozJxZBorYWjrx', 'i', '', 'WSOYdEKatHkWiCtlwsCbKRnXuKcLggbkBxoq', '']) from system.numbers limit 10; +select [4, 15, 0, 0, 0, 0, 5, 0, 5, 1, 0, 1, 13, 0, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('VcrBhHvWSFXnSEdYCYpU'), ['bhhVwSfXnSEd', 'DycyP', 'kEbKocUxLxmIAFQDiUNoAmJd', 'bsOjljbyCEcedqL', 'uJZxIXwICFBPDlUPRyDHMmTxv', 'BCIPfyArrdtv', 'hHv', 'eEMkLteHsuwsxkJKG', 'hHVWsFxNseDy', '', 'HsFlleAQfyVVCoOSLQqTNTaA', '', 'sEDY', 'UMCKQJY', 'j', 'rBhHvw']) from system.numbers limit 10; +select [1, 1, 0, 0, 1, 0, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('wZyCLyiWnNNdNAPWeGSQZcdqk'), ['w', '', 'vlgiXgFTplwqRbnwBumAjHvQuM', 'QoIRVKDHMlapLNiIZXvwYxluUivjY', 'WZY', 'gAFpUfPDAwgzARCIMrtbZUsNcR', 'egkLWqqdNiETeETsMG', 'dzSlJaoHKlQmENIboow', 'vPNBhcaIfsgLH', 'mlWPTCBDVTdKHxlvIUVcJXBrmTcJokAls']) from system.numbers limit 10; +select [0, 10, 0, 1, 7, 1, 6, 1, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('pqliUxqpRcOOKMjtrZSEsdW'), ['YhskuppNFdWaTaZo', 'Coo', 'mTEADzHXPeSMCQaYbKpikXBqcfIGKs', 'PQLiUxq', 'qpRCoOK', 'PQLIu', 'XQPrcoOK', '', 'pR', 'cTmgRtcSdRIklNQVcGZthwfarLtAYh']) from system.numbers limit 10; +select [16, 1, 1, 1, 1, 4, 17, 0, 0, 0, 1, 0, 0, 0, 20, 0] = multiSearchAllPositionsCaseInsensitive(materialize('kJyseeDFCeUWoqMfubYqJqWA'), ['fub', 'kJY', '', '', 'Kj', 's', 'uBYQJq', 'sUqCmHUZIBtZPswObXSrYCwrdxdznM', 'mtZDCJENYuikJnCcJfRcSCDYDPXU', 'IDXjRjHhmjqXmCOlQ', '', 'jiEwAxIsJDu', 'YXqcEKbHxlgUliIALorSKDMlGGWeCO', 'OstKrLpYuASEUrIlIuHIRdwLr', 'qJq', 'tnmvMTFvjsW']) from system.numbers limit 10; +select [11, 3, 1, 0, 9, 0, 0, 0, 0, 8, 3, 0] = multiSearchAllPositionsCaseInsensitive(materialize('EBSPtFpDaCIydASuyreS'), ['iyD', 'sptfpdAciyDAsuyR', 'EbS', 'IJlqfAcPMTUsTFXkvmtsma', 'AcIYda', 'fbWuKoCaCpRMddUr', 'srlRzZKeOQGGLtTLOwylLNpVM', 'ZeIgfTFxUyNwDkbnpeiPxQumD', 'j', 'daciydA', 'sp', 'dyGFtyfnngIIbcCRQzphoqIgIMt']) from system.numbers limit 10; +select [6, 0, 0, 0, 10, 0, 1, 4, 0, 15, 0, 2, 2, 6] = multiSearchAllPositionsCaseInsensitive(materialize('QvlLEEsgpydemRZAZcYbqPZHx'), ['eSgpYDEMRzAzcyBQPzH', 'NUabuIKDlDxoPXoZOKbUMdioqwQjQAiArv', 'pRFrIAGTrggEOBBxFmnZKRPtsUHEMUEg', 'CDvyjef', 'YdEMrzaZc', 'BO', '', 'leEsgPyDEmRzaZCYBqPz', 'EzcTkEbqVXaVKXNuoxqNWHM', 'Z', 'cuuHNcHCcLGb', 'V', 'vllEes', 'eS']) from system.numbers limit 10; +select [0, 0, 0, 0, 0, 3, 0, 0, 0, 0, 5, 7, 5, 0, 11, 1] = multiSearchAllPositionsCaseInsensitive(materialize('eiCZvPdGJSmwxMIrZvEzfYFOFJmV'), ['lSydrmJDeXDYHGFFiFOOJGyCbCCDbLzbSbub', 'ewsAVflvcTBQFtvWBwuZOJKkrUArIg', 'fpEkBWaBkRWypFWtMz', 'YatSURyNtcSuerWWlTBSdBNClO', 'YO', 'CZvpdg', 'uoH', 'gtGwQSVqSJDVROmsBIxjuVNfrQnxDhWGXLBH', 'IKNs', 'HElLuRMlsRgINaNp', 'V', 'DGjsMW', 'vPDgJSmW', 'SGCwNiAmNfHSwLGZkRYEqrxBTaDRAWcyHZYzn', 'mWXMiRZvezfYf', '']) from system.numbers limit 10; +select [23, 1, 0, 17, 0, 0, 9, 3, 0, 2] = multiSearchAllPositionsCaseInsensitive(materialize('BizUwoENfLxIIYVDflhOaxyPJw'), ['yPJ', '', 'gExRSJWtZwOptFTkNlBGuxyQrAu', 'FLH', 'hCqo', 'oVGcArersxMUCNewhTMmjpyZYAIU', 'FlXIiYVdflHoAX', 'ZuWOe', 'bhfAfNdgEAtGdHylxkjgvU', 'IZUWo']) from system.numbers limit 10; +select [0, 9, 0, 0, 0, 0, 1, 0, 0, 1, 3, 0, 13, 0, 3, 5] = multiSearchAllPositionsCaseInsensitive(materialize('loKxfFSIAjbRcguvSnCdTdyk'), ['UWLIDIermdFaQVqEsdpPpAJ', 'ajBrcg', 'xmDmuYoRpGu', 'wlNjlKhVzpC', 'MxIjTspHAQCDbGrIdepFmLHgQzfO', 'FybQUvFFJwMxpVQRrsKSNHfKyyf', '', 'vBWzlOChNgEf', 'DiCssjczvdDYZVXdCfdSDrWaxmgpPXDiD', '', 'kxFFSIAjBRCGUVSNcD', 'LrPRUqeehMZapsyNJdu', 'cGuVSNcdTdy', 'NmZpHGkBIHVSoOcj', 'KxffSIAjBr', 'ffsIaJB']) from system.numbers limit 10; +select [14, 0, 11, 0, 10, 0, 0, 0, 13, 1, 2, 11, 5, 0] = multiSearchAllPositionsCaseInsensitive(materialize('uijOrdZfWXamCseueEbq'), ['sE', 'VV', 'AmcsEu', 'fUNjxmUKgnDLHbbezdTOzyLaknQ', 'XAmCsE', 'HqprIpxIcOTkDIKcVK', 'NbmirQlNsTHnAVKlF', 'VVDNOxFKSnQGKPsTqgtwLhZnIPkL', 'c', '', 'IJ', 'aM', 'rDzF', 'YFwP']) from system.numbers limit 10; +select [0, 8, 17, 0, 1, 0, 0, 0, 0, 0, 5, 0] = multiSearchAllPositionsCaseInsensitive(materialize('PzIxktujxHZsaDlwSGQPgvA'), ['zrYlZdnUxlPrVJJeZEASwdCHlNEm', 'jxhZS', 'sGQPgV', 'MZMChmRBgsxhdgspUhALoxmrkZVp', 'pzIxktuJxHzsADlw', 'xavwOAibQuoKg', 'vuuETOrWLBNLhrMeWLgGQpeFPdcWmWu', 'TZrAgmdorqZIdudhyCMypHYKFO', 'ztcCyGxRKrcUTv', 'OUvwdMZrcZuwGtjuEBeGU', 'k', 'rFTpnfGIOCfwktWnyOMeXQZelkYwqZ']) from system.numbers limit 10; +select [3, 1, 4, 1, 0, 17, 13, 0, 0, 0, 0, 0, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('pUOaQLUvgmqvxaMsfJpud'), ['OaqLUvGm', '', 'aQ', '', 'VajqJSlkmQTOYcedjiwZwqNH', 'f', 'xaMsfj', 'CirvGMezpiIoacBGAGQhTJyr', 'vucKngiFjTlzltKHexFVFuUlVbey', 'ppalHtIYycBCEjsgsXbFeecpkQMNr', 'nEgIYVoGkhTsFgBUSHJvIcYCYbuOBP', 'efjBVRVzknGrikGHxExlFEtYf', 'v', 'QgRBCaGlwNYWRslDylOrfPxZxAOF']) from system.numbers limit 10; +select [14, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 1, 0, 20, 5, 0, 4, 0] = multiSearchAllPositionsCaseInsensitive(materialize('WZNWOCjFkCAAzIptkUtyPCyC'), ['iPTkuT', 'BngeNlFbKymzMYmNPfV', 'XKEjbLtADFMqS', 'dbRQKJGSFhzljAiZV', 'wZnwoCjFKCAAzIPTKuTYpc', 'yBaUvSSGOEL', 'iEYopROOYKxBwPdCgbPNPAsMwVksHgagnO', 'TljXPJVebHqrnhSiTGwpMaNeKy', 'wzNWocjF', 'bLxLrZnOCeIfxkfZEOcqDteUvc', 'CtHYpAZDANEv', '', 'XMAMpGYMiOb', 'y', 'o', 'floswnnFjXDTxantSvDYPSnaORL', 'WOcjFkcAaZIp', 'buqBHbZsLDnCUDhLdgd']) from system.numbers limit 10; +select [0, 20, 14, 0, 2, 0, 1, 14, 0, 0, 0, 1, 0, 26, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('XJMggEHaxfddDadtwKMCcPsMlSFVJ'), ['NzbUAZvCsnRnuzTglTsoT', 'ccP', 'ADTwKmc', 'JaUzcvWHMotuEMUtjsTfJzrsXqKf', 'jMGgEHaXfdddAdTWKMCcpsM', 'SMnb', '', 'AdTWkMccPSMlsfv', 'fVjPVafkp', 'goqsYAFqhhnCkGwhg', 'CNHNPZHZreFwhRMr', '', 'vcimNhmdbtoiCgVzNuvdgZG', 'sfvJ', 'AqKmroxmRMSFAKjfhwrzxmNSSjMHxKow', 'Xhub']) from system.numbers limit 10; +select [0, 0, 7, 0, 1, 1, 0, 0, 13, 0, 1, 1, 5, 0] = multiSearchAllPositionsCaseInsensitive(materialize('VQuEWycGbGcTcCCvWkujgdoWjKgVYy'), ['UevGaXmEAtBdWsPhBfqp', 'aQOrNMPmoVGSu', 'c', 'TMhzvbNJCaxtGNUgRBmTFEqgNBIBpSJ', '', 'vq', 'pVNUTCqXr', 'QSvkansbdPbvVmQpcQXDk', 'cCCvwkUjgdOWjKgVYy', 'EtCGaEzsSbJ', 'V', '', 'WycgBgCTCcCvwkujgdoWJKgv', 'xPBJqKrZbZHJawYvPxgqrgxPN']) from system.numbers limit 10; +select [4, 1, 0, 0, 0, 0, 0, 0, 0, 18] = multiSearchAllPositionsCaseInsensitive(materialize('LODBfQsqxfeNuoGtzvrUMRVWNKUKKs'), ['Bf', 'lOdbfQs', 'ZDSDfKXABsFiZRwsebyU', 'DT', 'GEUukPEwWZ', 'GNSbrGYqEDWNNCFRYokZbZEzGzc', 'kYCF', 'Kh', 'jRMxqdmGYpTkePeReXJNdnxagceitMJlmbbro', 'VrumrvWnKU']) from system.numbers limit 10; +select [1, 1, 3, 1, 10, 0, 9, 2, 2, 0, 0, 0, 0, 0, 8, 0, 1, 11, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('lStPVtsQypFlZQoQhCuP'), ['', '', 'tpV', 'L', 'PF', 'pGPggwbkQMZandXugTpUorlPOubk', 'yPFlz', 'sTPVTsQyPfLzQOqhCU', 'StPVtSq', 'cbCxBjAfJXYgueqMFNIoSguFm', 'AosIZKMPduRfumDZ', 'AGcNTHObH', 'oPaGpsQ', 'kwQCczyY', 'q', 'HHUYdzGAzVJyn', '', 'fLZQoqHcUp', 'q', 'SSonzfqLVwIGzdHtj']) from system.numbers limit 10; +select [0, 1, 2, 0, 0, 0, 13, 1, 27, 1, 0, 1, 3, 1, 0, 1, 3, 0] = multiSearchAllPositionsCaseInsensitive(materialize('NhKJtvBUddKWpseWwRiMyBsTWmlk'), ['toBjODDZoRAjFeppAdsne', '', 'HKjTvBu', 'QpFOZJzUHHQAExAqkdoBpSbXzPnTzuPd', 'gE', 'hLmXhcEOwCkatUrLGuEIJRkjATPlqBjKPOV', 'Ps', 'NH', 'l', '', 'aSZiWpmNKfglqAbMZpEwZKmIVNjyJTtDianY', 'NhKJTvBUDDkwpS', 'KJtvbUDDKWPSewwrimYbstwm', 'NHKJTvbudDKwpSEwwR', 'hmMeWEpksVAaXd', 'NHkJTvBUDd', 'kjTvbudd', 'kmwUzfEpWSIWkEylDeRPpJDGb']) from system.numbers limit 10; +select [0, 5, 0, 0, 0, 1, 1, 15, 2, 3, 4, 5] = multiSearchAllPositionsCaseInsensitive(materialize('NAfMyPcNINKcgsShJMascJunjJva'), ['ftHhHaJoHcALmFYVvNaazowvQlgxwqdTBkIF', 'yp', 'zDEdjPPkAdtkBqgLpBfCtsepRZScuQKbyxeYP', 'yPPTvdFcwNsUSeqdAUGySOGVIhxsJhMkZRGI', 'JQEqJOlnSSam', 'nAFmy', '', 'sHJmaScjUnJj', 'afmY', 'FmYpcnINKCg', 'MYPCniNkcgSS', 'YPCNiNkCgSsHjmasCJuNjJ']) from system.numbers limit 10; +select [0, 0, 6, 3, 2, 0, 8, 2, 2, 10, 0, 0, 14, 0, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('hgpZVERvggiLOpjMJhgUhpBKaN'), ['Nr', 'jMcd', 'e', 'PZVeRvggiLOPjmjh', 'GpZVe', 'cVbWQeTQGhYcWEANtAiihYzVGUoHKH', 'VGgilOPj', 'GPZVervgGiLopjmjHGuHp', 'GP', 'gil', 'fzwDPTewvwuCvpxNZDi', 'gLLycXDitSXUZTgwyeQgMSyC', 'PJmjh', 'bTQdrFiMiBtYBcEnYbKlqpTvGLmo', 'ggHxiDatVcGTiMogkIWDxmNnKyVDJth', 'pzv']) from system.numbers limit 10; +select [7, 1, 9, 3, 0, 0, 2, 0, 1, 11] = multiSearchAllPositionsCaseInsensitive(materialize('xUHVawrEvgeYyUZGmGZejClfinvNS'), ['RevGeYyuz', 'XUHvAWrev', 'Vg', 'hvawR', 'eRQbWyincvqjohEcYHMwmDbjU', 'nuQCxaoxEdadhptAhZMxkZl', 'UhVAwREvGEy', 'lHtwTFqlcQcoOAkujHSaj', '', 'eYYUzgMgzEjCLfIn']) from system.numbers limit 10; +select [0, 0, 8, 5, 9, 1, 0, 4, 12, 6, 4, 0, 0, 12] = multiSearchAllPositionsCaseInsensitive(materialize('DbtStWzfvScJMGVPQEGkGFoS'), ['CSjYiEgihaqQDxZsOiSDCWXPrBdiVg', 'aQukOYRCSLiildgifpuUXvepbXuAXnYMyk', 'fvsCjmgv', 'TWZFV', 'VscjMgVpQ', 'dBtSTwZfVsCjmGVP', 'wqpMklzJiEvqRFnZYMfd', 'StwZfVScJ', 'j', 'wzfVsCjmGV', 'STWZfVS', 'kdrDcqSnKFvKGAcsjcAPEwUUGWxh', 'UtrcmrgonvUlLnzWXvZI', 'jMgvP']) from system.numbers limit 10; +select [0, 0, 0, 0, 7, 3, 0, 11, 1, 10, 0, 0, 7, 1, 4, 0, 17, 3, 15, 0] = multiSearchAllPositionsCaseInsensitive(materialize('YSBdcQkWhYJMtqdEXFoLfDmSFeQrf'), ['TnclcrBJjLBtkdVtecaZQTUZjkXBC', 'SPwzygXYMrxKzdmBRTbppBQSvDADMUIWSEpVI', 'QnMXyFwUouXBoCGLtbBPDSxyaLTcjLcf', 'dOwcYyLWtJEhlXxiQLRYQBcU', 'KWhYjMtqdEXFo', 'BD', 'nnPsgvdYUIhjaMRVcbpPGWOgVjJxoUsliZi', 'j', '', 'YjmtQdeXF', 'peeOAjH', 'agVscUvPQNDwxyFfXpuUVPJZOjpSBv', 'kWh', '', 'dcQKWHYjmTQD', 'qjWSZOgiTCJyEvXYqaPFqbwvrwadJsGVTOhD', 'xfoL', 'b', 'DeXf', 'HyBR']) from system.numbers limit 10; +select [4, 0, 0, 13, 1, 0, 3, 13, 16, 1, 0, 1, 16, 1, 12, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('SoVPMQNqmaTGuzYxDvZvapSuPiaP'), ['pMqNQMAtGuzYxDVz', 'TEJtgLhyredMnIpoZfmWvNwpkxnm', 'XRWmsfWVOCHhk', 'u', '', 'HvkXtxFdhVIyccpzFFSL', 'VPM', 'uZyXDVzvAPsUpIaP', 'xDvzV', 'sovpmqNQmATguZYx', 'wEG', 'soVPmQnQ', 'XDVzV', '', 'GUZyXdvzva', 'FetUahWwGtwEpVdlJCJntL', 'B', 'lSCUttZM']) from system.numbers limit 10; +select [1, 0, 1, 2, 15, 0, 0, 0, 1, 0] = multiSearchAllPositionsCaseInsensitive(materialize('zFWmqRMtsDjSeWBSFoqvWsrV'), ['', 'GItrPyYRBwNUqwSaUBpbHJ', '', 'f', 'BsfOQvWsR', 'JgvsMUZzWaddD', 'wxRECkgoCBPjSMRorZpBwuOQL', 'xHKLLxUoWexAM', '', 'YlckoSedfStmFOumjm']) from system.numbers limit 10; +select [11, 1, 1, 1, 0, 0, 1, 0, 4, 0, 0, 0, 1, 0, 5, 8] = multiSearchAllPositionsCaseInsensitive(materialize('THBuPkHbMokPQgchYfBFFXme'), ['KpqGchyfBF', '', '', 'TH', 'NjnC', 'ssbzgYTybNDbtuwJnvCCM', 'tHbupKHBMOkPQgcHy', 'RpOBhT', 'uPKHbMoKpq', 'oNQLkpSKwocBuPglKvciSjttK', 'TaCqLisKvOjznOxnTuZe', 'HmQJhFyZrcfeWbXVXsnqpcgRlg', 'tHB', 'gkFGbYje', 'pkhbMokPq', 'Bm']) from system.numbers limit 10; +select [7, 10, 0, 0, 9, 0, 0, 3, 0, 10] = multiSearchAllPositionsCaseInsensitive(materialize('ESKeuHuVsDbiNtvxUrfPFjxblv'), ['uvsDBiNtV', 'DbInTvxu', 'YcLzbvwQghvrCtCGTWVuosE', 'cGMNo', 'SDb', 'nFIRTLImfrLpxsVFMBJKHBKdSeBy', 'EUSiPjqCXVOFOJkGnKYdrpuxzlbKizCURgQ', 'KeUHU', 'gStFdxQlrDcUEbOlhLjdtQlddJ', 'DBInTVx']) from system.numbers limit 10; +select [1, 0, 2, 18, 1, 3, 15, 8, 0, 0, 1, 3, 0, 23, 2, 0, 8, 0] = multiSearchAllPositionsCaseInsensitive(materialize('TzczIDSFtrkjCmDQyHxSlvYTNVKjMT'), ['', 'AmIFsYdYFaIYObkyiXtxgvnwMVZxLNlmytkSqAyb', 'ZcZI', 'HXsLVYTnvKjm', '', 'CZiDsFtRKJ', 'DQYhxSl', 'fTRKjCmdqYHxsLvYtNvk', 'hxVpKFQojYDnGjPaTNPhGkRFzkNhnMUeDLKnd', 'RBVNIxIvzjGYmQBNFhubBMOMvInMQMqXQnjnzyw', '', 'c', 'vcvyskDmNYOobeNSfmlWcpfpXHfdAdgZNXzNm', 'ytnvKJM', 'ZcZidsFtRKjcmdqy', 'IRNETsfz', 'fTR', 'POwVxuBifnvZmtBICqOWhbOmrcU']) from system.numbers limit 10; +select [14, 16, 10, 2, 6, 1, 0, 8, 0, 0, 12, 1, 0, 1, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('tejdZOLhjpFLkGBWTGPfmk'), ['GBWtgPF', 'Wt', 'PflkgBWTgpFmK', 'ejdZOLhJPFlKgb', 'o', 'TejDZ', 'HlQfCP', 'hJP', 'ydiyWEfPGyRwcKGfGVdYxAXmkY', 'QsOyrgkTGMpVUAmLjtnWEIW', 'LKGBw', 'tejDzolHJpFLKgbWT', 'IK', '', 'WrzLpcmudcIJEBapkToDbYSazKTwilW', 'DmEWOxoieDsQHYsLNelMc']) from system.numbers limit 10; +select [9, 0, 1, 4, 13, 0, 0, 1, 3, 7, 9, 0, 1, 1, 0, 7] = multiSearchAllPositionsCaseInsensitive(materialize('ZWHpzwUiXxltWPAIGGxIcJB'), ['XxLTWpA', 'YOv', '', 'pzwUIXXl', 'wp', 'lpMMLDAuflLnWMFrETXRethzCUZOWfQ', 'la', '', 'HPZ', 'UixxlTw', 'xXLTWP', 'YlfpbSBqkbddrVwTEmXxgymedH', '', '', 'QZWlplahlCRTMjmNBeoSlcBoKBTnNZAS', 'UiXxlTwPAiGG']) from system.numbers limit 10; +select [0, 9, 6, 0, 4, 0, 3, 0, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('NytxaLUvmiojEepjuCzwUYPoWL'), ['LcOnnmjbZSifx', 'm', 'lUvMIOjeE', 'vuZsNMSsutiLCDbClPUSsrziohmoZaQeXtKG', 'XaLuvm', 'hlUevDfTSEGOjvLNdRTYjJQvMvwrMpwy', 'TXALuVmioJeePjUczw', 'pKaQKZg', 'PAdX', 'FKLMfNAwNqeZeWplTLjd', 'DODpbzUmMCzfGZwfkjH', 'HMcEGRHLspYdJIiJXqwjDUBp']) from system.numbers limit 10; +select [2, 1, 0, 16, 8, 1, 6, 0, 0, 1, 8, 0, 7, 0, 9, 1, 1, 0, 0, 1] = multiSearchAllPositionsCaseInsensitive(materialize('WGVvkXuhsbzkLqiIEOuyiRfomy'), ['GVv', '', 'VbldWXHWzdziNcJKqIkDWrO', 'iEOUyIRFomy', 'hsBZklqiieOuy', '', 'X', 'emXjmIqLvXsNz', 'rxhVkujX', 'wgvvK', 'HsBzKLQiie', 'wVzJBMSdKOqjiNrXrfLEjjXozolCgYv', 'UHsbzklQiiEouyirf', 'UOvUsiKtUnwIt', 'SBZKLqiIEoUYIrfom', 'wg', '', 'BefhETEirL', 'WyTCSmbKLbkQ', '']) from system.numbers limit 10; +select [8, 1, 2, 8, 1, 0, 5, 0, 0, 4, 0, 1, 14, 0, 0, 7, 0, 1] = multiSearchAllPositionsCaseInsensitive(materialize('uyWhVSwxUFitYoVQqUaCVlsZN'), ['XufitYOVqqUACVlszn', '', 'ywH', 'XUFIT', 'uywHvSWXuFIt', 'dGhpjGRnQlrZhzGeInmOj', 'vswXuFitYovqQuA', 'dHCfJRAAQJUZeMJNXLqrqYCygdozjAC', 'rojpIwYfNLECl', 'hVswxufiTYov', 'bgJdgRoye', '', 'ovQ', 'AdVrJlq', 'krJFOKilvBTGZ', 'WxuFITYOV', 'AsskQjNPViwyTF', 'u']) from system.numbers limit 10; +select [0, 2, 0, 0, 0, 6, 0, 5, 0, 15, 0, 0, 3, 0] = multiSearchAllPositionsCaseInsensitive(materialize('BEKRRKLkptaZQvBxKoBL'), ['HTwmOxzMykTOkDVKjSbOqaAbg', 'eKrRKl', 'UrLKPVVwK', 'TyuqYmTlQDMXJUfbiTCr', 'fyHrUaoMGdq', 'KLkPtaZq', 'cPUJp', 'RKLk', 'yMnNgUOpDdP', 'BX', 'tXZScAuxcwYEfSKXzyfioYPWsrpuZz', 'dsiqhlAKbCXkyTjBbXGxOENd', 'k', 'juPjORNFlAoEeMAUVH']) from system.numbers limit 10; +select [9, 0, 0, 0, 1, 4, 2, 0, 0, 0, 0, 8, 0, 2, 0, 3, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('PFkLcrbouhBTisTkuUcO'), ['UhBtistKU', 'ioQunYMFWHD', 'VgYHTKZazRtfgRtvywtIgVoBqNBwVn', 'ijSNLKch', 'pFKlcrBOuhbtIsTku', 'lCRboUHBtI', 'fKLCRBOu', 'XTeBYUCBQVFwqRkElrvDOpZiZYmh', 'KzXfBUupnT', 'OgIjgQO', 'icmYVdmekJlUGSmPLXHc', 'OuH', 'BWDGzBZFhTKQErIRCbtUDIIjzw', 'F', 'LuWyPfSdNHIAOYwRMFhP', 'kL', 'PQmvXDCkEhrlFBkUmRqqWBxYi', 'kLcrbo']) from system.numbers limit 10; +select [0, 1, 1, 6, 14, 3, 0, 1, 9, 1, 9, 0, 1, 10, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('pfynpJvgIjSqXWlZzqSGPTTW'), ['ZzeqsJPmHmpoYyTnKcWJGReOSUCITAX', '', 'P', 'jvGIj', 'wLZzQsgP', 'YnPjVGij', 'DmpcmWsyilwHwAFcKpLhkiV', '', 'I', 'pFy', 'IjsqxwLZzqSgpT', 'pKpe', 'PfynpJvgiJSqXwlzZ', 'jsQXwLZZqs', 'onQyQzglEOJwMCO', 'GV']) from system.numbers limit 10; +select [1, 17, 1, 20, 0, 0, 5, 0, 0, 0, 24, 0] = multiSearchAllPositionsCaseInsensitive(materialize('BLNRADHLMQstZkAlKJVylmBUDHqEVa'), ['bLnRaDhLm', 'kJVYlmbuD', 'bLnr', 'yLMbU', 'eAZtcqAMoqPEgwtcrHTgooQcOOCmn', 'jPmVwqZfp', 'aDHlmqS', 'fmaauDbUAQsTeijxJFhpRFjkbYPX', 'aqIXStybzbcMjyDKRUFBrhfRcNjauljlqolfDX', 'WPIuzORuNbTGTNb', 'uDhqeVa', 'fQRglSARIviYABcjGeLK']) from system.numbers limit 10; +select [2, 0, 4, 5, 1, 15, 1, 9, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('BEwjMzphoTMoGikbrjTVyqDq'), ['E', 'sClgniMsZoGTEuLO', 'jmzphotmoGIKBRjtv', 'MZPhOtmo', '', 'Kb', '', 'otm', 'tVpxYRttoVpRLencV', 'SJAhAuMttGaeMsalRjeelAGG']) from system.numbers limit 10; +select [1, 0, 0, 0, 0, 0, 4, 0, 0, 19, 0, 7] = multiSearchAllPositionsCaseInsensitive(materialize('yNnYRQfcyemQdxUEPOiwRn'), ['', 'SJteoGNeIAMPWWBltkNKMrWDiVfR', 'kKnnKQhIPiekpnqTXJuyHfvWL', 'GPDUQEMWKzEEpvjLaIRYiuNfpzxsnSBX', 'oPrngRKwruyH', 'ukTSzFePSeVoeZeLQlAaOUe', 'yRqfcyemQDXUepo', 'CwmxidvpPHIbkJnVfSpbiZY', 'FUxmQdFVISApa', 'iwr', 'ciGHzDpMGNQbytsKpRP', 'Fcy']) from system.numbers limit 10; +select [0, 1, 0, 11, 2, 0, 1, 3, 0, 0, 0, 21] = multiSearchAllPositionsCaseInsensitive(materialize('EgGWQFaRsjTzAzejYhVrboju'), ['DVnaLFtCeuFJsFMLsfk', '', 'thaqudWdT', 'Tzazejy', 'GGW', 'RolbbeLLHOJpzmUgCN', '', 'gwqfarsjtzaZeJYHvR', 'KkaoIcijmfILoe', 'UofWvICTEbwVgISstVjIzkdrrGryxNB', 'UJEvDeESWShjvsJeioXMddXDkaWkOiCV', 'B']) from system.numbers limit 10; +select [0, 5, 2, 0, 0, 7, 0, 0, 0, 11, 0, 12, 22, 10, 0, 12] = multiSearchAllPositionsCaseInsensitive(materialize('ONgpDBjfRUCmkAOabDkgHXICkKuuL'), ['XiMhnzJKAulYUCAUkHa', 'dbj', 'nGpDbJFRU', 'xwbyFAiJjkohARSeXmaU', 'QgsJHnGqKZOsFCfxXEBexQHrNpewEBFgme', 'JFruCM', 'DLiobjNSVmQk', 'vx', 'HYQYzwiCArqkVOwnjoVNZxhbjFaMK', 'Cm', 'ckHlrEXBPMrVIlyD', 'M', 'xI', 'UcmkAOabdKg', 'jursqSsWYOLbXMLQAEhvnuHclcrNcKqB', 'mKaoaBdKghxiCkkUUL']) from system.numbers limit 10; +select [0, 1, 0, 1, 0, 0, 0, 0, 7, 21] = multiSearchAllPositionsCaseInsensitive(materialize('WhdlibCbKUmdiGbJRshgdOWe'), ['kDPiHmzbHUZB', '', 'CukBhVOzElTdbEBHyrspj', '', 'QOmMle', 'wiRqgNwjpdfgyQabxzksjg', 'RgilTJqakLrXnlWMn', 'bSPXSjkbypwqyazFLQ', 'CBkuMDiGbJRShGdOWe', 'dow']) from system.numbers limit 10; +select [0, 8, 0, 1, 1, 0, 1, 7, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('oOccAoDDoPzHUyRqdWhJxNmATEqtE'), ['LFuvoQkVx', 'DoPzh', 'YaBSTdWvmUzlgRloppaShkRmLC', 'oO', '', 'eeEpOSLSXbyaOxTscOPoaTcKcchPmSGThk', '', 'dDO', 'oFXmyIJtmcSnebywDlKruvPUgmPFzEnMvA', 'vCs', 'MsxHLTgQcaQYZdPWJshIMWbk', 'yqrjIzvrxd']) from system.numbers limit 10; +select [0, 16, 0, 0, 0, 0, 7, 1, 0, 0, 1, 2, 1, 4, 0, 3] = multiSearchAllPositionsCaseInsensitive(materialize('FtjOSBIjcnZecmFEoECoep'), ['FQQwzxsyauVUBufEBdLTKKSdxSxoMFpL', 'EOecoEP', 'HGWzNTDfHxLtKrIODGnDehl', 'ZxirLbookpoHaxvASAMfiZUhYlfuJJN', 'mKh', 'GZaxbwVOEEsApJgkLFBRXvmrymSp', 'Ij', '', 'X', 'AnCEVAe', 'fTj', 'tjOSbIjcNZECMfeoEC', '', 'OsBIjcN', 'LtdJpFximOmwYmawvlAIadIstt', 'JOsBiJCNzEc']) from system.numbers limit 10; +select [0, 2, 0, 0, 19, 0, 0, 12, 1, 0, 3, 1, 0, 0] = multiSearchAllPositionsCaseInsensitive(materialize('ugpnWWncvqSLsYUCVXRZk'), ['yOWnQmZuhppRVZamgmRIXXMDQdeUich', 'gPNww', 'jlyFSbvmjaYPsMe', 'fQUeGVxgQdmPbVH', 'rZk', 'ariCX', 'grAffMPlefMQvugtAzN', 'LsYuCVX', '', 'jZFoQdWEWJFfSmNDqxIyNjvxnZJ', 'P', 'UgPN', 'JmKMsbegxNvusaiGGAZKglq', 'qArXLxzdYvabPv']) from system.numbers limit 10; +select [0, 0, 0, 0, 0, 0, 8, 0, 0, 1, 1, 15, 0, 1, 7, 0] = multiSearchAllPositionsCaseInsensitive(materialize('nxwotjpplUAXvoQaHgQzr'), ['ABiEhaADbBLzPwhSfhu', 'TbIqtlkCnFdPgvXAYpUuLjqnnDjDD', 'oPszWpzxuhcyuWxiOyfMBi', 'fLkacEEeHXCYuGYQXbDHKTBntqCQOnD', 'GHGZkWVqyooxtKtFTh', 'CvHcLTbMOQBKNCizyEXIZSgFxJY', 'PlUAxVoQah', 'zrhYwNUzoYjUSswEFEQKvkI', 'c', 'NXWOt', '', 'qAhG', 'JNqCpsMJfOcDxWLVhSSqyNauaRxC', '', 'PpLuaxV', 'DLITYGE']) from system.numbers limit 10; +select [2, 0, 0, 1, 0, 0, 28, 1, 16, 1] = multiSearchAllPositionsCaseInsensitive(materialize('undxzJRxBhUkJpInxxJZvcUkINlya'), ['ndxzjRxbhuKjP', 'QdJVLzIyWazIfRcXU', 'oiXcYEsTIKdDZSyQ', 'U', 'dRLPRY', 'jTQRHyW', 'Y', '', 'nxxJZVcU', '']) from system.numbers limit 10; +select [1, 4, 1, 0, 4, 1, 0, 1, 16, 1, 0, 0, 0, 8, 12, 14, 0, 2] = multiSearchAllPositionsCaseInsensitive(materialize('lrDgweYHmpzOASVeiFcrDQUsv'), ['', 'gwEYhMP', 'LrDGwEyHmPzOaSVEifC', 'oMN', 'gwEYhMpZO', 'lrdGWEy', 'pOKrxN', 'lrDgwEyhmpZoaSv', 'eifcrdqU', 'LrDgw', 'dUvarZ', 'giYIvswNbNaBWprMd', 'pPPqKPhVaBhNdmZqrBmb', 'hmPzoASVEiF', 'O', 'SVEi', 'gIGLmHnctIkFsDFfeJWahtjDzjPXwY', 'rDGweyHmP']) from system.numbers limit 10; +select [0, 0, 11, 1, 1, 1, 0, 16, 0, 1, 5, 0, 0, 0, 2, 0, 2, 0] = multiSearchAllPositionsCaseInsensitive(materialize('XAtDvcDVPxZSQsnmVSXMvHcKVab'), ['bFLmyGwEdXiyNfnzjKxUlhweubGMeuHxaL', 'IhXOeTDqcamcAHzSh', 'ZSQsNMvsxmVHcK', '', '', '', 'dbrLiMzYMQotrvgwjh', 'MvsxMV', 'zMp', 'XaTDvCdvpXzsqSNMVSxm', 'v', 'LkUkcjfrhyFmgPXPmXNkuDjGYlSfzPi', 'ULpAlGowytswrAqYdaufOyWybVOhWMQrvxqMs', 'wGdptUwQtNaS', 'ATdVcdVPXzSqsnmVSXMvHcKVab', 'JnhhGhONmMlUvrKGjQcsWbQGgDCYSDOlor', 'atdvCdvpXzsqSnMVSxMVhCkvAb', 'ybNczkKjdlMoOavqBaouwI']) from system.numbers limit 10; +select [8, 0, 0, 0, 4, 0, 0, 5, 5, 2] = multiSearchAllPositionsCaseInsensitive(materialize('XPquCTjqgYymRuwolcgmcIqS'), ['qgyYMruW', 'tPWiStuETZYRkfjfqBeTfYlhmsjRjMVLJZ', 'PkTdqDkRpPpQAMksmkRNXydKBmrlOAzIKe', 'wDUMtn', 'UcTJQgYYMRuWoLCgMcI', 'PieFD', 'kCBaCC', 'Ct', 'C', 'pQuctjqgyymRuwOLCgmc']) from system.numbers limit 10; + +select [1, 0, 7, 1, 0, 24, 17, 0, 0, 0, 2, 0, 1, 7, 4, 1, 12, 8] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('гГБаДнФбпнЩврЩшЩЩМщЕБшЩПЖПчдт'), ['', 'таОХхрзИДжЛСдЖКЧжБВЩжЛкКХУКждАКРеаЗТгч', 'Ф', '', 'ЙЩИФМфАГщХзКЩЧТЙжмуГшСЛ', 'ПЖпчдТ', 'ЩМщЕбшЩПжПч', 'ФгА', 'гУД', 'зУцкжРоППЖчиШйЗЕшаНаЧаЦх', 'гбаДНФбПНЩВРЩШЩщМЩеБшЩпжПЧд', 'РДЧЖАбрФЦ', 'гГ', 'ФбпНщвр', 'адНфБПнщвРщШщщМщЕбШщ', 'ггб', 'ВРЩ', 'бПНщврЩш']) from system.numbers limit 10; +select [0, 12, 8, 0, 12, 0, 0, 10, 0, 8, 4, 6] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('айРВбЧБжКВИхБкчФЖЖНВнпФйФБДфЗ'), ['ЛрЦфуУДВК', 'хБкчфЖжНвнпфйфБдФ', 'жКВИХБкчФЖжНвнПф', 'кЖчвУцВСфЗБТИфбСжТИдРкшгзХвщ', 'хбк', 'штДезйААУЛчнЖофМисНЗо', 'нлнШЧВЙхОПежкцевчлКрайдХНчНб', 'вИХбкчфжжНВН', 'ЩдзЦТуоЛДСеШГфЦ', 'ЖКВихбКЧфжЖ', 'вбЧбЖкВихБкЧфЖжНВ', 'Чб']) from system.numbers limit 10; +select [18, 15, 0, 0, 0, 0, 5, 0, 14, 1, 0, 0, 0, 0, 0, 15] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('пМИОкоЗжГйНТПЙацччЧАЩгЕВБбЕ'), ['ЧЧАЩгЕВБ', 'а', 'ФбРВщшййпХдфаЗЖлЛСЗПРШПпАОинЧКзЩхждН', 'ЛфРКДЙВСУСЙОчтнИкРЗбСГфкЩреИхЛлчХчШСч', 'ШйвБПАдФдФепЗТкНУрААйеЧПВйТоЧмБГДгс', 'ФтЙлЖЕсИАХИФЗаЕМшсшуцлцАМФМгбО', 'КО', 'лиШБнлпОХИнБаФЩдмцпжЗИЛнвсЩЙ', 'йацччЧАщгевбБЕ', 'ПмИоКозжГйНТП', 'ИГНннСчКАИСБщцП', 'ПнжмЙЛвШтЩейХЛутОРЩжифбЗчгМУЛруГпх', 'ХжЗПлГЖЛйсбпрЩОТИеБвулДСиГзлЛНГ', 'учклЦНЕгжмщлжАшщжМд', 'ЩеПОЙтЖзСифОУ', 'АЦЧ']) from system.numbers limit 10; +select [10, 0, 1, 1, 6, 1, 7, 6, 0, 0, 0, 2, 12, 0, 6, 0, 4, 8, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('квхБнцхйзЕпйИмтЙхфзвгдФ'), ['еПйИМт', 'хгкиМжСБИТНщенЩИщНСкй', '', 'Квхб', 'цхЙЗЕПйИмТйХФЗ', 'к', 'хйЗЕПЙИмтй', 'Цх', 'нКлШбМЖГйШкРзадрЛ', 'ДштШвБШТг', 'СЦКйЕамЦщПглдСзМлоНШарУтМднЕтв', 'ВхБнцхйЗЕПйимТ', 'йимтЙХФЗВГД', 'жчссунЙаРцМкЖУЦщнцОЕхнРж', 'цХЙЗЕП', 'ОгНФдМЛПТИдшцмХИеКйРЛД', 'бнЦхЙ', 'ЙЗе', 'згЩищШ', 'фХлФчлХ']) from system.numbers limit 10; +select [0, 0, 0, 12, 0, 0, 27, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('хНпсРТХВдтоЦчдлеФПвнЛгЗКлПйнМВ'), ['ШиБфЗШПДЧхОЩшхфщЗЩ', 'иГйСЧЗтШЛуч', 'АЗХЦхедхОуРАСВЙС', 'цчдЛЕфП', 'СДбйГйВЕРмЙЩЛщнжен', 'НДлцСфТшАщижгфмуЖицжчзегЕСЕНп', 'й', '', 'йлчМкРИЙиМКЙжссЦТцРГзщнхТмОР', 'ПРцГувЧкйУХггОгЖНРРсшГДрлЧНжГМчрХЗфЧЕ']) from system.numbers limit 10; +select [0, 0, 2, 0, 10, 7, 1, 1, 0, 9, 0, 2, 0, 17, 0, 0, 0, 6, 5, 2] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЙзЗжпжДЕСУхчйдттСЙзоЗо'), ['щОЙУшееЧщкхГККреБкВ', 'жВ', 'ззЖпждЕсУХчЙДТТсЙ', 'ЙЦШЦЙЖзХШРвнкЕд', 'УхчйДтТсйЗОз', 'дЕСу', '', '', 'дсцеррищндЗдНкжаНЦ', 'сУхчЙдттсйзОзО', 'ЦЖРжмц', 'ЗЗ', 'СгЛГАГЕЖНгщОеЖЦДмБССцЩафзЗ', 'Сйзоз', 'ЦГХТЕвЕЗБМА', 'пмВоиеХГжВшдфАЖАшТйуСщШчИДРЙБнФц', 'Оа', 'ждЕ', 'ПжДесу', 'ЗзЖПждЕСУ']) from system.numbers limit 10; +select [0, 0, 0, 0, 5, 1, 0, 6, 0, 1, 17, 15, 1, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('уФШЙбШТоХВбзЦцЖОЕКТлщхнЖГ'), ['цЛ', 'ууМ', 'ТИгЙолМФсибтЕМнетквЦИЩИБккйн', 'оФОаМогсХЧЦооДТПхб', 'бШтОХВбЗцЦЖоЕКтЛ', 'уфШйбШтоХ', 'фдтщрФОЦсшигдПУхЛцнХрЦл', 'ШтО', 'НИкИТрбФБГИДКфшзЕмЙнДЖОсЙпЩцщкеЖхкР', 'уфШЙБш', 'екТлщ', 'ЖоекТл', 'уфШйБшТоХвбз', 'ТуОхдЗмгФеТаафЙм']) from system.numbers limit 10; +select [0, 1, 6, 1, 0, 1, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чМЩБЛЛПРлщкВУбПефХВФлАЗШао'), ['гаТкЛВнрвдПМоеКПОйр', 'ч', 'ЛпрЛЩКвуБпе', 'ЧмЩб', 'ц', '', 'жгаччЖйГЧацмдсИИВЩЩжВЛо', 'йГеЙнБзгнкЦЛБКдОЕЧ', 'ПоЦРвпЕЗСАШж', 'ЙОНЦОбиееО']) from system.numbers limit 10; +select [2, 0, 17, 1, 0, 0, 0, 5, 0, 4, 0, 0, 0, 0, 0, 2] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЕаЩичщМЦЖиЗБЛЧжуНМЧК'), ['АЩиЧЩ', 'ИлУсшДБнжщаатуРТтраПОЙКЩйТГ', 'НМЧк', 'Еа', 'зАВФЛЩбФрМВШбПФГгВЕвЖббИТйе', 'РЗНРБЩ', 'ЦдЙНГпефзЛчпУ', 'ч', 'НШШчПЗР', 'ИчЩмЦжИЗБлЧЖУНМч', 'аннвГДлмОнТЖЗЙ', 'ШдчЩшЕБвхПУсШпг', 'гФИШНфЖПжймРчхАБШкЖ', 'ЖзгЖАБлШЗДпд', 'Д', 'ащиЧ']) from system.numbers limit 10; +select [4, 1, 0, 7, 0, 7, 1, 1, 0, 3, 7, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('иОцХКЙвувМИжШдУМУЕйНсБ'), ['ХкйвуВмИжШдУм', '', 'звМАОМЩщЙПшкиТчЩдгТЦмфзеИ', 'вуВМиж', 'КДщчшЙВЕ', 'в', '', 'ИоЦхКЙВувМижШ', 'ЕвТАРи', 'цхКЙвувмИЖШДумуе', 'вУвМи', 'зПШИХчУщШХУвврХйсуЙЗеВЧКНмКШ']) from system.numbers limit 10; +select [0, 5, 0, 0, 0, 0, 0, 12, 0, 11] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЦОфбчУФсвТймЦчдщгЩжИАБ'), ['йлрк', 'ЧуФсвтйМцчдЩгщ', 'МНлЕжорв', 'иНзТЖМсмх', 'шЕМЖжпИчсБжмтЧЙчщФХб', 'жШХДнФКАЩГсОЩвЕаам', 'НпКЦХулЛвФчШЕЗкхХо', 'мЦчДЩгЩжиАб', 'мпцгВАЕ', 'Й']) from system.numbers limit 10; +select [1, 0, 0, 0, 8, 0, 2, 0, 0, 7] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чТХЙНщФфцИНБаеЖкОвлиУДР'), ['', 'рВХмжКцНцИЙраштМппсодЛнЧАКуЩ', 'ИХфХЖЧХВкзЩВЙхчфМрчдтКздиОфЙжУ', 'Гзлр', 'фЦи', 'абПф', 'тХЙНщффЦИн', 'нссГбВеЖх', 'амлЗщрсУ', 'фФ']) from system.numbers limit 10; +select [0, 9, 11, 0, 11, 1, 0, 0, 0, 1, 6, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('зДЗпщАцвТгРдврщхЩфЖл'), ['йХЛ', 'Т', 'рд', 'АИЦщгниДфВОе', 'Р', 'здзпщ', 'вКТвВШмгч', 'ввирАйбЗЕЕНПс', 'тХиХоОтхПК', '', 'аЦВТгРДврщ', '', 'уЗЗЖвУЕйтчудноЕКМЖцВРаНТЙЗСОиЕ', 'оЕфПхЕДжАаНхЕцЖжжофЦхкШоБЙр']) from system.numbers limit 10; +select [1, 1, 0, 0, 1, 7, 0, 0, 0, 2] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('йЛПЛшмЦШНЖРрЧрМцкЖзЕНжЧДелФжАн'), ['', 'йЛПлшМЦшНЖррч', 'ПНКдфтДейуиШзЗХАРУХизВ', 'ПценмщЧОФУСЙЖв', '', 'ЦшнжрРчрМЦКЖЗе', 'МрПзЕАгжРбТЧ', 'ЕДФмаФНвТЦгКТЧЦжцЛбещЛ', 'УтПУвЛкТасдЦкеИмОещНИАоИжЖдЛРгБЩнвЖКЛЕП', 'Л']) from system.numbers limit 10; +select [1, 5, 1, 1, 0, 0, 1, 1, 0, 2, 19, 0, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('сйДпмжнДРщКБгфцЖОчтГНБ'), ['', 'МЖнДРщ', 'Сй', '', 'пУщ', 'йгВИАЦнозаемТиХВвожКАПТдкПИаж', 'Сйд', 'СЙДпмжНдРщ', 'ФПщБцАпетаЙФГ', 'ЙдпМжНдрЩКбГфЦжОЧТГНб', 'т', 'гллрБВМнвУБгНаЙцМцТйЙФпзЧОЙЛвчЙ', 'йДПМжндРЩкБ', 'ЗмфОмГСНПщшЧкиССдГБУсчМ']) from system.numbers limit 10; +select [0, 18, 10, 5, 0, 2, 8, 1, 4, 11] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ХпИРддХрмВНйфчвгШиЧМКП'), ['хЗФДлДУБЙаЦтжРБЗсуйнЦпш', 'иЧмК', 'внЙ', 'д', 'зиМУЩГиГ', 'ПИр', 'РМвнЙфчвгШич', '', 'РдДхРМ', 'нЙфчВГШИ']) from system.numbers limit 10; +select [18, 0, 0, 1, 0, 0, 6, 0, 0, 9] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('нГгФкдуФШуИТбшпХфтаГт'), ['Таг', 'рРпшУйчГд', 'гК', '', 'лаВНбездпШШ', 'ЕБРйаНрОБожкКИсв', 'ДУфШУитБ', 'ГРиГШфШтйфЖлРФзфбащМЗ', 'мхЩжЛнК', 'ШуИтБШ']) from system.numbers limit 10; +select [13, 0, 0, 7, 0, 15, 0, 0, 15, 0, 0, 5, 6, 0, 18, 21, 11, 1] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('рлобшдПЦИхжФуХщжгПФукшзт'), ['УхщжГ', 'ТВщЦфФсчЩГ', 'ЕжФШойжуЛРМчУвк', 'пцИХжфуХЩж', 'бР', 'щЖГПфуКШЗТ', 'йжРГгЛуШКдлил', 'ТщЖГкбШНИщЩеЩлаАГхрАфЙНцЦгВкб', 'щжГПфУ', 'бкаДБЛХ', 'АЗ', 'шДПЦихжфух', 'дП', 'вфнЙобСцвЩмКОбЦсИббФКзЩ', 'пФУкшзТ', 'К', 'жфу', '']) from system.numbers limit 10; +select [12, 19, 8, 1, 0, 0, 0, 15, 0, 0, 12, 2, 0, 4, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЦкЛЗепкЕХЩГлКФрБдТрлвйАхдООШ'), ['лК', 'рЛв', 'Ехщ', '', 'еаПКБгЦЩАоЗВонйТЗгМхццСАаодМЕЩГ', 'ишОНиеБидфбФБЖриУЩЩ', 'дуж', 'РбДТ', 'пЗсГХКсгРущкЙРФкАНЩОржФвбЦнЩНЖЩ', 'щрОУАГФащзхффКвЕйизцсйВТШКбнБПеОГ', 'лкФрБдТРлвЙа', 'КЛзеп', 'УЛФЗРшкРщзеФуМвгПасШЧЛАЦр', 'зеПКеХщглкфР', 'ЦЖЗдХеМЕ', 'зЖжрт', 'уЩФрйрЖдЦз', 'МфцУГЩтвПАЦжтМТоеищЕфнЖй']) from system.numbers limit 10; +select [0, 0, 1, 0, 1, 0, 0, 7, 0, 5, 1, 6, 1, 1, 1, 5, 6, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('РННЕШвжМКФтшДЙлфЛИзЙ'), ['ГаМРош', 'Дтфс', '', 'еБбиаКщГхххШвхМЖКзЛАезФУчХо', 'РНн', 'сВбТМ', 'ЖЗЦПБчиСйе', 'жМкфтШДЙл', 'нЖХуеДзтЧтулиСХпТпеМлИа', 'ШВжМкФТШдЙлфл', '', 'вЖМКфТ', '', '', '', 'швЖМКфтШДЙЛфлИЗй', 'вЖмКФТ', 'еМ']) from system.numbers limit 10; +select [0, 0, 15, 1, 0, 0, 8, 1, 0, 0, 0, 4, 8, 10] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('РиучГийдХутДЕЙДпфиуд'), ['ЩмгцлЖрц', 'ЕСжСлЩЧИЖгЗЛлф', 'дП', '', 'щГЦаБтПШВзЦСрриСЙбД', 'тдРгОЛТШ', 'д', '', 'КЕбЗКСХЦТщЦДЖХпфаЧйоХАл', 'мТвзелНКрЖЧЦПпЕЙвдШтеШйБ', 'ЙОТКрБСШпШд', 'ЧГ', 'ДХУТДЕЙд', 'УТд']) from system.numbers limit 10; +select [0, 0, 0, 0, 15, 0, 0, 0, 11, 0, 0, 5, 1, 1, 0, 2, 3, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('МшазшргхОПивОлбмДоебАшцН'), ['ЦИшштН', 'еМСЗкФЕКДйОНМ', 'ЛСчГрбеРЕбЩМПМЗЦИп', 'ХнИПЧжЗдзФщЗ', 'бмдоЕ', 'гМОдйсбТСЦЩбФВЗШзшщбчегаЕмЕБаХаРР', 'фщнР', 'щмТчФчсМАОгчБщшг', 'иВ', 'УщцГОшТзпУХКоКЖБеМШ', 'мйаАЛцАегСмПОаСТИСфбЧДБКоИВчбЦЙ', 'шРгхоп', '', '', 'еИпАЩпнЛцФжЩХИрЧаИИТЛвшиСНЩ', 'шаЗ', 'АЗ', 'ФгдтфвКЩБреногуир', 'ДБжШгщШБЩпЖИЛК', 'ЧдРЩрбфЛзЙклхдМСФУЙЛн']) from system.numbers limit 10; +select [5, 0, 0, 18, 13, 0, 2, 7, 0, 0, 1, 15, 1, 0, 0, 0, 3, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('хщеКЗПчуНЙтрЧЩгфСбоКЕАДТййАрр'), ['зп', 'хчПЦшпДбзСфНВЧзНжЕМФОП', 'ЧЖхЕУк', 'БОКеАдтЙЙа', 'чЩГфС', 'шллддЩщеМжШйкЩн', 'щЕкзпЧуНЙТ', 'ЧунйтРЧщгФс', 'ввНздЙуоТЖРаВЙчМИчхРвфЛЖБН', 'ЗХМХПщПкктцАзщЙкдпжф', '', 'ГФСбОкеАДтйЙа', '', 'МБХВЕчпБМчуххРбнИМЛТшЩИщЙгаДцзЛАМвйаО', 'ЛкОзц', 'ЕцпАДЗСРрсЕвтВщДвцбЗузУннТИгХжхрцПДРДПм', 'екЗПЧунЙТРчщгФсбоК', 'шпИфЕчгШжцГВСйм', 'ЛхйЧбЧД', 'ВзЗоМцкЩНХГж']) from system.numbers limit 10; +select [0, 0, 6, 20, 0, 10, 0, 0, 0, 9, 10, 3, 23, 1, 0, 0, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('лцапШиХчЛДшдксСНИбшгикзчЙанми'), ['ХууатТдтбодМГЧгщЧнклШтЗПНчкЦОаЙг', 'МЦЧчпИхКЛаФхщХдРУДщжУчфлжахц', 'иХЧлдшдкСсНИбШГикзЧЙ', 'гикЗчйА', 'ГсТзЛОфИББлекЩАсЛвмБ', 'Д', 'ЦХрТЖощНрУШфнужзжецсНХВфЩБбДУоМШШиГйж', 'йуВдЕзоггПВДЖб', 'ЙфБГйХМбжоакЖЛфБаГИаБФСнБЖсТшбмЗЙТГОДКИ', 'ЛДШдКССНИБшГикзч', 'ДШдКССниБ', 'аПШИХчЛДШДКсс', 'з', '', 'ФоохПЩОГЖоУШлКшзЙДоуп', 'хАДХЩхлвУИсшчрбРШУдФА', 'ЦА', 'гвптУФлчУуРхпрмЖКИрБеЩКчН']) from system.numbers limit 10; +select [0, 4, 5, 7, 15, 3, 3, 17, 7, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('зЗАЩлЕЕЕПИохЧчШвКЧйрсКХдд'), ['пКРбуШОНТЙБГНзИРвЖБсхрЛщчИрлЧУ', 'ЩЛЕЕЕПиоХЧ', 'ЛеЕеп', 'Еепио', 'швкЧйрС', 'ащЛеееПИох', 'АЩлеЕЕпиОхЧЧШвкЧЙРсК', 'КчйРскхД', 'ЕЕПИохччшВКчй', 'у']) from system.numbers limit 10; +select [1, 12, 0, 8, 1, 1, 0, 1, 5, 0, 1, 0, 0, 0, 0, 3, 1, 0, 4, 5] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ПмКСйСКЖККмШеоигЙчПфжТ'), ['', 'Шео', 'РчвлдЙЙлПщуКмтН', 'жкКмшЕоИГЙЧ', '', '', 'йРмМЖнПиЙ', '', 'йс', 'тфФРСцл', '', 'щлЩХиКсС', 'кпнТЖпФЩиЙЛ', 'абкКптбИВгмЧкцфЦртЛДЦФФВоУхЗБн', 'чНшоВСГДМйДлтвфмхХВВуеЩЦВтЖтв', 'кС', '', 'фидБлйеЙЧШРЗЗОулщеЕЩщЙсЙшА', 'СЙс', 'йсКжкКМшЕо']) from system.numbers limit 10; +select [0, 0, 1, 0, 2, 2, 1, 2, 7, 0, 1, 2, 1, 0, 6, 8] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('УгЖЕугАЩХйчидаррлжНпфФГшр'), ['утвШ', 'кЕвФч', 'угжеУг', 'тШлТвЕШЗчЖеЛНджЦазЩХцж', 'гЖеугаЩхй', 'ГжЕугаЩХйЧидАР', 'УгжЕУГаЩХЙЧИда', 'гЖеу', 'ащхЙчИ', 'мЧлщгкЛдмЙЩРЧДИу', '', 'ГжеугАщХйЧиДаРРЛЖНП', '', 'зЕМвИКбУГКЩФшоГЧГ', 'ГАЩХйчИДАррлЖНпФфг', 'ЩХЙчИдАррЛЖНпфФгш']) from system.numbers limit 10; +select [1, 0, 0, 7, 0, 6, 0, 11, 0, 0, 0, 2, 0, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЗЕГЛЩПцГНтзЕЦШЧхНКГТХЙЙФШ'), ['', 'шзкиЗсаИщАБмаз', 'Ж', 'ц', 'гШуЕжЛСПодРнхе', 'пцГНтЗЕЦ', 'щРкЩАеНржЙПМАизшщКвЗщглТкКИф', 'ЗеЦшчхнКГтхЙЙ', 'пелгЩКкцвтфнжЖУуКосЙлкЛ', 'рф', 'хНШчНрАХМШщфЧкЩБНзХУкилЙмП', 'ЕгЛЩПЦгнтзецШЧ', 'ЩУчБчРнЖугабУоиХоИККтО', 'СГмЦШтФШЛмЙЩ', 'ауТПЛШВадоХМПиБу', 'ЩЩйр']) from system.numbers limit 10; +select [2, 2, 1, 0, 0, 0, 0, 0, 1, 0, 7, 9, 0, 15, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('гЙЧЙФХнЖБвомгАШГбОВГксИйцз'), ['ЙЧйфхНЖбвО', 'Й', 'гЙЧйфхнЖбв', 'хсЩмШЙЙММВЦмУБТчгзУЛР', 'зктшп', 'дЕоиЖлгШж', 'хКкаНЛБ', 'ЗКйСчсоЗшскГЩбИта', '', 'у', 'НжбВОмгашГ', 'БВо', 'ещфРШлчСчмаЖШПЧфоК', 'шгбо', 'ЙСтШШДЩшзМмдпЧдЙЖевТвоУСЕп', 'Л']) from system.numbers limit 10; +select [0, 9, 0, 0, 18, 13, 13, 11, 0, 0, 4, 1] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЙЛмоЦСдТаоФчШКЖЦСНРаРЦзоС'), ['ДфгЗАасВфаМмшхчлмР', 'аоФчШкЖцСнРАРЦзОС', 'зЩзнйтФРТЙжУлхФВт', 'чЦкШВчЕщДУМкхЛУЩФшА', 'н', 'Шк', 'шКЖцсНРаРцЗос', 'фчшкЖцснрАРЦз', 'лку', 'пЧШМЦквоемЕщ', 'о', 'йЛМоцСДТАофЧшкжЦСнРаРЦзос']) from system.numbers limit 10; +select [21, 0, 0, 17, 1, 11, 0, 2, 0, 7] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('кЧЖнЕбМЛпШЗХиЙжиМщлнСФрПЧЖВН'), ['сФ', 'гцХаШЛсаШЛкшфЧОКЛцзешХСиЩоаЕОш', 'Г', 'МщЛНСФРпч', '', 'зХ', 'ОАДепНпСГшгФАЦмлуНуШШЗфдЧРШфрБЛчРМ', 'чЖне', 'СфЕАбФн', 'М']) from system.numbers limit 10; +select [4, 0, 1, 1, 0, 2, 4, 16, 3, 6, 5, 0, 0, 6, 1, 0, 5, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('кдАпЩСШИСцРхтеСиФЖЧСсОоц'), ['пщСшиСцрХТЕсифЖчССоОц', 'рхнкикДТКДВШчиЖЦнВм', '', '', 'жПЛСнЦцн', 'дА', 'ПщсШИсцрХтЕс', 'иФжЧсСоОЦ', 'ап', 'с', 'щсШИ', 'МАзашДРПЩПзРТЛАсБцкСШнЕРЙцИЩлТЛеУ', 'ичцпДбАК', 'сшИСЦрхтЕсифжчСсООц', 'КдАПЩСшИСЦРХТЕсИфЖЧСсо', 'ЛнБсИПоМЩвЛпиЩЗЖСд', 'щс', 'шщДНБаСщЗАхкизжнЛАХЙ']) from system.numbers limit 10; +select [0, 13, 0, 2, 16, 1, 3, 0, 9, 0, 2, 0, 1, 4, 0, 0, 0, 1] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('иНхеЕкхЩщмгзМГхсгРБсхОКцУгуНБ'), ['ДиоУлФЖЛисУСЕтсЕалщн', 'МгХсгрБСХО', 'ЖХНцршПшгйО', 'нХЕЕкхЩ', 'сГРбсхОКцУг', '', 'х', 'Ж', 'щМгЗмгхСг', 'СрпхДГОУ', 'НхеЕкХщ', 'ПМтБцЦЙЖАЙКВБпФ', 'ИнхеЕ', 'еЕКхЩ', 'мМГлРзш', 'гтдоЙБСВещкЩАЩЦйТВИгоАЦлчКнНРНПДЖшСЧа', 'ЖшеН', '']) from system.numbers limit 10; +select [1, 5, 0, 0, 3, 0, 2, 0, 14, 14, 1, 0, 17, 13, 3, 25] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('айлзсЗБоГйтГжЙРККФхКшлНРОрЦкфо'), ['', 'с', 'Д', 'шиБраНИЦЧуИжп', 'Лз', 'ДРБСУфКСшцГДц', 'йЛЗСЗбОгЙтГЖйРК', 'ЕЙЦсвРЕШшщЕЗб', 'ЙркКфхкшЛнРОР', 'ЙРкКФхкШ', 'а', 'ГдоДКшСудНл', 'КФхКшлНРоР', 'ж', 'лзСзБогйТГЖйрККф', 'оР']) from system.numbers limit 10; +select [6, 0, 8, 10, 1, 0, 1, 13, 0, 0, 0, 2, 2, 0, 4, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('РучУлрХчЗУпИчДТЕфщИЙщрНлн'), ['РХЧ', 'оДсГСЛЙшйиЧРСКзчХВоХарцНШ', 'ЧЗУпИ', 'УПичдТе', 'Р', 'ВЙЩхжАутПСНЦфхКщеЩИуЧдчусцАесзМпмУв', '', 'ЧдТ', 'ООсШИ', 'ФШсВжХтБУШз', 'ЕЩуДдшкМУРЕБшщпДОСАцйауи', 'УЧ', 'УЧУЛрХчзуПИчдТеФщий', 'йнЦцДСхйШВЛнШКМСфмдЩВйлнеЖуВдС', 'улрхчзупиЧдтефщИ', 'СХТЧШшГТВвлЕИчНОВи']) from system.numbers limit 10; +select [0, 0, 0, 2, 1, 1, 0, 1, 19, 0, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('УецжлЦЦщМшРГгЩЩдБмхЖЗЧзШЙб'), ['НзИуАузуРЗРуКфоТМмлПкрсмЕЕЕнТ', 'ЕЩГХхЧш', 'ХоЙпООчфЖввИжЙшЖжЕФОтБхлВен', 'ЕЦЖЛЦцщ', '', '', 'ухогСИФвемдпаШЗуЛтлизОЧ', 'УецЖ', 'ХЖзЧЗ', 'П', 'мБкзХ', 'уБуОБхШ']) from system.numbers limit 10; +select [6, 1, 15, 5, 0, 0, 0, 3, 2, 4, 0, 12, 0, 2, 0, 3, 1, 6, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ГЖФеачМаКчПСпкВкхсПтг'), ['чмАкЧ', '', 'ВкХс', 'ачМА', 'КлтжУлОЛршБЕблФЩ', 'тцуМфж', 'л', 'фе', 'Жф', 'ЕАЧМак', 'лЖЕРТнФбЧЙТййвзШМСплИхбЙЛЖзДпм', 'СпкВК', 'ЩзчжИш', 'жФеАчМ', 'КбЦбйЕШмКтЩЕКдуЩтмпИЕВТЖл', 'ФЕаЧмАКчПСПквкхспТ', 'гжФеАЧмаКчпСп', 'ЧмАК', 'дцкДННМБцйЕгайхшжПГх', 'ТЩбвЦЖАНшрАШФДчОщй']) from system.numbers limit 10; +select [1, 6, 0, 1, 0, 0, 3, 1, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('МФННЧйОнцЛИЧЕПШПЧйоГФО'), ['', 'йОн', 'шУлгИЛЛРЙАсфЗоИЙЗРхуПбОЙсшдхо', 'МФННчЙоНц', 'лзВжбЦзфкзтуОйзуЗ', 'ЖГДщшЦзсжщцЦЖеЧвРфНИНОСАОщг', 'ННчйОНЦлИчЕПШ', '', 'Ф', 'ЩрИдНСлЙуАНЗвЕчмчАКмФУипндиП']) from system.numbers limit 10; +select [5, 0, 8, 13, 0, 0, 0, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('зВйймХЩМзЦГЕкЕКфоСтхПблуКМхц'), ['МХщмз', 'НАНрШоНДмурМлО', 'мзцгЕкек', 'кеКфоСтХПбЛУК', 'СУУксО', 'ЦоШжЧфйШЦаГЧйбЛШГЙггцРРчт', 'НбтвВбМ', '', 'тЩФкСтоСЧЦЦЙаСДЩСГЙГРИФЗОЗфбТДЙИб', 'ВГж']) from system.numbers limit 10; +select [0, 0, 0, 8, 19, 0, 3, 12, 1, 4] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ДпбЙЖНЗбПнЛбахБаХТуабШ'), ['цИаЩвгеИР', 'Ф', 'РЖиА', 'БпнЛб', 'У', 'Тфн', 'Б', 'БА', '', 'ЙЖНзБПнлбАхбаХ']) from system.numbers limit 10; +select [0, 0, 0, 0, 0, 1, 0, 17, 1, 0, 1, 1, 1, 11, 0, 1, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ТЦмЩОинХзоДДпПНЩигрРщОзКц'), ['ЕжЙВпПл', 'ВКфКТ', 'ШкДсЖхшфоПИадУбхФЩБчОАкпУеБхи', 'НТЕЙОШЦЖоЩбзВзшс', 'учГгуКФзлУдНУУуПУлкаЦЕ', '', 'фАПМКуЧйБЧзСоЗргШДб', 'ИГРрщОзк', '', 'йупОМшУйзВиВрЛЩЕеЩмп', '', '', '', 'дДППнщИгРР', 'ШФвИЧакеЦвШ', 'ТцМЩоинхЗОДдппнЩ', 'мрОгЩшЩеЧ', 'еЖРиркуаОТсАолЩДББВАМБфРфпШшРРРм']) from system.numbers limit 10; +select [3, 0, 0, 0, 0, 0, 1, 0, 0, 14, 0, 1, 0, 1, 1, 1, 0, 7] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('аОкиЛгКйхаОГОУзЦЛрбцш'), ['кИЛГкйхАогоУЗЦл', 'щЧДпХИхбпсГвфДФХкчХ', 'ШвАмБЗлДОИПткжхФТФН', 'щфсхФмЦсЛеувЙО', 'лВУЖц', 'еИщРшозЖАдцтКииДУлДОУФв', 'а', 'ХгЦРШ', 'ФзрЖкРЗЩЧИеЧцКФИфЧЧжаооИФк', 'уЗ', 'фЦФдцжжМчЗЖлиСЧзлщжжЦт', '', 'МдхжизИХфвбМААрйФНХдЕжп', 'аОкиЛг', 'АОКИЛгкйХАОГОУЗЦ', '', 'МбЖйрсумщиеОЩк', 'КйХАоГоУЗцлРБЦШ']) from system.numbers limit 10; +select [0, 0, 2, 1, 0, 0, 12, 0, 17, 0, 0, 0, 2, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('КУчЛХФчЛХшвбМЦинРвНрФМРкмиеЕп'), ['ТБЩБзхАмщПщЧПИФПашгЕТиКЦМБМпСЩСуЩМчтшеш', 'йлВЕЙшфшаШЗШЩВХЦчЛБс', 'УЧл', '', 'ЛДсЖщмНЦсКуфЗуГиука', 'РТТОТфГЕлЩЕгЛтДфлВЖШГзЦЖвнЗ', 'БМцИНРвнРф', 'ОЕИЕдИсАНаифТПмузЧчЖфШЕуеЩсслСШМоЖуЩЛМп', 'рвНРфМркМи', 'ЦзБМСиКчУжКУЩИИПУДвлбдБИОЙКТЛвтз', 'злСГе', 'ВдтцвОИРМЕжХО', 'учЛХфЧл', 'БшччШбУзЕТзфКпиШжнезвоеК']) from system.numbers limit 10; +select [0, 7, 0, 0, 0, 0, 7, 6, 0, 16, 12, 12, 15, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('оЖиогсфклШМСДрбхРбМбрЕщНЙЗйод'), ['иПмДКейууОклНХГЗсбаЙдШ', 'ФКлШмсДрБХРбМбрещНЙЗЙОд', 'арчжтСТнк', 'чбТНЛЕжооЗшзОУ', 'ощАЩучРСУгауДхГКлмОхЙцЕо', 'аЛбкиЦаКМбКхБМДнмФМкйРвРр', 'ФКлШмСДрбХРбм', 'СфклШ', 'еДйилкУлиИчХЙШтхцЗБУ', 'хрБ', 'СДрбХрбМБР', 'СдрбхРБ', 'бхрБМБРЕщНйз', 'КИб']) from system.numbers limit 10; +select [22, 1, 8, 0, 0, 1, 0, 3, 0, 6, 20, 0, 0, 0, 4, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('ЕЖДФбКужЙЦЦмсЖГГжБзеЙнПйЙри'), ['НПййР', '', 'Жй', 'Щ', 'ФхУО', 'ЕЖДфБКУЖйЦЦмСжГГ', 'НФЙзщЩГЧпфсфЦШОМЕЗгцрс', 'д', 'ЦтщДДЖтбвкгКонСк', 'кУЖЙЦЦм', 'ЕйНПййРИ', 'РчеЙйичФбдЦОтпчлТЖИлДучЙПгЗр', 'внчзшЗзОнФфхДгфзХТеНПШРшфБТЖДйф', 'кНснгмулМуГНурщЕББСузВмбнЧаХ', 'фбКУЖйЦцМсЖГгЖб', 'ЩСЕ']) from system.numbers limit 10; +select [0, 0, 0, 1, 10, 4, 0, 0, 5, 0, 1, 0, 7, 0, 3, 7, 0, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чБхлжгКЖХлЙнкКЦфжЕгЖАндЧ'), ['ПдмРрЖАтВнСдСБШпПЗГгшИ', 'цшцг', 'тчАЙЧОеЕАвГпЗцЖЧгдХуЛСЛНрвАЖщ', '', 'Лй', 'Л', 'ОйррцУжчуЦБАжтшл', 'вХУКк', 'жгКжхЛЙН', 'уцбЕЕОЧГКУПуШХВЕчГБнт', '', 'ПсАжБИКштЕаН', 'КжхлЙН', 'ЩгШухЦПАТКежхгХксгокбщФЙПсдТНШФЦ', 'Х', 'кЖХЛйНккЦФжЕГЖ', 'ЙзРДСПднаСтбЧЖхощ', 'пАПОУЧмИпслБЗПфУ']) from system.numbers limit 10; +select [0, 0, 0, 5, 2, 16, 4, 4, 11, 0, 0, 3, 3, 0, 0, 6] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('кпМаоуГГфвощолЦЩщЧПРОКепеА'), ['ЗзуФжНшщПТнЧЦКВОиАУсЧХОШбк', 'тмПкАпеайзуХсурШй', 'АЕЦавбШиСДвВДумВкиИУБШЕ', 'о', 'ПМаОУггФВощоЛЦЩЩЧПрокЕПеа', 'щЩ', 'аоУг', 'аОуГгФВ', 'оЩоЛЦЩщчПРОК', 'виХЛшчБсщ', 'УчАМаЦкйДЦфКСмГУЧт', 'мАоУ', 'МАО', 'щФФА', 'Н', 'У']) from system.numbers limit 10; +select [0, 3, 10, 8, 3, 0, 4, 0, 9, 4, 1, 9] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('щЙЧРпшИцхпргЦНуДййусЧЧнЖ'), ['ДлУцтееЖБКХгМзСВжА', 'чРпШИЦ', 'пргЦнУДЙЙУ', 'Ц', 'ЧРПш', 'нЩрЕвмрМеРйхтшЩче', 'РпШИЦхПРГцнУд', 'ПНоЙтПкоаОКгПОМЦпДЛФЩДНКПбСгЗНЗ', 'ХПРГцНудЙЙ', 'рПши', '', 'ХПРГ']) from system.numbers limit 10; +select [11, 4, 1, 0, 1, 0, 0, 0, 0, 12, 0, 9, 5, 0, 16, 0, 12, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('пкзщщЛНОНбфЦноИЧфхбФ'), ['ф', 'щщл', 'ПКзЩщЛНОн', 'ЩшФйЧБНДОИзМхеЖНЦцеЛлУЧ', '', 'сЗоЙТклйДШкДИЗгЖ', 'орЛФХПвБбУхНс', 'доЗмЩВу', 'ШиЕ', 'ЦНО', 'ндЩдРУЖШМпнзНссЖШДЦФвпТмуМЙйцН', 'НбФЦнОИч', 'ЩлНонБФ', 'ЛдРжКММЙм', 'чфх', 'ЦматДйиСфЦфааЦо', 'ЦНОИчФх', 'иржЦщн']) from system.numbers limit 10; +select [0, 0, 0, 0, 0, 1, 0, 0, 0, 1, 0, 1, 0, 3, 0, 5] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('чЖажцВбшЛттзДааАугШщАйПгщП'), ['ШгУТсчГОВЦЦеЛАСфдЗоЗЦВЛйлТДзчвЛва', 'УшЕшищЖткрвРСйиФЗйТФТЛЗаЗ', 'ВдикЙббщузоФХщХХГтЗоДпхбЕкМщц', 'срйеХ', 'рАшуПсЙоДнхчВкПЖ', '', 'гНЗбКРНСБВрАВФлнДШг', 'фХЧгмКнлПШлЩР', 'мкйЗбИФрЗахжгАдвЕ', 'чжаЖцВБШлТ', 'лХЕСрлПрОс', '', 'ЗЧПтчЙОцвОФУФО', 'ажцвБшЛТт', 'уНчЖШчМЕА', 'ц']) from system.numbers limit 10; +select [7, 1, 0, 7, 1, 19, 8, 6, 3, 0, 2, 13, 6, 0] = multiSearchAllPositionsCaseInsensitiveUTF8(materialize('НТКПпмБжДцбАКПНСЖоиТФД'), ['б', '', 'аУщЛМХЖбвИтНчГБМГдДнч', 'Б', 'НТкппм', 'и', 'Жд', 'МБждЦбАкП', 'кппмБждцБа', 'ПЕрнЦпМЦВгЧЧгГ', 'ткПпМБЖДцбаКпнСжО', 'кПнСЖоИ', 'МБжДцБакпН', 'гхОХжГуОвШШАкфКМщсшФДШеИжоАйг']) from system.numbers limit 10; + +select 0 = multiSearchAny(materialize('mpnsguhwsitzvuleiwebwjfitmsg'), ['wbirxqoabpblrnvvmjizj', 'cfcxhuvrexyzyjsh', 'oldhtubemyuqlqbwvwwkwin', 'bumoozxdkjglzu', 'intxlfohlxmajjomw', 'dxkeghohv', 'arsvmwwkjeopnlwnan', 'ouugllgowpqtaxslcopkytbfhifaxbgt', 'hkedmjlbcrzvryaopjqdjjc', 'tbqkljywstuahzh', 'o', 'wowoclosyfcuwotmvjygzuzhrery', 'vpefjiffkhlggntcu', 'ytdixvasrorhripzfhjdmlhqksmctyycwp']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('qjjzqexjpgkglgxpzrbqbnskq'), ['vaiatcjacmlffdzsejpdareqzy', 'xspcfzdufkmecud', 'bcvtbuqtctq', 'nkcopwbfytgemkqcfnnno', 'dylxnzuyhq', 'tno', 'scukuhufly', 'cdyquzuqlptv', 'ohluyfeksyxepezdhqmtfmgkvzsyph', 'ualzwtahvqvtijwp', 'jg', 'gwbawqlngzcknzgtmlj', 'qimvjcgbkkp', 'eaedbcgyrdvv', 'qcwrncjoewwedyyewcdkh', 'uqcvhngoqngmitjfxpznqomertqnqcveoqk', 'ydrgjiankgygpm', 'axepgap']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('fdkmtqmxnegwvnjhghjq'), ['vynkybvdmhgeezybbdqfrukibisj', 'knazzamgjjpavwhvdkwigykh', 'peumnifrmdhhmrqqnemw', 'lmsnyvqoisinlaqobxojlwfbi', 'oqwfzs', 'dymudxxeodwjpgbibnkvr', 'vomtfsnizkplgzktqyoiw', 'yoyfuhlpgrzds', 'cefao', 'gi', 'srpgxfjwl', 'etsjusdeiwbfe', 'ikvtzdopxo', 'ljfkavrau', 'soqdhxtenfrkmeic', 'ktprjwfcelzbup', 'pcvuoddqwsaurcqdtjfnczekwni', 'agkqkqxkfbkfgyqliahsljim']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('khljxzxlpcrxpkrfybbfk'), ['', 'lpc', 'rxpkrfybb', 'crxp', '', 'pkr', 'jxzxlpcrxpkrf', '', 'xzxlpcr', 'xpk', 'fyb', 'xzxlpcrxpkrfybbfk', 'k', 'lpcrxp', 'ljxzxlpcr', 'r', 'pkr', 'fk']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('rbrizgjbigvzfnpgmpkqxoqxvdj'), ['ee', 'cohqnb', 'msol', 'yhlujcvhklnhuomy', 'ietn', 'vgmnlkcsybtokrepzrm', 'wspiryefojxysgrzsxyrluykxfnnbzdstcel', 'mxisnsivndbefqxwznimwgazuulupbaihavg', 'vpzdjvqqeizascxmzdhuq', 'pgvncohlxcqjhfkm', 'mbaypcnfapltsegquurahlsruqvipfhrhq', 'ioxjbcyyqujfveujfhnfdfokfcrlsincjbdt', 'cnvlujyowompdrqjwjx', 'wobwed', 'kdfhaoxiuifotmptcmdbk', 'leoamsnorcvtlmokdomkzuo', 'jjw', 'ogugysetxuqmvggneosbsfbonszepsatq']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('uymwxzyjbfegbhgswiqhinf'), ['lizxzbzlwljkr', 'ukxygktlpzuyijcqeqktxenlaqi', 'onperabgbdiafsxwbvpjtyt', 'xfqgoqvhqph', 'aflmcwabtwgmajmmqelxwkaolyyhmdlc', 'yfz', 'meffuiaicvwed', 'hhzvgmifzamgftkifaeowayjrnnzw', 'nwewybtajv', 'ectiye', 'epjeiljegmqqjncubj', 'zsjgftqjrn', 'pssng', 'raqoarfhdoeujulvqmdo']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('omgghgnzjmecpzqmtcvw'), ['fjhlzbszodmzavzg', 'gfofrnwrxprkfiokv', 'jmjiiqpgznlmyrxwewzqzbe', 'pkyrsqkltlmxr', 'crqgkgqkkyujcyoc', 'endagbcxwqhueczuasykmajfsvtcmh', 'xytmxtrnkdysuwltqomehddp', 'etmdxyyfotfyifwvbykghijvwv', 'mwqtgrncyhkfhjdg', 'iuvymofrqpp', 'pgllsdanlhzqhkstwsmzzftp', 'disjylcceufxtjdvhy']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('mznihnmshftvnmmhnrulizzpslq'), ['nrul', 'mshftvnmmhnr', 'z', 'mhnrulizzps', 'hftvnmmhnrul', 'ihnmshftvnmmhnrulizzp', 'izz', '', 'uli', 'nihnmshftvnmmhnru', 'hnrulizzp', 'nrulizz']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('ruqmqrsxrbftvruvahonradau'), ['uqmqrsxrbft', 'ftv', 'tvruvahonrad', 'mqrsxrbftvruvahon', 'rbftvruvah', 'qrsxrbftvru', 'o', 'ahonradau', 'a', 'ft', '', 'u', 'rsxrbftvruvahonradau', 'ruvahon', 'bftvruvahonradau', 'qrsxrbftvru', 't', 'vahonrada', 'vruvahonradau', 'onra']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('gpsevxtcoeexrltyzduyidmtzxf'), ['exrltyzduyid', 'vxtcoeexrltyz', 'xr', 'ltyzduyidmt', 'yzduy', 'exr', 'coeexrltyzduy', 'coeexrltyzduy', 'rlty', 'rltyzduyidm', 'exrltyz', 'xtcoeexrlty', 'vxtcoeexrltyzduyidm', '', 'coeexrl', 'sevxtcoeexrltyzdu', 'dmt', '']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('dyhycfhzyewaikgursyxfkuv'), ['sktnofpugrmyxmbizzrivmhn', 'fhlgadpoqcvktbfzncxbllvwutdawmw', 'eewzjpcgzrqmltbgmhafwlwqb', 'tpogbkyj', 'rtllntxjgkzs', 'mirbvsqexscnzglogigbujgdwjvcv', 'iktwpgjsakemewmahgqza', 'xgfvzkvqgiuoihjjnxwwpznxhz', 'nxaumpaknreklbwynvxdsmatjekdlxvklh', 'zadzwqhgfxqllihuudozxeixyokhny', 'tdqpgfpzexlkslodps', 'slztannufxaabqfcjyfquafgfhfb', 'xvjldhfuwurvkb', 'aecv', 'uycfsughpikqsbcmwvqygdyexkcykhbnau', 'jr']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('vbcsettndwuntnruiyclvvwoo'), ['dwuntnru', '', 'ttndwuntnruiyclvv', 'ntnr', 'nruiyclvvw', 'wo', '', 'bcsettndwuntnruiycl', 'yc', 'untnruiyclvvw', 'csettndwuntnr', 'ntnruiyclvvwo']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('pqqnugshlczcuxhpjxjbcnro'), ['dpeedqy', 'rtsc', 'jdgla', 'qkgudqjiyzvlvsj', 'xmfxawhijgxxtydbd', 'ebgzazqthb', 'wyrjhvhwzhmpybnylirrn', 'iviqbyuclayqketooztwegtkgwnsezfl', 'bhvidy', 'hijctxxweboq', 't', 'osnzfbziidteiaifgaanm']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('loqchlxspwuvvccucskuytr'), ['', 'k', 'qchlxspwu', 'u', 'hlxspwuvv', 'wuvvccucsku', 'vcc', 'uyt', 'uvv', 'spwu', 'ytr', 'wuvvccucs', 'xspwuv', 'lxspwuvvccuc', 'spwuvvccu', 'oqchlxspwuvvccucskuy']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('pjjyzupzwllshlnatiujmwvaofr'), ['lnatiujmwvao', '', 'zupzwllsh', 'nati', 'wllshl', 'hlnatiujmwv', 'mwvao', 'shlnat', 'ati', 'wllshlnatiujmwvao', 'wllshlnatiujmwvaofr', 'nat']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('iketunkleyaqaxdlocci'), ['nkleyaqaxd', 'etunkleyaq', 'yaqaxdlocci', 'tunkleyaq', 'eyaqaxdlocc', 'leyaq', 'nkleyaqaxdl', 'tunkleya', 'kleyaqa', 'etunkleya', 'leyaqa', 'dlo', 'yaqa', 'leyaqaxd', 'etunkleyaq', '']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('drqianqtangmgbdwruvblkqd'), ['wusajejyucamkyl', 'wsgibljugzrpkniliy', 'lhwqqiuafwffyersqjgjvvvfurx', 'jfokpzzxfdonelorqu', 'ccwkpcgac', 'jmyulqpndkmzbfztobwtm', 'rwrgfkccgxht', 'ggldjecrgbngkonphtcxrkcviujihidjx', 'spwweavbiokizv', 'lv', 'krb', 'vstnhvkbwlqbconaxgbfobqky', 'pvxwdc', 'thrl', 'ahsblffdveamceonqwrbeyxzccmux', 'yozji', 'oejtaxwmeovtqtz', 'zsnzznvqpxdvdxhznxrjn', 'hse', 'kcmkrccxmljzizracxwmpoaggywhdfpxkq']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('yasnpckniistxcejowfijjsvkdajz'), ['slkpxhtsmrtvtm', 'crsbq', 'rdeshtxbfrlfwpsqojassxmvlfbzefldavmgme', 'ipetilcbpsfroefkjirquciwtxhrimbmwnlyv', 'knjpwkmdwbvdbapuyqbtsw', 'horueidziztxovqhsicnklmharuxhtgrsr', 'ofohrgpz', 'oneqnwyevbaqsonrcpmxcynflojmsnix', 'shg', 'nglqzczevgevwawdfperpeytuodjlf']) from system.numbers limit 10; +select 0 = multiSearchAny(materialize('ueptpscfgxhplwsueckkxs'), ['ohhygchclbpcdwmftperprn', 'dvpjdqmqckekndvcerqrpkxen', 'lohhvarnmyi', 'zppd', 'qmqxgfewitsunbuhffozcpjtc', 'hsjbioisycsrawktqssjovkmltxodjgv', 'dbzuunwbkrtosyvctdujqtvaawfnvuq', 'gupbvpqthqxae', 'abjdmijaaiasnccgxttmqdsz', 'uccyumqoyqe', 'kxxliepyzlc', 'wbqcqtbyyjbqcgdbpkmzugksmcxhvr', 'piedxm', 'uncpphzoif', 'exkdankwck', 'qeitzozdrqopsergzr', 'hesgrhaftgesnzflrrtjdobxhbepjoas', 'wfpexx']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('ldrzgttlqaphekkkdukgngl'), ['gttlqaphekkkdukgn', 'ekkkd', 'gttlqaphe', 'qaphek', 'h', 'kdu', 'he', 'phek', '', 'drzgttlqaphekkkd']) from system.numbers limit 10; +select 1 = multiSearchAny(materialize('ololo'), ['ololo', 'ololo', 'ololo']); + +select 1 = multiSearchAnyUTF8(materialize('иечбпрхгебилцмпфвжцс'), ['лцмпфвж', 'ечбпрхгебилц', 'фвж', 'мпфвж', 'вжцс', 'пфвжцс', 'ц', 'чбпрхгебил', 'илцмп', 'фвж', 'ечбпрхгеби', '', 'б', 'хгеб', '', '', 'ил', 'ебилцмпфвжцс']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('змейдмоодкшуищвеишчддуцпх'), ['здсщесгдкзмчбжчщчиоо', 'чфззцмудщхтфрмсзрвшйщ', 'рлунбнзрфубуббдочтвлзмпгскузохк', 'ктзлебцам', 'вчспмж', 'нгкк', 'гпзйа', 'щпйкччнабакцтлапсбваихншхфридб', 'афсузжнайхфи', 'йрздеучфдбсвпжохрз', 'ошбечпзлг', 'полшхидфр']) from system.numbers limit 10; +select 1 = multiSearchAnyUTF8(materialize('лшнуухевгплвйужчошгнкнгбпщф'), ['гбпщф', 'б', 'ф', 'чошгнкнг', 'йужчо', 'гплвйужчошгнкн', 'бпщф', 'плвйужч', 'шгнкнг', 'хевгплвй', 'плвйужчошгн', 'вй', 'лвйужчошгнкнгбпщф', 'лвйужчошгнкн']) from system.numbers limit 10; +select 1 = multiSearchAnyUTF8(materialize('кцпгуоойвщталпобщафибирад'), ['ойвщталпобща', 'щта', 'пгуоойвщтал', 'ф', 'общ', 'цпгуоойвщталпобща', 'побщ', 'ф', 'цпгуоойвщталпобщафиб', 'побщаф', 'лпобщафи', 'цпгуоойвщталпобщафи', 'пгуоойвщталпобщаф', 'талпоб', 'уоойвщталпо', 'гуоойвщтал', 'уоойвщталп', 'щ', '', 'цпгуоойвщталпобщафибирад']) from system.numbers limit 10; +select 1 = multiSearchAnyUTF8(materialize('фвгйсеккзбщвфтмблщходео'), ['еккзбщвфтмблщходе', 'йсеккзбщвфтм', 'вфтмблщходео', 'вгйсеккзбщ', '', 'йсеккзбщвфт', 'бщвфтмблщход', 'ккзбщвфтмблщход', 'ккзбщвфтм', 'еккзбщвфтмблщходе', 'еккзбщвфтмблщх', 'вгйсеккзбщвф', 'оде', 'оде', '', 'бщвфтмблщх', 'б', 'йсеккзбщвфтмблщходео', 'вфтмблщ', 'кзбщ']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('хбаипфшнкнлтбшрскшщдувчтг'), ['хгшгднфуркшщвфгдглххс', 'цогчщки', 'тдмщшйзйхиквмб', 'етелфмшвмтзгеурнтбгчнщпмйпйжжциш', 'чсбк', 'ибащлшздеуревжйфуепфхкузбзао', 'дкмбщдсбжййсвгкхбхпшноншлщ', 'щхбеехнцегрфжжу', 'збфлпгсмащр', 'скчдигцнсзфрещйлвзнбнл', 'освзелагррдоортлрз', 'утхрч', 'йкбрвруенчччпшрнгмхобщимантешищщбж', 'жгивтеншхкцаргдасгирфанебкзаспбдшж', 'ййекжшщцщ', 'ефдсфбунйчдбуй', 'бвжцирзшмзщ', 'випжцщйзхнгахчсцвфгщзкдтвчйцемшлй', 'лдрфгвднеиопннтчсйффвлхемввег', 'бмтцжжеоебщупфчазпгхггцегнрутр']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('фбуоойпцщишщлбхчрсллзвг'), ['уччхщ', 'вщчсарфмйшгшпйфгмжугмщжкцщгйжзфл', 'кклл', 'лпнжирпсиуо', 'нчипзфщхнтштхйхщрпзитко', 'вйпсдергвцзсцсгмхпбз', 'чфщдфоилгцевпц', 'чааиае', 'чгингршжтчпу', 'щетбнгутшйсгмвмучдхстнбрптничихб']) from system.numbers limit 10; +select 1 = multiSearchAnyUTF8(materialize('лйвзжфснтлгбгцерлзсжфещ'), ['зсжф', '', 'бгц', 'зжфснтлгбгц', 'л', 'цер', 'жфснтлгбгц', 'тлгбг', 'це', 'гбгцерл', 'нтлгбгцерлзсж', 'жфещ', 'взжфснтлг', 'фснтлгбгцерлзсжфещ', 'нтлгбгцерлзсж', 'зжфснтлгбг', 'взжфснтлгбгцерлз', 'взжфснтлгбгце']) from system.numbers limit 10; +select 1 = multiSearchAnyUTF8(materialize('нфдцжбхуучеинивсжуеблмйрзцршз'), ['чеинивсжуеблм', 'жуебл', 'блмйрзцрш', 'цр', 'м', 'фдцжбхуучеинивсжуеблмйрзцр', 'нивсж', 'ивсжуеблмй', 'й', 'всжуеблмйрзцршз']) from system.numbers limit 10; +select 1 = multiSearchAnyUTF8(materialize('всщромуцйсхрпчщрхгбцмхшуиоб'), ['муцйсхрп', '', 'уцйсхрп', 'сщромуцйсхрпчщ', 'схрпчщр', 'сщромуцйсхрп', '', 'уцйсхрпчщрхгбцмх', '', 'цмхшуиоб', 'гбц', 'пчщр', 'цйсхрпчщр', 'омуцйсхрпч', 'схрпчщрхгбцм', 'йсхрпчщрхгбцм', '', 'пчщрхгбцм', 'уцйсхрпчщрхгбцмх', 'омуцйсхрпчщ']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('уузшсржоцчтсачтедебозцвчвс'), ['бомбсзхйхкх', 'отвгстзихфойукарацуздшгбщеховпзкй', 'мфнев', 'вйийшшггилцохнзбхрлхи', 'втинбтпсщрбевзуокб', 'оиойвулхкзлифкзиххт', 'зацччзвибшицщрзиптвицзхщхкбйгшфи', 'кнузршшднмвтощрцвтрулхцх', 'рчбкагчкпзжвтбажиабиркдсройцл', 'щргчкзожийтпдзфч', 'щбошгщзсжтнжцтлкщитеееигзцлцсмч', 'сцкк']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('щчбслгзвйдйжрнщчвфшй'), ['пдашзбалйнзвузкдвймц', 'щхтшйоч', 'фднвфигозржаз', 'рйфопхкшщвщдвл', 'цдкйхтусожпешпджпатфуиткп', 'щпбчсслгщййлвскшц', 'жпснс', 'уиицуувешвмчмиеднлекшснчлйц', 'пххаедштхмчщчбч', 'ичтмжз', 'лсбкчу', 'бгфдвпзрл', 'йицц', 'цфйвфлнвопкмщк', 'бгщцвбелхефв', 'мймсвзаелхнжйчохомлизенфш', 'трйднхндшсщмпвщомашчнгхд', 'жфцнифлгдзйе', 'зспкшщщенбцжгл', 'рщтб']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('шщпееасбтхогвгвцниуевисгшгбч'), ['гпа', 'стимсркзебхрвфпиемзчзу', 'нзгофухвекудблкадбшшусбеулрлмгфнйгиух', 'кфиашфобакщворувгвкчавфзшх', 'гфпгщгедкмтгрдодфпуйддхзчждихгрчтб', 'тцтжр', 'рцйна', 'йцбпбдрреаолг', 'житсфосшлтгсщдцидгсгфтвлз', 'жвтнжедцфцтхжчщч']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('вхкшгфпфмнщаохтмизпврйопцуйзмк'), ['дтчбкхащаткифружжейабфйкйтрскбощиеч', 'фтоуабхмдааиснрбраттклмйонлфна', 'цадзиднщймшкщолттпгщбх', 'кштбчжтждпкцнтщвмухнлби', 'микудпдпумцдцгфахгб', 'ирик', 'емлжухвмк', 'чгуросфйдцшигцхжрухжпшдкфгдклмдцнмодкп', 'ттбнллквдувтфжвчттжщажзчлнбждчщцонцлуж', 'елцофйамкхзегхклйгглаувфтуувее', 'двкзчсифвтекб', 'шсус']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('йхцглкцвзтшщочпзмнчтуеао'), ['йечдай', 'дащжщзлосмй', 'афуккгугаазшрчпцнхщцтмлфф', 'чфтфскрфйщк', 'жлччкцшнфижтехппафхвщфс', 'бзжчв', 'щкщймнкщлпедидсу', 'оцбажцзшзйпптгщтфекртдпдзшодвойвох', 'йжддбссерхичгнчлкидвгбдзуфембрц', 'ктщвшкрщмдшчогхфхусдотсщтцхтищ', 'пшстккамнбнардпзчлшечхундргтоегцзр', 'нсрнфузгжррчнжначучиелебрб', 'шгжмквршжтккднгаткзтпвкгзхшйр', 'змквцефтулхфохбнхбакдичудфмйчп']) from system.numbers limit 10; +select 1 = multiSearchAnyUTF8(materialize('шждйрчйавщбйфвмнжоржмвдфжх'), ['ор', '', 'йрчйавщбйфвмнжо', 'вщбйфвмнжорж', 'ждйрчйавщбйфвмнжорж', 'йавщбйф', 'дф', 'вщбйф', 'бйфвмнжорж', 'мнж']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('кдшнсйршгвлицбенйбцфрсаччетфм'), ['асмун', 'йогкдчодиф', 'лштйбжнзфкикмпбитжшгкбоослщгзнщо', 'улштжцисцажзчштгжтфффабйлофедуфме', 'дрпгкчджихшзммймиамзфнуиорлищзгйвху', 'йиоршнйоввквбдвдзасма', 'члмвасмфрхжсхрбцро', 'лшкизщушборшчшастйсцкжцбонсшейрщ', 'масдфкршлупасвйфщфважсуфсейшзлащхрж', 'дгхшщферодщцнйна', 'цзфзждбавкжрткст', 'рфбожзееаце', 'кошомвгпрщсдквазчавожпечдиуйлщадфкгфи', 'бшпхнхсгшикеавааизцсйажсдийаачбхч']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('хтиелйтарквурйлжпеегфш'), ['зпмйвзуднцпвжкбмйрпушдуавднвцх', 'фбссчгчвжакуагдвижйтщтшоабпхабжш', 'щхшибаскрщбшрндххщт', 'сммрсцзмптисвим', 'цсргщфж', 'восжбшйштезвлкммвдхд', 'вбсапкефецщжквплуо', 'даеуфчвеби', 'бтптлжпин', 'шчддтнсйкщйщ', 'фжхщецпзчбйкц', 'цсвфпздхрщхцбуцвтег']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('апрчвзфжмбутццрйщкар'), ['индхжз', 'жилцовщччгстби', 'ажс', 'фктйамйтаг', 'шммнзачггоннксцушпчн', 'чдлйтзтоцдгзццисц', 'пнбтувщцдсчнщмсакрлгфмгрй', 'овмсйнщзушвщгуитщрхвйодф', 'бзлштезвлаижхбмигйзалчолшеунлц', 'фкжпеввгшгащз', 'тменбщжмсхщсогттршгек', 'чап', 'х', 'шкомегурлнйпшбщглав']) from system.numbers limit 10; +select 0 = multiSearchAnyUTF8(materialize('двхопооллаеийтпцчфжштнргкк'), ['йймчнздешхбццбжибопгктрнркевпиз', 'фйрохсамщцнмф', 'ййхфдпецжзгнуорвбплоахрфиле', 'пкллкацнвдббогг', 'йщдезамтжйзихщжмцлх', 'гдзувмщиеулиддердшпитвд', 'фхтунйшзхтщжтзхгцорошднпбс', 'фнситбеелцдкйщойлатиуухгффдвищсше', 'нзщщщндцрнищпхйвтбвмцтнуадцбву', 'вбщкапшнв', 'зйлмуимчскщнивтшлчмуузщепшйр', 'шжбвйдр', 'гддждбкначдттфшжшхпфиклртпгм', 'еншащцфафчнгбнщххнзочбтпушщорегшцзб', 'уунеущкззоетбучкц', 'щасифзоажребийещ', 'пщбххсдгйтт', 'хшсчуотрт', 'жкднйрозбцшужчшбкккагрщчхат', 'шачефцгч']) from system.numbers limit 10; + +select 0 = multiSearchAnyCaseInsensitive(materialize('QWyWngrQGrDmZxgRnlOMYHBtuMW'), ['ZnvckNbkeVHnIBwAwpPZIr', 'NCzFhWQmOqIGQzMORw', 'tDYaxfQXWpKNLsawBUUOmik', 'IMveCViyAvmoTEQqmbcTbdfjULnnl', 'NRvsdotmmfwumsDpDtZU', 'mnqVnwWOvMiD', 'HXpHrMvGQpbuhVgnUkfFPqjpoRdhXBrFB', 'awtr', 'IMIdOmMHZccbOZHhWOKcKjkwwgkJSfxHDCzR', 'jPLISbIwWJEKPwgvajTxVLws', 'HBfRrzEC', 'VXsysGnAsFbqNOvIaR', 'upCaeaIOK', 'GUDFkrzBiqrbZVnS', 'MoCOePXRlVqCQpSCaIKpEXkH', 'rfF', 'fjhMEpySIpevBVWLOpqi', 'KdeskLSktU', 'vjUuNUlBEGkQyRuojZLyrmf', 'SvSxotkTKCeVzNICcSZLsScKsf']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('gcDqqBCNqhQgVVgsxMXkevYIAxNl'), ['BHnoKRqOoKgmOVkjtehGSsInDvavDWOhkKAUL', 'nYqpmKPTWGdnyMcg', 'TIplHzsSXUz', 'SiQwpQgEdZ', 'YoJTWBJgsbJvq', 'CwyazvXERUFMCJWhTjvltxFBkkvMwAysRLe', 'tXUxqmPbYFeLUlNrNlvKFKAwLhCXg', 'vUbNusJGlwsOyAqxPS', 'ME', 'ASUzpELipnYwAknh', 'VtTdMpsQALpibryKQfPBzDFNLz', 'KmujbORrULAYfSBDyYvA', 'BaLGNBliWdgmqnzUx', 'IzwKIbbSUiwhFQrujMgRcigX', 'pnS', 'UKSZbRGwGtFyLMSxcinKvBvaX']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('HCPOPUUEVVsuZDbyRnbowGuOMhQ'), ['UzDbYrNBoWgUo', '', 'pUUEVVsUzdByrNB', 'nBO', 'SUZdbYrNbOWgUoMH', 'pOpuUevVSUZDbYRnb', 'bowGUoMh', 'VsUZDbyrNbo', 'suzdBYrN', 'uueVvsUZDBYRnBoW', 'gUom', 'eVvsuzDBYRNBoWgUOM']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('RIDPJWYYSGBFWyXikHofbTcZAnj'), ['aFxQyVe', 'OcnZBgPsA', 'iBQaH', 'oesSvsWtgQprSSIPaDHdW', 'EfytiMfW', 'qHiFjeUvQRm', 'LfQkfmhTMUfoTOmGJUnJpevIoPpfpzMuKKjv', 'scYbCYNzJhEMMg', 'yTLwClSbqklywqDiSKmEdyfU', 'HYlGFMM', 'TMQhjOMTImXbCv', 'AVtzpxurFkmpVkddQANedlyVlQsCXWcRjEr']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('GEsmYgXgMWWYsdhZaVvikXZiN'), ['wySd', 'smYgxGMWWYsDHZ', 'vIk', 'smyGxgmwWysDHzAvvikxZi', 'WYsdHZAvVI', 'YGxGmwWYSDhzavvI', 'XzI', 'ySDhZAvvIK', '', 'myGXgmwWySdHz', 'MYGxgmwWySdHZaVvik', 'wYsDhzAvvikXz', 'wwYsdHzav', 'Z']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('XKCeCpxYeaYOWzIDcreyPWJWdrck'), ['tTRLUYJTkSWOabLJlIBshARIkwVRKemt', 'jQgn', 'wdNRsKIVunGlvwqkwn', 'BsbKGBJlkWQDBwqqeIjENvtkQue', 'yLuUru', 'zoLGzThznNmsitmJFIjQ', 'WFKnfdrnoxOWcXBqxkvqrFbahQx', 'QHbgRXcfuESPcMkwGJuDN', 'NPqfqLS', 'bi', 'HnccYFPObXjeGYtrmAEHDZQiXTvbNcOiesqRPS', 'KobVCJewfUsjBXDfgSnPxzeJhz', 'AqYNUPOYDZjwXx', 'xbZydBGZFFYFsFHwm']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('AnIhBNnXKYQwRSuSqrDCnI'), ['', 'HBNNxkyqWRS', 'xKyqwrSUSQR', 'yQwr', 'ihbnnxKYQWrsUS', 'bnnXkYqwrSuS', 'qWRs', 'nXKyqWRSUS', 'qrdcN', 'NiHBnNXkYQWrS', 'NnXkYQwRSUsqRDCn', 'rSusqRd']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('OySHBUpomaqcWHcHgyufm'), ['lihJlyBiOyyqzeveErImIJuJlfl', 'WyfAXSwZPcxOEDtiCGBJvkCHNnYfA', 'hZ', 'fDQzngAutwHSVeoGVihUyvHXmAE', 'aCpcZqWKdNqTdLwBnQENgQptIyRuOT', 'PFQVrlctEwb', 'ggpNUNnWqoubvmAFdjhLXzohmT', 'VFsfaLwcwNME', 'nHuIzNMciJjmK', 'OryyjtFfIaxViPXRyzKiMu', 'XufDMKXzqKjYynmmZzZHcDm', 'xWbDgq', 'ArElRZqdLQmN', 'obzvBzKQuJXZHMVmEBgFdnnQvtZSV', 'ZEHSnSmlbfsjc', 'gjmWPiLylEkYMTFCOVFB']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('NwMuwbdjhSYlzKoAZIceDx'), ['ZKOaZ', 'wBDJhsYlZKo', 'hSy', 'MUwbDjHsyl', 'sYlzK', 'ylZKOAZ', 'y', 'lZKoaZICEdX', 'azIce', 'djHSylZkoAzice', 'djHsYLZKoAzi', 'dJHSYlZK', 'muWbDJHsYLzKOaziC', 'zi']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('gtBXzVqRbepHJVsMocOxn'), ['DidFXiqhRVBCHBVklLHudA', 'yEhumIpaYXlj', 'iaEmViTRLPM', 'vTwKBlbpaJZGYGdMifOVd', 'zvgfzWeLsMQNLutdAdCeuAgEBhy', 'Ca', 'iHabiaRoIeiJgSx', 'EBfgrJnzHbuinysDBKc', 'kT', 'SGIT', 'BTRuKgHDuXMzxwwEgvE', 'OWJIeTLqLfaPT', 'BQM', 'yMimBqutKovoBIvMBok', 'zIBCYVNYAwu', 'EFDEFWGqvuxygsLszSwSiWYEqJu', 'QJDIXvPOYtvhPyfIKqebhTfL', 'ssALaXRxjguUIVKMCdWRPkivww']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('MowjvqBkjnVTelCcXpoSuUowuzF'), ['Su', 'vqBkJNvTelC', 'Elccxp', 'vtElc', 'JVqBkJnVTELCcxpOsU', 'OsUuOWUz', 'ElccxPoSU', 'wJVQbkJNVtElCC', 'xpOSUUo', 'VQbkJnvTELCCXp', '', 'TeLcCxPOsuuO']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('VfVQmlYIDdGBpRyfoeuLffUUpMordC'), ['vqMLyIddgBPrYFoEulFFu', 'lyIDdgBPrYFOeul', 'dGBPRYFOeUlffUupmOrD', 'OEulffU', 'pMordc', 'FVqmlyiDdgBpRyFoeUlFfuUpMOrD', 'PmO', 'o', 'YiDDgbPRYFOe', 'DGBPryfoeU', 'yIDdgbpRyFOeULfFU', 'lyIddgBPryfoeulfFuU', 'gbPrYfOeUlFfuupmO', 'yFoeULF']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('CdnrzjzmwtMMPLjgcXWsbtrBs'), ['RfgIUeerlPIozKpRQR', 'QRoYzjZlgngJxX', 'mEbqlBIzTQH', 'UmrfJxKyTllktPfyHA', 'ukoZeOPA', 'pbbRaUcJijcxt', 'Rg', 'lSBG', 'HvuwuiqVy', 'Fo', 'aGpUVjaFCrOwFCvjc', 'zKhfkgymcWmXdsSrqAHBnxJhvcpplgUecg', 'ioTdwUnrJBGUEESnxKuaRM', 'QciYRCjRDUxPkafN']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('miTQkQcxbKMwGOyzzRJpfXLyGx'), ['yMwgQQJkeshUugm', 'wGVe', 'XncShWqjp', 'KWjGQCOsfMKWRcgCfebkXZwZ', 'SFWbU', 'WdFDMIcfWeApTteNfcDsHIjEB', 'XRuUJznPOCQbK', 'tibBMGZHiIKVAKuUAIwuRAAfG', 'VVCqVGGObZLQsuqUjrXrsBSQJKChGpZxb', 'bWYAOLuwMcwWYeECkpVYLGeWHRrIp', 'SLzCgfkRWmZQQcQzP', 'VvfOhFBhfiVezUSPdIbr']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('KXoTIgVktxiXoEwfoLCENiEhz'), ['oLCENie', 'xix', 'en', 'IgvktxIXoEWFOLCEnieHz', 'xOEWFoL', 'LC', 'ktxIxoEwfolCenie', 'ce', 'oTIGvktXIXOE', 'eW', 'otigVKTXIXOEwFolC', 'E', 'CEni', 'gVKtxIxoEwfOLCENieh']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('DXKzSivrdLuBdCrEYfMEgPhOZ'), ['', 'sIVRDlUBdcr', 'luBDcrE', 'rDLUbDCreY', 'KzSiVRdLuBDCr', 'dcREYFme', 'lUbdCReyFMEgph', 'sivrDlubdCr', 'BdcreYfMEgP', 'ZSiVrdluBDCrEYfmegpHOZ']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('lTvINMXVojkokvNBXPZOm'), ['ZQOJMEJfrjm', 'vIpmXnGlmWze', 'wbdDKcjrrIzBHypzJU', 'omotHOYbZjWfyVNeNtyOsfXPALJG', 'SXxu', 'yZPDFsZq', 'OVYVWUjQDSQTKRgKoHSovXbROLRQ', 'RnXWZfZwHipewOJimTeRoNRYIdcZGzv', 'sizoEJibbfzwqFb', 'vgFmePQYlajiqSyBpvaKdmMYZohM', 'ENsFoFCxDQofsBSkLZRtOcJNU', 'nG']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('LsTqxiGRdvQClVNBCGMOUHOAmOqPEC'), ['NdFuUQEUWaxS', 'fdOHzUzineBDnWJJvhPNZgB', 'rYAWGIBPxOLrjuquqGjLLoIHrHqSFmjh', 'IVgYBJARY', 'ToivVgUJAxRJoCIFo', 'yQXGrRjhIqFtC', 'PNYdEPsWVqjZOhanGNAq', 'nrQIDDOfETr', 'usJcPtiHKhgKtYO', 'vPKqumGhPbmAJGAoiyZHJvNBd', 'eXINlP', 'WQeESQJcJJV']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitive(materialize('gRzzQYOwLNiDcMFjXzSFleV'), ['XZSfLe', 'wLnIdcMFjxZSf', 'F', 'm', 'Le', 'qYoWLNidcMFjXzsf', 'zqyoWlNIdcMFj', '', 'oWlnIDCMfJxzsfL', 'wlNIdCmfjXzS']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitive(materialize('cYnMXJMJCdibMXoUQHEw'), ['BFrGFZRgzwHGkUVbBiZMe', 'piORdVIWHMBsBDeJRLbGZAHGBrzNg', 'bmDePbTPnFQiCFfBJUxAEYNSbgrOoM', 'gtzeAGwqjFrasTQUgAscfcangexE', 'okLG', 'l', 'EBkkGYNZZURgFgJPlb', 'HDQVngp', 'vEHhtBqWhZHCOrqEKO', 'fgqdFc', 'COig', 'VftTpSXAmTmvnShHJqJTdEFcyKPUN', 'WDI', 'knBm']) from system.numbers limit 10; + +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('мтдчЛВЖАгвзщущвкфИКмТбжВ'), ['щУщвкФИкМ', 'чЛвжАГвЗЩуЩвКФикм', 'ДчлвЖАГвзЩУЩвКфИКМтБЖВ', 'ЖагвзщуЩВКФикМТБжВ', 'ВжагВзЩУ', 'гВЗщущвкфИКмТБж', 'ГвЗщ', 'щВкФикМТБЖВ', 'вЖАГВзщущ', 'взЩуЩвКФИкМТ', 'ЧЛВЖагвЗщуЩВк', 'тДчлвЖагвзЩуЩвкфИк', 'ТДЧлвжаГВзЩущВ', 'тДчлВжАГВЗЩУ']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('дтрцФхИнпиОШфдАгзктвбУвсб'), ['чТрВиУРФсРпДЩОащчзЦНцхИДА', 'ЗжмПВтмиойУГхАЦПиДУЦноНуййЩХаФТофшЩ', 'уБшлОЙцМПгетЖЧетШжу', 'ЧзИАУХобФрачТеХОШбМщЖСамиМВАКРщАЦ', 'ВйвТзхЙФЧоАЖвщиушАз', 'ЦшИфххкжиФйСЛЛНЛчВоЙВПпхиИ', 'ОатЕтщкЦпбСБйцОшГШРОшхБцщЙЧиУЩЕеФлщ', 'цСПпЧА', 'ШЧНфПмФсКМКДВЦАоФчОУеТЦИзЦ', 'зАбдЛНДГИ', 'фхЩлЗДНСсКЖИФлУАбЛеТФЕпЖлпПхЙиТЕ', 'иВшкНслТКМШЗиДПйфвйНкМЛхеФДзИм', 'лпушПБванпцев', 'ЧОшЧЧмшЦЛЙйГСДФйЛАв']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('нщМаНдЧЛмиВврПокПШмКБичкхвРГ'), ['АЙбаЙйШЛЙРЦмЗчВеИЕощсЦ', 'щЦФдВжчТСЩВКЦСпачЙсумщАтЩувеиниХПДоМС', 'иоАкДРршуойиЩищпрфВаЦПж', 'еЖПйШкГжЧтоГЙМВ', 'ЩПалиБ', 'ТвВлт', 'оХжйЛФеКчхЗВвЕ', 'ерцЩ', 'ШХЖОАрзеп', 'ККМрфктКГишпГЩхаллхДиВИИЛЗДеКйХмжШ']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('вШЙчоМгОттЧАЕнЧаВеЦщчЧошМУ'), ['ЧОмГотТчАЕН', 'ОмГотТчАЕнчАвецЩчч', 'ЧАВецЩч', 'ТЧАеНЧаВ', 'ттчаЕнча', 'ТчАЕ', 'мготтЧАенчавЕЦЩ', 'НЧаВец', 'тТЧаенчАвецщчЧошм', 'Ав', 'ТЧаЕнчавецщчЧоШму', 'аЕнЧав', 'АеНЧав', 'шйЧомГОТТчаЕнчАВЕ', 'шйчоМгОтТЧаЕНчаВеЦщЧчош', 'МУ', 'ошМ', 'гОТтЧаеНЧА']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('фйадзЧмщЖШйЖЛшцГигцШ'), ['НТХеМРшДНУЗгадцуЧИ', 'жпСИКЩМлНлиоктлЦИвНЛ', 'КхшКРчХ', 'кгТЗаШИарХЧЛЖмСЖм', 'ОмиЛй', 'жЕРбФЩНуЕКЕАВоБМОнАЕнКщшзйПкОЗ', 'гиЗдадкбжХМЗслшВИШай', 'двтЗйЙНгПуТзД', 'ТНкмаВЕФ', 'Шеа']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ШЕшхмеЦХеАСКощеКИфлсТЧИЗЛ'), ['КифЛсТ', 'ХеаСКощЕк', 'КифлсТЧ', 'шХМеЦхЕаскОЩеКИ', 'ЕшхмЕцХеаСК', 'ХЕасКоЩ', 'чИ', 'ЕцхеАсКОЩек', 'ЩЕкИфлс', 'асКощЕкифЛсТ']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('шоКнВЕрОЖЛпУйХзСугКПВжиРсЙпо'), ['игВербфНахчжЙггч', 'лтимрдфЕг', 'нкеаЖАшНБвйСдКИВГДшАГиАТнФШ', 'МжсТЙМГОииУКВГнцткДнцсоАд', 'ХтпгУСдБдцАЖЛАННоЕцзЕшштккз', 'ншУЦгФСЖшмс', 'нЩшМ', 'гоЖхМшаЕмаДРЧБЛИТпмЗОоД', 'фГКШхчФбЕГЛйкчПИЙххуМГНШзхг', 'ХпХщПЦАзщтг']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('кЧбоЗХфвБХхусмШгНаШШаГзш'), ['Ури', 'лММшткфНзцЦСВАдЩПМШфйОМшефигЖлуЕП', 'сМтЕдчЦафйСТЖЗфлРЙПЦдипжШскцВКХЦЖ', 'АУкжИФцшЛБЦЧм', 'ФПлнАаДСХзфоХПСБоСгМТОкЗЧйЛ', 'ЦшСГЛрцДмнНнХщивППттжв', 'жзЕгнциФ', 'МШЛсЙЧтЛАГжд', 'уИиЕжцоРНх', 'ЧбйГуХтшОНкрЧИеПД', 'ЦдЩЕкКвРЦжщЧциекЗРйхрббЖуЧ', 'иВжен', 'ГчОржвБГсжштРЕБ', 'ШоЖдуЙфчсЧегумщс', 'йчЙГ', 'РДедвТ']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ткРНбЩаРкгГчХшецИкНЕнСЖкйзАуУЖ'), ['ХлЖхУИллрРННйЗйсРуШЧвМбЧЧщфФЦц', 'СЛчКБцСФДшлфщаФлЙСзШабмбхуБжТСТ', 'УКУиввЗЩуВМцпчбпнДГбпЕЖрПбИДркМРОеЧмЧдГ', 'ПчщвШЩвГсЛмММГБ', 'хКЦЧсчжХЩИЖХеНнтоФЦлнмЛЧРФКпмСшгСЧДБ', 'удсЗйУДНЧУнтЕйЦЗЖзВСх', 'хПЖЙИрцхмУкКоСмГсвПаДОаЦНЖПп', 'сВОей', 'ЩЦжщоабнСгдчрХнЩиМХзжЩмФцррвД', 'ЦИсйнЦДоЕДглЕЦД', 'жзйПфБфУФоцзмКЩГПЧХхщщПТпдодмап', 'ДНХГНипжШлСхХхСнШЩЛИснУйЧЩЖДССФфиС', 'ОйЩНнйЕшцФчБГЛвхЖ', 'КЧРВшИуШйВфрпБНМсУмнСЦРпхЗАщЗУСвЧйБХтшХЧ', 'зЛбНу', 'ЗСрзпшЕйРржПСсФсШиМдйМЦГхдйтРКЩКНцкбмгС', 'СУццБуКнчОищГ', 'уЕГЧлЗБНпУисЕЛ']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ВЦХсЖЗЧЙБЗНбРитщстеМНжвВ'), ['итщст', 'ЧйБЗНбрИтщстЕМнЖ', 'ХСЖЗЧйбзНБриТщ', 'Темнж', 'сЖзЧЙБзнб', 'хСжЗчйБзнБрИтЩстЕм', 'БзнБРиТщ', 'ЗчЙбзНбрИТщ', 'чйбЗНбри', 'зЧйбзНБРИ', 'нБРитщсТе', 'зНб', 'цхСжзчйБЗнБРИТЩСтЕм', 'жЗЧЙБЗнбрит']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ХцМШКАБАОххЕижгГХЩГиНциД'), ['ОРАБЕРВомЛфГНМИКупбхЛаАкЗдМзтш', 'лЗУЩнлбмиЛАфсгМРкцВтлснййишИНАС', 'ТлжлУоУгжукФжЖва', 'жоСШПоУНЩшРМГшОЛзЦБЛиЛдТхПДнфжн', 'чнСУЗбДаГогжДфвШКеЙПБПутрРпсалцоБ', 'ЙозоПщчакщаАлРХбЦгац', 'иаИСсчЙЧБШорлгЧТнчцйзоВБХбхЙФтоЩ', 'ПСзсБЗЕщурфДЛХйГИеПНрмииаРнвСФч', 'ЦйЖЕуТфЖбхЩМтйсЙОгЛбхгтКЕЩСАЩ', 'гтЗуЩлужДУцФВПЛмрБТсСНпА', 'тГвлбчЗМасМЖхдЕгхмЩксоЩдрквук', 'ВРаг']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('тУйВЖдНнщцЗЖфКгфжГфиХСБЕЩ'), ['КгФЖГФи', 'сБе', 'ЖФ', 'гфжгФИхсбе', 'ВЖДНнщЦзжфКГфЖгфИхсбещ', 'ВЖДНнЩЦзжфкГ', 'вЖДННЩЦзжФКГфЖгФ', 'ф', 'НщЦЗж', 'нщЦЗЖФк', 'Их', 'дННщцзЖФКгф', '', 'нщцзжФкг']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ШкКРаоПеЗалРсТОиовРжгЙЧМКЛШ'), ['рчсажЕК', 'пЧТМфУрУММждЛйжзУрбкмам', 'бАШеНмВШзлзтушШШсхОсцрчЙПКИБнКжфЧЕХ', 'ЖМЛшбсУМкшфзочщАЖцМбмШСбВб', 'гтРХсщхАИОащчлИЧуйиСпСДФПбРл', 'ЧуОРУаоойГбУппМйЩФДКПВ', 'уУпугйРЕетвцБес', 'ЙЖЦТбСЖж', 'ИБКЛ', 'ТДтвОШСХГКУИПСмФМтНМзвбЦрднлхвДРсРФ', 'вВгНЙХИрвйЕЗпчРГЩ', 'ПчмТуивШб']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('РлчгхзуВШежХЦуМмнВЙщдцО'), ['ХшвМЦДШпЩОСшЦПдруа', 'ФИЦчУвРкпнПшИЕСЧАувиХд', 'фшвбЦОИЗфпИУМщзОЧЗфВцЙПнмтаТгг', 'мЖЩйавтнМСЛ', 'НВбШ', 'ааФДДрВвЙТдПд', 'ЗнчЧущшхЙС', 'рзуСзнеДфЩПуХЙЕл', 'ШСЩсАгдЦбНиШмшКрКс', 'ггнЕфБГзрОнАГЙзЧеИП', 'вшТИпЧдЖРкМНшзпиоиЩчзДмлШКТдпЦчж', 'фЦТЙц', 'ОтУшмбптТКЗеПлЧцЛОкЩБпккфгИн', 'ЩпвхпЗлШБЦ']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ЙбйнхНщЧЖщчГОАпчФнЛШФбгЛа'), ['щчг', '', 'апЧфНЛШфб', 'ЙнхНЩЧЖщчгОАПЧф', 'ХНщЧжЩЧгоАпч', 'ХНщЧжщчГо', 'нщЧжщчГОа', 'чЖЩЧГоапЧФНл', 'оапчФ', 'щЧГОАпЧФНлшФ', 'ЩЧГОАпЧФНЛшфБг', 'БЙНхнщчЖщчГоаПЧФНЛШФБгЛ', 'ОапЧфн', 'ф', 'БглА', 'ш', 'шфбГ', 'ХнЩЧЖщчГоА', 'ХНщчжщЧгоапч', 'хНЩчжщЧГоапчфнлшФбгЛ']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('кдЙДТЩеВЕфйКЩЦДиКБМф'), ['щфЛ', 'фЧЩЩичрКйЦКхеИИАпоБВЙЗбДАФио', 'мИтиЦРоВЙсБбСлНзиЛЧОфФевТмижщК', 'тЙгнКШфНТЕБЛцтГШЦхШхБ', 'уаабРГрМЙпМаБуЗпБЙчНивЦеДК', 'мпВЛНДеКПУгРЛЛинзуЕщиВШ', 'ЩжКйШшпгллщУ', 'пршЙПцхХЗжБС', 'нбЗНЙШБш', 'йцхИщиоцаМРсвнНфКБекзЛкчТ', 'хсмЦмнТрЩкДТЖиХщцкЦМх', 'ГмЛАбМщЗцЦйаОНвзуЗмЕКПБЙмАЕЛГ', 'ОЦХРЗРмкжмРИЖИЙ', 'з', 'лЕТкпкдЗчЗшжНфо', 'ИТПфйгЖЛзУТсЩ', 'ОфрбЛпГА', 'МЖооШпЦмсуГцАвМЕ']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ЩГТРШКИОРБРеНЖПКиуМОкхЛугИе'), ['брЕнЖ', 'РбрЕНЖпКиУМокХЛу', 'ГТрШКИорБРеНЖпКиУМ', 'рШКиоРбрЕнЖпкИУМОК', 'ИорбрЕнЖПК', 'Окхл', 'шкИоРБРеНЖПк', 'ТРШкИоРБрЕнжПКИУМОкхл', 'КИОРБРЕнжпкиУм', 'Н', 'КиОРбРЕнЖпкИУмоКхл', 'к', 'ГтРшКИоРБРЕнЖпк', 'гтрШкиорбрЕНЖпк']) from system.numbers limit 10; +select 0 = multiSearchAnyCaseInsensitiveUTF8(materialize('ШНвпкфЗвгДжУЙГлрТШаШЛгНЗг'), ['нЗБенВшщрЛАрблцщшБАдзччммсцКЖ', 'бЗЩхзЗЗбФЕйМоазщугБбмМ', 'рЙсВжВсхдйлЩгБтХлчсщФ', 'пиБшКРнбВБгЕуЖ', 'жПшнхпШзУБрУЛРНЩДиаГШщКдЕвшоуПС', 'чЕщкЗмДуузуСдддзгКлИнгРмЙщВКТчхзЗЛ', 'кЖУЗЖС', 'щххОВМшуажвН', 'фбцЖМ', 'ДШитЧЩДсйНбдШеООУдг', 'ЛХПфБВХЦТИаФПЕвгкпкпщлхмЙхГбц', 'чЦсщЗщрМ']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ФРХгаСлчЧОцкШгзмКЗшФфББвЧ'), ['кзШфФб', 'ГАслЧЧОцкшг', 'ФфббВЧ', 'ЦкШ', '', 'АслчЧОЦКШгзМкЗШффбБвч', 'РХгаслЧчОЦКШГз', 'РхгаслчЧОцКШгзМкзшФфБбВ', 'Шг', 'Ф', 'ХГАслчЧоцКШГзМкзш', 'ШгЗмКЗшфФб']) from system.numbers limit 10; +select 1 = multiSearchAnyCaseInsensitiveUTF8(materialize('ЧдйШкхОлалщНйбССХКаФзОМрКЕЙР'), ['бссХкафзОм', 'ХОЛаЛщнйБссХкаФз', 'лаЛщнйБсСХ', 'ЩнЙбСсхКаФЗО', 'йБСсХКАФЗОмР', 'йшкХолаЛЩНйбсСхК', 'С', '', 'ЙшкхОлалщНЙБсСхКаФзом', 'Йр', 'щнЙБссхКАфзоМрК', 'рКе']) from system.numbers limit 10; + +select 1 = multiSearchFirstIndex(materialize('alhpvldsiwsydwhfdasqju'), ['sydwh', 'dwh', 'dwhfdasqj', 'w', 'briozrtpq', 'fdasq', 'lnuvpuxdhhuxjbolw', 'vldsiws', 'dasqju', 'uancllygwoifwnnp', 'wfxputfnen', 'hzaclvjumecnmweungz']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('kcwchxxyujbhrxkxgnomg'), ['jmvqipszutxfnhdfaxqwoxcz', 'nrgzkbsakdtdiiyphozjoauyughyvlz', 'qbszx', 'sllthykcnttqecpequommemygee', 'bvsbdiufrrrjxaxzxgbd', 'hdkpcmpdyjildw', 'frxkyukiywngfcxfzwkcun', 'dmvxf', 'esamivybor', 'eoggdynqwlnlxr']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('zggbeilrfpkleafjjldgyfgn'), ['rpypxkpgvljhqthneremvabcd', 'qchzlnsctuwkdxqcrjgihvtfxhqxfqsxm', 'vtozkivjyqcqetmqenuihq', 'fixcvjyzbzejmwdivjf', 'lydoolvnuuamwlnzbyuuwpqqjaxf', 'elkodwthxqpcybwezm', 'wpiju', 'wdzuuwumlqfvga', 'iokphkai', 'wkbwdstplhivjyk', 'wxfbhfturuqoymwklohawgwltptytc', 'jehprkzofqvurepbvuwdqj']) from system.numbers limit 10; +select 9 = multiSearchFirstIndex(materialize('bwhfigqufrbwsrnnkjdzjhplfck'), ['v', 'ovusuizkdn', 'ttnsliwvxbvck', 'uh', 'lfourtjqblwdtvbgtbejkygkdurerqqdwm', 'snmtctvqmyyqiz', 'ckpixecvternrg', 'gluetlfyforxcygqnj', 'igqufrbwsr', 'om', 'huwazltjsnohsrcbfttzwyvcrobdixsuerkle', 'gqufrbwsrnnkjdzj', 'hfigqufrbwsrn', 'lhhyosbtznyeqzsddnqkfxayiyyajggxb', 'igqufrbwsrnnkjdzjhplf', 'pl', 'jtbqaqakbkesnazbvlaaojppxlbxccs', 'gqufrbwsrnnkjdz']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('yevfiumtjatfdnqixatbprvzwqlfgu'), ['ozghvskaixje', 'vmdrvdjhwxdvajmkpcxigsjzmtuhdxgllhzrpqd', 'qfhnxpcmtzpociajidwlcvobjfyxfcugsxy', 'pgamvhedjibcghinjrnowqzkfzibmfmh', 'bcmrdzpcczhquy', 'czosacvwfsbdvwwyirpvbve', 'qu', 'fdkobwlnmxbpvjkapextlbcrny', 'bqutjqobkyobhtpevjvewyksnoqyjunnnmtocr', 'kjlgff', 'oitltmhdburybwfxrjtxdiry', 'kiokuquyllpeagxygqugfmtm', 'wlbkl', 'khubpmstqjzzjzmsvfmrbmknykszqvue', 'lqrbmyndsztyrkcgqxcsnsanqjigimaxce', 'nitnyonuzedorrtkxhhgedohqcojbvtvjx']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('wmvuoeqphsycrvtxghrcozortmdnh'), ['hv', 'ugcmpebvlzgdtcmgkbgzyfel', 'qvmofayljsvybupvvnbhhibsz', 'zvlihxmyxlxwbffwjzjrfjgimmltftqqre', 'mwassqvxptav', 'jrumvqzkiaewngoufhrleakcfrsaxhpxyg', 'sxlxwhvkpavgfhxrxcbnqbstyrejtosxwe', 'psnlqakyfhcupryqatrmwqlswwjylpaiqammx', 'ivozojwldsgtnxpvsi', 'epyzjs', 'legi', 'sdqxxahfbddhacqrglgdcmlslraxfaahhfyodon']) from system.numbers limit 10; +select 12 = multiSearchFirstIndex(materialize('lebwdwxfdzwquhqhbvmte'), ['mwhruilzxvlyrgxivavxbbsq', 'ubuiizuasp', 'xpkzcsf', 'qpeqitoqqqeivohajzhmjbo', 'kbftixqmqgonemmbfpazcvf', 'iyhluioqs', 'hws', 'tupfdksgc', 'ows', 'pngzkoedabstewcdtdc', 'zdmyczldeftgdlwedcjfcoqycjcivf', '', 'xt', 'syuojejhbblohzwvjzzedzgmwc']) from system.numbers limit 10; +select 7 = multiSearchFirstIndex(materialize('wcrqaoecjwkhnskrbahqxfqgf'), ['qegldkdmyaznlmlhzvxfgoukngzbatnuq', 'khgcvgrifwtc', 'hkwcpogbbdqulizrycmneqmqynvj', 'zkqjf', 'xfduxyy', 'ructdekcoywfxsvpumfefoglljptsuwd', 'wkhnskrbahq', 'crqaoecjwkh', 'ikmpbunpguleinptzfelysiqc', 'lhldcci', 'nooepfypkoxxbriztycqam', 'uxeroptbiqrjartlnxzhhnlvjp']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('psgkkcwttitgrjsobiofheyohadu'), ['achfrepey', 'minlzeiwgjfvvmhnevisky', 'oxfghfdthtyczzveppcoxrued', 'ydhaupodnezvxhcqahfkwtpvxnymriixf', 'slxsbxidylxyurq', 'socyyabwbjdabnuqswrtjtqogirctqsk', 'lvbnacirctyxxspjmispi', 'oj', 'ihmmuuqlosorrwhfxvpygfrzsqpmilcvjodmcz', 'idmtmemqfyrlbwhxz', 'hsqfsfdzvslwbtlwrfavez', 'gszl', 'ei', 'pnywjnezncpjtyazuudpaxulyv', 'iqgavdjfqmxufapuziwwzkdmovdprlhfpl', 'yigk', 'mjidozklrpedutllijluv', 'vixwko']) from system.numbers limit 10; +select 3 = multiSearchFirstIndex(materialize('xtjxvytsseiqrpkbspwipjns'), ['bwmoghrdbaeybrmsnucbd', 'zoslqabihtlcqatlczbf', 'sseiqrpkbspwipjn', 'mdnbzcvtayycqfbycwum', 'npueimpsprhfdfnbtyzcogqsb', 'ytsseiqrpkbspwipj', 'fzvhcobygkwqohwutfyauwocwid', 'naacyhhkirpqlywrrpforhkcjrjsnz', 'vezbzderculzpmsehxqrkoihfoziaxhghh', 'mvvdfqzskcyomjbaxjfrtmbduvm', 'pwipjns', 'tsseiqrpkbspwipjn', 'sseiqrpkbspwip', 'qgrtbcdqcbybzevizw', 'isjouwql', 'rlbeidykltcyopzsfstukduxabothywwbq']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('zxmeusmehplcgbqabjof'), ['hqxgrw', 'fydjyrr', 'cocwtbazwjrswygttvrna', 'wpkvowuq', 'mwnzdxihrxihzhqtl', 'ljkjtmrfbonhqkioyzotyeegrw', 'ofxo', 'rjubwtpbweratrelqlrqotl', 'wvxkcil', 'qvolxxgqs', 'afqlhjnlvxowtnuuzywxuob', 'slwbmq']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('tjcmtoisgbilkygushkpuxklis'), ['bkdohwx', 'dfohgzhcjqirlbrokwy', 'zaemgqgxltznvkccyumhgsftnfigbol', 'otgcaybejwe', 'qn', 'gvfzcyhvmsnbgkulsqrzeekmjkc', 'cajuyauvmhkrriehgwfmtqbkupysudle', 'pmcupysyllzpstolkfpdvieffxaupqtjty', 'elhlzvescbfpayngnnalzixxgunqdhx', 'cvxpgdnqcxeesk', 'etlewyipypeiiowuoewulkpalvcfe', 'ordhwrkwqq', 'wnroixlkrqnydblfrtlbywc', 'xshujuttvcdxzbetuvifiqi', 'meqqxqhntkvzwoptnwskdgsxsgjdawe', 'dnmicrfshqnzosxhnrftxxeifoqlnfdhheg']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('iepqqbvekaflprupsmnpoijrld'), ['kqomoeysekwcplpegdwcdoeh', 'mwdvr', 'aobviioktzwzmpilblbdwstndhimabfgct', 'vqustluciruiyfkoontehnwylnauwpol', 'utcqnitztcgr', 'ityszrqmlwzspnrwdcvdhtziob', 'hmll', 'ilfzvuxbkyppwejtp', 'euxdzqcqutnfeiivw', 'rbcjlmjniiznzaktsuawnfjzqjri', 'fzyxlzzretsshklrkwru', 'jrujmdevqqojloz']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('cufztqffwjhtlkysekklpaywemm'), ['cpawuauqodogaitybtvplknjrsb', 'ynsocxfnxshzwnhlrfilynvz', 'ylrpytgcvtiumdckm', 'mvgrkueaslpgnjvvhzairgldtl', 'iliorsjypskmxfuuplfagktoycywb', 'drvwngp', 'zviuhcxaspwmqqz', 'qfgmrmhycskus', 'szj', 'rooivliiqufztcqlhrqyqvp', 'tufdmsmwue', 'cssowtldgwksbzlqyfereodcpuedighwd', 'odcjdffchhabtaxjvnr', 'o']) from system.numbers limit 10; +select 7 = multiSearchFirstIndex(materialize('zqwvlarwmhhtjjgwrivwfpsjkvx'), ['zcwhagxehtswbdkey', 'okezglmrjoim', 'ilwdviqimijzgoopmxdswouh', 'aqztpsntwjqpluygrvwdyz', 'uzxhjuhiwpz', 'akgc', 'larwmhhtjjgwrivwfpsj', 'isqghxsmcrwlgyloslmlyeboywtttgejdyma', 'arwmhhtjjgwri', 'rwmhhtjj']) from system.numbers limit 10; +select 9 = multiSearchFirstIndex(materialize('fuddujwwcewlhthgwsrn'), ['shtzrrtukxmdovtixf', 'rkcnzzzojqvvysm', 'jlamctgphjqcxlvmpzyxtghnoaq', 'pthrwvbheydmrot', 'kpniaqbcrgtxdyxxdxonbbltbdo', 'igulngxgtauumhckvbdt', 'khgrmskijoxruzzzaigjxonsc', 'rxzeykfxwssltw', 'hthg', '']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('jtgvvkggpkqhbxptjgoy'), ['nplzawmacgtqfxsp', 'oosw', 'akw', 'hnsenqoqwiydiufozomkyirgjepeqw', 'fpafgahvfdxukzvskbuy', 'tqimmsqffiqfoni', 'rrxkjklmkdhxqwcpfyutqzxu', 'esfqeujcbqxwnvodkwwdbsyozptaf', 'rqnyguyz', 'fftl', 'ccfyavxtxrpi', 'wftpsblszgovfgf']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('steccxkwnptybaddcuau'), ['qagxfznhjaxtyclxdsi', 'rtxwptfyzgthkwrx', 'rmcoxxs', 'vlubx', 'siecygstzivz', 'tksiagm', 'kq', 'dgsqrobxegmdbjkanb', 'lxokyvhveklvdakrxyiqokr', 'tgpmehwdrirpfjonqzhqshbo', 'cqmkargvsfjoxrguymtzsfwkg', 'avkmufhoywprjw', 'xzywtvlpoozmgkrcavevwebv', 'hfiuwslapamiceaouznxm', 'tmfjhqddafhhjbybfphlbwu', 'mrigvhmjvdpny']) from system.numbers limit 10; +select 0 = multiSearchFirstIndex(materialize('ccbgxzoivbqtmyzqyooyepnmwufizz'), ['lcclseplkhxbrrzlnani', 'xggxivwqlpxmpypzovprdkmhrcgjkro', 'dbbmiegotfxjxybs', 'hqtcowpupsyqfx', 'znatfzjbeevbaqbmpofhywbyfxn', 'mnditiygex', 'lazqapwjswhkuimwmjoyseyucllnrfxrwnzj', 'jg', 'dmqwnuvsufgffuubhqeugwcanvflseorrydyyxvr', 'wpjfcfwfgjiybncrw', 'joucnvxxcyjyqlwhrzwnstyj', 'babtxkzasyaffxzd', 'wgcfdyhwxjoytbxffdxbdfinolbltnhqkvyzybc', 'yhrgwbdwopznltjtyidxawqg', 'bvrrt', 'bcwmsys', 'ijdjojhhzaiyjyai', 'eevxwppogogdbmqpbeqtembiqxeiwf']) from system.numbers limit 10; +select 2 = multiSearchFirstIndex(materialize('xrwjeznohtbdvijwsbdksf'), ['hwdfufmoemohatqafdrcvdk', 'tbdvijwsbdks', 'xzwjczbuteujfjifzkbxvezs', 'bdvijwsbd', 'eznohtbdvijwsbdks', 'xadezwhbbmlqz', 'b', 'socrdjxsibkb', 'dk', 'eznohtbdvijws', 'pavsosnncajr', 'jixlmxxmxnnbpebjhitvtsaiwzmtqq', 'yuxmmnrqz', 'mpzytweuycabvu', 'tbdvi', 'ip']) from system.numbers limit 10; + +select 0 = multiSearchFirstIndexUTF8(materialize('црвтгмсрооацволепкщкпнгшкамщ'), ['гйцбсханрейщжнфбхтщбйала', 'дирдфнжпнччхаоцшрийнйнечллтгцбфедгсш', 'жфйндбффаилбндмлточиирасдзйлжбдзег', 'жвоуйфсйойфцвгзшцитсчпкч', 'ршонтбгщжооилчхрзшгсдцпзчесххцп', 'пйучихссгнхщлутвменлмм', 'хишгешегдефесо', 'знупгж', 'щчфу', 'знвтжифбнщсибеноожжметачаохфхсжосдзйуп', 'ггтоцйпгхчсбохлрчлваисивжбшбохдурввагш', 'щлийбчштбсч']) from system.numbers limit 10; +select 5 = multiSearchFirstIndexUTF8(materialize('опднхссгртрхтотлпагхжипхпитраб'), ['шфршсцешушклудефцугщцмйщлошечедзг', 'нйумйхфщцгщклдожхвосочжжислцрц', 'згтпвзцбхйптцбагсвцгтнф', 'пшичси', 'ссгртрхтотлпа', 'апзазогвсбежзрйгщоитмдкн', 'непгайтзкгштглхифмзданоихц', 'пднхссгртрхтотлпагхжипхпитр', 'ждднфлрзалшптсбтущвошрйтхкцнегшхрсв', 'брп', 'сгртрхтотлпагхжипх', 'нхссгртрхтотлпагхжипхп', 'пагхж', 'мфкжм']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('овччцнтчайомсйййоуйуучщххиффсб'), ['жжрддцпнехйр', 'шзбвуооинпаххесйкпкошжмцзгхе', 'ррсннилщлщжгцтйрпхабкехахззнтщемагдйшпсч', 'пуфугнказепщ', 'гддхтплвд', 'сщсчи', 'бйрсахедщфкхиевкетнс', 'йфжцжшпхлййхачзхнфоц', 'цтмтжлщдщофисзрвтбо', 'кщсевбоуйб', 'щгаапзкн', 'осймщовшчозцййизм', 'фкмаат', 'бкзцсдонфгттнфтаглпрцтбхбсок', 'жлмичлйнйсжбгсейбсиезщдмутационжгмзп', 'нбищижнлпмтморлхцхвеибщщлкйкндлтпбд']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('фдситчщдвхмфйтхшдтуцтщжрочщо'), ['ейшфдннтйечгк', 'фуйщгбйшдцирзб', 'ехйцмчщрсртнк', 'увтцмдорщжфгцгзущпувтщкнрфсйбщрзй', 'хчщпхвуарнббпзсцшчщуносйгщпсбтх', 'жтдчрхфмхцххккзппзбнуббс', 'тчохнмбаваошернеймгготлузвсбрщезднеил', 'стссчкшрчррйбхдуефвеепщшзмербгц', 'жбезжпещ', 'вйтсрхптлкшвавдаакгохжцоощд', 'искеубочвчмдхе', 'щмлочпзбунщнхлрдлщтбеощчшчхцелшоп', 'екуийтсйукцн', 'дочахгжошвшйжцпчзвжйкис', 'лтеенешпсболгчиожпжобка', 'букзппщрчбпшвпопвйцач']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('гопвмрутфпфбхмидшлуб'), ['цнхшдойгщн', 'дкаежщрапщпщеа', 'фмогимдничрфтхмсцмчпдфтиофнтйц', 'фчмсщисхщуп', 'ощмвдчефозйжбеесбмещочевцчд', 'апкбцйщжщабвппофм', 'мтйоддлфцгдуммптднпщшрн', 'икхнсмжчбхнфхнссгл', 'ущмунинлбпрман', 'ллкнечрезп', 'ажтнвбиччджсзтйешйффдгдрувер', 'йрщ', 'чигдкйшфщжужзлвщулквдфщхубги', 'иккшсмаеодейнкмгхбдлоижххдан']) from system.numbers limit 10; +select 12 = multiSearchFirstIndexUTF8(materialize('срлцчуийдлрзтейоцгиз'), ['жщлнвбубжпф', 'оклвцедмиср', 'нлзхмчдзрззегщ', 'хоу', 'шайиуд', 'ерслщтзцфзвмйтжвфеблщдурстмйжо', 'жмгуйузнчгтт', 'стеглмрдмирйрумилвшнзззр', 'втедлчрчайвщнллнцдмурутш', 'цимхргмрвмщиогврнпиччубцйе', 'ктчтцбснзцйцймридвш', 'ейоц']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('лрицжленфилзсжпжйнцжжупупдфз'), ['чпбрмлрнцмвеуфу', 'рмпизмпжчшбхдудчшохтжш', 'гргцжчпгщищннусв', 'ийщтщвзчшпдзитщубакусхавслрсбткб', 'бйбакижцтибгбгхжцвйчжжщжсжкзф', 'чгрп', 'чуносжусжфчмфжхрщзлщрдвбашажаанча', 'чекршбш', 'лбцкхйсооцц', 'сгвнлегвфмпчтййлрмд', 'наатущркхйимхщщг', 'щпзоеимфощулбзхафпц', 'дцабцхлврк', 'умидмчуегтхпу', 'дщнаойрмчсуффиббдйопдииуефосжхнлржрйлз', 'щзжетезвндхптпфлк', 'бгчемкццдбжп', 'иихуеоцедгрсеужрииомкбззцнгфифоаневц']) from system.numbers limit 10; +select 3 = multiSearchFirstIndexUTF8(materialize('бхжвчашрощбмсбущлхевозожзуцгбе'), ['амидхмуеийхрнчйейтущлуегрртщрхвг', 'фнисцщггбщйа', 'хжвчашрощбмсбу', 'фщвщцнеспдддцчччекчвеещ', 'ущуджсшежчелмкдмщхашв', 'цкуфбиз', 'евозожз', 'ппт', 'лвцнелшхцш', 'ощбмсбущлхев', 'ефхсзишшвтмцжнвклцуо', 'цржсржмчвмфмнеещхмиркчмцойвйц', 'ашрощбмсбущлхевозожзу', 'гхщншфрщзтнтжкмлщанв', '', 'хевозо', 'ощбмсбущлхевозожзуц', 'возожзуц']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('мзчатйжщгтзлвефчшмлшт'), ['гхшфрунирйдзтеафщгк', 'ймхмфлц', 'звуумивмвштчтнтеобзщесакийгк', 'чщжетзнцишхрммтбцакиббчп', 'блмидикавущщдпгпчхйаатйанд', 'цмщшбклгцгмчредмущаофпткеф', 'бнетввйцзпдерхщ', 'ицйнцрввемсвтштчфрпжнатаихцклкц', 'дзлщсштофвздтмчвсефишс', 'пбзртдцвгкглцфесидлвваисщр', 'ммеилбзфнчищч', 'жш', 'лздиззтпемкх', 'байлужднфугмкшгвгулффмщзхомпав', 'рсзнббедсчзущафббзбйоелид', 'цфшйкцксйгуйо']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('жжмзмащйфжщлрффбпврзнидииейщ'), ['ржфзнлйщсздйткаоцруйцгцт', 'илинксщмгщшещееифвпданмйлж', 'кг', 'гпааймцщпмсочтеиффосицхйпруйшнццвс', 'кнзфгжйирблщлл', 'ищуушфчорзлкбцппидчннцвхщщжййнкфтлрдчм', 'тбтдчлвцилргоргжсфбоо', 'ехаех', 'нехщмдлйджждмрцпйкбрнщсифхфщ', 'тцжпснйофцжфивзфбхзузщтмдкцжплавозмше']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('биаризлрвххжкпщтккучфизуршткпн'), ['йбручвндбщвссаеха', 'ол', 'еузкмпогщзгзафшдшоплбфнфдккх', 'ибзихщйфбтаз', 'ибрчиейш', 'нафрпбснзрузнтмнйиомтечтшзбкпзутдилтф', 'тщтбапцчдий', 'щкнггмфцжрзщцзжвлкчбммхтхтуж', 'ваам', 'цкфиушзигбжтацнчдлжжзфшщммтнлж', 'туфовжтнкзщсщщизмрйкхкпц', 'пирзксзикфтшодожшчцг', 'жфчфцфвлйбмеглжйдазгптзщгж', 'тутириждкзчвтсоажп', 'мотзусбхту', 'слщкгхжщфщоцкцтрлгп', 'бругтбфесвсшцхнтулк', 'восур', 'ссежгнггщдтишхйнн', 'вгзосзгоукмтубахжнзгшн']) from system.numbers limit 10; +select 8 = multiSearchFirstIndexUTF8(materialize('мчслвбжвманджййсикнврцдчмш'), ['рлбмй', 'иб', 'жажлцсзхйфдцудппефвжфк', 'огггхзгтцфслхацбщ', 'дзтцкогаибевсйещпг', 'зпцтйзфмвгщшуоилл', 'етщзгцпдйчзмфнхпфцен', 'нджййсик', 'сикнврцдчмш', 'жййсикн', 'икнврцдч', 'паокаочввеулщв', '', '', 'кечзсшип', 'вбжвманджййсикнвр']) from system.numbers limit 10; +select 2 = multiSearchFirstIndexUTF8(materialize('нвррммппогдйншбшнехнвлхм'), ['нфошцншблеооту', 'лх', 'цртд', 'огдйншбшн', 'уулддйдщицчпшбоиоцшй', '', 'дрдужзжпцкслетгвп', 'й', 'мппогдйншбшнех', 'дйншб', 'лжвофчзвдд', 'рммппогдйншб', 'ехнв', 'втущсщзбчсжцмаанчлнасп']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('удехбкабиацхпгзнхжелшц'), ['фмнбтйезсфоахофофдблкжщжфмгхтзс', 'тщтамзафозхлз', 'цшжфсбл', 'йзгзилупшллвипучхавшнмщафзмнк', 'лу', 'гтебпднцчвмктщсзи', 'лпщлмцийгуеджекшд', 'пцдхфоецфрунзм', 'зис', 'хпж', 'цтцплхцжишфнплуеохн', 'впх', 'чцчдацлуецрчцжижфиквтйийкез', 'гчшмекотд', 'пйгкцчафеавзихзтххтсмкал', 'сжфхпцгдфицжслрдчлдхлсувчнрогнву']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('щщвфчгамтжашнуошбзшуйчтшх'), ['дийу', 'жеомлсжщймемрсччошдфажцтдп', 'нгопнцквбф', 'хопб', 'ив', 'чвфвшфрдфелрдбтатшвейтг', 'вхкцадмупдчбаушшлдксйв', 'жтжбсвмшшсйеуфдпбдлкквдиовж', 'гтсдолснхесйцкйкмищгсзедх', 'ошплп', 'ифпуррикбопйгиччи', 'чдфймудаибвфчжтзглс', 'зпцмвпнлтунвйж', 'еждрйитхччещлцч', 'вмофсужхгрнзехкх', 'щжгквкрфжмжжсефпахст']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('рфгигуужжцфмоаешщечувщгонт'), ['слащченщлуоцргврбаб', 'тцизут', 'лйрсцолзклжбчрзгббммоищщ', 'уицмлоилзф', 'зпхмшвфйккфщщп', 'ймижрпдщмшв', 'пуощжлрмжлщхмкйгщшщивдпчпжчл', 'ойахшафнж', 'гксомбвцрсбжепхкхжхнсббци', 'панлраптщмцмйфебцщемйахенг', 'сохлгожштлднчсзпгтифсйгфмфп', 'аждчвзну', 'дхшуфд', 'борзизцхнийбщгхепрнзшй', 'фщшздруггрке', 'оевупрйщктнолшбкунзжху']) from system.numbers limit 10; +select 8 = multiSearchFirstIndexUTF8(materialize('кщзпапйднучлктхжслмищ'), ['апмдйлсафхугшдезксш', 'кйрм', 'цйивайчшуалгащсхйш', 'злорнмхекг', 'сгщврурфопжнлхкбилдч', 'бнлпщшнвубддрлижпайм', 'нукдонццнрмовфнбгзщсшщшдичежффе', 'йднучлктхжс', 'зпапйднучлктхж', 'затйотдсмпбевлжаиутсуг']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('жцажссефррршнфмнупщаоафгкщваа'), ['жфпщкгзкрмтщчцтжйчпйдошбшоцд', 'бхгйлйдробптвущшппзуиидежнлтпбжащткцф', 'хлещазйцепдханпажчизнхгншйуазщхй', 'ашцк', 'фрбммхдднчзшс', 'нжцанилзжаречвучозрущцдщаон', 'длмчзцрмжщбневрхуонпйейм', 'шкбщттврлпреабпоиожнууупшмкере', 'вуцпщдиифпеоурчвибойбпкпбкйбшхдбхнаббж', 'нртжвкдйтнлншцанцпугтогщгчигзтоищпм', 'цкплнкщлкшемощмстздхпацефогтск', 'цвждйбсмпгацфн', 'шсжшрзрардтпщлгчфздумупд', 'цйииткглчжйвуейеиииинврщу', 'унлодтулшпймашоквббчйнибтвалалрвбцж', 'нбнфнвйишйжлзхкахчмнлшзуеенк', 'бшлпсщжквпцахигчдтибкййб', 'фчакпзовтрлкншзцулшщмпзж']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexUTF8(materialize('иматеччдфлггшпучумджпфпзмвх'), ['дахахпчлцлаачгцгтфпнжлшчйуцбшсг', 'атжйувхец', 'грдсбвиднницдвшпйршгмегцаоопнжгй', 'чзлхречмктфащмтеечуиагоуб', 'савбхлпилийщтихутйчдгфсойй', 'вбгочбзистзщшденусцофит', 'мар', 'дфшажхдсри', 'тжлмщшж', 'птсрсщгшммв', 'ре', 'зратамкткфкинййй', 'гуцмсизулвазужфдмхнелфнжббдтрудчтнфцр', 'нйчинеучкхнпчгнйвчвсвлгминуцахгщввжц', 'ечагчнуулфббгбел', 'йшжуговрйкащцофдокфчушжктнптйеззушфо']) from system.numbers limit 10; +select 11 = multiSearchFirstIndexUTF8(materialize('азтммйтшхцхлгдрнтхфжбдрлцхщ'), ['нпучщфвспндщшспзмшочгсщжчйгжбжзжжтн', 'хккдйшабисдузфртнллщпбоуооврайцз', 'йпхрфжждгпнйаспйппвхбргшйвжччт', 'ффеее', 'кежцновв', 'еххрчштарзмкпйззсйлмплхбчбулзибвчбщ', 'шфжйдотрщттфхобббг', 'ожоцжущопгоцимсфчйщцддзнфи', 'цуимеимймкфччц', 'прммщмтбт', 'хцхлгдрнтхфж', 'лгд', 'цжбдаичхпщзцасбиршшикджцунйохдлхй', 'пидхцмхйнспйокнттмййвчщпхап', 'йтйзмеаизкшйошзвфучйирг', 'хцхлгдр']) from system.numbers limit 10; + +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('gyhTlBTDPlwbsznFtODVUzGJtq'), ['seSqNDSccPGLUJjb', 'xHvtZaHNEwtPVTRHuTPZDFERaTsDoSdX', 'QCeZOYqoYDU', 'bsybOMriWGxpwvJhbPfYR', 'FFHhlxfSLzMYwLPPz', 'tvDAJjaLNCCsLPbN', 'kOykGaSibakfHcr', 'mWAZaefkrIuYafkCDegF', 'ILrFDapnEDGCZWEQxSDHjWnjJmeMJlcMXh', 'zHvaaTgspUDUx', 'tss', 'laUe', 'euUKFLSUqGCjgj', 'Kd', 'MxyBG', 'qRXMsQbNsmFKbYSfEKieYGOxfVvSOuQZw', 'PdBrNIsprvTHfTuLgObTt', 'kMekbxI']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('ZxTznPEbfoBfLElYOrRiHrDLMmTpIh'), ['bJhYwKLeeLvLmXwWvQHWFkDQp', 'dLyZmUicTZmUfjfsFjxxgOiMJn', 'UCYbbGcY', 'kpPiwfWHEuh', 'jviwmHeiTQGxlTKGVEnse', 'cVnEyLFjKXiLebXjjVxvVeNzPPhizhAWnfCFr', 'gkcoAlFFA', 'ahZFvTJLErKpnnqesNYueUzI', 'VIJXPlFhp', 'rxWeMpmRFMZYwHnUP', 'iFwXBONeEUkQTxczRgm', 'ZnbOGKnoWh', 'SokGzZpkdaMe', 'EfKstISJNTmwrJAsxJoAqAzmZgGCzVRoC', 'HTmHWsY', 'CpRDbhLIroWakVkTQujcAJgrHHxc']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('VELfidaBvVtAghxjkrdZnG'), ['fvEFyRHvixuAYbuXygKeD', 'zFNHINreSOFksEGssBI', 'hcdWEcKDGWvfu', 'KczaFjvN', 'nZLTZAYSbfqcNWzWuGatDPUBYaRzuMBO', 'UdOdfdyPWPlUVeBzLRPMnqKLSuHvHgKX', 'DgVLuvxPhqRdSHVRSeoJwWeJQKQnKqFM', 'NNfgQylawNsoRJNpmFJVjAtoYy', 'tWFyALHEAyladtnPaTsmFJQfafkFjL', 'lYIXNiApypgtQuziDNKYfjwAqT', 'QjbTezRorweORubheFFrj', 'htIjVIFzLlMJDsPnBPF', 'ltDTemMQEgITf', 'fprmapUHaSQNLkRLWAfhOZNy', 'dOJMvPoNCUjEk', 'm', 'vEEXwfF', 'aVIsuUeKGAcmBcxOHubKuk']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('kOzLaInSCOFHikwfkXaBfkyjdQ'), ['t', 'emHGfAiZSkZaVTSfplxRiPoDZUTT', 'YHnGJDTzxsboDsLPGHChMHwrHHICBIs', 'gbcbVHSlVeVDOeILWtSLkKfVVjG', 'fPaJjbnNthEwWZyg', 'qS', 'PCQxoLaSdQOjioMKPglmoWR', 'KLMNszm', 'TCErEFyxOvqnHs', 'dRbGzEJqvIGAcilZoHlXtZpjmLLZfsYueKqo', 'iKHmNSbGgaJYJEdMkbobXTdlFgAGEJMQ', 'mUGB']) from system.numbers limit 10; +select 1 = multiSearchFirstIndexCaseInsensitive(materialize('JGcICnWOGwFmJzHjtGJM'), ['fmJzHj', 'LhGTreYju', 'yCELHyNLiAJENFOLKOeuvEPxDPUQj', 'kWqx', 'OBnNMuaeQWmZqjWvQI', 'ektduDXTNNeelv', 'J', 'iCNwoGwfMJzhjtGJ', 'uiIipgCRWeKm', 'bNIWEfWyZlLd']) from system.numbers limit 10; +select 7 = multiSearchFirstIndexCaseInsensitive(materialize('fsoSePRpplvNyBVQYjRFHHIh'), ['ZqGBzyQJYuhTupkOLLqgXdtIkhZx', 'pouH', 'mzCauXdgBdEpuzzFkfJ', 'uOrjMmsHkPpGAhjJwVOFw', 'KbKrrCJrTtiuu', 'jxbLtHIrwYXDERFHfMzVJxgUAofwUrB', 'PLvNyBVQYjRfhhi', 'wTPkeRGqqYiIxwExFu', 'PplvNybvqyJ', 'qOWuzwzvWrvzamVTPUZPMmZkIESq', 'ZDGM', 'nLyiGwqGIcr', 'GdaWtNcVvIYClQBiomWUrBNNKWV', 'QQxsPMoliytEtQ', 'TVarlkYnCsDWm', 'BvqYJr', 'YJr', 'sePrPPLVNYbvqYJRFhh', 'ybvq', 'VQYjrFHh']) from system.numbers limit 10; +select 3 = multiSearchFirstIndexCaseInsensitive(materialize('aliAsDgMSDPISdriLduBFnuWaaRej'), ['gWOFTxMrQGQaLrpJamvRhgeHwk', 'iWsBLzLycWvbJXBNlBazmJqxNlaPX', 'Ri', 'FPLRURSsjvsySncekcxaWQFGKn', 'wgXSTVzddtSGJQWxucYorRjnQQlJcd', 'wOLJWZcjHEatZWYfIwGIqnuzdcHKSFqfARfNLky', 'eEECZMNmWcoEnVeSrDNJxcOKDz', 'duBF', 'EhfLOjeEOQ', 'dUbFNUWA']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('EUzxPFYxMsJaTDzAKRXgZIVSFXU'), ['TDKAgICICjzBKHRqgFAuPCSODemldGGd', 'LvMluSJTIlgL', 'srbRhQKjPIchsipVHsjxwhK', 'vdurVsYkUWiFQVaDOnoNIJEX', 'UzZsZqAUNjMvWJaTqSWMHpzlDhVOaLzHPZfV', 'XcnnPXXEJJv', 'JSwFBNnYzNbIRZdeMfYiAfxzWfnCQFqoTUjns', 'HBMeqdLkrhebQeYfPzfJKAZgtuWHl', 'cMfSOnWgJvGhFPjgZdMBncnqdX', 'orDafpQXkrADEikyLVTHYmbVxtD', 'Vz', 'bfYwQkUC', 'q', 'YqomKpmYpHGv']) from system.numbers limit 10; +select 4 = multiSearchFirstIndexCaseInsensitive(materialize('mDFzyOuNsuOCSzyjWXxePRRIAHi'), ['TfejIlXcxqqoVmNHsOocEogH', 'clyblaTFmyY', 'JQfxMAWVnQDucIQ', 'jw', 'fGetlRA', 'uWwCOCd', 'rInhyxSIFiogdCCdTPqJNrqVaKIPWvLFI', 'mimSJjfCWI', 'jqnJvNZXMEPorpIxpWkhCoiGzlcfqRGyWxQL', 'bxCJeVlWhqGHoakarZcK', 'unsUOcSZyjwxxe', 'E', 'PR', 'nsUoCSZyjwxXEPr', 'sfotzRPMmalUSjHkZDDOzjens', 'zYJwxx', 'DFzyouNsUocsZ', 'QBaQfeznthSEMIPFwuvtolRzrXjjhpUY', 'sQPVBaoeYlUyZRHtapfGM', 'lPiZLi']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('VOAJfSkbDvNWAZNLIwqUgvBOddX'), ['pHrGGgJ', 'VohjtPdQZSNeKAlChDCnRTelroghFbZXVpnD', 'rnWebvdsmiFypMKL', 'NtKRiJOfAkWyKvubXrkOODgmZxvfOohsnHJEO', 'nxsDisKarasSZwESIInCJnYREUcoRUTXHBUH', 'mXYYr', 'jujScxeTBWujKhKyAswXPRszFcOKMSbk', 'INEegRWNgEoxqwNaGZV', 'VVyjMXVWVyuaOwiVnEsYN', 'mkLXSmXppxJhFsmH', 'pRVnBrWjqPeUDHvhVuDbzUgy', 'PzchFdPTkOCIVhCKml', 'KXaGWnzqoHBd', 'PhzQVqIOLleqDSYNHLjAceHLKYPhCVq', 'aixxTqAtOAOylYGSYwtMkZbrKGnQLVxnq', 'ruEiaxeRaOOXGggRSPlUOGWSjxh', 'prSULtHvDMw', 'vEpaIIDbGvIePYIHHZVNSPYJl']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('ZHcEinZEFtfmHBLuCHntUhbIgY'), ['GKElMPEtmkLl', 'mkrzzjSRfXThuCQHkbZxRbhcymzTxcn', 'PREwQjxBJkpkiyuYEvtMZNFELgbINWsgf', 'lFEGlPtaDJSyoXzwREiRfpzNpsaBYo', 'tmVTuLPhqhgnFNhHvqpmc', 'NtijVhVfAwpRsvkUTkhwxcHJ', 'O', 'FSweqlUXdDcrlT', 'uljEFtKVjIzAEUBUeKZXzCWmG', 'dBIsjfm', 'CNaZCAQdKGiRUDOGMtUvFigloLEUr', 'yWjizKZ', 'QqPVdyIFXcweHz', 'uPmgGWGjhzt']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('AYMpbVsUQqAfoaMiJcYsulujYoSIx'), ['aXECumHNmAEefHPJy', 'hTosrERBdVCIilCYcMdHwaRh', 'PVDBpwrc', 'uFvQRPePvmzmocOauvEjqoxMhytzOwPSOCjmtm', 'kQqIlSCHDmWXCKN', 'ybAHGYDEDvvOJsF', 'WpkANi', 'cFGuzEcdahZtTdLFNBrRW', 'EBaybUFxO', 'mRlZUzHzMsMAgvtRtATEDLQvXZnZHw', 'uqxckjqpCBHiLgSPRz', 'Lv', 'AJcRfAvBmQVMOjaFfMfNHJt', 'FYsPM', 'pkKXTPgijOHFclqgVq', 'Ck']) from system.numbers limit 10; +select 11 = multiSearchFirstIndexCaseInsensitive(materialize('gmKSXWkNhKckrVNgvwiP'), ['bdJMecfCwQlrsgxkqA', 'NTgcYkMNDnTiQj', 'fmRZvPRkvNFnamMxyseerPoNBa', 'rfcRLxKJIVkLaRiUSTqnKYUrH', 'YSUWAyEvbUHc', 'PridoKqGiaCKp', 'quwOidiRRFT', 'yHmxxUyeVwXKnuAofwYD', 'gichY', 'QlNKUQpsQPxAg', 'knhkCKRVNGvWIp', 'jAuJorWkuxaGcEvpkXpqetHnWToeEp', 'KnHKCKrvNgVW', 'tCvFhhhzqegmltWKea', 'luZUmrtKmmgasVXS', 'mageZacuFgxBOkBfHsfJVBeAFx', 'hKC', 'hkRCMCgJScJusY', 'MKSXWknHkckrVNgv', 'osbRPcYXDxgYjSodlMgV']) from system.numbers limit 10; +select 15 = multiSearchFirstIndexCaseInsensitive(materialize('lcXsRFUrGxroGIcpdeSJGiSseJldX'), ['pBYVjxNcQiyAFfzBvHYHhheAHZpeLcieaTu', 'SQSQp', 'OQePajOcTpkOhSKmoIKCAcUDRGsQFln', 'AYMDhpMbxWpBXytgWYXjq', 'gkUC', 'oWcNKfmSTwoWNxrfXjyMpst', 'fQSqkjRNiBGSfceVgJsxgZLSnUu', 'LRrhUjQstxBlmPWLGFMwbLCaBEkWdNJ', 'cZnaActZVoCZhffIMlkMbvbT', 'Uxg', 'vlKdriGMajSlGdmrwoAEBrdI', 'Fl', 'XzcNdlUJShjddbUQiRtR', 'AqowAuWqVQMppR', 'SRFUrGXrOgiCP', 'k']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitive(materialize('KhwhbOzWvobUwJcteCHguFCn'), ['LkDYrpvDfPL', 'CIaTaShobVIaWjdbsNsCMdZKlGdtWuJmn', 'zYcsxxFyfuGrPdTPgEvGbXoYy', 'vDIeYpJbLMGMuRkIrPkAnqDDkqXPzy', 'Ievib', 'CREiuEsErFgvGEkQzThHtYtPmcL', 'JjRWKyALtSkoGmRxh', 'JxPhpijkDOpncCKyDEyXvKNua', 'jo', 'mKpFscuBEABMAlQO', 'qiFTgJpcnUMRKzTEuKY', 'pXBtITxCPRaXijM', 'guYVLpIbu', 'tSKYIxv', 'oDnWaFAmsXGRdGvRPhbCIvFSFQNlSVYB', 'phdckINUiYL']) from system.numbers limit 10; +select 14 = multiSearchFirstIndexCaseInsensitive(materialize('pXFoUGwVTAItBqgbBaQwAqmeh'), ['LfBevBpGnaSlmGhbeZ', 'NtBYzEksiXvYI', 'jMeRw', 'omtaduY', 'BsWyvNdkfXsTBxf', 'CtoOIvaesuca', 'pgJcRIBVbyaPBgGsNKP', 'bAwdUMnwKvMXfFHQWrtfMeqcORIJH', 'GDxZblrqWSxUJFjEuXArPtfHPdwSNGGL', 'LLxcfp', 'NrLghkFpwCdvHJBfPBgiMatNRaDKjO', 'XCzr', 'cCojPpfLkGZnaWBGpaZvrGMwgHNF', 'BaQWAQmE', 'AQ', 'RtxxEZDfcEZAgURg']) from system.numbers limit 10; +select 5 = multiSearchFirstIndexCaseInsensitive(materialize('KoLaGGWMRbPbKNChdKPGuNCDKZtWRX'), ['FBmf', 'QJxevrlVWhTDAJetlGoEBZWYz', 'tKoWKKXBOATZukMuBEaYYBPHuyncskOZYD', 'kgjgTpaHXji', '', 'xOJWVRvQoAYNVSN', 'YApQjWJCFuusXpTLfmLPinKNEuqfYAz', 'GXGfZJxhHcChCaoLwNNocnCjtIuw', 'ZLBHIwyivzQDbGsmVNBFDpVaWkIDRqsl', 'Kp', 'EyrNtIFdsoUWqLcVOpuqJBdMQ', 'AggwmRBpbknCHdKPgun', 'xNlnPtyQsdqH', 'hDk']) from system.numbers limit 10; +select 6 = multiSearchFirstIndexCaseInsensitive(materialize('OlyNppgrtlubvhpJfxeWsRHpr'), ['slbiGvzIFnqPgKZbzuh', 'fakuDHZWkYbXycUwNWC', 'HnVViUypZxAsLJocdwFFPgTDIkI', 'bLx', 'fmXVYOINsdIMmTJAQYWbBAuX', 'pjFXews', 'BG', 'vrSQLb', 'ub', 'pREPyIjRhXGKZovTqlDyYIuoYHewBH', 'hnNQpJmOKnGMlVbkSOyJxoQMdbGhTAsQU', 'UwaNyOQuYpkE', 'yHNlFVnuOLUxqHyzAtNgNohLT', 'YJRazuUZkP', 'z', 'lUbVhpjFxEWsRhP']) from system.numbers limit 10; +select 6 = multiSearchFirstIndexCaseInsensitive(materialize('ryHzepjmzFdLkCcYqoFCgnJh'), ['cLwBRJmuspkoOgKwtLXLbKFsj', 'YSgEdzTdYTZAEtaoJpjyfwymbERCVvveR', 'RzdDRzKjPXQzberVJRry', 'HUitVdjGjxYwIaLozmnKcCpFOjotfpAy', 'LWqtEkIiSvufymDiYjwt', 'FDlKCCYqoFCGNj', 'jmZfdlKCcyQOFcGnJ', 'OZCPsxgxYHdhqlnPnfRVGOJRL', 'JfhoyhbUhmDrKtYjZDCDFDcdNs', 'KCCYqo', 'EPJMzFDLKcCYQ', 'zLQb', 'qsqFDGqVnDX', 'MzfdLkCCyQOFc']) from system.numbers limit 10; +select 5 = multiSearchFirstIndexCaseInsensitive(materialize('oQLuuhKsqjdTaZmMiThIJrtwSrFv'), ['MsfVCGMIlgwomkNhkKn', 'fBzcso', 'meOeEdkEbFjgyAaeQeuqZXFFXqIxBkLbYiPk', 'tNV', 'i', 'EwuTkQnYCWktMAIdZEeJkgl', '', 'hUo', 'dtAzmMITHijRtwsrFV', 'vhnipYCl', 'puor', 'TazMmiTh', 'ITHIJRTWSrf', 'luuHksqJDTaz', 'uHkSQjDtazMMiThIjrtwSRFV', 'gpWugfu', 'QjdtazmmIthIjRTWSRFV', 'ZdJpc']) from system.numbers limit 10; + +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ИпрУщйжббКВНИчйацпцоЛП'), ['цШСкЕвеГЕЗЦщруИБтЦсБГАу', 'Хнщта', 'БшА', 'СалШйР', 'ЩфДГРРчшБДММГЧоноЖСчдпВХшшгйН', 'бЕжПШЦддожнЧоЕишчшЕЙфСщиВПФМ', 'ТЗзГФх', 'Чфл', 'КнНкнЖЕкППварНрхдгЙкДешмСКИЛкеО', 'ЖИсЧПСФФМДиТШХЦфмЗУпфрУщСЛщсфмвШ', 'ллЙумпхчОсЦМщУ', 'ГМУНЦФшНУбРжоПвШШщлВФАтоРфИ', 'БХцжеНЗкжЗЗшЦзфгдЖОзЗЖщКМИШАтЦАп', 'мтСкЕнбХШнЛхХГР']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('цмйвГЖруДлдЦавхЖАлоЕхЗКд'), ['ХфБПМДВХЙ', 'саЗваАбднХбЦттмКсМбШбВМУйНКСЖжХЦНц', 'плиЩщШАцЖсхГ', 'ЗнУЕФЗВаНА', 'ЧДйСаЗГЕшойСжбсуЩуЩщбПР', 'ЧЕуЩкФБВвчмабШЦтЖбОрЗп', 'йХбМсрТАФм', 'РЖСЗвЦлНВПЧщГУцЖ', 'ГГлЩрОХКнШРТуДФ', 'шСабРжла', 'ЕчБвгаРЧифаЙщХПпГЦхчШ', 'дайшйцВНЩЧуцйдМХг', 'УнзНКЧххВрцЩМлАнЖСДОДцбИгЛЛР', 'сЛЗзПбиАгзК']) from system.numbers limit 10; +select 2 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('дфЧлзОжММФкЖгиЗЩлоШжФТкцк'), ['ЗРТцИрсФСбПрщГЗ', '', 'ЖГИЗщлОш', 'АДПН', '', 'чЛЗОЖмМфКжг', 'Мфкж', 'ндаовк', 'зГЛРГАНШмСмШМефазшеБкзДвЕШиЖСЗЧПИфо', 'ФЧЛзОЖммфКжгиЗЩ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ИИКДМЛхРчнвЙЕкВЧелТйЛВТ'), ['АчшОЛтНЙуЦЛЙфАКУйуТЗМеЗщОХТМЗеТА', 'НЦУДбчфРТОпЛкОгВпоО', 'неДавнНРеАУфТтфАнДчтнУМЛПШнроАчжш', 'бГржВПЧлЛтСВТтаМЦШШ', 'БщГщРнБхЕЛоЛсмЙцВЕГ', 'цбАжЦРеу', 'ХсЦРаНиН', 'нббДдВЗРС', 'змОПпеЛЖзушлнДЛфчЗлцЙЛфЖрЛКг', 'фШиЖСУоаНПйИВшшаоуЙУА', 'ЛктХиШРП', 'МапщВйцХч', 'жмУТкуГбУ', 'сйпзДЩоНдШЕТбПзФтсрмАФГСз', 'ЛБУвйладЕижрКзШУАгНЩчЕмАа', 'мЧпФлМчРбШРблмтмПМоС']) from system.numbers limit 10; +select 8 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ПоДУЗАтХншЦатИшХвмИЖчГнжчНцух'), ['жЛЧХзкжлиЛцЩбЧСнЛУжЖпКРвиСРН', 'шадмЩеУШБврУдЕБЗИгмЗЕФШчЦБСзПидтАлб', 'йпГмШСз', 'хЖФЙиПГЗЩавиЗЩйПнБЗЦЩмАЧ', 'ХесщтлбСИуЦ', 'вар', 'ЙкМаСхаЩаЗнФЩфКжПщб', 'ОдУзАТХншЦатИШхвМиЖчгнЖч', 'ЗВЗДБпФфцвжУКвНсбухссбЙКЙйккЛиим', 'гХхсГЛшдфЖЛбгчоЕмоЧр']) from system.numbers limit 10; +select 7 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ихзКЖЩсЧРСЖсЖжЛАшкТхИйТгМБпск'), ['ДРОБм', 'нз', 'тОЛ', 'щРзуЖрТ', 'Мдд', 'АЦГРК', 'Чрсжсжжл', 'чРсжсЖжл', 'ктхИйтГмБ', 'аАзЙддМДЦЩФкРТЧзЧПУойоТхБиЧПлХДв', 'иЙтгМбп', 'РицлПн', 'йДГнЧкЕв', 'ВМЩцАш', 'хКЩнДшуБЕЛТФГВгцБПРихШЙХгГД', 'иЙТГМ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('жггкщцзщшамдбРЗжйТзвхшАпХСбе'), ['лВТвтлРБжиЛЦвРЦкАЦаНБгуОН', 'рШаавцжзМрзВЧДРСузб', 'оемрЗМгФБНмжп', 'ЛбмХбФЧШГЛХИуТСрфхп', 'ЖшТдтЧйчМР', 'ЧнИМбфУпмЙлШЗТрТИкКИЩОЧеМщПЩлдБ', 'ГвРдПжГдБаснилз', 'уТнТчТРЗИЛ', 'ИТЕВ', 'дИСЖпПнПСНОвсЩЩшНтХЧшВ', 'штабтлМнсчРЗтфсТЩублЕЧйцеЦТтХ', 'ХбхгУШвАзкшЖ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('нсЩЙЕМмЧЛСйФцГВМиатГХш'), ['КсОПЧИкВсКшРхнкхБжду', 'мШмпТащжФ', 'ББЖнианЧЦпмрГЩГМаЛКжА', 'арИжзжфГТУДИРРРбцил', 'дфдмшМИщТиЗПруКфОнСЦ', 'Рцч', 'гмДгВДАтсщКЗлхвжЦУеФДАТГЙЦЧОЗвРш', 'чфХЩсДбУбВжАМшРлКРщв', 'нцБйсУ', 'фасДЕчвчДмбтЖХвоД', 'аБЧшЖшЖАКргОИшпШЧзТбтфйвкЕц', 'ЗжжсмкжЛд', 'щщлПзг', 'бП']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('сКиурчоиаЦйхгаУДПфчИтИК'), ['МЧПцУАМрХКЧмАЦннУшмРчкЖКХвху', 'РвДуВиашрРКкмжшЖНШБфлцжБЦР', 'йМУиУчНЧчРшДйБЗфЩЦйПсцгкДС', 'НсмаЛзЧвНЦШФуВРпзБГзйКцп', 'ЖлМЛУХОБллСЗСКвМКМдГчЩ', 'ЩХПШиобЛх', 'аФАЖВтРиЦнжбкСожУЖЙипм', 'аУГжУНуМУВФлж', 'ШБчтЗкЖНЙк', 'ЩоГПГчНП', 'мВЗйЛаХПоЕМХиИйДлшРгзугЙЖлнМппКЦ', 'вчмДФхНеЦйЗсЗйкфпОщПтШпспИМдГйВМх', 'ИЗИжЧжаГЩСуцСЩдкскздмЖЦ', 'дАмфЕбгс', 'ГМттнхчЩжМЧДфщШБкфчтЧ', 'ШЕииФБпщЙИДцРиЖжЩл', 'ОпуОлБ', 'хБ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('риШМбгиПЖннНоЧргзГзеДпЛиНт'), ['икДкбйдройВУсвФзрПСусДнАШо', 'чуУеТкУВФхз', 'ЕГпйчехЗвЛлБблЧПДм', 'зеоЩЧожКЛбШЩдАрКБНйшКВШаЗгПш', 'виФКуЗОтгВмТкБ', 'цДрЙгЗРаЧКаМДдБЕЧзСРщВФзПВЧГвЩрАУшс', 'мБЗИУдчХХжТж', 'ФТНМмгЖилуЛйМ', 'ЗегЩЦнЦщцИк', 'оГОусхФсДЖДЩИЕХЗпсПЩХБТГЕп', 'АУКНзАДНкусВЧХвАж', 'КвКрбсВлНАоЗсфХОйЦхТ', 'вФдеХацЧБкрхМЖЗЧчКшпфВчс', 'йХшиОвХЗжТпДТбвУрпшЕ']) from system.numbers limit 10; +select 11 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('МойрЙлтЖйБдББЛЕЕЦузЛфпИЕГт'), ['ПОжЦЩа', 'СШзЧФтСЗохЦЗдФтцНТу', 'вЕдТ', 'ечУФаМДнХщЕНУи', 'вмеосТзБАБуроЙУЛгФжДсЧщтчЕзлепгк', 'ИЧтБрцПмРаВрйИвНЛСйпЖжУВдНрурКшоКХП', 'ЕН', 'щКЦЩгФБСХпкпит', 'ей', 'ЕахшеОМРдЕГХуГЖчвКХМЕ', 'Гт', 'НужЛЛЙОАл']) from system.numbers limit 10; +select 11 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('еззЦАвУаДнзИКЙнЙдртРоП'), ['КгЩбшПЛКвтИРцйчккгЧчЧмтГ', 'кЛппСФщзМмТйВЕтбЩЦлО', 'ШпдзиЖх', 'иИХ', 'пУаАФгсмтофНФХиЦЕтТЗсОШЗЙ', 'фаКАБТцФМиКЖрИКшГБЗБ', 'идЖЙдЦММУнХЦЦфсФМ', 'МиЦечЖЦЙмРВЙОХсБРНнрлйЙшц', 'ТфдСтМгтмимТМАучтхПНЦлуф', 'бейККЛСггУЦБсокЕЙпнРЧ', 'цавУАДНЗИКЙнЙд', 'ЩйЕЖчЧщаПшжФсхХЛЕТчвмЙнуце', 'РТРОП', 'цАВуАДнзИкЙНЙдРтРо', 'аЩПИд', 'ОСчКшОАчВмр', '', 'уЙЛИуЕУвцДшНОгбТбИШв', 'АВУаднзИКЙНйдР', 'жТйоП']) from system.numbers limit 10; +select 12 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('шйМЦУщвфщшбмлТНВохСЖНУ'), ['хшТАпТоШхКНсДпвДЕчДМНбАНччд', 'ХКуПСтфСйРжмБглОШЙлйДкСФВйВ', 'хпмНЦМУШеАД', 'чзмЧВвлбЧкАщПкзТгеуГущб', 'шзжрДд', 'еЗГОЙНйИБЗДщИИНицмсЙЗгФУл', 'кнщЙхооДТООе', 'всзЙнТшжФЗДБДрщВДлбвулДИаз', 'мп', 'уБОйцзнМпИсксхефбдЕЛйгИмГШГЗЩ', 'ОМпзШШщчФФнвУЧгжчиндЧч', 'щВФЩШбмЛТн', 'бм', 'БпфнкнйЗцПдЧЩбВ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('НЗБлОбшмОПктткоччКиКрФсбкШАХ'), ['нффЕББУЖГшЖвГфЦФГЕСщсЩЧлфнАшшктизУ', 'нСмпцхшИои', 'ЧИчЗУтйЦхГезппФРХХШуцЗШВ', 'РИнщН', 'НЩдВТсЙсОдхРбМФнСпАбОПкудБФСчмб', 'йхглпдКтртгош', 'ибгУРАБцх', 'ИЕиЛрИДафмЗИкТвАуГчШугбЧмЛШщсОЧбБкП', 'ЩСМуХМ', 'АУсмдЗБвКфЩ', 'пгбТНОйц', 'МоИ', 'КОйкзОЕИЗМЩ', 'чщттЛРНнГхЗхХй', 'ЩшцЧРКмШЖЩЦемтЧУЛГкХтВНзОжУХТпН', 'ЕшбБНчрДпЩЧМлераУЖХйфйдчтсчПШ', 'дбФйтИАшДйЩтбФйШуПиРлГмВОШаСлШЧИвфЖщгж', 'ОДжТЦщпщИжфуеЩмн', 'ПЛНЕзжСчВКДттуФРУ', 'БбмеГЩХшжрцОжХНииВКВлдиХБДСмнНфХЛТХ']) from system.numbers limit 10; +select 4 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ЕКаЖСЗЗЕЗгПдШкфцЙТцл'), ['ЙКМИХРОХ', 'НвМУХзфчДбАРЙДу', 'чмщжФшшжсЗТв', 'жСЗзеЗг', 'ЛФсКзВСдЦД', 'АЖсЗЗЕЗГ', 'Пдшкфц', 'усйсКщшрДрвнФЛедуГХ', '', 'цйтЦ', 'Ощс', 'ЕЗГпдшКф', 'ззеЗгп', 'УгЛйхШТтшрЛ', 'ЗзЕЗгП', 'КЛмТЩРтрзБбЩРгФбиОБазУнтУЦ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('чЕжАфАрБпКбДмшАшТШККауЩИхНВО'), ['ЧЙпЗЧЧлйПЙЖЙшККг', 'зйхуМЩАИПГЗА', 'ЙцехноХниИбзБЧ', 'чВомЗОфУроС', 'дбРхХЗрзоДДШщЕДжиФаЙ', 'еЛзТцЩДиДГрдМОНЧУнеТуДЩЧЦпГЕщПОРсйпЧ', 'ФчнпМРЧцПЙЩЩвфДХПнУхцЩСИ', 'цлШеУкМБнжЧлУцСуСЙуотшМфйс', 'лугГлкщКщкзЛйпбдсишргДДшОувр', 'ЗРИаФЛЗФрСзм', 'аЗвжВгхЩоЦ', 'чГКлеБНДнИЖЧеШЧДнИвсГДЖЖфБМНсУЦосВс', 'щЦнПУзЧщнЩЕ', 'рВУв']) from system.numbers limit 10; +select 20 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('анктгЦВВкЧвЖиБпфТйлр'), ['НшДПчтсСЧпкидаХжаЙчаДчЦГшГ', 'ХнцЛШИрХВаРхнЧИЙрОЛЛИТпППфгЖЩФ', 'ФАЛущПупмдМБмтйзУшрВМзцзШжгД', 'ГчЛЧеЛДХеипдшЦЦмаШНаРшУТ', 'фОЕфжО', 'ТНсУАнчшУЛЦкцчЙ', 'ЛйЦКБЗГЦйКЩиОПуТЦкБкБувснЙи', 'Бунф', 'ИтХЛШСУНЦВйРСЙчДчНвйшЗЦй', 'АцСКнзБаЖУДЖегавйБгужШАДЙтжИВк', 'ЦцХщфирДПрСуХзхЖМЕщ', 'кфдБЖКншвУФкЗДКуЙ', 'СкиСЦЗЦРмгЦНпБхфХДЙщЛзХ', 'йУепВЖАПНбАЩуЛжвЧпхМ', 'БпЧшпДочУвибщерйхйтОБАСПнЧМИОЩ', 'чФгНЗщвхавбшсООоВштбЧ', 'уДиЕцнЙХВЕйИАГдЕ', 'тп', 'ЧЕРЖсгВ', 'вЖибПФТЙЛ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('ипозйпхЛОЛТлСМХЩдМвМгШИвГиЛп'), ['ФСГзиГррБДНКГЛХбААФхИ', 'гегпАвхДЕ', 'ЦХжзщХИвхп', 'ЗЖ', 'ХОКцКзЩо', 'абИОрГПМТКшБ', 'кмХТмФихСЦсшУдхВбИШМНАНмпмХОЗйПЩч', 'еОжТСкфЕТУУжГ', 'НтщМЕПЧИКЙКйй', 'ежСикИвйЛж', 'ушЩФОтпБзЩЛЗЦЧЙиВгБЧоПХНгОуАДТЙж', 'фМЕРефнутпнцФРнрГЖ', 'хшДЧзнХпфорвЩжмГРЦуХГ', 'ЧЖн', 'вВзгОСхгНумм', 'ЗДоВлСжпфщСКсщХаолЛнЛЗбСхвЩвЩНоЩЩМ']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('МрЗтВФуЖРеЕШЧхПФбжжхчД'), ['щжОожЦндцШйТАй', 'йуРСЦУЗФУЦПвРфевСлфдРещЦтИтЩЩТг', 'ЕГЧдмХмРАлнЧ', 'йнкФизГСЗнуКбЙВЙчАТТрСхаЙШтсдгХ', 'ЧПрнРЖЙцХИщ', 'зЕ', 'СжВЩчГзБХбйТиклкдШШИееАлЧЩН', 'МШщГйБХжЙпйЕЗТзКмпе', 'НКбНщОМДзлдЧОс', 'НчзВХОпХХШМОХФумБгсрРЧИчВтгутВЩо']) from system.numbers limit 10; +select 0 = multiSearchFirstIndexCaseInsensitiveUTF8(materialize('упТУЖелФкЧЧУЦРжоБтХсжКщД'), ['щКшуОЖааЖйнЕбДИжМК', 'ЕкнШцХРВтНйШоНбЙйУоЧщУиРпШЧхмКЧжх', 'рвЩЗоЗхшЗвлизкСзебЩКМКжбша', 'ДииБНСШвцЦбаСсИжЕЗмхмВ', 'СЦоБЗПМтмшрУлрДТФГЖиувШЗууШзв', 'ЦЗБЕзВХЙбйВОмЗпхндЗ', 'ЗНизЧВШкГВтпсЖж', 'уШиБПЙЧтРаЕгИ', 'ЙшпПА', 'ЧоММаАйМСфбхуФкефФштгУА']) from system.numbers limit 10; + From 1adfa8e398914740f25183857bef4d09281d0869 Mon Sep 17 00:00:00 2001 From: whysage Date: Mon, 18 Jul 2022 10:41:57 +0300 Subject: [PATCH 327/659] Fix link in doc AggregateFunction --- docs/ru/sql-reference/data-types/aggregatefunction.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/data-types/aggregatefunction.md b/docs/ru/sql-reference/data-types/aggregatefunction.md index 4d3a6309576..551b8eac6e2 100644 --- a/docs/ru/sql-reference/data-types/aggregatefunction.md +++ b/docs/ru/sql-reference/data-types/aggregatefunction.md @@ -63,5 +63,4 @@ SELECT uniqMerge(state) FROM (SELECT uniqState(UserID) AS state FROM table GROUP ## Пример использования {#primer-ispolzovaniia} -Смотрите в описании движка [AggregatingMergeTree](../../sql-reference/data-types/aggregatefunction.md). - +Смотрите в описании движка [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md). From 32637cb1b99c9310866a103dadf969518ed1ce56 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Jul 2022 06:19:16 +0000 Subject: [PATCH 328/659] Fix build --- src/Processors/Formats/Impl/AvroRowOutputFormat.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index dce5a928690..b63b1e7b9b1 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -179,14 +179,14 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF if (traits->isStringAsString(column_name)) return {avro::StringSchema(), [](const IColumn & column, size_t row_num, avro::Encoder & encoder) { - const std::string_ref & s = assert_cast(column).getDataAt(row_num).toView(); + const std::string_view & s = assert_cast(column).getDataAt(row_num).toView(); encoder.encodeString(std::string(s)); } }; else return {avro::BytesSchema(), [](const IColumn & column, size_t row_num, avro::Encoder & encoder) { - const std::string_view & s = assert_cast(column).getDataAt(row_num).toString(); + const std::string_view & s = assert_cast(column).getDataAt(row_num).toView(); encoder.encodeBytes(reinterpret_cast(s.data()), s.size()); } }; @@ -343,7 +343,7 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF auto keys_serializer = [](const IColumn & column, size_t row_num, avro::Encoder & encoder) { - const StringRef & s = column.getDataAt(row_num).toView(); + const std::string_view & s = column.getDataAt(row_num).toView(); encoder.encodeString(std::string(s)); }; From c63512d255e16e5053ca5e93299da8d53815b9ed Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Mon, 18 Jul 2022 09:59:02 +0200 Subject: [PATCH 329/659] Use CH custom platform macro for OS Darwin --- utils/self-extracting-executable/compressor.cpp | 2 +- utils/self-extracting-executable/decompressor.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/utils/self-extracting-executable/compressor.cpp b/utils/self-extracting-executable/compressor.cpp index 5e055a34825..6ce4af2c362 100644 --- a/utils/self-extracting-executable/compressor.cpp +++ b/utils/self-extracting-executable/compressor.cpp @@ -9,7 +9,7 @@ #include #include #include -#if defined __APPLE__ +#if defined OS_DARWIN // dependencies #include diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 8914f611e10..0adeb33b3ce 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -1,6 +1,6 @@ #include #include -#if defined __APPLE__ +#if defined OS_DARWIN #include #else #include @@ -12,7 +12,7 @@ #include #include #include -#if defined __APPLE__ +#if defined OS_DARWIN // dependencies #include From caef6729f49280acc88a459f5e670f64d50f8f5a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 18 Jul 2022 10:32:45 +0200 Subject: [PATCH 330/659] Improve debug logging of github_helper --- tests/ci/github_helper.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/github_helper.py b/tests/ci/github_helper.py index 426ab19a520..46cf7d2b726 100644 --- a/tests/ci/github_helper.py +++ b/tests/ci/github_helper.py @@ -135,6 +135,7 @@ class GitHub(github.Github): if updated_at <= cached_pr.updated_at: logger.debug("Getting PR #%s from cache", number) return cached_pr + logger.debug("Getting PR #%s from API", number) for i in range(self.retries): try: pr = repo.get_pull(number) @@ -143,7 +144,7 @@ class GitHub(github.Github): if i == self.retries - 1: raise self.sleep_on_rate_limit() - logger.debug("Getting PR #%s from API", number) + logger.debug("Caching PR #%s from API in %s", number, pr_cache_file) with open(pr_cache_file, "wb") as prfd: self.dump(pr, prfd) # type: ignore return pr From 21fa5e6ece8d50068fe051d61f213fbdca9d0416 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 18 Jul 2022 10:51:02 +0300 Subject: [PATCH 331/659] Add check for CLICKHOUSE_SPLIT_BINARY=1 with USE_STATIC_LIBRARIES=1 Signed-off-by: Azat Khuzhin --- CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f2ab66afd9b..7aa6b5d5a50 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -79,8 +79,8 @@ option(USE_STATIC_LIBRARIES "Disable to use shared libraries" ON) option(SPLIT_SHARED_LIBRARIES "Keep all internal libraries as separate .so files" OFF) option(CLICKHOUSE_SPLIT_BINARY "Make several binaries (clickhouse-server, clickhouse-client etc.) instead of one bundled" OFF) -if (USE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES) - message(FATAL_ERROR "Defining SPLIT_SHARED_LIBRARIES=1 without USE_STATIC_LIBRARIES=0 has no effect.") +if (USE_STATIC_LIBRARIES AND (SPLIT_SHARED_LIBRARIES OR CLICKHOUSE_SPLIT_BINARY)) + message(FATAL_ERROR "SPLIT_SHARED_LIBRARIES=1 or CLICKHOUSE_SPLIT_BINARY=1 must not be used together with USE_STATIC_LIBRARIES=1") endif() if (NOT USE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES) From 4f41d216269cac896a30073981a3825a5f651a2b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 18 Jul 2022 12:31:32 +0300 Subject: [PATCH 332/659] Fix leaking of logger in clickhouse-disks CI found [1]: Direct leak of 256 byte(s) in 1 object(s) allocated from: 0 0xd8cb88d in operator new(unsigned long) (/usr/bin/clickhouse+0xd8cb88d) (BuildId: 7a3fd7b485701220) 1 0xde8943e in DB::DisksApp::main() build_docker/../programs/disks/DisksApp.cpp:157:41 2 0x38dca887 in Poco::Util::Application::run() build_docker/../contrib/poco/Util/src/Application.cpp:334:8 3 0xde8d72c in mainEntryClickHouseDisks(int, char**) build_docker/../programs/disks/DisksApp.cpp:219:20 4 0xd8cf47f in main build_docker/../programs/main.cpp:445:12 5 0x7f060ddce082 in __libc_start_main (/lib/x86_64-linux-gnu/libc.so.6+0x24082) (BuildId: 1878e6b475720c7c51969e69ab2d276fae6d1dee) CI: https://s3.amazonaws.com/clickhouse-test-reports/39299/37b4b52c12698e711aa931f10aec3909bca287b6/integration_tests__asan__actions__[2/3].html Signed-off-by: Azat Khuzhin --- programs/disks/DisksApp.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 99b4b099bd8..58a18f6ad2e 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -154,7 +154,7 @@ int DisksApp::main(const std::vector & /*args*/) Poco::Logger::root().setLevel(Poco::Logger::parseLevel(log_level)); auto log_path = config().getString("logger.clickhouse-disks", "/var/log/clickhouse-server/clickhouse-disks.log"); - Poco::Logger::root().setChannel(new Poco::FileChannel(log_path)); + Poco::Logger::root().setChannel(Poco::AutoPtr(new Poco::FileChannel(log_path))); } if (config().has("config-file") || fs::exists(getDefaultConfigFileName())) From 3b0ca82d568f6bfb928a46fc9b27e2170f3b315a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Jul 2022 09:41:01 +0000 Subject: [PATCH 333/659] Fix build, pt. II --- src/Functions/stringToH3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/stringToH3.cpp b/src/Functions/stringToH3.cpp index b5f8585152f..db13534b3d2 100644 --- a/src/Functions/stringToH3.cpp +++ b/src/Functions/stringToH3.cpp @@ -84,7 +84,7 @@ private: auto h3index = h3index_source.getWhole(); // convert to std::string and get the c_str to have the delimiting \0 at the end. - auto h3index_str = std::string(std::string_view(h3index.data(), h3index.size())); + auto h3index_str = std::string(reinterpret_cast(h3index.data), h3index.size); res_data[row_num] = stringToH3(h3index_str.c_str()); if (res_data[row_num] == 0) From a1932767379cda5870952ec8c1daa50055d6ee83 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 18 Jul 2022 12:50:49 +0300 Subject: [PATCH 334/659] Update DatabaseOnDisk.cpp --- src/Databases/DatabaseOnDisk.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 26ea3b81e3a..fe229ba6ee9 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -202,8 +202,9 @@ void DatabaseOnDisk::createTable( if (create.uuid != create_detached.uuid) throw Exception( ErrorCodes::TABLE_ALREADY_EXISTS, - "Table {}.{} already exist (detached permanently). To attach it back " - "you need to use short ATTACH syntax or a full statement with the same UUID", + "Table {}.{} already exist (detached or detached permanently). To attach it back " + "you need to use short ATTACH syntax (ATTACH TABLE {}.{};)", + backQuote(getDatabaseName()), backQuote(table_name), backQuote(getDatabaseName()), backQuote(table_name)); } From 2593dbcaa677b190e564fb394446c2e91adb388d Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Mon, 18 Jul 2022 12:21:44 +0200 Subject: [PATCH 335/659] reuse Common/ExponentiallySmoothedCounter.h --- src/Common/EventRateMeter.h | 66 ++++++++++++------------------- src/Common/ProgressIndication.cpp | 4 +- src/Common/ProgressIndication.h | 2 +- 3 files changed, 29 insertions(+), 43 deletions(-) diff --git a/src/Common/EventRateMeter.h b/src/Common/EventRateMeter.h index b8582585e4b..a32324de63c 100644 --- a/src/Common/EventRateMeter.h +++ b/src/Common/EventRateMeter.h @@ -2,9 +2,14 @@ #include #include + +#include + #include #include #include +#include + namespace DB { @@ -12,42 +17,23 @@ namespace DB /// Event count measurement with exponential smoothing intended for computing time derivatives class EventRateMeter { public: - explicit EventRateMeter(UInt64 period_, UInt64 resolution = 1000) - : period(std::max(period_, 1ul)) - , step(std::max(period / resolution, 1ul)) - , decay(1.0 - 1.0 / resolution) - {} + explicit EventRateMeter(double now, double period_) + : period(period_) + , half_decay_time(period * std::numbers::ln2) // for `ExponentiallySmoothedAverage::sumWeights()` to be equal to `1/period` + { + reset(now); + } /// Add `count` events happened at `now` instant. /// Previous events that are older than `period` from `now` will be forgotten /// in a way to keep average event rate the same, using exponential smoothing. /// NOTE: Adding events into distant past (further than `period`) must be avoided. - void add(UInt64 now, UInt64 count = 1) + void add(double now, double count) { - if (unlikely(end == 0)) - { - // Initialization during the first call - if (start == 0) - start = now; - end = start + period; - } - else if (now > end) - { - // Compute number of steps we have to move for `now <= end` to became true - UInt64 steps = (now - end + step - 1) / step; - end += steps * step; - assert(now <= end); - - // Forget old events, assuming all events are distributed evenly throughout whole `period`. - // This assumption leads to exponential decay in case no new events will come. - if (steps == 1) - events *= decay; - else - events *= std::pow(decay, steps); - } - - // Add new events - events += count; + if (now - period <= start) // precise counting mode + events = ExponentiallySmoothedAverage(events.value + count, now); + else // exponential smoothing mode + events.add(count, now, half_decay_time); } /// Compute average event rate thoughout `[now - period, now]` period. @@ -58,23 +44,23 @@ public: add(now, 0); if (unlikely(now <= start)) return 0; - return double(events) / std::min(period, now - start); + if (now - period <= start) // precise counting mode + return events.value / (now - start); + else // exponential smoothing mode + return events.get(half_decay_time); // equals to `events.value / period` } - void reset(UInt64 now) + void reset(double now) { - events = 0; start = now; - end = 0; + events = ExponentiallySmoothedAverage(); } private: - const UInt64 period; - const UInt64 step; - const double decay; - double events = 0; // Estimated number of events in [end - period, end] range - UInt64 start = 0; // Instant in past without events before it; when measurement started or reset - UInt64 end = 0; // Instant in future to start decay; moving in steps + const double period; + const double half_decay_time; + double start; // Instant in past without events before it; when measurement started or reset + ExponentiallySmoothedAverage events; // Estimated number of events in the last `period` }; } diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 2b4b5d9ade9..8ca1612e916 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -56,7 +56,7 @@ void ProgressIndication::resetProgress() write_progress_on_update = false; { std::lock_guard lock(profile_events_mutex); - cpu_usage_meter.reset(clock_gettime_ns()); + cpu_usage_meter.reset(static_cast(clock_gettime_ns())); thread_data.clear(); } } @@ -93,7 +93,7 @@ void ProgressIndication::updateThreadEventData(HostToThreadTimesMap & new_thread total_cpu_ns += aggregateCPUUsageNs(new_host_map.second); thread_data[new_host_map.first] = std::move(new_host_map.second); } - cpu_usage_meter.add(clock_gettime_ns(), total_cpu_ns); + cpu_usage_meter.add(static_cast(clock_gettime_ns()), total_cpu_ns); } size_t ProgressIndication::getUsedThreadsCount() const diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 6ff89294475..588a31beca7 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -91,7 +91,7 @@ private: bool write_progress_on_update = false; - EventRateMeter cpu_usage_meter{3'000'000'000 /*ns*/}; // average cpu utilization last 3 second + EventRateMeter cpu_usage_meter{static_cast(clock_gettime_ns()), 3'000'000'000 /*ns*/}; // average cpu utilization last 3 second HostToThreadTimesMap thread_data; /// In case of all of the above: /// - clickhouse-local From 00e5bdb84ab64febca291c846cb0110a491674e6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 18 Jul 2022 12:44:58 +0200 Subject: [PATCH 336/659] Fix build clang-13 --- base/base/defines.h | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/base/base/defines.h b/base/base/defines.h index 5a646f4dca2..c8c408b9c93 100644 --- a/base/base/defines.h +++ b/base/base/defines.h @@ -93,7 +93,6 @@ # define NO_SANITIZE_ADDRESS __attribute__((__no_sanitize__("address"))) # define NO_SANITIZE_THREAD __attribute__((__no_sanitize__("thread"))) # define ALWAYS_INLINE_NO_SANITIZE_UNDEFINED __attribute__((__always_inline__, __no_sanitize__("undefined"))) -# define DISABLE_SANITIZER_INSTRUMENTATION __attribute__((disable_sanitizer_instrumentation)) #else /// It does not work in GCC. GCC 7 cannot recognize this attribute and GCC 8 simply ignores it. # define NO_SANITIZE_UNDEFINED # define NO_SANITIZE_ADDRESS @@ -101,6 +100,13 @@ # define ALWAYS_INLINE_NO_SANITIZE_UNDEFINED ALWAYS_INLINE #endif +#if defined(__clang__) && defined(__clang_major__) && __clang_major__ >= 14 +# define DISABLE_SANITIZER_INSTRUMENTATION __attribute__((disable_sanitizer_instrumentation)) +#else +# define DISABLE_SANITIZER_INSTRUMENTATION +#endif + + #if !__has_include() || !defined(ADDRESS_SANITIZER) # define ASAN_UNPOISON_MEMORY_REGION(a, b) # define ASAN_POISON_MEMORY_REGION(a, b) From 0ffe5bd39cc08fd0ae77453b40095332daf4e9cf Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Mon, 18 Jul 2022 13:26:33 +0200 Subject: [PATCH 337/659] fix typo and style --- src/Common/EventRateMeter.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Common/EventRateMeter.h b/src/Common/EventRateMeter.h index a32324de63c..c0b2ee8c587 100644 --- a/src/Common/EventRateMeter.h +++ b/src/Common/EventRateMeter.h @@ -15,7 +15,8 @@ namespace DB { /// Event count measurement with exponential smoothing intended for computing time derivatives -class EventRateMeter { +class EventRateMeter +{ public: explicit EventRateMeter(double now, double period_) : period(period_) @@ -36,7 +37,7 @@ public: events.add(count, now, half_decay_time); } - /// Compute average event rate thoughout `[now - period, now]` period. + /// Compute average event rate throughout `[now - period, now]` period. /// If measurements are just started (`now - period < start`), then average /// is computed based on shorter `[start; now]` period to avoid initial linear growth. double rate(UInt64 now) From 30cb4e168e79b1cd870b5fe412e0ee0b638d73af Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Mon, 18 Jul 2022 13:31:32 +0200 Subject: [PATCH 338/659] cleanup --- src/Common/EventRateMeter.h | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Common/EventRateMeter.h b/src/Common/EventRateMeter.h index c0b2ee8c587..f70258faa9e 100644 --- a/src/Common/EventRateMeter.h +++ b/src/Common/EventRateMeter.h @@ -1,13 +1,9 @@ #pragma once #include -#include #include -#include -#include -#include #include @@ -40,7 +36,7 @@ public: /// Compute average event rate throughout `[now - period, now]` period. /// If measurements are just started (`now - period < start`), then average /// is computed based on shorter `[start; now]` period to avoid initial linear growth. - double rate(UInt64 now) + double rate(double now) { add(now, 0); if (unlikely(now <= start)) From 74a0f676761c64844987983b251013f152d19ac3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 18 Jul 2022 13:46:26 +0200 Subject: [PATCH 339/659] Fix setting name in integration tests. --- .../test_row_policy.py | 2 +- .../test_select_from_system_tables.py | 2 +- ..._select_from_system_tables.py => test_from_system_tables.py} | 2 +- .../test_select_access_rights/{test.py => test_main.py} | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) rename tests/integration/test_select_access_rights/{test_select_from_system_tables.py => test_from_system_tables.py} (98%) rename tests/integration/test_select_access_rights/{test.py => test_main.py} (99%) diff --git a/tests/integration/test_disabled_access_control_improvements/test_row_policy.py b/tests/integration/test_disabled_access_control_improvements/test_row_policy.py index 509b4de1a37..f083b0b61ef 100644 --- a/tests/integration/test_disabled_access_control_improvements/test_row_policy.py +++ b/tests/integration/test_disabled_access_control_improvements/test_row_policy.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="row_policy") node = cluster.add_instance( "node", main_configs=["configs/config.d/disable_access_control_improvements.xml"], diff --git a/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py b/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py index 5d760c9fc2c..a4151f44d1f 100644 --- a/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py +++ b/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="select_from_system_tables") node = cluster.add_instance( "node", main_configs=["configs/config.d/disable_access_control_improvements.xml"], diff --git a/tests/integration/test_select_access_rights/test_select_from_system_tables.py b/tests/integration/test_select_access_rights/test_from_system_tables.py similarity index 98% rename from tests/integration/test_select_access_rights/test_select_from_system_tables.py rename to tests/integration/test_select_access_rights/test_from_system_tables.py index ac938a9694a..5ff6fdd11c9 100644 --- a/tests/integration/test_select_access_rights/test_select_from_system_tables.py +++ b/tests/integration/test_select_access_rights/test_from_system_tables.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="from_system_tables") node = cluster.add_instance( "node", user_configs=[ diff --git a/tests/integration/test_select_access_rights/test.py b/tests/integration/test_select_access_rights/test_main.py similarity index 99% rename from tests/integration/test_select_access_rights/test.py rename to tests/integration/test_select_access_rights/test_main.py index 76940cdadb4..cd9e641ed44 100644 --- a/tests/integration/test_select_access_rights/test.py +++ b/tests/integration/test_select_access_rights/test_main.py @@ -3,7 +3,7 @@ from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance("instance") +instance = cluster.add_instance("instance", name="main") @pytest.fixture(scope="module", autouse=True) From cc1046f18aef8196a3d25d467230e9e48a26cae7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 18 Jul 2022 12:12:24 +0000 Subject: [PATCH 340/659] Possible fix for flaky test_keeper_force_recovery --- tests/integration/test_keeper_force_recovery/test.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_keeper_force_recovery/test.py b/tests/integration/test_keeper_force_recovery/test.py index 5f1b7d1e4e4..8eb759fae47 100644 --- a/tests/integration/test_keeper_force_recovery/test.py +++ b/tests/integration/test_keeper_force_recovery/test.py @@ -5,7 +5,7 @@ from helpers.cluster import ClickHouseCluster import time -from kazoo.client import KazooClient +from kazoo.client import KazooClient, KazooRetry CLUSTER_SIZE = 5 QUORUM_SIZE = CLUSTER_SIZE // 2 + 1 @@ -52,8 +52,10 @@ def started_cluster(): def get_fake_zk(nodename, timeout=30.0): _fake_zk_instance = KazooClient( - hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout, + command_retry=KazooRetry(max_tries=10) ) + _fake_zk_instance.start() return _fake_zk_instance @@ -117,7 +119,7 @@ def test_cluster_recovery(started_cluster): data_in_cluster = [] def add_data(zk, path, data): - zk.create(path, data.encode()) + zk.retry(zk.create, path, data.encode()) data_in_cluster.append((path, data)) def assert_all_data(zk): From 15882b1d8606ca08866a0fde252ace14d146710e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 18 Jul 2022 14:25:14 +0200 Subject: [PATCH 341/659] Auto set test's name in integration tests. --- tests/integration/helpers/cluster.py | 14 +++++++++++++- .../test_aggregate_fixed_key.py | 2 +- .../test_aggregate_function_state_avg.py | 2 +- .../test_convert_ordinary.py | 2 +- .../test_cte_distributed.py | 2 +- .../test_data_skipping_indices.py | 2 +- .../test_detach_part_wrong_partition_id.py | 2 +- .../test_insert_profile_events.py | 2 +- .../test_select_aggregate_alias_column.py | 2 +- .../test_short_strings_aggregation.py | 2 +- tests/integration/test_cluster_copier/test.py | 2 +- .../test_cluster_copier/test_three_nodes.py | 2 +- .../test_cluster_copier/test_trivial.py | 2 +- .../test_cluster_copier/test_two_nodes.py | 2 +- .../test_cassandra.py | 2 +- .../test_clickhouse_local.py | 2 +- .../test_clickhouse_remote.py | 2 +- .../test_executable_cache.py | 2 +- .../test_executable_hashed.py | 2 +- .../test_file.py | 2 +- .../test_http.py | 2 +- .../test_https.py | 2 +- .../test_mongo.py | 2 +- .../test_mongo_uri.py | 2 +- .../test_mysql.py | 2 +- .../test_dictionaries_redis/test_long.py | 2 +- .../test_default_reading.py | 2 +- .../test_default_string.py | 2 +- .../test_dict_get_or_default.py | 2 +- .../test_row_policy.py | 2 +- .../test_select_from_system_tables.py | 2 +- .../test_distributed_respect_user_timeouts/test.py | 2 -- .../test_allow_list.py | 2 +- tests/integration/test_log_levels_update/test.py | 2 +- .../integration/test_log_query_probability/test.py | 2 +- .../test_from_system_tables.py | 2 +- .../test_select_access_rights/test_main.py | 2 +- .../test_zookeeper_config/test_password.py | 2 +- .../test_zookeeper_config/test_secure.py | 1 - 39 files changed, 49 insertions(+), 40 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 219bc830a07..1c4add88dee 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -237,6 +237,18 @@ def enable_consistent_hash_plugin(rabbitmq_id): return p.returncode == 0 +def extract_test_name(base_path): + """Extracts the name of the test based to a path to its test*.py file + Must be unique in each test directory (because it's used to make instances dir and to stop docker containers from previous run) + """ + name = p.basename(base_path) + if name == "test.py": + name = "" + elif name.startswith("test_") and name.endswith(".py"): + name = name[len("test_"):(len(name)-len(".py"))] + return name + + def get_instances_dir(): if ( "INTEGRATION_TESTS_RUN_ID" in os.environ @@ -274,7 +286,7 @@ class ClickHouseCluster: logging.debug("ENV %40s %s" % (param, os.environ[param])) self.base_path = base_path self.base_dir = p.dirname(base_path) - self.name = name if name is not None else "" + self.name = name if name is not None else extract_test_name(base_path) self.base_config_dir = base_config_dir or os.environ.get( "CLICKHOUSE_TESTS_BASE_CONFIG_DIR", "/etc/clickhouse-server/" diff --git a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py index 35cdaeef9ac..01c9736c354 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__, name="aggregate_fixed_key") +cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", with_zookeeper=True, diff --git a/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py b/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py index 13dd28ee8af..1e54e6220d7 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__, name="aggregate_state") +cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", with_zookeeper=False, diff --git a/tests/integration/test_backward_compatibility/test_convert_ordinary.py b/tests/integration/test_backward_compatibility/test_convert_ordinary.py index 59ceca23a51..c509dade0b8 100644 --- a/tests/integration/test_backward_compatibility/test_convert_ordinary.py +++ b/tests/integration/test_backward_compatibility/test_convert_ordinary.py @@ -1,7 +1,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__, name="convert_ordinary") +cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", image="yandex/clickhouse-server", diff --git a/tests/integration/test_backward_compatibility/test_cte_distributed.py b/tests/integration/test_backward_compatibility/test_cte_distributed.py index 89a565b4b37..7ea0d2d9f21 100644 --- a/tests/integration/test_backward_compatibility/test_cte_distributed.py +++ b/tests/integration/test_backward_compatibility/test_cte_distributed.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__, name="cte_distributed") +cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance("node1", with_zookeeper=False) node2 = cluster.add_instance( "node2", diff --git a/tests/integration/test_backward_compatibility/test_data_skipping_indices.py b/tests/integration/test_backward_compatibility/test_data_skipping_indices.py index 60d709c257f..c65dc6d3841 100644 --- a/tests/integration/test_backward_compatibility/test_data_skipping_indices.py +++ b/tests/integration/test_backward_compatibility/test_data_skipping_indices.py @@ -5,7 +5,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__, name="skipping_indices") +cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", image="yandex/clickhouse-server", diff --git a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py index cb9929db48b..02fccfae4e5 100644 --- a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py +++ b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__, name="detach") +cluster = ClickHouseCluster(__file__) # Version 21.6.3.14 has incompatible partition id for tables with UUID in partition key. node_21_6 = cluster.add_instance( "node_21_6", diff --git a/tests/integration/test_backward_compatibility/test_insert_profile_events.py b/tests/integration/test_backward_compatibility/test_insert_profile_events.py index 8047c088e4c..0fd453e57d4 100644 --- a/tests/integration/test_backward_compatibility/test_insert_profile_events.py +++ b/tests/integration/test_backward_compatibility/test_insert_profile_events.py @@ -6,7 +6,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__, name="insert_profile_events") +cluster = ClickHouseCluster(__file__) upstream_node = cluster.add_instance("upstream_node") old_node = cluster.add_instance( "old_node", diff --git a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py index e98894d887a..8bdae54a889 100644 --- a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py +++ b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__, name="aggregate_alias_column") +cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance("node1", with_zookeeper=False) node2 = cluster.add_instance( "node2", diff --git a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py index 8053ad417ec..17a7282b7b5 100644 --- a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__, name="short_strings") +cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", with_zookeeper=False, diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index 14417f151ee..0aadcadc064 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -18,7 +18,7 @@ sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) COPYING_FAIL_PROBABILITY = 0.2 MOVING_FAIL_PROBABILITY = 0.2 -cluster = ClickHouseCluster(__file__, name="copier_test") +cluster = ClickHouseCluster(__file__) def generateRandomString(count): diff --git a/tests/integration/test_cluster_copier/test_three_nodes.py b/tests/integration/test_cluster_copier/test_three_nodes.py index c8039792fe8..31d6c0448f4 100644 --- a/tests/integration/test_cluster_copier/test_three_nodes.py +++ b/tests/integration/test_cluster_copier/test_three_nodes.py @@ -12,7 +12,7 @@ import docker CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) -cluster = ClickHouseCluster(__file__, name="copier_test_three_nodes") +cluster = ClickHouseCluster(__file__) @pytest.fixture(scope="module") diff --git a/tests/integration/test_cluster_copier/test_trivial.py b/tests/integration/test_cluster_copier/test_trivial.py index 84bf39f0d76..785186fded4 100644 --- a/tests/integration/test_cluster_copier/test_trivial.py +++ b/tests/integration/test_cluster_copier/test_trivial.py @@ -19,7 +19,7 @@ sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) COPYING_FAIL_PROBABILITY = 0.1 MOVING_FAIL_PROBABILITY = 0.1 -cluster = ClickHouseCluster(__file__, name="copier_test_trivial") +cluster = ClickHouseCluster(__file__) def generateRandomString(count): diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 6fdaaeea720..10ab7d03b00 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -12,7 +12,7 @@ import docker CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) -cluster = ClickHouseCluster(__file__, name="copier_test_two_nodes") +cluster = ClickHouseCluster(__file__) @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py index aa1eb614dd5..2213623379a 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py @@ -24,7 +24,7 @@ def setup_module(module): global complex_tester global ranged_tester - cluster = ClickHouseCluster(__file__, name=test_name) + cluster = ClickHouseCluster(__file__) SOURCE = SourceCassandra( "Cassandra", diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py index b7f8226960f..bb0e3b47414 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py @@ -38,7 +38,7 @@ def setup_module(module): ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__, name=test_name) + cluster = ClickHouseCluster(__file__) main_configs = [] main_configs.append(os.path.join("configs", "disable_ssl_verification.xml")) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py index 6790d11ed1a..bf4d05a154c 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py @@ -38,7 +38,7 @@ def setup_module(module): ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__, name=test_name) + cluster = ClickHouseCluster(__file__) main_configs = [] main_configs.append(os.path.join("configs", "disable_ssl_verification.xml")) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py index 5186139ddf6..6af5fa841c1 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py @@ -38,7 +38,7 @@ def setup_module(module): ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__, name=test_name) + cluster = ClickHouseCluster(__file__) main_configs = [] main_configs.append(os.path.join("configs", "disable_ssl_verification.xml")) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py index 63f4ff87cce..dfcf1e4fc64 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py @@ -38,7 +38,7 @@ def setup_module(module): ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__, name=test_name) + cluster = ClickHouseCluster(__file__) main_configs = [] main_configs.append(os.path.join("configs", "disable_ssl_verification.xml")) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py index 0147b95c786..e9bf93b3c8e 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py @@ -36,7 +36,7 @@ def setup_module(module): ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__, name=test_name) + cluster = ClickHouseCluster(__file__) main_configs = [] main_configs.append(os.path.join("configs", "disable_ssl_verification.xml")) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py index 96d17508880..94220d7c698 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py @@ -36,7 +36,7 @@ def setup_module(module): ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__, name=test_name) + cluster = ClickHouseCluster(__file__) main_configs = [] main_configs.append(os.path.join("configs", "disable_ssl_verification.xml")) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py index 007e318e037..0b7476faf2e 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py @@ -38,7 +38,7 @@ def setup_module(module): ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__, name=test_name) + cluster = ClickHouseCluster(__file__) main_configs = [] main_configs.append(os.path.join("configs", "disable_ssl_verification.xml")) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py index 4a9d054b08f..55639877ba0 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py @@ -24,7 +24,7 @@ def setup_module(module): global complex_tester global ranged_tester - cluster = ClickHouseCluster(__file__, name=test_name) + cluster = ClickHouseCluster(__file__) SOURCE = SourceMongo( "MongoDB", "localhost", diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py index c6551e0eb70..84c547b7a6b 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py @@ -24,7 +24,7 @@ def setup_module(module): global complex_tester global ranged_tester - cluster = ClickHouseCluster(__file__, name=test_name) + cluster = ClickHouseCluster(__file__) SOURCE = SourceMongoURI( "MongoDB", diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py index 96757c58e0c..77b2c0741b5 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py @@ -24,7 +24,7 @@ def setup_module(module): global complex_tester global ranged_tester - cluster = ClickHouseCluster(__file__, name=test_name) + cluster = ClickHouseCluster(__file__) SOURCE = SourceMySQL( "MySQL", diff --git a/tests/integration/test_dictionaries_redis/test_long.py b/tests/integration/test_dictionaries_redis/test_long.py index 19b03322b4d..094df789704 100644 --- a/tests/integration/test_dictionaries_redis/test_long.py +++ b/tests/integration/test_dictionaries_redis/test_long.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster import redis -cluster = ClickHouseCluster(__file__, name="long") +cluster = ClickHouseCluster(__file__) node = cluster.add_instance("node", with_redis=True) diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py index bb587efa7e9..85c45d5df3c 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py @@ -5,7 +5,7 @@ from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseKiller from helpers.network import PartitionManager -cluster = ClickHouseCluster(__file__, name="reading") +cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance("dictionary_node", stay_alive=True) main_node = cluster.add_instance( diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py index 7acc26a66e0..92d681698bc 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py @@ -7,7 +7,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV -cluster = ClickHouseCluster(__file__, name="string") +cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance("dictionary_node", stay_alive=True) main_node = cluster.add_instance( diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py index 54c5976f295..1da8fd3325a 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py @@ -5,7 +5,7 @@ from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseKiller from helpers.network import PartitionManager -cluster = ClickHouseCluster(__file__, name="default") +cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance("dictionary_node", stay_alive=True) main_node = cluster.add_instance( diff --git a/tests/integration/test_disabled_access_control_improvements/test_row_policy.py b/tests/integration/test_disabled_access_control_improvements/test_row_policy.py index f083b0b61ef..509b4de1a37 100644 --- a/tests/integration/test_disabled_access_control_improvements/test_row_policy.py +++ b/tests/integration/test_disabled_access_control_improvements/test_row_policy.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV -cluster = ClickHouseCluster(__file__, name="row_policy") +cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", main_configs=["configs/config.d/disable_access_control_improvements.xml"], diff --git a/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py b/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py index a4151f44d1f..5d760c9fc2c 100644 --- a/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py +++ b/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV -cluster = ClickHouseCluster(__file__, name="select_from_system_tables") +cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", main_configs=["configs/config.d/disable_access_control_improvements.xml"], diff --git a/tests/integration/test_distributed_respect_user_timeouts/test.py b/tests/integration/test_distributed_respect_user_timeouts/test.py index 567377aba0b..ea79a9544d5 100644 --- a/tests/integration/test_distributed_respect_user_timeouts/test.py +++ b/tests/integration/test_distributed_respect_user_timeouts/test.py @@ -8,8 +8,6 @@ from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager from helpers.test_tools import TSV -cluster = ClickHouseCluster(__file__) - NODES = {"node" + str(i): None for i in (1, 2)} IS_DEBUG = False diff --git a/tests/integration/test_keeper_four_word_command/test_allow_list.py b/tests/integration/test_keeper_four_word_command/test_allow_list.py index 026bd1d59af..4bf8ae1ab53 100644 --- a/tests/integration/test_keeper_four_word_command/test_allow_list.py +++ b/tests/integration/test_keeper_four_word_command/test_allow_list.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster import time -cluster = ClickHouseCluster(__file__, name="test_keeper_4lw_allow_list") +cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", main_configs=["configs/keeper_config_with_allow_list.xml"], stay_alive=True ) diff --git a/tests/integration/test_log_levels_update/test.py b/tests/integration/test_log_levels_update/test.py index 86719390f33..b0c003ea440 100644 --- a/tests/integration/test_log_levels_update/test.py +++ b/tests/integration/test_log_levels_update/test.py @@ -3,7 +3,7 @@ import re from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__, name="log_quries_probability") +cluster = ClickHouseCluster(__file__) node = cluster.add_instance("node", with_zookeeper=False) config = """ diff --git a/tests/integration/test_log_query_probability/test.py b/tests/integration/test_log_query_probability/test.py index d13ecc276cb..0ed7bf2c928 100644 --- a/tests/integration/test_log_query_probability/test.py +++ b/tests/integration/test_log_query_probability/test.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__, name="log_quries_probability") +cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance("node1", with_zookeeper=False) node2 = cluster.add_instance("node2", with_zookeeper=False) diff --git a/tests/integration/test_select_access_rights/test_from_system_tables.py b/tests/integration/test_select_access_rights/test_from_system_tables.py index 5ff6fdd11c9..ac938a9694a 100644 --- a/tests/integration/test_select_access_rights/test_from_system_tables.py +++ b/tests/integration/test_select_access_rights/test_from_system_tables.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV -cluster = ClickHouseCluster(__file__, name="from_system_tables") +cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", user_configs=[ diff --git a/tests/integration/test_select_access_rights/test_main.py b/tests/integration/test_select_access_rights/test_main.py index cd9e641ed44..76940cdadb4 100644 --- a/tests/integration/test_select_access_rights/test_main.py +++ b/tests/integration/test_select_access_rights/test_main.py @@ -3,7 +3,7 @@ from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance("instance", name="main") +instance = cluster.add_instance("instance") @pytest.fixture(scope="module", autouse=True) diff --git a/tests/integration/test_zookeeper_config/test_password.py b/tests/integration/test_zookeeper_config/test_password.py index 71f059b3277..55a06cd5f51 100644 --- a/tests/integration/test_zookeeper_config/test_password.py +++ b/tests/integration/test_zookeeper_config/test_password.py @@ -2,7 +2,7 @@ import time import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__, name="password") +cluster = ClickHouseCluster(__file__) # TODO ACL not implemented in Keeper. node1 = cluster.add_instance( diff --git a/tests/integration/test_zookeeper_config/test_secure.py b/tests/integration/test_zookeeper_config/test_secure.py index f540a504024..1bc7c62f92c 100644 --- a/tests/integration/test_zookeeper_config/test_secure.py +++ b/tests/integration/test_zookeeper_config/test_secure.py @@ -9,7 +9,6 @@ TEST_DIR = os.path.dirname(__file__) cluster = ClickHouseCluster( __file__, - name="secure", zookeeper_certfile=os.path.join(TEST_DIR, "configs_secure", "client.crt"), zookeeper_keyfile=os.path.join(TEST_DIR, "configs_secure", "client.key"), ) From bd563c210e0ecdb0211f97d3f873c7402842005c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 18 Jul 2022 14:55:21 +0200 Subject: [PATCH 342/659] Update tests/integration/helpers/cluster.py Co-authored-by: Azat Khuzhin --- tests/integration/helpers/cluster.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 1c4add88dee..fb7d05632bb 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -242,10 +242,9 @@ def extract_test_name(base_path): Must be unique in each test directory (because it's used to make instances dir and to stop docker containers from previous run) """ name = p.basename(base_path) - if name == "test.py": - name = "" - elif name.startswith("test_") and name.endswith(".py"): - name = name[len("test_"):(len(name)-len(".py"))] + name = p.splitext(name)[0] + name = name.removeprefix('test') + name = name.strip('-_') return name From e67a447fc349c3ea69e001dc353ba66745085461 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 18 Jul 2022 15:17:53 +0200 Subject: [PATCH 343/659] Fix style. --- tests/integration/helpers/cluster.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index fb7d05632bb..d25ab696e30 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -243,8 +243,8 @@ def extract_test_name(base_path): """ name = p.basename(base_path) name = p.splitext(name)[0] - name = name.removeprefix('test') - name = name.strip('-_') + name = name.removeprefix("test") + name = name.strip("-_") return name From 80b5bc5236b073d2265177c5c694ac70f9ba262f Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 18 Jul 2022 13:19:41 +0000 Subject: [PATCH 344/659] Add system table settings_changes --- src/Core/Settings.cpp | 23 ++-- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 115 +++++++++--------- .../System/StorageSystemSettingsChanges.cpp | 37 ++++++ .../System/StorageSystemSettingsChanges.h | 28 +++++ src/Storages/System/attachSystemTables.cpp | 2 + ...26_settings_changes_system_table.reference | 3 + .../02326_settings_changes_system_table.sql | 2 + 8 files changed, 147 insertions(+), 65 deletions(-) create mode 100644 src/Storages/System/StorageSystemSettingsChanges.cpp create mode 100644 src/Storages/System/StorageSystemSettingsChanges.h create mode 100644 tests/queries/0_stateless/02326_settings_changes_system_table.reference create mode 100644 tests/queries/0_stateless/02326_settings_changes_system_table.sql diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 65c0adfc4b4..7bac3f04fc6 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -146,7 +146,7 @@ std::vector Settings::getAllRegisteredNames() const return all_settings; } -void Settings::set(const std::string_view & name, const Field & value) +void Settings::set(std::string_view name, const Field & value) { BaseSettings::set(name, value); @@ -173,14 +173,23 @@ void Settings::applyCompatibilitySetting() return; ClickHouseVersion version(compatibility); - for (const auto & [setting_name, history] : settings_changes_history) + /// Iterate through ClickHouse version in descending order and apply reversed + /// changes for each version that is higher that version from compatibility setting + for (auto it = settings_changes_history.rbegin(); it != settings_changes_history.rend(); ++it) { - /// If this setting was changed manually, we don't change it - if (isChanged(setting_name)) - continue; + if (version >= it->first) + break; - BaseSettings::set(setting_name, history.getValueForVersion(version)); - settings_changed_by_compatibility_setting.insert(setting_name); + /// Apply reversed changes from this version. + for (const auto & change : it->second) + { + /// If this setting was changed manually, we don't change it + if (isChanged(change.name) && !settings_changed_by_compatibility_setting.contains(change.name)) + continue; + + BaseSettings::set(change.name, change.previous_value); + settings_changed_by_compatibility_setting.insert(change.name); + } } } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a374a60b405..595f398246b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -832,7 +832,7 @@ struct Settings : public BaseSettings, public IHints<2, Settings void addProgramOptionAsMultitoken(boost::program_options::options_description & options, const SettingFieldRef & field); - void set(const std::string_view & name, const Field & value) override; + void set(std::string_view name, const Field & value) override; private: void applyCompatibilitySetting(); diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index c5bd9623aaa..716f5a40196 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -21,6 +22,9 @@ public: Strings split; boost::split(split, version, [](char c){ return c == '.'; }); components.reserve(split.size()); + if (split.empty()) + throw Exception{ErrorCodes::BAD_ARGUMENTS, "Cannot parse ClickHouse version here: {}", version}; + for (const auto & split_element : split) { size_t component; @@ -32,80 +36,77 @@ public: ClickHouseVersion(const char * version) : ClickHouseVersion(String(version)) {} + String toString() const + { + String version = std::to_string(components[0]); + for (size_t i = 1; i < components.size(); ++i) + version += "." + std::to_string(components[i]); + + return version; + } + bool operator<(const ClickHouseVersion & other) const { return components < other.components; } + bool operator>=(const ClickHouseVersion & other) const + { + return components >= other.components; + } + private: std::vector components; }; -struct SettingChangesHistory +namespace SettingsChangesHistory { - struct Change + struct SettingChange { - Change(const Field & value_, const ClickHouseVersion & version_) : value(value_), version(version_) {} - - Field value; - ClickHouseVersion version; + String name; + Field previous_value; + Field new_value; + String reason; }; - SettingChangesHistory(const Field & initial_value_, const std::vector & changes_) : initial_value(initial_value_), changes(changes_) {} - - Field getValueForVersion(const ClickHouseVersion & version) const - { - Field value = initial_value; - for (const auto & change : changes) - { - if (version < change.version) - return value; - value = change.value; - } - return value; - } - - Field initial_value; - std::vector changes; -}; + using SettingsChanges = std::vector; +} /// History of settings changes that controls some backward incompatible changes -/// across all ClickHouse versions. It maps setting name to special struct -/// SettingChangesHistory {initial_value, {{changed_value_1, version1}, {changed_value_2, version_2}, ...}} +/// across all ClickHouse versions. It maps ClickHouse version to settings changes that were done +/// in this version. Settings changes is a vector of structs {setting_name, previous_value, new_value} /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) -const std::unordered_map settings_changes_history = +std::map settings_changes_history = { - {"enable_positional_arguments", {false, {{true, "22.7"}}}}, - {"output_format_json_named_tuples_as_objects", {false, {{true, "22.6"}}}}, - {"memory_overcommit_ratio_denominator", {0, {{1073741824, "22.5"}}}}, - {"memory_overcommit_ratio_denominator_for_user", {0, {{1073741824, "22.5"}}}}, - {"allow_settings_after_format_in_insert", {true, {{false, "22.4"}}}}, - {"cast_ipv4_ipv6_default_on_conversion_error", {true, {{false, "22.3"}}}}, - {"input_format_ipv4_default_on_conversion_error", {true, {{false, "22.3"}}}}, - {"input_format_ipv6_default_on_conversion_error", {true, {{false, "22.3"}}}}, - {"stream_like_engine_allow_direct_select", {true, {{false, "21.12"}}}}, - {"output_format_decimal_trailing_zeros", {true, {{false, "21.9"}}}}, - {"use_hedged_requests", {false, {{true, "21.9"}}}}, - {"legacy_column_name_of_tuple_literal", {true, {{false, "21.7"}}}}, - {"async_socket_for_remote", {true, {{false, "21.3"}, {true, "21.5"}}}}, - {"optimize_normalize_count_variants", {false, {{true, "21.3"}}}}, - {"normalize_function_names", {false, {{true, "21.3"}}}}, - {"enable_global_with_statement", {false, {{true, "21.2"}}}}, - {"insert_quorum_parallel", {false, {{true, "21.1"}}}}, - {"input_format_null_as_default", {false, {{true, "21.1"}}}}, - {"optimize_on_insert", {false, {{true, "21.1"}}}}, - {"use_compact_format_in_distributed_parts_names", {false, {{true, "21.1"}}}}, - {"format_regexp_escaping_rule", {"Escaped", {{"Raw", "20.10"}}}}, - {"show_table_uuid_in_table_create_query_if_not_nil", {true, {{false, "20.7"}}}}, - {"input_format_with_names_use_header", {false, {{true, "20.5"}}}}, - {"allow_suspicious_codecs", {true, {{false, "20.5"}}}}, - {"validate_polygons", {false, {{true, "20.4"}}}}, - {"enable_scalar_subquery_optimization", {false, {{true, "19.18"}}}}, - {"any_join_distinct_right_table_keys", {true, {{false, "19.14"}}}}, - {"input_format_defaults_for_omitted_fields", {false, {{true, "19.12"}}}}, - {"max_partitions_per_insert_block", {0, {{100, "19.5"}}}}, - {"enable_optimize_predicate_expression", {0, {{1, "18.12.17"}}}}, + {"22.7", {{"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}}}, + {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}}}, + {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, + {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, + {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, + {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, + {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, + {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, + {"use_hedged_requests", false, true, "Enable Hedged Requests feature bu default"}}}, + {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, + {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, + {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, + {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, + {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, + {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, + {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, + {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, + {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, + {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, + {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, + {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, + {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, + {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, + {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, + {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, + {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, + {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for ommitted fields for some input formats, because it should be the expected behaviour"}}}, + {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, + {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, }; - } diff --git a/src/Storages/System/StorageSystemSettingsChanges.cpp b/src/Storages/System/StorageSystemSettingsChanges.cpp new file mode 100644 index 00000000000..e84fd44fcc3 --- /dev/null +++ b/src/Storages/System/StorageSystemSettingsChanges.cpp @@ -0,0 +1,37 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ +NamesAndTypesList StorageSystemSettingsChanges::getNamesAndTypes() +{ + return { + {"version", std::make_shared()}, + {"changes", + std::make_shared(std::make_shared( + DataTypes{ + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared()}, + Names{"name", "previous_value", "new_value", "reason"}))}, + }; +} + +void StorageSystemSettingsChanges::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +{ + for (auto it = settings_changes_history.rbegin(); it != settings_changes_history.rend(); ++it) + { + res_columns[0]->insert(it->first.toString()); + Array changes; + for (const auto & change : it->second) + changes.push_back(Tuple{change.name, toString(change.previous_value), toString(change.new_value), change.reason}); + res_columns[1]->insert(changes); + } +} + +} diff --git a/src/Storages/System/StorageSystemSettingsChanges.h b/src/Storages/System/StorageSystemSettingsChanges.h new file mode 100644 index 00000000000..283487df51b --- /dev/null +++ b/src/Storages/System/StorageSystemSettingsChanges.h @@ -0,0 +1,28 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +/** Implements system table "settings_changes", which allows to get information + * about the settings changes through different ClickHouse versions. + */ +class StorageSystemSettingsChanges final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemSettingsChanges"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index a86a04c4444..dbef2df953b 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -39,6 +39,7 @@ #include #include #include +#include #include #include #include @@ -100,6 +101,7 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) attach(context, system_database, "functions"); attach(context, system_database, "events"); attach(context, system_database, "settings"); + attach(context, system_database, "settings_changes"); attach>(context, system_database, "merge_tree_settings"); attach>(context, system_database, "replicated_merge_tree_settings"); attach(context, system_database, "build_options"); diff --git a/tests/queries/0_stateless/02326_settings_changes_system_table.reference b/tests/queries/0_stateless/02326_settings_changes_system_table.reference new file mode 100644 index 00000000000..c4a3c71edfd --- /dev/null +++ b/tests/queries/0_stateless/02326_settings_changes_system_table.reference @@ -0,0 +1,3 @@ +version String +changes Array(Tuple(name String, previous_value String, new_value String, reason String)) +22.5 [('memory_overcommit_ratio_denominator','0','1073741824','Enable memory overcommit feature by default'),('memory_overcommit_ratio_denominator_for_user','0','1073741824','Enable memory overcommit feature by default')] diff --git a/tests/queries/0_stateless/02326_settings_changes_system_table.sql b/tests/queries/0_stateless/02326_settings_changes_system_table.sql new file mode 100644 index 00000000000..e56cd62ce55 --- /dev/null +++ b/tests/queries/0_stateless/02326_settings_changes_system_table.sql @@ -0,0 +1,2 @@ +DESC system.settings_changes; +SELECT * FROM system.settings_changes WHERE version = '22.5' From ae68a641a852138213d7c63f63f46539a9e21d2d Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 18 Jul 2022 15:29:13 +0200 Subject: [PATCH 345/659] Update src/Core/SettingsChangesHistory.h Co-authored-by: Vladimir C --- src/Core/SettingsChangesHistory.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 716f5a40196..cb86234851b 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -78,7 +78,8 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) std::map settings_changes_history = { - {"22.7", {{"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}}}, + {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, + {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}}}, {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}}}, {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, From 6f5e0172f450ff8f587f546b1d0634b3ee1a7c80 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 18 Jul 2022 15:30:44 +0200 Subject: [PATCH 346/659] Fix spaces --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index cb86234851b..a1274f24e21 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -79,7 +79,7 @@ namespace SettingsChangesHistory std::map settings_changes_history = { {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}}}, + {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}}}, {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}}}, {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, From 51e7c41883f9e0f290c9b5c7a1853111ac52492a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Jul 2022 14:52:41 +0300 Subject: [PATCH 347/659] Add a test for LSan v2: fix type check Signed-off-by: Azat Khuzhin --- src/Common/tests/gtest_lsan.cpp | 33 +++++++++++++++++++++++++++++++++ 1 file changed, 33 insertions(+) create mode 100644 src/Common/tests/gtest_lsan.cpp diff --git a/src/Common/tests/gtest_lsan.cpp b/src/Common/tests/gtest_lsan.cpp new file mode 100644 index 00000000000..f6e1984ec58 --- /dev/null +++ b/src/Common/tests/gtest_lsan.cpp @@ -0,0 +1,33 @@ +#include // ADDRESS_SANITIZER + +#ifdef ADDRESS_SANITIZER + +#include +#include + +#include +#include + +/// Test that ensures that LSan works. +/// +/// Regression test for the case when it may not work, +/// because of broken getauxval() [1]. +/// +/// [1]: https://github.com/ClickHouse/ClickHouse/pull/33957 +TEST(Common, LSan) +{ + int sanitizers_exit_code = 1; + + ASSERT_EXIT({ + std::thread leak_in_thread([]() + { + void * leak = malloc(4096); + ASSERT_NE(leak, nullptr); + }); + leak_in_thread.join(); + + __lsan_do_leak_check(); + }, ::testing::ExitedWithCode(sanitizers_exit_code), ".*LeakSanitizer: detected memory leaks.*"); +} + +#endif From f187c3499a9efe05af109aa8011f384f0b14297e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Jul 2022 17:26:00 +0300 Subject: [PATCH 348/659] Fix getauxval() by using procfs with fallback to environ getauxval() from glibc-compatibility did not work always correctly: - It does not work after setenv(), and this breaks vsyscalls, like sched_getcpu() [1] (and BaseDaemon.cpp always set TZ if timezone is defined, which is true for CI [2]). Also note, that fixing setenv() will not fix LSan, since the culprit is getauxval() [1]: https://bugzilla.redhat.com/show_bug.cgi?id=1163404 [2]: ClickHouse#32928 (comment) - Another think that is definitely broken is LSan (Leak Sanitizer), it relies on worked getauxval() but it does not work if __environ is not initialized yet (there is even a commit about this). And because of, at least, one leak had been introduced [3]: [3]: ClickHouse#33840 Fix this by using /proc/self/auxv with fallback to environ solution to make it compatible with environment that does not allow reading from auxv (or no procfs). v2: add fallback to environ solution v3: fix return value for __auxv_init_procfs() Refs: #33957 Signed-off-by: Azat Khuzhin --- base/glibc-compatibility/musl/getauxval.c | 156 ++++++++++++++++++---- 1 file changed, 132 insertions(+), 24 deletions(-) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index dad7aa938d7..616e43ef342 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -1,68 +1,176 @@ -#include #include "atomic.h" -#include // __environ +#include +#include // open +#include // O_RDONLY +#include // read, close +#include // ssize_t +#include // perror, fprintf +#include // ElfW #include -// We don't have libc struct available here. Compute aux vector manually. -static unsigned long * __auxv = NULL; -static unsigned long __auxv_secure = 0; +#define ARRAY_SIZE(a) sizeof((a))/sizeof((a[0])) +// We don't have libc struct available here. +// Compute aux vector manually (from /proc/self/auxv). +// +// Right now there is only 51 AT_* constants, +// so 64 should be enough until this implementation will be replaced with musl. +static unsigned long __auxv_procfs[64]; +static unsigned long __auxv_secure = 0; +// Common +static unsigned long * __auxv_environ = NULL; + +static void * volatile getauxval_func; + +static unsigned long __auxv_init_environ(unsigned long type); + +// +// auxv from procfs interface +// +ssize_t __retry_read(int fd, void * buf, size_t count) +{ + for (;;) + { + ssize_t ret = read(fd, buf, count); + if (ret == -1) + { + if (errno == EINTR) + { + continue; + } + perror("Cannot read /proc/self/auxv"); + abort(); + } + return ret; + } +} +unsigned long __getauxval_procfs(unsigned long type) +{ + if (type == AT_SECURE) + { + return __auxv_secure; + } + + if (type >= ARRAY_SIZE(__auxv_procfs)) + { + errno = ENOENT; + return 0; + } + + return __auxv_procfs[type]; +} +static unsigned long __auxv_init_procfs(unsigned long type) +{ + // For debugging: + // - od -t dL /proc/self/auxv + // - LD_SHOW_AUX= ls + int fd = open("/proc/self/auxv", O_RDONLY); + // It is possible in case of: + // - no procfs mounted + // - on android you are not able to read it unless running from shell or debugging + // - some other issues + if (fd == -1) + { + // Fallback to environ. + a_cas_p(&getauxval_func, (void *)__auxv_init_procfs, (void *)__auxv_init_environ); + return __auxv_init_environ(type); + } + + ElfW(auxv_t) aux; + + /// NOTE: sizeof(aux) is very small (less then PAGE_SIZE), so partial read should not be possible. + _Static_assert(sizeof(aux) < 4096, "Unexpected sizeof(aux)"); + while (__retry_read(fd, &aux, sizeof(aux)) == sizeof(aux)) + { + if (aux.a_type >= ARRAY_SIZE(__auxv_procfs)) + { + fprintf(stderr, "AT_* is out of range: %li (maximum allowed is %zu)\n", aux.a_type, ARRAY_SIZE(__auxv_procfs)); + abort(); + } + if (__auxv_procfs[aux.a_type]) + { + fprintf(stderr, "AUXV already has value (%zu)\n", __auxv_procfs[aux.a_type]); + abort(); + } + __auxv_procfs[aux.a_type] = aux.a_un.a_val; + } + close(fd); + + __auxv_secure = __getauxval_procfs(AT_SECURE); + + // Now we've initialized __auxv_procfs, next time getauxval() will only call __get_auxval(). + a_cas_p(&getauxval_func, (void *)__auxv_init_procfs, (void *)__getauxval_procfs); + + return __getauxval_procfs(type); +} + +// +// auxv from environ interface +// +// NOTE: environ available only after static initializers, +// so you cannot rely on this if you need getauxval() before. +// +// Good example of such user is sanitizers, for example +// LSan will not work with __auxv_init_environ(), +// since it needs getauxval() before. +// static size_t __find_auxv(unsigned long type) { size_t i; - for (i = 0; __auxv[i]; i += 2) + for (i = 0; __auxv_environ[i]; i += 2) { - if (__auxv[i] == type) + if (__auxv_environ[i] == type) + { return i + 1; + } } return (size_t) -1; } - -unsigned long __getauxval(unsigned long type) +unsigned long __getauxval_environ(unsigned long type) { if (type == AT_SECURE) return __auxv_secure; - if (__auxv) + if (__auxv_environ) { size_t index = __find_auxv(type); if (index != ((size_t) -1)) - return __auxv[index]; + return __auxv_environ[index]; } errno = ENOENT; return 0; } - -static void * volatile getauxval_func; - -static unsigned long __auxv_init(unsigned long type) +static unsigned long __auxv_init_environ(unsigned long type) { if (!__environ) { - // __environ is not initialized yet so we can't initialize __auxv right now. + // __environ is not initialized yet so we can't initialize __auxv_environ right now. // That's normally occurred only when getauxval() is called from some sanitizer's internal code. errno = ENOENT; return 0; } - // Initialize __auxv and __auxv_secure. + // Initialize __auxv_environ and __auxv_secure. size_t i; for (i = 0; __environ[i]; i++); - __auxv = (unsigned long *) (__environ + i + 1); + __auxv_environ = (unsigned long *) (__environ + i + 1); size_t secure_idx = __find_auxv(AT_SECURE); if (secure_idx != ((size_t) -1)) - __auxv_secure = __auxv[secure_idx]; + __auxv_secure = __auxv_environ[secure_idx]; - // Now we've initialized __auxv, next time getauxval() will only call __get_auxval(). - a_cas_p(&getauxval_func, (void *)__auxv_init, (void *)__getauxval); + // Now we need to switch to __getauxval_environ for all later calls, since + // everything is initialized. + a_cas_p(&getauxval_func, (void *)__auxv_init_environ, (void *)__getauxval_environ); - return __getauxval(type); + return __getauxval_environ(type); } -// First time getauxval() will call __auxv_init(). -static void * volatile getauxval_func = (void *)__auxv_init; +// Callchain: +// - __auxv_init_procfs -> __getauxval_environ +// - __auxv_init_procfs -> __auxv_init_environ -> __getauxval_environ +static void * volatile getauxval_func = (void *)__auxv_init_procfs; unsigned long getauxval(unsigned long type) { From b687b5406ee18257d8d90e12fcc654c1389824fa Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 18 Jul 2022 15:58:39 +0200 Subject: [PATCH 349/659] Fix include and typo --- src/Core/SettingsChangesHistory.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index a1274f24e21..f59cd546bab 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB { @@ -105,7 +105,7 @@ std::map settings_ch {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for ommitted fields for some input formats, because it should be the expected behaviour"}}}, + {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, }; From dfccb677f52652a5c7882959b7d89c99985a4475 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 18 Jul 2022 16:03:06 +0200 Subject: [PATCH 350/659] try fix flaky test_store_cleanup --- .../integration/test_broken_detached_part_clean_up/test.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_broken_detached_part_clean_up/test.py b/tests/integration/test_broken_detached_part_clean_up/test.py index 019ae3d52a1..1321778f2ca 100644 --- a/tests/integration/test_broken_detached_part_clean_up/test.py +++ b/tests/integration/test_broken_detached_part_clean_up/test.py @@ -197,7 +197,10 @@ def test_store_cleanup(started_cluster): node1.exec_in_container(["mkdir", f"{path_to_data}/store/kek"]) node1.exec_in_container(["touch", f"{path_to_data}/store/12"]) - node1.exec_in_container(["mkdir", f"{path_to_data}/store/456"]) + try: + node1.exec_in_container(["mkdir", f"{path_to_data}/store/456"]) + except Exception as e: + print("Failed to create 456/:", str(e)) node1.exec_in_container(["mkdir", f"{path_to_data}/store/456/testgarbage"]) node1.exec_in_container( ["mkdir", f"{path_to_data}/store/456/30000000-1000-4000-8000-000000000003"] @@ -218,7 +221,7 @@ def test_store_cleanup(started_cluster): timeout=60, look_behind_lines=1000, ) - node1.wait_for_log_line("directories from store") + node1.wait_for_log_line("directories from store", look_behind_lines=1000) store = node1.exec_in_container(["ls", f"{path_to_data}/store"]) assert "100" in store From 456ae5ea440fe279438f28671da63205b0bc9f43 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 18 Jul 2022 10:39:20 -0400 Subject: [PATCH 351/659] add comments for readCompressedDataBlockForAsynchronous --- cmake/cpu_features.cmake | 2 +- src/Compression/CompressedReadBufferBase.cpp | 2 +- src/Compression/CompressedReadBufferBase.h | 20 +++++++++++++++++-- .../CompressedReadBufferFromFile.cpp | 2 +- 4 files changed, 21 insertions(+), 5 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 7b966e1acac..ad8d55e6c60 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -18,7 +18,7 @@ option (ENABLE_PCLMULQDQ "Use pclmulqdq instructions on x86_64" 1) option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1) option (ENABLE_AVX "Use AVX instructions on x86_64" 0) option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0) -option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 0) +option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 1) option (ENABLE_BMI "Use BMI instructions on x86_64" 0) option (ENABLE_AVX2_FOR_SPEC_OP "Use avx2 instructions for specific operations on x86_64" 0) option (ENABLE_AVX512_FOR_SPEC_OP "Use avx512 instructions for specific operations on x86_64" 0) diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 5a51d8377d5..b2cd222f7d8 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -187,7 +187,7 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, return size_compressed_without_checksum + sizeof(Checksum); } -size_t CompressedReadBufferBase::readCompressedDataBlockHold(size_t & size_decompressed, size_t & size_compressed_without_checksum) +size_t CompressedReadBufferBase::readCompressedDataBlockForAsynchronous(size_t & size_decompressed, size_t & size_compressed_without_checksum) { UInt8 header_size = ICompressionCodec::getHeaderSize(); if (compressed_in->eof() || (compressed_in->available() < (header_size + sizeof(Checksum)))) diff --git a/src/Compression/CompressedReadBufferBase.h b/src/Compression/CompressedReadBufferBase.h index 85f49bd3510..0457d59058c 100644 --- a/src/Compression/CompressedReadBufferBase.h +++ b/src/Compression/CompressedReadBufferBase.h @@ -38,15 +38,31 @@ protected: /// /// Returns number of compressed bytes read. size_t readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum, bool always_copy); - size_t readCompressedDataBlockHold(size_t & size_decompressed, size_t & size_compressed_without_checksum); + + /// Read compressed data into compressed_buffer for asynchronous decompression to avoid the situation of "read compressed block across the compressed_in". + /// + /// Compressed block may not be completely contained in "compressed_in" buffer which means compressed block may be read across the "compressed_in". + /// For native LZ4/ZSTD, it has no probem in facing situation above because they are sychronous. + /// But for asynchronous decompression, such as QPL deflate, it requires source and target buffer for decompression can not be overwritten until execution complete. + /// + /// Returns number of compressed bytes read. + /// If Returns value > 0, means the address range for current block are maintained in "compressed_in", then asynchronous decompression can be called to boost performance. + /// If Returns value == 0, it means current block cannot be decompressed asynchronously.Meanwhile, asynchronous requests for previous blocks should be flushed if any. + size_t readCompressedDataBlockForAsynchronous(size_t & size_decompressed, size_t & size_compressed_without_checksum); + /// Decompress into memory pointed by `to` void decompressTo(char * to, size_t size_decompressed, size_t size_compressed_without_checksum); /// This method can change location of `to` to avoid unnecessary copy if data is uncompressed. /// It is more efficient for compression codec NONE but not suitable if you want to decompress into specific location. void decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum); - /// Flush all asynchronous decompress request + + /// Flush all asynchronous decompress request. void flushAsynchronousDecompressRequests() const; + + /// Set decompression mode: Synchronous/Asynchronous/SoftwareFallback. + /// The mode is "Synchronous" by default. + /// flushAsynchronousDecompressRequests must be called subsequently once set "Asynchronous" mode. void setDecompressMode(ICompressionCodec::CodecMode mode); public: diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index 0b9dfa7be9e..f34665a2fb4 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -104,7 +104,7 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) size_t size_decompressed = 0; size_t size_compressed_without_checksum = 0; - size_t new_size_compressed = readCompressedDataBlockHold(size_decompressed, size_compressed_without_checksum); + size_t new_size_compressed = readCompressedDataBlockForAsynchronous(size_decompressed, size_compressed_without_checksum); if (new_size_compressed) { From 23312164bf1b54b63697dff52d3c246e40b328be Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 18 Jul 2022 17:05:03 +0200 Subject: [PATCH 352/659] do not start on unexpected Ordinary metadata --- src/Databases/DatabaseFactory.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index df1e58ca852..6213fa62e3c 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -126,6 +126,20 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (!create.attach && !context->getSettingsRef().allow_deprecated_database_ordinary) throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Ordinary database engine is deprecated (see also allow_deprecated_database_ordinary setting)"); + + /// Before 20.7 metadata/db_name.sql file might absent and Ordinary database was attached if there's metadata/db_name/ dir. + /// Between 20.7 and 22.7 metadata/db_name.sql was created in this case as well. + /// Since 20.7 `default` database is created with Atomic engine on the very first server run. + /// The problem is that if server crashed during the very first run and metadata/db_name/ -> store/whatever symlink was created + /// then it's considered as Ordinary database. And it even works somehow + /// until background task tries to remove onused dir from store/... + if (fs::is_symlink(metadata_path)) + throw Exception(ErrorCodes::CANNOT_CREATE_DATABASE, "Metadata directory {} for Ordinary database {} is a symbolic link to {}. " + "It may be a result of manual intervention, crash on very first server start or a bug. " + "Database cannot be attached (it's kind of protection from potential data loss). " + "Metadata directory must not be a symlink and must contain tables metadata files itself. " + "You have to resolve this manually.", + metadata_path, database_name, fs::read_symlink(metadata_path).string()); return std::make_shared(database_name, metadata_path, context); } From 225a5c3c7e5b779ef31b6cc6061489409efd7ac7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 18 Jul 2022 18:19:02 +0300 Subject: [PATCH 353/659] Update tests/integration/helpers/cluster.py --- tests/integration/helpers/cluster.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index d25ab696e30..1c4add88dee 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -242,9 +242,10 @@ def extract_test_name(base_path): Must be unique in each test directory (because it's used to make instances dir and to stop docker containers from previous run) """ name = p.basename(base_path) - name = p.splitext(name)[0] - name = name.removeprefix("test") - name = name.strip("-_") + if name == "test.py": + name = "" + elif name.startswith("test_") and name.endswith(".py"): + name = name[len("test_"):(len(name)-len(".py"))] return name From 362694ea709eb2623f6a7791d56e2ccf98428961 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 18 Jul 2022 11:20:53 -0400 Subject: [PATCH 354/659] switch from mkdocs to Docusaurus --- docs/README.md | 18 ++++----- docs/tools/README.md | 96 ++++++++++++++++++++++++++++++++------------ 2 files changed, 79 insertions(+), 35 deletions(-) diff --git a/docs/README.md b/docs/README.md index b328a3ee125..fa8b6bed85c 100644 --- a/docs/README.md +++ b/docs/README.md @@ -38,9 +38,9 @@ Writing the docs is extremely useful for project's users and developers, and gro The documentation contains information about all the aspects of the ClickHouse lifecycle: developing, testing, installing, operating, and using. The base language of the documentation is English. The English version is the most actual. All other languages are supported as much as they can by contributors from different countries. -At the moment, [documentation](https://clickhouse.com/docs) exists in English, Russian, Chinese, Japanese. We store the documentation besides the ClickHouse source code in the [GitHub repository](https://github.com/ClickHouse/ClickHouse/tree/master/docs). +At the moment, [documentation](https://clickhouse.com/docs) exists in English, Russian, and Chinese. We store the reference documentation besides the ClickHouse source code in the [GitHub repository](https://github.com/ClickHouse/ClickHouse/tree/master/docs), and user guides in a separate repo [Clickhouse/clickhouse-docs](https://github.com/ClickHouse/clickhouse-docs). -Each language lays in the corresponding folder. Files that are not translated from English are the symbolic links to the English ones. +Each language lies in the corresponding folder. Files that are not translated from English are symbolic links to the English ones. @@ -48,9 +48,9 @@ Each language lays in the corresponding folder. Files that are not translated fr You can contribute to the documentation in many ways, for example: -- Fork the ClickHouse repository, edit, commit, push, and open a pull request. +- Fork the ClickHouse and ClickHouse-docs repositories, edit, commit, push, and open a pull request. - Add the `documentation` label to this pull request for proper automatic checks applying. If you have no permissions for adding labels, the reviewer of your PR adds it. + Add the `pr-documentation` label to this pull request for proper automatic checks applying. If you do not have permission to add labels, then the reviewer of your PR will add it. - Open a required file in the ClickHouse repository and edit it from the GitHub web interface. @@ -158,15 +158,15 @@ When everything is ready, we will add the new language to the website. -### Documentation for Different Audience +### Documentation for Different Audiences -When writing documentation, think about people who read it. Each audience has specific requirements for terms they use in communications. +When writing documentation, think about the people who read it. Each audience has specific requirements for terms they use in communications. -ClickHouse documentation can be divided by the audience for the following parts: +ClickHouse documentation can be divided up by the audience for the following parts: -- Conceptual topics in [Introduction](https://clickhouse.com/docs/en/), tutorials and overviews, changelog. +- Conceptual topics like tutorials and overviews. - These topics are for the most common auditory. When editing text in them, use the most common terms that are comfortable for the audience with basic technical skills. + These topics are for the most common audience. When editing text in them, use the most common terms that are comfortable for the audience with basic technical skills. - Query language reference and related topics. diff --git a/docs/tools/README.md b/docs/tools/README.md index 163600804c6..7cf3540d108 100644 --- a/docs/tools/README.md +++ b/docs/tools/README.md @@ -1,50 +1,94 @@ -## How ClickHouse documentation is generated? {#how-clickhouse-documentation-is-generated} +## Generating ClickHouse documentation {#how-clickhouse-documentation-is-generated} -ClickHouse documentation is built using [build.py](build.py) script that uses [mkdocs](https://www.mkdocs.org) library and it’s dependencies to separately build all version of documentations (all languages in either single and multi page mode) as static HTMLs for each single page version. The results are then put in the correct directory structure. It is recommended to use Python 3.7 to run this script. +ClickHouse documentation is built using [Docusaurus](https://docusaurus.io). -[release.sh](release.sh) also pulls static files needed for [official ClickHouse website](https://clickhouse.com) from [../../website](../../website) folder then pushes to specified GitHub repo to be served via [GitHub Pages](https://pages.github.com). +## Check the look of your documentation changes {#how-to-check-if-the-documentation-will-look-fine} -## How to check if the documentation will look fine? {#how-to-check-if-the-documentation-will-look-fine} +There are a few options that are all useful depending on how large or complex your edits are. -There are few options that are all useful depending on how large or complex your edits are. +### Use the GitHub web interface to edit -### Use GitHub web interface to edit +Every page in the docs has an **Edit this page** link that opens the page in the GitHub editor. GitHub has Markdown support with a preview feature. The details of GitHub Markdown and the documentation Markdown are a bit different but generally this is close enough, and the person merging your PR will build the docs and check them. -GitHub has Markdown support with preview feature, but the details of GitHub Markdown dialect are a bit different in ClickHouse documentation. +### Install a Markdown editor or plugin for your IDE {#install-markdown-editor-or-plugin-for-your-ide} -### Install Markdown editor or plugin for your IDE {#install-markdown-editor-or-plugin-for-your-ide} +Usually, these plugins provide a preview of how the markdown will render, and they catch basic errors like unclosed tags very early. -Usually those also have some way to preview how Markdown will look like, which allows to catch basic errors like unclosed tags very early. -### Use build.py {#use-build-py} +## Build the docs locally {#use-build-py} -It’ll take some effort to go through, but the result will be very close to production documentation. +You can build the docs locally. It takes a few minutes to set up, but once you have done it the first time, the process is very simple. -For the first time you’ll need to: +### Clone the repos -#### 1. Set up virtualenv +The documentation is in two repos, clone both of them: +- [ClickHouse/ClickHouse](https://github.com/ClickHouse/ClickHouse) +- [ClickHouse/ClickHouse-docs](https://github.com/ClickHouse/clickhouse-docs) -``` bash -$ cd ClickHouse/docs/tools -$ mkdir venv -$ virtualenv -p $(which python3) venv -$ source venv/bin/activate -$ pip3 install -r requirements.txt +### Install Node.js + +The documentation is built with Docusaurus, which requires Node.js. We recommend version 16. Install [Node.js](https://nodejs.org/en/download/). + +### Copy files into place + +Docusaurus expects all of the markdown files to be located in the directory tree `clickhouse-docs/docs/`. This is not the way our repos are set up, so some copying of files is needed to build the docs: + +```bash +# from the parent directory of both the ClickHouse/ClickHouse and ClickHouse-clickhouse-docs repos: +cp -r ClickHouse/docs/en/development clickhouse-docs/docs/en/ +cp -r ClickHouse/docs/en/engines clickhouse-docs/docs/en/ +cp -r ClickHouse/docs/en/getting-started clickhouse-docs/docs/en/ +cp -r ClickHouse/docs/en/interfaces clickhouse-docs/docs/en/ +cp -r ClickHouse/docs/en/operations clickhouse-docs/docs/en/ +cp -r ClickHouse/docs/en/sql-reference clickhouse-docs/docs/en/ + +cp -r ClickHouse/docs/ru/* clickhouse-docs/docs/ru/ +cp -r ClickHouse/docs/zh clickhouse-docs/docs/ ``` -#### 2. Run build.py +#### Note: Symlinks will not work. +### Setup Docusaurus -When all prerequisites are installed, running `build.py` without args (there are some, check `build.py --help`) will generate `ClickHouse/docs/build` folder with complete static html website. +There are two commands that you may need to use with Docusaurus: +- `yarn install` +- `yarn start` -The easiest way to see the result is to use `--livereload=8888` argument of build.py. Alternatively, you can manually launch a HTTP server to serve the docs, for example by running `cd ClickHouse/docs/build && python3 -m http.server 8888`. Then go to http://localhost:8888 in browser. Feel free to use any other port instead of 8888. +#### Install Docusaurus and its dependencies: + +```bash +cd clickhouse-docs +yarn install +``` + +#### Start a development Docusaurus environment + +This command will start Docusaurus in development mode, which means that as you edit source (for example, `.md` files) files the changes will be rendered into HTML files and served by the Docusaurus development server. + +```bash +yarn start +``` + +### Make your changes to the markdown files + +Edit your files. Remember that if you are editing files in the `ClickHouse/ClickHouse` repo then you should edit them +in that repo and then copy the edited file into the `ClickHouse/clickhouse-docs/` directory structure so that they are updated in your develoment environment. + +`yarn start` probably opened a browser for you when you ran it; if not, open a browser to `http://localhost:3000/docs/en/intro` and navigate to the documentation that you are changing. If you have already made the changes, you can verify them here; if not, make them, and you will see the page update as you save the changes. ## How to change code highlighting? {#how-to-change-code-hl} -ClickHouse does not use mkdocs `highlightjs` feature. It uses modified pygments styles instead. -If you want to change code highlighting, edit the `website/css/highlight.css` file. -Currently, an [eighties](https://github.com/idleberg/base16-pygments/blob/master/css/base16-eighties.dark.css) theme -is used. +Code highlighting is based on the language chosen for your code blocks. Specify the language when you start the code block: +
```sql
+SELECT firstname from imdb.actors;
+```
+
+ +```sql +SELECT firstname from imdb.actors; +``` + +If you need a language supported then open an issue in [ClickHouse-docs](https://github.com/ClickHouse/clickhouse-docs/issues). ## How to subscribe on documentation changes? {#how-to-subscribe-on-documentation-changes} At the moment there’s no easy way to do just that, but you can consider: From ae96911735d122f7be55ca6155a470dd5ecc19b2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 18 Jul 2022 17:31:36 +0200 Subject: [PATCH 355/659] fix flaky 01174_select_insert_isolation --- tests/queries/0_stateless/01174_select_insert_isolation.sh | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index 58dadb68117..0c2b10cab41 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -39,11 +39,12 @@ function thread_select() # Result of `uniq | wc -l` must be 1 if the first and the last queries got the same result $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; - SELECT arraySort(groupArray(n)), arraySort(groupArray(m)), arraySort(groupArray(_part)) FROM mt; + SET throw_on_unsupported_query_inside_transaction=0; + CREATE TEMPORARY TABLE tmp AS SELECT arraySort(groupArray(n)), arraySort(groupArray(m)), arraySort(groupArray(_part)) FROM mt FORMAT Null; SELECT throwIf((SELECT sum(n) FROM mt) != 0) FORMAT Null; SELECT throwIf((SELECT count() FROM mt) % 2 != 0) FORMAT Null; - SELECT arraySort(groupArray(n)), arraySort(groupArray(m)), arraySort(groupArray(_part)) FROM mt; - COMMIT;" | uniq | wc -l | grep -v "^1$" ||: + select throwIf((SELECT * FROM tmp) != (SELECT arraySort(groupArray(n)), arraySort(groupArray(m)), arraySort(groupArray(_part)) FROM mt)) FORMAT Null; + COMMIT;" done } From 2367f40b70ad2aca8e30bfc33108380e60c1d9e1 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 18 Jul 2022 15:36:33 +0000 Subject: [PATCH 356/659] Better exception messages in schema inference --- src/Formats/ReadSchemaUtils.cpp | 23 +++++++++++++++++------ 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 11a91bd50dc..39812f5ba56 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -66,7 +66,7 @@ ColumnsDescription readSchemaFromFormat( } catch (const DB::Exception & e) { - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}", format_name, e.message()); + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}. You can specify the structure manually", format_name, e.message()); } } else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name)) @@ -75,16 +75,27 @@ ColumnsDescription readSchemaFromFormat( SchemaReaderPtr schema_reader; size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference; size_t iterations = 0; - while ((buf = read_buffer_iterator())) + while (true) { + bool is_eof = false; + try + { + buf = read_buffer_iterator(); + is_eof = buf->eof(); + } + catch (...) + { + auto exception_message = getCurrentExceptionMessage(false); + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file: {}. You can specify the structure manually", format_name, exception_message); + } ++iterations; - if (buf->eof()) + if (is_eof) { auto exception_message = fmt::format("Cannot extract table structure from {} format file, file is empty", format_name); if (!retry) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, exception_message); + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "{}. You can specify the structure manually", exception_message); exception_messages += "\n" + exception_message; continue; @@ -118,14 +129,14 @@ ColumnsDescription readSchemaFromFormat( } if (!retry || !isRetryableSchemaInferenceError(getCurrentExceptionCode())) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}", format_name, exception_message); + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}. You can specify the structure manually", format_name, exception_message); exception_messages += "\n" + exception_message; } } if (names_and_types.empty()) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "All attempts to extract table structure from files failed. Errors:{}", exception_messages); + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "All attempts to extract table structure from files failed. Errors:{}\nYou can specify the structure manually", exception_messages); /// If we have "INSERT SELECT" query then try to order /// columns as they are ordered in table schema for formats From 51c4ffad8c07a37e509684d934c469834285f884 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 18 Jul 2022 17:40:09 +0200 Subject: [PATCH 357/659] block memory exceptions in TransactionLog --- src/Interpreters/MergeTreeTransaction.cpp | 2 ++ src/Interpreters/TransactionLog.cpp | 2 ++ 2 files changed, 4 insertions(+) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 3142e6e7c54..f438194b87b 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -210,6 +210,7 @@ scope_guard MergeTreeTransaction::beforeCommit() void MergeTreeTransaction::afterCommit(CSN assigned_csn) noexcept { + LockMemoryExceptionInThread memory_tracker_lock(VariableContext::Global); /// Write allocated CSN into version metadata, so we will know CSN without reading it from transaction log /// and we will be able to remove old entries from transaction log in ZK. /// It's not a problem if server crash before CSN is written, because we already have TID in data part and entry in the log. @@ -245,6 +246,7 @@ void MergeTreeTransaction::afterCommit(CSN assigned_csn) noexcept bool MergeTreeTransaction::rollback() noexcept { + LockMemoryExceptionInThread memory_tracker_lock(VariableContext::Global); CSN expected = Tx::UnknownCSN; bool need_rollback = csn.compare_exchange_strong(expected, Tx::RolledBackCSN); diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index ec7969952f5..b349fdc1b23 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -457,6 +457,7 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn, bool CSN TransactionLog::finalizeCommittedTransaction(MergeTreeTransaction * txn, CSN allocated_csn, scope_guard & state_guard) noexcept { + LockMemoryExceptionInThread memory_tracker_lock(VariableContext::Global); chassert(!allocated_csn == txn->isReadOnly()); if (allocated_csn) { @@ -502,6 +503,7 @@ bool TransactionLog::waitForCSNLoaded(CSN csn) const void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) noexcept { + LockMemoryExceptionInThread memory_tracker_lock(VariableContext::Global); LOG_TRACE(log, "Rolling back transaction {}{}", txn->tid, std::uncaught_exceptions() ? fmt::format(" due to uncaught exception (code: {})", getCurrentExceptionCode()) : ""); From bc9c359abe9d84dd9a5b8947d603c9bf37f3e013 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 18 Jul 2022 18:47:35 +0300 Subject: [PATCH 358/659] Update 01174_select_insert_isolation.sh --- tests/queries/0_stateless/01174_select_insert_isolation.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index 0c2b10cab41..dc5c1d7a722 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -36,7 +36,7 @@ function thread_insert_rollback() function thread_select() { while true; do - # Result of `uniq | wc -l` must be 1 if the first and the last queries got the same result + # The first and the last queries must get the same result $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; SET throw_on_unsupported_query_inside_transaction=0; From c6f7698f9ad6ae22199182ebf7c3b2dac77d69d8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 18 Jul 2022 18:32:18 +0300 Subject: [PATCH 359/659] Fix possible UB in MergeTreeBackgroundExecutor Lambda erase_from_active() captures the item (TaskRuntimeDataPtr), most of the code path is OK, since it explicitly reset the item->task. However one is not, when it moves the item to pending list, which will be cleaned up when the table will be DROP/DETACH'ed, from MergeTreeBackgroundExecutor::removeTasksCorrespondingToStorage(), and in this case if IStorage will be already removed, then it will lead to use-after-free on destroying the lambda, since it captures the item by value. And I belive that CI founds this issue here [1]:
stack trace 4 0x268d1354 in DB::ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting() build_docker/../src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp:1510:33 5 0x268ead1a in std::__1::default_delete::operator()(DB::ReplicatedMergeTreeQueue::CurrentlyExecuting*) const build_docker/../contrib/libcxx/include/__memory/unique_ptr.h:54:5 6 0x268ead1a in std::__1::unique_ptr >::reset(DB::ReplicatedMergeTreeQueue::CurrentlyExecuting*) build_docker/../contrib/libcxx/include/__memory/unique_ptr.h:315:7 7 0x268ead1a in std::__1::unique_ptr >::~unique_ptr() build_docker/../contrib/libcxx/include/__memory/unique_ptr.h:269:19 8 0x268ead1a in DB::ReplicatedMergeTreeQueue::SelectedEntry::~SelectedEntry() build_docker/../src/Storages/MergeTree/ReplicatedMergeTreeQueue.h:351:12 9 0x268ead1a in void std::__1::__destroy_at(DB::ReplicatedMergeTreeQueue::SelectedEntry*) build_docker/../contrib/libcxx/include/__memory/construct_at.h:56:13 ... 16 0x265e9abb in DB::MergeTreeBackgroundExecutor::routine(std::__1::shared_ptr)::'lambda'()::~() build_docker/../src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp:127:30 17 0x265e9abb in DB::MergeTreeBackgroundExecutor::routine(std::__1::shared_ptr) build_docker/../src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp:235:1 18 0x265ea730 in DB::MergeTreeBackgroundExecutor::threadFunction() build_docker/../src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp:261:13
[1]: https://s3.amazonaws.com/clickhouse-test-reports/0/533c7888038453c047df816f3f65292cca05a54f/stateless_tests__ubsan__actions_.html And I also I think that the following issue will also be fixed [2]:
stack trace [ 680 ] {} : Logical error: 'Memory tracker: allocations not allowed.'. ... [ 23430 ] {} BaseDaemon: 23. /build/build_docker/../src/Common/formatReadable.cpp:46: formatReadableSizeWithBinarySuffix(double, int) @ 0x1713bb98 in /usr/bin/clickhouse ... [ 23430 ] {} BaseDaemon: 31. ../src/Common/MemoryTracker.cpp:112: MemoryTracker::logPeakMemoryUsage() @ 0x170e0ab9 in /usr/bin/clickhouse [ 23430 ] {} BaseDaemon: 32. /build/build_docker/../src/Common/MemoryTracker.cpp:98: MemoryTracker::~MemoryTracker() @ 0x170e063a in /usr/bin/clickhouse [ 23430 ] {} BaseDaemon: 33. /build/build_docker/../src/Storages/MergeTree/MergeList.cpp:144: DB::MergeListElement::~MergeListElement() @ 0x279fb290 in /usr/bin/clickhouse ... [ 23430 ] {} BaseDaemon: 38. /build/build_docker/../src/Storages/MergeTree/BackgroundProcessList.h:41: DB::BackgroundProcessListEntry::~BackgroundProcessListEntry() @ 0x276ce6c7 in /usr/bin/clickhouse [ 23430 ] {} BaseDaemon: 39. /build/build_docker/../contrib/libcxx/include/__memory/unique_ptr.h:54: std::__1::default_delete >::operator()(DB::BackgroundProcessListEntry*) const @ 0x276ce60b in /usr/bin/clickhouse [ 23430 ] {} BaseDaemon: 40. /build/build_docker/../contrib/libcxx/include/__memory/unique_ptr.h:316: std::__1::unique_ptr, std::__1::default_delete > >::reset(DB::BackgroundProcessListEntry*) @ 0x276ce57c in /usr/bin/clickhouse [ 23430 ] {} BaseDaemon: 41. /build/build_docker/../contrib/libcxx/include/__memory/unique_ptr.h:269: std::__1::unique_ptr, std::__1::default_delete > >::~unique_ptr() @ 0x276ce399 in /usr/bin/clickhouse [ 23430 ] {} BaseDaemon: 42. /build/build_docker/../src/Storages/MergeTree/MutatePlainMergeTreeTask.h:22: DB::MutatePlainMergeTreeTask::~MutatePlainMergeTreeTask() @ 0x27defceb in /usr/bin/clickhouse [ 23430 ] {} BaseDaemon: 43. /build/build_docker/../contrib/libcxx/include/__memory/construct_at.h:57: void std::__1::__destroy_at(DB::MutatePlainMergeTreeTask*) @ 0x27dd69c1 in /usr/bin/clickhouse [ 23430 ] {} BaseDaemon: 44. /build/build_docker/../contrib/libcxx/include/__memory/construct_at.h:82: void std::__1::destroy_at(DB::MutatePlainMergeTreeTask*) @ 0x27dd6955 in /usr/bin/clickhouse [ 23430 ] {} BaseDaemon: Integrity check of the executable skipped because the reference checksum could not be read. (calculated checksum: 91F5937571C11255DFE73230B52CE9C0) [ 602 ] {} Application: Child process was terminated by signal 6.
[2]: https://s3.amazonaws.com/clickhouse-test-reports/39222/a068c397dfd7943359a8b554566c3c70b78baf8d/stateless_tests__debug__actions__%5B1/3%5D.html Refs: https://github.com/ClickHouse/ClickHouse/pull/29614#discussion_r720455032 (cc @nikitamikhaylov) Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 08d39091cfd..234487763d7 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -124,7 +124,7 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) /// All operations with queues are considered no to do any allocations - auto erase_from_active = [this, item]() TSA_REQUIRES(mutex) + auto erase_from_active = [this, &item]() TSA_REQUIRES(mutex) { active.erase(std::remove(active.begin(), active.end(), item), active.end()); }; @@ -157,11 +157,10 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) if (need_execute_again) { std::lock_guard guard(mutex); + erase_from_active(); if (item->is_currently_deleting) { - erase_from_active(); - /// This is significant to order the destructors. { NOEXCEPT_SCOPE({ @@ -179,7 +178,6 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) /// Otherwise the destruction of the task won't be ordered with the destruction of the /// storage. pending.push(std::move(item)); - erase_from_active(); has_tasks.notify_one(); item = nullptr; return; From e165e68386daaef105a4875dda1477d3d4237cca Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 18 Jul 2022 18:12:13 +0200 Subject: [PATCH 360/659] Addressed review comment - updated creating ForkWriteBuffer in ClientBase to avoid duplication --- src/Client/ClientBase.cpp | 22 +++++++--------------- 1 file changed, 7 insertions(+), 15 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index cbb5ec9f004..c8a1dd3dcb0 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -549,26 +549,18 @@ try range.second); } + out_file_buf = wrapWriteBufferWithCompressionMethod( + std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), + compression_method, + compression_level + ); + if (query_with_output->is_into_outfile_with_stdout) { select_into_file_and_stdout = true; - WriteBufferPtr file_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), - compression_method, - compression_level - ); - - out_file_buf = std::make_unique(std::vector{file_buf, + out_file_buf = std::make_unique(std::vector{std::move(out_file_buf), std::make_shared(STDOUT_FILENO)}); } - else - { - out_file_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), - compression_method, - compression_level - ); - } // We are writing to file, so default format is the same as in non-interactive mode. if (is_interactive && is_default_format) From c7569d8b054169fb50f74536d77011e6cad3c0e2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 18 Jul 2022 18:28:42 +0200 Subject: [PATCH 361/659] Fix build --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index f59cd546bab..6d907395221 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -76,7 +76,7 @@ namespace SettingsChangesHistory /// across all ClickHouse versions. It maps ClickHouse version to settings changes that were done /// in this version. Settings changes is a vector of structs {setting_name, previous_value, new_value} /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) -std::map settings_changes_history = +static std::map settings_changes_history = { {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}}}, From 7f6b175c3eecd3e37d1199ffc44185d46e3eb8e3 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 18 Jul 2022 19:39:10 +0200 Subject: [PATCH 362/659] impl --- .../external-dictionaries/external-dicts-dict-sources.md | 3 ++- .../external-dictionaries/external-dicts-dict-sources.md | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index ec62205a36d..1a5308b5569 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -553,6 +553,7 @@ Setting fields: :::note The `table` or `where` fields cannot be used together with the `query` field. And either one of the `table` or `query` fields must be declared. +There is no explicit parameter `secure`. Both cases: when establishing SSL-connection is mandatory and when it's not are handled automatically. ::: MySQL can be connected on a local host via sockets. To do this, set `host` and `socket`. @@ -815,4 +816,4 @@ Setting fields: :::note The `table` or `where` fields cannot be used together with the `query` field. And either one of the `table` or `query` fields must be declared. -::: \ No newline at end of file +::: diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 36475449cf8..ac03dd39047 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -542,6 +542,7 @@ SOURCE(MYSQL( :::info "Примечание" Поля `table` или `where` не могут быть использованы вместе с полем `query`. Также обязательно должен быть один из источников данных: `table` или `query`. + Явный параметр `secure` отсутствует. Автоматически поддержана работа в обоих случаях: когда установка SSL-соединения необходима и когда нет. MySQL можно подключить на локальном хосте через сокеты, для этого необходимо задать `host` и `socket`. From 3f81aadb6051a31c6bf0c78bf19e342c0d630790 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 18 Jul 2022 17:53:18 +0000 Subject: [PATCH 363/659] Fix schema inference in case of empty messages in Protobuf/CapnProto formats --- src/Formats/CapnProtoUtils.cpp | 8 ++++++- src/Formats/ProtobufSerializer.cpp | 6 ++++- ...apnproto_protobuf_empty_messages.reference | 4 ++++ ...02327_capnproto_protobuf_empty_messages.sh | 24 +++++++++++++++++++ .../format_schemas/02327_schema.capnp | 10 ++++++++ .../format_schemas/02327_schema.proto | 8 +++++++ 6 files changed, 58 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.reference create mode 100755 tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh create mode 100644 tests/queries/0_stateless/format_schemas/02327_schema.capnp create mode 100644 tests/queries/0_stateless/format_schemas/02327_schema.proto diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index add5220414f..4ae2590ee60 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -16,6 +16,8 @@ #include #include +#include + namespace DB { @@ -490,6 +492,10 @@ static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type) { auto struct_schema = capnp_type.asStruct(); + + if (struct_schema.getFields().size() == 0) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Empty messages are not supported"); + /// Check if it can be Nullable. if (checkIfStructIsNamedUnion(struct_schema)) { @@ -525,7 +531,7 @@ static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type) NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema) { if (checkIfStructContainsUnnamedUnion(schema)) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported 2"); NamesAndTypesList names_and_types; for (auto field : schema.getNonUnionFields()) diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index b9af9d61da0..02054d0c1ed 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -3489,7 +3489,11 @@ namespace case FieldTypeId::TYPE_MESSAGE: { const auto * message_descriptor = field_descriptor->message_type(); - if (message_descriptor->field_count() == 1) + if (message_descriptor->field_count() == 0) + { + throw Exception("Empty messages are not supported", ErrorCodes::BAD_ARGUMENTS); + } + else if (message_descriptor->field_count() == 1) { const auto * nested_field_descriptor = message_descriptor->field(0); auto nested_name_and_type = getNameAndDataTypeFromField(nested_field_descriptor); diff --git a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.reference b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.reference new file mode 100644 index 00000000000..b462a5a7baa --- /dev/null +++ b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.reference @@ -0,0 +1,4 @@ +OK +OK +OK +OK diff --git a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh new file mode 100755 index 00000000000..3890f013b3b --- /dev/null +++ b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel, no-replicated-database + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +touch $USER_FILES_PATH/data.capnp + +SCHEMADIR=$(clickhouse-client --query "select * from file('data.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") +CLIENT_SCHEMADIR=$CURDIR/format_schemas +SERVER_SCHEMADIR=test_02327 +mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR +cp -r $CLIENT_SCHEMADIR/02327_* $SCHEMADIR/$SERVER_SCHEMADIR/ + + +$CLICKHOUSE_CLIENT --query="desc file(data.pb) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="desc file(data.capnp) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; + +$CLICKHOUSE_CLIENT --query="create table t engine=File(Protobuf) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="create table t engine=File(CapnProto) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; + +rm -rf ${SCHEMADIR:?}/${SERVER_SCHEMADIR:?} diff --git a/tests/queries/0_stateless/format_schemas/02327_schema.capnp b/tests/queries/0_stateless/format_schemas/02327_schema.capnp new file mode 100644 index 00000000000..c882dcab8d4 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02327_schema.capnp @@ -0,0 +1,10 @@ +@0x9ef128e10a8010b8; + +struct Nested1 +{ +} + +struct Message +{ + tuple1 @0 : Nested1; +} diff --git a/tests/queries/0_stateless/format_schemas/02327_schema.proto b/tests/queries/0_stateless/format_schemas/02327_schema.proto new file mode 100644 index 00000000000..ae1f440d279 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02327_schema.proto @@ -0,0 +1,8 @@ +syntax = "proto3"; + +message Nested { +} + +message Message { + Nested nested = 1; +}; From 24c94676413b38fa65e3568db524b2b72d62e435 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 18 Jul 2022 19:55:14 +0200 Subject: [PATCH 364/659] Fix --- src/Formats/CapnProtoUtils.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index 4ae2590ee60..65954315c0d 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -16,8 +16,6 @@ #include #include -#include - namespace DB { @@ -531,7 +529,7 @@ static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type) NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema) { if (checkIfStructContainsUnnamedUnion(schema)) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported 2"); + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); NamesAndTypesList names_and_types; for (auto field : schema.getNonUnionFields()) From 9de72d995a14d18b49d5dd9b29898e42352ffbff Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 12 Jul 2022 13:25:14 +0200 Subject: [PATCH 365/659] POC lightweight delete using __row_exists virtual column and prewhere-like filtering --- src/Core/Settings.h | 1 + src/Interpreters/MutationsInterpreter.cpp | 6 +- src/Processors/QueryPlan/FilterStep.cpp | 4 +- .../QueryPlan/ReadFromMergeTree.cpp | 1 + .../Transforms/FilterSortedStreamByRange.h | 2 +- src/Processors/Transforms/FilterTransform.cpp | 15 +++-- src/Processors/Transforms/FilterTransform.h | 2 +- src/Storages/MergeTree/MergeTask.cpp | 20 +++++- .../MergeTreeBaseSelectProcessor.cpp | 64 ++++++++++++++++++- .../MergeTree/MergeTreeBaseSelectProcessor.h | 1 + .../MergeTree/MergeTreeBlockReadUtils.cpp | 14 +++- .../MergeTree/MergeTreeBlockReadUtils.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 1 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 ++ .../MergeTree/MergeTreeRangeReader.cpp | 27 ++++++-- src/Storages/MergeTree/MergeTreeRangeReader.h | 2 + src/Storages/MergeTree/MergeTreeReadPool.cpp | 4 +- src/Storages/MergeTree/MergeTreeReadPool.h | 3 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 8 ++- .../MergeTree/MergeTreeSequentialSource.cpp | 2 +- .../MergeTreeThreadSelectProcessor.cpp | 20 ++++++ src/Storages/MergeTree/MutateTask.cpp | 14 ++++ src/Storages/StorageDistributed.cpp | 1 + .../02352_ligthweight_delete.reference | 11 ++++ .../0_stateless/02352_ligthweight_delete.sql | 37 +++++++++++ 25 files changed, 238 insertions(+), 27 deletions(-) create mode 100644 tests/queries/0_stateless/02352_ligthweight_delete.reference create mode 100644 tests/queries/0_stateless/02352_ligthweight_delete.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7c559330e38..4e1024a07df 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -459,6 +459,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \ M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ M(Bool, allow_experimental_lightweight_delete, false, "Enable lightweight DELETE mutations for mergetree tables. Work in progress", 0) \ + M(Bool, allow_experimental_lwd2, false, "Enable lightweight DELETE mutations using __rows_exists column for mergetree tables. Work in progress", 0) \ M(Bool, lightweight_delete_mutation, true, "Enable to make ordinary ALTER DELETE queries lightweight for mergetree tables", 0) \ M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index cc658bfa764..21160ac5dd8 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -297,7 +297,9 @@ MutationsInterpreter::MutationsInterpreter( , is_lightweight(is_lightweight_) { if (is_lightweight) + { mutation_ast = prepareLightweightDelete(!can_execute); + } else mutation_ast = prepare(!can_execute); } @@ -354,7 +356,7 @@ static void validateUpdateColumns( } } - if (!found) + if (!found && column_name != "__row_exists") /// TODO: properly handle updating __row_exists column for LWD { for (const auto & col : metadata_snapshot->getColumns().getMaterialized()) { @@ -507,7 +509,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) /// /// Outer CAST is added just in case if we don't trust the returning type of 'if'. - const auto & type = columns_desc.getPhysical(column).type; + const auto type = (column == "__row_exists" ? std::make_shared() : columns_desc.getPhysical(column).type); auto type_literal = std::make_shared(type->getName()); const auto & update_expr = kv.second; diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index ff58abf8874..becfe1062fb 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -34,7 +34,7 @@ FilterStep::FilterStep( input_stream_, FilterTransform::transformHeader( input_stream_.header, - *actions_dag_, + actions_dag_.get(), filter_column_name_, remove_filter_column_), getTraits(actions_dag_)) @@ -109,7 +109,7 @@ void FilterStep::updateOutputStream() { output_stream = createOutputStream( input_streams.front(), - FilterTransform::transformHeader(input_streams.front().header, *actions_dag, filter_column_name, remove_filter_column), + FilterTransform::transformHeader(input_streams.front().header, actions_dag.get(), filter_column_name, remove_filter_column), getDataStreamTraits()); } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 59f6ec558e7..9a65cd4f17e 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -185,6 +185,7 @@ Pipe ReadFromMergeTree::readFromPool( storage_snapshot, prewhere_info, required_columns, + virt_column_names, backoff_settings, settings.preferred_block_size_bytes, false); diff --git a/src/Processors/Transforms/FilterSortedStreamByRange.h b/src/Processors/Transforms/FilterSortedStreamByRange.h index 04ef3afd69c..e1141ebd299 100644 --- a/src/Processors/Transforms/FilterSortedStreamByRange.h +++ b/src/Processors/Transforms/FilterSortedStreamByRange.h @@ -23,7 +23,7 @@ public: bool on_totals_ = false) : ISimpleTransform( header_, - FilterTransform::transformHeader(header_, expression_->getActionsDAG(), filter_column_name_, remove_filter_column_), + FilterTransform::transformHeader(header_, &expression_->getActionsDAG(), filter_column_name_, remove_filter_column_), true) , filter_transform(header_, expression_, filter_column_name_, remove_filter_column_, on_totals_) { diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index 9164599f3b1..5d75bdc483c 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -29,11 +29,12 @@ static void replaceFilterToConstant(Block & block, const String & filter_column_ Block FilterTransform::transformHeader( Block header, - const ActionsDAG & expression, + const ActionsDAG * expression, const String & filter_column_name, bool remove_filter_column) { - header = expression.updateHeader(std::move(header)); + if (expression) + header = expression->updateHeader(std::move(header)); if (remove_filter_column) header.erase(filter_column_name); @@ -51,7 +52,7 @@ FilterTransform::FilterTransform( bool on_totals_) : ISimpleTransform( header_, - transformHeader(header_, expression_->getActionsDAG(), filter_column_name_, remove_filter_column_), + transformHeader(header_, expression_ ? &expression_->getActionsDAG() : nullptr, filter_column_name_, remove_filter_column_), true) , expression(std::move(expression_)) , filter_column_name(std::move(filter_column_name_)) @@ -59,7 +60,8 @@ FilterTransform::FilterTransform( , on_totals(on_totals_) { transformed_header = getInputPort().getHeader(); - expression->execute(transformed_header); + if (expression) + expression->execute(transformed_header); filter_column_position = transformed_header.getPositionByName(filter_column_name); auto & column = transformed_header.getByPosition(filter_column_position).column; @@ -74,7 +76,7 @@ IProcessor::Status FilterTransform::prepare() /// Optimization for `WHERE column in (empty set)`. /// The result will not change after set was created, so we can skip this check. /// It is implemented in prepare() stop pipeline before reading from input port. - || (!are_prepared_sets_initialized && expression->checkColumnIsAlwaysFalse(filter_column_name)))) + || (!are_prepared_sets_initialized && expression && expression->checkColumnIsAlwaysFalse(filter_column_name)))) { input.close(); output.finish(); @@ -106,7 +108,8 @@ void FilterTransform::transform(Chunk & chunk) Block block = getInputPort().getHeader().cloneWithColumns(columns); columns.clear(); - expression->execute(block, num_rows_before_filtration); + if (expression) + expression->execute(block, num_rows_before_filtration); columns = block.getColumns(); } diff --git a/src/Processors/Transforms/FilterTransform.h b/src/Processors/Transforms/FilterTransform.h index 3340fe230b7..be892414c96 100644 --- a/src/Processors/Transforms/FilterTransform.h +++ b/src/Processors/Transforms/FilterTransform.h @@ -24,7 +24,7 @@ public: static Block transformHeader( Block header, - const ActionsDAG & expression, + const ActionsDAG * expression, const String & filter_column_name, bool remove_filter_column); diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 7426b384394..cc93e17bda3 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -810,11 +811,28 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() for (const auto & part : global_ctx->future_part->parts) { + auto columns = global_ctx->merging_column_names; + + if (part->getColumns().contains("__row_exists")) + columns.emplace_back("__row_exists"); + + auto input = std::make_unique( - *global_ctx->data, global_ctx->storage_snapshot, part, global_ctx->merging_column_names, ctx->read_with_direct_io, true); + *global_ctx->data, global_ctx->storage_snapshot, part, columns, ctx->read_with_direct_io, true); Pipe pipe(std::move(input)); + +///////////// + if (part->getColumns().contains("__row_exists")) + { + pipe.addSimpleTransform([](const Block & header) + { + return std::make_shared(header, nullptr, "__row_exists", "__row_exists"); + }); + } +///////////// + if (global_ctx->metadata_snapshot->hasSortingKey()) { pipe.addSimpleTransform([this](const Block & header) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 31ca533a9ec..933f9144c6a 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -61,6 +61,10 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( { non_const_virtual_column_names.emplace_back(*it); } + else if (*it == "__row_exists") + { + non_const_virtual_column_names.emplace_back(*it); + } else { /// Remove virtual columns that are going to be filled with const values @@ -219,10 +223,20 @@ void MergeTreeBaseSelectProcessor::initializeRangeReaders(MergeTreeReadTask & cu { MergeTreeRangeReader* prev_reader = nullptr; bool last_reader = false; + size_t pre_readers_shift = 0; + + if (!reader_settings.skip_deleted_mask && current_task.data_part->getColumns().contains("__row_exists")) + { +// last_reader = !prewhere_actions || prewhere_actions->steps.empty(); + current_task.pre_range_readers.push_back( + MergeTreeRangeReader(pre_reader_for_step[0].get(), prev_reader, &lwd_filter_step, last_reader, non_const_virtual_column_names)); + prev_reader = ¤t_task.pre_range_readers.back(); + pre_readers_shift++; + } if (prewhere_info) { - if (prewhere_actions->steps.size() != pre_reader_for_step.size()) + if (prewhere_actions->steps.size() + pre_readers_shift != pre_reader_for_step.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "PREWHERE steps count mismatch, actions: {}, readers: {}", prewhere_actions->steps.size(), pre_reader_for_step.size()); @@ -232,7 +246,7 @@ void MergeTreeBaseSelectProcessor::initializeRangeReaders(MergeTreeReadTask & cu { last_reader = reader->getColumns().empty() && (i + 1 == prewhere_actions->steps.size()); current_task.pre_range_readers.push_back( - MergeTreeRangeReader(pre_reader_for_step[i].get(), prev_reader, &prewhere_actions->steps[i], last_reader, non_const_virtual_column_names)); + MergeTreeRangeReader(pre_reader_for_step[i + pre_readers_shift].get(), prev_reader, &prewhere_actions->steps[i], last_reader, non_const_virtual_column_names)); prev_reader = ¤t_task.pre_range_readers.back(); } @@ -339,7 +353,10 @@ Chunk MergeTreeBaseSelectProcessor::readFromPartImpl() /// Reorder columns. TODO: maybe skip for default case. for (size_t ps = 0; ps < header_without_virtual_columns.columns(); ++ps) { - auto pos_in_sample_block = sample_block.getPositionByName(header_without_virtual_columns.getByPosition(ps).name); + const auto & name = header_without_virtual_columns.getByPosition(ps).name; + if (name == "__row_exists" && !sample_block.has(name)) + continue; /// TODO: properly deal with cases when __row_exists is not read and is filled later + auto pos_in_sample_block = sample_block.getPositionByName(name); ordered_columns.emplace_back(std::move(read_result.columns[pos_in_sample_block])); } @@ -365,6 +382,7 @@ namespace virtual void insertArrayOfStringsColumn(const ColumnPtr & column, const String & name) = 0; virtual void insertStringColumn(const ColumnPtr & column, const String & name) = 0; + virtual void insertUInt8Column(const ColumnPtr & column, const String & name) = 0; virtual void insertUInt64Column(const ColumnPtr & column, const String & name) = 0; virtual void insertUUIDColumn(const ColumnPtr & column, const String & name) = 0; @@ -390,6 +408,8 @@ static void injectNonConstVirtualColumns( { if (virtual_column_name == "_part_offset") inserter.insertUInt64Column(DataTypeUInt64().createColumn(), virtual_column_name); + if (virtual_column_name == "__row_exists") + inserter.insertUInt8Column(DataTypeUInt8().createColumn(), virtual_column_name); } } @@ -485,6 +505,11 @@ namespace block.insert({column, std::make_shared(), name}); } + void insertUInt8Column(const ColumnPtr & column, const String & name) final + { + block.insert({column, std::make_shared(), name}); + } + void insertUInt64Column(const ColumnPtr & column, const String & name) final { block.insert({column, std::make_shared(), name}); @@ -525,6 +550,11 @@ namespace columns.push_back(column); } + void insertUInt8Column(const ColumnPtr & column, const String &) final + { + columns.push_back(column); + } + void insertUInt64Column(const ColumnPtr & column, const String &) final { columns.push_back(column); @@ -569,6 +599,34 @@ void MergeTreeBaseSelectProcessor::injectVirtualColumns( auto columns = chunk.detachColumns(); VirtualColumnsInserterIntoColumns inserter{columns}; + +///////////////////////// +// TODO: implement properly + for (const auto & virtual_column_name : virtual_columns) + { + + if (virtual_column_name == "__row_exists") + { + if (task->data_part->getColumns().contains(virtual_column_name)) + { + /// If this column is present in the part it must be read from the data + assert(task->task_columns.columns.contains(virtual_column_name)); + } + else + { + /// If __row_exists column isn't present in the part then + ColumnPtr column; + if (num_rows) + column = DataTypeUInt8().createColumnConst(num_rows, 1)->convertToFullColumnIfConst(); + else + column = DataTypeUInt8().createColumn(); + + inserter.insertUInt8Column(column, virtual_column_name); + } + } + } +/////////////////////////// + /// Only add const virtual columns because non-const ones have already been added injectPartConstVirtualColumns(num_rows, inserter, task, partition_value_type, virtual_columns); diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 368dce7deaa..15a088d115c 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -89,6 +89,7 @@ protected: const MergeTreeData & storage; StorageSnapshotPtr storage_snapshot; + PrewhereExprStep lwd_filter_step { nullptr, "__row_exists", true, true }; PrewhereInfoPtr prewhere_info; std::unique_ptr prewhere_actions; diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 50f4c34f004..ad208f6b041 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -108,7 +108,9 @@ NameSet injectRequiredColumns( for (size_t i = 0; i < columns.size(); ++i) { /// We are going to fetch only physical columns - if (!storage_snapshot->tryGetColumn(options, columns[i])) + const bool is_real_column = storage_snapshot->tryGetColumn(options, columns[i]).has_value(); + const bool is_virtual_column = storage.isVirtualColumn(columns[i], storage_snapshot->getMetadataForQuery()); + if (!is_real_column && !is_virtual_column) throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no physical column or subcolumn {} in table", columns[i]); have_at_least_one_physical_column |= injectRequiredColumnsRecursively( @@ -272,18 +274,26 @@ MergeTreeReadTaskColumns getReadTaskColumns( const StorageSnapshotPtr & storage_snapshot, const MergeTreeData::DataPartPtr & data_part, const Names & required_columns, + const Names & non_const_virtual_columns, const PrewhereInfoPtr & prewhere_info, bool with_subcolumns) { Names column_names = required_columns; Names pre_column_names; + /// read non-const virtual column from data if it exists + for (const auto & name : non_const_virtual_columns) + { + if (data_part->getColumns().contains(name)) + column_names.push_back(name); + } + /// inject columns required for defaults evaluation injectRequiredColumns( storage, storage_snapshot, data_part, with_subcolumns, column_names); MergeTreeReadTaskColumns result; - auto options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects(); + auto options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); if (with_subcolumns) options.withSubcolumns(); diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index f9f82dbd1f2..d6277167555 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -79,6 +79,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( const StorageSnapshotPtr & storage_snapshot, const MergeTreeData::DataPartPtr & data_part, const Names & required_columns, + const Names & non_const_virtual_columns, const PrewhereInfoPtr & prewhere_info, bool with_subcolumns); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c887552b35b..4df47eb7765 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6562,6 +6562,7 @@ NamesAndTypesList MergeTreeData::getVirtuals() const NameAndTypePair("_partition_value", getPartitionValueType()), NameAndTypePair("_sample_factor", std::make_shared()), NameAndTypePair("_part_offset", std::make_shared()), + NameAndTypePair("__row_exists", std::make_shared()), }; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 85231aca253..36d45430cff 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1214,6 +1214,10 @@ static void selectColumnNames( { virt_column_names.push_back(name); } + else if (name == "__row_exists") + { + virt_column_names.push_back(name); + } else if (name == "_part_uuid") { virt_column_names.push_back(name); diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 2f34d75b2c4..cf90da36ace 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -657,21 +657,28 @@ MergeTreeRangeReader::MergeTreeRangeReader( , prewhere_info(prewhere_info_) , last_reader_in_chain(last_reader_in_chain_) , is_initialized(true) - , non_const_virtual_column_names(non_const_virtual_column_names_) +// , non_const_virtual_column_names() { + + if (prev_reader) sample_block = prev_reader->getSampleBlock(); for (const auto & name_and_type : merge_tree_reader->getColumns()) sample_block.insert({name_and_type.type->createColumn(), name_and_type.type, name_and_type.name}); - - for (const auto & column_name : non_const_virtual_column_names) + + for (const auto & column_name : non_const_virtual_column_names_) { if (sample_block.has(column_name)) continue; + non_const_virtual_column_names.push_back(column_name); + if (column_name == "_part_offset") sample_block.insert(ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), column_name)); + +// if (column_name == "__row_exists") +// sample_block.insert(ColumnWithTypeAndName(ColumnUInt8::create(), std::make_shared(), column_name)); } if (merge_tree_reader->needReadDeletedMask()) @@ -861,7 +868,11 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar if (read_result.num_rows) { /// Physical columns go first and then some virtual columns follow - const size_t physical_columns_count = read_result.columns.size() - non_const_virtual_column_names.size(); + size_t physical_columns_count = read_result.columns.size() - read_result.extra_columns_filled.size(); +/////////// +// TODO: properly account for "virtual columns" that are overridden with real data in the part + +///////////// Columns physical_columns(read_result.columns.begin(), read_result.columns.begin() + physical_columns_count); bool should_evaluate_missing_defaults; @@ -989,6 +1000,7 @@ void MergeTreeRangeReader::fillPartOffsetColumn(ReadResult & result, UInt64 lead } result.columns.emplace_back(std::move(column)); + result.extra_columns_filled.push_back("_part_offset"); } /// Fill deleted_row_mask column, referenced from fillPartOffsetColumn(). void MergeTreeRangeReader::fillDeletedRowMaskColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset) @@ -1184,7 +1196,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r size_t num_columns = header.size(); /// Check that we have columns from previous steps and newly read required columns - if (result.columns.size() < num_columns + non_const_virtual_column_names.size()) + if (result.columns.size() < num_columns + result.extra_columns_filled.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid number of columns passed to MergeTreeRangeReader. Expected {}, got {}", num_columns, result.columns.size()); @@ -1227,6 +1239,11 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r block.insert({result.columns[pos], std::make_shared(), column_name}); } + else if (column_name == "__row_exists") + { + /// do nothing, it will be added later + /// TODO: properly implement reading non-const virtual columns or filling them with default values + } else throw Exception("Unexpected non-const virtual column: " + column_name, ErrorCodes::LOGICAL_ERROR); ++pos; diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 0e6ace18be9..8f063786cbc 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -244,6 +244,8 @@ public: static size_t numZerosInTail(const UInt8 * begin, const UInt8 * end); std::map filter_bytes_map; + + Names extra_columns_filled; }; ReadResult read(size_t max_rows, MarkRanges & ranges); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index d44d250149e..9bcc6535abb 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -26,6 +26,7 @@ MergeTreeReadPool::MergeTreeReadPool( const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const Names & column_names_, + const Names & virtual_column_names_, const BackoffSettings & backoff_settings_, size_t preferred_block_size_bytes_, bool do_not_steal_tasks_) @@ -34,6 +35,7 @@ MergeTreeReadPool::MergeTreeReadPool( , data{data_} , storage_snapshot{storage_snapshot_} , column_names{column_names_} + , virtual_column_names{virtual_column_names_} , do_not_steal_tasks{do_not_steal_tasks_} , predict_block_size_bytes{preferred_block_size_bytes_ > 0} , prewhere_info{prewhere_info_} @@ -213,7 +215,7 @@ std::vector MergeTreeReadPool::fillPerPartInfo(const RangesInDataParts & auto task_columns = getReadTaskColumns( data, storage_snapshot, part.data_part, - column_names, prewhere_info, /*with_subcolumns=*/ true); + column_names, virtual_column_names /*TODO: fill non-const virtual columns*/, prewhere_info, /*with_subcolumns=*/ true); auto size_predictor = !predict_block_size_bytes ? nullptr : MergeTreeBaseSelectProcessor::getSizePredictor(part.data_part, task_columns, sample_block); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index d882c0d761f..01a1280b6cb 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -73,7 +73,7 @@ public: size_t threads_, size_t sum_marks_, size_t min_marks_for_concurrent_read_, RangesInDataParts && parts_, const MergeTreeData & data_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, - const Names & column_names_, + const Names & column_names_, const Names & virtual_column_names_, const BackoffSettings & backoff_settings_, size_t preferred_block_size_bytes_, bool do_not_steal_tasks_ = false); @@ -97,6 +97,7 @@ private: const MergeTreeData & data; StorageSnapshotPtr storage_snapshot; const Names column_names; + const Names virtual_column_names; bool do_not_steal_tasks; bool predict_block_size_bytes; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index aa6c457f412..47bcf72d611 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -52,7 +52,7 @@ void MergeTreeSelectProcessor::initializeReaders() { task_columns = getReadTaskColumns( storage, storage_snapshot, data_part, - required_columns, prewhere_info, /*with_subcolumns=*/ true); + required_columns, non_const_virtual_column_names, prewhere_info, /*with_subcolumns=*/ true); /// Will be used to distinguish between PREWHERE and WHERE columns when applying filter const auto & column_names = task_columns.columns.getNames(); @@ -68,6 +68,12 @@ void MergeTreeSelectProcessor::initializeReaders() pre_reader_for_step.clear(); + if (!reader_settings.skip_deleted_mask && data_part->getColumns().contains("__row_exists")) + { + pre_reader_for_step.push_back(data_part->getReader({{"__row_exists", std::make_shared()}}, storage_snapshot->getMetadataForQuery(), + all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, {}, {})); + } + if (prewhere_info) { for (const auto & pre_columns_for_step : task_columns.pre_columns) diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 47c45058088..53a56bad97e 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -46,7 +46,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( NamesAndTypesList columns_for_reader; if (take_column_types_from_storage) { - auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects(); + auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects().withVirtuals(); columns_for_reader = storage_snapshot->getColumnsByNames(options, columns_to_read); } else diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp index 574ce4479f2..4c0eac95593 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp @@ -116,6 +116,16 @@ void MergeTreeThreadSelectProcessor::finalizeNewTask() IMergeTreeReader::ValueSizeMap{}, profile_callback); pre_reader_for_step.clear(); + + + if (!reader_settings.skip_deleted_mask && task->data_part->getColumns().contains("__row_exists")) + { + pre_reader_for_step.push_back(task->data_part->getReader({{"__row_exists", std::make_shared()}}, metadata_snapshot, task->mark_ranges, + owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, + IMergeTreeReader::ValueSizeMap{}, profile_callback)); + } + + if (prewhere_info) { for (const auto & pre_columns_per_step : task->task_columns.pre_columns) @@ -137,6 +147,16 @@ void MergeTreeThreadSelectProcessor::finalizeNewTask() reader->getAvgValueSizeHints(), profile_callback); pre_reader_for_step.clear(); + + if (!reader_settings.skip_deleted_mask && task->data_part->getColumns().contains("__row_exists")) + { + pre_reader_for_step.push_back(task->data_part->getReader({{"__row_exists", std::make_shared()}}, metadata_snapshot, task->mark_ranges, + owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, + reader->getAvgValueSizeHints(), profile_callback)); + } + + + if (prewhere_info) { for (const auto & pre_columns_per_step : task->task_columns.pre_columns) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 50c37ba5b08..b47f0cab6ab 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -174,6 +174,15 @@ getColumnsForNewDataPart( /// All commands are validated in AlterCommand so we don't care about order for (const auto & command : commands_for_removes) { + if (command.type == MutationCommand::UPDATE) + { + for (const auto & [column_name, _] : command.column_to_update_expression) + { + if (column_name == "__row_exists" && !storage_columns.contains(column_name)) + storage_columns.emplace_back("__row_exists", std::make_shared()); + } + } + /// If we don't have this column in source part, than we don't need to materialize it if (!part_columns.has(command.column_name)) continue; @@ -1682,6 +1691,11 @@ bool MutateTask::prepare() need_mutate_all_columns = need_mutate_all_columns || (ctx->mutation_kind == MutationsInterpreter::MutationKind::MUTATE_OTHER && ctx->interpreter->isAffectingAllColumns()); if (!need_mutate_all_columns && ctx->source_part->hasLightweightDelete() && !ctx->is_lightweight_mutation) ctx->interpreter->setSkipDeletedMask(true); + +///// + ctx->interpreter->setSkipDeletedMask(true); +///// + ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); ctx->progress_callback = MergeProgressCallback((*ctx->mutate_entry)->ptr(), ctx->watch_prev_elapsed, *ctx->stage_progress); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 03eb400a8ad..32e0fcffca6 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -312,6 +312,7 @@ NamesAndTypesList StorageDistributed::getVirtuals() const NameAndTypePair("_partition_id", std::make_shared()), NameAndTypePair("_sample_factor", std::make_shared()), NameAndTypePair("_part_offset", std::make_shared()), + NameAndTypePair("__row_exists", std::make_shared()), NameAndTypePair("_shard_num", std::make_shared()), /// deprecated }; } diff --git a/tests/queries/0_stateless/02352_ligthweight_delete.reference b/tests/queries/0_stateless/02352_ligthweight_delete.reference new file mode 100644 index 00000000000..26cce75896c --- /dev/null +++ b/tests/queries/0_stateless/02352_ligthweight_delete.reference @@ -0,0 +1,11 @@ +Rows in parts 10000000 +Count 10000000 +First row 0 10 +Delete 3M rows using light weight delete +Rows in parts 10000000 +Count 7000000 +First row 3000000 10 +Force merge to cleanup deleted rows +Rows in parts 7000000 +Count 7000000 +First row 3000000 10 diff --git a/tests/queries/0_stateless/02352_ligthweight_delete.sql b/tests/queries/0_stateless/02352_ligthweight_delete.sql new file mode 100644 index 00000000000..a472a927424 --- /dev/null +++ b/tests/queries/0_stateless/02352_ligthweight_delete.sql @@ -0,0 +1,37 @@ +DROP TABLE IF EXISTS lwd_test; + +CREATE TABLE lwd_test (id UInt64 , value String) ENGINE MergeTree() ORDER BY id; + +INSERT INTO lwd_test SELECT number, randomString(10) FROM system.numbers LIMIT 10000000; + +SET mutations_sync = 1; +--SET allow_experimental_lightweight_delete = 1; + +SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; + +SELECT 'Count', count() FROM lwd_test WHERE id >= 0; + +SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; + + +SELECT 'Delete 3M rows using light weight delete'; +ALTER TABLE lwd_test UPDATE __row_exists = 0 WHERE id < 3000000; + +SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; + +SELECT 'Count', count() FROM lwd_test WHERE id >= 0; + +SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; + + +SELECT 'Force merge to cleanup deleted rows'; +OPTIMIZE TABLE lwd_test FINAL; + +SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; + +SELECT 'Count', count() FROM lwd_test WHERE id >= 0; + +SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; + + +DROP TABLE lwd_test; \ No newline at end of file From f956810fdd791a9deab34abfbba5dcdaf05ec672 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 12 Jul 2022 22:56:15 +0200 Subject: [PATCH 366/659] Rewrite DELETE FROM into UPDATE __row_exist=0 --- src/Interpreters/InterpreterDeleteQuery.cpp | 56 +++++++++++++++---- ...nce => 02352_lightweight_delete.reference} | 6 +- ...elete.sql => 02352_lightweight_delete.sql} | 16 +++++- 3 files changed, 62 insertions(+), 16 deletions(-) rename tests/queries/0_stateless/{02352_ligthweight_delete.reference => 02352_lightweight_delete.reference} (58%) rename tests/queries/0_stateless/{02352_ligthweight_delete.sql => 02352_lightweight_delete.sql} (71%) diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 3a786997ae3..33662b94614 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include #include #include @@ -31,7 +33,7 @@ InterpreterDeleteQuery::InterpreterDeleteQuery(const ASTPtr & query_ptr_, Contex BlockIO InterpreterDeleteQuery::execute() { - if (!getContext()->getSettingsRef().allow_experimental_lightweight_delete) + if (!getContext()->getSettingsRef().allow_experimental_lightweight_delete && !getContext()->getSettingsRef().allow_experimental_lwd2) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Lightweight delete mutate is experimental. Set `allow_experimental_lightweight_delete` setting to enable it"); FunctionNameNormalizer().visit(query_ptr.get()); @@ -68,20 +70,50 @@ BlockIO InterpreterDeleteQuery::execute() MutationCommands mutation_commands; MutationCommand mut_command; - mut_command.type = MutationCommand::Type::DELETE; - mut_command.predicate = delete_query.predicate; + if (getContext()->getSettingsRef().allow_experimental_lwd2) + { + /// UPDATE __row_exists = 0 WHERE predicate + mut_command.type = MutationCommand::Type::UPDATE; + mut_command.predicate = delete_query.predicate; - auto command = std::make_shared(); - command->type = ASTAlterCommand::DELETE; - command->predicate = delete_query.predicate; - command->children.push_back(command->predicate); - mut_command.ast = command->ptr(); + auto command = std::make_shared(); + command->type = ASTAlterCommand::UPDATE; + command->predicate = delete_query.predicate; + command->update_assignments = std::make_shared(); + auto set_row_exists = std::make_shared(); + set_row_exists->column_name = "__row_exists"; + auto zero_value = std::make_shared(DB::Field(UInt8(0))); + set_row_exists->children.push_back(zero_value); + command->update_assignments->children.push_back(set_row_exists); + command->children.push_back(command->predicate); + command->children.push_back(command->update_assignments); + mut_command.column_to_update_expression[set_row_exists->column_name] = zero_value; + mut_command.ast = command->ptr(); - mutation_commands.emplace_back(mut_command); + mutation_commands.emplace_back(mut_command); + + table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); + MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false, false).validate(); + storage_merge_tree->mutate(mutation_commands, getContext(), MutationType::Ordinary); + } + else + { + mut_command.type = MutationCommand::Type::DELETE; + mut_command.predicate = delete_query.predicate; + + auto command = std::make_shared(); + command->type = ASTAlterCommand::DELETE; + command->predicate = delete_query.predicate; + command->children.push_back(command->predicate); + mut_command.ast = command->ptr(); + + mutation_commands.emplace_back(mut_command); + + table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); + MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false, false).validate(); + storage_merge_tree->mutate(mutation_commands, getContext(), MutationType::Lightweight); + } - table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); - MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false, false).validate(); - storage_merge_tree->mutate(mutation_commands, getContext(), MutationType::Lightweight); return {}; } diff --git a/tests/queries/0_stateless/02352_ligthweight_delete.reference b/tests/queries/0_stateless/02352_lightweight_delete.reference similarity index 58% rename from tests/queries/0_stateless/02352_ligthweight_delete.reference rename to tests/queries/0_stateless/02352_lightweight_delete.reference index 26cce75896c..8ae6b1d3195 100644 --- a/tests/queries/0_stateless/02352_ligthweight_delete.reference +++ b/tests/queries/0_stateless/02352_lightweight_delete.reference @@ -1,7 +1,7 @@ Rows in parts 10000000 Count 10000000 First row 0 10 -Delete 3M rows using light weight delete +Delete 3M rows using UPDATE __row_exists Rows in parts 10000000 Count 7000000 First row 3000000 10 @@ -9,3 +9,7 @@ Force merge to cleanup deleted rows Rows in parts 7000000 Count 7000000 First row 3000000 10 +Delete 3M more rows using light weight DELETE +Rows in parts 7000000 +Count 4000000 +First row 6000000 10 diff --git a/tests/queries/0_stateless/02352_ligthweight_delete.sql b/tests/queries/0_stateless/02352_lightweight_delete.sql similarity index 71% rename from tests/queries/0_stateless/02352_ligthweight_delete.sql rename to tests/queries/0_stateless/02352_lightweight_delete.sql index a472a927424..c5f636181cd 100644 --- a/tests/queries/0_stateless/02352_ligthweight_delete.sql +++ b/tests/queries/0_stateless/02352_lightweight_delete.sql @@ -5,7 +5,6 @@ CREATE TABLE lwd_test (id UInt64 , value String) ENGINE MergeTree() ORDER BY id; INSERT INTO lwd_test SELECT number, randomString(10) FROM system.numbers LIMIT 10000000; SET mutations_sync = 1; ---SET allow_experimental_lightweight_delete = 1; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; @@ -14,7 +13,7 @@ SELECT 'Count', count() FROM lwd_test WHERE id >= 0; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; -SELECT 'Delete 3M rows using light weight delete'; +SELECT 'Delete 3M rows using UPDATE __row_exists'; ALTER TABLE lwd_test UPDATE __row_exists = 0 WHERE id < 3000000; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; @@ -34,4 +33,15 @@ SELECT 'Count', count() FROM lwd_test WHERE id >= 0; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; -DROP TABLE lwd_test; \ No newline at end of file +SET allow_experimental_lwd2 = 1; +SELECT 'Delete 3M more rows using light weight DELETE'; +DELETE FROM lwd_test WHERE id < 6000000; + +SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; + +SELECT 'Count', count() FROM lwd_test WHERE id >= 0; + +SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; + + +DROP TABLE lwd_test; From f324ca992155b85dd3b4ef5e901f094fa90473b2 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sun, 17 Jul 2022 20:41:17 +0200 Subject: [PATCH 367/659] Cleanups --- src/Core/Settings.h | 2 +- src/Interpreters/InterpreterDeleteQuery.cpp | 19 +- src/Interpreters/MutationsInterpreter.cpp | 18 +- src/Storages/AlterCommands.cpp | 7 +- src/Storages/MergeTree/MergeTask.cpp | 18 +- .../MergeTreeBaseSelectProcessor.cpp | 294 +++++++----------- .../MergeTree/MergeTreeBaseSelectProcessor.h | 28 +- .../MergeTree/MergeTreeRangeReader.cpp | 1 + src/Storages/MergeTree/MergeTreeRangeReader.h | 6 +- .../MergeTreeReverseSelectProcessor.cpp | 6 +- .../MergeTreeReverseSelectProcessor.h | 4 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 23 +- .../MergeTreeThreadSelectProcessor.cpp | 64 +--- src/Storages/MergeTree/MutateTask.cpp | 16 +- .../MergeTree/registerStorageMergeTree.cpp | 3 + src/Storages/StorageInMemoryMetadata.cpp | 2 + src/Storages/StorageInMemoryMetadata.h | 2 + src/Storages/TTLDescription.h | 7 + .../02352_lightweight_delete.reference | 8 + .../0_stateless/02352_lightweight_delete.sql | 31 +- 20 files changed, 256 insertions(+), 303 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4e1024a07df..cb932843fc2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -459,7 +459,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \ M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ M(Bool, allow_experimental_lightweight_delete, false, "Enable lightweight DELETE mutations for mergetree tables. Work in progress", 0) \ - M(Bool, allow_experimental_lwd2, false, "Enable lightweight DELETE mutations using __rows_exists column for mergetree tables. Work in progress", 0) \ + M(Bool, allow_experimental_lightweight_delete_with_row_exists, false, "Enable lightweight DELETE mutations using __rows_exists column for mergetree tables. Work in progress", 0) \ M(Bool, lightweight_delete_mutation, true, "Enable to make ordinary ALTER DELETE queries lightweight for mergetree tables", 0) \ M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 33662b94614..0b7fdbd264c 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -33,8 +33,11 @@ InterpreterDeleteQuery::InterpreterDeleteQuery(const ASTPtr & query_ptr_, Contex BlockIO InterpreterDeleteQuery::execute() { - if (!getContext()->getSettingsRef().allow_experimental_lightweight_delete && !getContext()->getSettingsRef().allow_experimental_lwd2) + if (!getContext()->getSettingsRef().allow_experimental_lightweight_delete && + !getContext()->getSettingsRef().allow_experimental_lightweight_delete_with_row_exists) + { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Lightweight delete mutate is experimental. Set `allow_experimental_lightweight_delete` setting to enable it"); + } FunctionNameNormalizer().visit(query_ptr.get()); const ASTDeleteQuery & delete_query = query_ptr->as(); @@ -70,9 +73,9 @@ BlockIO InterpreterDeleteQuery::execute() MutationCommands mutation_commands; MutationCommand mut_command; - if (getContext()->getSettingsRef().allow_experimental_lwd2) + if (getContext()->getSettingsRef().allow_experimental_lightweight_delete_with_row_exists) { - /// UPDATE __row_exists = 0 WHERE predicate + /// Build "UPDATE __row_exists = 0 WHERE predicate" query mut_command.type = MutationCommand::Type::UPDATE; mut_command.predicate = delete_query.predicate; @@ -80,14 +83,14 @@ BlockIO InterpreterDeleteQuery::execute() command->type = ASTAlterCommand::UPDATE; command->predicate = delete_query.predicate; command->update_assignments = std::make_shared(); - auto set_row_exists = std::make_shared(); - set_row_exists->column_name = "__row_exists"; + auto set_row_does_not_exist = std::make_shared(); + set_row_does_not_exist->column_name = metadata_snapshot->lightweight_delete_description.filter_column.name; auto zero_value = std::make_shared(DB::Field(UInt8(0))); - set_row_exists->children.push_back(zero_value); - command->update_assignments->children.push_back(set_row_exists); + set_row_does_not_exist->children.push_back(zero_value); + command->update_assignments->children.push_back(set_row_does_not_exist); command->children.push_back(command->predicate); command->children.push_back(command->update_assignments); - mut_command.column_to_update_expression[set_row_exists->column_name] = zero_value; + mut_command.column_to_update_expression[set_row_does_not_exist->column_name] = zero_value; mut_command.ast = command->ptr(); mutation_commands.emplace_back(mut_command); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 21160ac5dd8..ffe80a9502a 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -29,7 +29,6 @@ #include #include - namespace DB { @@ -297,9 +296,7 @@ MutationsInterpreter::MutationsInterpreter( , is_lightweight(is_lightweight_) { if (is_lightweight) - { mutation_ast = prepareLightweightDelete(!can_execute); - } else mutation_ast = prepare(!can_execute); } @@ -356,7 +353,11 @@ static void validateUpdateColumns( } } - if (!found && column_name != "__row_exists") /// TODO: properly handle updating __row_exists column for LWD + /// Allow to override values of virtual columns + if (!found && column_name == metadata_snapshot->lightweight_delete_description.filter_column.name) + found = true; + + if (!found) { for (const auto & col : metadata_snapshot->getColumns().getMaterialized()) { @@ -509,7 +510,14 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) /// /// Outer CAST is added just in case if we don't trust the returning type of 'if'. - const auto type = (column == "__row_exists" ? std::make_shared() : columns_desc.getPhysical(column).type); + DataTypePtr type; + if (auto physical_column = columns_desc.tryGetPhysical(column)) + type = physical_column->type; + else if (column == metadata_snapshot->lightweight_delete_description.filter_column.name) + type = metadata_snapshot->lightweight_delete_description.filter_column.type; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown column {}", column); + auto type_literal = std::make_shared(type->getName()); const auto & update_expr = kv.second; diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 845aae52582..03053eb9b18 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -785,7 +785,8 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada /// Drop alias is metadata alter, in other case mutation is required. if (type == DROP_COLUMN) - return metadata.columns.hasColumnOrNested(GetColumnsOptions::AllPhysical, column_name); + return metadata.columns.hasColumnOrNested(GetColumnsOptions::AllPhysical, column_name) || + column_name == metadata.lightweight_delete_description.filter_column.name; if (type != MODIFY_COLUMN || data_type == nullptr) return false; @@ -1149,7 +1150,9 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const } else if (command.type == AlterCommand::DROP_COLUMN) { - if (all_columns.has(command.column_name) || all_columns.hasNested(command.column_name)) + if (all_columns.has(command.column_name) || + all_columns.hasNested(command.column_name) || + (command.clear && column_name == metadata.lightweight_delete_description.filter_column.name)) { if (!command.clear) /// CLEAR column is Ok even if there are dependencies. { diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index cc93e17bda3..65c9523f861 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -813,25 +813,25 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() { auto columns = global_ctx->merging_column_names; - if (part->getColumns().contains("__row_exists")) - columns.emplace_back("__row_exists"); - + /// The part might have some rows masked by lightweight deletes + const auto lwd_filter_column = global_ctx->metadata_snapshot->lightweight_delete_description.filter_column.name; + const bool need_to_filter_deleted_rows = !lwd_filter_column.empty() && part->getColumns().contains(lwd_filter_column); + if (need_to_filter_deleted_rows) + columns.emplace_back(lwd_filter_column); auto input = std::make_unique( *global_ctx->data, global_ctx->storage_snapshot, part, columns, ctx->read_with_direct_io, true); Pipe pipe(std::move(input)); - -///////////// - if (part->getColumns().contains("__row_exists")) + /// Add filtering step that discards deleted rows + if (need_to_filter_deleted_rows) { - pipe.addSimpleTransform([](const Block & header) + pipe.addSimpleTransform([lwd_filter_column](const Block & header) { - return std::make_shared(header, nullptr, "__row_exists", "__row_exists"); + return std::make_shared(header, nullptr, lwd_filter_column, true); }); } -///////////// if (global_ctx->metadata_snapshot->hasSortingKey()) { diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 933f9144c6a..ce48a03ce8b 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -208,16 +208,58 @@ Chunk MergeTreeBaseSelectProcessor::generate() auto res = readFromPart(); - if (res.hasRows()) + if (res.row_count) { - injectVirtualColumns(res, task.get(), partition_value_type, virt_column_names); - return res; + injectVirtualColumns(res.block, res.row_count, task.get(), partition_value_type, virt_column_names); + + /// Reorder the columns according to output header + const auto & output_header = output.getHeader(); + Columns ordered_columns; + ordered_columns.reserve(output_header.columns()); + for (size_t i = 0; i < output_header.columns(); ++i) + { + auto name = output_header.getByPosition(i).name; + ordered_columns.push_back(res.block.getByName(name).column); + } + + return Chunk(ordered_columns, res.row_count); } } return {}; } +void MergeTreeBaseSelectProcessor::initializeMergeTreeReadersForPart( + MergeTreeData::DataPartPtr & data_part, + const MergeTreeReadTaskColumns & task_columns, const StorageMetadataPtr & metadata_snapshot, + const MarkRanges & mark_ranges, const IMergeTreeReader::ValueSizeMap & value_size_map, + const ReadBufferFromFileBase::ProfileCallback & profile_callback) +{ + reader = data_part->getReader(task_columns.columns, metadata_snapshot, mark_ranges, + owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, + value_size_map, profile_callback); + + pre_reader_for_step.clear(); + + /// Add lightweight delete filtering step + const auto & lightweigth_delete_info = metadata_snapshot->lightweight_delete_description; + if (!reader_settings.skip_deleted_mask && data_part->getColumns().contains(lightweigth_delete_info.filter_column.name)) + { + pre_reader_for_step.push_back(data_part->getReader({lightweigth_delete_info.filter_column}, metadata_snapshot, mark_ranges, + owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, + value_size_map, profile_callback)); + } + + if (prewhere_info) + { + for (const auto & pre_columns_per_step : task_columns.pre_columns) + { + pre_reader_for_step.push_back(data_part->getReader(pre_columns_per_step, metadata_snapshot, mark_ranges, + owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, + value_size_map, profile_callback)); + } + } +} void MergeTreeBaseSelectProcessor::initializeRangeReaders(MergeTreeReadTask & current_task) { @@ -225,9 +267,10 @@ void MergeTreeBaseSelectProcessor::initializeRangeReaders(MergeTreeReadTask & cu bool last_reader = false; size_t pre_readers_shift = 0; - if (!reader_settings.skip_deleted_mask && current_task.data_part->getColumns().contains("__row_exists")) + /// Add filtering step with lightweight delete mask + const auto & lightweigth_delete_info = storage_snapshot->metadata->lightweight_delete_description; + if (!reader_settings.skip_deleted_mask && current_task.data_part->getColumns().contains(lightweigth_delete_info.filter_column.name)) { -// last_reader = !prewhere_actions || prewhere_actions->steps.empty(); current_task.pre_range_readers.push_back( MergeTreeRangeReader(pre_reader_for_step[0].get(), prev_reader, &lwd_filter_step, last_reader, non_const_virtual_column_names)); prev_reader = ¤t_task.pre_range_readers.back(); @@ -241,7 +284,6 @@ void MergeTreeBaseSelectProcessor::initializeRangeReaders(MergeTreeReadTask & cu "PREWHERE steps count mismatch, actions: {}, readers: {}", prewhere_actions->steps.size(), pre_reader_for_step.size()); - for (size_t i = 0; i < prewhere_actions->steps.size(); ++i) { last_reader = reader->getColumns().empty() && (i + 1 == prewhere_actions->steps.size()); @@ -304,7 +346,7 @@ static UInt64 estimateNumRows(const MergeTreeReadTask & current_task, UInt64 cur } -Chunk MergeTreeBaseSelectProcessor::readFromPartImpl() +MergeTreeBaseSelectProcessor::BlockAndRowCount MergeTreeBaseSelectProcessor::readFromPartImpl() { if (task->size_predictor) task->size_predictor->startBlock(); @@ -347,24 +389,13 @@ Chunk MergeTreeBaseSelectProcessor::readFromPartImpl() if (read_result.num_rows == 0) return {}; - Columns ordered_columns; - ordered_columns.reserve(header_without_virtual_columns.columns()); + BlockAndRowCount res = { sample_block.cloneWithColumns(read_result.columns), read_result.num_rows }; - /// Reorder columns. TODO: maybe skip for default case. - for (size_t ps = 0; ps < header_without_virtual_columns.columns(); ++ps) - { - const auto & name = header_without_virtual_columns.getByPosition(ps).name; - if (name == "__row_exists" && !sample_block.has(name)) - continue; /// TODO: properly deal with cases when __row_exists is not read and is filled later - auto pos_in_sample_block = sample_block.getPositionByName(name); - ordered_columns.emplace_back(std::move(read_result.columns[pos_in_sample_block])); - } - - return Chunk(std::move(ordered_columns), read_result.num_rows); + return res; } -Chunk MergeTreeBaseSelectProcessor::readFromPart() +MergeTreeBaseSelectProcessor::BlockAndRowCount MergeTreeBaseSelectProcessor::readFromPart() { if (!task->range_reader.isInitialized()) initializeRangeReaders(*task); @@ -375,22 +406,46 @@ Chunk MergeTreeBaseSelectProcessor::readFromPart() namespace { - /// Simple interfaces to insert virtual columns. struct VirtualColumnsInserter { - virtual ~VirtualColumnsInserter() = default; + explicit VirtualColumnsInserter(Block & block_) : block(block_) {} - virtual void insertArrayOfStringsColumn(const ColumnPtr & column, const String & name) = 0; - virtual void insertStringColumn(const ColumnPtr & column, const String & name) = 0; - virtual void insertUInt8Column(const ColumnPtr & column, const String & name) = 0; - virtual void insertUInt64Column(const ColumnPtr & column, const String & name) = 0; - virtual void insertUUIDColumn(const ColumnPtr & column, const String & name) = 0; + bool columnExists(const String & name) const { return block.has(name); } - virtual void insertPartitionValueColumn( - size_t rows, - const Row & partition_value, - const DataTypePtr & partition_value_type, - const String & name) = 0; + void insertStringColumn(const ColumnPtr & column, const String & name) + { + block.insert({column, std::make_shared(), name}); + } + + void insertUInt8Column(const ColumnPtr & column, const String & name) + { + block.insert({column, std::make_shared(), name}); + } + + void insertUInt64Column(const ColumnPtr & column, const String & name) + { + block.insert({column, std::make_shared(), name}); + } + + void insertUUIDColumn(const ColumnPtr & column, const String & name) + { + block.insert({column, std::make_shared(), name}); + } + + void insertPartitionValueColumn( + size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String & name) + { + ColumnPtr column; + if (rows) + column = partition_value_type->createColumnConst(rows, Tuple(partition_value.begin(), partition_value.end())) + ->convertToFullColumnIfConst(); + else + column = partition_value_type->createColumn(); + + block.insert({column, partition_value_type, name}); + } + + Block & block; }; } @@ -400,16 +455,34 @@ static void injectNonConstVirtualColumns( VirtualColumnsInserter & inserter, const Names & virtual_columns) { - if (unlikely(rows)) - throw Exception("Cannot insert non-constant virtual column to non-empty chunk.", - ErrorCodes::LOGICAL_ERROR); - for (const auto & virtual_column_name : virtual_columns) { if (virtual_column_name == "_part_offset") - inserter.insertUInt64Column(DataTypeUInt64().createColumn(), virtual_column_name); + { + if (!rows) + { + inserter.insertUInt64Column(DataTypeUInt64().createColumn(), virtual_column_name); + } + else + { + if (!inserter.columnExists(virtual_column_name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Column {} must have been filled part reader", + virtual_column_name); + } + } + if (virtual_column_name == "__row_exists") - inserter.insertUInt8Column(DataTypeUInt8().createColumn(), virtual_column_name); + { + /// If __row_exists column isn't present in the part then fill it here with 1s + ColumnPtr column; + if (rows) + column = DataTypeUInt8().createColumnConst(rows, 1)->convertToFullColumnIfConst(); + else + column = DataTypeUInt8().createColumn(); + + inserter.insertUInt8Column(column, virtual_column_name); + } } } @@ -489,148 +562,15 @@ static void injectPartConstVirtualColumns( } } -namespace -{ - struct VirtualColumnsInserterIntoBlock : public VirtualColumnsInserter - { - explicit VirtualColumnsInserterIntoBlock(Block & block_) : block(block_) {} - - void insertArrayOfStringsColumn(const ColumnPtr & column, const String & name) final - { - block.insert({column, std::make_shared(std::make_shared()), name}); - } - - void insertStringColumn(const ColumnPtr & column, const String & name) final - { - block.insert({column, std::make_shared(), name}); - } - - void insertUInt8Column(const ColumnPtr & column, const String & name) final - { - block.insert({column, std::make_shared(), name}); - } - - void insertUInt64Column(const ColumnPtr & column, const String & name) final - { - block.insert({column, std::make_shared(), name}); - } - - void insertUUIDColumn(const ColumnPtr & column, const String & name) final - { - block.insert({column, std::make_shared(), name}); - } - - void insertPartitionValueColumn( - size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String & name) final - { - ColumnPtr column; - if (rows) - column = partition_value_type->createColumnConst(rows, Tuple(partition_value.begin(), partition_value.end())) - ->convertToFullColumnIfConst(); - else - column = partition_value_type->createColumn(); - - block.insert({column, partition_value_type, name}); - } - - Block & block; - }; - - struct VirtualColumnsInserterIntoColumns : public VirtualColumnsInserter - { - explicit VirtualColumnsInserterIntoColumns(Columns & columns_) : columns(columns_) {} - - void insertArrayOfStringsColumn(const ColumnPtr & column, const String &) final - { - columns.push_back(column); - } - - void insertStringColumn(const ColumnPtr & column, const String &) final - { - columns.push_back(column); - } - - void insertUInt8Column(const ColumnPtr & column, const String &) final - { - columns.push_back(column); - } - - void insertUInt64Column(const ColumnPtr & column, const String &) final - { - columns.push_back(column); - } - - void insertUUIDColumn(const ColumnPtr & column, const String &) final - { - columns.push_back(column); - } - - void insertPartitionValueColumn( - size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String &) final - { - ColumnPtr column; - if (rows) - column = partition_value_type->createColumnConst(rows, Tuple(partition_value.begin(), partition_value.end())) - ->convertToFullColumnIfConst(); - else - column = partition_value_type->createColumn(); - columns.push_back(column); - } - - Columns & columns; - }; -} - void MergeTreeBaseSelectProcessor::injectVirtualColumns( - Block & block, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns) + Block & block, size_t row_count, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns) { - VirtualColumnsInserterIntoBlock inserter{block}; + VirtualColumnsInserter inserter{block}; /// First add non-const columns that are filled by the range reader and then const columns that we will fill ourselves. /// Note that the order is important: virtual columns filled by the range reader must go first - injectNonConstVirtualColumns(block.rows(), inserter, virtual_columns); - injectPartConstVirtualColumns(block.rows(), inserter, task, partition_value_type, virtual_columns); -} - -void MergeTreeBaseSelectProcessor::injectVirtualColumns( - Chunk & chunk, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns) -{ - UInt64 num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - - VirtualColumnsInserterIntoColumns inserter{columns}; - -///////////////////////// -// TODO: implement properly - for (const auto & virtual_column_name : virtual_columns) - { - - if (virtual_column_name == "__row_exists") - { - if (task->data_part->getColumns().contains(virtual_column_name)) - { - /// If this column is present in the part it must be read from the data - assert(task->task_columns.columns.contains(virtual_column_name)); - } - else - { - /// If __row_exists column isn't present in the part then - ColumnPtr column; - if (num_rows) - column = DataTypeUInt8().createColumnConst(num_rows, 1)->convertToFullColumnIfConst(); - else - column = DataTypeUInt8().createColumn(); - - inserter.insertUInt8Column(column, virtual_column_name); - } - } - } -/////////////////////////// - - /// Only add const virtual columns because non-const ones have already been added - injectPartConstVirtualColumns(num_rows, inserter, task, partition_value_type, virtual_columns); - - chunk.setColumns(columns, num_rows); + injectNonConstVirtualColumns(row_count, inserter, virtual_columns); + injectPartConstVirtualColumns(row_count, inserter, task, partition_value_type, virtual_columns); } Block MergeTreeBaseSelectProcessor::transformHeader( @@ -676,7 +616,7 @@ Block MergeTreeBaseSelectProcessor::transformHeader( } } - injectVirtualColumns(block, nullptr, partition_value_type, virtual_columns); + injectVirtualColumns(block, 0, nullptr, partition_value_type, virtual_columns); return block; } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 15a088d115c..299feed5a49 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -57,6 +58,12 @@ public: const Block & sample_block); protected: + /// This struct allow to return block with no columns but with non-zero number of rows similar to Chunk + struct BlockAndRowCount + { + Block block; + size_t row_count = 0; + }; Chunk generate() final; @@ -74,22 +81,29 @@ protected: /// Closes readers and unlock part locks virtual void finish() = 0; - virtual Chunk readFromPart(); + virtual BlockAndRowCount readFromPart(); - Chunk readFromPartImpl(); + BlockAndRowCount readFromPartImpl(); - /// Two versions for header and chunk. + /// Used for filling header with no rows as well as block with data static void - injectVirtualColumns(Block & block, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns); - static void - injectVirtualColumns(Chunk & chunk, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns); + injectVirtualColumns(Block & block, size_t row_count, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns); + /// Sets up data readers for each step of prewhere and where + void initializeMergeTreeReadersForPart( + MergeTreeData::DataPartPtr & data_part, + const MergeTreeReadTaskColumns & task_columns, const StorageMetadataPtr & metadata_snapshot, + const MarkRanges & mark_ranges, const IMergeTreeReader::ValueSizeMap & value_size_map, + const ReadBufferFromFileBase::ProfileCallback & profile_callback); + + /// Sets up range readers corresponding to data readers void initializeRangeReaders(MergeTreeReadTask & task); const MergeTreeData & storage; StorageSnapshotPtr storage_snapshot; - PrewhereExprStep lwd_filter_step { nullptr, "__row_exists", true, true }; + /// This step is added when the part has lightweight delete mask + const PrewhereExprStep lwd_filter_step { nullptr, "__row_exists", true, true }; PrewhereInfoPtr prewhere_info; std::unique_ptr prewhere_actions; diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index cf90da36ace..867d43e20b3 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -1002,6 +1002,7 @@ void MergeTreeRangeReader::fillPartOffsetColumn(ReadResult & result, UInt64 lead result.columns.emplace_back(std::move(column)); result.extra_columns_filled.push_back("_part_offset"); } + /// Fill deleted_row_mask column, referenced from fillPartOffsetColumn(). void MergeTreeRangeReader::fillDeletedRowMaskColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset) { diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 8f063786cbc..ba71f1898f6 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -161,9 +161,6 @@ public: /// The number of bytes read from disk. size_t numBytesRead() const { return num_bytes_read; } - /// Similar as filter that you need to apply to newly-read columns - ColumnPtr deleted_mask_filter_holder; - private: /// Only MergeTreeRangeReader is supposed to access ReadResult internals. friend class MergeTreeRangeReader; @@ -245,6 +242,9 @@ public: std::map filter_bytes_map; + /// Similar as filter that you need to apply to newly-read columns + ColumnPtr deleted_mask_filter_holder; + Names extra_columns_filled; }; diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp index 84548d357b7..c009e6f1165 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp @@ -44,9 +44,9 @@ catch (...) throw; } -Chunk MergeTreeReverseSelectProcessor::readFromPart() +MergeTreeBaseSelectProcessor::BlockAndRowCount MergeTreeReverseSelectProcessor::readFromPart() { - Chunk res; + BlockAndRowCount res; if (!chunks.empty()) { @@ -60,7 +60,7 @@ Chunk MergeTreeReverseSelectProcessor::readFromPart() while (!task->isFinished()) { - Chunk chunk = readFromPartImpl(); + auto chunk = readFromPartImpl(); chunks.push_back(std::move(chunk)); } diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h index 38dcc1a2352..06a218abafa 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h @@ -27,9 +27,9 @@ private: bool getNewTaskImpl() override; void finalizeNewTask() override {} - Chunk readFromPart() override; + BlockAndRowCount readFromPart() override; - Chunks chunks; + std::vector chunks; Poco::Logger * log = &Poco::Logger::get("MergeTreeReverseSelectProcessor"); }; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 47bcf72d611..3e346df6662 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -52,7 +52,7 @@ void MergeTreeSelectProcessor::initializeReaders() { task_columns = getReadTaskColumns( storage, storage_snapshot, data_part, - required_columns, non_const_virtual_column_names, prewhere_info, /*with_subcolumns=*/ true); + required_columns, virt_column_names, prewhere_info, /*with_subcolumns=*/ true); /// Will be used to distinguish between PREWHERE and WHERE columns when applying filter const auto & column_names = task_columns.columns.getNames(); @@ -63,25 +63,8 @@ void MergeTreeSelectProcessor::initializeReaders() owned_mark_cache = storage.getContext()->getMarkCache(); - reader = data_part->getReader(task_columns.columns, storage_snapshot->getMetadataForQuery(), - all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, {}, {}); - - pre_reader_for_step.clear(); - - if (!reader_settings.skip_deleted_mask && data_part->getColumns().contains("__row_exists")) - { - pre_reader_for_step.push_back(data_part->getReader({{"__row_exists", std::make_shared()}}, storage_snapshot->getMetadataForQuery(), - all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, {}, {})); - } - - if (prewhere_info) - { - for (const auto & pre_columns_for_step : task_columns.pre_columns) - { - pre_reader_for_step.push_back(data_part->getReader(pre_columns_for_step, storage_snapshot->getMetadataForQuery(), - all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, {}, {})); - } - } + initializeMergeTreeReadersForPart(data_part, task_columns, storage_snapshot->getMetadataForQuery(), + all_mark_ranges, {}, {}); } diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp index 4c0eac95593..04b7f6094e4 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp @@ -105,68 +105,24 @@ void MergeTreeThreadSelectProcessor::finalizeNewTask() auto profile_callback = [this](ReadBufferFromFileBase::ProfileInfo info_) { pool->profileFeedback(info_); }; const auto & metadata_snapshot = storage_snapshot->metadata; + IMergeTreeReader::ValueSizeMap value_size_map; + if (!reader) { if (use_uncompressed_cache) owned_uncompressed_cache = storage.getContext()->getUncompressedCache(); owned_mark_cache = storage.getContext()->getMarkCache(); - - reader = task->data_part->getReader(task->task_columns.columns, metadata_snapshot, task->mark_ranges, - owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, - IMergeTreeReader::ValueSizeMap{}, profile_callback); - - pre_reader_for_step.clear(); - - - if (!reader_settings.skip_deleted_mask && task->data_part->getColumns().contains("__row_exists")) - { - pre_reader_for_step.push_back(task->data_part->getReader({{"__row_exists", std::make_shared()}}, metadata_snapshot, task->mark_ranges, - owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, - IMergeTreeReader::ValueSizeMap{}, profile_callback)); - } - - - if (prewhere_info) - { - for (const auto & pre_columns_per_step : task->task_columns.pre_columns) - { - pre_reader_for_step.push_back(task->data_part->getReader(pre_columns_per_step, metadata_snapshot, task->mark_ranges, - owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, - IMergeTreeReader::ValueSizeMap{}, profile_callback)); - } - } } - else + else if (part_name != last_readed_part_name) { - /// in other case we can reuse readers, anyway they will be "seeked" to required mark - if (part_name != last_readed_part_name) - { - /// retain avg_value_size_hints - reader = task->data_part->getReader(task->task_columns.columns, metadata_snapshot, task->mark_ranges, - owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, - reader->getAvgValueSizeHints(), profile_callback); + value_size_map = reader->getAvgValueSizeHints(); + } - pre_reader_for_step.clear(); - - if (!reader_settings.skip_deleted_mask && task->data_part->getColumns().contains("__row_exists")) - { - pre_reader_for_step.push_back(task->data_part->getReader({{"__row_exists", std::make_shared()}}, metadata_snapshot, task->mark_ranges, - owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, - reader->getAvgValueSizeHints(), profile_callback)); - } - - - - if (prewhere_info) - { - for (const auto & pre_columns_per_step : task->task_columns.pre_columns) - { - pre_reader_for_step.push_back(task->data_part->getReader(pre_columns_per_step, metadata_snapshot, task->mark_ranges, - owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, - reader->getAvgValueSizeHints(), profile_callback)); - } - } - } + const bool init_new_readers = !reader || part_name != last_readed_part_name; + if (init_new_readers) + { + initializeMergeTreeReadersForPart(task->data_part, task->task_columns, metadata_snapshot, + task->mark_ranges, value_size_map, profile_callback); } last_readed_part_name = part_name; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b47f0cab6ab..ae64b08e351 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -170,6 +170,15 @@ getColumnsForNewDataPart( NameToNameMap renamed_columns_to_from; NameToNameMap renamed_columns_from_to; ColumnsDescription part_columns(source_part->getColumns()); + const auto all_virtual_columns = source_part->storage.getVirtuals(); + + /// Preserve virtual columns that have persisted values in the source_part +/// TODO: only allow LWD mask to be overriden!!!!! + for (const auto & virtual_column : all_virtual_columns) + { + if (part_columns.has(virtual_column.name) && !storage_columns.contains(virtual_column.name)) + storage_columns.emplace_back(virtual_column); + } /// All commands are validated in AlterCommand so we don't care about order for (const auto & command : commands_for_removes) @@ -178,8 +187,11 @@ getColumnsForNewDataPart( { for (const auto & [column_name, _] : command.column_to_update_expression) { - if (column_name == "__row_exists" && !storage_columns.contains(column_name)) - storage_columns.emplace_back("__row_exists", std::make_shared()); + /// Allow to update and persist values of virtual column +/// TODO: only allow LWD mask to be overriden!!!!! + auto virtual_column = all_virtual_columns.tryGetByName(column_name); + if (virtual_column && !storage_columns.contains(column_name)) + storage_columns.emplace_back(column_name, virtual_column->type); } } diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index e52a0fed674..9390514d299 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -10,6 +10,7 @@ #include #include #include +#include "DataTypes/DataTypesNumber.h" #include #include @@ -677,6 +678,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (arg_num != arg_cnt) throw Exception("Wrong number of engine arguments.", ErrorCodes::BAD_ARGUMENTS); + metadata.lightweight_delete_description.filter_column = { "__row_exists", std::make_shared() }; + if (replicated) { auto storage_policy = args.getContext()->getStoragePolicy(storage_settings->storage_policy); diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 66dcc938aef..de12467bdec 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -38,6 +38,7 @@ StorageInMemoryMetadata::StorageInMemoryMetadata(const StorageInMemoryMetadata & , sampling_key(other.sampling_key) , column_ttls_by_name(other.column_ttls_by_name) , table_ttl(other.table_ttl) + , lightweight_delete_description(other.lightweight_delete_description) , settings_changes(other.settings_changes ? other.settings_changes->clone() : nullptr) , select(other.select) , comment(other.comment) @@ -63,6 +64,7 @@ StorageInMemoryMetadata & StorageInMemoryMetadata::operator=(const StorageInMemo sampling_key = other.sampling_key; column_ttls_by_name = other.column_ttls_by_name; table_ttl = other.table_ttl; + lightweight_delete_description = other.lightweight_delete_description; if (other.settings_changes) settings_changes = other.settings_changes->clone(); else diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index a9ab96909f4..84a3bcb3046 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -43,6 +43,8 @@ struct StorageInMemoryMetadata TTLColumnsDescription column_ttls_by_name; /// TTL expressions for table (Move and Rows) TTLTableDescription table_ttl; + /// Lightweight delete filter column if the storage supports it. + LightweightDeleteDescription lightweight_delete_description; /// SETTINGS expression. Supported for MergeTree, Buffer, Kafka, RabbitMQ. ASTPtr settings_changes; /// SELECT QUERY. Supported for MaterializedView and View (have to support LiveView). diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index 8f60eb604b5..5170b7d326c 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -6,6 +6,8 @@ #include #include #include +#include "Core/NamesAndTypes.h" +#include "DataTypes/Serializations/ISerialization.h" namespace DB { @@ -127,4 +129,9 @@ struct TTLTableDescription static TTLTableDescription parse(const String & str, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key); }; +struct LightweightDeleteDescription +{ + NameAndTypePair filter_column; +}; + } diff --git a/tests/queries/0_stateless/02352_lightweight_delete.reference b/tests/queries/0_stateless/02352_lightweight_delete.reference index 8ae6b1d3195..02a34d7b82d 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete.reference +++ b/tests/queries/0_stateless/02352_lightweight_delete.reference @@ -13,3 +13,11 @@ Delete 3M more rows using light weight DELETE Rows in parts 7000000 Count 4000000 First row 6000000 10 +Do UPDATE mutation +Rows in parts 7000000 +Count 4000000 +First row 6000000 1 +Force merge to cleanup deleted rows +Rows in parts 4000000 +Count 4000000 +First row 6000000 1 diff --git a/tests/queries/0_stateless/02352_lightweight_delete.sql b/tests/queries/0_stateless/02352_lightweight_delete.sql index c5f636181cd..b9daf5e124d 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete.sql +++ b/tests/queries/0_stateless/02352_lightweight_delete.sql @@ -5,21 +5,20 @@ CREATE TABLE lwd_test (id UInt64 , value String) ENGINE MergeTree() ORDER BY id; INSERT INTO lwd_test SELECT number, randomString(10) FROM system.numbers LIMIT 10000000; SET mutations_sync = 1; +SET allow_experimental_lightweight_delete_with_row_exists = 1; +SET allow_experimental_lightweight_delete = 0; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; - SELECT 'Count', count() FROM lwd_test WHERE id >= 0; - SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; SELECT 'Delete 3M rows using UPDATE __row_exists'; -ALTER TABLE lwd_test UPDATE __row_exists = 0 WHERE id < 3000000; +--ALTER TABLE lwd_test UPDATE __row_exists = 0 WHERE id < 3000000; +DELETE FROM lwd_test WHERE id < 3000000; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; - SELECT 'Count', count() FROM lwd_test WHERE id >= 0; - SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; @@ -27,21 +26,33 @@ SELECT 'Force merge to cleanup deleted rows'; OPTIMIZE TABLE lwd_test FINAL; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; - SELECT 'Count', count() FROM lwd_test WHERE id >= 0; - SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; -SET allow_experimental_lwd2 = 1; SELECT 'Delete 3M more rows using light weight DELETE'; DELETE FROM lwd_test WHERE id < 6000000; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; - SELECT 'Count', count() FROM lwd_test WHERE id >= 0; - SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; +SELECT 'Do UPDATE mutation'; +ALTER TABLE lwd_test UPDATE value = 'v' WHERE id % 2 == 0; + +SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; +SELECT 'Count', count() FROM lwd_test WHERE id >= 0; +SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; + + +SELECT 'Force merge to cleanup deleted rows'; +OPTIMIZE TABLE lwd_test FINAL; + +SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; +SELECT 'Count', count() FROM lwd_test WHERE id >= 0; +SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; + + + DROP TABLE lwd_test; From ae0d00083c4870e5e735cc22775d7a552a92a529 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 18 Jul 2022 09:36:28 +0200 Subject: [PATCH 368/659] Renamed __row_exists to _row_exists --- src/Interpreters/InterpreterDeleteQuery.cpp | 2 +- src/Interpreters/MutationsInterpreter.cpp | 2 +- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 4 ++-- src/Storages/MergeTree/registerStorageMergeTree.cpp | 2 +- src/Storages/StorageDistributed.cpp | 2 +- .../queries/0_stateless/02352_lightweight_delete.reference | 4 ++-- tests/queries/0_stateless/02352_lightweight_delete.sql | 6 +++--- 11 files changed, 17 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 0b7fdbd264c..aeeb72ad06c 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -75,7 +75,7 @@ BlockIO InterpreterDeleteQuery::execute() if (getContext()->getSettingsRef().allow_experimental_lightweight_delete_with_row_exists) { - /// Build "UPDATE __row_exists = 0 WHERE predicate" query + /// Build "UPDATE _row_exists = 0 WHERE predicate" query mut_command.type = MutationCommand::Type::UPDATE; mut_command.predicate = delete_query.predicate; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index ffe80a9502a..f896c2269e8 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -353,7 +353,7 @@ static void validateUpdateColumns( } } - /// Allow to override values of virtual columns + /// Allow to override value of lightweight delete filter virtual column if (!found && column_name == metadata_snapshot->lightweight_delete_description.filter_column.name) found = true; diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index ce48a03ce8b..a5649c0b41b 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -61,7 +61,7 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( { non_const_virtual_column_names.emplace_back(*it); } - else if (*it == "__row_exists") + else if (*it == "_row_exists") { non_const_virtual_column_names.emplace_back(*it); } @@ -472,9 +472,9 @@ static void injectNonConstVirtualColumns( } } - if (virtual_column_name == "__row_exists") + if (virtual_column_name == "_row_exists") { - /// If __row_exists column isn't present in the part then fill it here with 1s + /// If _row_exists column isn't present in the part then fill it here with 1s ColumnPtr column; if (rows) column = DataTypeUInt8().createColumnConst(rows, 1)->convertToFullColumnIfConst(); diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 299feed5a49..51805fa83a2 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -103,7 +103,7 @@ protected: StorageSnapshotPtr storage_snapshot; /// This step is added when the part has lightweight delete mask - const PrewhereExprStep lwd_filter_step { nullptr, "__row_exists", true, true }; + const PrewhereExprStep lwd_filter_step { nullptr, "_row_exists", true, true }; PrewhereInfoPtr prewhere_info; std::unique_ptr prewhere_actions; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4df47eb7765..c9106b2f7b6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6562,7 +6562,7 @@ NamesAndTypesList MergeTreeData::getVirtuals() const NameAndTypePair("_partition_value", getPartitionValueType()), NameAndTypePair("_sample_factor", std::make_shared()), NameAndTypePair("_part_offset", std::make_shared()), - NameAndTypePair("__row_exists", std::make_shared()), + NameAndTypePair("_row_exists", std::make_shared()), }; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 36d45430cff..c78c187db8f 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1214,7 +1214,7 @@ static void selectColumnNames( { virt_column_names.push_back(name); } - else if (name == "__row_exists") + else if (name == "_row_exists") { virt_column_names.push_back(name); } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 867d43e20b3..cbc409af4e8 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -677,7 +677,7 @@ MergeTreeRangeReader::MergeTreeRangeReader( if (column_name == "_part_offset") sample_block.insert(ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), column_name)); -// if (column_name == "__row_exists") +// if (column_name == "_row_exists") // sample_block.insert(ColumnWithTypeAndName(ColumnUInt8::create(), std::make_shared(), column_name)); } @@ -1240,7 +1240,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r block.insert({result.columns[pos], std::make_shared(), column_name}); } - else if (column_name == "__row_exists") + else if (column_name == "_row_exists") { /// do nothing, it will be added later /// TODO: properly implement reading non-const virtual columns or filling them with default values diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 9390514d299..beeb980a786 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -678,7 +678,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (arg_num != arg_cnt) throw Exception("Wrong number of engine arguments.", ErrorCodes::BAD_ARGUMENTS); - metadata.lightweight_delete_description.filter_column = { "__row_exists", std::make_shared() }; + metadata.lightweight_delete_description.filter_column = { "_row_exists", std::make_shared() }; if (replicated) { diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 32e0fcffca6..b3ea2cb9f5b 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -312,7 +312,7 @@ NamesAndTypesList StorageDistributed::getVirtuals() const NameAndTypePair("_partition_id", std::make_shared()), NameAndTypePair("_sample_factor", std::make_shared()), NameAndTypePair("_part_offset", std::make_shared()), - NameAndTypePair("__row_exists", std::make_shared()), + NameAndTypePair("_row_exists", std::make_shared()), NameAndTypePair("_shard_num", std::make_shared()), /// deprecated }; } diff --git a/tests/queries/0_stateless/02352_lightweight_delete.reference b/tests/queries/0_stateless/02352_lightweight_delete.reference index 02a34d7b82d..8d7be361ba1 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete.reference +++ b/tests/queries/0_stateless/02352_lightweight_delete.reference @@ -1,7 +1,7 @@ Rows in parts 10000000 Count 10000000 First row 0 10 -Delete 3M rows using UPDATE __row_exists +Delete 3M rows using lightweight DELETE Rows in parts 10000000 Count 7000000 First row 3000000 10 @@ -9,7 +9,7 @@ Force merge to cleanup deleted rows Rows in parts 7000000 Count 7000000 First row 3000000 10 -Delete 3M more rows using light weight DELETE +Delete 3M more rows using lightweight DELETE Rows in parts 7000000 Count 4000000 First row 6000000 10 diff --git a/tests/queries/0_stateless/02352_lightweight_delete.sql b/tests/queries/0_stateless/02352_lightweight_delete.sql index b9daf5e124d..0eb30f260dd 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete.sql +++ b/tests/queries/0_stateless/02352_lightweight_delete.sql @@ -13,8 +13,8 @@ SELECT 'Count', count() FROM lwd_test WHERE id >= 0; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; -SELECT 'Delete 3M rows using UPDATE __row_exists'; ---ALTER TABLE lwd_test UPDATE __row_exists = 0 WHERE id < 3000000; +SELECT 'Delete 3M rows using lightweight DELETE'; +--ALTER TABLE lwd_test UPDATE _row_exists = 0 WHERE id < 3000000; DELETE FROM lwd_test WHERE id < 3000000; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; @@ -30,7 +30,7 @@ SELECT 'Count', count() FROM lwd_test WHERE id >= 0; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; -SELECT 'Delete 3M more rows using light weight DELETE'; +SELECT 'Delete 3M more rows using lightweight DELETE'; DELETE FROM lwd_test WHERE id < 6000000; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; From ab29e18b26fcf44874fc5f014fcd89970b8193f8 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 18 Jul 2022 12:11:31 +0200 Subject: [PATCH 369/659] Speedup 02352_lightweight_delete test --- .../02352_lightweight_delete.reference | 38 +++++++++---------- .../0_stateless/02352_lightweight_delete.sql | 10 ++--- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/tests/queries/0_stateless/02352_lightweight_delete.reference b/tests/queries/0_stateless/02352_lightweight_delete.reference index 8d7be361ba1..2c62a8cf5ea 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete.reference +++ b/tests/queries/0_stateless/02352_lightweight_delete.reference @@ -1,23 +1,23 @@ -Rows in parts 10000000 -Count 10000000 +Rows in parts 1000000 +Count 1000000 First row 0 10 -Delete 3M rows using lightweight DELETE -Rows in parts 10000000 -Count 7000000 -First row 3000000 10 +Delete 300K rows using lightweight DELETE +Rows in parts 1000000 +Count 700000 +First row 300000 10 Force merge to cleanup deleted rows -Rows in parts 7000000 -Count 7000000 -First row 3000000 10 -Delete 3M more rows using lightweight DELETE -Rows in parts 7000000 -Count 4000000 -First row 6000000 10 +Rows in parts 700000 +Count 700000 +First row 300000 10 +Delete 300K more rows using lightweight DELETE +Rows in parts 700000 +Count 400000 +First row 600000 10 Do UPDATE mutation -Rows in parts 7000000 -Count 4000000 -First row 6000000 1 +Rows in parts 700000 +Count 400000 +First row 600000 1 Force merge to cleanup deleted rows -Rows in parts 4000000 -Count 4000000 -First row 6000000 1 +Rows in parts 400000 +Count 400000 +First row 600000 1 diff --git a/tests/queries/0_stateless/02352_lightweight_delete.sql b/tests/queries/0_stateless/02352_lightweight_delete.sql index 0eb30f260dd..ff0d21136d3 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete.sql +++ b/tests/queries/0_stateless/02352_lightweight_delete.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS lwd_test; CREATE TABLE lwd_test (id UInt64 , value String) ENGINE MergeTree() ORDER BY id; -INSERT INTO lwd_test SELECT number, randomString(10) FROM system.numbers LIMIT 10000000; +INSERT INTO lwd_test SELECT number, randomString(10) FROM system.numbers LIMIT 1000000; SET mutations_sync = 1; SET allow_experimental_lightweight_delete_with_row_exists = 1; @@ -13,9 +13,9 @@ SELECT 'Count', count() FROM lwd_test WHERE id >= 0; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; -SELECT 'Delete 3M rows using lightweight DELETE'; +SELECT 'Delete 300K rows using lightweight DELETE'; --ALTER TABLE lwd_test UPDATE _row_exists = 0 WHERE id < 3000000; -DELETE FROM lwd_test WHERE id < 3000000; +DELETE FROM lwd_test WHERE id < 300000; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; SELECT 'Count', count() FROM lwd_test WHERE id >= 0; @@ -30,8 +30,8 @@ SELECT 'Count', count() FROM lwd_test WHERE id >= 0; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; -SELECT 'Delete 3M more rows using lightweight DELETE'; -DELETE FROM lwd_test WHERE id < 6000000; +SELECT 'Delete 300K more rows using lightweight DELETE'; +DELETE FROM lwd_test WHERE id < 600000; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; SELECT 'Count', count() FROM lwd_test WHERE id >= 0; From 614cb9a87f3f99e0a970da65cc10ad67fe5f387e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 18 Jul 2022 13:50:23 +0200 Subject: [PATCH 370/659] Disable trivial count() optimization if _row_exists column is present --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- .../queries/0_stateless/02352_lightweight_delete.sql | 12 ++++++------ 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c9106b2f7b6..0223561cdb6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1077,7 +1077,7 @@ void MergeTreeData::loadDataPartsFromDisk( has_adaptive_parts.store(true, std::memory_order_relaxed); /// Check if there is lightweight delete in part - if (part->hasLightweightDelete()) + if (part->hasLightweightDelete() || part->getColumns().contains("_row_exists")) // TODO: implement properly has_lightweight_in_parts.store(true, std::memory_order_relaxed); part->modification_time = part_disk_ptr->getLastModified(fs::path(relative_data_path) / part_name).epochTime(); @@ -2872,7 +2872,7 @@ bool MergeTreeData::renameTempPartAndReplaceImpl( throw Exception("MergeTreeData::Transaction for one table cannot be used with another. It is a bug.", ErrorCodes::LOGICAL_ERROR); - if (part->hasLightweightDelete()) + if (part->hasLightweightDelete() || part->getColumns().contains("_row_exists")) // TODO: implement properly has_lightweight_delete_parts.store(true); checkPartCanBeAddedToTable(part, lock); diff --git a/tests/queries/0_stateless/02352_lightweight_delete.sql b/tests/queries/0_stateless/02352_lightweight_delete.sql index ff0d21136d3..cc66898d749 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete.sql +++ b/tests/queries/0_stateless/02352_lightweight_delete.sql @@ -9,7 +9,7 @@ SET allow_experimental_lightweight_delete_with_row_exists = 1; SET allow_experimental_lightweight_delete = 0; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; -SELECT 'Count', count() FROM lwd_test WHERE id >= 0; +SELECT 'Count', count() FROM lwd_test; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; @@ -18,7 +18,7 @@ SELECT 'Delete 300K rows using lightweight DELETE'; DELETE FROM lwd_test WHERE id < 300000; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; -SELECT 'Count', count() FROM lwd_test WHERE id >= 0; +SELECT 'Count', count() FROM lwd_test; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; @@ -26,7 +26,7 @@ SELECT 'Force merge to cleanup deleted rows'; OPTIMIZE TABLE lwd_test FINAL; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; -SELECT 'Count', count() FROM lwd_test WHERE id >= 0; +SELECT 'Count', count() FROM lwd_test; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; @@ -34,7 +34,7 @@ SELECT 'Delete 300K more rows using lightweight DELETE'; DELETE FROM lwd_test WHERE id < 600000; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; -SELECT 'Count', count() FROM lwd_test WHERE id >= 0; +SELECT 'Count', count() FROM lwd_test; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; @@ -42,7 +42,7 @@ SELECT 'Do UPDATE mutation'; ALTER TABLE lwd_test UPDATE value = 'v' WHERE id % 2 == 0; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; -SELECT 'Count', count() FROM lwd_test WHERE id >= 0; +SELECT 'Count', count() FROM lwd_test; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; @@ -50,7 +50,7 @@ SELECT 'Force merge to cleanup deleted rows'; OPTIMIZE TABLE lwd_test FINAL; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; -SELECT 'Count', count() FROM lwd_test WHERE id >= 0; +SELECT 'Count', count() FROM lwd_test; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; From f31788ed2a769c994e6675d20c35cf5774e8c4a3 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 18 Jul 2022 16:04:07 +0200 Subject: [PATCH 371/659] Perf test for read after deleting many rows --- tests/performance/lightweight_delete.xml | 36 ++++++++++++++++++++++++ 1 file changed, 36 insertions(+) create mode 100644 tests/performance/lightweight_delete.xml diff --git a/tests/performance/lightweight_delete.xml b/tests/performance/lightweight_delete.xml new file mode 100644 index 00000000000..af7103f02d0 --- /dev/null +++ b/tests/performance/lightweight_delete.xml @@ -0,0 +1,36 @@ + + + + CREATE TABLE lwd_test ( + id UInt64, + value String, + ) ENGINE=MergeTree() ORDER BY id; + + + + + INSERT INTO lwd_test SELECT number, randomString(100) FROM system.numbers LIMIT 10000000; + + + + OPTIMIZE TABLE lwd_test FINAL; + + + + 1 + 1 + 0 + 1 + + + + + DELETE FROM lwd_test WHERE id < 9999999; + + + + SELECT id, length(value) FROM lwd_test ORDER BY id LIMIT 1 + + DROP TABLE lwd_test + + From 9dd370e2ec408202b82617b820eadac154493c04 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 18 Jul 2022 20:15:18 +0200 Subject: [PATCH 372/659] Fix typo --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a28af944868..e060535c1ae 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2099,7 +2099,7 @@ if __name__ == "__main__": group.add_argument( "--backward-compatibility-check", action="store_true", - help="Run tests for further backwoard compatibility testing by ignoring all" + help="Run tests for further backward compatibility testing by ignoring all" "drop queries in tests for collecting data from new version of server", ) parser.add_argument( From 332d3fd1a7e486d2c07a313b44d6944a0fd14e53 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 18 Jul 2022 14:40:49 -0400 Subject: [PATCH 373/659] add comments for doDecompressDataAsynchronous --- cmake/cpu_features.cmake | 2 +- src/Compression/CompressionCodecDeflateQpl.h | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index ad8d55e6c60..7b966e1acac 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -18,7 +18,7 @@ option (ENABLE_PCLMULQDQ "Use pclmulqdq instructions on x86_64" 1) option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1) option (ENABLE_AVX "Use AVX instructions on x86_64" 0) option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0) -option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 1) +option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 0) option (ENABLE_BMI "Use BMI instructions on x86_64" 0) option (ENABLE_AVX2_FOR_SPEC_OP "Use avx2 instructions for specific operations on x86_64" 0) option (ENABLE_AVX512_FOR_SPEC_OP "Use avx512 instructions for specific operations on x86_64" 0) diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index f64cf4c0c6c..40fc7a00a72 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -68,8 +68,12 @@ public: HardwareCodecDeflateQpl(); ~HardwareCodecDeflateQpl(); Int32 doCompressData(const char * source, UInt32 source_size, char * dest, UInt32 dest_size) const; + + ///Submit job request to the IAA hardware. IAA hardware will process decompression jobs automatically. Int32 doDecompressDataAsynchronous(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size); - /// Flush result for previous asynchronous decompression requests.Must be used following with several calls of doDecompressDataReq. + + /// Flush result for all previous requests which means busy waiting till all the jobs in "decomp_async_job_map" are finished. + /// Must be called subsequently after several calls of doDecompressDataReq. void flushAsynchronousDecompressRequests(); private: From 725ad5bf22233b623812c7d5e6da491a0bee3336 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 18 Jul 2022 20:49:30 +0200 Subject: [PATCH 374/659] Update cluster.py --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 1c4add88dee..5f1d1a32588 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -245,7 +245,7 @@ def extract_test_name(base_path): if name == "test.py": name = "" elif name.startswith("test_") and name.endswith(".py"): - name = name[len("test_"):(len(name)-len(".py"))] + name = name[len("test_") : (len(name) - len(".py"))] return name From 1513285166425eb726bdd491db4338c5b9d1be1c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 18 Jul 2022 20:54:13 +0200 Subject: [PATCH 375/659] Fix typo --- src/Formats/CapnProtoUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index add5220414f..f0d7ddf6fc3 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -52,7 +52,7 @@ capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaIn if (description.find("Parse error") != String::npos) throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, "Cannot parse CapnProto schema {}:{}", schema_info.schemaPath(), e.getLine()); - throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception while parsing CapnProro schema: {}, schema dir and file: {}, {}", description, schema_info.schemaDirectory(), schema_info.schemaPath()); + throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception while parsing CapnProto schema: {}, schema dir and file: {}, {}", description, schema_info.schemaDirectory(), schema_info.schemaPath()); } auto message_maybe = schema.findNested(schema_info.messageName()); From 6eb234a1cc76c0042262df15fb0b641250179ffc Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 18 Jul 2022 19:53:24 +0000 Subject: [PATCH 376/659] Avoid abort() in capnproto on exception descruction --- .../Formats/Impl/CapnProtoRowInputFormat.cpp | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index ad173e449d6..81e594e1153 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -264,20 +264,26 @@ bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension if (in->eof()) return false; - auto array = readMessage(); + try + { + auto array = readMessage(); #if CAPNP_VERSION >= 7000 && CAPNP_VERSION < 8000 - capnp::UnalignedFlatArrayMessageReader msg(array); + capnp::UnalignedFlatArrayMessageReader msg(array); #else - capnp::FlatArrayMessageReader msg(array); + capnp::FlatArrayMessageReader msg(array); #endif - auto root_reader = msg.getRoot(root); - - for (size_t i = 0; i != columns.size(); ++i) + auto root_reader = msg.getRoot(root); + for (size_t i = 0; i != columns.size(); ++i) + { + auto value = getReaderByColumnName(root_reader, column_names[i]); + insertValue(*columns[i], column_types[i], value, format_settings.capn_proto.enum_comparing_mode); + } + } + catch (const kj::Exception & e) { - auto value = getReaderByColumnName(root_reader, column_names[i]); - insertValue(*columns[i], column_types[i], value, format_settings.capn_proto.enum_comparing_mode); + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read row: {}", e.getDescription().cStr()); } return true; From 8b523bec167289e182f6cbb58a5ff94335241d22 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 18 Jul 2022 22:12:26 +0200 Subject: [PATCH 377/659] Addressed review comments - Updated destructor of ForkWriteBuffer to clear buffer --- src/IO/ForkWriteBuffer.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/IO/ForkWriteBuffer.cpp b/src/IO/ForkWriteBuffer.cpp index 876a924a72b..e50c8e2409c 100644 --- a/src/IO/ForkWriteBuffer.cpp +++ b/src/IO/ForkWriteBuffer.cpp @@ -46,6 +46,7 @@ void ForkWriteBuffer::nextImpl() ForkWriteBuffer::~ForkWriteBuffer() { finalize(); + set(nullptr, 0); } From d82f378a9d67d8e4782fb9aa7a222bdea6f3d1b7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 18 Jul 2022 23:37:07 +0200 Subject: [PATCH 378/659] do not enqueue uneeded parts for check --- src/Storages/MergeTree/DataPartsExchange.cpp | 9 +++++---- src/Storages/MergeTree/MergeTreeData.cpp | 4 +++- src/Storages/MergeTree/MergeTreeData.h | 7 +------ .../MergeTreeInOrderSelectProcessor.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 6 ++++++ .../MergeTree/ReplicatedMergeTreeQueue.h | 2 ++ src/Storages/StorageReplicatedMergeTree.cpp | 20 ++++++++++++++++--- 7 files changed, 35 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 9e18dbc6281..3609a65bc71 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -127,12 +127,13 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write { if (part && part->isProjectionPart()) { - data.reportBrokenPart(part->getParentPart()->name); + auto parent_part = part->getParentPart()->shared_from_this(); + data.reportBrokenPart(parent_part); } + else if (part) + data.reportBrokenPart(part); else - { - data.reportBrokenPart(part_name); - } + LOG_TRACE(log, "Part {} was not found, do not report it as broken", part_name); }; try diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3b39100b3de..727ebc9c3cc 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6031,8 +6031,10 @@ void MergeTreeData::reportBrokenPart(MergeTreeData::DataPartPtr & data_part) con broken_part_callback(part->name); } } - else + else if (data_part && data_part->getState() == IMergeTreeDataPart::State::Active) broken_part_callback(data_part->name); + else + LOG_DEBUG(log, "Will not check potentially broken part {} because it's not active", data_part->getNameWithState()); } MergeTreeData::MatcherFn MergeTreeData::getPartitionMatcher(const ASTPtr & partition_ast, ContextPtr local_context) const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 0b6e757ab49..26ac4d362ec 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -669,12 +669,7 @@ public: AlterLockHolder & table_lock_holder); /// Should be called if part data is suspected to be corrupted. - void reportBrokenPart(const String & name) const - { - broken_part_callback(name); - } - - /// Same as above but has the ability to check all other parts + /// Has the ability to check all other parts /// which reside on the same disk of the suspicious part. void reportBrokenPart(MergeTreeData::DataPartPtr & data_part) const; diff --git a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp index 280ce82cfce..655ca003deb 100644 --- a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp @@ -44,7 +44,7 @@ catch (...) { /// Suspicion of the broken part. A part is added to the queue for verification. if (getCurrentExceptionCode() != ErrorCodes::MEMORY_LIMIT_EXCEEDED) - storage.reportBrokenPart(data_part->name); + storage.reportBrokenPart(data_part); throw; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 7967726edca..2c32d9f266c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -2336,6 +2336,12 @@ bool ReplicatedMergeTreeMergePredicate::hasDropRange(const MergeTreePartInfo & n return queue.hasDropRange(new_drop_range_info); } +String ReplicatedMergeTreeMergePredicate::getCoveringVirtualPart(const String & part_name) const +{ + std::lock_guard lock(queue.state_mutex); + return queue.virtual_parts.getContainingPart(MergeTreePartInfo::fromPartName(part_name, queue.format_version)); +} + ReplicatedMergeTreeQueue::SubscriberHandler ReplicatedMergeTreeQueue::addSubscriber(ReplicatedMergeTreeQueue::SubscriberCallBack && callback) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index a88d9182bbf..a830815f760 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -521,6 +521,8 @@ public: bool hasDropRange(const MergeTreePartInfo & new_drop_range_info) const; + String getCoveringVirtualPart(const String & part_name) const; + private: const ReplicatedMergeTreeQueue & queue; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 520b5534fe3..c2ed93ca074 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1837,8 +1837,8 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) LOG_TRACE(log, "Executing DROP_RANGE {}", entry.new_part_name); auto drop_range_info = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version); getContext()->getMergeList().cancelInPartition(getStorageID(), drop_range_info.partition_id, drop_range_info.max_block); - part_check_thread.cancelRemovedPartsCheck(drop_range_info); queue.removePartProducingOpsInRange(getZooKeeper(), drop_range_info, entry); + part_check_thread.cancelRemovedPartsCheck(drop_range_info); /// Delete the parts contained in the range to be deleted. /// It's important that no old parts remain (after the merge), because otherwise, @@ -1906,8 +1906,8 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (replace) { getContext()->getMergeList().cancelInPartition(getStorageID(), drop_range.partition_id, drop_range.max_block); - part_check_thread.cancelRemovedPartsCheck(drop_range); queue.removePartProducingOpsInRange(getZooKeeper(), drop_range, entry); + part_check_thread.cancelRemovedPartsCheck(drop_range); } else { @@ -7953,12 +7953,26 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP while (true) { + auto pred = queue.getMergePredicate(zookeeper); + String covering_virtual = pred.getCoveringVirtualPart(lost_part_name); + if (covering_virtual.empty()) + { + LOG_WARNING(log, "Will not create empty part instead of lost {}, because there's no covering part in replication queue", lost_part_name); + return false; + } + if (pred.hasDropRange(MergeTreePartInfo::fromPartName(covering_virtual, format_version))) + { + LOG_WARNING(log, "Will not create empty part instead of lost {}, because it's covered by DROP_RANGE", lost_part_name); + return false; + } Coordination::Requests ops; Coordination::Stat replicas_stat; auto replicas_path = fs::path(zookeeper_path) / "replicas"; Strings replicas = zookeeper->getChildren(replicas_path, &replicas_stat); + ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/log", pred.getVersion())); + /// In rare cases new replica can appear during check ops.emplace_back(zkutil::makeCheckRequest(replicas_path, replicas_stat.version)); @@ -7988,7 +8002,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP } else if (code == Coordination::Error::ZBADVERSION) { - LOG_INFO(log, "Looks like new replica appearead while creating new empty part, will retry"); + LOG_INFO(log, "Looks like log was updated or new replica appeared while creating new empty part, will retry"); } else { From a53ed21dba57571bd3e8b7345ab3912fef125079 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 18 Jul 2022 17:44:15 -0400 Subject: [PATCH 379/659] restore the function doDecompressDataSynchronous --- .../CompressionCodecDeflateQpl.cpp | 42 +++++++++++++++++-- src/Compression/CompressionCodecDeflateQpl.h | 5 ++- 2 files changed, 42 insertions(+), 5 deletions(-) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index a0fd6b7757e..341b3db52cd 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -171,6 +171,42 @@ Int32 HardwareCodecDeflateQpl::doCompressData(const char * source, UInt32 source } } +Int32 HardwareCodecDeflateQpl::doDecompressDataSynchronous(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) +{ + UInt32 job_id = 0; + qpl_job * job_ptr = nullptr; + UInt32 decompressed_size = 0; + if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(job_id))) + { + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataSynchronous->acquireJob fail, probably job pool exhausted)"); + return RET_ERROR; + } + + // Performing a decompression operation + job_ptr->op = qpl_op_decompress; + job_ptr->next_in_ptr = reinterpret_cast(const_cast(source)); + job_ptr->next_out_ptr = reinterpret_cast(dest); + job_ptr->available_in = source_size; + job_ptr->available_out = uncompressed_size; + job_ptr->flags = QPL_FLAG_FIRST | QPL_FLAG_LAST; + + if (auto status = qpl_submit_job(job_ptr); status != QPL_STS_OK) + { + DeflateQplJobHWPool::instance().releaseJob(job_id); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataSynchronous->qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + return RET_ERROR; + } + /// Busy waiting till job complete. + do + { + _tpause(1, __rdtsc() + 1000); + } while (qpl_check_job(job_ptr) == QPL_STS_BEING_PROCESSED); + + decompressed_size = job_ptr->total_out; + DeflateQplJobHWPool::instance().releaseJob(job_id); + return decompressed_size; +} + Int32 HardwareCodecDeflateQpl::doDecompressDataAsynchronous(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) { UInt32 job_id = 0; @@ -337,10 +373,8 @@ void CompressionCodecDeflateQpl::doDecompressData(const char * source, UInt32 so Int32 res = HardwareCodecDeflateQpl::RET_ERROR; if (DeflateQplJobHWPool::instance().isJobPoolReady()) { - res = hw_codec->doDecompressDataAsynchronous(source, source_size, dest, uncompressed_size); - if(res != HardwareCodecDeflateQpl::RET_ERROR) - hw_codec->flushAsynchronousDecompressRequests(); - else + res = hw_codec->doDecompressDataSynchronous(source, source_size, dest, uncompressed_size); + if(res == HardwareCodecDeflateQpl::RET_ERROR) sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); } else diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 40fc7a00a72..125f35eb015 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -69,7 +69,10 @@ public: ~HardwareCodecDeflateQpl(); Int32 doCompressData(const char * source, UInt32 source_size, char * dest, UInt32 dest_size) const; - ///Submit job request to the IAA hardware. IAA hardware will process decompression jobs automatically. + ///Submit job request to the IAA hardware and then busy waiting till it complete. + Int32 doDecompressDataSynchronous(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size); + + ///Submit job request to the IAA hardware and return immediately. IAA hardware will process decompression jobs automatically. Int32 doDecompressDataAsynchronous(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size); /// Flush result for all previous requests which means busy waiting till all the jobs in "decomp_async_job_map" are finished. From e7af88b6887749f34f6409b39d73aeec66c97e11 Mon Sep 17 00:00:00 2001 From: jiahui-97 Date: Sun, 10 Jul 2022 19:39:45 +0800 Subject: [PATCH 380/659] implementation of parseTimeDelta function Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- .../functions/other-functions.md | 41 +++ src/Functions/parseTimeDelta.cpp | 291 ++++++++++++++++++ src/Functions/registerFunctionsFormatting.cpp | 2 + tests/fuzz/all.dict | 1 + tests/fuzz/dictionaries/functions.dict | 1 + .../queries/0_stateless/00534_filimonov.data | 1 + .../02354_parse_timedelta.reference | 7 + .../0_stateless/02354_parse_timedelta.sql | 21 ++ 8 files changed, 365 insertions(+) create mode 100644 src/Functions/parseTimeDelta.cpp create mode 100644 tests/queries/0_stateless/02354_parse_timedelta.reference create mode 100644 tests/queries/0_stateless/02354_parse_timedelta.sql diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 9e6f0effcf9..6b01ee31501 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -681,6 +681,47 @@ SELECT └────────────┴─────────────────────────────────────────────────────────────────┘ ``` +## parseTimeDelta + +Parse a sequence of numbers followed by something resembling a time unit. + +**Syntax** + +```sql +parseTimeDelta(timestr) +``` + +**Arguments** + +- `timestr` — A sequence of numbers followed by something resembling a time unit. + + +**Returned value** + +- A floating-point number with the number of seconds. + +**Example** + +```sql +SELECT parseTimeDelta('11s+22min') +``` + +```text +┌─parseTimeDelta('11s+22min')─┐ +│ 1331 │ +└─────────────────────────────┘ +``` + +```sql +SELECT parseTimeDelta('1yr2mo') +``` + +```text +┌─parseTimeDelta('1yr2mo')─┐ +│ 36806400 │ +└──────────────────────────┘ +``` + ## least(a, b) Returns the smallest value from a and b. diff --git a/src/Functions/parseTimeDelta.cpp b/src/Functions/parseTimeDelta.cpp new file mode 100644 index 00000000000..fb5a7621a53 --- /dev/null +++ b/src/Functions/parseTimeDelta.cpp @@ -0,0 +1,291 @@ +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int BAD_ARGUMENTS; +} + +namespace +{ + const std::unordered_map time_unit_to_float = { + {"years", 365 * 24 * 3600}, + {"year", 365 * 24 * 3600}, + {"yr", 365 * 24 * 3600}, + {"y", 365 * 24 * 3600}, + + {"months", 30.5 * 24 * 3600}, + {"month", 30.5 * 24 * 3600}, + {"mo", 30.5 * 24 * 3600}, + + {"weeks", 7 * 24 * 3600}, + {"week", 7 * 24 * 3600}, + {"w", 7 * 24 * 3600}, + + {"days", 24 * 3600}, + {"day", 24 * 3600}, + {"d", 24 * 3600}, + + {"hours", 3600}, + {"hour", 3600}, + {"hr", 3600}, + {"h", 3600}, + + {"minutes", 60}, + {"minute", 60}, + {"min", 60}, + {"m", 60}, + + {"seconds", 1}, + {"second", 1}, + {"sec", 1}, + {"s", 1}, + }; + + /** Prints amount of seconds in form of: + * "1 year 2 months 4 weeks 12 days 3 hours 1 minute 33 seconds". + * ' ', ';', '-', '+', ',', ':' can be used as separator, eg. "1yr-2mo", "2m:6s" + * + * valid expressions: + * SELECT parseTimeDelta('1 min 35 sec'); + * SELECT parseTimeDelta('0m;11.23s.'); + * SELECT parseTimeDelta('11hr 25min 3.1s'); + * SELECT parseTimeDelta('0.00123 seconds'); + * SELECT parseTimeDelta('1yr2mo'); + * SELECT parseTimeDelta('11s+22min'); + * SELECT parseTimeDelta('1yr-2mo-4w + 12 days, 3 hours : 1 minute ; 33 seconds'); + * + * invalid expressions: + * SELECT parseTimeDelta(); + * SELECT parseTimeDelta('1yr', 1); + * SELECT parseTimeDelta(1); + * SELECT parseTimeDelta(' '); + * SELECT parseTimeDelta('-1yr'); + * SELECT parseTimeDelta('1yr-'); + * SELECT parseTimeDelta('yr2mo'); + * SELECT parseTimeDelta('1.yr2mo'); + * SELECT parseTimeDelta('1-yr'); + * SELECT parseTimeDelta('1 1yr'); + * SELECT parseTimeDelta('1yyr'); + * SELECT parseTimeDelta('1yr-2mo-4w + 12 days, 3 hours : 1 minute ;. 33 seconds'); + * + * The length of years and months (and even days in presence of time adjustments) are rough: + * year is just 365 days, month is 30.5 days, day is 86400 seconds, similarly to what formatReadableTimeDelta is doing. + */ + class FunctionParseTimeDelta : public IFunction + { + public: + static constexpr auto name = "parseTimeDelta"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 1.", + getName(), + toString(arguments.size())); + + if (arguments.size() > 1) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 1.", + getName(), + toString(arguments.size())); + + const IDataType & type = *arguments[0]; + + if (!isString(type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot format {} as time string.", type.getName()); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + auto col_to = ColumnFloat64::create(); + auto & res_data = col_to->getData(); + + for (size_t i = 0; i < input_rows_count; ++i) + { + std::string_view str{arguments[0].column->getDataAt(i)}; + Int64 token_tail = 0; + Int64 token_front = 0; + Int64 last_pos = str.length() - 1; + Float64 result = 0; + + /// ignore '.' and ' ' at the end of string + while (last_pos >= 0 && (str[last_pos] == ' ' || str[last_pos] == '.')) + --last_pos; + + /// no valid characters + if (last_pos < 0) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Invalid expression for function {}, don't find valid characters, str: \"{}\".", + getName(), + String(str)); + } + + /// last pos character must be character and not be separator or number after ignoring '.' and ' ' + if (!isalpha(str[last_pos])) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid expression for function {}, str: \"{}\".", getName(), String(str)); + } + + /// scan spaces at the beginning + scanSpaces(str, token_tail, last_pos); + token_front = token_tail; + + while (token_tail <= last_pos) + { + /// scan unsigned integer + if (!scanUnsignedInteger(str, token_tail, last_pos)) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Invalid expression for function {}, find number failed, str: \"{}\".", + getName(), + String(str)); + } + + /// if there is a '.', then scan another integer to get a float number + if (token_tail <= last_pos && str[token_tail] == '.') + { + token_tail++; + if (!scanUnsignedInteger(str, token_tail, last_pos)) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Invalid expression for function {}, find number after '.' failed, str: \"{}\".", + getName(), + String(str)); + } + } + + /// convert float/integer string to float + Float64 base = 0; + std::string_view base_str = str.substr(token_front, token_tail - token_front); + auto value = boost::convert(base_str, boost::cnv::strtol()); + if (!value.has_value()) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Invalid expression for function {}, convert string to float64 failed: \"{}\".", + getName(), + String(base_str)); + } + base = value.get(); + + scanSpaces(str, token_tail, last_pos); + token_front = token_tail; + + /// scan a unit + if (!scanUnit(str, token_tail, last_pos)) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Invalid expression for function {}, find unit failed, str: \"{}\".", + getName(), + String(str)); + } + + /// get unit number + std::string_view unit = str.substr(token_front, token_tail - token_front); + auto iter = time_unit_to_float.find(unit); + if (iter == time_unit_to_float.end()) /// not find unit + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Invalid expression for function {}, parse unit failed: \"{}\".", getName(), unit); + } + result += base * iter->second; + + /// scan separator between two tokens + scanSeparator(str, token_tail, last_pos); + token_front = token_tail; + } + + res_data.emplace_back(result); + } + + return col_to; + } + + /// scan an unsigned integer number + static bool scanUnsignedInteger(std::string_view & str, Int64 & index, Int64 last_pos) + { + int64_t begin_index = index; + while (index <= last_pos && isdigit(str[index])) + { + index++; + } + return index != begin_index; + } + + /// scan a unit + static bool scanUnit(std::string_view & str, Int64 & index, Int64 last_pos) + { + int64_t begin_index = index; + while (index <= last_pos && isalpha(str[index])) + { + index++; + } + return index != begin_index; + } + + /// scan spaces + static void scanSpaces(std::string_view & str, Int64 & index, Int64 last_pos) + { + while (index <= last_pos && (str[index] == ' ')) + { + index++; + } + } + + /// scan for characters to ignore + static void scanSeparator(std::string_view & str, Int64 & index, Int64 last_pos) + { + /// ignore spaces + scanSpaces(str, index, last_pos); + + /// ignore separator + if (index <= last_pos + && (str[index] == ';' || str[index] == '-' || str[index] == '+' || str[index] == ',' || str[index] == ':')) + { + index++; + } + + scanSpaces(str, index, last_pos); + } + }; + +} + +void registerFunctionParseTimeDelta(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/registerFunctionsFormatting.cpp b/src/Functions/registerFunctionsFormatting.cpp index e434b0e49f0..02353fac812 100644 --- a/src/Functions/registerFunctionsFormatting.cpp +++ b/src/Functions/registerFunctionsFormatting.cpp @@ -7,6 +7,7 @@ void registerFunctionsBitToArray(FunctionFactory &); void registerFunctionFormatReadableSize(FunctionFactory &); void registerFunctionFormatReadableQuantity(FunctionFactory &); void registerFunctionFormatReadableTimeDelta(FunctionFactory &); +void registerFunctionParseTimeDelta(FunctionFactory &); void registerFunctionsFormatting(FunctionFactory & factory) { @@ -14,6 +15,7 @@ void registerFunctionsFormatting(FunctionFactory & factory) registerFunctionFormatReadableSize(factory); registerFunctionFormatReadableQuantity(factory); registerFunctionFormatReadableTimeDelta(factory); + registerFunctionParseTimeDelta(factory); } } diff --git a/tests/fuzz/all.dict b/tests/fuzz/all.dict index 356428a0b86..dff62cd68a7 100644 --- a/tests/fuzz/all.dict +++ b/tests/fuzz/all.dict @@ -898,6 +898,7 @@ "parseDateTimeBestEffortUS" "parseDateTimeBestEffortUSOrNull" "parseDateTimeBestEffortUSOrZero" +"parseTimeDelta" "PARTITION" "PARTITION BY" "partitionId" diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index 1bdaed5ee1b..cbcad3c05da 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -68,6 +68,7 @@ "reinterpretAsUInt8" "atanh" "formatReadableTimeDelta" +"parseTimeDelta" "geohashEncode" "atan2" "acos" diff --git a/tests/queries/0_stateless/00534_filimonov.data b/tests/queries/0_stateless/00534_filimonov.data index f07d79ed1bf..911a8e4d1f3 100644 --- a/tests/queries/0_stateless/00534_filimonov.data +++ b/tests/queries/0_stateless/00534_filimonov.data @@ -176,6 +176,7 @@ SELECT toInt16OrZero(NULL); SELECT formatReadableSize(NULL); SELECT formatReadableQuantity(NULL); SELECT formatReadableTimeDelta(NULL); +SELECT parseTimeDelta(NULL); SELECT concatAssumeInjective(NULL); SELECT toString(NULL); SELECT MACStringToNum(NULL); diff --git a/tests/queries/0_stateless/02354_parse_timedelta.reference b/tests/queries/0_stateless/02354_parse_timedelta.reference new file mode 100644 index 00000000000..9bc208ada9d --- /dev/null +++ b/tests/queries/0_stateless/02354_parse_timedelta.reference @@ -0,0 +1,7 @@ +95 +11.23 +41103.1 +0.00123 +36806400 +1331 +40273293 diff --git a/tests/queries/0_stateless/02354_parse_timedelta.sql b/tests/queries/0_stateless/02354_parse_timedelta.sql new file mode 100644 index 00000000000..3cf282d715a --- /dev/null +++ b/tests/queries/0_stateless/02354_parse_timedelta.sql @@ -0,0 +1,21 @@ +SELECT parseTimeDelta('1 min 35 sec'); +SELECT parseTimeDelta('0m;11.23s.'); +SELECT parseTimeDelta('11hr 25min 3.1s'); +SELECT parseTimeDelta('0.00123 seconds'); +SELECT parseTimeDelta('1yr2mo'); +SELECT parseTimeDelta('11s+22min'); +SELECT parseTimeDelta('1yr-2mo-4w + 12 days, 3 hours : 1 minute ; 33 seconds'); + +-- invalid expressions +SELECT parseTimeDelta(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT parseTimeDelta('1yr', 1); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT parseTimeDelta(1); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT parseTimeDelta(' '); -- {serverError BAD_ARGUMENTS} +SELECT parseTimeDelta('-1yr'); -- {serverError BAD_ARGUMENTS} +SELECT parseTimeDelta('1yr-'); -- {serverError BAD_ARGUMENTS} +SELECT parseTimeDelta('yr2mo'); -- {serverError BAD_ARGUMENTS} +SELECT parseTimeDelta('1.yr2mo'); -- {serverError BAD_ARGUMENTS} +SELECT parseTimeDelta('1-yr'); -- {serverError BAD_ARGUMENTS} +SELECT parseTimeDelta('1 1yr'); -- {serverError BAD_ARGUMENTS} +SELECT parseTimeDelta('1yyr'); -- {serverError BAD_ARGUMENTS} +SELECT parseTimeDelta('1yr-2mo-4w + 12 days, 3 hours : 1 minute ;. 33 seconds'); -- {serverError BAD_ARGUMENTS} From 81ef1099ccb88f049383a851ce8e6418d19f946f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Jul 2022 17:31:34 +0000 Subject: [PATCH 381/659] Even less usage of StringRef --> see #39300 --- src/Common/TraceSender.cpp | 10 ++-- .../Serializations/SerializationEnum.cpp | 6 +- .../Serializations/SerializationString.cpp | 6 +- src/Functions/extractGroups.cpp | 6 +- src/Functions/formatReadableTimeDelta.cpp | 8 +-- src/Functions/geohashDecode.cpp | 4 +- src/Functions/isIPAddressContainedIn.cpp | 12 ++-- src/IO/Operators.h | 2 - src/IO/WriteHelpers.h | 57 +++---------------- src/IO/tests/gtest_manip.cpp | 5 -- src/Processors/Merges/Algorithms/Graphite.cpp | 8 +-- src/Processors/Merges/Algorithms/Graphite.h | 3 +- .../GraphiteRollupSortedAlgorithm.cpp | 4 +- .../GraphiteRollupSortedAlgorithm.h | 2 +- src/Storages/Distributed/DirectoryMonitor.cpp | 1 - .../graphite-rollup/graphite-rollup-bench.cpp | 14 ++--- 16 files changed, 50 insertions(+), 98 deletions(-) diff --git a/src/Common/TraceSender.cpp b/src/Common/TraceSender.cpp index ce8adb98740..f1287b11ac4 100644 --- a/src/Common/TraceSender.cpp +++ b/src/Common/TraceSender.cpp @@ -42,13 +42,13 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int char buffer[buf_size]; WriteBufferFromFileDescriptorDiscardOnFailure out(pipe.fds_rw[1], buf_size, buffer); - StringRef query_id; + std::string_view query_id; UInt64 thread_id; if (CurrentThread::isInitialized()) { - query_id = StringRef(CurrentThread::getQueryId()); - query_id.size = std::min(query_id.size, QUERY_ID_MAX_LEN); + query_id = CurrentThread::getQueryId(); + query_id = std::string_view(query_id.data(), std::min(query_id.size(), QUERY_ID_MAX_LEN)); thread_id = CurrentThread::get().thread_id; } @@ -59,8 +59,8 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int writeChar(false, out); /// true if requested to stop the collecting thread. - writeBinary(static_cast(query_id.size), out); - out.write(query_id.data, query_id.size); + writeBinary(static_cast(query_id.size()), out); + out.write(query_id.data(), query_id.size()); size_t stack_trace_size = stack_trace.getSize(); size_t stack_trace_offset = stack_trace.getOffset(); diff --git a/src/DataTypes/Serializations/SerializationEnum.cpp b/src/DataTypes/Serializations/SerializationEnum.cpp index 39e9885fe17..a1b9c8bf95a 100644 --- a/src/DataTypes/Serializations/SerializationEnum.cpp +++ b/src/DataTypes/Serializations/SerializationEnum.cpp @@ -18,7 +18,7 @@ void SerializationEnum::serializeText(const IColumn & column, size_t row_n template void SerializationEnum::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeEscapedString(this->getNameForValue(assert_cast(column).getData()[row_num]), ostr); + writeEscapedString(this->getNameForValue(assert_cast(column).getData()[row_num]).toView(), ostr); } template @@ -69,13 +69,13 @@ void SerializationEnum::deserializeWholeText(IColumn & column, ReadBuffer template void SerializationEnum::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - writeJSONString(this->getNameForValue(assert_cast(column).getData()[row_num]), ostr, settings); + writeJSONString(this->getNameForValue(assert_cast(column).getData()[row_num]).toView(), ostr, settings); } template void SerializationEnum::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeXMLStringForTextElement(this->getNameForValue(assert_cast(column).getData()[row_num]), ostr); + writeXMLStringForTextElement(this->getNameForValue(assert_cast(column).getData()[row_num]).toView(), ostr); } template diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 5614e970315..e07fd4f26cf 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -213,7 +213,7 @@ void SerializationString::serializeText(const IColumn & column, size_t row_num, void SerializationString::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeEscapedString(assert_cast(column).getDataAt(row_num), ostr); + writeEscapedString(assert_cast(column).getDataAt(row_num).toView(), ostr); } @@ -266,7 +266,7 @@ void SerializationString::deserializeTextQuoted(IColumn & column, ReadBuffer & i void SerializationString::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - writeJSONString(assert_cast(column).getDataAt(row_num), ostr, settings); + writeJSONString(assert_cast(column).getDataAt(row_num).toView(), ostr, settings); } @@ -278,7 +278,7 @@ void SerializationString::deserializeTextJSON(IColumn & column, ReadBuffer & ist void SerializationString::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeXMLStringForTextElement(assert_cast(column).getDataAt(row_num), ostr); + writeXMLStringForTextElement(assert_cast(column).getDataAt(row_num).toView(), ostr); } diff --git a/src/Functions/extractGroups.cpp b/src/Functions/extractGroups.cpp index 940e76df1c0..c6633732aaa 100644 --- a/src/Functions/extractGroups.cpp +++ b/src/Functions/extractGroups.cpp @@ -87,10 +87,10 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { - StringRef current_row = column_haystack->getDataAt(i); + std::string_view current_row = column_haystack->getDataAt(i).toView(); - if (re2->Match(re2_st::StringPiece(current_row.data, current_row.size), - 0, current_row.size, re2_st::RE2::UNANCHORED, matched_groups.data(), matched_groups.size())) + if (re2->Match(re2_st::StringPiece(current_row.data(), current_row.size()), + 0, current_row.size(), re2_st::RE2::UNANCHORED, matched_groups.data(), matched_groups.size())) { // 1 is to exclude group #0 which is whole re match. for (size_t group = 1; group <= groups_count; ++group) diff --git a/src/Functions/formatReadableTimeDelta.cpp b/src/Functions/formatReadableTimeDelta.cpp index d781d227c64..8efd13a734a 100644 --- a/src/Functions/formatReadableTimeDelta.cpp +++ b/src/Functions/formatReadableTimeDelta.cpp @@ -94,19 +94,19 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - StringRef maximum_unit_str; + std::string_view maximum_unit_str; if (arguments.size() == 2) { const ColumnPtr & maximum_unit_column = arguments[1].column; const ColumnConst * maximum_unit_const_col = checkAndGetColumnConstStringOrFixedString(maximum_unit_column.get()); if (maximum_unit_const_col) - maximum_unit_str = maximum_unit_const_col->getDataColumn().getDataAt(0); + maximum_unit_str = maximum_unit_const_col->getDataColumn().getDataAt(0).toView(); } Unit max_unit; /// Default means "use all available units". - if (maximum_unit_str.size == 0 || maximum_unit_str == "years") + if (maximum_unit_str.empty() || maximum_unit_str == "years") max_unit = Years; else if (maximum_unit_str == "months") max_unit = Months; @@ -122,7 +122,7 @@ public: throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected value of maximum unit argument ({}) for function {}, the only allowed values are:" " 'seconds', 'minutes', 'hours', 'days', 'months', 'years'.", - maximum_unit_str.toString(), getName()); + std::string(maximum_unit_str), getName()); auto col_to = ColumnString::create(); diff --git a/src/Functions/geohashDecode.cpp b/src/Functions/geohashDecode.cpp index 199d1a62f1d..b4e5d8e46e9 100644 --- a/src/Functions/geohashDecode.cpp +++ b/src/Functions/geohashDecode.cpp @@ -64,8 +64,8 @@ public: for (size_t i = 0; i < count; ++i) { - StringRef encoded_string = encoded->getDataAt(i); - geohashDecode(encoded_string.data, encoded_string.size, &lon_data[i], &lat_data[i]); + std::string_view encoded_string = encoded->getDataAt(i).toView(); + geohashDecode(encoded_string.data(), encoded_string.size(), &lon_data[i], &lat_data[i]); } MutableColumns result; diff --git a/src/Functions/isIPAddressContainedIn.cpp b/src/Functions/isIPAddressContainedIn.cpp index 1ba719cda63..a6f94c77ad1 100644 --- a/src/Functions/isIPAddressContainedIn.cpp +++ b/src/Functions/isIPAddressContainedIn.cpp @@ -27,7 +27,7 @@ class IPAddressVariant { public: - explicit IPAddressVariant(StringRef address_str) + explicit IPAddressVariant(std::string_view address_str) { /// IP address parser functions require that the input is /// NULL-terminated so we need to copy it. @@ -85,7 +85,7 @@ IPAddressCIDR parseIPWithCIDR(std::string_view cidr_str) throw DB::Exception("The text does not contain '/': " + std::string(cidr_str), DB::ErrorCodes::CANNOT_PARSE_TEXT); std::string_view addr_str = cidr_str.substr(0, pos_slash); - IPAddressVariant addr(StringRef{addr_str.data(), addr_str.size()}); + IPAddressVariant addr(addr_str); uint8_t prefix = 0; auto prefix_str = cidr_str.substr(pos_slash+1); @@ -188,7 +188,7 @@ namespace DB const auto & col_addr = col_addr_const.getDataColumn(); const auto & col_cidr = col_cidr_const.getDataColumn(); - const auto addr = IPAddressVariant(col_addr.getDataAt(0)); + const auto addr = IPAddressVariant(col_addr.getDataAt(0).toView()); const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(0).toView()); ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(1); @@ -204,7 +204,7 @@ namespace DB { const auto & col_addr = col_addr_const.getDataColumn(); - const auto addr = IPAddressVariant(col_addr.getDataAt (0)); + const auto addr = IPAddressVariant(col_addr.getDataAt(0).toView()); ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); ColumnUInt8::Container & vec_res = col_res->getData(); @@ -228,7 +228,7 @@ namespace DB ColumnUInt8::Container & vec_res = col_res->getData(); for (size_t i = 0; i < input_rows_count; ++i) { - const auto addr = IPAddressVariant(col_addr.getDataAt(i)); + const auto addr = IPAddressVariant(col_addr.getDataAt(i).toView()); vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; } return col_res; @@ -242,7 +242,7 @@ namespace DB for (size_t i = 0; i < input_rows_count; ++i) { - const auto addr = IPAddressVariant(col_addr.getDataAt(i)); + const auto addr = IPAddressVariant(col_addr.getDataAt(i).toView()); const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i).toView()); vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; diff --git a/src/IO/Operators.h b/src/IO/Operators.h index 114ab692dc3..93807ea39fc 100644 --- a/src/IO/Operators.h +++ b/src/IO/Operators.h @@ -54,9 +54,7 @@ template WriteBuffer & operator<< (QuoteManipWriteBuffer buf, template WriteBuffer & operator<< (DoubleQuoteManipWriteBuffer buf, const T & x) { writeDoubleQuoted(x, buf.get()); return buf; } template WriteBuffer & operator<< (BinaryManipWriteBuffer buf, const T & x) { writeBinary(x, buf.get()); return buf; } -inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const String & x) { writeEscapedString(x, buf); return buf; } inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, std::string_view x) { writeEscapedString(x, buf); return buf; } -inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, StringRef x) { writeEscapedString(x, buf); return buf; } inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const char * x) { writeEscapedString(x, strlen(x), buf); return buf; } inline WriteBuffer & operator<< (QuoteManipWriteBuffer buf, const char * x) { writeAnyQuotedString<'\''>(x, x + strlen(x), buf.get()); return buf; } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 6f35dae8300..2903a70b61a 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -360,19 +360,9 @@ void writeAnyEscapedString(const char * begin, const char * end, WriteBuffer & b } -inline void writeJSONString(StringRef s, WriteBuffer & buf, const FormatSettings & settings) -{ - writeJSONString(s.data, s.data + s.size, buf, settings); -} - inline void writeJSONString(std::string_view s, WriteBuffer & buf, const FormatSettings & settings) { - writeJSONString(StringRef{s}, buf, settings); -} - -inline void writeJSONString(const String & s, WriteBuffer & buf, const FormatSettings & settings) -{ - writeJSONString(StringRef{s}, buf, settings); + writeJSONString(s.data(), s.data() + s.size(), buf, settings); } template @@ -417,7 +407,7 @@ void writeJSONNumber(T x, WriteBuffer & ostr, const FormatSettings & settings) template -void writeAnyEscapedString(const String & s, WriteBuffer & buf) +void writeAnyEscapedString(std::string_view s, WriteBuffer & buf) { writeAnyEscapedString(s.data(), s.data() + s.size(), buf); } @@ -428,18 +418,6 @@ inline void writeEscapedString(const char * str, size_t size, WriteBuffer & buf) writeAnyEscapedString<'\''>(str, str + size, buf); } - -inline void writeEscapedString(const String & s, WriteBuffer & buf) -{ - writeEscapedString(s.data(), s.size(), buf); -} - - -inline void writeEscapedString(StringRef ref, WriteBuffer & buf) -{ - writeEscapedString(ref.data, ref.size, buf); -} - inline void writeEscapedString(std::string_view ref, WriteBuffer & buf) { writeEscapedString(ref.data(), ref.size(), buf); @@ -455,16 +433,9 @@ void writeAnyQuotedString(const char * begin, const char * end, WriteBuffer & bu template -void writeAnyQuotedString(const String & s, WriteBuffer & buf) +void writeAnyQuotedString(std::string_view ref, WriteBuffer & buf) { - writeAnyQuotedString(s.data(), s.data() + s.size(), buf); -} - - -template -void writeAnyQuotedString(StringRef ref, WriteBuffer & buf) -{ - writeAnyQuotedString(ref.data, ref.data + ref.size, buf); + writeAnyQuotedString(ref.data(), ref.data() + ref.size(), buf); } @@ -475,7 +446,7 @@ inline void writeQuotedString(const String & s, WriteBuffer & buf) inline void writeQuotedString(StringRef ref, WriteBuffer & buf) { - writeAnyQuotedString<'\''>(ref, buf); + writeAnyQuotedString<'\''>(ref.toView(), buf); } inline void writeQuotedString(std::string_view ref, WriteBuffer & buf) @@ -490,7 +461,7 @@ inline void writeDoubleQuotedString(const String & s, WriteBuffer & buf) inline void writeDoubleQuotedString(StringRef s, WriteBuffer & buf) { - writeAnyQuotedString<'"'>(s, buf); + writeAnyQuotedString<'"'>(s.toView(), buf); } inline void writeDoubleQuotedString(std::string_view s, WriteBuffer & buf) @@ -501,7 +472,7 @@ inline void writeDoubleQuotedString(std::string_view s, WriteBuffer & buf) /// Outputs a string in backquotes. inline void writeBackQuotedString(StringRef s, WriteBuffer & buf) { - writeAnyQuotedString<'`'>(s, buf); + writeAnyQuotedString<'`'>(s.toView(), buf); } /// Outputs a string in backquotes for MySQL. @@ -611,16 +582,11 @@ inline void writeXMLStringForTextElementOrAttributeValue(const char * begin, con } } -inline void writeXMLStringForTextElementOrAttributeValue(const String & s, WriteBuffer & buf) +inline void writeXMLStringForTextElementOrAttributeValue(std::string_view s, WriteBuffer & buf) { writeXMLStringForTextElementOrAttributeValue(s.data(), s.data() + s.size(), buf); } -inline void writeXMLStringForTextElementOrAttributeValue(StringRef s, WriteBuffer & buf) -{ - writeXMLStringForTextElementOrAttributeValue(s.data, s.data + s.size, buf); -} - /// Writing a string to a text node in XML (not into an attribute - otherwise you need more escaping). inline void writeXMLStringForTextElement(const char * begin, const char * end, WriteBuffer & buf) { @@ -652,16 +618,11 @@ inline void writeXMLStringForTextElement(const char * begin, const char * end, W } } -inline void writeXMLStringForTextElement(const String & s, WriteBuffer & buf) +inline void writeXMLStringForTextElement(std::string_view s, WriteBuffer & buf) { writeXMLStringForTextElement(s.data(), s.data() + s.size(), buf); } -inline void writeXMLStringForTextElement(StringRef s, WriteBuffer & buf) -{ - writeXMLStringForTextElement(s.data, s.data + s.size, buf); -} - template void formatHex(IteratorSrc src, IteratorDst dst, size_t num_bytes); void formatUUID(const UInt8 * src16, UInt8 * dst36); diff --git a/src/IO/tests/gtest_manip.cpp b/src/IO/tests/gtest_manip.cpp index afcffc7dfb9..09cc3d16619 100644 --- a/src/IO/tests/gtest_manip.cpp +++ b/src/IO/tests/gtest_manip.cpp @@ -32,11 +32,6 @@ TEST(OperatorsManipTest, EscapingTest) checkString(sv1, escape, "Hello \\'world\\'"); std::string_view sv2 = s2; checkString(sv2, escape, "Hello \\\\world\\\\"); // NOLINT - - StringRef sr1 = s1; - checkString(sr1, escape, "Hello \\'world\\'"); - StringRef sr2 = s2; - checkString(sr2, escape, "Hello \\\\world\\\\"); // NOLINT } TEST(OperatorsManipTest, QuouteTest) diff --git a/src/Processors/Merges/Algorithms/Graphite.cpp b/src/Processors/Merges/Algorithms/Graphite.cpp index 6c4ca5ef85b..2448a1e2a94 100644 --- a/src/Processors/Merges/Algorithms/Graphite.cpp +++ b/src/Processors/Merges/Algorithms/Graphite.cpp @@ -71,11 +71,11 @@ static const Graphite::Pattern undef_pattern = .type = undef_pattern.TypeUndef, }; -inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, StringRef path) +inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, std::string_view path) { if (params.patterns_typed) { - std::string_view path_view = path.toView(); + std::string_view path_view = path; if (path_view.find("?"sv) == path_view.npos) return params.patterns_plain; else @@ -89,7 +89,7 @@ inline static const Patterns & selectPatternsForMetricType(const Graphite::Param Graphite::RollupRule selectPatternForPath( const Graphite::Params & params, - StringRef path) + std::string_view path) { const Graphite::Pattern * first_match = &undef_pattern; @@ -119,7 +119,7 @@ Graphite::RollupRule selectPatternForPath( } else { - if (pattern.regexp->match(path.data, path.size)) + if (pattern.regexp->match(path.data(), path.size())) { /// General pattern with matched path if (pattern.type == pattern.TypeAll) diff --git a/src/Processors/Merges/Algorithms/Graphite.h b/src/Processors/Merges/Algorithms/Graphite.h index 05306ebe30f..46b1bbbfcad 100644 --- a/src/Processors/Merges/Algorithms/Graphite.h +++ b/src/Processors/Merges/Algorithms/Graphite.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include @@ -147,7 +146,7 @@ struct Params using RollupRule = std::pair; -Graphite::RollupRule selectPatternForPath(const Graphite::Params & params, StringRef path); +Graphite::RollupRule selectPatternForPath(const Graphite::Params & params, std::string_view path); void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params); diff --git a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp index eff62d73f50..467ded19f4d 100644 --- a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp @@ -120,7 +120,7 @@ IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge() return Status(current.impl->order); } - StringRef next_path = current->all_columns[columns_definition.path_column_num]->getDataAt(current->getRow()); + std::string_view next_path = current->all_columns[columns_definition.path_column_num]->getDataAt(current->getRow()).toView(); bool new_path = is_first || next_path != current_group_path; is_first = false; @@ -190,7 +190,7 @@ IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge() current_subgroup_newest_row.set(current, sources[current.impl->order].chunk); /// Small hack: group and subgroups have the same path, so we can set current_group_path here instead of startNextGroup - /// But since we keep in memory current_subgroup_newest_row's block, we could use StringRef for current_group_path and don't + /// But since we keep in memory current_subgroup_newest_row's block, we could use string_view for current_group_path and don't /// make deep copy of the path. current_group_path = next_path; } diff --git a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h index 4968cbfc470..d6d2f66fb82 100644 --- a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h @@ -92,7 +92,7 @@ private: */ /// Path name of current bucket - StringRef current_group_path; + std::string_view current_group_path; static constexpr size_t max_row_refs = 2; /// current_subgroup_newest_row, current_row. /// Last row with maximum version for current primary key (time bucket). diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 5e9da48fc68..bf2638f7bc3 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include diff --git a/utils/graphite-rollup/graphite-rollup-bench.cpp b/utils/graphite-rollup/graphite-rollup-bench.cpp index 4c11f90b3ff..49a3d509be6 100644 --- a/utils/graphite-rollup/graphite-rollup-bench.cpp +++ b/utils/graphite-rollup/graphite-rollup-bench.cpp @@ -20,9 +20,9 @@ using namespace DB; static SharedContextHolder shared_context = Context::createShared(); -std::vector loadMetrics(const std::string & metrics_file) +std::vector loadMetrics(const std::string & metrics_file) { - std::vector metrics; + std::vector metrics; FILE * stream; char * line = nullptr; @@ -47,7 +47,7 @@ std::vector loadMetrics(const std::string & metrics_file) } if (l > 0) { - metrics.push_back(StringRef(strdup(line), l)); + metrics.emplace_back(std::string_view(strdup(line), l)); } } } @@ -80,7 +80,7 @@ void bench(const std::string & config_path, const std::string & metrics_file, si Graphite::Params params; setGraphitePatternsFromConfig(context, "graphite_rollup", params); - std::vector metrics = loadMetrics(metrics_file); + std::vector metrics = loadMetrics(metrics_file); std::vector durations(metrics.size()); size_t j, i; @@ -99,15 +99,15 @@ void bench(const std::string & config_path, const std::string & metrics_file, si if (j == 0 && verbose) { - std::cout << metrics[i].data << ": rule with regexp '" << rule.second->regexp_str << "' found\n"; + std::cout << metrics[i].data() << ": rule with regexp '" << rule.second->regexp_str << "' found\n"; } } } for (i = 0; i < metrics.size(); i++) { - std::cout << metrics[i].data << " " << durations[i] / n << " ns\n"; - free(const_cast(static_cast(metrics[i].data))); + std::cout << metrics[i].data() << " " << durations[i] / n << " ns\n"; + free(const_cast(static_cast(metrics[i].data()))); } } From de2a0ca05e52126e7f9ccdec45e00a3b21d91895 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 19 Jul 2022 09:03:58 +0000 Subject: [PATCH 382/659] black --- tests/integration/test_keeper_force_recovery/test.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_force_recovery/test.py b/tests/integration/test_keeper_force_recovery/test.py index 8eb759fae47..3109562f1c3 100644 --- a/tests/integration/test_keeper_force_recovery/test.py +++ b/tests/integration/test_keeper_force_recovery/test.py @@ -52,8 +52,9 @@ def started_cluster(): def get_fake_zk(nodename, timeout=30.0): _fake_zk_instance = KazooClient( - hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout, - command_retry=KazooRetry(max_tries=10) + hosts=cluster.get_instance_ip(nodename) + ":9181", + timeout=timeout, + command_retry=KazooRetry(max_tries=10), ) _fake_zk_instance.start() From 3e7414b356a2c9b659da17ff966e96c819e2f464 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 19 Jul 2022 12:47:20 +0200 Subject: [PATCH 383/659] add comment --- src/Storages/StorageReplicatedMergeTree.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c2ed93ca074..1bc4c26e40e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7953,6 +7953,11 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP while (true) { + /// We should be careful when creating an empty part, because we are not sure that this part is still needed. + /// For example, it's possible that part (or partition) was dropped (or replaced) concurrently. + /// We can enqueue part for check from DataPartExchange or SelectProcessor + /// and it's hard to synchronize it with ReplicatedMergeTreeQueue and PartCheckThread... + /// But at least we can ignore parts that are definitely not needed according to virtual parts and drop ranges. auto pred = queue.getMergePredicate(zookeeper); String covering_virtual = pred.getCoveringVirtualPart(lost_part_name); if (covering_virtual.empty()) From 9d0e3d107b806835f769bd03396720e0399c759a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 19 Jul 2022 13:17:33 +0200 Subject: [PATCH 384/659] Update src/Functions/formatReadableTimeDelta.cpp Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- src/Functions/formatReadableTimeDelta.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/formatReadableTimeDelta.cpp b/src/Functions/formatReadableTimeDelta.cpp index 8efd13a734a..219c2d95353 100644 --- a/src/Functions/formatReadableTimeDelta.cpp +++ b/src/Functions/formatReadableTimeDelta.cpp @@ -122,7 +122,7 @@ public: throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected value of maximum unit argument ({}) for function {}, the only allowed values are:" " 'seconds', 'minutes', 'hours', 'days', 'months', 'years'.", - std::string(maximum_unit_str), getName()); + maximum_unit_str, getName()); auto col_to = ColumnString::create(); From a54033b749aab4c794c7b05a2c7d3db8aac7d656 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 7 Jul 2022 20:31:42 +0000 Subject: [PATCH 385/659] EXPLAIN AST rewrite: additional query in test --- .../02353_explain_ast_rewrite.reference | 28 +++++++++++++++++++ .../0_stateless/02353_explain_ast_rewrite.sql | 2 ++ 2 files changed, 30 insertions(+) diff --git a/tests/queries/0_stateless/02353_explain_ast_rewrite.reference b/tests/queries/0_stateless/02353_explain_ast_rewrite.reference index 5ee3e0d126c..1a5202b8e98 100644 --- a/tests/queries/0_stateless/02353_explain_ast_rewrite.reference +++ b/tests/queries/0_stateless/02353_explain_ast_rewrite.reference @@ -23,3 +23,31 @@ SelectWithUnionQuery (children 1) Function numbers (children 1) ExpressionList (children 1) Literal UInt64_0 +EXPLAIN AST rewrite=0 SELECT countDistinct(number) FROM numbers(0); +SelectWithUnionQuery (children 1) + ExpressionList (children 1) + SelectQuery (children 2) + ExpressionList (children 1) + Function countDistinct (children 1) + ExpressionList (children 1) + Identifier number + TablesInSelectQuery (children 1) + TablesInSelectQueryElement (children 1) + TableExpression (children 1) + Function numbers (children 1) + ExpressionList (children 1) + Literal UInt64_0 +EXPLAIN AST rewrite=1 SELECT countDistinct(number) FROM numbers(0); +SelectWithUnionQuery (children 1) + ExpressionList (children 1) + SelectQuery (children 2) + ExpressionList (children 1) + Function uniqExact (children 1) + ExpressionList (children 1) + Identifier number + TablesInSelectQuery (children 1) + TablesInSelectQueryElement (children 1) + TableExpression (children 1) + Function numbers (children 1) + ExpressionList (children 1) + Literal UInt64_0 diff --git a/tests/queries/0_stateless/02353_explain_ast_rewrite.sql b/tests/queries/0_stateless/02353_explain_ast_rewrite.sql index 7310aa62704..48082ffadfc 100644 --- a/tests/queries/0_stateless/02353_explain_ast_rewrite.sql +++ b/tests/queries/0_stateless/02353_explain_ast_rewrite.sql @@ -1,4 +1,6 @@ -- { echoOn } EXPLAIN AST rewrite=0 SELECT * FROM numbers(0); EXPLAIN AST rewrite=1 SELECT * FROM numbers(0); +EXPLAIN AST rewrite=0 SELECT countDistinct(number) FROM numbers(0); +EXPLAIN AST rewrite=1 SELECT countDistinct(number) FROM numbers(0); -- { echoOff } From a761da9f1baa1e0387c900926beed981a55a3a03 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 19 Jul 2022 13:58:59 +0200 Subject: [PATCH 386/659] add comment --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index a830815f760..f4cae7152ef 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -519,8 +519,10 @@ public: /// The version of "log" node that is used to check that no new merges have appeared. int32_t getVersion() const { return merges_version; } + /// Returns true if there's a drop range covering new_drop_range_info bool hasDropRange(const MergeTreePartInfo & new_drop_range_info) const; + /// Returns virtual part covering part_name (if any) or empty string String getCoveringVirtualPart(const String & part_name) const; private: From 30eb2cbcb37ee8c99ba5e549ed14930108600c6f Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 19 Jul 2022 14:03:13 +0200 Subject: [PATCH 387/659] removed Base58 domain type --- .../data-types/domains/base58.md | 12 ---- src/DataTypes/DataTypeCustomBase58.cpp | 19 ----- src/DataTypes/DataTypeFactory.cpp | 1 - src/DataTypes/DataTypeFactory.h | 1 - .../Serializations/SerializationBase58.cpp | 72 ------------------- .../Serializations/SerializationBase58.h | 17 ----- 6 files changed, 122 deletions(-) delete mode 100644 docs/en/sql-reference/data-types/domains/base58.md delete mode 100644 src/DataTypes/DataTypeCustomBase58.cpp delete mode 100644 src/DataTypes/Serializations/SerializationBase58.cpp delete mode 100644 src/DataTypes/Serializations/SerializationBase58.h diff --git a/docs/en/sql-reference/data-types/domains/base58.md b/docs/en/sql-reference/data-types/domains/base58.md deleted file mode 100644 index 5c2afe106cc..00000000000 --- a/docs/en/sql-reference/data-types/domains/base58.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -sidebar_position: 61 -sidebar_label: Base58 ---- - -## Base58 - -`Base58` is a domain based on `String` type created for storing Base58-encoded strings values. - -Usage is the same as for general `String` type. THe only difference is that string of `Base58` type must be a correct Base58-encoded string. - -[Original article](https://clickhouse.com/docs/en/data_types/domains/base58) diff --git a/src/DataTypes/DataTypeCustomBase58.cpp b/src/DataTypes/DataTypeCustomBase58.cpp deleted file mode 100644 index 67c4399c9a0..00000000000 --- a/src/DataTypes/DataTypeCustomBase58.cpp +++ /dev/null @@ -1,19 +0,0 @@ -#include -#include -#include - -namespace DB -{ - -void registerDataTypeDomainBase58(DataTypeFactory & factory) -{ - factory.registerSimpleDataTypeCustom("Base58", [] - { - auto type = DataTypeFactory::instance().get("String"); - return std::make_pair(type, std::make_unique( - std::make_unique("Base58"), std::make_unique(type->getDefaultSerialization()))); - }); - -} - -} diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 336934c7a56..dd08656dccc 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -219,7 +219,6 @@ DataTypeFactory::DataTypeFactory() registerDataTypeDomainGeo(*this); registerDataTypeMap(*this); registerDataTypeObject(*this); - registerDataTypeDomainBase58(*this); } DataTypeFactory & DataTypeFactory::instance() diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index d46dbbade7a..704d8926bf0 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -88,6 +88,5 @@ void registerDataTypeDomainBool(DataTypeFactory & factory); void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory); void registerDataTypeDomainGeo(DataTypeFactory & factory); void registerDataTypeObject(DataTypeFactory & factory); -void registerDataTypeDomainBase58(DataTypeFactory & factory); } diff --git a/src/DataTypes/Serializations/SerializationBase58.cpp b/src/DataTypes/Serializations/SerializationBase58.cpp deleted file mode 100644 index c6cbee843a2..00000000000 --- a/src/DataTypes/Serializations/SerializationBase58.cpp +++ /dev/null @@ -1,72 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; - extern const int ILLEGAL_COLUMN; -} - -SerializationBase58::SerializationBase58(const SerializationPtr & nested_) : SerializationCustomSimpleText(nested_) -{ -} - -void SerializationBase58::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const -{ - const ColumnString * col = checkAndGetColumn(&column); - if (!col) - { - throw Exception("Base58 type can only serialize columns of type String." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); - } - - auto value = col->getDataAtWithTerminatingZero(row_num); - char buffer[value.size * 2 + 1]; - char * ptr = buffer; - encodeBase58(reinterpret_cast(value.data), reinterpret_cast(ptr)); - ostr.write(buffer, strlen(buffer)); -} - -void SerializationBase58::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const -{ - ColumnString * col = typeid_cast(&column); - if (!col) - { - throw Exception("Base58 type can only deserialize columns of type String." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); - } - - size_t allocated = 32; - std::string encoded(allocated, '\0'); - - size_t read_position = 0; - while (istr.read(encoded[read_position])) - { - ++read_position; - if (read_position == allocated) - { - allocated *= 2; - encoded.resize(allocated, '\0'); - } - } - - char buffer[read_position + 1]; - if (!decodeBase58(reinterpret_cast(encoded.c_str()), reinterpret_cast(buffer))) - { - throw Exception("Invalid Base58 encoded value, cannot parse." + column.getName(), ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); - } - - col->insertDataWithTerminatingZero(buffer, read_position+1); - - if (whole && !istr.eof()) - throwUnexpectedDataAfterParsedValue(column, istr, settings, "Base58"); -} -} diff --git a/src/DataTypes/Serializations/SerializationBase58.h b/src/DataTypes/Serializations/SerializationBase58.h deleted file mode 100644 index e6586d1fb03..00000000000 --- a/src/DataTypes/Serializations/SerializationBase58.h +++ /dev/null @@ -1,17 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class SerializationBase58 final : public SerializationCustomSimpleText -{ -public: - explicit SerializationBase58(const SerializationPtr & nested_); - - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; - void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override; -}; - -} From 246614229fb2d8997db0b92271f05e75c36f74c3 Mon Sep 17 00:00:00 2001 From: Constantine Peresypkin Date: Wed, 6 Jul 2022 20:51:30 +0200 Subject: [PATCH 388/659] fix long wait for process exit in ShellCommand fixes #38889 --- src/Common/ShellCommand.cpp | 43 +------- src/Common/waitForPid.cpp | 192 ++++++++++++++++++++++++++++++++++++ src/Common/waitForPid.h | 12 +++ 3 files changed, 206 insertions(+), 41 deletions(-) create mode 100644 src/Common/waitForPid.cpp create mode 100644 src/Common/waitForPid.h diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 86adeeaf7e5..0050288b1cf 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -1,9 +1,7 @@ #include #include -#include #include #include -#include #include #include @@ -13,6 +11,7 @@ #include #include #include +#include namespace @@ -94,53 +93,15 @@ ShellCommand::~ShellCommand() bool ShellCommand::tryWaitProcessWithTimeout(size_t timeout_in_seconds) { - int status = 0; - LOG_TRACE(getLogger(), "Try wait for shell command pid {} with timeout {}", pid, timeout_in_seconds); wait_called = true; - struct timespec interval {.tv_sec = 1, .tv_nsec = 0}; in.close(); out.close(); err.close(); - if (timeout_in_seconds == 0) - { - /// If there is no timeout before signal try to waitpid 1 time without block so we can avoid sending - /// signal if process is already normally terminated. - - int waitpid_res = waitpid(pid, &status, WNOHANG); - bool process_terminated_normally = (waitpid_res == pid); - return process_terminated_normally; - } - - /// If timeout is positive try waitpid without block in loop until - /// process is normally terminated or waitpid return error - - while (timeout_in_seconds != 0) - { - int waitpid_res = waitpid(pid, &status, WNOHANG); - bool process_terminated_normally = (waitpid_res == pid); - - if (process_terminated_normally) - { - return true; - } - else if (waitpid_res == 0) - { - --timeout_in_seconds; - nanosleep(&interval, nullptr); - - continue; - } - else if (waitpid_res == -1 && errno != EINTR) - { - return false; - } - } - - return false; + return waitForPid(pid, timeout_in_seconds); } void ShellCommand::logCommand(const char * filename, char * const argv[]) diff --git a/src/Common/waitForPid.cpp b/src/Common/waitForPid.cpp new file mode 100644 index 00000000000..38f43ae2f6a --- /dev/null +++ b/src/Common/waitForPid.cpp @@ -0,0 +1,192 @@ +#include +#include +#include +#include + +#include +#include +#include + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wgnu-statement-expression" +#define HANDLE_EINTR(x) ({ \ + decltype(x) eintr_wrapper_result; \ + do { \ + eintr_wrapper_result = (x); \ + } while (eintr_wrapper_result == -1 && errno == EINTR); \ + eintr_wrapper_result; \ +}) + +#if defined(OS_LINUX) + +#include +#include + +#if !defined(__NR_pidfd_open) + #if defined(__x86_64__) + #define SYS_pidfd_open 434 + #elif defined(__aarch64__) + #define SYS_pidfd_open 434 + #elif defined(__ppc64__) + #define SYS_pidfd_open 434 + #elif defined(__riscv) + #define SYS_pidfd_open 434 + #else + #error "Unsupported architecture" + #endif +#else + #define SYS_pidfd_open __NR_pidfd_open +#endif + +namespace DB +{ + +static int syscall_pidfd_open(pid_t pid) +{ + // pidfd_open cannot be interrupted, no EINTR handling + return syscall(SYS_pidfd_open, pid, 0); +} + +static int dir_pidfd_open(pid_t pid) +{ + std::string path = "/proc/" + std::to_string(pid); + return HANDLE_EINTR(open(path.c_str(), O_DIRECTORY)); +} + +static bool supportsPidFdOpen() +{ + VersionNumber pidfd_open_minimal_version(5, 3, 0); + VersionNumber linux_version(Poco::Environment::osVersion()); + return linux_version >= pidfd_open_minimal_version; +} + +static int pidFdOpen(pid_t pid) +{ + // use pidfd_open or just plain old /proc/[pid] open for Linux + if (supportsPidFdOpen()) + { + return syscall_pidfd_open(pid); + } + else + { + return dir_pidfd_open(pid); + } +} + +static int pollPid(pid_t pid, int timeout_in_ms) +{ + struct pollfd pollfd; + + int pid_fd = pidFdOpen(pid); + if (pid_fd == -1) + { + return false; + } + pollfd.fd = pid_fd; + pollfd.events = POLLIN; + int ready = poll(&pollfd, 1, timeout_in_ms); + int save_errno = errno; + close(pid_fd); + errno = save_errno; + return ready; +} +#elif defined(OS_DARWIN) || defined(OS_FREEBSD) + +#include +#include + +namespace DB +{ + +static int pollPid(pid_t pid, int timeout_in_ms) +{ + int status = 0; + int kq = HANDLE_EINTR(kqueue()); + if (kq == -1) + { + return false; + } + struct kevent change = {.ident = NULL}; + EV_SET(&change, pid, EVFILT_PROC, EV_ADD, NOTE_EXIT, 0, NULL); + int result = HANDLE_EINTR(kevent(kq, &change, 1, NULL, 0, NULL)); + if (result == -1) + { + if (errno != ESRCH) + { + return false; + } + // check if pid already died while we called kevent() + if (waitpid(pid, &status, WNOHANG) == pid) + { + return true; + } + return false; + } + + struct kevent event = {.ident = NULL}; + struct timespec remaining_timespec = {.tv_sec = timeout_in_ms / 1000, .tv_nsec = (timeout_in_ms % 1000) * 1000000}; + int ready = kevent(kq, nullptr, 0, &event, 1, &remaining_timespec); + int save_errno = errno; + close(kq); + errno = save_errno; + return ready; +} +#else + #error "Unsupported OS type" +#endif + +bool waitForPid(pid_t pid, size_t timeout_in_seconds) +{ + int status = 0; + + Stopwatch watch; + + if (timeout_in_seconds == 0) + { + /// If there is no timeout before signal try to waitpid 1 time without block so we can avoid sending + /// signal if process is already normally terminated. + + int waitpid_res = waitpid(pid, &status, WNOHANG); + bool process_terminated_normally = (waitpid_res == pid); + return process_terminated_normally; + } + + /// If timeout is positive try waitpid without block in loop until + /// process is normally terminated or waitpid return error + + int timeout_in_ms = timeout_in_seconds * 1000; + while (timeout_in_ms > 0) + { + int waitpid_res = waitpid(pid, &status, WNOHANG); + bool process_terminated_normally = (waitpid_res == pid); + if (process_terminated_normally) + { + return true; + } + else if (waitpid_res == 0) + { + watch.restart(); + int ready = pollPid(pid, timeout_in_ms); + if (ready <= 0) + { + if (errno == EINTR || errno == EAGAIN) + { + timeout_in_ms -= watch.elapsedMilliseconds(); + } + else + { + return false; + } + } + continue; + } + else if (waitpid_res == -1 && errno != EINTR) + { + return false; + } + } + return false; +} + +} +#pragma GCC diagnostic pop diff --git a/src/Common/waitForPid.h b/src/Common/waitForPid.h new file mode 100644 index 00000000000..71c1a74712c --- /dev/null +++ b/src/Common/waitForPid.h @@ -0,0 +1,12 @@ +#pragma once +#include + +namespace DB +{ +/* + * Waits for a specific pid with timeout, using modern Linux and OSX facilities + * Returns `true` if process terminated successfully or `false` otherwise + */ +bool waitForPid(pid_t pid, size_t timeout_in_seconds); + +} From 329acfd6a80194eb1c75e3dec359c2c38a810515 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 19 Jul 2022 12:08:57 +0000 Subject: [PATCH 389/659] Fix tests --- .../0_stateless/02149_external_schema_inference.reference | 1 - .../format_schemas/00825_protobuf_format_persons.proto | 6 ------ 2 files changed, 7 deletions(-) diff --git a/tests/queries/0_stateless/02149_external_schema_inference.reference b/tests/queries/0_stateless/02149_external_schema_inference.reference index 875659c7fb6..ebc30e874da 100644 --- a/tests/queries/0_stateless/02149_external_schema_inference.reference +++ b/tests/queries/0_stateless/02149_external_schema_inference.reference @@ -46,7 +46,6 @@ age String isOnline Enum8(\'offline\' = 0, \'online\' = 1) someRatio Float64 visitTime UInt64 -newMessage Tuple(empty Array(Tuple()), z Float32) randomBigNumber Int64 newFieldInt Array(Int32) color Array(Float32) diff --git a/tests/queries/0_stateless/format_schemas/00825_protobuf_format_persons.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_persons.proto index b588619f488..ecebb1ba452 100644 --- a/tests/queries/0_stateless/format_schemas/00825_protobuf_format_persons.proto +++ b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_persons.proto @@ -75,11 +75,6 @@ message AltPerson { male = 0; female = 1; }; - message Dummy { - message Empty {}; - repeated Empty empty = 1; - float z = 2; - }; repeated int32 location = 101 [packed=false]; float pi = 103; bytes uuid = 300; @@ -92,7 +87,6 @@ message AltPerson { OnlineStatus isOnline = 1; double someRatio = 100; fixed64 visitTime = 15; - Dummy newMessage = 1000; sfixed64 randomBigNumber = 140; repeated int32 newFieldInt = 104; repeated float color = 14; From 4f020654bef5bd793c27c3f0d7f6e08b848a1964 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 19 Jul 2022 12:12:40 +0000 Subject: [PATCH 390/659] Get rid of unneded ifdefs --- src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index 81e594e1153..8ca874103a8 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -267,13 +267,7 @@ bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension try { auto array = readMessage(); - -#if CAPNP_VERSION >= 7000 && CAPNP_VERSION < 8000 - capnp::UnalignedFlatArrayMessageReader msg(array); -#else capnp::FlatArrayMessageReader msg(array); -#endif - auto root_reader = msg.getRoot(root); for (size_t i = 0; i != columns.size(); ++i) { From 5ad4e9d28fe34fc756a12f0ae01817d4d42046ec Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 19 Jul 2022 14:33:50 +0200 Subject: [PATCH 391/659] Update LRUFileCache.cpp --- src/Common/LRUFileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/LRUFileCache.cpp b/src/Common/LRUFileCache.cpp index 0ce76dbdec6..b14df39331a 100644 --- a/src/Common/LRUFileCache.cpp +++ b/src/Common/LRUFileCache.cpp @@ -45,7 +45,7 @@ void LRUFileCache::initialize() catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); - return; + throw; } } else From 37af9685481d623a06df2fb33350ae19f82976a4 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 19 Jul 2022 12:40:50 +0000 Subject: [PATCH 392/659] Rename option rewrite -> optimize --- src/Interpreters/InterpreterExplainQuery.cpp | 6 +++--- ...ite.reference => 02353_explain_ast_optimize.reference} | 8 ++++---- tests/queries/0_stateless/02353_explain_ast_optimize.sql | 6 ++++++ tests/queries/0_stateless/02353_explain_ast_rewrite.sql | 6 ------ 4 files changed, 13 insertions(+), 13 deletions(-) rename tests/queries/0_stateless/{02353_explain_ast_rewrite.reference => 02353_explain_ast_optimize.reference} (86%) create mode 100644 tests/queries/0_stateless/02353_explain_ast_optimize.sql delete mode 100644 tests/queries/0_stateless/02353_explain_ast_rewrite.sql diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 3fad4374abe..6715947da43 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -146,14 +146,14 @@ namespace struct QueryASTSettings { bool graph = false; - bool rewrite = false; + bool optimize = false; constexpr static char name[] = "AST"; std::unordered_map> boolean_settings = { {"graph", graph}, - {"rewrite", rewrite} + {"optimize", optimize} }; }; @@ -280,7 +280,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() case ASTExplainQuery::ParsedAST: { auto settings = checkAndGetSettings(ast.getSettings()); - if (settings.rewrite) + if (settings.optimize) { ExplainAnalyzedSyntaxVisitor::Data data(getContext()); ExplainAnalyzedSyntaxVisitor(data).visit(query); diff --git a/tests/queries/0_stateless/02353_explain_ast_rewrite.reference b/tests/queries/0_stateless/02353_explain_ast_optimize.reference similarity index 86% rename from tests/queries/0_stateless/02353_explain_ast_rewrite.reference rename to tests/queries/0_stateless/02353_explain_ast_optimize.reference index 1a5202b8e98..f4e0de5ca98 100644 --- a/tests/queries/0_stateless/02353_explain_ast_rewrite.reference +++ b/tests/queries/0_stateless/02353_explain_ast_optimize.reference @@ -1,5 +1,5 @@ -- { echoOn } -EXPLAIN AST rewrite=0 SELECT * FROM numbers(0); +EXPLAIN AST optimize=0 SELECT * FROM numbers(0); SelectWithUnionQuery (children 1) ExpressionList (children 1) SelectQuery (children 2) @@ -11,7 +11,7 @@ SelectWithUnionQuery (children 1) Function numbers (children 1) ExpressionList (children 1) Literal UInt64_0 -EXPLAIN AST rewrite=1 SELECT * FROM numbers(0); +EXPLAIN AST optimize=1 SELECT * FROM numbers(0); SelectWithUnionQuery (children 1) ExpressionList (children 1) SelectQuery (children 2) @@ -23,7 +23,7 @@ SelectWithUnionQuery (children 1) Function numbers (children 1) ExpressionList (children 1) Literal UInt64_0 -EXPLAIN AST rewrite=0 SELECT countDistinct(number) FROM numbers(0); +EXPLAIN AST optimize=0 SELECT countDistinct(number) FROM numbers(0); SelectWithUnionQuery (children 1) ExpressionList (children 1) SelectQuery (children 2) @@ -37,7 +37,7 @@ SelectWithUnionQuery (children 1) Function numbers (children 1) ExpressionList (children 1) Literal UInt64_0 -EXPLAIN AST rewrite=1 SELECT countDistinct(number) FROM numbers(0); +EXPLAIN AST optimize=1 SELECT countDistinct(number) FROM numbers(0); SelectWithUnionQuery (children 1) ExpressionList (children 1) SelectQuery (children 2) diff --git a/tests/queries/0_stateless/02353_explain_ast_optimize.sql b/tests/queries/0_stateless/02353_explain_ast_optimize.sql new file mode 100644 index 00000000000..a46a47a2e64 --- /dev/null +++ b/tests/queries/0_stateless/02353_explain_ast_optimize.sql @@ -0,0 +1,6 @@ +-- { echoOn } +EXPLAIN AST optimize=0 SELECT * FROM numbers(0); +EXPLAIN AST optimize=1 SELECT * FROM numbers(0); +EXPLAIN AST optimize=0 SELECT countDistinct(number) FROM numbers(0); +EXPLAIN AST optimize=1 SELECT countDistinct(number) FROM numbers(0); +-- { echoOff } diff --git a/tests/queries/0_stateless/02353_explain_ast_rewrite.sql b/tests/queries/0_stateless/02353_explain_ast_rewrite.sql deleted file mode 100644 index 48082ffadfc..00000000000 --- a/tests/queries/0_stateless/02353_explain_ast_rewrite.sql +++ /dev/null @@ -1,6 +0,0 @@ --- { echoOn } -EXPLAIN AST rewrite=0 SELECT * FROM numbers(0); -EXPLAIN AST rewrite=1 SELECT * FROM numbers(0); -EXPLAIN AST rewrite=0 SELECT countDistinct(number) FROM numbers(0); -EXPLAIN AST rewrite=1 SELECT countDistinct(number) FROM numbers(0); --- { echoOff } From af7acb17828d61abc7dc2a058e8f0d915dde5580 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 19 Jul 2022 14:41:31 +0200 Subject: [PATCH 393/659] Update LRUFileCache.cpp --- src/Common/LRUFileCache.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/LRUFileCache.cpp b/src/Common/LRUFileCache.cpp index b14df39331a..a6fc2c95038 100644 --- a/src/Common/LRUFileCache.cpp +++ b/src/Common/LRUFileCache.cpp @@ -841,7 +841,9 @@ void LRUFileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_l /// cache_base_path / key_prefix / key / offset if (!files.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache already initialized"); + throw Exception( + REMOTE_FS_OBJECT_CACHE_ERROR, + "Cache already initialized: this can be result of a first attempt during cache initialization. Please, check log for error messages"); fs::directory_iterator key_prefix_it{cache_base_path}; for (; key_prefix_it != fs::directory_iterator(); ++key_prefix_it) From f7bb69ef7abafe91652e68cb3d5129c5916bb2a8 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 19 Jul 2022 14:43:11 +0200 Subject: [PATCH 394/659] Update LRUFileCache.cpp --- src/Common/LRUFileCache.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/LRUFileCache.cpp b/src/Common/LRUFileCache.cpp index a6fc2c95038..4289502374b 100644 --- a/src/Common/LRUFileCache.cpp +++ b/src/Common/LRUFileCache.cpp @@ -843,7 +843,9 @@ void LRUFileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_l if (!files.empty()) throw Exception( REMOTE_FS_OBJECT_CACHE_ERROR, - "Cache already initialized: this can be result of a first attempt during cache initialization. Please, check log for error messages"); + "Cache initialization is partially made. " + "This can be a result of a first attempt to initialize cache. " + "Please, check log for error messages"); fs::directory_iterator key_prefix_it{cache_base_path}; for (; key_prefix_it != fs::directory_iterator(); ++key_prefix_it) From 2e3ceb351e4865bc3ccb8f02111a062207244b6a Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 19 Jul 2022 14:44:56 +0200 Subject: [PATCH 395/659] Update LRUFileCache.cpp --- src/Common/LRUFileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/LRUFileCache.cpp b/src/Common/LRUFileCache.cpp index 4289502374b..b7472362823 100644 --- a/src/Common/LRUFileCache.cpp +++ b/src/Common/LRUFileCache.cpp @@ -844,7 +844,7 @@ void LRUFileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_l throw Exception( REMOTE_FS_OBJECT_CACHE_ERROR, "Cache initialization is partially made. " - "This can be a result of a first attempt to initialize cache. " + "This can be a result of a failed first attempt to initialize cache. " "Please, check log for error messages"); fs::directory_iterator key_prefix_it{cache_base_path}; From 88d59520a2b64aed4b3863ac8b0df20239a36b71 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 19 Jul 2022 15:20:56 +0200 Subject: [PATCH 396/659] Fix --- src/Formats/ReadSchemaUtils.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 39812f5ba56..058f9b7059b 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -81,6 +81,8 @@ ColumnsDescription readSchemaFromFormat( try { buf = read_buffer_iterator(); + if (!buf) + break; is_eof = buf->eof(); } catch (...) From 3a7fa8887fe84d2cb7b4896330eed73628751c7d Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 19 Jul 2022 10:23:25 -0400 Subject: [PATCH 397/659] refine readCompressedDataBlockForAsynchronous --- src/Compression/CompressedReadBufferBase.cpp | 115 ++++++++----------- 1 file changed, 49 insertions(+), 66 deletions(-) diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index b2cd222f7d8..30e5d0c6655 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -106,21 +106,15 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c throw Exception(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH); } - -/// Read compressed data into compressed_buffer. Get size of decompressed data from block header. Checksum if need. -/// Returns number of compressed bytes read. -size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum, bool always_copy) +static void readHeaderAndGetCodecAndSize( + const char * compressed_buffer, + UInt8 header_size, + CompressionCodecPtr & codec, + size_t & size_decompressed, + size_t & size_compressed_without_checksum, + bool allow_different_codecs) { - if (compressed_in->eof()) - return 0; - - UInt8 header_size = ICompressionCodec::getHeaderSize(); - own_compressed_buffer.resize(header_size + sizeof(Checksum)); - - compressed_in->readStrict(own_compressed_buffer.data(), sizeof(Checksum) + header_size); - char * compressed_header = own_compressed_buffer.data() + sizeof(Checksum); - - uint8_t method = ICompressionCodec::readMethod(compressed_header); + uint8_t method = ICompressionCodec::readMethod(compressed_buffer); if (!codec) { @@ -142,8 +136,8 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, } } - size_compressed_without_checksum = ICompressionCodec::readCompressedBlockSize(compressed_header); - size_decompressed = ICompressionCodec::readDecompressedBlockSize(compressed_header); + size_compressed_without_checksum = ICompressionCodec::readCompressedBlockSize(compressed_buffer); + size_decompressed = ICompressionCodec::readDecompressedBlockSize(compressed_buffer); /// This is for clang static analyzer. assert(size_decompressed > 0); @@ -159,6 +153,27 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, + ", this should include header size) is less than the header size (" + toString(header_size) + ")", ErrorCodes::CORRUPTED_DATA); ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum)); +} + +/// Read compressed data into compressed_buffer. Get size of decompressed data from block header. Checksum if need. +/// Returns number of compressed bytes read. +size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum, bool always_copy) +{ + if (compressed_in->eof()) + return 0; + + UInt8 header_size = ICompressionCodec::getHeaderSize(); + own_compressed_buffer.resize(header_size + sizeof(Checksum)); + + compressed_in->readStrict(own_compressed_buffer.data(), sizeof(Checksum) + header_size); + + readHeaderAndGetCodecAndSize( + own_compressed_buffer.data() + sizeof(Checksum), + header_size, + codec, + size_decompressed, + size_compressed_without_checksum, + allow_different_codecs); auto additional_size_at_the_end_of_buffer = codec->getAdditionalSizeAtTheEndOfBuffer(); @@ -187,60 +202,29 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, return size_compressed_without_checksum + sizeof(Checksum); } +/// Read compressed data into compressed_buffer for asynchronous decompression to avoid the situation of "read compressed block across the compressed_in". size_t CompressedReadBufferBase::readCompressedDataBlockForAsynchronous(size_t & size_decompressed, size_t & size_compressed_without_checksum) { UInt8 header_size = ICompressionCodec::getHeaderSize(); + /// Make sure the whole header located in 'compressed_in->' buffer. if (compressed_in->eof() || (compressed_in->available() < (header_size + sizeof(Checksum)))) return 0; own_compressed_buffer.resize(header_size + sizeof(Checksum)); - compressed_in->readStrict(own_compressed_buffer.data(), sizeof(Checksum) + header_size); - char * compressed_header = own_compressed_buffer.data() + sizeof(Checksum); - uint8_t method = ICompressionCodec::readMethod(compressed_header); - - if (!codec) - { - codec = CompressionCodecFactory::instance().get(method); - } - else if (method != codec->getMethodByte()) - { - if (allow_different_codecs) - { - codec = CompressionCodecFactory::instance().get(method); - } - else - { - throw Exception("Data compressed with different methods, given method byte 0x" - + getHexUIntLowercase(method) - + ", previous method byte 0x" - + getHexUIntLowercase(codec->getMethodByte()), - ErrorCodes::CANNOT_DECOMPRESS); - } - } - - size_compressed_without_checksum = ICompressionCodec::readCompressedBlockSize(compressed_header); - size_decompressed = ICompressionCodec::readDecompressedBlockSize(compressed_header); - - /// This is for clang static analyzer. - assert(size_decompressed > 0); - - if (size_compressed_without_checksum > DBMS_MAX_COMPRESSED_SIZE) - throw Exception("Too large size_compressed_without_checksum: " - + toString(size_compressed_without_checksum) - + ". Most likely corrupted data.", - ErrorCodes::TOO_LARGE_SIZE_COMPRESSED); - - if (size_compressed_without_checksum < header_size) - throw Exception("Can't decompress data: the compressed data size (" + toString(size_compressed_without_checksum) - + ", this should include header size) is less than the header size (" + toString(header_size) + ")", ErrorCodes::CORRUPTED_DATA); - - ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum)); + readHeaderAndGetCodecAndSize( + own_compressed_buffer.data() + sizeof(Checksum), + header_size, + codec, + size_decompressed, + size_compressed_without_checksum, + allow_different_codecs); auto additional_size_at_the_end_of_buffer = codec->getAdditionalSizeAtTheEndOfBuffer(); - /// Is whole compressed block located in 'compressed_in->' buffer? + /// Make sure the whole compressed block located in 'compressed_in->' buffer. + /// Otherwise, abandon header and restore original offset of compressed_in if (compressed_in->offset() >= header_size + sizeof(Checksum) && compressed_in->available() >= (size_compressed_without_checksum - header_size) + additional_size_at_the_end_of_buffer + sizeof(Checksum)) { @@ -320,19 +304,18 @@ void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_d void CompressedReadBufferBase::flushAsynchronousDecompressRequests() const { - if (codec) - { - codec->flushAsynchronousDecompressRequests(); - } + /// The codec should be created at the latest in readHeaderAndGetCodec + assert(codec.get()); + codec->flushAsynchronousDecompressRequests(); } void CompressedReadBufferBase::setDecompressMode(ICompressionCodec::CodecMode mode) { - if (codec) - { - codec->setDecompressMode(mode); - } + /// The codec should be created at the latest in readHeaderAndGetCodec + assert(codec.get()); + codec->setDecompressMode(mode); } + /// 'compressed_in' could be initialized lazily, but before first call of 'readCompressedData'. CompressedReadBufferBase::CompressedReadBufferBase(ReadBuffer * in, bool allow_different_codecs_) : compressed_in(in), own_compressed_buffer(0), allow_different_codecs(allow_different_codecs_) From 10e4ef135d8c62ede00b4af9cd0c0711e57ceb22 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 18 Jul 2022 15:18:39 +0200 Subject: [PATCH 398/659] Set default value cross_to_inner_join_rewrite = 2 for comma join --- src/Core/Settings.h | 2 +- src/Interpreters/CrossToInnerJoinVisitor.cpp | 20 ++++++++++++++--- ...4_setting_cross_to_inner_rewrite.reference | 7 ++++++ .../02364_setting_cross_to_inner_rewrite.sql | 22 +++++++++++++++++++ 4 files changed, 47 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.reference create mode 100644 tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bda72f089eb..ea08658d851 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -758,7 +758,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, output_format_pretty_row_numbers, false, "Add row numbers before each row for pretty output format", 0) \ M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \ \ - M(UInt64, cross_to_inner_join_rewrite, 1, "Use inner join instead of comma/cross join if possible. Possible values: 0 - no rewrite, 1 - apply if possible, 2 - force rewrite all cross joins", 0) \ + M(UInt64, cross_to_inner_join_rewrite, 2, "Use inner join instead of comma/cross join if possible. Possible values: 0 - no rewrite, 1 - apply if possible for comma/cross, 2 - force rewrite all comma joins, cross - if possible", 0) \ \ M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ M(Bool, output_format_arrow_string_as_string, false, "Use Arrow String type instead of Binary for String columns", 0) \ diff --git a/src/Interpreters/CrossToInnerJoinVisitor.cpp b/src/Interpreters/CrossToInnerJoinVisitor.cpp index d438ea9394e..be6c1101fb4 100644 --- a/src/Interpreters/CrossToInnerJoinVisitor.cpp +++ b/src/Interpreters/CrossToInnerJoinVisitor.cpp @@ -39,7 +39,10 @@ struct JoinedElement : element(table_element) { if (element.table_join) + { join = element.table_join->as(); + original_kind = join->kind; + } } void checkTableName(const DatabaseAndTableWithAlias & table, const String & current_database) const @@ -61,6 +64,8 @@ struct JoinedElement join->kind = ASTTableJoin::Kind::Cross; } + ASTTableJoin::Kind getOriginalKind() const { return original_kind; } + bool rewriteCrossToInner(ASTPtr on_expression) { if (join->kind != ASTTableJoin::Kind::Cross) @@ -83,6 +88,8 @@ struct JoinedElement private: const ASTTablesInSelectQueryElement & element; ASTTableJoin * join = nullptr; + + ASTTableJoin::Kind original_kind; }; bool isAllowedToRewriteCrossJoin(const ASTPtr & node, const Aliases & aliases) @@ -251,10 +258,17 @@ void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & da } } - if (data.cross_to_inner_join_rewrite > 1 && !rewritten) + if (joined.getOriginalKind() == ASTTableJoin::Kind::Comma && + data.cross_to_inner_join_rewrite > 1 && + !rewritten) { - throw Exception(ErrorCodes::INCORRECT_QUERY, "Failed to rewrite '{} WHERE {}' to INNER JOIN", - query_before, queryToString(select.where())); + throw Exception( + ErrorCodes::INCORRECT_QUERY, + "Failed to rewrite comma join to INNER. " + "Please, try to simplify WHERE section " + "or set the setting `cross_to_inner_join_rewrite` to 1 to allow slow CROSS JOIN for this case" + "(cannot rewrite '{} WHERE {}' to INNER JOIN)", + query_before, queryToString(select.where())); } } } diff --git a/tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.reference b/tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.reference new file mode 100644 index 00000000000..fcb49fa9945 --- /dev/null +++ b/tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.reference @@ -0,0 +1,7 @@ +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.sql b/tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.sql new file mode 100644 index 00000000000..8deddbaa037 --- /dev/null +++ b/tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.sql @@ -0,0 +1,22 @@ + + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 ( x Int ) Engine = Memory; +INSERT INTO t1 VALUES ( 1 ), ( 2 ), ( 3 ); + +CREATE TABLE t2 ( x Int ) Engine = Memory; +INSERT INTO t2 VALUES ( 2 ), ( 3 ), ( 4 ); + +SET cross_to_inner_join_rewrite = 1; +SELECT count() = 1 FROM t1, t2 WHERE t1.x > t2.x; +SELECT count() = 1 2ROM t1, t2 WHERE t1.x = t2.x; +SELECT count() = 1 2ROM t1 CROSS JOIN t2 WHERE t1.x = t2.x; +SELECT count() = 1 FROM t1 CROSS JOIN t2 WHERE t1.x > t2.x; + +SET cross_to_inner_join_rewrite = 2; +SELECT count() = 1 FROM t1, t2 WHERE t1.x > t2.x; -- { serverError INCORRECT_QUERY } +SELECT count() = 2 FROM t1, t2 WHERE t1.x = t2.x; +SELECT count() = 2 FROM t1 CROSS JOIN t2 WHERE t1.x = t2.x; +SELECT count() = 1 FROM t1 CROSS JOIN t2 WHERE t1.x > t2.x; -- do not force rewrite explicit CROSS From c0547bb09cc7d450003bcdaf63d903272cefdab3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 19 Jul 2022 12:32:04 +0200 Subject: [PATCH 399/659] fix space --- src/Interpreters/CrossToInnerJoinVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/CrossToInnerJoinVisitor.cpp b/src/Interpreters/CrossToInnerJoinVisitor.cpp index be6c1101fb4..cfa979f4036 100644 --- a/src/Interpreters/CrossToInnerJoinVisitor.cpp +++ b/src/Interpreters/CrossToInnerJoinVisitor.cpp @@ -266,7 +266,7 @@ void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & da ErrorCodes::INCORRECT_QUERY, "Failed to rewrite comma join to INNER. " "Please, try to simplify WHERE section " - "or set the setting `cross_to_inner_join_rewrite` to 1 to allow slow CROSS JOIN for this case" + "or set the setting `cross_to_inner_join_rewrite` to 1 to allow slow CROSS JOIN for this case " "(cannot rewrite '{} WHERE {}' to INNER JOIN)", query_before, queryToString(select.where())); } From b58f9adce9b9a36c8f83b55231e177582584d395 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 19 Jul 2022 12:32:27 +0200 Subject: [PATCH 400/659] Fix tests for cross_to_inner_join_rewrite --- tests/queries/0_stateless/00849_multiple_comma_join_2.sql | 1 + tests/queries/0_stateless/00950_test_gorilla_codec.sql | 2 ++ tests/queries/0_stateless/01095_tpch_like_smoke.sql | 2 ++ tests/queries/0_stateless/01479_cross_join_9855.sql | 2 ++ tests/queries/0_stateless/01911_logical_error_minus.sql | 2 ++ tests/queries/0_stateless/02313_test_fpc_codec.sql | 2 ++ .../0_stateless/02364_setting_cross_to_inner_rewrite.sql | 4 ++-- 7 files changed, 13 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00849_multiple_comma_join_2.sql b/tests/queries/0_stateless/00849_multiple_comma_join_2.sql index eabede3ff00..eb803450ff7 100644 --- a/tests/queries/0_stateless/00849_multiple_comma_join_2.sql +++ b/tests/queries/0_stateless/00849_multiple_comma_join_2.sql @@ -1,5 +1,6 @@ SET enable_optimize_predicate_expression = 0; SET convert_query_to_cnf = 0; +SET cross_to_inner_join_rewrite = 1; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/00950_test_gorilla_codec.sql b/tests/queries/0_stateless/00950_test_gorilla_codec.sql index a6e0f1d7b11..e9582480bcb 100644 --- a/tests/queries/0_stateless/00950_test_gorilla_codec.sql +++ b/tests/queries/0_stateless/00950_test_gorilla_codec.sql @@ -1,5 +1,7 @@ DROP TABLE IF EXISTS codecTest; +SET cross_to_inner_join_rewrite = 1; + CREATE TABLE codecTest ( key UInt64, name String, diff --git a/tests/queries/0_stateless/01095_tpch_like_smoke.sql b/tests/queries/0_stateless/01095_tpch_like_smoke.sql index 5971178ade5..1ac9ec229f0 100644 --- a/tests/queries/0_stateless/01095_tpch_like_smoke.sql +++ b/tests/queries/0_stateless/01095_tpch_like_smoke.sql @@ -7,6 +7,8 @@ DROP TABLE IF EXISTS lineitem; DROP TABLE IF EXISTS nation; DROP TABLE IF EXISTS region; +SET cross_to_inner_join_rewrite = 1; + CREATE TABLE part ( p_partkey Int32, -- PK diff --git a/tests/queries/0_stateless/01479_cross_join_9855.sql b/tests/queries/0_stateless/01479_cross_join_9855.sql index 0b549619489..6dc76f22057 100644 --- a/tests/queries/0_stateless/01479_cross_join_9855.sql +++ b/tests/queries/0_stateless/01479_cross_join_9855.sql @@ -1,3 +1,5 @@ +SET cross_to_inner_join_rewrite = 1; + SELECT count() FROM numbers(4) AS n1, numbers(3) AS n2 WHERE n1.number > (select avg(n.number) from numbers(3) n); diff --git a/tests/queries/0_stateless/01911_logical_error_minus.sql b/tests/queries/0_stateless/01911_logical_error_minus.sql index 9813c1a8a5d..3dcdedd38f5 100644 --- a/tests/queries/0_stateless/01911_logical_error_minus.sql +++ b/tests/queries/0_stateless/01911_logical_error_minus.sql @@ -1,6 +1,8 @@ -- This test case is almost completely generated by fuzzer. -- It appeared to trigger assertion. +SET cross_to_inner_join_rewrite = 1; + DROP TABLE IF EXISTS codecTest; CREATE TABLE codecTest ( diff --git a/tests/queries/0_stateless/02313_test_fpc_codec.sql b/tests/queries/0_stateless/02313_test_fpc_codec.sql index 3b1127350f0..4fe54b87c9c 100644 --- a/tests/queries/0_stateless/02313_test_fpc_codec.sql +++ b/tests/queries/0_stateless/02313_test_fpc_codec.sql @@ -1,5 +1,7 @@ DROP TABLE IF EXISTS codecTest; +SET cross_to_inner_join_rewrite = 1; + CREATE TABLE codecTest ( key UInt64, name String, diff --git a/tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.sql b/tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.sql index 8deddbaa037..cdbac93937e 100644 --- a/tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.sql +++ b/tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.sql @@ -11,8 +11,8 @@ INSERT INTO t2 VALUES ( 2 ), ( 3 ), ( 4 ); SET cross_to_inner_join_rewrite = 1; SELECT count() = 1 FROM t1, t2 WHERE t1.x > t2.x; -SELECT count() = 1 2ROM t1, t2 WHERE t1.x = t2.x; -SELECT count() = 1 2ROM t1 CROSS JOIN t2 WHERE t1.x = t2.x; +SELECT count() = 2 FROM t1, t2 WHERE t1.x = t2.x; +SELECT count() = 2 FROM t1 CROSS JOIN t2 WHERE t1.x = t2.x; SELECT count() = 1 FROM t1 CROSS JOIN t2 WHERE t1.x > t2.x; SET cross_to_inner_join_rewrite = 2; From 02ed126533129b69c434e4a9fe62da29989c97da Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 19 Jul 2022 18:44:31 +0200 Subject: [PATCH 401/659] Update src/Common/TraceSender.cpp Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- src/Common/TraceSender.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/TraceSender.cpp b/src/Common/TraceSender.cpp index f1287b11ac4..ad88e508d06 100644 --- a/src/Common/TraceSender.cpp +++ b/src/Common/TraceSender.cpp @@ -48,7 +48,8 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int if (CurrentThread::isInitialized()) { query_id = CurrentThread::getQueryId(); - query_id = std::string_view(query_id.data(), std::min(query_id.size(), QUERY_ID_MAX_LEN)); + if (query_id.size() > QUERY_ID_MAX_LEN) + query_id.remove_suffix(query_id.size() - QUERY_ID_MAX_LEN); thread_id = CurrentThread::get().thread_id; } From 51396aa08326e6cebb8e9562016d9e949559a810 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Jul 2022 16:46:52 +0000 Subject: [PATCH 402/659] Fix scale for DateTime64 dictionary attribute. --- src/DataTypes/DataTypesDecimal.h | 14 +++++++++++--- src/DataTypes/getLeastSupertype.cpp | 2 +- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/src/DataTypes/DataTypesDecimal.h b/src/DataTypes/DataTypesDecimal.h index 00b5e2b9e37..617bffbb8b6 100644 --- a/src/DataTypes/DataTypesDecimal.h +++ b/src/DataTypes/DataTypesDecimal.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -13,6 +14,7 @@ namespace DB namespace ErrorCodes { extern const int DECIMAL_OVERFLOW; + extern const int LOGICAL_ERROR; } /// Implements Decimal(P, S), where P is precision, S is scale. @@ -58,7 +60,7 @@ inline const DataTypeDecimal * checkDecimal(const IDataType & data_type) return typeid_cast *>(&data_type); } -inline UInt32 getDecimalScale(const IDataType & data_type, UInt32 default_value = std::numeric_limits::max()) +inline UInt32 getDecimalScale(const IDataType & data_type) //, UInt32 default_value = std::numeric_limits::max()) { if (const auto * decimal_type = checkDecimal(data_type)) return decimal_type->getScale(); @@ -68,7 +70,10 @@ inline UInt32 getDecimalScale(const IDataType & data_type, UInt32 default_value return decimal_type->getScale(); if (const auto * decimal_type = checkDecimal(data_type)) return decimal_type->getScale(); - return default_value; + if (const auto * date_time_type = typeid_cast(&data_type)) + return date_time_type->getScale(); + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get decimal scale from type {}", data_type.getName()); } inline UInt32 getDecimalPrecision(const IDataType & data_type) @@ -81,7 +86,10 @@ inline UInt32 getDecimalPrecision(const IDataType & data_type) return decimal_type->getPrecision(); if (const auto * decimal_type = checkDecimal(data_type)) return decimal_type->getPrecision(); - return 0; + if (const auto * date_time_type = typeid_cast(&data_type)) + return date_time_type->getPrecision(); + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get decimal precision from type {}", data_type.getName()); } template diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 65ed37f1dcf..e09b3cbe938 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -554,7 +554,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types) UInt32 max_scale = 0; for (const auto & type : types) { - UInt32 scale = getDecimalScale(*type, 0); + UInt32 scale = getDecimalScale(*type); if (scale > max_scale) max_scale = scale; } From 8f6cc6971b955a7fe78645ed867901143aa07ccc Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 19 Jul 2022 14:24:00 -0400 Subject: [PATCH 403/659] add comments for readBig --- src/Compression/CompressedReadBufferFromFile.cpp | 10 +++++++++- src/Compression/CompressionCodecDeflateQpl.cpp | 1 + 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index f34665a2fb4..3d2db3cdf93 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -91,6 +91,7 @@ void CompressedReadBufferFromFile::seek(size_t offset_in_compressed_file, size_t size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) { size_t bytes_read = 0; + /// The codec mode is only relevant for codecs which support hardware offloading. ICompressionCodec::CodecMode decompress_mode = ICompressionCodec::CodecMode::Synchronous; bool read_tail = false; @@ -104,15 +105,21 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) size_t size_decompressed = 0; size_t size_compressed_without_checksum = 0; + ///Try to read block which is entirely located in a single 'compressed_in->' buffer. size_t new_size_compressed = readCompressedDataBlockForAsynchronous(size_decompressed, size_compressed_without_checksum); if (new_size_compressed) { + /// Current block is entirely located in a single 'compressed_in->' buffer. + /// We can set asynchronous decompression mode if supported to boost performance. decompress_mode = ICompressionCodec::CodecMode::Asynchronous; } else { - flushAsynchronousDecompressRequests(); /// here switch to unhold block in compress_in, we must flush for previous blocks completely hold in compress_in + /// Current block cannot be decompressed asynchronously, means it probably span across two compressed_in buffers. + /// Meanwhile, asynchronous requests for previous blocks should be flushed if any. + flushAsynchronousDecompressRequests(); + /// Fallback to generic API new_size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum, false); decompress_mode = ICompressionCodec::CodecMode::Synchronous; } @@ -169,6 +176,7 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) } } + /// Here we must make sure all asynchronous requests above are completely done. flushAsynchronousDecompressRequests(); if (read_tail) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 341b3db52cd..aae415aac69 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -401,6 +401,7 @@ void CompressionCodecDeflateQpl::flushAsynchronousDecompressRequests() { if (DeflateQplJobHWPool::instance().isJobPoolReady()) hw_codec->flushAsynchronousDecompressRequests(); + /// After flush previous all async requests, we must restore mode to be synchronous by default. setDecompressMode(CodecMode::Synchronous); } void registerCodecDeflateQpl(CompressionCodecFactory & factory) From b0c6fcf3a03c46f04c5124a27d0b65584a8fccab Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 19 Jul 2022 15:30:25 -0400 Subject: [PATCH 404/659] demoted log to INFO when the job pool is exhausted --- src/Compression/CompressionCodecDeflateQpl.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index aae415aac69..7d566e14a87 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -145,7 +145,7 @@ Int32 HardwareCodecDeflateQpl::doCompressData(const char * source, UInt32 source UInt32 compressed_size = 0; if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(job_id))) { - LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doCompressData->acquireJob fail, probably job pool exhausted)"); + LOG_INFO(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doCompressData->acquireJob fail, probably job pool exhausted)"); return RET_ERROR; } @@ -178,7 +178,7 @@ Int32 HardwareCodecDeflateQpl::doDecompressDataSynchronous(const char * source, UInt32 decompressed_size = 0; if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(job_id))) { - LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataSynchronous->acquireJob fail, probably job pool exhausted)"); + LOG_INFO(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataSynchronous->acquireJob fail, probably job pool exhausted)"); return RET_ERROR; } @@ -213,7 +213,7 @@ Int32 HardwareCodecDeflateQpl::doDecompressDataAsynchronous(const char * source, qpl_job * job_ptr = nullptr; if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(job_id))) { - LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataAsynchronous->acquireJob fail, probably job pool exhausted)"); + LOG_INFO(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataAsynchronous->acquireJob fail, probably job pool exhausted)"); return RET_ERROR; } From 94332d876a3615016ffbf4010260c0286411a120 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 19 Jul 2022 17:48:41 -0400 Subject: [PATCH 405/659] Fixed type&style issue --- src/Compression/CompressedReadBufferBase.cpp | 8 ++++---- src/Compression/CompressedReadBufferBase.h | 2 +- src/Compression/CompressionCodecDeflateQpl.cpp | 8 ++++---- src/Compression/CompressionCodecDeflateQpl.h | 3 ++- src/Compression/ICompressionCodec.h | 2 +- 5 files changed, 12 insertions(+), 11 deletions(-) diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 30e5d0c6655..244450414ba 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -151,8 +151,6 @@ static void readHeaderAndGetCodecAndSize( if (size_compressed_without_checksum < header_size) throw Exception("Can't decompress data: the compressed data size (" + toString(size_compressed_without_checksum) + ", this should include header size) is less than the header size (" + toString(header_size) + ")", ErrorCodes::CORRUPTED_DATA); - - ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum)); } /// Read compressed data into compressed_buffer. Get size of decompressed data from block header. Checksum if need. @@ -199,6 +197,7 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum); } + ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum)); return size_compressed_without_checksum + sizeof(Checksum); } @@ -231,11 +230,14 @@ size_t CompressedReadBufferBase::readCompressedDataBlockForAsynchronous(size_t & compressed_in->position() -= header_size; compressed_buffer = compressed_in->position(); compressed_in->position() += size_compressed_without_checksum; + if (!disable_checksum) { Checksum & checksum = *reinterpret_cast(own_compressed_buffer.data()); validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum); } + + ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum)); return size_compressed_without_checksum + sizeof(Checksum); } else @@ -273,14 +275,12 @@ static void readHeaderAndGetCodec(const char * compressed_buffer, size_t size_de } } - void CompressedReadBufferBase::decompressTo(char * to, size_t size_decompressed, size_t size_compressed_without_checksum) { readHeaderAndGetCodec(compressed_buffer, size_decompressed, codec, allow_different_codecs); codec->decompress(compressed_buffer, size_compressed_without_checksum, to); } - void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum) { readHeaderAndGetCodec(compressed_buffer, size_decompressed, codec, allow_different_codecs); diff --git a/src/Compression/CompressedReadBufferBase.h b/src/Compression/CompressedReadBufferBase.h index 0457d59058c..4d76fe72fdc 100644 --- a/src/Compression/CompressedReadBufferBase.h +++ b/src/Compression/CompressedReadBufferBase.h @@ -42,7 +42,7 @@ protected: /// Read compressed data into compressed_buffer for asynchronous decompression to avoid the situation of "read compressed block across the compressed_in". /// /// Compressed block may not be completely contained in "compressed_in" buffer which means compressed block may be read across the "compressed_in". - /// For native LZ4/ZSTD, it has no probem in facing situation above because they are sychronous. + /// For native LZ4/ZSTD, it has no problem in facing situation above because they are synchronous. /// But for asynchronous decompression, such as QPL deflate, it requires source and target buffer for decompression can not be overwritten until execution complete. /// /// Returns number of compressed bytes read. diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 7d566e14a87..81ec7ee5dca 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -109,11 +109,11 @@ bool DeflateQplJobHWPool::tryLockJob(UInt32 index) return hw_job_ptr_locks[index].compare_exchange_strong(expected, true); } - void DeflateQplJobHWPool::unLockJob(UInt32 index) - { +void DeflateQplJobHWPool::unLockJob(UInt32 index) +{ assert(index < MAX_HW_JOB_NUMBER); hw_job_ptr_locks[index].store(false); - } +} //HardwareCodecDeflateQpl HardwareCodecDeflateQpl::HardwareCodecDeflateQpl() @@ -374,7 +374,7 @@ void CompressionCodecDeflateQpl::doDecompressData(const char * source, UInt32 so if (DeflateQplJobHWPool::instance().isJobPoolReady()) { res = hw_codec->doDecompressDataSynchronous(source, source_size, dest, uncompressed_size); - if(res == HardwareCodecDeflateQpl::RET_ERROR) + if (res == HardwareCodecDeflateQpl::RET_ERROR) sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); } else diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 125f35eb015..c15f537fd3f 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -12,7 +12,8 @@ class Logger; namespace DB { -/// DeflateQplJobHWPool is resource pool for provide the job objects which is required to save context infomation during offload asynchronous compression to IAA. +/// DeflateQplJobHWPool is resource pool to provide the job objects. +/// Job object is used for storing context information during offloading compression job to HW Accelerator. class DeflateQplJobHWPool { public: diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 51d08ae8f33..f40404a84f3 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -52,7 +52,7 @@ public: /// Synchronous mode which is commonly used by default; /// --- For the codec with HW decompressor, it means submit request to HW and busy wait till complete. /// Asynchronous mode which required HW decompressor support; - /// --- For the codec with HW decompressor, it means submit request to HW and return immeditately. + /// --- For the codec with HW decompressor, it means submit request to HW and return immediately. /// --- Must be used in pair with flushAsynchronousDecompressRequests. /// SoftwareFallback mode is exclusively defined for the codec with HW decompressor, enable its capability of "fallback to SW codec". enum class CodecMode From 5fbc5b25de08e7958efa75b21ecdf49a8a14506b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Jul 2022 16:50:32 +0000 Subject: [PATCH 406/659] Added test. --- ...ctionary_datetime_64_attribute_crash.reference | 2 ++ ...364_dictionary_datetime_64_attribute_crash.sql | 15 +++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/02364_dictionary_datetime_64_attribute_crash.reference create mode 100644 tests/queries/0_stateless/02364_dictionary_datetime_64_attribute_crash.sql diff --git a/tests/queries/0_stateless/02364_dictionary_datetime_64_attribute_crash.reference b/tests/queries/0_stateless/02364_dictionary_datetime_64_attribute_crash.reference new file mode 100644 index 00000000000..cd97db4debd --- /dev/null +++ b/tests/queries/0_stateless/02364_dictionary_datetime_64_attribute_crash.reference @@ -0,0 +1,2 @@ +2022-01-24 02:30:00.008122000 +1 diff --git a/tests/queries/0_stateless/02364_dictionary_datetime_64_attribute_crash.sql b/tests/queries/0_stateless/02364_dictionary_datetime_64_attribute_crash.sql new file mode 100644 index 00000000000..77fc9e1183b --- /dev/null +++ b/tests/queries/0_stateless/02364_dictionary_datetime_64_attribute_crash.sql @@ -0,0 +1,15 @@ +create table dat (blockNum Decimal(10,0), eventTimestamp DateTime64(9)) Engine=MergeTree() primary key eventTimestamp; +insert into dat values (1, '2022-01-24 02:30:00.008122000'); + +CREATE DICTIONARY datDictionary +( + `blockNum` Decimal(10, 0), + `eventTimestamp` DateTime64(9) +) +PRIMARY KEY blockNum +SOURCE(CLICKHOUSE(TABLE 'dat')) +LIFETIME(MIN 0 MAX 1000) +LAYOUT(FLAT()); + +select (select eventTimestamp from datDictionary); +select count(*) from dat where eventTimestamp >= (select eventTimestamp from datDictionary); From 775b96fe13b5a002bcbcf6128bb239280aed3082 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Jul 2022 17:05:44 +0000 Subject: [PATCH 407/659] Remove comment --- src/DataTypes/DataTypesDecimal.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/DataTypesDecimal.h b/src/DataTypes/DataTypesDecimal.h index 617bffbb8b6..7bcc6593435 100644 --- a/src/DataTypes/DataTypesDecimal.h +++ b/src/DataTypes/DataTypesDecimal.h @@ -60,7 +60,7 @@ inline const DataTypeDecimal * checkDecimal(const IDataType & data_type) return typeid_cast *>(&data_type); } -inline UInt32 getDecimalScale(const IDataType & data_type) //, UInt32 default_value = std::numeric_limits::max()) +inline UInt32 getDecimalScale(const IDataType & data_type) { if (const auto * decimal_type = checkDecimal(data_type)) return decimal_type->getScale(); From a4450225c76c4f1bac16c531ef931da17f0c9d89 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 19 Jul 2022 17:14:44 +0000 Subject: [PATCH 408/659] Disable digest by default --- src/Coordination/KeeperServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 7c6ed227a06..b5e5eac7bd7 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -111,7 +111,7 @@ KeeperServer::KeeperServer( configuration_and_settings_->snapshot_storage_path, coordination_settings, checkAndGetSuperdigest(configuration_and_settings_->super_digest), - config.getBool("keeper_server.digest_enabled", true))) + config.getBool("keeper_server.digest_enabled", false))) , state_manager(nuraft::cs_new( server_id, "keeper_server", configuration_and_settings_->log_storage_path, configuration_and_settings_->state_file_path, config, coordination_settings)) , log(&Poco::Logger::get("KeeperServer")) From 891dbccdc73ea9d1ddae484ea01107bd5cb8108e Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 19 Jul 2022 19:18:42 +0200 Subject: [PATCH 409/659] Update LRUFileCache.cpp --- src/Common/LRUFileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/LRUFileCache.cpp b/src/Common/LRUFileCache.cpp index b7472362823..45977b20dda 100644 --- a/src/Common/LRUFileCache.cpp +++ b/src/Common/LRUFileCache.cpp @@ -842,7 +842,7 @@ void LRUFileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_l if (!files.empty()) throw Exception( - REMOTE_FS_OBJECT_CACHE_ERROR, + ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Cache initialization is partially made. " "This can be a result of a failed first attempt to initialize cache. " "Please, check log for error messages"); From 5c16d6b55302f235e844fe2fefb24108bf26d943 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 19 Jul 2022 19:21:30 +0000 Subject: [PATCH 410/659] Fix WriteBuffer finalize in destructor when cacnel query --- src/Processors/Formats/IOutputFormat.cpp | 4 +++- src/Storages/HDFS/StorageHDFS.cpp | 7 +++++++ src/Storages/StorageFile.cpp | 7 +++++++ src/Storages/StorageS3.cpp | 7 +++++++ src/Storages/StorageURL.cpp | 7 +++++++ src/Storages/StorageURL.h | 1 + 6 files changed, 32 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index 3c4e6861151..47ebaa9c5f5 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -73,7 +73,6 @@ void IOutputFormat::work() setRowsBeforeLimit(rows_before_limit_counter->get()); finalize(); - finalized = true; return; } @@ -120,9 +119,12 @@ void IOutputFormat::write(const Block & block) void IOutputFormat::finalize() { + if (finalized) + return; writePrefixIfNot(); writeSuffixIfNot(); finalizeImpl(); + finalized = true; } } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 57e893e9683..1e9f9286633 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -430,6 +430,13 @@ public: writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } + void onCancel() override + { + if (!writer) + return; + onFinish(); + } + void onException() override { if (!writer) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index d138104018a..30e5042fb06 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -813,6 +813,13 @@ public: writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } + void onCancel() override + { + if (!writer) + return; + onFinish(); + } + void onException() override { if (!writer) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 130bc75a65c..6a7c682199d 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -602,6 +602,13 @@ public: writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } + void onCancel() override + { + if (!writer) + return; + onFinish(); + } + void onException() override { if (!writer) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 15ae23305f3..cdc288ba788 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -450,6 +450,13 @@ void StorageURLSink::consume(Chunk chunk) writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } +void StorageURLSink::onCancel() +{ + if (!writer) + return; + onFinish(); +} + void StorageURLSink::onException() { if (!writer) diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 79371242bb1..320c6258ee5 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -114,6 +114,7 @@ public: std::string getName() const override { return "StorageURLSink"; } void consume(Chunk chunk) override; + void onCancel() override; void onException() override; void onFinish() override; From be1d40916a095962d29f1b4372b8a19381b27ffb Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 19 Jul 2022 18:22:21 -0300 Subject: [PATCH 411/659] Update formats.md --- docs/en/interfaces/formats.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 5d8ed9cdacd..d4449a6aa8e 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1632,6 +1632,8 @@ kafka_topic_list = 'topic1', kafka_group_name = 'group1', kafka_format = 'AvroConfluent'; +-- for debug purposes you can set format_avro_schema_registry_url in session. +-- this way cannot be used in production SET format_avro_schema_registry_url = 'http://schema-registry'; SELECT * FROM topic1_stream; From 1d5b2b770e82fe48f877777c8a1a2c219b51600d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 19 Jul 2022 18:23:02 -0300 Subject: [PATCH 412/659] Update formats.md --- docs/en/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index d4449a6aa8e..e499849426b 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1632,7 +1632,7 @@ kafka_topic_list = 'topic1', kafka_group_name = 'group1', kafka_format = 'AvroConfluent'; --- for debug purposes you can set format_avro_schema_registry_url in session. +-- for debug purposes you can set format_avro_schema_registry_url in a session. -- this way cannot be used in production SET format_avro_schema_registry_url = 'http://schema-registry'; From 150e058be959ea8fc641417cc2e8277edac291b5 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 20 Jul 2022 09:04:18 +0200 Subject: [PATCH 413/659] lockTablesForReading() comes back. --- src/Backups/BackupEntriesCollector.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 6c1b0413368..d2e4b1f8c4b 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -239,6 +239,7 @@ bool BackupEntriesCollector::tryGatherMetadataAndCompareWithPrevious(std::option table_infos.clear(); gatherDatabasesMetadata(); gatherTablesMetadata(); + lockTablesForReading(); } catch (Exception & e) { From 18eb8b6d48394e34870a9a5ae60d5bb56afb74fe Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 19 Jul 2022 14:31:56 +0300 Subject: [PATCH 414/659] Fix UB (stack-use-after-scope) in extactAll() After #37544 OptimizedRegularExpressionImpl started to be moved, but StringSearcher is not copyable since it holds pointers that goes out of scope after move (before Regexps::get() returns std::shared_ptr but it had been replaced with Regexps::createRegexp() that returns Regexp object).
ASan report ==48348==ERROR: AddressSanitizer: stack-use-after-scope on address 0x7fff577239a9 at pc 0x00001518209b bp 0x7fff57723820 sp 0x7fff57723818 READ of size 1 at 0x7fff577239a9 thread T0 0 0x1518209a in char8_t const* DB::StringSearcher::search(char8_t const*, char8_t const*) const /bld/./src/Common/StringSearcher.h:730:41 1 0x1518dd3f in char8_t const* DB::StringSearcher::search(char8_t const*, unsigned long) const /bld/./src/Common/StringSearcher.h:751:16 2 0x1518dd3f in OptimizedRegularExpressionImpl::match(char const*, unsigned long, std::__1::vector >&, unsigned int) const /bld/./src/Common/OptimizedRegularExpression.cpp:463:54 3 0x1811cb42 in DB::ExtractAllImpl::get(char const*&, char const*&) /bld/./src/Functions/FunctionsStringArray.h:588:18 4 0x1811aa62 in DB::FunctionTokens::executeImpl(std::__1::vector > const&, std::__1::shared_ptr const&, unsigned long) const /bld/./src/Functions/FunctionsStringArray.h:704:30 5 0x14fe17b4 in DB::IFunction::executeImplDryRun(std::__1::vector > const&, std::__1::shared_ptr const&, unsigned long) const /bld/./src/Functions/IFunction.h:409:16 Address 0x7fff577239a9 is located in stack of thread T0 at offset 201 in frame 0 0x1518d98f in OptimizedRegularExpressionImpl::match(char const*, unsigned long, std::__1::vector >&, unsigned int) const /bld/./src/Common/OptimizedRegularExpression.cpp:439
CI: https://s3.amazonaws.com/clickhouse-test-reports/39342/c6f7698f9ad6ae22199182ebf7c3b2dac77d69d8/fuzzer_astfuzzerasan,actions//report.html Fixes: #37544 (cc @rschu1ze) Signed-off-by: Azat Khuzhin --- src/Common/OptimizedRegularExpression.cpp | 17 +++++++++++++++++ src/Common/OptimizedRegularExpression.h | 3 +++ .../02370_extractAll_regress.reference | 1 + .../0_stateless/02370_extractAll_regress.sql | 5 +++++ 4 files changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/02370_extractAll_regress.reference create mode 100644 tests/queries/0_stateless/02370_extractAll_regress.sql diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index cfc364929a3..60efab69433 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -342,6 +342,23 @@ OptimizedRegularExpressionImpl::OptimizedRegularExpressionImpl(cons } } +template +OptimizedRegularExpressionImpl::OptimizedRegularExpressionImpl(OptimizedRegularExpressionImpl && rhs) noexcept + : is_trivial(rhs.is_trivial) + , required_substring_is_prefix(rhs.required_substring_is_prefix) + , is_case_insensitive(rhs.is_case_insensitive) + , required_substring(std::move(rhs.required_substring)) + , re2(std::move(rhs.re2)) + , number_of_subpatterns(rhs.number_of_subpatterns) +{ + if (!required_substring.empty()) + { + if (is_case_insensitive) + case_insensitive_substring_searcher.emplace(required_substring.data(), required_substring.size()); + else + case_sensitive_substring_searcher.emplace(required_substring.data(), required_substring.size()); + } +} template bool OptimizedRegularExpressionImpl::match(const char * subject, size_t subject_size) const diff --git a/src/Common/OptimizedRegularExpression.h b/src/Common/OptimizedRegularExpression.h index eaa7b06e309..dad8706a50d 100644 --- a/src/Common/OptimizedRegularExpression.h +++ b/src/Common/OptimizedRegularExpression.h @@ -56,6 +56,9 @@ public: using StringPieceType = std::conditional_t; OptimizedRegularExpressionImpl(const std::string & regexp_, int options = 0); /// NOLINT + /// StringSearcher store pointers to required_substring, it must be updated on move. + OptimizedRegularExpressionImpl(OptimizedRegularExpressionImpl && rhs) noexcept; + OptimizedRegularExpressionImpl(const OptimizedRegularExpressionImpl & rhs) = delete; bool match(const std::string & subject) const { diff --git a/tests/queries/0_stateless/02370_extractAll_regress.reference b/tests/queries/0_stateless/02370_extractAll_regress.reference new file mode 100644 index 00000000000..aad46128e52 --- /dev/null +++ b/tests/queries/0_stateless/02370_extractAll_regress.reference @@ -0,0 +1 @@ +{"a":"1","b":"2","c":"","d":"4"}{"a":"1","b":"2","c":"","d":"4"}{"a":"1","b":"2","c":"","d":"4"}{"a":"1","b":"2","c":"","d":"4"} ['a','b','c','d','a','b','c','d','a','b','c','d','a','b','c','d'] [':"',':"',':"',':"',':"',':"',':"',':"',':"',':"',':"',':"',':"',':"',':"',':"'] diff --git a/tests/queries/0_stateless/02370_extractAll_regress.sql b/tests/queries/0_stateless/02370_extractAll_regress.sql new file mode 100644 index 00000000000..6d255124948 --- /dev/null +++ b/tests/queries/0_stateless/02370_extractAll_regress.sql @@ -0,0 +1,5 @@ +-- Regression for UB (stack-use-after-scope) in extactAll() +SELECT + '{"a":"1","b":"2","c":"","d":"4"}{"a":"1","b":"2","c":"","d":"4"}{"a":"1","b":"2","c":"","d":"4"}{"a":"1","b":"2","c":"","d":"4"}' AS json, + extractAll(json, '"([^"]*)":') AS keys, + extractAll(json, ':"\0[^"]*)"') AS values; From df53a556eb398c6b62ddfeef9a2b3e05f657537c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 20 Jul 2022 07:59:02 +0000 Subject: [PATCH 415/659] Fix test failures --- src/IO/Operators.h | 2 ++ src/IO/tests/gtest_manip.cpp | 5 +++++ 2 files changed, 7 insertions(+) diff --git a/src/IO/Operators.h b/src/IO/Operators.h index 93807ea39fc..06ff20c43e8 100644 --- a/src/IO/Operators.h +++ b/src/IO/Operators.h @@ -54,7 +54,9 @@ template WriteBuffer & operator<< (QuoteManipWriteBuffer buf, template WriteBuffer & operator<< (DoubleQuoteManipWriteBuffer buf, const T & x) { writeDoubleQuoted(x, buf.get()); return buf; } template WriteBuffer & operator<< (BinaryManipWriteBuffer buf, const T & x) { writeBinary(x, buf.get()); return buf; } +inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const String & x) { writeEscapedString(x, buf); return buf; } inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, std::string_view x) { writeEscapedString(x, buf); return buf; } +inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, StringRef x) { writeEscapedString(x.toView(), buf); return buf; } inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const char * x) { writeEscapedString(x, strlen(x), buf); return buf; } inline WriteBuffer & operator<< (QuoteManipWriteBuffer buf, const char * x) { writeAnyQuotedString<'\''>(x, x + strlen(x), buf.get()); return buf; } diff --git a/src/IO/tests/gtest_manip.cpp b/src/IO/tests/gtest_manip.cpp index 09cc3d16619..afcffc7dfb9 100644 --- a/src/IO/tests/gtest_manip.cpp +++ b/src/IO/tests/gtest_manip.cpp @@ -32,6 +32,11 @@ TEST(OperatorsManipTest, EscapingTest) checkString(sv1, escape, "Hello \\'world\\'"); std::string_view sv2 = s2; checkString(sv2, escape, "Hello \\\\world\\\\"); // NOLINT + + StringRef sr1 = s1; + checkString(sr1, escape, "Hello \\'world\\'"); + StringRef sr2 = s2; + checkString(sr2, escape, "Hello \\\\world\\\\"); // NOLINT } TEST(OperatorsManipTest, QuouteTest) From 0840dfe2e0011227a6aff6fcbe9d3b27755299db Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 20 Jul 2022 11:42:24 +0200 Subject: [PATCH 416/659] Update LRUFileCache.cpp --- src/Common/LRUFileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/LRUFileCache.cpp b/src/Common/LRUFileCache.cpp index 45977b20dda..6306b6de059 100644 --- a/src/Common/LRUFileCache.cpp +++ b/src/Common/LRUFileCache.cpp @@ -842,7 +842,7 @@ void LRUFileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_l if (!files.empty()) throw Exception( - ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, + ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Cache initialization is partially made. " "This can be a result of a failed first attempt to initialize cache. " "Please, check log for error messages"); From 8873d16afe96fbacf70fd116efafbcff945084dd Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 20 Jul 2022 11:51:55 +0200 Subject: [PATCH 417/659] Fix interactive client with older server after https://github.com/ClickHouse/ClickHouse/pull/38970 --- programs/client/Client.cpp | 47 +++++++++++++++++++++++--------------- programs/client/Client.h | 1 + src/Client/Suggest.h | 4 ++-- src/Core/ProtocolDefines.h | 4 +++- 4 files changed, 34 insertions(+), 22 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index cf9b7cbafea..74559920954 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -102,9 +102,34 @@ void Client::processError(const String & query) const } +void Client::showWarnings() +{ + try + { + std::vector messages = loadWarningMessages(); + if (!messages.empty()) + { + std::cout << "Warnings:" << std::endl; + for (const auto & message : messages) + std::cout << " * " << message << std::endl; + std::cout << std::endl; + } + } + catch (...) + { + /// Ignore exception + } +} + /// Make query to get all server warnings std::vector Client::loadWarningMessages() { + /// Older server versions cannot execute the query loading warnings. + constexpr UInt64 MIN_SERVER_REVISION_TO_LOAD_WARNINGS = DBMS_MIN_PROTOCOL_VERSION_WITH_VIEW_IF_PERMITTED; + + if (server_revision < MIN_SERVER_REVISION_TO_LOAD_WARNINGS) + return {}; + std::vector messages; connection->sendQuery(connection_parameters.timeouts, "SELECT * FROM viewIfPermitted(SELECT message FROM system.warnings ELSE null('message String'))", @@ -226,25 +251,9 @@ try connect(); - /// Load Warnings at the beginning of connection + /// Show warnings at the beginning of connection. if (is_interactive && !config().has("no-warnings")) - { - try - { - std::vector messages = loadWarningMessages(); - if (!messages.empty()) - { - std::cout << "Warnings:" << std::endl; - for (const auto & message : messages) - std::cout << " * " << message << std::endl; - std::cout << std::endl; - } - } - catch (...) - { - /// Ignore exception - } - } + showWarnings(); if (is_interactive && !delayed_interactive) { @@ -370,7 +379,7 @@ void Client::connect() } server_version = toString(server_version_major) + "." + toString(server_version_minor) + "." + toString(server_version_patch); - load_suggestions = is_interactive && (server_revision >= Suggest::MIN_SERVER_REVISION && !config().getBool("disable_suggestion", false)); + load_suggestions = is_interactive && (server_revision >= Suggest::MIN_SERVER_REVISION) && !config().getBool("disable_suggestion", false); if (server_display_name = connection->getServerDisplayName(connection_parameters.timeouts); server_display_name.empty()) server_display_name = config().getString("host", "localhost"); diff --git a/programs/client/Client.h b/programs/client/Client.h index 164b8e2ebaa..1fec282be51 100644 --- a/programs/client/Client.h +++ b/programs/client/Client.h @@ -45,6 +45,7 @@ protected: private: void printChangedSettings() const; + void showWarnings(); std::vector loadWarningMessages(); }; } diff --git a/src/Client/Suggest.h b/src/Client/Suggest.h index 65b60ceffc4..25d45f7ffaf 100644 --- a/src/Client/Suggest.h +++ b/src/Client/Suggest.h @@ -28,8 +28,8 @@ public: template void load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit); - /// Older server versions cannot execute the query above. - static constexpr int MIN_SERVER_REVISION = 54406; + /// Older server versions cannot execute the query loading suggestions. + static constexpr int MIN_SERVER_REVISION = DBMS_MIN_PROTOCOL_VERSION_WITH_VIEW_IF_PERMITTED; private: void fetch(IServerConnection & connection, const ConnectionTimeouts & timeouts, const std::string & query); diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 2df48a79776..584720694d7 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -52,8 +52,10 @@ /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54456 +#define DBMS_TCP_PROTOCOL_VERSION 54457 #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 #define DBMS_MIN_PROTOCOL_VERSION_WITH_PROFILE_EVENTS_IN_INSERT 54456 + +#define DBMS_MIN_PROTOCOL_VERSION_WITH_VIEW_IF_PERMITTED 54457 From e5e9f6e60bc63c5071493b8328833966d04238d7 Mon Sep 17 00:00:00 2001 From: Ladislav Snizek Date: Wed, 20 Jul 2022 11:53:33 +0200 Subject: [PATCH 418/659] Documentation: Correct http_receive_timeout and http_send_timeout defaults (changed in #31450) --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 75c2aa57b32..9f66d5d29a9 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2626,7 +2626,7 @@ Possible values: - Any positive integer. - 0 - Disabled (infinite timeout). -Default value: 1800. +Default value: 180. ## http_receive_timeout {#http_receive_timeout} @@ -2637,7 +2637,7 @@ Possible values: - Any positive integer. - 0 - Disabled (infinite timeout). -Default value: 1800. +Default value: 180. ## check_query_single_value_result {#check_query_single_value_result} From 4b46d80169469076552dc184a891e955163ca643 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 20 Jul 2022 18:03:34 +0800 Subject: [PATCH 419/659] update codes --- src/Storages/Hive/HiveFile.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 671cb11deaa..112798fea4e 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -88,7 +88,6 @@ std::optional IHiveFile::getRows() has_init_rows = true; } } - return rows; } From 28b1fe89395ee713008158b78b606e08f04c2fcf Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 20 Jul 2022 12:04:59 +0200 Subject: [PATCH 420/659] Update docs/en/operations/settings/settings.md Co-authored-by: Antonio Andelic --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f40683dd25b..0b6e45fbcf0 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3332,7 +3332,7 @@ Default value: `1GiB`. ## compatibility {#compatibility} This setting changes other settings according to provided ClickHouse version. -If we know that we changed some behaviour in ClickHouse by changing some settings in some version, this compatibility setting will control these settings if they were not set by user. +If a behaviour in ClickHouse was changed by using a different default value for some setting, this compatibility setting allows you to use default values from previous versions for all the settings that were not set by the user. This setting takes ClickHouse version number as a string, like `21.3`, `21.8`. Empty value means that this setting is disabled. From a1b63b4a02ab99e0137f060f224506c85cf373a5 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 20 Jul 2022 12:07:22 +0200 Subject: [PATCH 421/659] Fix style --- src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index 8ca874103a8..50145fd5bc0 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -30,6 +30,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int INCORRECT_DATA; } CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params_, const FormatSchemaInfo & info, const FormatSettings & format_settings_) From 840ace663b205f35a42b28228f165f8561e6bbb7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 20 Jul 2022 13:19:53 +0300 Subject: [PATCH 422/659] Update run.sh --- docker/test/stress/run.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 06bf05a1727..ffa0b12b8a3 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -362,6 +362,8 @@ else # FIXME Not sure if it's expected, but some tests from BC check may not be finished yet when we restarting server. # Let's just ignore all errors from queries ("} TCPHandler: Code:", "} executeQuery: Code:") # FIXME https://github.com/ClickHouse/ClickHouse/issues/39197 ("Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'") + # NOTE Incompatibility was introduced in https://github.com/ClickHouse/ClickHouse/pull/39263, it's expected + # ("This engine is deprecated and is not supported in transactions", "[Queue = DB::MergeMutateRuntimeQueue]: Code: 235. DB::Exception: Part") echo "Check for Error messages in server log:" zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \ -e "Code: 236. DB::Exception: Cancelled mutating parts" \ @@ -389,6 +391,8 @@ else -e "} TCPHandler: Code:" \ -e "} executeQuery: Code:" \ -e "Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'" \ + -e "This engine is deprecated and is not supported in transactions" \ + -e "[Queue = DB::MergeMutateRuntimeQueue]: Code: 235. DB::Exception: Part" \ /var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv From a3cd9dc923e07e2452fc92794cc37c8b97ebf19f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 20 Jul 2022 12:45:37 +0200 Subject: [PATCH 423/659] Fix flaky test test_login_as_dropped_user_xml. --- .../integration/test_create_user_and_login/test.py | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_create_user_and_login/test.py b/tests/integration/test_create_user_and_login/test.py index fd052ba9716..25346c22d3b 100644 --- a/tests/integration/test_create_user_and_login/test.py +++ b/tests/integration/test_create_user_and_login/test.py @@ -81,18 +81,23 @@ EOF""", ["bash", "-c", "rm /etc/clickhouse-server/users.d/user_c.xml"] ) - expected_error = "no user with such name" + expected_errors = ["no user with such name", "not found in user directories"] while True: out, err = instance.query_and_get_answer_with_error("SELECT 1", user="C") - if expected_error in err: - logging.debug(f"Got error '{expected_error}' just as expected") + found_error = [ + expected_error + for expected_error in expected_errors + if (expected_error in err) + ] + if found_error: + logging.debug(f"Got error '{found_error}' just as expected") break if out == "1\n": logging.debug(f"Got output '1', retrying...") time.sleep(0.5) continue raise Exception( - f"Expected either output '1' or error '{expected_error}', got output={out} and error={err}" + f"Expected either output '1' or one of errors '{expected_errors}', got output={out} and error={err}" ) assert instance.query("SELECT name FROM system.users WHERE name='C'") == "" From 784ee115945761ca7fc1e8262891d6cc18728070 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 20 Jul 2022 11:16:25 +0000 Subject: [PATCH 424/659] Add settings to skip fields with unsupported types in Protobuf/CapnProto schema inference --- src/Core/Settings.h | 2 + src/Formats/CapnProtoUtils.cpp | 43 +++++++++-- src/Formats/CapnProtoUtils.h | 2 +- src/Formats/FormatFactory.cpp | 2 + src/Formats/FormatSettings.h | 2 + src/Formats/ProtobufSerializer.cpp | 74 +++++++++++++------ src/Formats/ProtobufSerializer.h | 2 +- .../Formats/Impl/CapnProtoRowInputFormat.cpp | 2 +- .../Formats/Impl/ProtobufListInputFormat.cpp | 3 +- .../Formats/Impl/ProtobufListInputFormat.h | 1 + .../Formats/Impl/ProtobufRowInputFormat.cpp | 6 +- .../Formats/Impl/ProtobufRowInputFormat.h | 1 + ...apnproto_protobuf_empty_messages.reference | 8 +- ...02327_capnproto_protobuf_empty_messages.sh | 17 ++++- .../format_schemas/02327_schema.capnp | 5 +- .../format_schemas/02327_schema.proto | 5 +- 16 files changed, 129 insertions(+), 46 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bda72f089eb..17514839b58 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -686,6 +686,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, input_format_csv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in CSV format", 0) \ M(Bool, input_format_tsv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in TSV format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Allow to skip columns with unsupported types while schema inference for format Parquet", 0) \ + M(Bool, input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference, false, "Allow to skip fields with unsupported types while schema inference for format Protobuf", 0) \ + M(Bool, input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference, false, "Allow to skip columns with unsupported types while schema inference for format CapnProto", 0) \ M(Bool, input_format_orc_skip_columns_with_unsupported_types_in_schema_inference, false, "Allow to skip columns with unsupported types while schema inference for format ORC", 0) \ M(Bool, input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference, false, "Allow to skip columns with unsupported types while schema inference for format Arrow", 0) \ M(String, column_names_for_schema_inference, "", "The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'", 0) \ diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index 65954315c0d..774ab00e2e3 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -450,7 +450,7 @@ static DataTypePtr getEnumDataTypeFromEnumSchema(const capnp::EnumSchema & enum_ throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "ClickHouse supports only 8 and 16-bit Enums"); } -static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type) +static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type, bool skip_unsupported_fields) { switch (capnp_type.which()) { @@ -483,7 +483,9 @@ static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type) case capnp::schema::Type::LIST: { auto list_schema = capnp_type.asList(); - auto nested_type = getDataTypeFromCapnProtoType(list_schema.getElementType()); + auto nested_type = getDataTypeFromCapnProtoType(list_schema.getElementType(), skip_unsupported_fields); + if (!nested_type) + return nullptr; return std::make_shared(nested_type); } case capnp::schema::Type::STRUCT: @@ -492,19 +494,33 @@ static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type) if (struct_schema.getFields().size() == 0) + { + if (skip_unsupported_fields) + return nullptr; throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Empty messages are not supported"); + } /// Check if it can be Nullable. if (checkIfStructIsNamedUnion(struct_schema)) { auto fields = struct_schema.getUnionFields(); if (fields.size() != 2 || (!fields[0].getType().isVoid() && !fields[1].getType().isVoid())) + { + if (skip_unsupported_fields) + return nullptr; throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unions are not supported"); + } auto value_type = fields[0].getType().isVoid() ? fields[1].getType() : fields[0].getType(); if (value_type.isStruct() || value_type.isList()) + { + if (skip_unsupported_fields) + return nullptr; throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Tuples and Lists cannot be inside Nullable"); + } - auto nested_type = getDataTypeFromCapnProtoType(value_type); + auto nested_type = getDataTypeFromCapnProtoType(value_type, skip_unsupported_fields); + if (!nested_type) + return nullptr; return std::make_shared(nested_type); } @@ -516,17 +532,26 @@ static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type) Names nested_names; for (auto field : struct_schema.getNonUnionFields()) { + auto nested_type = getDataTypeFromCapnProtoType(field.getType(), skip_unsupported_fields); + if (!nested_type) + continue; nested_names.push_back(field.getProto().getName()); - nested_types.push_back(getDataTypeFromCapnProtoType(field.getType())); + nested_types.push_back(nested_type); } + if (nested_types.empty()) + return nullptr; return std::make_shared(std::move(nested_types), std::move(nested_names)); } default: + { + if (skip_unsupported_fields) + return nullptr; throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unsupported CapnProtoType: {}", getCapnProtoFullTypeName(capnp_type)); + } } } -NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema) +NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema, bool skip_unsupported_fields) { if (checkIfStructContainsUnnamedUnion(schema)) throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); @@ -535,9 +560,13 @@ NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema) for (auto field : schema.getNonUnionFields()) { auto name = field.getProto().getName(); - auto type = getDataTypeFromCapnProtoType(field.getType()); - names_and_types.emplace_back(name, type); + auto type = getDataTypeFromCapnProtoType(field.getType(), skip_unsupported_fields); + if (type) + names_and_types.emplace_back(name, type); } + if (names_and_types.empty()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot convert CapnProto schema to ClickHouse table schema, all fields have unsupported types"); + return names_and_types; } diff --git a/src/Formats/CapnProtoUtils.h b/src/Formats/CapnProtoUtils.h index 47fe3ada7cd..50f146a05f6 100644 --- a/src/Formats/CapnProtoUtils.h +++ b/src/Formats/CapnProtoUtils.h @@ -38,7 +38,7 @@ capnp::DynamicValue::Reader getReaderByColumnName(const capnp::DynamicStruct::Re void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Block & header, FormatSettings::EnumComparingMode mode); -NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema); +NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema, bool skip_unsupported_fields); } #endif diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 756b33d3eb2..50d2ee12082 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -110,6 +110,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.pretty.output_format_pretty_row_numbers = settings.output_format_pretty_row_numbers; format_settings.protobuf.input_flatten_google_wrappers = settings.input_format_protobuf_flatten_google_wrappers; format_settings.protobuf.output_nullables_with_google_wrappers = settings.output_format_protobuf_nullables_with_google_wrappers; + format_settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference; format_settings.regexp.escaping_rule = settings.format_regexp_escaping_rule; format_settings.regexp.regexp = settings.format_regexp; format_settings.regexp.skip_unmatched = settings.format_regexp_skip_unmatched; @@ -151,6 +152,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.orc.output_string_as_string = settings.output_format_orc_string_as_string; format_settings.defaults_for_omitted_fields = settings.input_format_defaults_for_omitted_fields; format_settings.capn_proto.enum_comparing_mode = settings.format_capn_proto_enum_comparising_mode; + format_settings.capn_proto.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference; format_settings.seekable_read = settings.input_format_allow_seeks; format_settings.msgpack.number_of_columns = settings.input_format_msgpack_number_of_columns; format_settings.msgpack.output_uuid_representation = settings.output_format_msgpack_uuid_representation; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 70bf8979383..b7c55d11beb 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -185,6 +185,7 @@ struct FormatSettings * because Protobuf without delimiters is not generally useful. */ bool allow_multiple_rows_without_delimiter = false; + bool skip_fields_with_unsupported_types_in_schema_inference = false; } protobuf; struct @@ -255,6 +256,7 @@ struct FormatSettings struct { EnumComparingMode enum_comparing_mode = EnumComparingMode::BY_VALUES; + bool skip_fields_with_unsupported_types_in_schema_inference = false; } capn_proto; enum class MsgPackUUIDRepresentation diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 02054d0c1ed..203502150ad 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -3427,19 +3427,23 @@ namespace return std::make_shared>(std::move(values)); } - NameAndTypePair getNameAndDataTypeFromField(const google::protobuf::FieldDescriptor * field_descriptor, bool allow_repeat = true) + std::optional getNameAndDataTypeFromField(const google::protobuf::FieldDescriptor * field_descriptor, bool skip_unsupported_fields, bool allow_repeat = true) { if (allow_repeat && field_descriptor->is_map()) { - auto name_and_type = getNameAndDataTypeFromField(field_descriptor, false); - const auto * tuple_type = assert_cast(name_and_type.type.get()); - return {name_and_type.name, std::make_shared(tuple_type->getElements())}; + auto name_and_type = getNameAndDataTypeFromField(field_descriptor, skip_unsupported_fields, false); + if (!name_and_type) + return std::nullopt; + const auto * tuple_type = assert_cast(name_and_type->type.get()); + return NameAndTypePair{name_and_type->name, std::make_shared(tuple_type->getElements())}; } if (allow_repeat && field_descriptor->is_repeated()) { - auto name_and_type = getNameAndDataTypeFromField(field_descriptor, false); - return {name_and_type.name, std::make_shared(name_and_type.type)}; + auto name_and_type = getNameAndDataTypeFromField(field_descriptor, skip_unsupported_fields, false); + if (!name_and_type) + return std::nullopt; + return NameAndTypePair{name_and_type->name, std::make_shared(name_and_type->type)}; } switch (field_descriptor->type()) @@ -3447,31 +3451,35 @@ namespace case FieldTypeId::TYPE_SFIXED32: case FieldTypeId::TYPE_SINT32: case FieldTypeId::TYPE_INT32: - return {field_descriptor->name(), std::make_shared()}; + return NameAndTypePair{field_descriptor->name(), std::make_shared()}; case FieldTypeId::TYPE_SFIXED64: case FieldTypeId::TYPE_SINT64: case FieldTypeId::TYPE_INT64: - return {field_descriptor->name(), std::make_shared()}; + return NameAndTypePair{field_descriptor->name(), std::make_shared()}; case FieldTypeId::TYPE_BOOL: - return {field_descriptor->name(), std::make_shared()}; + return NameAndTypePair{field_descriptor->name(), std::make_shared()}; case FieldTypeId::TYPE_FLOAT: - return {field_descriptor->name(), std::make_shared()}; + return NameAndTypePair{field_descriptor->name(), std::make_shared()}; case FieldTypeId::TYPE_DOUBLE: - return {field_descriptor->name(), std::make_shared()}; + return NameAndTypePair{field_descriptor->name(), std::make_shared()}; case FieldTypeId::TYPE_UINT32: case FieldTypeId::TYPE_FIXED32: - return {field_descriptor->name(), std::make_shared()}; + return NameAndTypePair{field_descriptor->name(), std::make_shared()}; case FieldTypeId::TYPE_UINT64: case FieldTypeId::TYPE_FIXED64: - return {field_descriptor->name(), std::make_shared()}; + return NameAndTypePair{field_descriptor->name(), std::make_shared()}; case FieldTypeId::TYPE_BYTES: case FieldTypeId::TYPE_STRING: - return {field_descriptor->name(), std::make_shared()}; + return NameAndTypePair{field_descriptor->name(), std::make_shared()}; case FieldTypeId::TYPE_ENUM: { const auto * enum_descriptor = field_descriptor->enum_type(); if (enum_descriptor->value_count() == 0) + { + if (skip_unsupported_fields) + return std::nullopt; throw Exception("Empty enum field", ErrorCodes::BAD_ARGUMENTS); + } int max_abs = std::abs(enum_descriptor->value(0)->number()); for (int i = 1; i != enum_descriptor->value_count(); ++i) { @@ -3479,11 +3487,15 @@ namespace max_abs = std::abs(enum_descriptor->value(i)->number()); } if (max_abs < 128) - return {field_descriptor->name(), getEnumDataType(enum_descriptor)}; + return NameAndTypePair{field_descriptor->name(), getEnumDataType(enum_descriptor)}; else if (max_abs < 32768) - return {field_descriptor->name(), getEnumDataType(enum_descriptor)}; + return NameAndTypePair{field_descriptor->name(), getEnumDataType(enum_descriptor)}; else + { + if (skip_unsupported_fields) + return std::nullopt; throw Exception("ClickHouse supports only 8-bit and 16-bit enums", ErrorCodes::BAD_ARGUMENTS); + } } case FieldTypeId::TYPE_GROUP: case FieldTypeId::TYPE_MESSAGE: @@ -3491,13 +3503,17 @@ namespace const auto * message_descriptor = field_descriptor->message_type(); if (message_descriptor->field_count() == 0) { + if (skip_unsupported_fields) + return std::nullopt; throw Exception("Empty messages are not supported", ErrorCodes::BAD_ARGUMENTS); } else if (message_descriptor->field_count() == 1) { const auto * nested_field_descriptor = message_descriptor->field(0); - auto nested_name_and_type = getNameAndDataTypeFromField(nested_field_descriptor); - return {field_descriptor->name() + "_" + nested_name_and_type.name, nested_name_and_type.type}; + auto nested_name_and_type = getNameAndDataTypeFromField(nested_field_descriptor, skip_unsupported_fields); + if (!nested_name_and_type) + return std::nullopt; + return NameAndTypePair{field_descriptor->name() + "_" + nested_name_and_type->name, nested_name_and_type->type}; } else { @@ -3505,11 +3521,16 @@ namespace Strings nested_names; for (int i = 0; i != message_descriptor->field_count(); ++i) { - auto nested_name_and_type = getNameAndDataTypeFromField(message_descriptor->field(i)); - nested_types.push_back(nested_name_and_type.type); - nested_names.push_back(nested_name_and_type.name); + auto nested_name_and_type = getNameAndDataTypeFromField(message_descriptor->field(i), skip_unsupported_fields); + if (!nested_name_and_type) + continue; + nested_types.push_back(nested_name_and_type->type); + nested_names.push_back(nested_name_and_type->name); } - return {field_descriptor->name(), std::make_shared(std::move(nested_types), std::move(nested_names))}; + + if (nested_types.empty()) + return std::nullopt; + return NameAndTypePair{field_descriptor->name(), std::make_shared(std::move(nested_types), std::move(nested_names))}; } } } @@ -3544,11 +3565,16 @@ std::unique_ptr ProtobufSerializer::create( return ProtobufSerializerBuilder(writer).buildMessageSerializer(column_names, data_types, missing_column_indices, message_descriptor, with_length_delimiter, with_envelope, defaults_for_nullable_google_wrappers); } -NamesAndTypesList protobufSchemaToCHSchema(const google::protobuf::Descriptor * message_descriptor) +NamesAndTypesList protobufSchemaToCHSchema(const google::protobuf::Descriptor * message_descriptor, bool skip_unsupported_fields) { NamesAndTypesList schema; for (int i = 0; i != message_descriptor->field_count(); ++i) - schema.push_back(getNameAndDataTypeFromField(message_descriptor->field(i))); + { + if (auto name_and_type = getNameAndDataTypeFromField(message_descriptor->field(i), skip_unsupported_fields)) + schema.push_back(*name_and_type); + } + if (schema.empty()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot convert Protobuf schema to ClickHouse table schema, all fields have unsupported types"); return schema; } diff --git a/src/Formats/ProtobufSerializer.h b/src/Formats/ProtobufSerializer.h index 607d0b15b63..7cbfe5fd42c 100644 --- a/src/Formats/ProtobufSerializer.h +++ b/src/Formats/ProtobufSerializer.h @@ -54,7 +54,7 @@ public: ProtobufWriter & writer); }; -NamesAndTypesList protobufSchemaToCHSchema(const google::protobuf::Descriptor * message_descriptor); +NamesAndTypesList protobufSchemaToCHSchema(const google::protobuf::Descriptor * message_descriptor, bool skip_unsupported_fields); } #endif diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index ad173e449d6..8da36fecd92 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -298,7 +298,7 @@ NamesAndTypesList CapnProtoSchemaReader::readSchema() auto schema_parser = CapnProtoSchemaParser(); auto schema = schema_parser.getMessageSchema(schema_info); - return capnProtoSchemaToCHSchema(schema); + return capnProtoSchemaToCHSchema(schema, format_settings.capn_proto.skip_fields_with_unsupported_types_in_schema_inference); } void registerInputFormatCapnProto(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp index 4599734591f..d4d80fe3a23 100644 --- a/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp @@ -58,13 +58,14 @@ ProtobufListSchemaReader::ProtobufListSchemaReader(const FormatSettings & format true, format_settings.schema.is_server, format_settings.schema.format_schema_path) + , skip_unsopported_fields(format_settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference) { } NamesAndTypesList ProtobufListSchemaReader::readSchema() { const auto * message_descriptor = ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info, ProtobufSchemas::WithEnvelope::Yes); - return protobufSchemaToCHSchema(message_descriptor); + return protobufSchemaToCHSchema(message_descriptor, skip_unsopported_fields); } void registerInputFormatProtobufList(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/ProtobufListInputFormat.h b/src/Processors/Formats/Impl/ProtobufListInputFormat.h index 03f56077dec..2f334048ad2 100644 --- a/src/Processors/Formats/Impl/ProtobufListInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufListInputFormat.h @@ -50,6 +50,7 @@ public: private: const FormatSchemaInfo schema_info; + bool skip_unsopported_fields; }; } diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index 0376bf2c292..f4329b7ecfe 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -78,15 +78,15 @@ ProtobufSchemaReader::ProtobufSchemaReader(const FormatSettings & format_setting format_settings.schema.format_schema, "Protobuf", true, - format_settings.schema.is_server, - format_settings.schema.format_schema_path) + format_settings.schema.is_server, format_settings.schema.format_schema_path) + , skip_unsupported_fields(format_settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference) { } NamesAndTypesList ProtobufSchemaReader::readSchema() { const auto * message_descriptor = ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info, ProtobufSchemas::WithEnvelope::No); - return protobufSchemaToCHSchema(message_descriptor); + return protobufSchemaToCHSchema(message_descriptor, skip_unsupported_fields); } void registerProtobufSchemaReader(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h index c9ba573f103..3d00ee4794e 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h @@ -57,6 +57,7 @@ public: private: const FormatSchemaInfo schema_info; + bool skip_unsupported_fields; }; } diff --git a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.reference b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.reference index b462a5a7baa..0c7da0c3ce4 100644 --- a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.reference +++ b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.reference @@ -1,4 +1,8 @@ OK OK -OK -OK +FAIL +FAIL +str String +text String +str String +text String diff --git a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh index 3890f013b3b..a569a6435f6 100755 --- a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh +++ b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh @@ -18,7 +18,20 @@ cp -r $CLIENT_SCHEMADIR/02327_* $SCHEMADIR/$SERVER_SCHEMADIR/ $CLICKHOUSE_CLIENT --query="desc file(data.pb) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; $CLICKHOUSE_CLIENT --query="desc file(data.capnp) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="create table t engine=File(Protobuf) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="create table t engine=File(CapnProto) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="create table test_protobuf engine=File(Protobuf) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="create table test_capnp engine=File(CapnProto) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; + +$CLICKHOUSE_CLIENT --query="desc file(data.pb) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message', input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference=1"; +$CLICKHOUSE_CLIENT --query="desc file(data.capnp) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message', input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference=1"; + +$CLICKHOUSE_CLIENT --query="drop table if exists test_protobuf"; +$CLICKHOUSE_CLIENT --query="create table test_protobuf engine=File(Protobuf) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message', input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference=1"; +$CLICKHOUSE_CLIENT --query="desc test_protobuf"; +$CLICKHOUSE_CLIENT --query="drop table test_protobuf"; + +$CLICKHOUSE_CLIENT --query="drop table if exists test_capnp"; +$CLICKHOUSE_CLIENT --query="create table test_capnp engine=File(CapnProto) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message', input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference=1"; +$CLICKHOUSE_CLIENT --query="desc test_capnp"; +$CLICKHOUSE_CLIENT --query="drop table test_capnp"; rm -rf ${SCHEMADIR:?}/${SERVER_SCHEMADIR:?} diff --git a/tests/queries/0_stateless/format_schemas/02327_schema.capnp b/tests/queries/0_stateless/format_schemas/02327_schema.capnp index c882dcab8d4..12ccc7308c9 100644 --- a/tests/queries/0_stateless/format_schemas/02327_schema.capnp +++ b/tests/queries/0_stateless/format_schemas/02327_schema.capnp @@ -1,10 +1,11 @@ @0x9ef128e10a8010b8; -struct Nested1 +struct Empty { } struct Message { - tuple1 @0 : Nested1; + tuple1 @0 : Empty; + text @1 : Text; } diff --git a/tests/queries/0_stateless/format_schemas/02327_schema.proto b/tests/queries/0_stateless/format_schemas/02327_schema.proto index ae1f440d279..b5067393558 100644 --- a/tests/queries/0_stateless/format_schemas/02327_schema.proto +++ b/tests/queries/0_stateless/format_schemas/02327_schema.proto @@ -1,8 +1,9 @@ syntax = "proto3"; -message Nested { +message Empty { } message Message { - Nested nested = 1; + Empty empty = 1; + string str = 2; }; From d43b1d62800ccd65656bea2f8f9dda583b3fcc29 Mon Sep 17 00:00:00 2001 From: Niek <93536181+nikoloko@users.noreply.github.com> Date: Wed, 20 Jul 2022 13:21:06 +0200 Subject: [PATCH 425/659] Corrected structure of the users section I received feedback from a user that the structure of the user's section was not fully correct. I changed accordingly. --- docs/en/operations/settings/settings-users.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings-users.md b/docs/en/operations/settings/settings-users.md index 6b3340bfce2..101ad46e55a 100644 --- a/docs/en/operations/settings/settings-users.md +++ b/docs/en/operations/settings/settings-users.md @@ -29,7 +29,7 @@ Structure of the `users` section: profile_name default - default + default From 11b8d788caf3a9f2cf50bbfc0d6f7a7736a7fb5d Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 20 Jul 2022 13:36:47 +0200 Subject: [PATCH 426/659] small improvements --- src/Common/base58.h | 16 +++++++-------- src/Functions/FunctionBase58Conversion.h | 26 +++++++++++------------- 2 files changed, 20 insertions(+), 22 deletions(-) diff --git a/src/Common/base58.h b/src/Common/base58.h index 00eaaa7c5e4..5dbe48ac4b2 100644 --- a/src/Common/base58.h +++ b/src/Common/base58.h @@ -5,7 +5,7 @@ namespace DB { -inline bool encodeBase58(const char8_t * src, char8_t * dst) +inline size_t encodeBase58(const char8_t * src, char8_t * dst) { const char * base58_encoding_alphabet = "123456789ABCDEFGHJKLMNPQRSTUVWXYZabcdefghijkmnopqrstuvwxyz"; @@ -38,10 +38,10 @@ inline bool encodeBase58(const char8_t * src, char8_t * dst) dst[c_idx] = base58_encoding_alphabet[static_cast(dst[c_idx])]; } dst[idx] = '\0'; - return true; + return idx + 1; } -inline bool decodeBase58(const char8_t * src, char8_t * dst) +inline size_t decodeBase58(const char8_t * src, char8_t * dst) { const char map_digits[128] = {-1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, @@ -53,12 +53,12 @@ inline bool decodeBase58(const char8_t * src, char8_t * dst) for (; *src; ++src) { - unsigned int carry = static_cast(map_digits[static_cast(*src)]); - if (carry == UINT_MAX || *src < '1' || map_digits[static_cast(*src)] == map_digits[0]) + unsigned int carry = map_digits[*src]; + if (unlikely(carry == UINT_MAX)) { - return false; + return 0; } - for (size_t j = 0; j < idx; j++) + for (size_t j = 0; j < idx; ++j) { carry += static_cast(dst[j]) * 58; dst[j] = static_cast(carry & 0xff); @@ -79,7 +79,7 @@ inline bool decodeBase58(const char8_t * src, char8_t * dst) dst[idx - (i + 1)] = s; } dst[idx] = '\0'; - return true; + return idx + 1; } } diff --git a/src/Functions/FunctionBase58Conversion.h b/src/Functions/FunctionBase58Conversion.h index 7842232961c..82e2a2caac0 100644 --- a/src/Functions/FunctionBase58Conversion.h +++ b/src/Functions/FunctionBase58Conversion.h @@ -39,7 +39,7 @@ struct Base58Encode const ColumnString::Offsets & src_offsets = src_column.getOffsets(); - const auto * source = src_column.getChars().data(); + const auto * src = src_column.getChars().data(); auto * dst = dst_data.data(); auto * dst_pos = dst; @@ -47,13 +47,11 @@ struct Base58Encode for (size_t row = 0; row < input_rows_count; ++row) { - size_t srclen = src_offsets[row] - src_offset_prev - 1; - encodeBase58(source, dst_pos); + size_t srclen = src_offsets[row] - src_offset_prev; + auto encoded_size = encodeBase58(src, dst_pos); - size_t encoded_length = strlen(reinterpret_cast(dst_pos)); - - source += srclen + 1; - dst_pos += encoded_length + 1; + src += srclen; + dst_pos += encoded_size; dst_offsets[row] = dst_pos - dst; src_offset_prev = src_offsets[row]; @@ -82,7 +80,7 @@ struct Base58Decode const ColumnString::Offsets & src_offsets = src_column.getOffsets(); - const auto * source = src_column.getChars().data(); + const auto * src = src_column.getChars().data(); auto * dst = dst_data.data(); auto * dst_pos = dst; @@ -90,14 +88,14 @@ struct Base58Decode for (size_t row = 0; row < input_rows_count; ++row) { - size_t srclen = src_offsets[row] - src_offset_prev - 1; - if (!decodeBase58(source, dst_pos)) + size_t srclen = src_offsets[row] - src_offset_prev; + + auto decoded_size = decodeBase58(src, dst_pos); + if (!decoded_size) throw Exception("Invalid Base58 value, cannot be decoded", ErrorCodes::BAD_ARGUMENTS); - size_t encoded_length = strlen(reinterpret_cast(dst_pos)); - - source += srclen + 1; - dst_pos += encoded_length + 1; + src += srclen; + dst_pos += decoded_size; dst_offsets[row] = dst_pos - dst; src_offset_prev = src_offsets[row]; From 260d4e54554b330fc218b304ed44cc1e9bfcacc4 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 20 Jul 2022 15:02:30 +0200 Subject: [PATCH 427/659] fix remaining basex files --- contrib/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index d4a3f164214..489f788a512 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -156,7 +156,6 @@ endif() add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) -add_contrib (base-x-cmake base-x) # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear From 562148e0e2a03ab0c66b9d868e2d8424c69a45c4 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 20 Jul 2022 15:04:15 +0200 Subject: [PATCH 428/659] remove basex --- contrib/base-x | 1 - contrib/base-x-cmake/CMakeLists.txt | 28 ---------------------------- 2 files changed, 29 deletions(-) delete mode 160000 contrib/base-x delete mode 100644 contrib/base-x-cmake/CMakeLists.txt diff --git a/contrib/base-x b/contrib/base-x deleted file mode 160000 index a85f98fb4ed..00000000000 --- a/contrib/base-x +++ /dev/null @@ -1 +0,0 @@ -Subproject commit a85f98fb4ed52c2f4029a4b6ac1ef0bafdfc56f5 diff --git a/contrib/base-x-cmake/CMakeLists.txt b/contrib/base-x-cmake/CMakeLists.txt deleted file mode 100644 index b0c11a176c6..00000000000 --- a/contrib/base-x-cmake/CMakeLists.txt +++ /dev/null @@ -1,28 +0,0 @@ -option (ENABLE_BASEX "Enable base-x" ${ENABLE_LIBRARIES}) - -if (NOT ENABLE_BASEX) - message(STATUS "Not using base-x") - return() -endif() - -set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/base-x") - -set (SRCS - ${LIBRARY_DIR}/base_x.hh - ${LIBRARY_DIR}/uinteger_t.hh - ) - -add_library(_base-x INTERFACE) -target_include_directories(_base-x SYSTEM BEFORE INTERFACE "${ClickHouse_SOURCE_DIR}/contrib/base-x") - -if (XCODE OR XCODE_VERSION) - # https://gitlab.kitware.com/cmake/cmake/issues/17457 - # Some native build systems may not like targets that have only object files, so consider adding at least one real source file - # This applies to Xcode. - if (NOT EXISTS "${CMAKE_CURRENT_BINARY_DIR}/dummy.c") - file(WRITE "${CMAKE_CURRENT_BINARY_DIR}/dummy.c" "") - endif () - target_sources(_base-x PRIVATE "${CMAKE_CURRENT_BINARY_DIR}/dummy.c") -endif () - -add_library(ch_contrib::base-x ALIAS _base-x) \ No newline at end of file From 61a6117a331a6ad53887f5a45d54735ccffd0ecc Mon Sep 17 00:00:00 2001 From: lokax Date: Fri, 8 Jul 2022 14:16:01 +0800 Subject: [PATCH 429/659] overload getPostitionByName function --- src/DataTypes/DataTypeTuple.cpp | 14 ++++++++++++++ src/DataTypes/DataTypeTuple.h | 1 + 2 files changed, 15 insertions(+) diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 558b13927c1..855aeb248ed 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -214,6 +214,20 @@ size_t DataTypeTuple::getPositionByName(const String & name) const throw Exception("Tuple doesn't have element with name '" + name + "'", ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); } +bool DataTypeTuple::getPositionByName(const String & name, size_t & index) const +{ + size_t size = elems.size(); + for (size_t i = 0; i < size; ++i) + { + if (names[i] == name) + { + index = i; + return true; + } + } + return false; +} + String DataTypeTuple::getNameByPosition(size_t i) const { if (i == 0 || i > names.size()) diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 009a2284a0a..6cee8a3685b 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -60,6 +60,7 @@ public: const Strings & getElementNames() const { return names; } size_t getPositionByName(const String & name) const; + bool getPositionByName(const String & name, size_t & index) const; String getNameByPosition(size_t i) const; bool haveExplicitNames() const { return have_explicit_names; } From 6e23d2cb8537cae97252be7a1381c3f00002b8dc Mon Sep 17 00:00:00 2001 From: lokax Date: Fri, 8 Jul 2022 14:21:59 +0800 Subject: [PATCH 430/659] feat(function): tupleElement with default value --- src/Functions/tupleElement.cpp | 48 +++++++++++++++---- .../0_stateless/02116_tuple_element.reference | 9 ++++ .../0_stateless/02116_tuple_element.sql | 10 ++++ 3 files changed, 58 insertions(+), 9 deletions(-) diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index 023dc266b43..bea2385fb49 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -18,6 +18,8 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_INDEX; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int NOT_FOUND_COLUMN_IN_BLOCK; } namespace @@ -40,9 +42,11 @@ public: return name; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { - return 2; + return 0; } bool useDefaultImplementationForConstants() const override @@ -59,6 +63,13 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { + const size_t number_of_arguments = arguments.size(); + + if (number_of_arguments < 2 || number_of_arguments > 3) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(number_of_arguments) + ", should be 2 or 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + size_t count_arrays = 0; const IDataType * tuple_col = arguments[0].type.get(); @@ -72,7 +83,12 @@ public: if (!tuple) throw Exception("First argument for function " + getName() + " must be tuple or array of tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - size_t index = getElementNum(arguments[1].column, *tuple); + size_t index = 0; + if (!getElementNum(arguments[1].column, *tuple, index, number_of_arguments)) + { + return arguments[2].type; + } + DataTypePtr out_return_type = tuple->getElements()[index]; for (; count_arrays; --count_arrays) @@ -81,7 +97,7 @@ public: return out_return_type; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { Columns array_offsets; @@ -103,7 +119,11 @@ public: if (!tuple_type_concrete || !tuple_col_concrete) throw Exception("First argument for function " + getName() + " must be tuple or array of tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - size_t index = getElementNum(arguments[1].column, *tuple_type_concrete); + size_t index = 0; + if (!getElementNum(arguments[1].column, *tuple_type_concrete, index, arguments.size())) + { + return ColumnConst::create(arguments[2].column, input_rows_count); + } ColumnPtr res = tuple_col_concrete->getColumns()[index]; /// Wrap into Arrays @@ -114,7 +134,7 @@ public: } private: - size_t getElementNum(const ColumnPtr & index_column, const DataTypeTuple & tuple) const + bool getElementNum(const ColumnPtr & index_column, const DataTypeTuple & tuple, size_t & index, const size_t argument_size) const { if ( checkAndGetColumnConst(index_column.get()) @@ -123,19 +143,29 @@ private: || checkAndGetColumnConst(index_column.get()) ) { - size_t index = index_column->getUInt(0); + index = index_column->getUInt(0); if (index == 0) throw Exception("Indices in tuples are 1-based.", ErrorCodes::ILLEGAL_INDEX); if (index > tuple.getElements().size()) throw Exception("Index for tuple element is out of range.", ErrorCodes::ILLEGAL_INDEX); - - return index - 1; + index--; + return true; } else if (const auto * name_col = checkAndGetColumnConst(index_column.get())) { - return tuple.getPositionByName(name_col->getValue()); + if (tuple.getPositionByName(name_col->getValue(), index)) + { + return true; + } + + if (argument_size == 2) + { + throw Exception("Tuple doesn't have element with name '" + name_col->getValue() + "'", ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); + } + + return false; } else throw Exception("Second argument to " + getName() + " must be a constant UInt or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/tests/queries/0_stateless/02116_tuple_element.reference b/tests/queries/0_stateless/02116_tuple_element.reference index 121b08d02f1..4146b13b463 100644 --- a/tests/queries/0_stateless/02116_tuple_element.reference +++ b/tests/queries/0_stateless/02116_tuple_element.reference @@ -13,6 +13,15 @@ FROM t_tuple_element 2 SELECT `t2.1` FROM t_tuple_element +z +SELECT tupleElement(t1, \'z\', \'z\') +FROM t_tuple_element +0 +SELECT tupleElement(t1, \'z\', 0) +FROM t_tuple_element +z +SELECT tupleElement(t2, \'z\', \'z\') +FROM t_tuple_element 1 2 WITH (1, 2) AS t SELECT diff --git a/tests/queries/0_stateless/02116_tuple_element.sql b/tests/queries/0_stateless/02116_tuple_element.sql index 4ce6e5cf136..26d576b8443 100644 --- a/tests/queries/0_stateless/02116_tuple_element.sql +++ b/tests/queries/0_stateless/02116_tuple_element.sql @@ -27,6 +27,16 @@ EXPLAIN SYNTAX SELECT t2.1 FROM t_tuple_element; SELECT tupleElement(t2, 1) FROM t_tuple_element; EXPLAIN SYNTAX SELECT tupleElement(t2, 1) FROM t_tuple_element; +SELECT tupleElement(t1, 'z', 'z') FROM t_tuple_element; +EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 'z') FROM t_tuple_element; +SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element; +EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element; +SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element; +EXPLAIN SYNTAX SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element; + +SELECT tupleElement(t1, 3, 'z') FROM t_tuple_element; -- { serverError 127 } +SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element; -- { serverError 127 } + SELECT tupleElement(t2) FROM t_tuple_element; -- { serverError 42 } SELECT tupleElement(t2, 'a') FROM t_tuple_element; -- { serverError 47 } SELECT tupleElement(t2, 0) FROM t_tuple_element; -- { serverError 127 } From 5c6b18a9bd9e2c97ac09ee1b51a0df629ea1ec06 Mon Sep 17 00:00:00 2001 From: lokax Date: Fri, 8 Jul 2022 18:17:31 +0800 Subject: [PATCH 431/659] fix: 3rd parameter must be constant --- src/Functions/tupleElement.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index bea2385fb49..3763d878b1d 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -56,7 +56,7 @@ public: ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { - return {1}; + return {1, 2}; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } @@ -97,7 +97,7 @@ public: return out_return_type; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { Columns array_offsets; @@ -122,7 +122,7 @@ public: size_t index = 0; if (!getElementNum(arguments[1].column, *tuple_type_concrete, index, arguments.size())) { - return ColumnConst::create(arguments[2].column, input_rows_count); + return arguments[2].column; } ColumnPtr res = tuple_col_concrete->getColumns()[index]; From f1e9c23f4630905fcdf80084c998596b28f0998c Mon Sep 17 00:00:00 2001 From: lokax Date: Wed, 13 Jul 2022 14:48:43 +0800 Subject: [PATCH 432/659] add tryGetPostitionByName function --- src/DataTypes/DataTypeTuple.cpp | 7 +++---- src/DataTypes/DataTypeTuple.h | 3 ++- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 855aeb248ed..1ef86a8c12f 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -214,18 +214,17 @@ size_t DataTypeTuple::getPositionByName(const String & name) const throw Exception("Tuple doesn't have element with name '" + name + "'", ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); } -bool DataTypeTuple::getPositionByName(const String & name, size_t & index) const +std::optional DataTypeTuple::tryGetPositionByName(const String & name) const { size_t size = elems.size(); for (size_t i = 0; i < size; ++i) { if (names[i] == name) { - index = i; - return true; + return std::optional(i); } } - return false; + return std::nullopt; } String DataTypeTuple::getNameByPosition(size_t i) const diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 6cee8a3685b..eed04631528 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -60,7 +61,7 @@ public: const Strings & getElementNames() const { return names; } size_t getPositionByName(const String & name) const; - bool getPositionByName(const String & name, size_t & index) const; + std::optional tryGetPositionByName(const String & name) const; String getNameByPosition(size_t i) const; bool haveExplicitNames() const { return have_explicit_names; } From 764037f508fb297868e51189be53ee0d58889172 Mon Sep 17 00:00:00 2001 From: lokax Date: Wed, 13 Jul 2022 14:59:23 +0800 Subject: [PATCH 433/659] test(function): tupleElement with default --- .../0_stateless/02116_tuple_element.sql | 10 ----- ...02354_tuple_element_with_default.reference | 21 +++++++++ .../02354_tuple_element_with_default.sql | 43 +++++++++++++++++++ 3 files changed, 64 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02354_tuple_element_with_default.reference create mode 100644 tests/queries/0_stateless/02354_tuple_element_with_default.sql diff --git a/tests/queries/0_stateless/02116_tuple_element.sql b/tests/queries/0_stateless/02116_tuple_element.sql index 26d576b8443..4ce6e5cf136 100644 --- a/tests/queries/0_stateless/02116_tuple_element.sql +++ b/tests/queries/0_stateless/02116_tuple_element.sql @@ -27,16 +27,6 @@ EXPLAIN SYNTAX SELECT t2.1 FROM t_tuple_element; SELECT tupleElement(t2, 1) FROM t_tuple_element; EXPLAIN SYNTAX SELECT tupleElement(t2, 1) FROM t_tuple_element; -SELECT tupleElement(t1, 'z', 'z') FROM t_tuple_element; -EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 'z') FROM t_tuple_element; -SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element; -EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element; -SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element; -EXPLAIN SYNTAX SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element; - -SELECT tupleElement(t1, 3, 'z') FROM t_tuple_element; -- { serverError 127 } -SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element; -- { serverError 127 } - SELECT tupleElement(t2) FROM t_tuple_element; -- { serverError 42 } SELECT tupleElement(t2, 'a') FROM t_tuple_element; -- { serverError 47 } SELECT tupleElement(t2, 0) FROM t_tuple_element; -- { serverError 127 } diff --git a/tests/queries/0_stateless/02354_tuple_element_with_default.reference b/tests/queries/0_stateless/02354_tuple_element_with_default.reference new file mode 100644 index 00000000000..1da17c1b42e --- /dev/null +++ b/tests/queries/0_stateless/02354_tuple_element_with_default.reference @@ -0,0 +1,21 @@ +z +SELECT tupleElement(t1, \'z\', \'z\') +FROM t_tuple_element_default +0 +SELECT tupleElement(t1, \'z\', 0) +FROM t_tuple_element_default +z +SELECT tupleElement(t2, \'z\', \'z\') +FROM t_tuple_element_default +[(3,4)] +SELECT tupleElement([(1, 2)], \'a\', [(3, 4)]) +[[1,2,3]] +SELECT tupleElement([[tuple(1)]], \'a\', [[1, 2, 3]]) +SELECT tupleElement(t1, \'a\', [tuple(1)]) +FROM t_tuple_element_default +[(0)] +SELECT tupleElement(t1, \'a\', [tuple(0)]) +FROM t_tuple_element_default +[(0)] +SELECT tupleElement(t1, \'a\', [tuple(0)]) +FROM t_tuple_element_default diff --git a/tests/queries/0_stateless/02354_tuple_element_with_default.sql b/tests/queries/0_stateless/02354_tuple_element_with_default.sql new file mode 100644 index 00000000000..5adf0b74d4c --- /dev/null +++ b/tests/queries/0_stateless/02354_tuple_element_with_default.sql @@ -0,0 +1,43 @@ +DROP TABLE IF EXISTS t_tuple_element_default; + +CREATE TABLE t_tuple_element_default(t1 Tuple(a UInt32, s String), t2 Tuple(UInt32, String)) ENGINE = Memory; +INSERT INTO t_tuple_element_default VALUES ((1, 'a'), (2, 'b')); + +SELECT tupleElement(t1, 'z', 'z') FROM t_tuple_element_default; +EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 'z') FROM t_tuple_element_default; +SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element_default; +EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element_default; +SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default; +EXPLAIN SYNTAX SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default; + +SELECT tupleElement(t1, 3, 'z') FROM t_tuple_element_default; -- { serverError 127 } +SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element_default; -- { serverError 127 } + +DROP TABLE t_tuple_element_default; + +SELECT tupleElement(array(tuple(1, 2)), 'a', 0); -- { serverError 645 } +SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(1, 2), tuple(3, 4))); -- { serverError 190 } + +SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(3, 4))); +EXPLAIN SYNTAX SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(3, 4))); + +SELECT tupleElement(array(array(tuple(1))), 'a', array(array(1, 2, 3))); +EXPLAIN SYNTAX SELECT tupleElement(array(array(tuple(1))), 'a', array(array(1, 2, 3))); + +CREATE TABLE t_tuple_element_default(t1 Array(Tuple(UInt32)), t2 UInt32) ENGINE = Memory; + +SELECT tupleElement(t1, 'a', array(tuple(1))) FROM t_tuple_element_default; +EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(tuple(1))) FROM t_tuple_element_default; + +INSERT INTO t_tuple_element_default VALUES ([(1)], 100); + +SELECT tupleElement(t1, 'a', array(tuple(0))) FROM t_tuple_element_default; +EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(tuple(0))) FROM t_tuple_element_default; + +INSERT INTO t_tuple_element_default VALUES ([(2), (3), (4)], 234); + +SELECT tupleElement(t1, 'a', array(tuple(0))) FROM t_tuple_element_default; -- { serverError 190 } +EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(tuple(0))) FROM t_tuple_element_default; + +DROP TABLE t_tuple_element_default; + From 647eafa00e65fd029a35a60c1df5ef2041a037b4 Mon Sep 17 00:00:00 2001 From: lokax Date: Wed, 13 Jul 2022 15:01:32 +0800 Subject: [PATCH 434/659] support for array type --- src/Functions/tupleElement.cpp | 117 ++++++++++++++++++++++++++------- 1 file changed, 94 insertions(+), 23 deletions(-) diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index 3763d878b1d..68ee7d07152 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -20,6 +20,8 @@ namespace ErrorCodes extern const int ILLEGAL_INDEX; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NOT_FOUND_COLUMN_IN_BLOCK; + extern const int NUMBER_OF_DIMENSIONS_MISMATHED; + extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; } namespace @@ -56,7 +58,7 @@ public: ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { - return {1, 2}; + return {1}; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } @@ -71,7 +73,6 @@ public: ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); size_t count_arrays = 0; - const IDataType * tuple_col = arguments[0].type.get(); while (const DataTypeArray * array = checkAndGetDataType(tuple_col)) { @@ -83,21 +84,33 @@ public: if (!tuple) throw Exception("First argument for function " + getName() + " must be tuple or array of tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - size_t index = 0; - if (!getElementNum(arguments[1].column, *tuple, index, number_of_arguments)) + auto index = getElementNum(arguments[1].column, *tuple, number_of_arguments); + if (index.has_value()) { + DataTypePtr out_return_type = tuple->getElements()[index.value()]; + + for (; count_arrays; --count_arrays) + out_return_type = std::make_shared(out_return_type); + + return out_return_type; + } else + { + const IDataType * default_col = arguments[2].type.get(); + size_t default_argument_count_arrays = 0; + if (const DataTypeArray * array = checkAndGetDataType(default_col)) + { + default_argument_count_arrays = array->getNumberOfDimensions(); + } + + if (count_arrays != default_argument_count_arrays) + { + throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATHED, "Dimension of types mismatched between first argument and third argument. Dimension of 1st argument: {}. Dimension of 3rd argument: {}.",count_arrays, default_argument_count_arrays); + } return arguments[2].type; } - - DataTypePtr out_return_type = tuple->getElements()[index]; - - for (; count_arrays; --count_arrays) - out_return_type = std::make_shared(out_return_type); - - return out_return_type; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { Columns array_offsets; @@ -105,6 +118,12 @@ public: const IDataType * tuple_type = first_arg.type.get(); const IColumn * tuple_col = first_arg.column.get(); + bool first_arg_is_const = false; + if (typeid_cast(tuple_col)) + { + tuple_col = assert_cast(tuple_col)->getDataColumnPtr().get(); + first_arg_is_const = true; + } while (const DataTypeArray * array_type = checkAndGetDataType(tuple_type)) { const ColumnArray * array_col = assert_cast(tuple_col); @@ -119,22 +138,74 @@ public: if (!tuple_type_concrete || !tuple_col_concrete) throw Exception("First argument for function " + getName() + " must be tuple or array of tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - size_t index = 0; - if (!getElementNum(arguments[1].column, *tuple_type_concrete, index, arguments.size())) + auto index = getElementNum(arguments[1].column, *tuple_type_concrete, arguments.size()); + + if (!index.has_value()) { + if (!array_offsets.empty()) + { + CheckArrayOffsets(arguments[0].column, arguments[2].column); + } return arguments[2].column; } - ColumnPtr res = tuple_col_concrete->getColumns()[index]; + + ColumnPtr res = tuple_col_concrete->getColumns()[index.value()]; /// Wrap into Arrays for (auto it = array_offsets.rbegin(); it != array_offsets.rend(); ++it) res = ColumnArray::create(res, *it); + if (first_arg_is_const) + { + res = ColumnConst::create(res, input_rows_count); + } return res; } private: - bool getElementNum(const ColumnPtr & index_column, const DataTypeTuple & tuple, size_t & index, const size_t argument_size) const + + void CheckArrayOffsets(ColumnPtr col_x, ColumnPtr col_y) const + { + if (isColumnConst(*col_x)) + { + CheckArrayOffsetsWithFirstArgConst(col_x, col_y); + } else if (isColumnConst(*col_y)) + { + CheckArrayOffsetsWithFirstArgConst(col_y, col_x); + } else + { + const auto & array_x = *assert_cast(col_x.get()); + const auto & array_y = *assert_cast(col_y.get()); + if (!array_x.hasEqualOffsets(array_y)) + { + throw Exception("The argument 1 and argument 3 of function " + getName() + " have different array sizes", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + } + } + } + + void CheckArrayOffsetsWithFirstArgConst(ColumnPtr col_x, ColumnPtr col_y) const + { + col_x = assert_cast(col_x.get())->getDataColumnPtr(); + col_y = col_y->convertToFullColumnIfConst(); + const auto & array_x = *assert_cast(col_x.get()); + const auto & array_y = *assert_cast(col_y.get()); + + const auto & offsets_x = array_x.getOffsets(); + const auto & offsets_y = array_y.getOffsets(); + + ColumnArray::Offset prev_offset = 0; + size_t row_size = offsets_y.size(); + for (size_t row = 0; row < row_size; ++row) + { + if (unlikely(offsets_x[0] != offsets_y[row] - prev_offset)) + { + throw Exception("The argument 1 and argument 3 of function " + getName() + " have different array sizes", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + } + prev_offset = offsets_y[row]; + } + } + + std::optional getElementNum(const ColumnPtr & index_column, const DataTypeTuple & tuple, const size_t argument_size) const { if ( checkAndGetColumnConst(index_column.get()) @@ -143,29 +214,29 @@ private: || checkAndGetColumnConst(index_column.get()) ) { - index = index_column->getUInt(0); + size_t index = index_column->getUInt(0); if (index == 0) throw Exception("Indices in tuples are 1-based.", ErrorCodes::ILLEGAL_INDEX); if (index > tuple.getElements().size()) throw Exception("Index for tuple element is out of range.", ErrorCodes::ILLEGAL_INDEX); - index--; - return true; + + return std::optional(index - 1); } else if (const auto * name_col = checkAndGetColumnConst(index_column.get())) { - if (tuple.getPositionByName(name_col->getValue(), index)) + auto index = tuple.tryGetPositionByName(name_col->getValue()); + if (index.has_value()) { - return true; + return index; } if (argument_size == 2) { throw Exception("Tuple doesn't have element with name '" + name_col->getValue() + "'", ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); } - - return false; + return std::nullopt; } else throw Exception("Second argument to " + getName() + " must be a constant UInt or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); From f42e0e8fe1595bbd6446f2ba5550ca99d49db8ac Mon Sep 17 00:00:00 2001 From: lokax Date: Wed, 13 Jul 2022 15:11:12 +0800 Subject: [PATCH 435/659] fix --- tests/queries/0_stateless/02116_tuple_element.reference | 9 --------- 1 file changed, 9 deletions(-) diff --git a/tests/queries/0_stateless/02116_tuple_element.reference b/tests/queries/0_stateless/02116_tuple_element.reference index 4146b13b463..121b08d02f1 100644 --- a/tests/queries/0_stateless/02116_tuple_element.reference +++ b/tests/queries/0_stateless/02116_tuple_element.reference @@ -13,15 +13,6 @@ FROM t_tuple_element 2 SELECT `t2.1` FROM t_tuple_element -z -SELECT tupleElement(t1, \'z\', \'z\') -FROM t_tuple_element -0 -SELECT tupleElement(t1, \'z\', 0) -FROM t_tuple_element -z -SELECT tupleElement(t2, \'z\', \'z\') -FROM t_tuple_element 1 2 WITH (1, 2) AS t SELECT From ff433c1c0196132cb046f9f59ded334839ff6449 Mon Sep 17 00:00:00 2001 From: lokax Date: Wed, 13 Jul 2022 15:52:12 +0800 Subject: [PATCH 436/659] fix build Signed-off-by: lokax --- src/Functions/tupleElement.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index 68ee7d07152..1c67a733d6a 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes extern const int ILLEGAL_INDEX; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NOT_FOUND_COLUMN_IN_BLOCK; - extern const int NUMBER_OF_DIMENSIONS_MISMATHED; + extern const int NUMBER_OF_DIMENSIONS_MISMATCHED; extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; } @@ -104,7 +104,7 @@ public: if (count_arrays != default_argument_count_arrays) { - throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATHED, "Dimension of types mismatched between first argument and third argument. Dimension of 1st argument: {}. Dimension of 3rd argument: {}.",count_arrays, default_argument_count_arrays); + throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATCHED, "Dimension of types mismatched between first argument and third argument. Dimension of 1st argument: {}. Dimension of 3rd argument: {}.",count_arrays, default_argument_count_arrays); } return arguments[2].type; } From d40f04b860a4d97ed1e65910f23b7c4b82ad2328 Mon Sep 17 00:00:00 2001 From: lokax Date: Wed, 13 Jul 2022 22:22:54 +0800 Subject: [PATCH 437/659] fix function name Signed-off-by: lokax --- src/Functions/tupleElement.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index 1c67a733d6a..3d9c2e4f766 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -144,7 +144,7 @@ public: { if (!array_offsets.empty()) { - CheckArrayOffsets(arguments[0].column, arguments[2].column); + checkArrayOffsets(arguments[0].column, arguments[2].column); } return arguments[2].column; } @@ -164,14 +164,14 @@ public: private: - void CheckArrayOffsets(ColumnPtr col_x, ColumnPtr col_y) const + void checkArrayOffsets(ColumnPtr col_x, ColumnPtr col_y) const { if (isColumnConst(*col_x)) { - CheckArrayOffsetsWithFirstArgConst(col_x, col_y); + checkArrayOffsetsWithFirstArgConst(col_x, col_y); } else if (isColumnConst(*col_y)) { - CheckArrayOffsetsWithFirstArgConst(col_y, col_x); + checkArrayOffsetsWithFirstArgConst(col_y, col_x); } else { const auto & array_x = *assert_cast(col_x.get()); @@ -183,7 +183,7 @@ private: } } - void CheckArrayOffsetsWithFirstArgConst(ColumnPtr col_x, ColumnPtr col_y) const + void checkArrayOffsetsWithFirstArgConst(ColumnPtr col_x, ColumnPtr col_y) const { col_x = assert_cast(col_x.get())->getDataColumnPtr(); col_y = col_y->convertToFullColumnIfConst(); From 140f5e66852582d6260bc7d80feb20cbacdc75e7 Mon Sep 17 00:00:00 2001 From: lokax Date: Fri, 15 Jul 2022 13:39:04 +0800 Subject: [PATCH 438/659] recursive check array offsets Signed-off-by: lokax --- src/Functions/tupleElement.cpp | 13 ++++++++++++- .../02354_tuple_element_with_default.reference | 13 +++++++++---- .../02354_tuple_element_with_default.sql | 17 ++++++++++++----- 3 files changed, 33 insertions(+), 10 deletions(-) diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index 3d9c2e4f766..426eaebacfc 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -144,7 +144,7 @@ public: { if (!array_offsets.empty()) { - checkArrayOffsets(arguments[0].column, arguments[2].column); + recursiveCheckArrayOffsets(arguments[0].column, arguments[2].column, array_offsets.size()); } return arguments[2].column; } @@ -164,6 +164,17 @@ public: private: + void recursiveCheckArrayOffsets(ColumnPtr col_x, ColumnPtr col_y, size_t depth) const + { + for (size_t i = 1; i < depth; ++i) + { + checkArrayOffsets(col_x, col_y); + col_x = assert_cast(col_x.get())->getDataPtr(); + col_y = assert_cast(col_y.get())->getDataPtr(); + } + checkArrayOffsets(col_x, col_y); + } + void checkArrayOffsets(ColumnPtr col_x, ColumnPtr col_y) const { if (isColumnConst(*col_x)) diff --git a/tests/queries/0_stateless/02354_tuple_element_with_default.reference b/tests/queries/0_stateless/02354_tuple_element_with_default.reference index 1da17c1b42e..d5dfff17ef1 100644 --- a/tests/queries/0_stateless/02354_tuple_element_with_default.reference +++ b/tests/queries/0_stateless/02354_tuple_element_with_default.reference @@ -7,15 +7,20 @@ FROM t_tuple_element_default z SELECT tupleElement(t2, \'z\', \'z\') FROM t_tuple_element_default +-------------------- [(3,4)] SELECT tupleElement([(1, 2)], \'a\', [(3, 4)]) -[[1,2,3]] -SELECT tupleElement([[tuple(1)]], \'a\', [[1, 2, 3]]) +-------------------- SELECT tupleElement(t1, \'a\', [tuple(1)]) FROM t_tuple_element_default +-------------------- [(0)] SELECT tupleElement(t1, \'a\', [tuple(0)]) FROM t_tuple_element_default -[(0)] -SELECT tupleElement(t1, \'a\', [tuple(0)]) +[0] +SELECT tupleElement(t1, \'a\', [0]) +FROM t_tuple_element_default +[0] +[0] +SELECT tupleElement(t1, \'a\', [0]) FROM t_tuple_element_default diff --git a/tests/queries/0_stateless/02354_tuple_element_with_default.sql b/tests/queries/0_stateless/02354_tuple_element_with_default.sql index 5adf0b74d4c..908a869885b 100644 --- a/tests/queries/0_stateless/02354_tuple_element_with_default.sql +++ b/tests/queries/0_stateless/02354_tuple_element_with_default.sql @@ -15,29 +15,36 @@ SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element_default; -- { serverError 1 DROP TABLE t_tuple_element_default; +SELECT '--------------------'; + SELECT tupleElement(array(tuple(1, 2)), 'a', 0); -- { serverError 645 } SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(1, 2), tuple(3, 4))); -- { serverError 190 } +SELECT tupleElement(array(array(tuple(1))), 'a', array(array(1, 2, 3))); -- { serverError 190 } SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(3, 4))); EXPLAIN SYNTAX SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(3, 4))); -SELECT tupleElement(array(array(tuple(1))), 'a', array(array(1, 2, 3))); -EXPLAIN SYNTAX SELECT tupleElement(array(array(tuple(1))), 'a', array(array(1, 2, 3))); +SELECT '--------------------'; CREATE TABLE t_tuple_element_default(t1 Array(Tuple(UInt32)), t2 UInt32) ENGINE = Memory; SELECT tupleElement(t1, 'a', array(tuple(1))) FROM t_tuple_element_default; EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(tuple(1))) FROM t_tuple_element_default; +SELECT '--------------------'; + INSERT INTO t_tuple_element_default VALUES ([(1)], 100); SELECT tupleElement(t1, 'a', array(tuple(0))) FROM t_tuple_element_default; EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(tuple(0))) FROM t_tuple_element_default; -INSERT INTO t_tuple_element_default VALUES ([(2), (3), (4)], 234); +SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default; +EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default; -SELECT tupleElement(t1, 'a', array(tuple(0))) FROM t_tuple_element_default; -- { serverError 190 } -EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(tuple(0))) FROM t_tuple_element_default; +INSERT INTO t_tuple_element_default VALUES ([(2)], 200); + +SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default; +EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default; DROP TABLE t_tuple_element_default; From 22792d1c6a67d72897c332c91905c516a7656c32 Mon Sep 17 00:00:00 2001 From: lokax Date: Sat, 16 Jul 2022 19:50:08 +0800 Subject: [PATCH 439/659] fix style Signed-off-by: lokax --- src/Functions/tupleElement.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index 426eaebacfc..92ca6b85714 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -93,7 +93,8 @@ public: out_return_type = std::make_shared(out_return_type); return out_return_type; - } else + } + else { const IDataType * default_col = arguments[2].type.get(); size_t default_argument_count_arrays = 0; @@ -180,10 +181,12 @@ private: if (isColumnConst(*col_x)) { checkArrayOffsetsWithFirstArgConst(col_x, col_y); - } else if (isColumnConst(*col_y)) + } + else if (isColumnConst(*col_y)) { checkArrayOffsetsWithFirstArgConst(col_y, col_x); - } else + } + else { const auto & array_x = *assert_cast(col_x.get()); const auto & array_y = *assert_cast(col_y.get()); From 466fceb3ee6530847e3e697e8806322408c0a463 Mon Sep 17 00:00:00 2001 From: Harry Lee <96150659+HarryLeeIBM@users.noreply.github.com> Date: Wed, 20 Jul 2022 09:25:33 -0400 Subject: [PATCH 440/659] Fix exception in AsynchronousMetrics for s390x (#39193) --- src/Interpreters/AsynchronousMetrics.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index b057b6ee641..9fd27fc28b6 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -989,9 +989,15 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti if (s.rfind("processor", 0) == 0) { + /// s390x example: processor 0: version = FF, identification = 039C88, machine = 3906 + /// non s390x example: processor : 0 if (auto colon = s.find_first_of(':')) { +#ifdef __s390x__ + core_id = std::stoi(s.substr(10)); /// 10: length of "processor" plus 1 +#else core_id = std::stoi(s.substr(colon + 2)); +#endif } } else if (s.rfind("cpu MHz", 0) == 0) From c411763c2fb5208f2828a5ff25f77525c899a2e9 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 20 Jul 2022 09:41:00 -0400 Subject: [PATCH 441/659] split the note into two notes --- .../external-dicts-dict-sources.md | 33 ++++++++++--------- 1 file changed, 18 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 1a5308b5569..280dc1f54f4 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -5,9 +5,9 @@ sidebar_label: Sources of External Dictionaries # Sources of External Dictionaries -An external dictionary can be connected from many different sources. +An external dictionary can be connected to ClickHouse from many different sources. -If dictionary is configured using xml-file, the configuration looks like this: +If the dictionary is configured using an xml-file, the configuration looks like this: ``` xml @@ -24,7 +24,7 @@ If dictionary is configured using xml-file, the configuration looks like this: ``` -In case of [DDL-query](../../../sql-reference/statements/create/dictionary.md), equal configuration will looks like: +In case of [DDL-query](../../../sql-reference/statements/create/dictionary.md), the configuration described above will look like: ``` sql CREATE DICTIONARY dict_name (...) @@ -96,7 +96,7 @@ Setting fields: - `path` – The absolute path to the file. - `format` – The file format. All the formats described in [Formats](../../../interfaces/formats.md#formats) are supported. -When dictionary with source `FILE` is created via DDL command (`CREATE DICTIONARY ...`), the source file needs to be located in `user_files` directory, to prevent DB users accessing arbitrary file on ClickHouse node. +When a dictionary with source `FILE` is created via DDL command (`CREATE DICTIONARY ...`), the source file needs to be located in the `user_files` directory to prevent DB users from accessing arbitrary files on the ClickHouse node. **See Also** @@ -104,7 +104,7 @@ When dictionary with source `FILE` is created via DDL command (`CREATE DICTIONAR ## Executable File -Working with executable files depends on [how the dictionary is stored in memory](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file’s STDIN. Otherwise, ClickHouse starts executable file and treats its output as dictionary data. +Working with executable files depends on [how the dictionary is stored in memory](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file’s STDIN. Otherwise, ClickHouse starts the executable file and treats its output as dictionary data. Example of settings: @@ -120,22 +120,22 @@ Example of settings: Setting fields: -- `command` — The absolute path to the executable file, or the file name (if the program directory is written to `PATH`). +- `command` — The absolute path to the executable file, or the file name (if the command's directory is in the `PATH`). - `format` — The file format. All the formats described in [Formats](../../../interfaces/formats.md#formats) are supported. -- `command_termination_timeout` — executable script should contain main read-write loop. After dictionary is destroyed, pipe is closed, and executable file will have `command_termination_timeout` seconds to shutdown, before ClickHouse will send SIGTERM signal to child process. Specified in seconds. Default value is 10. Optional parameter. -- `command_read_timeout` - timeout for reading data from command stdout in milliseconds. Default value 10000. Optional parameter. -- `command_write_timeout` - timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter. +- `command_termination_timeout` — The executable script should contain a main read-write loop. After the dictionary is destroyed, the pipe is closed, and the executable file will have `command_termination_timeout` seconds to shutdown before ClickHouse will send a SIGTERM signal to the child process. `command_termination_timeout` is specified in seconds. Default value is 10. Optional parameter. +- `command_read_timeout` - Timeout for reading data from command stdout in milliseconds. Default value 10000. Optional parameter. +- `command_write_timeout` - Timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter. - `implicit_key` — The executable source file can return only values, and the correspondence to the requested keys is determined implicitly — by the order of rows in the result. Default value is false. -- `execute_direct` - If `execute_direct` = `1`, then `command` will be searched inside user_scripts folder specified by [user_scripts_path](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_scripts_path). Additional script arguments can be specified using whitespace separator. Example: `script_name arg1 arg2`. If `execute_direct` = `0`, `command` is passed as argument for `bin/sh -c`. Default value is `0`. Optional parameter. +- `execute_direct` - If `execute_direct` = `1`, then `command` will be searched inside user_scripts folder specified by [user_scripts_path](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_scripts_path). Additional script arguments can be specified using a whitespace separator. Example: `script_name arg1 arg2`. If `execute_direct` = `0`, `command` is passed as argument for `bin/sh -c`. Default value is `0`. Optional parameter. - `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. -That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled, otherwise, the DB user would be able to execute arbitrary binary on ClickHouse node. +That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled; otherwise, the DB user would be able to execute arbitrary binaries on the ClickHouse node. ## Executable Pool -Executable pool allows loading data from pool of processes. This source does not work with dictionary layouts that need to load all data from source. Executable pool works if the dictionary [is stored](external-dicts-dict-layout.md#ways-to-store-dictionaries-in-memory) using `cache`, `complex_key_cache`, `ssd_cache`, `complex_key_ssd_cache`, `direct`, `complex_key_direct` layouts. +Executable pool allows loading data from pool of processes. This source does not work with dictionary layouts that need to load all data from source. Executable pool works if the dictionary [is stored](external-dicts-dict-layout.md#ways-to-store-dictionaries-in-memory) using `cache`, `complex_key_cache`, `ssd_cache`, `complex_key_ssd_cache`, `direct`, or `complex_key_direct` layouts. -Executable pool will spawn pool of processes with specified command and keep them running until they exit. The program should read data from STDIN while it is available and output result to STDOUT, and it can wait for next block of data on STDIN. ClickHouse will not close STDIN after processing a block of data but will pipe another chunk of data when needed. The executable script should be ready for this way of data processing — it should poll STDIN and flush data to STDOUT early. +Executable pool will spawn a pool of processes with the specified command and keep them running until they exit. The program should read data from STDIN while it is available and output the result to STDOUT. It can wait for the next block of data on STDIN. ClickHouse will not close STDIN after processing a block of data, but will pipe another chunk of data when needed. The executable script should be ready for this way of data processing — it should poll STDIN and flush data to STDOUT early. Example of settings: @@ -553,10 +553,13 @@ Setting fields: :::note The `table` or `where` fields cannot be used together with the `query` field. And either one of the `table` or `query` fields must be declared. -There is no explicit parameter `secure`. Both cases: when establishing SSL-connection is mandatory and when it's not are handled automatically. ::: -MySQL can be connected on a local host via sockets. To do this, set `host` and `socket`. +:::note +There is no explicit parameter `secure`. When establishing an SSL-connection security is mandatory. +::: + +MySQL can be connected to on a local host via sockets. To do this, set `host` and `socket`. Example of settings: From 7d05ae786fc3ed0782bad11a67acac5384bfa74d Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 20 Jul 2022 16:27:11 +0200 Subject: [PATCH 442/659] Update 02327_capnproto_protobuf_empty_messages.reference --- .../02327_capnproto_protobuf_empty_messages.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.reference b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.reference index 0c7da0c3ce4..842cf482414 100644 --- a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.reference +++ b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.reference @@ -1,7 +1,7 @@ OK OK -FAIL -FAIL +OK +OK str String text String str String From 17a271ec30a4598d0eb4a29ec27afb58b4746e11 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 20 Jul 2022 14:33:46 +0000 Subject: [PATCH 443/659] Fix error codes --- src/Formats/CapnProtoUtils.cpp | 2 +- src/Formats/ProtobufSerializer.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index 774ab00e2e3..3db8672b6f9 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -565,7 +565,7 @@ NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema, names_and_types.emplace_back(name, type); } if (names_and_types.empty()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot convert CapnProto schema to ClickHouse table schema, all fields have unsupported types"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot convert CapnProto schema to ClickHouse table schema, all fields have unsupported types"); return names_and_types; } diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 203502150ad..42e02fd4f45 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -3574,7 +3574,7 @@ NamesAndTypesList protobufSchemaToCHSchema(const google::protobuf::Descriptor * schema.push_back(*name_and_type); } if (schema.empty()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot convert Protobuf schema to ClickHouse table schema, all fields have unsupported types"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot convert Protobuf schema to ClickHouse table schema, all fields have unsupported types"); return schema; } From 0e95dba3ef0cb37b73257573d313dddad29b42c6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 19 Jul 2022 19:55:21 +0300 Subject: [PATCH 444/659] Fix make clean (due to crosscompile of llvm) Without ADDITIONAL_CLEAN_FILES it reports an error: Cleaning... ninja: error: remove(contrib/llvm/llvm/NATIVE): Directory not empty ninja: error: remove(/bld/contrib/llvm/llvm/NATIVE): Directory not empty 0 files. Note, that ADDITIONAL_CLEAN_FILES had been added since cmake 3.15. Signed-off-by: Azat Khuzhin --- CMakeLists.txt | 2 +- contrib/llvm-cmake/CMakeLists.txt | 12 ++++++++++++ docs/en/development/build.md | 2 +- 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 7aa6b5d5a50..05f88f3530e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -1,4 +1,4 @@ -cmake_minimum_required(VERSION 3.14) +cmake_minimum_required(VERSION 3.15) project(ClickHouse LANGUAGES C CXX ASM) diff --git a/contrib/llvm-cmake/CMakeLists.txt b/contrib/llvm-cmake/CMakeLists.txt index a108e6537c9..4a4a5cef62e 100644 --- a/contrib/llvm-cmake/CMakeLists.txt +++ b/contrib/llvm-cmake/CMakeLists.txt @@ -93,6 +93,18 @@ set (CMAKE_CXX_STANDARD 17) set (LLVM_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/llvm/llvm") set (LLVM_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/llvm/llvm") add_subdirectory ("${LLVM_SOURCE_DIR}" "${LLVM_BINARY_DIR}") +set_directory_properties (PROPERTIES + # due to llvm crosscompile cmake does not know how to clean it, and on clean + # will lead to the following error: + # + # ninja: error: remove(contrib/llvm/llvm/NATIVE): Directory not empty + # + ADDITIONAL_CLEAN_FILES "${LLVM_BINARY_DIR}" + # llvm's cmake configuring this file only when cmake runs, + # and after clean cmake will not know that it should re-run, + # add explicitly depends from llvm-config.h + CMAKE_CONFIGURE_DEPENDS "${LLVM_BINARY_DIR}/include/llvm/Config/llvm-config.h" +) add_library (_llvm INTERFACE) target_link_libraries (_llvm INTERFACE ${REQUIRED_LLVM_LIBRARIES}) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index dbb90f8e537..e12884b61c4 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -75,7 +75,7 @@ This will create the `programs/clickhouse` executable, which can be used with `c The build requires the following components: - Git (is used only to checkout the sources, it’s not needed for the build) -- CMake 3.14 or newer +- CMake 3.15 or newer - Ninja - C++ compiler: clang-14 or newer - Linker: lld From d60a02829f6303d2a246534bfb0b47ecb9361588 Mon Sep 17 00:00:00 2001 From: James Morrison Date: Mon, 18 Jul 2022 12:35:08 +0000 Subject: [PATCH 445/659] Remove broken optimisation in Direct dictionary dictHas implementation I noticed this while working on another feature - if a set of keys being passed to `hasKeys` contains duplicates, then only one of the result slots for these keys will be populated. My fix uses to a simpler implementation which is likely slower, but is correct, which seems more important. No doubt faster approaches exist which are also correct. --- src/Dictionaries/DirectDictionary.cpp | 27 +++----- .../02366_direct_dictionary_dicthas.reference | 62 +++++++++++++++++++ .../02366_direct_dictionary_dicthas.sql | 56 +++++++++++++++++ 3 files changed, 128 insertions(+), 17 deletions(-) create mode 100644 tests/queries/0_stateless/02366_direct_dictionary_dicthas.reference create mode 100644 tests/queries/0_stateless/02366_direct_dictionary_dicthas.sql diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 6ecc216e370..20d8706ca54 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -171,15 +171,6 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys( auto requested_keys = requested_keys_extractor.extractAllKeys(); size_t requested_keys_size = requested_keys.size(); - HashMap requested_key_to_index; - requested_key_to_index.reserve(requested_keys_size); - - for (size_t i = 0; i < requested_keys.size(); ++i) - { - auto requested_key = requested_keys[i]; - requested_key_to_index[requested_key] = i; - } - auto result = ColumnUInt8::create(requested_keys_size, false); auto & result_data = result->getData(); @@ -205,15 +196,17 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys( { auto block_key = block_keys_extractor.extractCurrentKey(); - const auto * it = requested_key_to_index.find(block_key); - assert(it); + size_t index; + for (index = 0; index < requested_keys_size; ++index) + { + if (!result_data[index] && requested_keys[index] == block_key) + { + keys_found++; + result_data[index] = true; - size_t result_data_found_index = it->getMapped(); - /// block_keys_size cannot be used, due to duplicates. - keys_found += !result_data[result_data_found_index]; - result_data[result_data_found_index] = true; - - block_keys_extractor.rollbackCurrentKey(); + block_keys_extractor.rollbackCurrentKey(); + } + } } block_key_columns.clear(); diff --git a/tests/queries/0_stateless/02366_direct_dictionary_dicthas.reference b/tests/queries/0_stateless/02366_direct_dictionary_dicthas.reference new file mode 100644 index 00000000000..49b34f828cd --- /dev/null +++ b/tests/queries/0_stateless/02366_direct_dictionary_dicthas.reference @@ -0,0 +1,62 @@ +0 +0 +0 +1 +0 +1 +0 +2 +1 +0 +0 0 1 +1 0 1 +2 0 1 +3 1 0 +4 0 1 +5 1 0 +6 0 1 +7 2 0 +8 1 0 +9 0 1 +1 +1 +1 +0 +1 +0 +1 +0 +0 +1 +1 +1 +1 +0 +1 +0 +1 +0 +0 +1 +1 +1 +1 +0 +1 +0 +1 +0 +0 +1 +value_0 +value_0 +value_0 +UNKNOWN +value_0 +UNKNOWN +value_0 +UNKNOWN +UNKNOWN +value_0 +4 0 +6 1 diff --git a/tests/queries/0_stateless/02366_direct_dictionary_dicthas.sql b/tests/queries/0_stateless/02366_direct_dictionary_dicthas.sql new file mode 100644 index 00000000000..b111415b56c --- /dev/null +++ b/tests/queries/0_stateless/02366_direct_dictionary_dicthas.sql @@ -0,0 +1,56 @@ +-- Tags: no-backward-compatibility-check +DROP DATABASE IF EXISTS 02366_dictionary_db; +CREATE DATABASE 02366_dictionary_db; + +CREATE TABLE 02366_dictionary_db.dict_data +( + id UInt64, + val String +) +ENGINE = Memory; + +CREATE TABLE 02366_dictionary_db.lookup_data +( + id UInt64, + lookup_key UInt64, +) +ENGINE = Memory; + +INSERT INTO 02366_dictionary_db.dict_data VALUES(0, 'value_0'); + +INSERT INTO 02366_dictionary_db.lookup_data VALUES(0, 0); +INSERT INTO 02366_dictionary_db.lookup_data VALUES(1, 0); +INSERT INTO 02366_dictionary_db.lookup_data VALUES(2, 0); +INSERT INTO 02366_dictionary_db.lookup_data VALUES(3, 1); +INSERT INTO 02366_dictionary_db.lookup_data VALUES(4, 0); +INSERT INTO 02366_dictionary_db.lookup_data VALUES(5, 1); +INSERT INTO 02366_dictionary_db.lookup_data VALUES(6, 0); +INSERT INTO 02366_dictionary_db.lookup_data VALUES(7, 2); +INSERT INTO 02366_dictionary_db.lookup_data VALUES(8, 1); +INSERT INTO 02366_dictionary_db.lookup_data VALUES(9, 0); + +CREATE DICTIONARY 02366_dictionary_db.dict0 +( + id UInt64, + val String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data')) +LAYOUT(DIRECT()); + +SELECT lookup_key FROM 02366_dictionary_db.lookup_data ORDER BY id ASC; +SELECT id, lookup_key, dictHas(02366_dictionary_db.dict0, lookup_key) FROM 02366_dictionary_db.lookup_data ORDER BY id ASC; + +-- Nesting this way seems to help it make all the lookups as a single block, although even then it isn't guaranteed +SELECT dictHas(02366_dictionary_db.dict0, lk) FROM (SELECT any(lookup_key) as lk FROM 02366_dictionary_db.lookup_data group by id ORDER BY id ASC); +-- Same with this group by +SELECT dictHas(02366_dictionary_db.dict0, any(lookup_key)) FROM 02366_dictionary_db.lookup_data GROUP BY id ORDER BY id ASC; + + +SELECT dictHas(02366_dictionary_db.dict0, lookup_key) FROM 02366_dictionary_db.lookup_data ORDER BY id ASC; +SELECT dictGetOrDefault(02366_dictionary_db.dict0, 'val', lookup_key, 'UNKNOWN') FROM 02366_dictionary_db.lookup_data ORDER BY id ASC; +SELECT count(), has FROM 02366_dictionary_db.lookup_data group by dictHas(02366_dictionary_db.dict0, lookup_key) as has; + +DROP DICTIONARY 02366_dictionary_db.dict0; +DROP TABLE 02366_dictionary_db.lookup_data; +DROP TABLE 02366_dictionary_db.dict_data; From 658c39625fee7486486c79cbff52949ccc36d9f8 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 20 Jul 2022 08:27:45 -0700 Subject: [PATCH 446/659] Introduce a dependency to libuv when building NATS --- src/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 2424f2c42d8..cf8ae28df08 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -376,7 +376,7 @@ if (TARGET ch_contrib::rdkafka) endif() if (TARGET ch_contrib::nats_io) - dbms_target_link_libraries(PRIVATE ch_contrib::nats_io) + dbms_target_link_libraries(PRIVATE ch_contrib::nats_io ch_contrib::uv) endif() if (TARGET ch_contrib::sasl2) From eb4dcdb6a0457f550e6973151666027463949804 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 20 Jul 2022 17:47:47 +0200 Subject: [PATCH 447/659] Disable format_csv_allow_single_quotes by default --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8a1f47ec00e..9c02e2e0ba1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -660,7 +660,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) #define FORMAT_FACTORY_SETTINGS(M) \ M(Char, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \ - M(Bool, format_csv_allow_single_quotes, true, "If it is set to true, allow strings in single quotes.", 0) \ + M(Bool, format_csv_allow_single_quotes, false, "If it is set to true, allow strings in single quotes.", 0) \ M(Bool, format_csv_allow_double_quotes, true, "If it is set to true, allow strings in double quotes.", 0) \ M(Bool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \ M(Bool, input_format_csv_enum_as_number, false, "Treat inserted enum values in CSV formats as enum indices", 0) \ From d4c399d7f776c6ff4580471b76fecf17d5d24895 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 20 Jul 2022 17:50:24 +0200 Subject: [PATCH 448/659] Update settings history --- src/Core/SettingsChangesHistory.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 6d907395221..0bda25302b5 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -79,7 +79,8 @@ namespace SettingsChangesHistory static std::map settings_changes_history = { {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}}}, + {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"} + {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}}}, {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, From 55c625deb17008eed961be7f0237d96332346418 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 20 Jul 2022 17:52:18 +0200 Subject: [PATCH 449/659] Fix typo --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 0bda25302b5..ba60fb99308 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -79,7 +79,7 @@ namespace SettingsChangesHistory static std::map settings_changes_history = { {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"} + {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}}}, {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, From 85e55977988fff18bf08d1f2346ffc07a5de79e5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 20 Jul 2022 15:54:12 +0000 Subject: [PATCH 450/659] Fix some tests. --- src/DataTypes/getLeastSupertype.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index e09b3cbe938..fee3cf1553e 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -554,6 +554,10 @@ DataTypePtr getLeastSupertype(const DataTypes & types) UInt32 max_scale = 0; for (const auto & type : types) { + auto type_id = type->getTypeId(); + if (type_id != TypeIndex::Decimal32 && type_id != TypeIndex::Decimal64 && type_id != TypeIndex::Decimal128) + continue; + UInt32 scale = getDecimalScale(*type); if (scale > max_scale) max_scale = scale; From 828de2a674850ba1a2abefd35fc825c83f223449 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 20 Jul 2022 14:09:38 -0300 Subject: [PATCH 451/659] Add host regexp multiple PTR records integration tests --- .../runner/compose/docker_compose_coredns.yml | 9 ++ tests/integration/helpers/cluster.py | 55 ++++++++++++ .../__init__.py | 0 .../configs/host_regexp.xml | 11 +++ .../configs/listen_host.xml | 5 ++ .../coredns_config/Corefile | 8 ++ .../coredns_config/example.com | 1 + .../test.py | 88 +++++++++++++++++++ 8 files changed, 177 insertions(+) create mode 100644 docker/test/integration/runner/compose/docker_compose_coredns.yml create mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/__init__.py create mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml create mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml create mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile create mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com create mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/test.py diff --git a/docker/test/integration/runner/compose/docker_compose_coredns.yml b/docker/test/integration/runner/compose/docker_compose_coredns.yml new file mode 100644 index 00000000000..b329d4e0a46 --- /dev/null +++ b/docker/test/integration/runner/compose/docker_compose_coredns.yml @@ -0,0 +1,9 @@ +version: "2.3" + +services: + coredns: + image: coredns/coredns:latest + restart: always + volumes: + - ${COREDNS_CONFIG_DIR}/example.com:/example.com + - ${COREDNS_CONFIG_DIR}/Corefile:/Corefile diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0d32547358c..39b465a9cbb 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -364,6 +364,7 @@ class ClickHouseCluster: self.with_jdbc_bridge = False self.with_nginx = False self.with_hive = False + self.with_coredns = False self.with_minio = False self.minio_dir = os.path.join(self.instances_dir, "minio") @@ -407,6 +408,10 @@ class ClickHouseCluster: self.schema_registry_port = get_free_port() self.kafka_docker_id = self.get_instance_docker_id(self.kafka_host) + self.coredns_host = "coredns" + self.coredns_port = 53 + self.coredns_docker_id = self.get_instance_docker_id(self.coredns_host) + # available when with_kerberozed_kafka == True self.kerberized_kafka_host = "kerberized_kafka1" self.kerberized_kafka_port = get_free_port() @@ -1056,6 +1061,25 @@ class ClickHouseCluster: ] return self.base_mongo_cmd + def setup_coredns_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_coredns = True + env_variables["COREDNS_CONFIG_DIR"] = instance.path + "/" + "coredns_config" + self.base_cmd.extend( + ["--file", p.join(docker_compose_yml_dir, "docker_compose_coredns.yml")] + ) + + self.base_coredns_cmd = [ + "docker-compose", + "--env-file", + instance.env_file, + "--project-name", + self.project_name, + "--file", + p.join(docker_compose_yml_dir, "docker_compose_coredns.yml"), + ] + + return self.base_coredns_cmd + def setup_meili_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_meili = True env_variables["MEILI_HOST"] = self.meili_host @@ -1218,6 +1242,7 @@ class ClickHouseCluster: with_cassandra=False, with_jdbc_bridge=False, with_hive=False, + with_coredns=False, hostname=None, env_variables=None, image="clickhouse/integration-test", @@ -1301,6 +1326,7 @@ class ClickHouseCluster: with_cassandra=with_cassandra, with_jdbc_bridge=with_jdbc_bridge, with_hive=with_hive, + with_coredns=with_coredns, server_bin_path=self.server_bin_path, odbc_bridge_bin_path=self.odbc_bridge_bin_path, library_bridge_bin_path=self.library_bridge_bin_path, @@ -1460,6 +1486,11 @@ class ClickHouseCluster: ) ) + if with_coredns and not self.with_coredns: + cmds.append( + self.setup_coredns_cmd(instance, env_variables, docker_compose_yml_dir) + ) + if with_meili and not self.with_meili: cmds.append( self.setup_meili_cmd(instance, env_variables, docker_compose_yml_dir) @@ -1576,6 +1607,16 @@ class ClickHouseCluster: "IPAddress" ] + def get_instance_global_ipv6(self, instance_name): + logging.debug("get_instance_ip instance_name={}".format(instance_name)) + docker_id = self.get_instance_docker_id(instance_name) + # for cont in self.docker_client.containers.list(): + # logging.debug("CONTAINERS LIST: ID={} NAME={} STATUS={}".format(cont.id, cont.name, cont.status)) + handle = self.docker_client.containers.get(docker_id) + return list(handle.attrs["NetworkSettings"]["Networks"].values())[0][ + "GlobalIPv6Address" + ] + def get_container_id(self, instance_name): return self.get_instance_docker_id(instance_name) # docker_id = self.get_instance_docker_id(instance_name) @@ -2380,6 +2421,12 @@ class ClickHouseCluster: self.up_called = True self.wait_mongo_to_start(30, secure=self.with_mongo_secure) + if self.with_coredns and self.base_coredns_cmd: + logging.debug("Setup coredns") + run_and_check(self.base_coredns_cmd + common_opts) + self.up_called = True + time.sleep(10) + if self.with_meili and self.base_meili_cmd: logging.debug("Setup MeiliSearch") run_and_check(self.base_meili_cmd + common_opts) @@ -2717,6 +2764,7 @@ class ClickHouseInstance: with_azurite, with_jdbc_bridge, with_hive, + with_coredns, with_cassandra, server_bin_path, odbc_bridge_bin_path, @@ -2799,6 +2847,8 @@ class ClickHouseInstance: self.with_cassandra = with_cassandra self.with_jdbc_bridge = with_jdbc_bridge self.with_hive = with_hive + self.with_coredns = with_coredns + self.coredns_config_dir = p.abspath(p.join(base_path, "coredns_config")) self.main_config_name = main_config_name self.users_config_name = users_config_name @@ -3703,6 +3753,11 @@ class ClickHouseInstance: self.kerberos_secrets_dir, p.abspath(p.join(self.path, "secrets")) ) + if self.with_coredns: + shutil.copytree( + self.coredns_config_dir, p.abspath(p.join(self.path, "coredns_config")) + ) + # Copy config.d configs logging.debug( f"Copy custom test config files {self.custom_main_config_paths} to {self.config_d_dir}" diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/__init__.py b/tests/integration/test_host_regexp_multiple_ptr_records/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml b/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml new file mode 100644 index 00000000000..7a2141e6c7e --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml @@ -0,0 +1,11 @@ + + + + + + test1\.example\.com$ + + default + + + \ No newline at end of file diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml b/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml new file mode 100644 index 00000000000..58ef55cd3f3 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml @@ -0,0 +1,5 @@ + + :: + 0.0.0.0 + 1 + diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile new file mode 100644 index 00000000000..0dd198441dc --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile @@ -0,0 +1,8 @@ +. { + hosts /example.com { + reload "200ms" + fallthrough + } + forward . 127.0.0.11 + log +} diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com new file mode 100644 index 00000000000..9beb415c290 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com @@ -0,0 +1 @@ +filled in runtime, but needs to exist in order to be volume mapped in docker \ No newline at end of file diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/test.py b/tests/integration/test_host_regexp_multiple_ptr_records/test.py new file mode 100644 index 00000000000..cbfb140c905 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/test.py @@ -0,0 +1,88 @@ +import pytest +from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check +import os + +DOCKER_COMPOSE_PATH = get_docker_compose_path() +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +cluster = ClickHouseCluster(__file__) + +ch_server = cluster.add_instance( + "clickhouse-server", + with_coredns=True, + main_configs=["configs/listen_host.xml"], + user_configs=["configs/host_regexp.xml"], + ipv6_address="2001:3984:3989::1:1111", +) + +client = cluster.add_instance( + "clickhouse-client", + ipv6_address="2001:3984:3989::1:1112", +) + + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def setup_dns_server(ip): + domains_string = "test3.example.com test2.example.com test1.example.com" + example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com' + run_and_check(f"echo '{example_file_path}' > /custom_log", shell=True) + run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True) + + +def setup_ch_server(dns_server_ip): + ch_server.exec_in_container((["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"])) + ch_server.exec_in_container((["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"])) + ch_server.query("SYSTEM DROP DNS CACHE") + + +def build_endpoint_v4(ip): + return f"'http://{ip}:8123/?query=SELECT+1&user=test_dns'" + + +def build_endpoint_v6(ip): + return build_endpoint_v4(f"[{ip}]") + + +def test_host_regexp_multiple_ptr_v4_fails_with_wrong_resolution(started_cluster): + server_ip = cluster.get_instance_ip("clickhouse-server") + random_ip = "9.9.9.9" + dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) + + setup_dns_server(random_ip) + setup_ch_server(dns_server_ip) + + endpoint = build_endpoint_v4(server_ip) + + assert "1\n" != client.exec_in_container((["bash", "-c", f"curl {endpoint}"])) + + +def test_host_regexp_multiple_ptr_v4(started_cluster): + server_ip = cluster.get_instance_ip("clickhouse-server") + client_ip = cluster.get_instance_ip("clickhouse-client") + dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) + + setup_dns_server(client_ip) + setup_ch_server(dns_server_ip) + + endpoint = build_endpoint_v4(server_ip) + + assert "1\n" == client.exec_in_container((["bash", "-c", f"curl {endpoint}"])) + + +def test_host_regexp_multiple_ptr_v6(started_cluster): + setup_dns_server(client.ipv6_address) + setup_ch_server(cluster.get_instance_global_ipv6(cluster.coredns_host)) + + endpoint = build_endpoint_v6(ch_server.ipv6_address) + + assert "1\n" == client.exec_in_container((["bash", "-c", f"curl -6 {endpoint}"])) From 29613f4277403a5c6f256d62d4edb1e6226b62a1 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 20 Jul 2022 14:20:28 -0300 Subject: [PATCH 452/659] remove unused coredns symbols --- tests/integration/helpers/cluster.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 39b465a9cbb..162f9c069a6 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -409,8 +409,6 @@ class ClickHouseCluster: self.kafka_docker_id = self.get_instance_docker_id(self.kafka_host) self.coredns_host = "coredns" - self.coredns_port = 53 - self.coredns_docker_id = self.get_instance_docker_id(self.coredns_host) # available when with_kerberozed_kafka == True self.kerberized_kafka_host = "kerberized_kafka1" From 8b49952c7a70f1f7f133e63da61bdaadbe1a6f2a Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 20 Jul 2022 14:21:06 -0300 Subject: [PATCH 453/659] remove custom debugging log --- tests/integration/test_host_regexp_multiple_ptr_records/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/test.py b/tests/integration/test_host_regexp_multiple_ptr_records/test.py index cbfb140c905..a07ea5b924d 100644 --- a/tests/integration/test_host_regexp_multiple_ptr_records/test.py +++ b/tests/integration/test_host_regexp_multiple_ptr_records/test.py @@ -35,7 +35,6 @@ def started_cluster(): def setup_dns_server(ip): domains_string = "test3.example.com test2.example.com test1.example.com" example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com' - run_and_check(f"echo '{example_file_path}' > /custom_log", shell=True) run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True) From 0341c6c54bd7ac77200b4ca123208b195514ef20 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 20 Jul 2022 17:30:52 +0000 Subject: [PATCH 454/659] Try to firx keeper build. --- src/Core/SettingsFields.cpp | 6 ++++++ src/Core/SettingsFields.h | 4 ++++ 2 files changed, 10 insertions(+) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index eaf4e63a71a..d77a510d7f9 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -54,6 +54,7 @@ namespace return applyVisitor(FieldVisitorConvertToNumber(), f); } +#ifndef KEEPER_STANDALONE_BUILD Map stringToMap(const String & str) { /// Allow empty string as an empty map @@ -81,6 +82,8 @@ namespace return f.safeGet(); } +#endif + } template @@ -321,6 +324,8 @@ void SettingFieldString::readBinary(ReadBuffer & in) *this = std::move(str); } +#ifndef KEEPER_STANDALONE_BUILD + SettingFieldMap::SettingFieldMap(const Field & f) : value(fieldToMap(f)) {} String SettingFieldMap::toString() const @@ -360,6 +365,7 @@ void SettingFieldMap::readBinary(ReadBuffer & in) *this = map; } +#endif namespace { diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index a1434fc3f35..20f2b34084e 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -168,6 +168,8 @@ struct SettingFieldString void readBinary(ReadBuffer & in); }; +#ifndef KEEPER_STANDALONE_BUILD + struct SettingFieldMap { public: @@ -191,6 +193,8 @@ public: void readBinary(ReadBuffer & in); }; +#endif + struct SettingFieldChar { public: From 605fc5f12186438277e621166a75b53e48175c38 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 20 Jul 2022 21:02:54 +0200 Subject: [PATCH 455/659] Addressed review comments - Added finalizeImpl() override in ForkWriteBuffer to call finalize() of all the buffers. - Removed clearing buffer in ForkWriteBuffer destructor. --- src/IO/ForkWriteBuffer.cpp | 9 ++++++++- src/IO/ForkWriteBuffer.h | 1 + 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/IO/ForkWriteBuffer.cpp b/src/IO/ForkWriteBuffer.cpp index e50c8e2409c..e4868d5c9a8 100644 --- a/src/IO/ForkWriteBuffer.cpp +++ b/src/IO/ForkWriteBuffer.cpp @@ -43,10 +43,17 @@ void ForkWriteBuffer::nextImpl() } +void ForkWriteBuffer::finalizeImpl() +{ + for (const WriteBufferPtr & buffer : sources) + { + buffer->finalize(); + } +} + ForkWriteBuffer::~ForkWriteBuffer() { finalize(); - set(nullptr, 0); } diff --git a/src/IO/ForkWriteBuffer.h b/src/IO/ForkWriteBuffer.h index 56e9c445842..17fc82028a9 100644 --- a/src/IO/ForkWriteBuffer.h +++ b/src/IO/ForkWriteBuffer.h @@ -25,6 +25,7 @@ public: protected: void nextImpl() override; + void finalizeImpl() override; private: WriteBufferPtrs sources; From 6578d4f2b4a5f125682195cbf4023000797a5d3f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 20 Jul 2022 22:23:52 +0200 Subject: [PATCH 456/659] Mix lightweight deletes with old-style alter-delete's --- ...lightweight_delete_on_merge_tree.reference | 4 +- ...02319_lightweight_delete_on_merge_tree.sql | 3 +- .../02352_lightweight_delete.reference | 41 ++++++++++++------- .../0_stateless/02352_lightweight_delete.sql | 34 +++++++++++++-- 4 files changed, 61 insertions(+), 21 deletions(-) diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference index bc30d677348..fc646843eee 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference @@ -8,8 +8,8 @@ 1 1 1 -1 DELETE WHERE (c % 5) = 1 1 -1 DELETE WHERE c = 4 1 +0 UPDATE _row_exists = 0 WHERE (c % 5) = 1 1 +0 UPDATE _row_exists = 0 WHERE c = 4 1 0 MATERIALIZE INDEX i_c 1 0 UPDATE b = -1 WHERE a < 3 1 0 DROP INDEX i_c 1 diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql index 3c3df06915f..24afa5fb196 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql @@ -5,7 +5,8 @@ CREATE TABLE merge_table_standard_delete(id Int32, name String) ENGINE = MergeTr INSERT INTO merge_table_standard_delete select number, toString(number) from numbers(100); SET mutations_sync = 1; -SET allow_experimental_lightweight_delete = 1; +SET allow_experimental_lightweight_delete = 0; +SET allow_experimental_lightweight_delete_with_row_exists = 1; DELETE FROM merge_table_standard_delete WHERE id = 10; diff --git a/tests/queries/0_stateless/02352_lightweight_delete.reference b/tests/queries/0_stateless/02352_lightweight_delete.reference index 2c62a8cf5ea..3386b3294c3 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete.reference +++ b/tests/queries/0_stateless/02352_lightweight_delete.reference @@ -1,23 +1,36 @@ Rows in parts 1000000 Count 1000000 First row 0 10 -Delete 300K rows using lightweight DELETE +Delete 100K rows using lightweight DELETE Rows in parts 1000000 -Count 700000 -First row 300000 10 +Count 900000 +First row 100000 10 Force merge to cleanup deleted rows -Rows in parts 700000 -Count 700000 -First row 300000 10 -Delete 300K more rows using lightweight DELETE -Rows in parts 700000 -Count 400000 -First row 600000 10 +Rows in parts 900000 +Count 900000 +First row 100000 10 +Delete 100K more rows using lightweight DELETE +Rows in parts 900000 +Count 800000 +First row 200000 10 Do UPDATE mutation -Rows in parts 700000 -Count 400000 -First row 600000 1 +Rows in parts 900000 +Count 800000 +First row 200000 1 +Force merge to cleanup deleted rows +Rows in parts 800000 +Count 800000 +First row 200000 1 +Delete 100K more rows using lightweight DELETE +Rows in parts 800000 +Count 700000 +First row 300000 1 +Do ALTER DELETE mutation that does a "heavyweight" delete +Rows in parts 533333 +Count 466666 +First row 300001 10 +Delete 100K more rows using lightweight DELETE Force merge to cleanup deleted rows Rows in parts 400000 Count 400000 -First row 600000 1 +First row 400000 1 diff --git a/tests/queries/0_stateless/02352_lightweight_delete.sql b/tests/queries/0_stateless/02352_lightweight_delete.sql index cc66898d749..46336a57c3a 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete.sql +++ b/tests/queries/0_stateless/02352_lightweight_delete.sql @@ -13,9 +13,9 @@ SELECT 'Count', count() FROM lwd_test; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; -SELECT 'Delete 300K rows using lightweight DELETE'; +SELECT 'Delete 100K rows using lightweight DELETE'; --ALTER TABLE lwd_test UPDATE _row_exists = 0 WHERE id < 3000000; -DELETE FROM lwd_test WHERE id < 300000; +DELETE FROM lwd_test WHERE id < 100000; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; SELECT 'Count', count() FROM lwd_test; @@ -30,8 +30,8 @@ SELECT 'Count', count() FROM lwd_test; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; -SELECT 'Delete 300K more rows using lightweight DELETE'; -DELETE FROM lwd_test WHERE id < 600000; +SELECT 'Delete 100K more rows using lightweight DELETE'; +DELETE FROM lwd_test WHERE id < 200000; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; SELECT 'Count', count() FROM lwd_test; @@ -54,5 +54,31 @@ SELECT 'Count', count() FROM lwd_test; SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; +SELECT 'Delete 100K more rows using lightweight DELETE'; +DELETE FROM lwd_test WHERE id < 300000; + +SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; +SELECT 'Count', count() FROM lwd_test; +SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; + + +SELECT 'Do ALTER DELETE mutation that does a "heavyweight" delete'; +ALTER TABLE lwd_test DELETE WHERE id % 3 == 0; + +SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; +SELECT 'Count', count() FROM lwd_test; +SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; + +SELECT 'Delete 100K more rows using lightweight DELETE'; +DELETE FROM lwd_test WHERE id >= 300000 and id < 400000; + + +SELECT 'Force merge to cleanup deleted rows'; +OPTIMIZE TABLE lwd_test FINAL; + +SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; +SELECT 'Count', count() FROM lwd_test; +SELECT 'First row', id, length(value) FROM lwd_test ORDER BY id LIMIT 1; + DROP TABLE lwd_test; From 91043351aa4030855df3105c2d8b095f19ebd5a6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 20 Jul 2022 20:30:16 +0000 Subject: [PATCH 457/659] Fixing build. --- programs/disks/CommandCopy.cpp | 1 + programs/disks/CommandLink.cpp | 1 + programs/disks/CommandList.cpp | 1 + programs/disks/CommandListDisks.cpp | 1 + programs/disks/CommandMove.cpp | 1 + programs/disks/CommandRead.cpp | 1 + programs/disks/CommandRemove.cpp | 1 + programs/disks/CommandWrite.cpp | 1 + src/Access/DiskAccessStorage.cpp | 1 + src/AggregateFunctions/parseAggregateFunctionParameters.h | 2 ++ src/Backups/registerBackupEnginesFileAndDisk.cpp | 1 + src/Common/ShellCommand.h | 1 + src/Coordination/CoordinationSettings.cpp | 2 +- src/Coordination/KeeperServer.cpp | 1 + src/Disks/DiskEncrypted.cpp | 2 ++ src/Disks/IDisk.h | 6 ++++-- src/Disks/IO/CachedReadBufferFromRemoteFS.cpp | 1 + .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 1 + .../AzureBlobStorage/registerDiskAzureBlobStorage.cpp | 1 + src/Disks/ObjectStorages/DiskObjectStorage.cpp | 1 + src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp | 1 + src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h | 1 + src/Disks/ObjectStorages/IObjectStorage.cpp | 1 + src/Disks/ObjectStorages/IObjectStorage.h | 1 + src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 1 + src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 1 + src/IO/ReadBufferFromFileDescriptor.cpp | 1 + src/IO/ReadBufferFromFileDescriptor.h | 2 +- src/Parsers/IAST.h | 3 +-- src/Parsers/MySQL/ASTDeclareOption.h | 1 + src/Processors/Formats/Impl/ArrowBufferedStreams.cpp | 1 + src/Processors/QueryPlan/ReadFromMergeTree.cpp | 1 + src/Processors/QueryPlan/ReadFromMergeTree.h | 3 +++ src/Processors/TTL/TTLAggregationAlgorithm.cpp | 1 + src/QueryPipeline/RemoteInserter.cpp | 1 + src/Server/KeeperTCPHandler.h | 1 - src/Storages/MergeTree/BackgroundJobsAssignee.cpp | 1 + src/Storages/MergeTree/IMergeTreeReader.cpp | 1 + src/Storages/MergeTree/MergePlainMergeTreeTask.h | 1 + src/Storages/MergeTree/MergeTreeData.h | 3 +++ src/Storages/MergeTree/MergeTreePartition.cpp | 1 + src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp | 1 + src/Storages/ReadInOrderOptimizer.cpp | 1 + src/Storages/System/StorageSystemParts.cpp | 1 + 44 files changed, 52 insertions(+), 7 deletions(-) diff --git a/programs/disks/CommandCopy.cpp b/programs/disks/CommandCopy.cpp index f9cd7444287..1e5852fe651 100644 --- a/programs/disks/CommandCopy.cpp +++ b/programs/disks/CommandCopy.cpp @@ -1,6 +1,7 @@ #pragma once #include "ICommand.h" +#include namespace DB { diff --git a/programs/disks/CommandLink.cpp b/programs/disks/CommandLink.cpp index 6e9a7e64324..af48f0de097 100644 --- a/programs/disks/CommandLink.cpp +++ b/programs/disks/CommandLink.cpp @@ -1,6 +1,7 @@ #pragma once #include "ICommand.h" +#include namespace DB { diff --git a/programs/disks/CommandList.cpp b/programs/disks/CommandList.cpp index 8c6bfac3a9b..e76bb9e65fb 100644 --- a/programs/disks/CommandList.cpp +++ b/programs/disks/CommandList.cpp @@ -1,6 +1,7 @@ #pragma once #include "ICommand.h" +#include namespace DB { diff --git a/programs/disks/CommandListDisks.cpp b/programs/disks/CommandListDisks.cpp index 2bcbb045d67..22cffdd21fd 100644 --- a/programs/disks/CommandListDisks.cpp +++ b/programs/disks/CommandListDisks.cpp @@ -1,6 +1,7 @@ #pragma once #include "ICommand.h" +#include namespace DB { diff --git a/programs/disks/CommandMove.cpp b/programs/disks/CommandMove.cpp index 4a377cc7225..6322cf4b47d 100644 --- a/programs/disks/CommandMove.cpp +++ b/programs/disks/CommandMove.cpp @@ -1,6 +1,7 @@ #pragma once #include "ICommand.h" +#include namespace DB { diff --git a/programs/disks/CommandRead.cpp b/programs/disks/CommandRead.cpp index aa472fa217e..6b77a27e918 100644 --- a/programs/disks/CommandRead.cpp +++ b/programs/disks/CommandRead.cpp @@ -1,6 +1,7 @@ #pragma once #include "ICommand.h" +#include namespace DB { diff --git a/programs/disks/CommandRemove.cpp b/programs/disks/CommandRemove.cpp index d9925fbd93e..c1d3129bb8d 100644 --- a/programs/disks/CommandRemove.cpp +++ b/programs/disks/CommandRemove.cpp @@ -1,6 +1,7 @@ #pragma once #include "ICommand.h" +#include namespace DB { diff --git a/programs/disks/CommandWrite.cpp b/programs/disks/CommandWrite.cpp index c8ae91ea8d5..0b1c5823c81 100644 --- a/programs/disks/CommandWrite.cpp +++ b/programs/disks/CommandWrite.cpp @@ -1,6 +1,7 @@ #pragma once #include "ICommand.h" +#include namespace DB { diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 994abc7b53a..0cbe420f345 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include diff --git a/src/AggregateFunctions/parseAggregateFunctionParameters.h b/src/AggregateFunctions/parseAggregateFunctionParameters.h index a67bc081303..41a04324f6d 100644 --- a/src/AggregateFunctions/parseAggregateFunctionParameters.h +++ b/src/AggregateFunctions/parseAggregateFunctionParameters.h @@ -8,6 +8,8 @@ namespace DB { +struct Array; + Array getAggregateFunctionParametersArray( const ASTPtr & expression_list, const std::string & error_context, diff --git a/src/Backups/registerBackupEnginesFileAndDisk.cpp b/src/Backups/registerBackupEnginesFileAndDisk.cpp index 050a51939b6..380ae36a8e3 100644 --- a/src/Backups/registerBackupEnginesFileAndDisk.cpp +++ b/src/Backups/registerBackupEnginesFileAndDisk.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB diff --git a/src/Common/ShellCommand.h b/src/Common/ShellCommand.h index 190b5bc664e..dfc4a826f62 100644 --- a/src/Common/ShellCommand.h +++ b/src/Common/ShellCommand.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 34d69967828..c3decc071de 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -1,5 +1,5 @@ #include -#include +//#include #include #include #include diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 7c6ed227a06..864bb477786 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -21,6 +21,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index 8edb00e5a67..e6479727aad 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -8,6 +8,8 @@ #include #include #include +#include +#include namespace DB diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 941df99298b..2337fa00af5 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include @@ -41,6 +40,10 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +class IDisk; +using DiskPtr = std::shared_ptr; +using DisksMap = std::map; + class IReservation; using ReservationPtr = std::unique_ptr; using Reservations = std::vector; @@ -363,7 +366,6 @@ private: std::unique_ptr executor; }; -using DiskPtr = std::shared_ptr; using Disks = std::vector; /** diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp index b929cea0236..a3d5cfc408d 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace ProfileEvents diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 6df093ebd43..34b3d86b355 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp index 44976b7cf2d..dc70008649e 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index ca414a7ee72..0b7d16bd895 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp b/src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp index 99606a18517..b8ab2f49202 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h index b6426df1e9a..6d5ae12a157 100644 --- a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index f9c5c139b95..f3ac94768d8 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 06398b11aec..1ab2d75ff86 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -7,6 +7,7 @@ #include #include +#include #include #include #include diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 55c466d45f6..d36bf655c02 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -28,6 +28,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 98397224629..8f20671d841 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index 406b519df79..920e76cd7d0 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #ifdef HAS_RESERVED_IDENTIFIER diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index 40b0717c8b1..73c651189cd 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index b73919f4f36..1999eff37a8 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include @@ -26,7 +25,7 @@ namespace ErrorCodes using IdentifierNameSet = std::set; class WriteBuffer; - +using Strings = std::vector; /** Element of the syntax tree (hereinafter - directed acyclic graph with elements of semantics) */ diff --git a/src/Parsers/MySQL/ASTDeclareOption.h b/src/Parsers/MySQL/ASTDeclareOption.h index a9529924567..c493c49c61b 100644 --- a/src/Parsers/MySQL/ASTDeclareOption.h +++ b/src/Parsers/MySQL/ASTDeclareOption.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp index 5232d9166af..ebd9783b4fd 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index b4e143cc002..473798b1a2b 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 5d5c7e9cb2c..46be5ea1d7d 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB { @@ -9,6 +10,8 @@ using PartitionIdToMaxBlock = std::unordered_map; class Pipe; +using MergeTreeReadTaskCallback = std::function(PartitionReadRequest)>; + struct MergeTreeDataSelectSamplingData { bool use_sampling = false; diff --git a/src/Processors/TTL/TTLAggregationAlgorithm.cpp b/src/Processors/TTL/TTLAggregationAlgorithm.cpp index d8b022f0acb..0d160b8d32d 100644 --- a/src/Processors/TTL/TTLAggregationAlgorithm.cpp +++ b/src/Processors/TTL/TTLAggregationAlgorithm.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { diff --git a/src/QueryPipeline/RemoteInserter.cpp b/src/QueryPipeline/RemoteInserter.cpp index ce2ba23576d..58fed6e5466 100644 --- a/src/QueryPipeline/RemoteInserter.cpp +++ b/src/QueryPipeline/RemoteInserter.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index 9895c335c96..ee83c4fa21b 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -9,7 +9,6 @@ #include #include "IServer.h" #include -#include #include #include #include diff --git a/src/Storages/MergeTree/BackgroundJobsAssignee.cpp b/src/Storages/MergeTree/BackgroundJobsAssignee.cpp index 81445f40ed6..9617d16f6f1 100644 --- a/src/Storages/MergeTree/BackgroundJobsAssignee.cpp +++ b/src/Storages/MergeTree/BackgroundJobsAssignee.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index b8aeb8e6a5a..3acb4910e28 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.h b/src/Storages/MergeTree/MergePlainMergeTreeTask.h index 0f6d38d2cbf..7488b9655fe 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 26ac4d362ec..7c3bc21f391 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -56,6 +56,9 @@ struct ZeroCopyLock; class IBackupEntry; using BackupEntries = std::vector>>; +class MergeTreeTransaction; +using MergeTreeTransactionPtr = std::shared_ptr; + /// Auxiliary struct holding information about the future merged or mutated part. struct EmergingPartInfo { diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 81026989f95..4ea6ec11ecc 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 3eb638d15c0..9ed8fe0ad14 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/ReadInOrderOptimizer.cpp b/src/Storages/ReadInOrderOptimizer.cpp index 3ff4baa0b11..b188cef065e 100644 --- a/src/Storages/ReadInOrderOptimizer.cpp +++ b/src/Storages/ReadInOrderOptimizer.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 01bba669c0e..1b207d1d165 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB { From 965f96bd8476d0edbc50521029345db03d9f249f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 20 Jul 2022 20:44:47 +0000 Subject: [PATCH 458/659] DISTINCT in order: perf improvement + reduce allocations in DistinctSortedChunkTransform + use it for final distinct as well --- src/Processors/QueryPlan/DistinctStep.cpp | 23 ++------- .../DistinctSortedChunkTransform.cpp | 49 ++++++++++++------- .../Transforms/DistinctSortedChunkTransform.h | 16 +++--- ...ct_in_order_optimization_explain.reference | 4 +- ..._distinct_in_order_optimization_explain.sh | 2 +- 5 files changed, 49 insertions(+), 45 deletions(-) diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index 553732fbcc5..d1ca985bb2a 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -94,8 +94,10 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil SortDescription distinct_sort_desc = getSortDescription(input_stream.sort_description, columns); if (!distinct_sort_desc.empty()) { - /// pre-distinct for sorted chunks - if (pre_distinct) + const bool sorted_stream = input_stream.sort_mode == DataStream::SortMode::Stream; + /// pre-distinct for sorted chunks or + /// final distinct for sorted stream (sorting inside and among chunks) + if (pre_distinct || sorted_stream) { pipeline.addSimpleTransform( [&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr @@ -104,22 +106,7 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil return nullptr; return std::make_shared( - header, set_size_limits, limit_hint, distinct_sort_desc, columns); - }); - return; - } - /// final distinct for sorted stream (sorting inside and among chunks) - if (input_stream.sort_mode == DataStream::SortMode::Stream) - { - assert(input_stream.has_single_port); - - pipeline.addSimpleTransform( - [&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr - { - if (stream_type != QueryPipelineBuilder::StreamType::Main) - return nullptr; - - return std::make_shared(header, distinct_sort_desc, set_size_limits, limit_hint, columns); + header, set_size_limits, limit_hint, distinct_sort_desc, columns, sorted_stream); }); return; } diff --git a/src/Processors/Transforms/DistinctSortedChunkTransform.cpp b/src/Processors/Transforms/DistinctSortedChunkTransform.cpp index 064c827a8cc..8604cca5a5c 100644 --- a/src/Processors/Transforms/DistinctSortedChunkTransform.cpp +++ b/src/Processors/Transforms/DistinctSortedChunkTransform.cpp @@ -13,11 +13,13 @@ DistinctSortedChunkTransform::DistinctSortedChunkTransform( const SizeLimits & output_size_limits_, UInt64 limit_hint_, const SortDescription & sorted_columns_descr_, - const Names & source_columns) + const Names & source_columns, + const bool sorted_stream_) : ISimpleTransform(header_, header_, true) , limit_hint(limit_hint_) , output_size_limits(output_size_limits_) , sorted_columns_descr(sorted_columns_descr_) + , sorted_stream(sorted_stream_) { /// calculate sorted columns positions sorted_columns_pos.reserve(sorted_columns_descr.size()); @@ -43,7 +45,7 @@ DistinctSortedChunkTransform::DistinctSortedChunkTransform( /// reserve space in auxiliary column vectors for processing sorted_columns.reserve(sorted_columns_pos.size()); other_columns.reserve(other_columns_pos.size()); - current_key.reserve(sorted_columns.size()); + prev_chunk_latest_key.reserve(sorted_columns.size()); } void DistinctSortedChunkTransform::initChunkProcessing(const Columns & input_columns) @@ -101,28 +103,40 @@ size_t DistinctSortedChunkTransform::buildFilterForRange( return count; } -void DistinctSortedChunkTransform::setCurrentKey(const size_t row_pos) +void DistinctSortedChunkTransform::saveLatestKey(const size_t row_pos) { - current_key.clear(); + prev_chunk_latest_key.clear(); for (auto const & col : sorted_columns) { - current_key.emplace_back(col->cloneEmpty()); - current_key.back()->insertFrom(*col, row_pos); + prev_chunk_latest_key.emplace_back(col->cloneEmpty()); + prev_chunk_latest_key.back()->insertFrom(*col, row_pos); } } -bool DistinctSortedChunkTransform::isCurrentKey(const size_t row_pos) const +bool DistinctSortedChunkTransform::isKey(const size_t key_pos, const size_t row_pos) const { for (size_t i = 0; i < sorted_columns.size(); ++i) { - int res = current_key[i]->compareAt(0, row_pos, *sorted_columns[i], sorted_columns_descr[i].nulls_direction); + const int res = sorted_columns[i]->compareAt(key_pos, row_pos, *sorted_columns[i], sorted_columns_descr[i].nulls_direction); if (res != 0) return false; } return true; } -size_t DistinctSortedChunkTransform::getRangeEnd(size_t begin, size_t end) const +bool DistinctSortedChunkTransform::isLatestKeyFromPrevChunk(const size_t row_pos) const +{ + for (size_t i = 0; i < sorted_columns.size(); ++i) + { + const int res = prev_chunk_latest_key[i]->compareAt(0, row_pos, *sorted_columns[i], sorted_columns_descr[i].nulls_direction); + if (res != 0) + return false; + } + return true; +} + +template +size_t DistinctSortedChunkTransform::getRangeEnd(size_t begin, size_t end, Predicate pred) const { assert(begin < end); @@ -133,7 +147,7 @@ size_t DistinctSortedChunkTransform::getRangeEnd(size_t begin, size_t end) const for (size_t pos = begin; pos < linear_probe_end; ++pos) { - if (!isCurrentKey(pos)) + if (!pred(begin, pos)) return pos; } @@ -142,7 +156,7 @@ size_t DistinctSortedChunkTransform::getRangeEnd(size_t begin, size_t end) const while (low <= high) { size_t mid = low + (high - low) / 2; - if (isCurrentKey(mid)) + if (pred(begin, mid)) low = mid + 1; else { @@ -155,13 +169,13 @@ size_t DistinctSortedChunkTransform::getRangeEnd(size_t begin, size_t end) const std::pair DistinctSortedChunkTransform::continueWithPrevRange(const size_t chunk_rows, IColumn::Filter & filter) { - /// current_key is empty on very first transform() call + /// prev_chunk_latest_key is empty on very first transform() call /// or first row doesn't match a key from previous transform() - if (current_key.empty() || !isCurrentKey(0)) + if (prev_chunk_latest_key.empty() || !isLatestKeyFromPrevChunk(0)) return {0, 0}; size_t output_rows = 0; - const size_t range_end = getRangeEnd(0, chunk_rows); + const size_t range_end = getRangeEnd(0, chunk_rows, [&](size_t, size_t row_pos) { return isLatestKeyFromPrevChunk(row_pos); }); if (other_columns.empty()) std::fill(filter.begin(), filter.begin() + range_end, 0); /// skip rows already included in distinct on previous transform() else @@ -191,11 +205,8 @@ void DistinctSortedChunkTransform::transform(Chunk & chunk) size_t range_end = range_begin; while (range_end != chunk_rows) { - // set current key to find range - setCurrentKey(range_begin); - // find new range [range_begin, range_end) - range_end = getRangeEnd(range_begin, chunk_rows); + range_end = getRangeEnd(range_begin, chunk_rows, [&](size_t key_pos, size_t row_pos) { return isKey(key_pos, row_pos); }); // update filter for range if (other_columns.empty()) @@ -214,6 +225,8 @@ void DistinctSortedChunkTransform::transform(Chunk & chunk) range_begin = range_end; } + saveLatestKey(chunk_rows - 1); + /// apply the built filter for (auto & input_column : input_columns) input_column = input_column->filter(filter, output_rows); diff --git a/src/Processors/Transforms/DistinctSortedChunkTransform.h b/src/Processors/Transforms/DistinctSortedChunkTransform.h index 2e21c36f7dc..0ce8addbf7e 100644 --- a/src/Processors/Transforms/DistinctSortedChunkTransform.h +++ b/src/Processors/Transforms/DistinctSortedChunkTransform.h @@ -32,9 +32,10 @@ public: const SizeLimits & output_size_limits_, UInt64 limit_hint_, const SortDescription & sorted_columns_descr_, - const Names & source_columns_); + const Names & source_columns_, + bool sorted_stream_); - String getName() const override { return "DistinctSortedChunkTransform"; } + String getName() const override { return (!sorted_stream ? "DistinctSortedChunkTransform" : "DistinctSortedStreamTransform"); } protected: void transform(Chunk & chunk) override; @@ -43,9 +44,11 @@ private: void initChunkProcessing(const Columns & input_columns); std::pair continueWithPrevRange(size_t chunk_rows, IColumn::Filter & filter); size_t ordinaryDistinctOnRange(IColumn::Filter & filter, size_t range_begin, size_t range_end, bool clear_data); - inline void setCurrentKey(size_t row_pos); - inline bool isCurrentKey(size_t row_pos) const; - inline size_t getRangeEnd(size_t range_begin, size_t range_end) const; + inline void saveLatestKey(size_t row_pos); + inline bool isLatestKeyFromPrevChunk(size_t row_pos) const; + inline bool isKey(size_t key_pos, size_t row_pos) const; + template + inline size_t getRangeEnd(size_t range_begin, size_t range_end, Predicate pred) const; template size_t buildFilterForRange(Method & method, IColumn::Filter & filter, size_t range_begin, size_t range_end, bool clear_data); @@ -66,7 +69,8 @@ private: Sizes other_columns_sizes; ColumnRawPtrs other_columns; // used during processing - MutableColumns current_key; + MutableColumns prev_chunk_latest_key; + const bool sorted_stream = false; }; } diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index 2dac69edc41..f30d3fa30ea 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -7,13 +7,13 @@ DistinctSortedChunkTransform -- distinct with primary key prefix -> pre-distinct optimization only DistinctSortedChunkTransform -- distinct with primary key prefix and order by on column in distinct -> pre-distinct and final distinct optimization -DistinctSortedTransform +DistinctSortedStreamTransform DistinctSortedChunkTransform -- distinct with primary key prefix and order by on column _not_ in distinct -> pre-distinct optimization only DistinctSortedChunkTransform -- distinct with non-primary key prefix -> no optimizations No optimizations -- distinct with non-primary key prefix and order by on column in distinct -> final distinct optimization only -DistinctSortedTransform +DistinctSortedStreamTransform -- distinct with non-primary key prefix and order by on column _not_ in distinct -> no optimizations No optimizations diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh index 21f50a147ac..9af0e98ecf4 100755 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DISABLE_OPTIMIZATION="set optimize_distinct_in_order=0" ENABLE_OPTIMIZATION="set optimize_distinct_in_order=1" -GREP_OPTIMIZATIONS="grep 'DistinctSortedChunkTransform\|DistinctSortedTransform'" +GREP_OPTIMIZATIONS="grep 'DistinctSortedChunkTransform\|DistinctSortedStreamTransform'" TRIM_LEADING_SPACES="sed -e 's/^[ \t]*//'" FIND_OPTIMIZATIONS="$GREP_OPTIMIZATIONS | $TRIM_LEADING_SPACES" From b0e7d348f131114d283d22c70984c90cf916ed4b Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 20 Jul 2022 21:16:59 +0000 Subject: [PATCH 459/659] Fix tests --- tests/queries/0_stateless/02098_with_types_use_header.sh | 6 +++--- .../0_stateless/02155_csv_with_strings_with_slash.sh | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02098_with_types_use_header.sh b/tests/queries/0_stateless/02098_with_types_use_header.sh index 5d88a994052..457182a08f2 100755 --- a/tests/queries/0_stateless/02098_with_types_use_header.sh +++ b/tests/queries/0_stateless/02098_with_types_use_header.sh @@ -19,9 +19,9 @@ echo -e "y\tz\tx\nString\tDate\tUInt32\ntext\t2020-01-01\t1" | $CLICKHOUSE_CLIEN echo -e "x\tz\ty\nUInt32\tString\tDate\n1\ttext\t2020-01-01" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CustomSeparatedWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' echo "CSVWithNamesAndTypes" -echo -e "'x','y','z'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' -echo -e "'y','z','x'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" && echo 'OK' || echo 'FAIL' -echo -e "'x','z','y'\n'UInt32','String',Date'\n1,'text','2020-01-01'" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e "'x','y','z'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --format_csv_allow_single_quotes=1 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e "'y','z','x'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --format_csv_allow_single_quotes=1 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" && echo 'OK' || echo 'FAIL' +echo -e "'x','z','y'\n'UInt32','String',Date'\n1,'text','2020-01-01'" | $CLICKHOUSE_CLIENT --format_csv_allow_single_quotes=1 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' echo "JSONCompactEachRowWithNamesAndTypes" diff --git a/tests/queries/0_stateless/02155_csv_with_strings_with_slash.sh b/tests/queries/0_stateless/02155_csv_with_strings_with_slash.sh index 08d380bf559..4f38d662590 100755 --- a/tests/queries/0_stateless/02155_csv_with_strings_with_slash.sh +++ b/tests/queries/0_stateless/02155_csv_with_strings_with_slash.sh @@ -10,13 +10,13 @@ ${CLICKHOUSE_CLIENT} --query="create table test_02155_csv (A Int64, S String, D echo "input_format_null_as_default = 1" -cat $CUR_DIR/data_csv/csv_with_slash.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_02155_csv SETTINGS input_format_null_as_default = 1 FORMAT CSV" +cat $CUR_DIR/data_csv/csv_with_slash.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_02155_csv SETTINGS input_format_null_as_default = 1, format_csv_allow_single_quotes=1 FORMAT CSV" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM test_02155_csv" ${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE test_02155_csv" echo "input_format_null_as_default = 0" -cat $CUR_DIR/data_csv/csv_with_slash.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_02155_csv SETTINGS input_format_null_as_default = 0 FORMAT CSV" +cat $CUR_DIR/data_csv/csv_with_slash.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_02155_csv SETTINGS format_csv_allow_single_quotes = 1, input_format_null_as_default = 0 FORMAT CSV" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM test_02155_csv" From 635a566bec3317e93955de59bf84d4cd14f1c309 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 20 Jul 2022 21:37:46 +0000 Subject: [PATCH 460/659] Comment change --- src/Processors/QueryPlan/DistinctStep.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index d1ca985bb2a..b9a8932b409 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -95,8 +95,7 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil if (!distinct_sort_desc.empty()) { const bool sorted_stream = input_stream.sort_mode == DataStream::SortMode::Stream; - /// pre-distinct for sorted chunks or - /// final distinct for sorted stream (sorting inside and among chunks) + /// pre-distinct for sorted chunks or final distinct for sorted stream (sorting inside and among chunks) if (pre_distinct || sorted_stream) { pipeline.addSimpleTransform( From a3eb75becab03810d6c6ec5d4cc1254d2347e34b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 20 Jul 2022 22:26:17 +0000 Subject: [PATCH 461/659] Test: compare result of DISTINCT with and w/o optimization --- .../02317_distinct_in_order_optimization.reference | 1 + .../02317_distinct_in_order_optimization.sql | 11 +++++++++++ 2 files changed, 12 insertions(+) diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference index b53b561137e..05e65c92805 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference @@ -77,3 +77,4 @@ 2 2 1 1 0 0 +-- check that distinct with and w/o optimization produce the same result diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql index afe53a95b26..8b1385768ac 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql @@ -44,3 +44,14 @@ select '-- distinct with non-key prefix and non-sorted column, order by non-sort select distinct b,c from distinct_in_order order by c desc; drop table if exists distinct_in_order sync; + +select '-- check that distinct with and w/o optimization produce the same result'; +drop table if exists distinct_in_order sync; +drop table if exists ordinary_distinct sync; +create table distinct_in_order (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate); +insert into distinct_in_order select distinct CounterID, EventDate from hits_v1 order by CounterID, EventDate settings optimize_distinct_in_order=1; +create table ordinary_distinct (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate); +insert into ordinary_distinct select distinct CounterID, EventDate from hits_v1 settings optimize_distinct_in_order=0; +select distinct * from distinct_in_order except select * from ordinary_distinct; +drop table if exists distinct_in_order sync; +drop table if exists ordinary_distinct sync; From 052e7d3fbcdfce833e1709694948b4dea5cb32b1 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 20 Jul 2022 22:43:55 +0000 Subject: [PATCH 462/659] hits_v1 -> test.hits --- .../0_stateless/02317_distinct_in_order_optimization.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql index 8b1385768ac..29fcf47fa20 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql @@ -49,9 +49,9 @@ select '-- check that distinct with and w/o optimization produce the same result drop table if exists distinct_in_order sync; drop table if exists ordinary_distinct sync; create table distinct_in_order (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate); -insert into distinct_in_order select distinct CounterID, EventDate from hits_v1 order by CounterID, EventDate settings optimize_distinct_in_order=1; +insert into distinct_in_order select distinct CounterID, EventDate from test.hits order by CounterID, EventDate settings optimize_distinct_in_order=1; create table ordinary_distinct (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate); -insert into ordinary_distinct select distinct CounterID, EventDate from hits_v1 settings optimize_distinct_in_order=0; +insert into ordinary_distinct select distinct CounterID, EventDate from test.hits settings optimize_distinct_in_order=0; select distinct * from distinct_in_order except select * from ordinary_distinct; drop table if exists distinct_in_order sync; drop table if exists ordinary_distinct sync; From 40d5627510a742d200c437e76e3ff947685afa56 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 21 Jul 2022 02:26:28 +0300 Subject: [PATCH 463/659] Avoid loading toolchain file multiple times to avoid confusing ccache (#39387) During first run of cmake the toolchain file will be loaded twice, - /usr/share/cmake-3.23/Modules/CMakeDetermineSystem.cmake - /bld/CMakeFiles/3.23.2/CMakeSystem.cmake But once you already have non-empty cmake cache it will be loaded only once: - /bld/CMakeFiles/3.23.2/CMakeSystem.cmake This has no harm except for double load of toolchain will add --gcc-toolchain multiple times that will not allow ccache to reuse the cache. Signed-off-by: Azat Khuzhin --- cmake/linux/toolchain-x86_64.cmake | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/cmake/linux/toolchain-x86_64.cmake b/cmake/linux/toolchain-x86_64.cmake index 965ea024ab7..bdcfcfa013a 100644 --- a/cmake/linux/toolchain-x86_64.cmake +++ b/cmake/linux/toolchain-x86_64.cmake @@ -1,3 +1,19 @@ +if (_CLICKHOUSE_TOOLCHAIN_FILE_LOADED) + # During first run of cmake the toolchain file will be loaded twice, + # - /usr/share/cmake-3.23/Modules/CMakeDetermineSystem.cmake + # - /bld/CMakeFiles/3.23.2/CMakeSystem.cmake + # + # But once you already have non-empty cmake cache it will be loaded only + # once: + # - /bld/CMakeFiles/3.23.2/CMakeSystem.cmake + # + # This has no harm except for double load of toolchain will add + # --gcc-toolchain multiple times that will not allow ccache to reuse the + # cache. + return() +endif() +set (_CLICKHOUSE_TOOLCHAIN_FILE_LOADED ON) + set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) set (CMAKE_SYSTEM_NAME "Linux") From 4b3f90870db414e31eed551beb04c9e863a2105a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 21 Jul 2022 02:29:35 +0300 Subject: [PATCH 464/659] Remove specialization global lock/unlock from ActionLocksManager (#39215) This had been done in InterpreterSystemQuery explicitly, with grants checking. Signed-off-by: Azat Khuzhin --- src/Interpreters/ActionLocksManager.cpp | 22 ---------------------- src/Interpreters/ActionLocksManager.h | 4 ---- 2 files changed, 26 deletions(-) diff --git a/src/Interpreters/ActionLocksManager.cpp b/src/Interpreters/ActionLocksManager.cpp index 8f081f3d470..7b57b8803cd 100644 --- a/src/Interpreters/ActionLocksManager.cpp +++ b/src/Interpreters/ActionLocksManager.cpp @@ -23,20 +23,6 @@ ActionLocksManager::ActionLocksManager(ContextPtr context_) : WithContext(contex { } -template -inline void forEachTable(F && f, ContextPtr context) -{ - for (auto & elem : DatabaseCatalog::instance().getDatabases()) - for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next()) - if (auto table = iterator->table()) - f(table); -} - -void ActionLocksManager::add(StorageActionBlockType action_type, ContextPtr context_) -{ - forEachTable([&](const StoragePtr & table) { add(table, action_type); }, context_); -} - void ActionLocksManager::add(const StorageID & table_id, StorageActionBlockType action_type) { if (auto table = DatabaseCatalog::instance().tryGetTable(table_id, getContext())) @@ -54,14 +40,6 @@ void ActionLocksManager::add(const StoragePtr & table, StorageActionBlockType ac } } -void ActionLocksManager::remove(StorageActionBlockType action_type) -{ - std::lock_guard lock(mutex); - - for (auto & storage_elem : storage_locks) - storage_elem.second.erase(action_type); -} - void ActionLocksManager::remove(const StorageID & table_id, StorageActionBlockType action_type) { if (auto table = DatabaseCatalog::instance().tryGetTable(table_id, getContext())) diff --git a/src/Interpreters/ActionLocksManager.h b/src/Interpreters/ActionLocksManager.h index be112e71950..d1da81a8dd4 100644 --- a/src/Interpreters/ActionLocksManager.h +++ b/src/Interpreters/ActionLocksManager.h @@ -20,14 +20,10 @@ class ActionLocksManager : WithContext public: explicit ActionLocksManager(ContextPtr context); - /// Adds new locks for each table - void add(StorageActionBlockType action_type, ContextPtr context); /// Add new lock for a table if it has not been already added void add(const StorageID & table_id, StorageActionBlockType action_type); void add(const StoragePtr & table, StorageActionBlockType action_type); - /// Remove locks for all tables - void remove(StorageActionBlockType action_type); /// Removes a lock for a table if it exists void remove(const StorageID & table_id, StorageActionBlockType action_type); void remove(const StoragePtr & table, StorageActionBlockType action_type); From 92995a832b32330c0f60dbd7e56171711dae2774 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 21 Jul 2022 01:45:16 +0200 Subject: [PATCH 465/659] Revert "Fix WriteBuffer finalize in destructor when cacnel query" --- src/Processors/Formats/IOutputFormat.cpp | 4 +--- src/Storages/HDFS/StorageHDFS.cpp | 7 ------- src/Storages/StorageFile.cpp | 7 ------- src/Storages/StorageS3.cpp | 7 ------- src/Storages/StorageURL.cpp | 7 ------- src/Storages/StorageURL.h | 1 - 6 files changed, 1 insertion(+), 32 deletions(-) diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index 47ebaa9c5f5..3c4e6861151 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -73,6 +73,7 @@ void IOutputFormat::work() setRowsBeforeLimit(rows_before_limit_counter->get()); finalize(); + finalized = true; return; } @@ -119,12 +120,9 @@ void IOutputFormat::write(const Block & block) void IOutputFormat::finalize() { - if (finalized) - return; writePrefixIfNot(); writeSuffixIfNot(); finalizeImpl(); - finalized = true; } } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 1e9f9286633..57e893e9683 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -430,13 +430,6 @@ public: writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } - void onCancel() override - { - if (!writer) - return; - onFinish(); - } - void onException() override { if (!writer) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 30e5042fb06..d138104018a 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -813,13 +813,6 @@ public: writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } - void onCancel() override - { - if (!writer) - return; - onFinish(); - } - void onException() override { if (!writer) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 6a7c682199d..130bc75a65c 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -602,13 +602,6 @@ public: writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } - void onCancel() override - { - if (!writer) - return; - onFinish(); - } - void onException() override { if (!writer) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index cdc288ba788..15ae23305f3 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -450,13 +450,6 @@ void StorageURLSink::consume(Chunk chunk) writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } -void StorageURLSink::onCancel() -{ - if (!writer) - return; - onFinish(); -} - void StorageURLSink::onException() { if (!writer) diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 320c6258ee5..79371242bb1 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -114,7 +114,6 @@ public: std::string getName() const override { return "StorageURLSink"; } void consume(Chunk chunk) override; - void onCancel() override; void onException() override; void onFinish() override; From 72dda351086b51810034d4865651bf58f5ea22d6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 21 Jul 2022 04:34:04 +0300 Subject: [PATCH 466/659] Revert "[RFC] Fix LSan by fixing getauxval()" --- base/glibc-compatibility/musl/getauxval.c | 154 ++++------------------ src/Common/tests/gtest_lsan.cpp | 33 ----- 2 files changed, 23 insertions(+), 164 deletions(-) delete mode 100644 src/Common/tests/gtest_lsan.cpp diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index 616e43ef342..dad7aa938d7 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -1,176 +1,68 @@ -#include "atomic.h" #include -#include // open -#include // O_RDONLY -#include // read, close -#include // ssize_t -#include // perror, fprintf -#include // ElfW +#include "atomic.h" +#include // __environ #include -#define ARRAY_SIZE(a) sizeof((a))/sizeof((a[0])) - -// We don't have libc struct available here. -// Compute aux vector manually (from /proc/self/auxv). -// -// Right now there is only 51 AT_* constants, -// so 64 should be enough until this implementation will be replaced with musl. -static unsigned long __auxv_procfs[64]; +// We don't have libc struct available here. Compute aux vector manually. +static unsigned long * __auxv = NULL; static unsigned long __auxv_secure = 0; -// Common -static unsigned long * __auxv_environ = NULL; -static void * volatile getauxval_func; - -static unsigned long __auxv_init_environ(unsigned long type); - -// -// auxv from procfs interface -// -ssize_t __retry_read(int fd, void * buf, size_t count) -{ - for (;;) - { - ssize_t ret = read(fd, buf, count); - if (ret == -1) - { - if (errno == EINTR) - { - continue; - } - perror("Cannot read /proc/self/auxv"); - abort(); - } - return ret; - } -} -unsigned long __getauxval_procfs(unsigned long type) -{ - if (type == AT_SECURE) - { - return __auxv_secure; - } - - if (type >= ARRAY_SIZE(__auxv_procfs)) - { - errno = ENOENT; - return 0; - } - - return __auxv_procfs[type]; -} -static unsigned long __auxv_init_procfs(unsigned long type) -{ - // For debugging: - // - od -t dL /proc/self/auxv - // - LD_SHOW_AUX= ls - int fd = open("/proc/self/auxv", O_RDONLY); - // It is possible in case of: - // - no procfs mounted - // - on android you are not able to read it unless running from shell or debugging - // - some other issues - if (fd == -1) - { - // Fallback to environ. - a_cas_p(&getauxval_func, (void *)__auxv_init_procfs, (void *)__auxv_init_environ); - return __auxv_init_environ(type); - } - - ElfW(auxv_t) aux; - - /// NOTE: sizeof(aux) is very small (less then PAGE_SIZE), so partial read should not be possible. - _Static_assert(sizeof(aux) < 4096, "Unexpected sizeof(aux)"); - while (__retry_read(fd, &aux, sizeof(aux)) == sizeof(aux)) - { - if (aux.a_type >= ARRAY_SIZE(__auxv_procfs)) - { - fprintf(stderr, "AT_* is out of range: %li (maximum allowed is %zu)\n", aux.a_type, ARRAY_SIZE(__auxv_procfs)); - abort(); - } - if (__auxv_procfs[aux.a_type]) - { - fprintf(stderr, "AUXV already has value (%zu)\n", __auxv_procfs[aux.a_type]); - abort(); - } - __auxv_procfs[aux.a_type] = aux.a_un.a_val; - } - close(fd); - - __auxv_secure = __getauxval_procfs(AT_SECURE); - - // Now we've initialized __auxv_procfs, next time getauxval() will only call __get_auxval(). - a_cas_p(&getauxval_func, (void *)__auxv_init_procfs, (void *)__getauxval_procfs); - - return __getauxval_procfs(type); -} - -// -// auxv from environ interface -// -// NOTE: environ available only after static initializers, -// so you cannot rely on this if you need getauxval() before. -// -// Good example of such user is sanitizers, for example -// LSan will not work with __auxv_init_environ(), -// since it needs getauxval() before. -// static size_t __find_auxv(unsigned long type) { size_t i; - for (i = 0; __auxv_environ[i]; i += 2) + for (i = 0; __auxv[i]; i += 2) { - if (__auxv_environ[i] == type) - { + if (__auxv[i] == type) return i + 1; - } } return (size_t) -1; } -unsigned long __getauxval_environ(unsigned long type) + +unsigned long __getauxval(unsigned long type) { if (type == AT_SECURE) return __auxv_secure; - if (__auxv_environ) + if (__auxv) { size_t index = __find_auxv(type); if (index != ((size_t) -1)) - return __auxv_environ[index]; + return __auxv[index]; } errno = ENOENT; return 0; } -static unsigned long __auxv_init_environ(unsigned long type) + +static void * volatile getauxval_func; + +static unsigned long __auxv_init(unsigned long type) { if (!__environ) { - // __environ is not initialized yet so we can't initialize __auxv_environ right now. + // __environ is not initialized yet so we can't initialize __auxv right now. // That's normally occurred only when getauxval() is called from some sanitizer's internal code. errno = ENOENT; return 0; } - // Initialize __auxv_environ and __auxv_secure. + // Initialize __auxv and __auxv_secure. size_t i; for (i = 0; __environ[i]; i++); - __auxv_environ = (unsigned long *) (__environ + i + 1); + __auxv = (unsigned long *) (__environ + i + 1); size_t secure_idx = __find_auxv(AT_SECURE); if (secure_idx != ((size_t) -1)) - __auxv_secure = __auxv_environ[secure_idx]; + __auxv_secure = __auxv[secure_idx]; - // Now we need to switch to __getauxval_environ for all later calls, since - // everything is initialized. - a_cas_p(&getauxval_func, (void *)__auxv_init_environ, (void *)__getauxval_environ); + // Now we've initialized __auxv, next time getauxval() will only call __get_auxval(). + a_cas_p(&getauxval_func, (void *)__auxv_init, (void *)__getauxval); - return __getauxval_environ(type); + return __getauxval(type); } -// Callchain: -// - __auxv_init_procfs -> __getauxval_environ -// - __auxv_init_procfs -> __auxv_init_environ -> __getauxval_environ -static void * volatile getauxval_func = (void *)__auxv_init_procfs; +// First time getauxval() will call __auxv_init(). +static void * volatile getauxval_func = (void *)__auxv_init; unsigned long getauxval(unsigned long type) { diff --git a/src/Common/tests/gtest_lsan.cpp b/src/Common/tests/gtest_lsan.cpp deleted file mode 100644 index f6e1984ec58..00000000000 --- a/src/Common/tests/gtest_lsan.cpp +++ /dev/null @@ -1,33 +0,0 @@ -#include // ADDRESS_SANITIZER - -#ifdef ADDRESS_SANITIZER - -#include -#include - -#include -#include - -/// Test that ensures that LSan works. -/// -/// Regression test for the case when it may not work, -/// because of broken getauxval() [1]. -/// -/// [1]: https://github.com/ClickHouse/ClickHouse/pull/33957 -TEST(Common, LSan) -{ - int sanitizers_exit_code = 1; - - ASSERT_EXIT({ - std::thread leak_in_thread([]() - { - void * leak = malloc(4096); - ASSERT_NE(leak, nullptr); - }); - leak_in_thread.join(); - - __lsan_do_leak_check(); - }, ::testing::ExitedWithCode(sanitizers_exit_code), ".*LeakSanitizer: detected memory leaks.*"); -} - -#endif From 6d7403d78c37fd8d6bdda0240f4c0324f71a80c6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 21 Jul 2022 03:39:34 +0200 Subject: [PATCH 467/659] Add changelog for 22.7 --- CHANGELOG.md | 87 ++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 87 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index dfc51952250..6c1af61e115 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v22.7, 2022-07-21](#226)**
**[ClickHouse release v22.6, 2022-06-16](#226)**
**[ClickHouse release v22.5, 2022-05-19](#225)**
**[ClickHouse release v22.4, 2022-04-20](#224)**
@@ -7,6 +8,92 @@ **[ClickHouse release v22.1, 2022-01-18](#221)**
**[Changelog for 2021](https://clickhouse.com/docs/en/whats-new/changelog/2021/)**
+###
ClickHouse release 22.7, 2022-07-21 + +#### Upgrade Notes +* Force rewriting comma join to inner by default (set default value `cross_to_inner_join_rewrite = 2`). To have old behavior set `cross_to_inner_join_rewrite = 1`. [#39326](https://github.com/ClickHouse/ClickHouse/pull/39326) ([Vladimir C](https://github.com/vdimir)). If you will face any incompatibilities, you can turn this setting back. + +#### New Feature +* Add new `direct` join algorithm for `EmbeddedRocksDB` tables, see [#33582](https://github.com/ClickHouse/ClickHouse/issues/33582). [#35363](https://github.com/ClickHouse/ClickHouse/pull/35363) ([Vladimir C](https://github.com/vdimir)). +* Added full sorting merge join algorithm. [#35796](https://github.com/ClickHouse/ClickHouse/pull/35796) ([Vladimir C](https://github.com/vdimir)). +* Implement NATS table engine, which allows to pub/sub to NATS. Closes [#32388](https://github.com/ClickHouse/ClickHouse/issues/32388). [#37171](https://github.com/ClickHouse/ClickHouse/pull/37171) ([tchepavel](https://github.com/tchepavel)). ([Kseniia Sumarokova](https://github.com/kssenii)) +* Implement table function `mongodb`. Allow writes into `MongoDB` storage / table function. [#37213](https://github.com/ClickHouse/ClickHouse/pull/37213) ([aaapetrenko](https://github.com/aaapetrenko)). ([Kseniia Sumarokova](https://github.com/kssenii)) +* Add `compatibility` setting and `system.settings_changes` system table that contains information about changes in settings through ClickHouse versions. Closes [#35972](https://github.com/ClickHouse/ClickHouse/issues/35972). [#38957](https://github.com/ClickHouse/ClickHouse/pull/38957) ([Kruglov Pavel](https://github.com/Avogar)). +* Add functions `translate(string, from_string, to_string)` and `translateUTF8(string, from_string, to_string)`. It translates some characters to another. [#38935](https://github.com/ClickHouse/ClickHouse/pull/38935) ([Nikolay Degterinsky](https://github.com/evillique)). +* Support `parseTimeDelta` function. It can be used like ` ;-+,:` can be used as separators, eg. `1yr-2mo`, `2m:6s`: `SELECT parseTimeDelta('1yr-2mo-4w + 12 days, 3 hours : 1 minute ; 33 seconds')`. [#39071](https://github.com/ClickHouse/ClickHouse/pull/39071) ([jiahui-97](https://github.com/jiahui-97)). +* Added options to limit IO operations with remote storage: `max_remote_read_network_bandwidth_for_server` and `max_remote_write_network_bandwidth_for_server`. [#39095](https://github.com/ClickHouse/ClickHouse/pull/39095) ([Sergei Trifonov](https://github.com/serxa)). +* Add `group_by_use_nulls` setting to make aggregation key columns nullable in the case of ROLLUP, CUBE and GROUPING SETS. Closes [#37359](https://github.com/ClickHouse/ClickHouse/issues/37359). [#38642](https://github.com/ClickHouse/ClickHouse/pull/38642) ([Dmitry Novik](https://github.com/novikd)). +* Add the ability to specify compression level during data export. [#38907](https://github.com/ClickHouse/ClickHouse/pull/38907) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add an option to require explicit grants to SELECT from the `system` database. Details: [#38970](https://github.com/ClickHouse/ClickHouse/pull/38970) ([Vitaly Baranov](https://github.com/vitlibar)). +* Functions `multiMatchAny`, `multiMatchAnyIndex`, `multiMatchAllIndices` and their fuzzy variants now accept non-const pattern array argument. [#38485](https://github.com/ClickHouse/ClickHouse/pull/38485) ([Robert Schulze](https://github.com/rschu1ze)). SQL function `multiSearchAllPositions` now accepts non-const needle arguments. [#39167](https://github.com/ClickHouse/ClickHouse/pull/39167) ([Robert Schulze](https://github.com/rschu1ze)). +* Add `send_logs_source_regexp` setting. Send server text logs with specified regexp to match log source name. Empty means all sources. [#39161](https://github.com/ClickHouse/ClickHouse/pull/39161) ([Amos Bird](https://github.com/amosbird)). +* Support `ALTER` for `Hive` tables. [#38214](https://github.com/ClickHouse/ClickHouse/pull/38214) ([lgbo](https://github.com/lgbo-ustc)). +* Support `isNullable` function. This function checks whether it's argument is nullable and return 1 or 0. Closes [#38611](https://github.com/ClickHouse/ClickHouse/issues/38611). [#38841](https://github.com/ClickHouse/ClickHouse/pull/38841) ([lokax](https://github.com/lokax)). + +#### Performance Improvement +* Add AVX-512 VBMI optimized `copyOverlap32Shuffle` for LZ4 decompression. In other words, LZ4 decompression performance is improved. [#37891](https://github.com/ClickHouse/ClickHouse/pull/37891) ([Guo Wangyang](https://github.com/guowangy)). +* `ORDER BY (a, b)` will use all the same benefits as `ORDER BY a, b`. [#38873](https://github.com/ClickHouse/ClickHouse/pull/38873) ([Igor Nikonov](https://github.com/devcrafter)). +* Align branches within a 32B boundary to make benchmark more stable. [#38988](https://github.com/ClickHouse/ClickHouse/pull/38988) ([Guo Wangyang](https://github.com/guowangy)). It improves performance 1..2% on average for Intel. +* Executable UDF, executable dictionaries, and Executable tables will avoid wasting one second during wait for subprocess termination. [#38929](https://github.com/ClickHouse/ClickHouse/pull/38929) ([Constantine Peresypkin](https://github.com/pkit)). +* TODO remove? Pushdown filter to the right side of sorting join. [#39123](https://github.com/ClickHouse/ClickHouse/pull/39123) ([Vladimir C](https://github.com/vdimir)). +* Optimize accesses to `system.stack_trace` table if not all columns are selected. [#39177](https://github.com/ClickHouse/ClickHouse/pull/39177) ([Azat Khuzhin](https://github.com/azat)). +* Improve isNullable/isConstant/isNull/isNotNull performance for LowCardinality argument. [#39192](https://github.com/ClickHouse/ClickHouse/pull/39192) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Improvement +* Bugfixes and performance improvements for `parallel_hash` JOIN method. [#37648](https://github.com/ClickHouse/ClickHouse/pull/37648) ([Vladimir C](https://github.com/vdimir)). +* S3 single objects are now removed with `RemoveObjectRequest`. Implement compatibility with GCP which did not allow to use `removeFileIfExists` effectively breaking approximately half of `remove` functionality. Automatic detection for `DeleteObjects` S3 API, that is not supported by GCS. This will allow to use GCS without explicit `support_batch_delete=0` in configuration. [#37882](https://github.com/ClickHouse/ClickHouse/pull/37882) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Expose basic ClickHouse Keeper related monitoring data (via ProfileEvents and CurrentMetrics). [#38072](https://github.com/ClickHouse/ClickHouse/pull/38072) ([lingpeng0314](https://github.com/lingpeng0314)). +* Support `auto_close` option for PostgreSQL engine connection. Closes [#31486](https://github.com/ClickHouse/ClickHouse/issues/31486). [#38363](https://github.com/ClickHouse/ClickHouse/pull/38363) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Allow `NULL` modifier in columns declaration for table functions. [#38816](https://github.com/ClickHouse/ClickHouse/pull/38816) ([Kruglov Pavel](https://github.com/Avogar)). +* Deactivate `mutations_finalizing_task` before shutdown to avoid benign `TABLE_IS_READ_ONLY` errors during shutdown. [#38851](https://github.com/ClickHouse/ClickHouse/pull/38851) ([Raúl Marín](https://github.com/Algunenano)). +* Eliminate unnecessary waiting of SELECT queries after ALTER queries in presence of INSERT queries if you use deprecated Ordinary databases. [#38864](https://github.com/ClickHouse/ClickHouse/pull/38864) ([Azat Khuzhin](https://github.com/azat)). +* New option `rewrite` in `EXPLAIN AST`. If enabled, it shows AST after it's rewritten, otherwise AST of original query. Disabled by default. [#38910](https://github.com/ClickHouse/ClickHouse/pull/38910) ([Igor Nikonov](https://github.com/devcrafter)). +* Stop reporting Zookeeper "Node exists" exceptions in system.errors when they are expected. [#38961](https://github.com/ClickHouse/ClickHouse/pull/38961) ([Raúl Marín](https://github.com/Algunenano)). +* Allow to specify globs `* or {expr1, expr2, expr3}` inside a key for `clickhouse-extract-from-config` tool. [#38966](https://github.com/ClickHouse/ClickHouse/pull/38966) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* clearOldLogs: Don't report KEEPER_EXCEPTION on concurrent deletes. [#39016](https://github.com/ClickHouse/ClickHouse/pull/39016) ([Raúl Marín](https://github.com/Algunenano)). +* clickhouse-keeper improvement: persist meta-information about keeper servers to disk. [#39069](https://github.com/ClickHouse/ClickHouse/pull/39069) ([Antonio Andelic](https://github.com/antonio2368)). This will make it easier to operate if you shutdown or restart all keeper nodes at the same time. +* Continue without exception when running out of disk space when using filesystem cache. [#39106](https://github.com/ClickHouse/ClickHouse/pull/39106) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Handling SIGTERM signals from k8s. [#39130](https://github.com/ClickHouse/ClickHouse/pull/39130) ([Timur Solodovnikov](https://github.com/tsolodov)). +* Add `merge_algorithm` column (Undecided, Horizontal, Vertical) to system.part_log. [#39181](https://github.com/ClickHouse/ClickHouse/pull/39181) ([Azat Khuzhin](https://github.com/azat)). +* Don't increment a counter in `system.errors` when the disk is not rotational. [#39216](https://github.com/ClickHouse/ClickHouse/pull/39216) ([Raúl Marín](https://github.com/Algunenano)). +* The metric `result_bytes` for `INSERT` queries in `system.query_log` shows number of bytes inserted. Previously value was incorrect and stored the same value as `result_rows`. [#39225](https://github.com/ClickHouse/ClickHouse/pull/39225) ([Ilya Yatsishin](https://github.com/qoega)). +* The CPU usage metric in clickhouse-client will be displayed in a better way. Fixes [#38756](https://github.com/ClickHouse/ClickHouse/issues/38756). [#39280](https://github.com/ClickHouse/ClickHouse/pull/39280) ([Sergei Trifonov](https://github.com/serxa)). +* Rethrow exception on filesystem cache initialization on server startup, better error message. [#39386](https://github.com/ClickHouse/ClickHouse/pull/39386) ([Kseniia Sumarokova](https://github.com/kssenii)). +* OpenTelemetry now collects traces without Processors spans by default (there are too many). To enable Processors spans collection `opentelemetry_trace_processors` setting. [#39170](https://github.com/ClickHouse/ClickHouse/pull/39170) ([Ilya Yatsishin](https://github.com/qoega)). +* Functions `multiMatch[Fuzzy](AllIndices/Any/AnyIndex)` - don't throw a logical error if the needle argument is empty. [#39012](https://github.com/ClickHouse/ClickHouse/pull/39012) ([Robert Schulze](https://github.com/rschu1ze)). +* Allow to declare `RabbitMQ` queue without default arguments `x-max-length` and `x-overflow`. [#39259](https://github.com/ClickHouse/ClickHouse/pull/39259) ([rnbondarenko](https://github.com/rnbondarenko)). + +#### Build/Testing/Packaging Improvement +* Adapt universal installation script for FreeBSD. [#39302](https://github.com/ClickHouse/ClickHouse/pull/39302) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Preparation for building on `s390x` platform. [#39193](https://github.com/ClickHouse/ClickHouse/pull/39193) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Fix a bug in `jemalloc` library [#38757](https://github.com/ClickHouse/ClickHouse/pull/38757) ([Azat Khuzhin](https://github.com/azat)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Fixed crash caused by data race in storage `Hive` (integration table engine). [#38887](https://github.com/ClickHouse/ClickHouse/pull/38887) ([lgbo](https://github.com/lgbo-ustc)). +* Correct glob expansion in case of `{0..10}` forms. Fixes [#38498](https://github.com/ClickHouse/ClickHouse/issues/38498) Current Implementation is similar to what shell does mentiond by @rschu1ze [here](https://github.com/ClickHouse/ClickHouse/pull/38502#issuecomment-1169057723). [#38502](https://github.com/ClickHouse/ClickHouse/pull/38502) ([Heena Bansal](https://github.com/HeenaBansal2009)). +* Fix crash for `mapUpdate`, `mapFilter` functions when using with constant map argument. Closes [#38547](https://github.com/ClickHouse/ClickHouse/issues/38547). [#38553](https://github.com/ClickHouse/ClickHouse/pull/38553) ([hexiaoting](https://github.com/hexiaoting)). +* Fix `toHour` monotonicity information for query optimization which can lead to incorrect query result (incorrect index analysis). This fixes [#38333](https://github.com/ClickHouse/ClickHouse/issues/38333). [#38675](https://github.com/ClickHouse/ClickHouse/pull/38675) ([Amos Bird](https://github.com/amosbird)). +* Fix checking whether s3 storage support parallel writes. It resulted in s3 parallel writes not working. [#38792](https://github.com/ClickHouse/ClickHouse/pull/38792) ([chen](https://github.com/xiedeyantu)). +* Fix s3 seekable reads with parallel read buffer. (Affected memory usage during query). Closes [#38258](https://github.com/ClickHouse/ClickHouse/issues/38258). [#38802](https://github.com/ClickHouse/ClickHouse/pull/38802) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update `simdjson`. This fixes [#38621](https://github.com/ClickHouse/ClickHouse/issues/38621) - a buffer overflow on machines with the latest Intel CPUs with AVX-512 VBMI. [#38838](https://github.com/ClickHouse/ClickHouse/pull/38838) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix possible logical error for Vertical merges. [#38859](https://github.com/ClickHouse/ClickHouse/pull/38859) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix settings profile with seconds unit. [#38896](https://github.com/ClickHouse/ClickHouse/pull/38896) ([Raúl Marín](https://github.com/Algunenano)). +* Fix incorrect partition pruning when there is a nullable partition key. Note: most likely you don't use nullable partition keys - this is an obscure feature you should not use. Nullable keys are a nonsense and this feature is only needed for some crazy use-cases. This fixes [#38941](https://github.com/ClickHouse/ClickHouse/issues/38941). [#38946](https://github.com/ClickHouse/ClickHouse/pull/38946) ([Amos Bird](https://github.com/amosbird)). +* Improve `fsync_part_directory` for fetches. [#38993](https://github.com/ClickHouse/ClickHouse/pull/38993) ([Azat Khuzhin](https://github.com/azat)). +* Fix possible dealock inside `OvercommitTracker`. Fixes [#37794](https://github.com/ClickHouse/ClickHouse/issues/37794). [#39030](https://github.com/ClickHouse/ClickHouse/pull/39030) ([Dmitry Novik](https://github.com/novikd)). +* Fix bug in filesystem cache that could happen in some corner case which coincided with cache capacity hitting the limit. Closes [#39066](https://github.com/ClickHouse/ClickHouse/issues/39066). [#39070](https://github.com/ClickHouse/ClickHouse/pull/39070) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix some corner cases of interpretation of the arguments of window expressions. Fixes [#38538](https://github.com/ClickHouse/ClickHouse/issues/38538) Allow using of higher-order functions in window expressions. [#39112](https://github.com/ClickHouse/ClickHouse/pull/39112) ([Dmitry Novik](https://github.com/novikd)). +* Keep `LowCardinality` type in `tuple` function. Previously `LowCardinality` type was dropped and elements of created tuple had underlying type of `LowCardinality`. [#39113](https://github.com/ClickHouse/ClickHouse/pull/39113) ([Anton Popov](https://github.com/CurtizJ)). +* Fix error `Block structure mismatch` which could happen for INSERT into table with attached MATERIALIZED VIEW and enabled setting `extremes = 1`. Closes [#29759](https://github.com/ClickHouse/ClickHouse/issues/29759) and [#38729](https://github.com/ClickHouse/ClickHouse/issues/38729). [#39125](https://github.com/ClickHouse/ClickHouse/pull/39125) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix unexpected query result when both `optimize_trivial_count_query` and `empty_result_for_aggregation_by_empty_set` are set to true. This fixes [#39140](https://github.com/ClickHouse/ClickHouse/issues/39140). [#39155](https://github.com/ClickHouse/ClickHouse/pull/39155) ([Amos Bird](https://github.com/amosbird)). +* Fixed error `Not found column Type in block` in selects with `PREWHERE` and read-in-order optimizations. [#39157](https://github.com/ClickHouse/ClickHouse/pull/39157) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix extremely rare race condition in during hardlinks for remote filesystem. The only way to reproduce it is concurrent run of backups. [#39190](https://github.com/ClickHouse/ClickHouse/pull/39190) ([alesapin](https://github.com/alesapin)). +* (zero-copy replication is an experimental feature that should not be used in production) Fix fetch of in-memory part with `allow_remote_fs_zero_copy_replication`. [#39214](https://github.com/ClickHouse/ClickHouse/pull/39214) ([Azat Khuzhin](https://github.com/azat)). +* (MaterializedPostgreSQL - experimental feature). Fix segmentation fault in MaterializedPostgreSQL database engine, which could happen if some exception occurred at replication initialisation. Closes [#36939](https://github.com/ClickHouse/ClickHouse/issues/36939). [#39272](https://github.com/ClickHouse/ClickHouse/pull/39272) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix incorrect fetch of table metadata from PostgreSQL database engine. Closes [#33502](https://github.com/ClickHouse/ClickHouse/issues/33502). [#39283](https://github.com/ClickHouse/ClickHouse/pull/39283) ([Kseniia Sumarokova](https://github.com/kssenii)). + + ### ClickHouse release 22.6, 2022-06-16 #### Backward Incompatible Change From b7665524817281f261461a24893a67e3a2806216 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 20 Jul 2022 23:09:05 -0400 Subject: [PATCH 468/659] add native build for cross-compilation --- CMakeLists.txt | 43 +++++++++++++++++++ programs/self-extracting/CMakeLists.txt | 14 +++++- .../self-extracting-executable/CMakeLists.txt | 2 + 3 files changed, 58 insertions(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index d01d2e2179d..e5b83346137 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -13,6 +13,16 @@ else() set(RECONFIGURE_MESSAGE_LEVEL WARNING) endif() +# Setup NATIVE_BUILD_TARGETS list for further native build +# when cross-compiling and host architecture differs from target architecture. +# Add target to the list: +# add_native_target (...) +set_property (GLOBAL PROPERTY NATIVE_BUILD_TARGETS) +function (add_native_target) + get_property (tmp GLOBAL PROPERTY NATIVE_BUILD_TARGETS) + set_property (GLOBAL PROPERTY NATIVE_BUILD_TARGETS "${tmp}" "${ARGV}") +endfunction (add_native_target) + include (cmake/arch.cmake) include (cmake/target.cmake) include (cmake/tools.cmake) @@ -557,3 +567,36 @@ add_subdirectory (tests) add_subdirectory (utils) include (cmake/sanitize_target_link_libraries.cmake) + +# Build native targets if necessary +get_property(NATIVE_BUILD_TARGETS GLOBAL PROPERTY NATIVE_BUILD_TARGETS) +if (NATIVE_BUILD_TARGETS + AND NOT( + CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME + AND CMAKE_HOST_SYSTEM_PROCESSOR STREQUAL CMAKE_SYSTEM_PROCESSOR + ) +) + message (STATUS "Building native targets...") + + set (NATIVE_BUILD_DIR "${CMAKE_BINARY_DIR}/native") + + execute_process( + COMMAND ${CMAKE_COMMAND} -E make_directory "${NATIVE_BUILD_DIR}" + COMMAND_ECHO STDOUT) + + execute_process( + COMMAND ${CMAKE_COMMAND} + "-G${CMAKE_GENERATOR}" + "-DCMAKE_BUILD_TYPE=${CMAKE_BUILD_TYPE}" + "-DCMAKE_MAKE_PROGRAM=${CMAKE_MAKE_PROGRAM}" + "-DCMAKE_C_COMPILER=${CMAKE_C_COMPILER}" + "-DCMAKE_CXX_COMPILER=${CMAKE_CXX_COMPILER}" + ${CMAKE_SOURCE_DIR} + WORKING_DIRECTORY "${NATIVE_BUILD_DIR}" + COMMAND_ECHO STDOUT) + + execute_process( + COMMAND ${CMAKE_COMMAND} --build "${NATIVE_BUILD_DIR}" --target ${NATIVE_BUILD_TARGETS} + COMMAND_ECHO STDOUT) +endif () + diff --git a/programs/self-extracting/CMakeLists.txt b/programs/self-extracting/CMakeLists.txt index 89f6b56b826..2cc26926b38 100644 --- a/programs/self-extracting/CMakeLists.txt +++ b/programs/self-extracting/CMakeLists.txt @@ -1,6 +1,18 @@ + +if (NOT( + CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME + AND CMAKE_HOST_SYSTEM_PROCESSOR STREQUAL CMAKE_SYSTEM_PROCESSOR + ) +) + set (COMPRESSOR "${CMAKE_BINARY_DIR}/native/utils/self-extracting-executable/pre_compressor") + set (DECOMPRESSOR "--decompressor=${CMAKE_BINARY_DIR}/utils/self-extracting-executable/decompressor") +else () + set (COMPRESSOR "${CMAKE_BINARY_DIR}/utils/self-extracting-executable/compressor") +endif () + add_custom_target (self-extracting ALL ${CMAKE_COMMAND} -E remove clickhouse - COMMAND ${CMAKE_BINARY_DIR}/utils/self-extracting-executable/compressor clickhouse ../clickhouse + COMMAND ${COMPRESSOR} ${DECOMPRESSOR} clickhouse ../clickhouse DEPENDS clickhouse compressor ) diff --git a/utils/self-extracting-executable/CMakeLists.txt b/utils/self-extracting-executable/CMakeLists.txt index 8b554d791c9..2b115eaaf65 100644 --- a/utils/self-extracting-executable/CMakeLists.txt +++ b/utils/self-extracting-executable/CMakeLists.txt @@ -1,6 +1,8 @@ add_executable (pre_compressor compressor.cpp) target_link_libraries(pre_compressor PUBLIC ch_contrib::zstd) +add_native_target (pre_compressor) + add_executable (decompressor decompressor.cpp) target_link_libraries(decompressor PUBLIC ch_contrib::zstd) From ac6b9ef05180a9e5e4ea8811e4e44e1561af8326 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 21 Jul 2022 05:18:35 +0200 Subject: [PATCH 469/659] Prepare changelog --- CHANGELOG.md | 83 ++++++++++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 81 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6c1af61e115..b3ad0d3a4e9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -11,26 +11,50 @@ ### ClickHouse release 22.7, 2022-07-21 #### Upgrade Notes +* Enable setting `enable_positional_arguments` by default. It allows queries like `SELECT ... ORDER BY 1, 2` where 1, 2 are the references to the select clause. If you need to return the old behavior, disable this setting. [#38204](https://github.com/ClickHouse/ClickHouse/pull/38204) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* `Ordinary` database engine and old storage definition syntax for `*MergeTree` tables are deprecated. By default it's not possible to create new databases with `Ordinary` engine. If `system` database has `Ordinary` engine it will be automatically converted to `Atomic` on server startup. There are settings to keep old behavior (`allow_deprecated_database_ordinary` and `allow_deprecated_syntax_for_merge_tree`), but these settings may be removed in future releases. [#38335](https://github.com/ClickHouse/ClickHouse/pull/38335) ([Alexander Tokmakov](https://github.com/tavplubix)). * Force rewriting comma join to inner by default (set default value `cross_to_inner_join_rewrite = 2`). To have old behavior set `cross_to_inner_join_rewrite = 1`. [#39326](https://github.com/ClickHouse/ClickHouse/pull/39326) ([Vladimir C](https://github.com/vdimir)). If you will face any incompatibilities, you can turn this setting back. #### New Feature +* Support expressions with window functions. Closes [#19857](https://github.com/ClickHouse/ClickHouse/issues/19857). [#37848](https://github.com/ClickHouse/ClickHouse/pull/37848) ([Dmitry Novik](https://github.com/novikd)). * Add new `direct` join algorithm for `EmbeddedRocksDB` tables, see [#33582](https://github.com/ClickHouse/ClickHouse/issues/33582). [#35363](https://github.com/ClickHouse/ClickHouse/pull/35363) ([Vladimir C](https://github.com/vdimir)). * Added full sorting merge join algorithm. [#35796](https://github.com/ClickHouse/ClickHouse/pull/35796) ([Vladimir C](https://github.com/vdimir)). * Implement NATS table engine, which allows to pub/sub to NATS. Closes [#32388](https://github.com/ClickHouse/ClickHouse/issues/32388). [#37171](https://github.com/ClickHouse/ClickHouse/pull/37171) ([tchepavel](https://github.com/tchepavel)). ([Kseniia Sumarokova](https://github.com/kssenii)) * Implement table function `mongodb`. Allow writes into `MongoDB` storage / table function. [#37213](https://github.com/ClickHouse/ClickHouse/pull/37213) ([aaapetrenko](https://github.com/aaapetrenko)). ([Kseniia Sumarokova](https://github.com/kssenii)) +* Add SQLInsert output format. Closes [#38441](https://github.com/ClickHouse/ClickHouse/issues/38441). [#38477](https://github.com/ClickHouse/ClickHouse/pull/38477) ([Kruglov Pavel](https://github.com/Avogar)). * Add `compatibility` setting and `system.settings_changes` system table that contains information about changes in settings through ClickHouse versions. Closes [#35972](https://github.com/ClickHouse/ClickHouse/issues/35972). [#38957](https://github.com/ClickHouse/ClickHouse/pull/38957) ([Kruglov Pavel](https://github.com/Avogar)). * Add functions `translate(string, from_string, to_string)` and `translateUTF8(string, from_string, to_string)`. It translates some characters to another. [#38935](https://github.com/ClickHouse/ClickHouse/pull/38935) ([Nikolay Degterinsky](https://github.com/evillique)). * Support `parseTimeDelta` function. It can be used like ` ;-+,:` can be used as separators, eg. `1yr-2mo`, `2m:6s`: `SELECT parseTimeDelta('1yr-2mo-4w + 12 days, 3 hours : 1 minute ; 33 seconds')`. [#39071](https://github.com/ClickHouse/ClickHouse/pull/39071) ([jiahui-97](https://github.com/jiahui-97)). +* Added `CREATE TABLE ... EMPTY AS SELECT` query. It automatically deduces table structure from the SELECT query, but does not fill the table after creation. Resolves [#38049](https://github.com/ClickHouse/ClickHouse/issues/38049). [#38272](https://github.com/ClickHouse/ClickHouse/pull/38272) ([Alexander Tokmakov](https://github.com/tavplubix)). * Added options to limit IO operations with remote storage: `max_remote_read_network_bandwidth_for_server` and `max_remote_write_network_bandwidth_for_server`. [#39095](https://github.com/ClickHouse/ClickHouse/pull/39095) ([Sergei Trifonov](https://github.com/serxa)). * Add `group_by_use_nulls` setting to make aggregation key columns nullable in the case of ROLLUP, CUBE and GROUPING SETS. Closes [#37359](https://github.com/ClickHouse/ClickHouse/issues/37359). [#38642](https://github.com/ClickHouse/ClickHouse/pull/38642) ([Dmitry Novik](https://github.com/novikd)). * Add the ability to specify compression level during data export. [#38907](https://github.com/ClickHouse/ClickHouse/pull/38907) ([Nikolay Degterinsky](https://github.com/evillique)). * Add an option to require explicit grants to SELECT from the `system` database. Details: [#38970](https://github.com/ClickHouse/ClickHouse/pull/38970) ([Vitaly Baranov](https://github.com/vitlibar)). * Functions `multiMatchAny`, `multiMatchAnyIndex`, `multiMatchAllIndices` and their fuzzy variants now accept non-const pattern array argument. [#38485](https://github.com/ClickHouse/ClickHouse/pull/38485) ([Robert Schulze](https://github.com/rschu1ze)). SQL function `multiSearchAllPositions` now accepts non-const needle arguments. [#39167](https://github.com/ClickHouse/ClickHouse/pull/39167) ([Robert Schulze](https://github.com/rschu1ze)). +* Add a setting `zstd_window_log_max` to configure max memory usage on zstd decoding when importing external files. Closes [#35693](https://github.com/ClickHouse/ClickHouse/issues/35693). [#37015](https://github.com/ClickHouse/ClickHouse/pull/37015) ([wuxiaobai24](https://github.com/wuxiaobai24)). * Add `send_logs_source_regexp` setting. Send server text logs with specified regexp to match log source name. Empty means all sources. [#39161](https://github.com/ClickHouse/ClickHouse/pull/39161) ([Amos Bird](https://github.com/amosbird)). * Support `ALTER` for `Hive` tables. [#38214](https://github.com/ClickHouse/ClickHouse/pull/38214) ([lgbo](https://github.com/lgbo-ustc)). * Support `isNullable` function. This function checks whether it's argument is nullable and return 1 or 0. Closes [#38611](https://github.com/ClickHouse/ClickHouse/issues/38611). [#38841](https://github.com/ClickHouse/ClickHouse/pull/38841) ([lokax](https://github.com/lokax)). +* Added Base58 encoding/decoding. [#38159](https://github.com/ClickHouse/ClickHouse/pull/38159) ([Andrey Zvonov](https://github.com/zvonand)). +* Add chart visualization to Play UI. [#38197](https://github.com/ClickHouse/ClickHouse/pull/38197) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Added L2 Squared distance and norm functions for both arrays and tuples. [#38545](https://github.com/ClickHouse/ClickHouse/pull/38545) ([Julian Gilyadov](https://github.com/israelg99)). +* Add ability to pass HTTP headers to the `url` table function / storage via SQL. Closes [#37897](https://github.com/ClickHouse/ClickHouse/issues/37897). [#38176](https://github.com/ClickHouse/ClickHouse/pull/38176) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add `clickhouse-diagnostics` binary to the packages. [#38647](https://github.com/ClickHouse/ClickHouse/pull/38647) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Experimental Feature +* Adds new setting `implicit_transaction` to run standalone queries inside a transaction. It handles both creation and closing (via COMMIT if the query succeeded or ROLLBACK if it didn't) of the transaction automatically. [#38344](https://github.com/ClickHouse/ClickHouse/pull/38344) ([Raúl Marín](https://github.com/Algunenano)). #### Performance Improvement +* Distinct optimization for sorted columns. Use specialized distinct transformation in case input stream is sorted by column(s) in distinct. Optimization can be applied to pre-distinct, final distinct, or both. Initial implementation by @dimarub2000. [#37803](https://github.com/ClickHouse/ClickHouse/pull/37803) ([Igor Nikonov](https://github.com/devcrafter)). +* Improve performance of `ORDER BY`, `MergeTree` merges, window functions using batch version of `BinaryHeap`. [#38022](https://github.com/ClickHouse/ClickHouse/pull/38022) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix significant join performance regression which was introduced in https://github.com/ClickHouse/ClickHouse/pull/35616 . It's interesting that common join queries such as ssb queries have been 10 times slower for almost 3 months while no one complains. [#38052](https://github.com/ClickHouse/ClickHouse/pull/38052) ([Amos Bird](https://github.com/amosbird)). +* Migrate from the Intel hyperscan library to vectorscan, this speeds up many string matching on non-x86 platforms. [#38171](https://github.com/ClickHouse/ClickHouse/pull/38171) ([Robert Schulze](https://github.com/rschu1ze)). +* Increased parallelism of query plan steps executed after aggregation. [#38295](https://github.com/ClickHouse/ClickHouse/pull/38295) ([Nikita Taranov](https://github.com/nickitat)). +* Improve performance of insertion to columns of type `JSON`. [#38320](https://github.com/ClickHouse/ClickHouse/pull/38320) ([Anton Popov](https://github.com/CurtizJ)). +* Optimized insertion and lookups in the HashTable. [#38413](https://github.com/ClickHouse/ClickHouse/pull/38413) ([Nikita Taranov](https://github.com/nickitat)). +* Fix performance degradation from [#32493](https://github.com/ClickHouse/ClickHouse/issues/32493). [#38417](https://github.com/ClickHouse/ClickHouse/pull/38417) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve performance of joining with numeric columns using SIMD instructions. [#37235](https://github.com/ClickHouse/ClickHouse/pull/37235) ([zzachimed](https://github.com/zzachimed)). [#38565](https://github.com/ClickHouse/ClickHouse/pull/38565) ([Maksim Kita](https://github.com/kitaisreal)). +* Norm and Distance functions for arrays speed up 1.2-2 times. [#38740](https://github.com/ClickHouse/ClickHouse/pull/38740) ([Alexander Gololobov](https://github.com/davenger)). * Add AVX-512 VBMI optimized `copyOverlap32Shuffle` for LZ4 decompression. In other words, LZ4 decompression performance is improved. [#37891](https://github.com/ClickHouse/ClickHouse/pull/37891) ([Guo Wangyang](https://github.com/guowangy)). * `ORDER BY (a, b)` will use all the same benefits as `ORDER BY a, b`. [#38873](https://github.com/ClickHouse/ClickHouse/pull/38873) ([Igor Nikonov](https://github.com/devcrafter)). * Align branches within a 32B boundary to make benchmark more stable. [#38988](https://github.com/ClickHouse/ClickHouse/pull/38988) ([Guo Wangyang](https://github.com/guowangy)). It improves performance 1..2% on average for Intel. @@ -38,9 +62,31 @@ * TODO remove? Pushdown filter to the right side of sorting join. [#39123](https://github.com/ClickHouse/ClickHouse/pull/39123) ([Vladimir C](https://github.com/vdimir)). * Optimize accesses to `system.stack_trace` table if not all columns are selected. [#39177](https://github.com/ClickHouse/ClickHouse/pull/39177) ([Azat Khuzhin](https://github.com/azat)). * Improve isNullable/isConstant/isNull/isNotNull performance for LowCardinality argument. [#39192](https://github.com/ClickHouse/ClickHouse/pull/39192) ([Kruglov Pavel](https://github.com/Avogar)). +* Optimized processing of ORDER BY in window functions. [#34632](https://github.com/ClickHouse/ClickHouse/pull/34632) ([Vladimir Chebotarev](https://github.com/excitoon)). +* The table `system.asynchronous_metric_log` is further optimized for storage space. This closes [#38134](https://github.com/ClickHouse/ClickHouse/issues/38134). See the [YouTube video](https://www.youtube.com/watch?v=0fSp9SF8N8A). [#38428](https://github.com/ClickHouse/ClickHouse/pull/38428) ([Alexey Milovidov](https://github.com/alexey-milovidov)). #### Improvement +* Support SQL standard CREATE INDEX and DROP INDEX syntax. [#35166](https://github.com/ClickHouse/ClickHouse/pull/35166) ([Jianmei Zhang](https://github.com/zhangjmruc)). +* Send profile events for INSERT queries (previously only SELECT was supported). [#37391](https://github.com/ClickHouse/ClickHouse/pull/37391) ([Azat Khuzhin](https://github.com/azat)). +* Implement in order aggregation (`optimize_aggregation_in_order`) for fully materialized projections. [#37469](https://github.com/ClickHouse/ClickHouse/pull/37469) ([Azat Khuzhin](https://github.com/azat)). +* Remove subprocess run for kerberos initialization. Added new integration test. Closes [#27651](https://github.com/ClickHouse/ClickHouse/issues/27651). [#38105](https://github.com/ClickHouse/ClickHouse/pull/38105) ([Roman Vasin](https://github.com/rvasin)). +* * Add setting `multiple_joins_try_to_keep_original_names` to not rewrite identifier name on multiple JOINs rewrite, close [#34697](https://github.com/ClickHouse/ClickHouse/issues/34697). [#38149](https://github.com/ClickHouse/ClickHouse/pull/38149) ([Vladimir C](https://github.com/vdimir)). +* Improved trace-visualizer UX. [#38169](https://github.com/ClickHouse/ClickHouse/pull/38169) ([Sergei Trifonov](https://github.com/serxa)). +* Enable stack trace collection and query profiler for AArch64. [#38181](https://github.com/ClickHouse/ClickHouse/pull/38181) ([Maksim Kita](https://github.com/kitaisreal)). +* Do not skip symlinks in `user_defined` directory during SQL user defined functions loading. Closes [#38042](https://github.com/ClickHouse/ClickHouse/issues/38042). [#38184](https://github.com/ClickHouse/ClickHouse/pull/38184) ([Maksim Kita](https://github.com/kitaisreal)). +* Added background cleanup of subdirectories in `store/`. In some cases clickhouse-server might left garbage subdirectories in `store/` (for example, on unsuccessful table creation) and those dirs were never been removed. Fixes [#33710](https://github.com/ClickHouse/ClickHouse/issues/33710). [#38265](https://github.com/ClickHouse/ClickHouse/pull/38265) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add `DESCRIBE CACHE` query to show cache settings from config. Add `SHOW CACHES` query to show available filesystem caches list. [#38279](https://github.com/ClickHouse/ClickHouse/pull/38279) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add access check for `system drop filesystem cache`. Support ON CLUSTER. [#38319](https://github.com/ClickHouse/ClickHouse/pull/38319) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix PostgreSQL database engine incompatibility on upgrade from 21.3 to 22.3. Closes [#36659](https://github.com/ClickHouse/ClickHouse/issues/36659). [#38369](https://github.com/ClickHouse/ClickHouse/pull/38369) ([Kseniia Sumarokova](https://github.com/kssenii)). +* `filesystemAvailable` and similar functions now work in `clickhouse-local`. This closes [#38423](https://github.com/ClickHouse/ClickHouse/issues/38423). [#38424](https://github.com/ClickHouse/ClickHouse/pull/38424) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add `revision` function. [#38555](https://github.com/ClickHouse/ClickHouse/pull/38555) ([Azat Khuzhin](https://github.com/azat)). +* Fix GCS via proxy tunnel usage. [#38726](https://github.com/ClickHouse/ClickHouse/pull/38726) ([Azat Khuzhin](https://github.com/azat)). +* Support `\i file` in clickhouse client / local (similar to psql \i). [#38813](https://github.com/ClickHouse/ClickHouse/pull/38813) ([Kseniia Sumarokova](https://github.com/kssenii)). +* New option `optimize = 1` in `EXPLAIN AST`. If enabled, it shows AST after it's rewritten, otherwise AST of original query. Disabled by default. [#38910](https://github.com/ClickHouse/ClickHouse/pull/38910) ([Igor Nikonov](https://github.com/devcrafter)). +* Allow trailing comma in columns list. closes [#38425](https://github.com/ClickHouse/ClickHouse/issues/38425). [#38440](https://github.com/ClickHouse/ClickHouse/pull/38440) ([chen](https://github.com/xiedeyantu)). * Bugfixes and performance improvements for `parallel_hash` JOIN method. [#37648](https://github.com/ClickHouse/ClickHouse/pull/37648) ([Vladimir C](https://github.com/vdimir)). +* Support hadoop secure RPC transfer (hadoop.rpc.protection=privacy and hadoop.rpc.protection=integrity). [#37852](https://github.com/ClickHouse/ClickHouse/pull/37852) ([Peng Liu](https://github.com/michael1589)). +* Add struct type support in `StorageHive`. [#38118](https://github.com/ClickHouse/ClickHouse/pull/38118) ([lgbo](https://github.com/lgbo-ustc)). * S3 single objects are now removed with `RemoveObjectRequest`. Implement compatibility with GCP which did not allow to use `removeFileIfExists` effectively breaking approximately half of `remove` functionality. Automatic detection for `DeleteObjects` S3 API, that is not supported by GCS. This will allow to use GCS without explicit `support_batch_delete=0` in configuration. [#37882](https://github.com/ClickHouse/ClickHouse/pull/37882) ([Vladimir Chebotarev](https://github.com/excitoon)). * Expose basic ClickHouse Keeper related monitoring data (via ProfileEvents and CurrentMetrics). [#38072](https://github.com/ClickHouse/ClickHouse/pull/38072) ([lingpeng0314](https://github.com/lingpeng0314)). * Support `auto_close` option for PostgreSQL engine connection. Closes [#31486](https://github.com/ClickHouse/ClickHouse/issues/31486). [#38363](https://github.com/ClickHouse/ClickHouse/pull/38363) ([Kseniia Sumarokova](https://github.com/kssenii)). @@ -49,6 +95,7 @@ * Eliminate unnecessary waiting of SELECT queries after ALTER queries in presence of INSERT queries if you use deprecated Ordinary databases. [#38864](https://github.com/ClickHouse/ClickHouse/pull/38864) ([Azat Khuzhin](https://github.com/azat)). * New option `rewrite` in `EXPLAIN AST`. If enabled, it shows AST after it's rewritten, otherwise AST of original query. Disabled by default. [#38910](https://github.com/ClickHouse/ClickHouse/pull/38910) ([Igor Nikonov](https://github.com/devcrafter)). * Stop reporting Zookeeper "Node exists" exceptions in system.errors when they are expected. [#38961](https://github.com/ClickHouse/ClickHouse/pull/38961) ([Raúl Marín](https://github.com/Algunenano)). +* `clickhouse-keeper`: add support for real-time digest calculation and verification. It is disabled by default. [#37555](https://github.com/ClickHouse/ClickHouse/pull/37555) ([Antonio Andelic](https://github.com/antonio2368)). * Allow to specify globs `* or {expr1, expr2, expr3}` inside a key for `clickhouse-extract-from-config` tool. [#38966](https://github.com/ClickHouse/ClickHouse/pull/38966) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * clearOldLogs: Don't report KEEPER_EXCEPTION on concurrent deletes. [#39016](https://github.com/ClickHouse/ClickHouse/pull/39016) ([Raúl Marín](https://github.com/Algunenano)). * clickhouse-keeper improvement: persist meta-information about keeper servers to disk. [#39069](https://github.com/ClickHouse/ClickHouse/pull/39069) ([Antonio Andelic](https://github.com/antonio2368)). This will make it easier to operate if you shutdown or restart all keeper nodes at the same time. @@ -64,13 +111,18 @@ * Allow to declare `RabbitMQ` queue without default arguments `x-max-length` and `x-overflow`. [#39259](https://github.com/ClickHouse/ClickHouse/pull/39259) ([rnbondarenko](https://github.com/rnbondarenko)). #### Build/Testing/Packaging Improvement +* Apply Clang Thread Safety Analysis (TSA) annotations to ClickHouse. [#38068](https://github.com/ClickHouse/ClickHouse/pull/38068) ([Robert Schulze](https://github.com/rschu1ze)). * Adapt universal installation script for FreeBSD. [#39302](https://github.com/ClickHouse/ClickHouse/pull/39302) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Preparation for building on `s390x` platform. [#39193](https://github.com/ClickHouse/ClickHouse/pull/39193) ([Harry Lee](https://github.com/HarryLeeIBM)). * Fix a bug in `jemalloc` library [#38757](https://github.com/ClickHouse/ClickHouse/pull/38757) ([Azat Khuzhin](https://github.com/azat)). +* Hardware benchmark now has support for automatic results uploading. [#38427](https://github.com/ClickHouse/ClickHouse/pull/38427) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* System table "system.licenses" is now correctly populated on Mac (Darwin). [#38294](https://github.com/ClickHouse/ClickHouse/pull/38294) ([Robert Schulze](https://github.com/rschu1ze)). +* Change `all|noarch` packages to architecture-dependent - Fix some documentation for it - Push aarch64|arm64 packages to artifactory and release assets - Fixes [#36443](https://github.com/ClickHouse/ClickHouse/issues/36443). [#38580](https://github.com/ClickHouse/ClickHouse/pull/38580) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). #### Bug Fix (user-visible misbehavior in official stable or prestable release) - +* Fix rounding for `Decimal128/Decimal256` with more than 19-digits long scale. [#38027](https://github.com/ClickHouse/ClickHouse/pull/38027) ([Igor Nikonov](https://github.com/devcrafter)). * Fixed crash caused by data race in storage `Hive` (integration table engine). [#38887](https://github.com/ClickHouse/ClickHouse/pull/38887) ([lgbo](https://github.com/lgbo-ustc)). +* Fix crash when executing GRANT ALL ON *.* with ON CLUSTER. It was broken in https://github.com/ClickHouse/ClickHouse/pull/35767. This closes [#38618](https://github.com/ClickHouse/ClickHouse/issues/38618). [#38674](https://github.com/ClickHouse/ClickHouse/pull/38674) ([Vitaly Baranov](https://github.com/vitlibar)). * Correct glob expansion in case of `{0..10}` forms. Fixes [#38498](https://github.com/ClickHouse/ClickHouse/issues/38498) Current Implementation is similar to what shell does mentiond by @rschu1ze [here](https://github.com/ClickHouse/ClickHouse/pull/38502#issuecomment-1169057723). [#38502](https://github.com/ClickHouse/ClickHouse/pull/38502) ([Heena Bansal](https://github.com/HeenaBansal2009)). * Fix crash for `mapUpdate`, `mapFilter` functions when using with constant map argument. Closes [#38547](https://github.com/ClickHouse/ClickHouse/issues/38547). [#38553](https://github.com/ClickHouse/ClickHouse/pull/38553) ([hexiaoting](https://github.com/hexiaoting)). * Fix `toHour` monotonicity information for query optimization which can lead to incorrect query result (incorrect index analysis). This fixes [#38333](https://github.com/ClickHouse/ClickHouse/issues/38333). [#38675](https://github.com/ClickHouse/ClickHouse/pull/38675) ([Amos Bird](https://github.com/amosbird)). @@ -92,7 +144,34 @@ * (zero-copy replication is an experimental feature that should not be used in production) Fix fetch of in-memory part with `allow_remote_fs_zero_copy_replication`. [#39214](https://github.com/ClickHouse/ClickHouse/pull/39214) ([Azat Khuzhin](https://github.com/azat)). * (MaterializedPostgreSQL - experimental feature). Fix segmentation fault in MaterializedPostgreSQL database engine, which could happen if some exception occurred at replication initialisation. Closes [#36939](https://github.com/ClickHouse/ClickHouse/issues/36939). [#39272](https://github.com/ClickHouse/ClickHouse/pull/39272) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix incorrect fetch of table metadata from PostgreSQL database engine. Closes [#33502](https://github.com/ClickHouse/ClickHouse/issues/33502). [#39283](https://github.com/ClickHouse/ClickHouse/pull/39283) ([Kseniia Sumarokova](https://github.com/kssenii)). - +* Fix projection exception when aggregation keys are wrapped inside other functions. This fixes [#37151](https://github.com/ClickHouse/ClickHouse/issues/37151). [#37155](https://github.com/ClickHouse/ClickHouse/pull/37155) ([Amos Bird](https://github.com/amosbird)). +* Fix possible logical error `... with argument with type Nothing and default implementation for Nothing is expected to return result with type Nothing, got ...` in some functions. Closes: [#37610](https://github.com/ClickHouse/ClickHouse/issues/37610) Closes: [#37741](https://github.com/ClickHouse/ClickHouse/issues/37741). [#37759](https://github.com/ClickHouse/ClickHouse/pull/37759) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix incorrect columns order in subqueries of UNION (in case of duplicated columns in subselects may produce incorrect result). [#37887](https://github.com/ClickHouse/ClickHouse/pull/37887) ([Azat Khuzhin](https://github.com/azat)). +* Fix incorrect work of MODIFY ALTER Column with column names that contain dots. Closes [#37907](https://github.com/ClickHouse/ClickHouse/issues/37907). [#37971](https://github.com/ClickHouse/ClickHouse/pull/37971) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix reading of sparse columns from `MergeTree` tables that store their data in S3. [#37978](https://github.com/ClickHouse/ClickHouse/pull/37978) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible crash in `Distributed` async insert in case of removing a replica from config. [#38029](https://github.com/ClickHouse/ClickHouse/pull/38029) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix "Missing columns" for GLOBAL JOIN with CTE without alias. [#38056](https://github.com/ClickHouse/ClickHouse/pull/38056) ([Azat Khuzhin](https://github.com/azat)). +* Rewrite tuple functions as literals in backwards-compatibility mode. [#38096](https://github.com/ClickHouse/ClickHouse/pull/38096) ([Anton Kozlov](https://github.com/tonickkozlov)). +* Fix redundant memory reservation for output block during `ORDER BY`. [#38127](https://github.com/ClickHouse/ClickHouse/pull/38127) ([iyupeng](https://github.com/iyupeng)). +* Fix possible logical error `Bad cast from type DB::IColumn* to DB::ColumnNullable*` in array mapped functions. Closes [#38006](https://github.com/ClickHouse/ClickHouse/issues/38006). [#38132](https://github.com/ClickHouse/ClickHouse/pull/38132) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix temporary name clash in partial merge join, close [#37928](https://github.com/ClickHouse/ClickHouse/issues/37928). [#38135](https://github.com/ClickHouse/ClickHouse/pull/38135) ([Vladimir C](https://github.com/vdimir)). +* Some minr issue with queries like `CREATE TABLE nested_name_tuples (`a` Tuple(x String, y Tuple(i Int32, j String))) ENGINE = Memory;` [#38136](https://github.com/ClickHouse/ClickHouse/pull/38136) ([lgbo](https://github.com/lgbo-ustc)). +* Fix bug with nested short-circuit functions that led to execution of arguments even if condition is false. Closes [#38040](https://github.com/ClickHouse/ClickHouse/issues/38040). [#38173](https://github.com/ClickHouse/ClickHouse/pull/38173) ([Kruglov Pavel](https://github.com/Avogar)). +* (Window View is a experimental feature) Fix LOGICAL_ERROR for WINDOW VIEW with incorrect structure. [#38205](https://github.com/ClickHouse/ClickHouse/pull/38205) ([Azat Khuzhin](https://github.com/azat)). +* Update librdkafka submodule to fix crash when an OAUTHBEARER refresh callback is set. [#38225](https://github.com/ClickHouse/ClickHouse/pull/38225) ([Rafael Acevedo](https://github.com/racevedoo)). +* Fix INSERT into Distributed hung due to ProfileEvents. [#38307](https://github.com/ClickHouse/ClickHouse/pull/38307) ([Azat Khuzhin](https://github.com/azat)). +* Fix retries in PostgreSQL engine. [#38310](https://github.com/ClickHouse/ClickHouse/pull/38310) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix optimization in PartialSortingTransform (SIGSEGV and possible incorrect result). [#38324](https://github.com/ClickHouse/ClickHouse/pull/38324) ([Azat Khuzhin](https://github.com/azat)). +* Fix RabbitMQ with formats based on PeekableReadBuffer. Closes [#38061](https://github.com/ClickHouse/ClickHouse/issues/38061). [#38356](https://github.com/ClickHouse/ClickHouse/pull/38356) ([Kseniia Sumarokova](https://github.com/kssenii)). +* MaterializedPostgreSQL - experimentail feature. Fix possible `Invalid number of rows in Chunk` in MaterializedPostgreSQL. Closes [#37323](https://github.com/ClickHouse/ClickHouse/issues/37323). [#38360](https://github.com/ClickHouse/ClickHouse/pull/38360) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix RabbitMQ configuration with connection string setting. Closes [#36531](https://github.com/ClickHouse/ClickHouse/issues/36531). [#38365](https://github.com/ClickHouse/ClickHouse/pull/38365) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix PostgreSQL engine not using PostgreSQL schema when retrieving array dimension size. Closes [#36755](https://github.com/ClickHouse/ClickHouse/issues/36755). Closes [#36772](https://github.com/ClickHouse/ClickHouse/issues/36772). [#38366](https://github.com/ClickHouse/ClickHouse/pull/38366) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix possibly incorrect result of distributed queries with `DISTINCT` and `LIMIT`. Fixes [#38282](https://github.com/ClickHouse/ClickHouse/issues/38282). [#38371](https://github.com/ClickHouse/ClickHouse/pull/38371) ([Anton Popov](https://github.com/CurtizJ)). +* Fix wrong results of countSubstrings() & position() on patterns with 0-bytes. [#38589](https://github.com/ClickHouse/ClickHouse/pull/38589) ([Robert Schulze](https://github.com/rschu1ze)). +* Now it's possible to start a clickhouse-server and attach/detach tables even for tables with the incorrect values of IPv4/IPv6 representation. Proper fix for issue [#35156](https://github.com/ClickHouse/ClickHouse/issues/35156). [#38590](https://github.com/ClickHouse/ClickHouse/pull/38590) ([alesapin](https://github.com/alesapin)). +* `rankCorr` function will work correctly if some arguments are NaNs. This closes [#38396](https://github.com/ClickHouse/ClickHouse/issues/38396). [#38722](https://github.com/ClickHouse/ClickHouse/pull/38722) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix `parallel_view_processing=1` with `optimize_trivial_insert_select=1`. Fix `max_insert_threads` while pushing to views. [#38731](https://github.com/ClickHouse/ClickHouse/pull/38731) ([Azat Khuzhin](https://github.com/azat)). +* Fix use-after-free for aggregate functions with `Map` combinator that leads to incorrect result. [#38748](https://github.com/ClickHouse/ClickHouse/pull/38748) ([Azat Khuzhin](https://github.com/azat)). ### ClickHouse release 22.6, 2022-06-16 From 19c4c0bd6ec31d76636426abb692f7c3e7076b2e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 21 Jul 2022 06:41:42 +0300 Subject: [PATCH 470/659] Update CoordinationSettings.cpp --- src/Coordination/CoordinationSettings.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index c3decc071de..4733adcf67a 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -1,5 +1,4 @@ #include -//#include #include #include #include From 41045ba70479122628567ab3642e0db15c558a9e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 21 Jul 2022 06:42:51 +0300 Subject: [PATCH 471/659] Update Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e6992d0765f..09d888a26aa 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -607,7 +607,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) \ M(String, compatibility, "", "Changes other settings according to provided ClickHouse version. If we know that we changed some behaviour in ClickHouse by changing some settings in some version, this compatibility setting will control these settings", 0) \ \ - M(Map, additional_table_filters, "", "Additional filter expression which would be applied after reading from specified table. Syntax : {'table1' : 'expression', 'database.table2' : 'expression'}", 0) \ + M(Map, additional_table_filters, "", "Additional filter expression which would be applied after reading from specified table. Syntax: {'table1': 'expression', 'database.table2': 'expression'}", 0) \ M(String, additional_result_filter, "", "Additional filter expression which would be applied to query result", 0) \ \ /** Experimental functions */ \ From 3f76c8d7fd11a48c8bbb6e4d0b5f074672ab021a Mon Sep 17 00:00:00 2001 From: Mingliang Pan Date: Thu, 21 Jul 2022 11:56:35 +0800 Subject: [PATCH 472/659] optimize select query from cluster table function Use local node as first priority to get Structure Of Remote Table. we have many distributed queries( like select xx from cluster('xx',view (xxxx) ) on a clickhouse cluster. we found that the first node (shard_num=1) have 2 times of query number compared to other shards. The reason is that the getStructureOfRemoteTableInShard func always take the first shard to execute "DESC TABLE xx" query. The better way is to use local node as first priority which save the network rpc and reduce the pressure of first shard . --- src/Storages/getStructureOfRemoteTable.cpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 8fa4d02e8e1..4d30b04cd5e 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -122,6 +122,18 @@ ColumnsDescription getStructureOfRemoteTable( const auto & shards_info = cluster.getShardsInfo(); std::string fail_messages; + + // use local shard as first priority, as it needs no network communication + for (const auto & shard_info : shards_info) + { + if(shard_info.isLocal()){ + const auto & res = getStructureOfRemoteTableInShard(cluster, shard_info, table_id, context, table_func_ptr); + if (res.empty()) + continue; + + return res; + } + } for (const auto & shard_info : shards_info) { From e019b6ba697c50819a579f1225c091fe50c2a167 Mon Sep 17 00:00:00 2001 From: Mingliang Pan Date: Thu, 21 Jul 2022 12:11:33 +0800 Subject: [PATCH 473/659] Update getStructureOfRemoteTable.cpp --- src/Storages/getStructureOfRemoteTable.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 4d30b04cd5e..82a20b8d578 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -126,10 +126,11 @@ ColumnsDescription getStructureOfRemoteTable( // use local shard as first priority, as it needs no network communication for (const auto & shard_info : shards_info) { - if(shard_info.isLocal()){ + if(shard_info.isLocal()) + { const auto & res = getStructureOfRemoteTableInShard(cluster, shard_info, table_id, context, table_func_ptr); if (res.empty()) - continue; + break; return res; } From 29ed2083d13eb660be13be8b6d158fee98e1b7a6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Jul 2022 00:20:23 -0400 Subject: [PATCH 474/659] endian.h in different path on __APPLE__ and __FreeBSD__ --- utils/self-extracting-executable/compressor.cpp | 7 ++++++- utils/self-extracting-executable/decompressor.cpp | 7 ++++++- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/utils/self-extracting-executable/compressor.cpp b/utils/self-extracting-executable/compressor.cpp index f6bbc33aa16..58b1442af18 100644 --- a/utils/self-extracting-executable/compressor.cpp +++ b/utils/self-extracting-executable/compressor.cpp @@ -9,7 +9,12 @@ #include #include #include -#include + +#if (defined(__APPLE__) || defined(__FreeBSD__)) && defined(__GNUC__) +# include +#elif !defined(_MSC_VER) +# include +#endif #include "types.h" diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 97bcda763e3..60e734a99cb 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -12,7 +12,12 @@ #include #include #include -#include + +#if (defined(__APPLE__) || defined(__FreeBSD__)) && defined(__GNUC__) +# include +#elif !defined(_MSC_VER) +# include +#endif #include "types.h" From 8bd423f7dc662a4878eac76c16a01e11f8992672 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 21 Jul 2022 05:27:31 +0000 Subject: [PATCH 475/659] Fix test: move test to stateful tests --- .../02317_distinct_in_order_optimization.reference | 1 - .../02317_distinct_in_order_optimization.sql | 11 ----------- .../1_stateful/00174_distinct_in_order.reference | 1 + tests/queries/1_stateful/00174_distinct_in_order.sql | 10 ++++++++++ 4 files changed, 11 insertions(+), 12 deletions(-) create mode 100644 tests/queries/1_stateful/00174_distinct_in_order.reference create mode 100644 tests/queries/1_stateful/00174_distinct_in_order.sql diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference index 05e65c92805..b53b561137e 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference @@ -77,4 +77,3 @@ 2 2 1 1 0 0 --- check that distinct with and w/o optimization produce the same result diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql index 29fcf47fa20..afe53a95b26 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql @@ -44,14 +44,3 @@ select '-- distinct with non-key prefix and non-sorted column, order by non-sort select distinct b,c from distinct_in_order order by c desc; drop table if exists distinct_in_order sync; - -select '-- check that distinct with and w/o optimization produce the same result'; -drop table if exists distinct_in_order sync; -drop table if exists ordinary_distinct sync; -create table distinct_in_order (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate); -insert into distinct_in_order select distinct CounterID, EventDate from test.hits order by CounterID, EventDate settings optimize_distinct_in_order=1; -create table ordinary_distinct (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate); -insert into ordinary_distinct select distinct CounterID, EventDate from test.hits settings optimize_distinct_in_order=0; -select distinct * from distinct_in_order except select * from ordinary_distinct; -drop table if exists distinct_in_order sync; -drop table if exists ordinary_distinct sync; diff --git a/tests/queries/1_stateful/00174_distinct_in_order.reference b/tests/queries/1_stateful/00174_distinct_in_order.reference new file mode 100644 index 00000000000..f3389600167 --- /dev/null +++ b/tests/queries/1_stateful/00174_distinct_in_order.reference @@ -0,0 +1 @@ +-- check that distinct with and w/o optimization produce the same result diff --git a/tests/queries/1_stateful/00174_distinct_in_order.sql b/tests/queries/1_stateful/00174_distinct_in_order.sql new file mode 100644 index 00000000000..6d79990a0e0 --- /dev/null +++ b/tests/queries/1_stateful/00174_distinct_in_order.sql @@ -0,0 +1,10 @@ +select '-- check that distinct with and w/o optimization produce the same result'; +drop table if exists distinct_in_order sync; +drop table if exists ordinary_distinct sync; +create table distinct_in_order (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate); +insert into distinct_in_order select distinct CounterID, EventDate from test.hits order by CounterID, EventDate settings optimize_distinct_in_order=1; +create table ordinary_distinct (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate); +insert into ordinary_distinct select distinct CounterID, EventDate from test.hits settings optimize_distinct_in_order=0; +select distinct * from distinct_in_order except select * from ordinary_distinct; +drop table if exists distinct_in_order sync; +drop table if exists ordinary_distinct sync; From c4073328faa2502142f4bd98e864643ec16c0691 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Sun, 17 Jul 2022 20:22:32 -0400 Subject: [PATCH 476/659] add get_finished_status function --- tests/ci/rerun_helper.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/ci/rerun_helper.py b/tests/ci/rerun_helper.py index 35363593db6..0d523640f56 100644 --- a/tests/ci/rerun_helper.py +++ b/tests/ci/rerun_helper.py @@ -36,3 +36,9 @@ class RerunHelper: ): return True return False + + def get_finished_status(self): + for status in self.statuses: + if self.check_name in status.context: + return status + return None From f30b54a3924b19fddb9574ec3e8eb82018ae2e0e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Tue, 19 Jul 2022 08:57:03 -0400 Subject: [PATCH 477/659] fail_simple_check is added --- tests/ci/commit_status_helper.py | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index a53ce6715d5..1e34534e299 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -3,7 +3,7 @@ import time import os import csv -from env_helper import GITHUB_REPOSITORY +from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL from ci_config import CI_CONFIG RETRY = 5 @@ -73,3 +73,15 @@ def post_labels(gh, pr_info, labels_names): pull_request = repo.get_pull(pr_info.number) for label in labels_names: pull_request.add_to_labels(label) + + +def fail_simple_check(gh, pr_info, description) + if SKIP_SIMPLE_CHECK_LABEL in pr_info.labels: + return + commit = get_commit(gh, pr_info.sha) + commit.create_status( + context="Simple Check", + description=description, + state="failure", + target_url=GITHUB_RUN_URL, + ) From baff8327b949099950ce78d3e8857754d7cfff1b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Mon, 18 Jul 2022 15:15:21 -0400 Subject: [PATCH 478/659] do not recreate Simple Check if already exists --- tests/ci/commit_status_helper.py | 16 +++++++++++++++- tests/ci/fast_test_check.py | 22 +++++++--------------- tests/ci/run_check.py | 14 +++++++------- tests/ci/style_check.py | 24 +++++++----------------- 4 files changed, 36 insertions(+), 40 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 1e34534e299..420ca7a0ff7 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -5,6 +5,7 @@ import os import csv from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL from ci_config import CI_CONFIG +from pr_info import SKIP_SIMPLE_CHECK_LABEL RETRY = 5 @@ -75,7 +76,7 @@ def post_labels(gh, pr_info, labels_names): pull_request.add_to_labels(label) -def fail_simple_check(gh, pr_info, description) +def fail_simple_check(gh, pr_info, description): if SKIP_SIMPLE_CHECK_LABEL in pr_info.labels: return commit = get_commit(gh, pr_info.sha) @@ -85,3 +86,16 @@ def fail_simple_check(gh, pr_info, description) state="failure", target_url=GITHUB_RUN_URL, ) + + +def create_simple_check(gh, pr_info): + commit = get_commit(gh, pr_info.sha) + for status in commit.get_statuses(): + if "Simple Check" in status.context: + return + commit.create_status( + context="Simple Check", + description="Skipped", + state="success", + target_url=GITHUB_RUN_URL, + ) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index ce5a4195ceb..2e4d54f34c2 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -8,13 +8,16 @@ import sys from github import Github -from env_helper import CACHES_PATH, TEMP_PATH, GITHUB_SERVER_URL, GITHUB_REPOSITORY -from pr_info import FORCE_TESTS_LABEL, PRInfo, SKIP_SIMPLE_CHECK_LABEL +from env_helper import CACHES_PATH, TEMP_PATH +from pr_info import FORCE_TESTS_LABEL, PRInfo from s3_helper import S3Helper from get_robot_token import get_best_robot_token from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version -from commit_status_helper import post_commit_status, get_commit +from commit_status_helper import ( + post_commit_status, + fail_simple_check, +) from clickhouse_helper import ( ClickHouseHelper, mark_flaky_tests, @@ -219,16 +222,5 @@ if __name__ == "__main__": if FORCE_TESTS_LABEL in pr_info.labels and state != "error": print(f"'{FORCE_TESTS_LABEL}' enabled, will report success") else: - if SKIP_SIMPLE_CHECK_LABEL not in pr_info.labels: - url = ( - f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/" - "blob/master/.github/PULL_REQUEST_TEMPLATE.md?plain=1" - ) - commit = get_commit(gh, pr_info.sha) - commit.create_status( - context="Simple Check", - description=f"{NAME} failed", - state="failed", - target_url=url, - ) + fail_simple_check(gh, pr_info, f"{NAME} failed") sys.exit(1) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index b6d654c7bed..a39d97ce81d 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -6,7 +6,12 @@ from typing import Tuple from github import Github -from commit_status_helper import get_commit, post_labels, remove_labels +from commit_status_helper import ( + get_commit, + post_labels, + remove_labels, + create_simple_check, +) from env_helper import GITHUB_RUN_URL, GITHUB_REPOSITORY, GITHUB_SERVER_URL from get_robot_token import get_best_robot_token from pr_info import FORCE_TESTS_LABEL, PRInfo @@ -223,12 +228,7 @@ if __name__ == "__main__": if pr_labels_to_remove: remove_labels(gh, pr_info, pr_labels_to_remove) - commit.create_status( - context="Simple Check", - description="Skipped", - state="success", - target_url=GITHUB_RUN_URL, - ) + create_simple_check(gh, pr_info) if description_error: print( diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 84ed9e5a124..ce638c7fac5 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -10,15 +10,16 @@ from github import Github from env_helper import ( RUNNER_TEMP, GITHUB_WORKSPACE, - GITHUB_REPOSITORY, - GITHUB_SERVER_URL, ) from s3_helper import S3Helper -from pr_info import PRInfo, SKIP_SIMPLE_CHECK_LABEL +from pr_info import PRInfo from get_robot_token import get_best_robot_token from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version -from commit_status_helper import post_commit_status, get_commit +from commit_status_helper import ( + post_commit_status, + fail_simple_check, +) from clickhouse_helper import ( ClickHouseHelper, mark_flaky_tests, @@ -124,17 +125,6 @@ if __name__ == "__main__": ) ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) - if state == "error": - if SKIP_SIMPLE_CHECK_LABEL not in pr_info.labels: - url = ( - f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/" - "blob/master/.github/PULL_REQUEST_TEMPLATE.md?plain=1" - ) - commit = get_commit(gh, pr_info.sha) - commit.create_status( - context="Simple Check", - description=f"{NAME} failed", - state="failed", - target_url=url, - ) + if state in ["error", "failure"]: + fail_simple_check(gh, pr_info, f"{NAME} failed") sys.exit(1) From bdd11f671d51df3b6fb0b2e31a68d4311f65983c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Jul 2022 02:08:07 -0400 Subject: [PATCH 479/659] different path of endian.h for FreeBSD --- .../self-extracting-executable/compressor.cpp | 19 +++++++++---------- .../decompressor.cpp | 19 +++++++++---------- 2 files changed, 18 insertions(+), 20 deletions(-) diff --git a/utils/self-extracting-executable/compressor.cpp b/utils/self-extracting-executable/compressor.cpp index 061fe228c6a..98cb895bb3a 100644 --- a/utils/self-extracting-executable/compressor.cpp +++ b/utils/self-extracting-executable/compressor.cpp @@ -10,17 +10,16 @@ #include #include +#if (defined(__APPLE__) || defined(__FreeBSD__)) && defined(__GNUC__) +# include +#elif !defined(_MSC_VER) +# include +#endif + #if defined OS_DARWIN - -// dependencies -#include -#include - -// define 64 bit macros -#define htole64(x) OSSwapHostToLittleInt64(x) - -#else -#include +# include + // define 64 bit macros +# define htole64(x) OSSwapHostToLittleInt64(x) #endif #include "types.h" diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index f509469e9f1..c02b092f9d9 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -13,17 +13,16 @@ #include #include +#if (defined(__APPLE__) || defined(__FreeBSD__)) && defined(__GNUC__) +# include +#elif !defined(_MSC_VER) +# include +#endif + #if defined OS_DARWIN - -// dependencies -#include -#include - -// define 64 bit macros -#define le64toh(x) OSSwapLittleToHostInt64(x) - -#else -#include +# include + // define 64 bit macros +# define le64toh(x) OSSwapLittleToHostInt64(x) #endif #include "types.h" From 43a01294251cf7cf220a6680eadb1265409e0907 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 21 Jul 2022 08:43:38 +0200 Subject: [PATCH 480/659] Set default value cross_to_inner_join_rewrite = 1 --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9034fb924ba..e9cba533f42 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -760,7 +760,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, output_format_pretty_row_numbers, false, "Add row numbers before each row for pretty output format", 0) \ M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \ \ - M(UInt64, cross_to_inner_join_rewrite, 2, "Use inner join instead of comma/cross join if possible. Possible values: 0 - no rewrite, 1 - apply if possible for comma/cross, 2 - force rewrite all comma joins, cross - if possible", 0) \ + M(UInt64, cross_to_inner_join_rewrite, 1, "Use inner join instead of comma/cross join if there're joining expressions in the WHERE section. Values: 0 - no rewrite, 1 - apply if possible for comma/cross, 2 - force rewrite all comma joins, cross - if possible", 0) \ \ M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ M(Bool, output_format_arrow_string_as_string, false, "Use Arrow String type instead of Binary for String columns", 0) \ From 394fcf33ef8d4038093f52c80aa453478d44bc60 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Jul 2022 07:53:11 +0000 Subject: [PATCH 481/659] Fix test --- ...46_tsv_csv_best_effort_schema_inference.sh | 40 ++++++++++--------- 1 file changed, 21 insertions(+), 19 deletions(-) diff --git a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.sh b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.sh index 6589765f739..e8aa5914912 100755 --- a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.sh +++ b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.sh @@ -158,8 +158,10 @@ echo "CSV" echo -e "42,Some string,'[1, 2, 3, 4]','[(1, 2, 3)]' 42\,abcd,'[]','[(4, 5, 6)]'" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" +CLIENT_CMD="$CLICKHOUSE_CLIENT --format_csv_allow_single_quotes=1" + +$CLIENT_CMD -q "desc file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "select * from file('$FILE_NAME', 'CSV')" echo -e "\"[({'key' : 42.42}, ['String', 'String2'], 42.42), ({}, [], -42), ({'key2' : NULL}, [NULL], NULL)]\" '[]' @@ -168,8 +170,8 @@ echo -e "\"[({'key' : 42.42}, ['String', 'String2'], 42.42), ({}, [], -42), ({'k \"[({}, ['String3'], NULL)]\" \"[({'key3': NULL}, []), NULL]\""> $DATA_FILE -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV') settings input_format_csv_use_best_effort_in_schema_inference=false" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV') settings input_format_csv_use_best_effort_in_schema_inference=false" +$CLIENT_CMD -q "desc file('$FILE_NAME', 'CSV') settings input_format_csv_use_best_effort_in_schema_inference=false" +$CLIENT_CMD -q "select * from file('$FILE_NAME', 'CSV') settings input_format_csv_use_best_effort_in_schema_inference=false" echo -e "\"[({'key' : 42.42}, ['String', 'String2'], 42.42), ({}, [], -42), ({'key2' : NULL}, [NULL], NULL)]\" '[]' @@ -178,43 +180,43 @@ echo -e "\"[({'key' : 42.42}, ['String', 'String2'], 42.42), ({}, [], -42), ({'k \"[({}, ['String3'], NULL)]\" \"[({'key3': NULL}, [], NULL)]\""> $DATA_FILE -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "desc file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "select * from file('$FILE_NAME', 'CSV')" echo -e "true false \N" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "desc file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "select * from file('$FILE_NAME', 'CSV')" echo -e "'[true, NULL]' '[]' '[NULL]' '[false]'" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "desc file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "select * from file('$FILE_NAME', 'CSV')" echo -e "'(1, 2, 3)'"> $DATA_FILE -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "desc file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "select * from file('$FILE_NAME', 'CSV')" -echo -e "'123.123'"> $DATA_FILE +echo -e '"123.123"'> $DATA_FILE -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "desc file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "select * from file('$FILE_NAME', 'CSV')" echo -e "'[(1, 2, 3)]'"> $DATA_FILE -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "desc file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "select * from file('$FILE_NAME', 'CSV')" echo -e "\"[(1, 2, 3)]\""> $DATA_FILE -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "desc file('$FILE_NAME', 'CSV')" +$CLIENT_CMD -q "select * from file('$FILE_NAME', 'CSV')" From 98ff54700a7d9510b44f1b29329fd4ad3b8226a3 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 21 Jul 2022 16:23:25 +0800 Subject: [PATCH 482/659] Fix trivial count optimization with array join --- src/Interpreters/TreeRewriter.cpp | 2 +- ...2367_optimize_trivial_count_with_array_join.reference | 1 + .../02367_optimize_trivial_count_with_array_join.sql | 9 +++++++++ 3 files changed, 11 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.reference create mode 100644 tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index b389c3eb705..d5a3b26ffc6 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1016,7 +1016,7 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select has_explicit_columns = !required.empty(); if (is_select && !has_explicit_columns) { - optimize_trivial_count = true; + optimize_trivial_count = !columns_context.has_array_join; /// You need to read at least one column to find the number of rows. /// We will find a column with minimum . diff --git a/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.reference b/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql b/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql new file mode 100644 index 00000000000..8d812c56ffb --- /dev/null +++ b/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql @@ -0,0 +1,9 @@ +drop table if exists t; + +create table t(id UInt32) engine MergeTree order by id; + +insert into t values (1); + +select count() from t array join range(2) as a settings optimize_trivial_count_query = 1; + +drop table t; From da0bdd474b1fd3c0ed60f56533ba9af205a9d73c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 21 Jul 2022 10:48:41 +0200 Subject: [PATCH 483/659] Fix code style. --- programs/client/Client.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 74559920954..584806951cf 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -125,9 +125,9 @@ void Client::showWarnings() std::vector Client::loadWarningMessages() { /// Older server versions cannot execute the query loading warnings. - constexpr UInt64 MIN_SERVER_REVISION_TO_LOAD_WARNINGS = DBMS_MIN_PROTOCOL_VERSION_WITH_VIEW_IF_PERMITTED; + constexpr UInt64 min_server_revision_to_load_warnings = DBMS_MIN_PROTOCOL_VERSION_WITH_VIEW_IF_PERMITTED; - if (server_revision < MIN_SERVER_REVISION_TO_LOAD_WARNINGS) + if (server_revision < min_server_revision_to_load_warnings) return {}; std::vector messages; From 49d980d16c537e0e5f774aeb65b201135ab3132a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Jul 2022 08:55:51 +0000 Subject: [PATCH 484/659] Extend time units in parseTimeDelta function --- src/Functions/parseTimeDelta.cpp | 26 ++++++++++++++++--- .../02354_parse_timedelta.reference | 4 +++ .../0_stateless/02354_parse_timedelta.sql | 4 +++ 3 files changed, 30 insertions(+), 4 deletions(-) diff --git a/src/Functions/parseTimeDelta.cpp b/src/Functions/parseTimeDelta.cpp index fb5a7621a53..185c36170a1 100644 --- a/src/Functions/parseTimeDelta.cpp +++ b/src/Functions/parseTimeDelta.cpp @@ -18,7 +18,8 @@ namespace ErrorCodes namespace { - const std::unordered_map time_unit_to_float = { + const std::unordered_map time_unit_to_float = + { {"years", 365 * 24 * 3600}, {"year", 365 * 24 * 3600}, {"yr", 365 * 24 * 3600}, @@ -50,8 +51,26 @@ namespace {"second", 1}, {"sec", 1}, {"s", 1}, + + {"milliseconds", 1e-3}, + {"millisecond", 1e-3}, + {"millisec", 1e-3}, + {"ms", 1e-3}, + + {"microseconds", 1e-6}, + {"microsecond", 1e-6}, + {"microsec", 1e-6}, + {"μs", 1e-6}, + {"us", 1e-6}, + + {"nanoseconds", 1e-9}, + {"nanosecond", 1e-9}, + {"nanosec", 1e-9}, + {"ns", 1e-9}, }; + const std::unordered_set separators = {' ', ';', '-', '+', ',', ':'}; + /** Prints amount of seconds in form of: * "1 year 2 months 4 weeks 12 days 3 hours 1 minute 33 seconds". * ' ', ';', '-', '+', ',', ':' can be used as separator, eg. "1yr-2mo", "2m:6s" @@ -248,7 +267,7 @@ namespace static bool scanUnit(std::string_view & str, Int64 & index, Int64 last_pos) { int64_t begin_index = index; - while (index <= last_pos && isalpha(str[index])) + while (index <= last_pos && !isdigit(str[index]) && !separators.contains(str[index])) { index++; } @@ -271,8 +290,7 @@ namespace scanSpaces(str, index, last_pos); /// ignore separator - if (index <= last_pos - && (str[index] == ';' || str[index] == '-' || str[index] == '+' || str[index] == ',' || str[index] == ':')) + if (index <= last_pos && (separators.contains(str[index]))) { index++; } diff --git a/tests/queries/0_stateless/02354_parse_timedelta.reference b/tests/queries/0_stateless/02354_parse_timedelta.reference index 9bc208ada9d..f9dd7879057 100644 --- a/tests/queries/0_stateless/02354_parse_timedelta.reference +++ b/tests/queries/0_stateless/02354_parse_timedelta.reference @@ -5,3 +5,7 @@ 36806400 1331 40273293 +1.001001001 +1.001001001 +1.001001001 +1.11111111111 diff --git a/tests/queries/0_stateless/02354_parse_timedelta.sql b/tests/queries/0_stateless/02354_parse_timedelta.sql index 3cf282d715a..29f2bf9fdfc 100644 --- a/tests/queries/0_stateless/02354_parse_timedelta.sql +++ b/tests/queries/0_stateless/02354_parse_timedelta.sql @@ -5,6 +5,10 @@ SELECT parseTimeDelta('0.00123 seconds'); SELECT parseTimeDelta('1yr2mo'); SELECT parseTimeDelta('11s+22min'); SELECT parseTimeDelta('1yr-2mo-4w + 12 days, 3 hours : 1 minute ; 33 seconds'); +SELECT parseTimeDelta('1s1ms1us1ns'); +SELECT parseTimeDelta('1s1ms1μs1ns'); +SELECT parseTimeDelta('1s - 1ms : 1μs ; 1ns'); +SELECT parseTimeDelta('1.11s1.11ms1.11us1.11ns'); -- invalid expressions SELECT parseTimeDelta(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} From d63786e7097623a55ef2ad139878656d3cd0c7c7 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Jul 2022 09:07:23 +0000 Subject: [PATCH 485/659] Better --- src/Functions/parseTimeDelta.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Functions/parseTimeDelta.cpp b/src/Functions/parseTimeDelta.cpp index 185c36170a1..8cb7c229ae8 100644 --- a/src/Functions/parseTimeDelta.cpp +++ b/src/Functions/parseTimeDelta.cpp @@ -69,8 +69,6 @@ namespace {"ns", 1e-9}, }; - const std::unordered_set separators = {' ', ';', '-', '+', ',', ':'}; - /** Prints amount of seconds in form of: * "1 year 2 months 4 weeks 12 days 3 hours 1 minute 33 seconds". * ' ', ';', '-', '+', ',', ':' can be used as separator, eg. "1yr-2mo", "2m:6s" @@ -267,7 +265,7 @@ namespace static bool scanUnit(std::string_view & str, Int64 & index, Int64 last_pos) { int64_t begin_index = index; - while (index <= last_pos && !isdigit(str[index]) && !separators.contains(str[index])) + while (index <= last_pos && !isdigit(str[index]) && !isSeparator(str[index])) { index++; } @@ -290,13 +288,18 @@ namespace scanSpaces(str, index, last_pos); /// ignore separator - if (index <= last_pos && (separators.contains(str[index]))) + if (index <= last_pos && isSeparator(str[index])) { index++; } scanSpaces(str, index, last_pos); } + + static bool isSeparator(char symbol) + { + return symbol == ';' || symbol == '-' || symbol == '+' || symbol == ',' || symbol == ':' || symbol == ' '; + } }; } From 1ea9f143ff5bc9ffd4dbd999d1730fd5f7512f3e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 21 Jul 2022 09:32:29 +0200 Subject: [PATCH 486/659] Leave only _row_exists-based implementation of lightweight delete --- src/Compression/CompressionFactory.cpp | 4 +- src/Compression/CompressionFactory.h | 2 +- src/Core/Settings.h | 1 - src/Formats/NativeWriter.cpp | 2 +- src/Formats/NativeWriter.h | 2 - src/Interpreters/Context.h | 6 +- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Interpreters/InterpreterDeleteQuery.cpp | 64 ++--- src/Interpreters/MutationsInterpreter.cpp | 84 +----- src/Interpreters/MutationsInterpreter.h | 13 +- .../QueryPlan/ReadFromMergeTree.cpp | 2 +- .../MergeTree/DataPartStorageOnDisk.cpp | 26 -- .../MergeTree/DataPartStorageOnDisk.h | 3 - .../MergeTree/FutureMergedMutatedPart.h | 2 - src/Storages/MergeTree/IDataPartStorage.h | 4 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 29 -- src/Storages/MergeTree/IMergeTreeDataPart.h | 5 - src/Storages/MergeTree/IMergeTreeReader.h | 2 +- .../MergeTreeBaseSelectProcessor.cpp | 5 +- .../MergeTreeDataPartDeletedMask.cpp | 162 ----------- .../MergeTree/MergeTreeDataPartDeletedMask.h | 34 --- src/Storages/MergeTree/MergeTreeIOSettings.h | 4 +- .../MergeTree/MergeTreeMutationEntry.cpp | 26 +- .../MergeTree/MergeTreeMutationEntry.h | 10 +- .../MergeTree/MergeTreeMutationStatus.h | 2 - .../MergeTree/MergeTreeRangeReader.cpp | 102 +------ src/Storages/MergeTree/MergeTreeRangeReader.h | 8 - .../MergeTree/MergeTreeSequentialSource.cpp | 121 +++----- .../MergeTree/MergeTreeSequentialSource.h | 4 - src/Storages/MergeTree/MutateTask.cpp | 258 +----------------- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 1 - .../MergeTree/StorageFromMergeTreeDataPart.h | 5 + src/Storages/StorageJoin.cpp | 2 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 30 +- src/Storages/StorageMergeTree.h | 4 +- .../System/StorageSystemMutations.cpp | 2 - .../02117_show_create_table_system.reference | 1 - ...lightweight_delete_on_merge_tree.reference | 10 +- ...02319_lightweight_delete_on_merge_tree.sql | 4 +- .../0_stateless/02352_lightweight_delete.sql | 2 +- 41 files changed, 132 insertions(+), 920 deletions(-) delete mode 100644 src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp delete mode 100644 src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 94fb30af1bc..b8a1c5877a4 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -98,14 +98,14 @@ CompressionCodecPtr CompressionCodecFactory::get( } -CompressionCodecPtr CompressionCodecFactory::get(uint8_t byte_code, const IDataType * column_type) const +CompressionCodecPtr CompressionCodecFactory::get(uint8_t byte_code) const { const auto family_code_and_creator = family_code_with_codec.find(byte_code); if (family_code_and_creator == family_code_with_codec.end()) throw Exception("Unknown codec family code: " + toString(byte_code), ErrorCodes::UNKNOWN_CODEC); - return family_code_and_creator->second({}, column_type); + return family_code_and_creator->second({}, nullptr); } diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index c386784686e..a4451f9ed2e 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -63,7 +63,7 @@ public: } /// Get codec by method byte (no params available) - CompressionCodecPtr get(uint8_t byte_code, const IDataType * column_type = nullptr) const; + CompressionCodecPtr get(uint8_t byte_code) const; /// For backward compatibility with config settings CompressionCodecPtr get(const String & family_name, std::optional level) const; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cb932843fc2..672b8c5b1fb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -458,7 +458,6 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \ M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ - M(Bool, allow_experimental_lightweight_delete, false, "Enable lightweight DELETE mutations for mergetree tables. Work in progress", 0) \ M(Bool, allow_experimental_lightweight_delete_with_row_exists, false, "Enable lightweight DELETE mutations using __rows_exists column for mergetree tables. Work in progress", 0) \ M(Bool, lightweight_delete_mutation, true, "Enable to make ordinary ALTER DELETE queries lightweight for mergetree tables", 0) \ M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ diff --git a/src/Formats/NativeWriter.cpp b/src/Formats/NativeWriter.cpp index 004c75182a7..77692eec6b6 100644 --- a/src/Formats/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -46,7 +46,7 @@ void NativeWriter::flush() } -void NativeWriter::writeData(const ISerialization & serialization, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit) +static void writeData(const ISerialization & serialization, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit) { /** If there are columns-constants - then we materialize them. * (Since the data type does not know how to serialize / deserialize constants.) diff --git a/src/Formats/NativeWriter.h b/src/Formats/NativeWriter.h index 02fc53b60fe..010a03ec722 100644 --- a/src/Formats/NativeWriter.h +++ b/src/Formats/NativeWriter.h @@ -32,8 +32,6 @@ public: static String getContentType() { return "application/octet-stream"; } - static void writeData(const ISerialization & serialization, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit); - private: WriteBuffer & ostr; UInt64 client_revision; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c83e38a0ed1..7e3bbf43f39 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -361,7 +361,7 @@ private: inline static ContextPtr global_context_instance; /// A flag, used to mark if reader needs to apply deleted rows mask. - bool skip_deleted_mask = false; + bool apply_deleted_mask = true; public: // Top-level OpenTelemetry trace context for the query. Makes sense only for a query context. @@ -915,8 +915,8 @@ public: bool isInternalQuery() const { return is_internal_query; } void setInternalQuery(bool internal) { is_internal_query = internal; } - bool skipDeletedMask() const { return skip_deleted_mask; } - void setSkipDeletedMask(bool skip) { skip_deleted_mask = skip; } + bool applyDeletedMask() const { return apply_deleted_mask; } + void setApplyDeletedMask(bool apply) { apply_deleted_mask = apply; } ActionLocksManagerPtr getActionLocksManager(); diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 03b0e1d5894..056a3d9f7b4 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -144,7 +144,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) if (!mutation_commands.empty()) { table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); - MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false, false).validate(); + MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate(); table->mutate(mutation_commands, getContext()); } diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index aeeb72ad06c..cb4bc363d18 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -33,8 +33,7 @@ InterpreterDeleteQuery::InterpreterDeleteQuery(const ASTPtr & query_ptr_, Contex BlockIO InterpreterDeleteQuery::execute() { - if (!getContext()->getSettingsRef().allow_experimental_lightweight_delete && - !getContext()->getSettingsRef().allow_experimental_lightweight_delete_with_row_exists) + if (!getContext()->getSettingsRef().allow_experimental_lightweight_delete_with_row_exists) { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Lightweight delete mutate is experimental. Set `allow_experimental_lightweight_delete` setting to enable it"); } @@ -73,50 +72,29 @@ BlockIO InterpreterDeleteQuery::execute() MutationCommands mutation_commands; MutationCommand mut_command; - if (getContext()->getSettingsRef().allow_experimental_lightweight_delete_with_row_exists) - { - /// Build "UPDATE _row_exists = 0 WHERE predicate" query - mut_command.type = MutationCommand::Type::UPDATE; - mut_command.predicate = delete_query.predicate; + /// Build "UPDATE _row_exists = 0 WHERE predicate" query + mut_command.type = MutationCommand::Type::UPDATE; + mut_command.predicate = delete_query.predicate; - auto command = std::make_shared(); - command->type = ASTAlterCommand::UPDATE; - command->predicate = delete_query.predicate; - command->update_assignments = std::make_shared(); - auto set_row_does_not_exist = std::make_shared(); - set_row_does_not_exist->column_name = metadata_snapshot->lightweight_delete_description.filter_column.name; - auto zero_value = std::make_shared(DB::Field(UInt8(0))); - set_row_does_not_exist->children.push_back(zero_value); - command->update_assignments->children.push_back(set_row_does_not_exist); - command->children.push_back(command->predicate); - command->children.push_back(command->update_assignments); - mut_command.column_to_update_expression[set_row_does_not_exist->column_name] = zero_value; - mut_command.ast = command->ptr(); + auto command = std::make_shared(); + command->type = ASTAlterCommand::UPDATE; + command->predicate = delete_query.predicate; + command->update_assignments = std::make_shared(); + auto set_row_does_not_exist = std::make_shared(); + set_row_does_not_exist->column_name = metadata_snapshot->lightweight_delete_description.filter_column.name; + auto zero_value = std::make_shared(DB::Field(UInt8(0))); + set_row_does_not_exist->children.push_back(zero_value); + command->update_assignments->children.push_back(set_row_does_not_exist); + command->children.push_back(command->predicate); + command->children.push_back(command->update_assignments); + mut_command.column_to_update_expression[set_row_does_not_exist->column_name] = zero_value; + mut_command.ast = command->ptr(); - mutation_commands.emplace_back(mut_command); - - table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); - MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false, false).validate(); - storage_merge_tree->mutate(mutation_commands, getContext(), MutationType::Ordinary); - } - else - { - mut_command.type = MutationCommand::Type::DELETE; - mut_command.predicate = delete_query.predicate; - - auto command = std::make_shared(); - command->type = ASTAlterCommand::DELETE; - command->predicate = delete_query.predicate; - command->children.push_back(command->predicate); - mut_command.ast = command->ptr(); - - mutation_commands.emplace_back(mut_command); - - table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); - MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false, false).validate(); - storage_merge_tree->mutate(mutation_commands, getContext(), MutationType::Lightweight); - } + mutation_commands.emplace_back(mut_command); + table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); + MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate(); + storage_merge_tree->mutate(mutation_commands, getContext()); return {}; } diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index f896c2269e8..7778e316b8b 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -285,20 +285,15 @@ MutationsInterpreter::MutationsInterpreter( const StorageMetadataPtr & metadata_snapshot_, MutationCommands commands_, ContextPtr context_, - bool can_execute_, - bool is_lightweight_) + bool can_execute_) : storage(std::move(storage_)) , metadata_snapshot(metadata_snapshot_) , commands(std::move(commands_)) , context(Context::createCopy(context_)) , can_execute(can_execute_) , select_limits(SelectQueryOptions().analyze(!can_execute).ignoreLimits().ignoreProjections()) - , is_lightweight(is_lightweight_) { - if (is_lightweight) - mutation_ast = prepareLightweightDelete(!can_execute); - else - mutation_ast = prepare(!can_execute); + mutation_ast = prepare(!can_execute); } static NameSet getKeyColumns(const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot) @@ -777,6 +772,13 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects(); auto all_columns = storage_snapshot->getColumns(options); + // TODO: add _row_exists column if it is present in the part??? + if (auto part_storage = dynamic_pointer_cast(storage)) + { + if (part_storage->hasLightweightDeleteColumn()) + all_columns.push_back({metadata_snapshot->lightweight_delete_description.filter_column}); + } + /// Next, for each stage calculate columns changed by this and previous stages. for (size_t i = 0; i < prepared_stages.size(); ++i) { @@ -905,70 +907,6 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & return select; } -/// Prepare for lightweight delete -ASTPtr MutationsInterpreter::prepareLightweightDelete(bool dry_run) -{ - if (is_prepared) - throw Exception("MutationsInterpreter is already prepared. It is a bug.", ErrorCodes::LOGICAL_ERROR); - - if (commands.empty()) - throw Exception("Empty mutation commands list", ErrorCodes::LOGICAL_ERROR); - - /// For lightweight DELETE, we use predicate expression to get deleted rows. - /// Collect predicates in the commands - for (auto & command : commands) - { - if (command.type == MutationCommand::DELETE) - { - mutation_kind.set(MutationKind::MUTATE_OTHER); - if (stages.empty()) - stages.emplace_back(context); - - auto mask_predicate = getPartitionAndPredicateExpressionForMutationCommand(command); - stages.back().filters.push_back(mask_predicate); - } - else - throw Exception("Unsupported lightweight mutation command type: " + DB::toString(command.type), ErrorCodes::UNKNOWN_MUTATION_COMMAND); - } - - /// The updated_header is empty for lightweight delete. - updated_header = std::make_unique(); - - is_prepared = true; - - return prepareInterpreterSelectQueryLightweight(stages, dry_run); -} - -ASTPtr MutationsInterpreter::prepareInterpreterSelectQueryLightweight(std::vector & prepared_stages, bool) -{ - /// Construct a SELECT statement for lightweight delete is like "select _part_offset from db.table where " - auto select = std::make_shared(); - - /// DELETEs only query just need the _part_offset virtual column without real columns - select->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); - select->select()->children.push_back(std::make_shared("_part_offset")); - - ASTPtr where_expression; - if (!prepared_stages[0].filters.empty()) - { - if (prepared_stages[0].filters.size() == 1) - where_expression = prepared_stages[0].filters[0]; - else - { - auto coalesced_predicates = std::make_shared(); - coalesced_predicates->name = "or"; - coalesced_predicates->arguments = std::make_shared(); - coalesced_predicates->children.push_back(coalesced_predicates->arguments); - coalesced_predicates->arguments->children = prepared_stages[0].filters; - where_expression = std::move(coalesced_predicates); - } - - select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where_expression)); - } - - return select; -} - QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const { for (size_t i_stage = 1; i_stage < prepared_stages.size(); ++i_stage) @@ -1053,10 +991,10 @@ QueryPipelineBuilder MutationsInterpreter::execute() if (!select_interpreter) { /// Skip to apply deleted mask for MutateSomePartColumn cases when part has lightweight delete. - if (!is_lightweight && skip_deleted_mask) + if (!apply_deleted_mask) { auto context_for_reading = Context::createCopy(context); - context_for_reading->setSkipDeletedMask(skip_deleted_mask); + context_for_reading->setApplyDeletedMask(apply_deleted_mask); select_interpreter = std::make_unique(mutation_ast, context_for_reading, storage, metadata_snapshot, select_limits); } else diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 49165c6f9ad..10f764caaee 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -43,8 +43,7 @@ public: const StorageMetadataPtr & metadata_snapshot_, MutationCommands commands_, ContextPtr context_, - bool can_execute_, - bool is_lightweight_); + bool can_execute_); void validate(); @@ -79,16 +78,14 @@ public: MutationKind::MutationKindEnum getMutationKind() const { return mutation_kind.mutation_kind; } - void setSkipDeletedMask(bool skip) { skip_deleted_mask = skip; } + void setApplyDeletedMask(bool apply) { apply_deleted_mask = apply; } private: ASTPtr prepare(bool dry_run); - ASTPtr prepareLightweightDelete(bool dry_run); struct Stage; ASTPtr prepareInterpreterSelectQuery(std::vector &prepared_stages, bool dry_run); - static ASTPtr prepareInterpreterSelectQueryLightweight(std::vector &prepared_stages, bool dry_run); QueryPipelineBuilder addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const; @@ -103,10 +100,8 @@ private: bool can_execute; SelectQueryOptions select_limits; - /// True for lightweight delete. - bool is_lightweight = false; - /// True for MutateSomePartColumns on part with lightweight. - bool skip_deleted_mask = false; + /// TODO: is it needed? + bool apply_deleted_mask = true; ASTPtr mutation_ast; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 9a65cd4f17e..f060d42c718 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -62,7 +62,7 @@ static MergeTreeReaderSettings getMergeTreeReaderSettings( .save_marks_in_cache = true, .checksum_on_read = settings.checksum_on_read, .read_in_order = query_info.input_order_info != nullptr, - .skip_deleted_mask = context->skipDeletedMask(), + .apply_deleted_mask = context->applyDeletedMask(), }; } diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 1c4d8d9186a..03d24d84bb0 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -652,32 +652,6 @@ bool DataPartStorageOnDisk::shallParticipateInMerges(const IStoragePolicy & stor return !volume_ptr->areMergesAvoided(); } -void DataPartStorageOnDisk::loadDeletedRowsMask(MergeTreeDataPartDeletedMask & deleted_mask) const -{ - String deleted_mask_path = fs::path(getRelativePath()) / deleted_mask.name; - auto disk = volume->getDisk(); - - if (disk->isFile(deleted_mask_path)) - { - auto read_buf = openForReading(disk, deleted_mask_path); - deleted_mask.read(*read_buf); - assertEOF(*read_buf); - } -} - -void DataPartStorageOnDisk::writeDeletedRowsMask(const MergeTreeDataPartDeletedMask & deleted_mask) const -{ - const String final_path = fs::path(getRelativePath()) / deleted_mask.name; - const String tmp_path = final_path + ".tmp"; - - { - auto out = volume->getDisk()->writeFile(tmp_path, 4096); - deleted_mask.write(*out); - } - - volume->getDisk()->moveFile(tmp_path, final_path); -} - void DataPartStorageOnDisk::backup( TemporaryFilesOnDisks & temp_dirs, const MergeTreeDataPartChecksums & checksums, diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index 2362d30a92d..2426b5eee80 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -86,9 +86,6 @@ public: bool shallParticipateInMerges(const IStoragePolicy &) const override; - void loadDeletedRowsMask(MergeTreeDataPartDeletedMask & deleted_mask) const override; - void writeDeletedRowsMask(const MergeTreeDataPartDeletedMask & deleted_mask) const override; - void backup( TemporaryFilesOnDisks & temp_dirs, const MergeTreeDataPartChecksums & checksums, diff --git a/src/Storages/MergeTree/FutureMergedMutatedPart.h b/src/Storages/MergeTree/FutureMergedMutatedPart.h index 06659249cae..4447687c3d9 100644 --- a/src/Storages/MergeTree/FutureMergedMutatedPart.h +++ b/src/Storages/MergeTree/FutureMergedMutatedPart.h @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB @@ -24,7 +23,6 @@ struct FutureMergedMutatedPart MergeTreePartInfo part_info; MergeTreeData::DataPartsVector parts; MergeType merge_type = MergeType::Regular; - MutationType mutation_type = MutationType::Ordinary; const MergeTreePartition & getPartition() const { return parts.front()->partition; } diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index cfd29a550e1..f0173baecb7 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -3,7 +3,6 @@ #include #include #include -#include #include namespace DB @@ -170,9 +169,6 @@ public: /// A leak of abstraction virtual bool shallParticipateInMerges(const IStoragePolicy &) const { return true; } - virtual void loadDeletedRowsMask(MergeTreeDataPartDeletedMask & deleted_mask) const = 0; - virtual void writeDeletedRowsMask(const MergeTreeDataPartDeletedMask & deleted_mask) const = 0; - /// Create a backup of a data part. /// This method adds a new entry to backup_entries. /// Also creates a new tmp_dir for internal disk (if disk is mentioned the first time). diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index b8fd70193f1..3f8000f3136 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1217,35 +1217,6 @@ bool IMergeTreeDataPart::supportLightweightDeleteMutate() const return part_type == MergeTreeDataPartType::Wide && parent_part == nullptr && projection_parts.empty(); } -bool IMergeTreeDataPart::getDeletedMask(MergeTreeDataPartDeletedMask & deleted_mask) const -{ - bool found = false; - - /// Check if deleted mask file exists. - if (data_part_storage->exists(String(deleted_mask.name))) - { - data_part_storage->loadDeletedRowsMask(deleted_mask); - - if (deleted_mask.getDeletedRows().size() != rows_count) - throw Exception(ErrorCodes::CORRUPTED_DATA, - "Size of deleted mask loaded from '{}':'{}' doesn't match expected " - "for part {}" - "(loaded {} rows, expected {} rows).", - data_part_storage->getDiskPath(), deleted_mask.name, name, deleted_mask.getDeletedRows().size(), rows_count); - - found = true; - } - - return found; -} - -void IMergeTreeDataPart::writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask) const -{ - MergeTreeDataPartDeletedMask deleted_mask {}; - deleted_mask.setDeletedRows(new_mask); - data_part_storage->writeDeletedRowsMask(deleted_mask); -} - void IMergeTreeDataPart::assertHasVersionMetadata(MergeTreeTransaction * txn) const { TransactionID expected_tid = txn ? txn->tid : Tx::PrehistoricTID; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 7869ca52969..a9c4590c045 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -12,7 +12,6 @@ #include #include #include -#include #include #include #include @@ -465,10 +464,6 @@ public: /// True if here is lightweight deleted mask file in part. bool hasLightweightDelete() const { return data_part_storage->exists(DELETED_ROWS_MARK_FILE_NAME); } - /// Read lightweight deleted mask when needed. - bool getDeletedMask(MergeTreeDataPartDeletedMask & deleted_mask) const; - void writeDeletedMask(MergeTreeDataPartDeletedMask::DeletedRows new_mask) const; - protected: /// Total size of all columns, calculated once in calcuateColumnSizesOnDisk diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 229f62da293..41030e522ac 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -62,7 +62,7 @@ public: MergeTreeData::DataPartPtr data_part; - bool needReadDeletedMask() { return !settings.skip_deleted_mask && data_part->hasLightweightDelete(); } + bool needReadDeletedMask() { return settings.apply_deleted_mask && data_part->hasLightweightDelete(); } protected: /// Returns actual column type in part, which can differ from table metadata. diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index a5649c0b41b..4cae54492c8 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -243,7 +243,7 @@ void MergeTreeBaseSelectProcessor::initializeMergeTreeReadersForPart( /// Add lightweight delete filtering step const auto & lightweigth_delete_info = metadata_snapshot->lightweight_delete_description; - if (!reader_settings.skip_deleted_mask && data_part->getColumns().contains(lightweigth_delete_info.filter_column.name)) + if (reader_settings.apply_deleted_mask && data_part->getColumns().contains(lightweigth_delete_info.filter_column.name)) { pre_reader_for_step.push_back(data_part->getReader({lightweigth_delete_info.filter_column}, metadata_snapshot, mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, @@ -269,7 +269,7 @@ void MergeTreeBaseSelectProcessor::initializeRangeReaders(MergeTreeReadTask & cu /// Add filtering step with lightweight delete mask const auto & lightweigth_delete_info = storage_snapshot->metadata->lightweight_delete_description; - if (!reader_settings.skip_deleted_mask && current_task.data_part->getColumns().contains(lightweigth_delete_info.filter_column.name)) + if (reader_settings.apply_deleted_mask && current_task.data_part->getColumns().contains(lightweigth_delete_info.filter_column.name)) { current_task.pre_range_readers.push_back( MergeTreeRangeReader(pre_reader_for_step[0].get(), prev_reader, &lwd_filter_step, last_reader, non_const_virtual_column_names)); @@ -292,7 +292,6 @@ void MergeTreeBaseSelectProcessor::initializeRangeReaders(MergeTreeReadTask & cu prev_reader = ¤t_task.pre_range_readers.back(); } - } if (!last_reader) diff --git a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp deleted file mode 100644 index d1a78623278..00000000000 --- a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.cpp +++ /dev/null @@ -1,162 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - - -namespace DB::ErrorCodes -{ - extern const int UNKNOWN_FORMAT_VERSION; - extern const int CORRUPTED_DATA; -} - -namespace DB -{ - -namespace -{ -struct DeletedRowsHash -{ -private: - char value[16]; - -public: - DeletedRowsHash() = default; - explicit DeletedRowsHash(SipHash & hasher) - { - hasher.get128(value); - - static_assert(std::is_pod_v, "Expected to be a POD-type"); - static_assert(sizeof(DeletedRowsHash) * 8 == 128); - } - - bool operator==(const DeletedRowsHash & other) const - { - return memcmp(value, other.value, sizeof(value)) == 0; - } -}; - -constexpr UInt8 FORMAT_VERSION = 1; -constexpr UInt8 DEFAULT_CODEC = static_cast(CompressionMethodByte::T64); -constexpr UInt8 PADDING_SIZE = 7; // just in case -constexpr UInt8 HEADER_SIZE = 0 - + sizeof(FORMAT_VERSION) - + sizeof(UInt64) // number of rows in mask - + sizeof(DeletedRowsHash) // column data hash - + PADDING_SIZE // padding: zero-bytes - + sizeof(DeletedRowsHash); // header hash -} - -MergeTreeDataPartDeletedMask::MergeTreeDataPartDeletedMask() - : deleted_rows(ColumnUInt8::create()) -{} - -const ColumnUInt8 & MergeTreeDataPartDeletedMask::getDeletedRows() const -{ - return *deleted_rows; -} - -void MergeTreeDataPartDeletedMask::setDeletedRows(DeletedRows new_rows) -{ - deleted_rows.swap(new_rows); -} - -void MergeTreeDataPartDeletedMask::setDeletedRows(size_t rows, bool value) -{ - setDeletedRows(ColumnUInt8::create(rows, value)); -} - -void MergeTreeDataPartDeletedMask::read(ReadBuffer & in) -{ - std::array header_buffer_data; - in.readStrict(header_buffer_data.data(), header_buffer_data.size()); - {// validate hash of the header first - SipHash hash; - hash.update(header_buffer_data.data(), header_buffer_data.size()); - const DeletedRowsHash computed_hash(hash); - - DeletedRowsHash read_hash; - readPODBinary(read_hash, in); - if (read_hash != computed_hash) - throw Exception(ErrorCodes::CORRUPTED_DATA, - "Invalid deleted masks file header hash"); - } - - UInt8 format_version = FORMAT_VERSION; - UInt64 stored_rows = 0; - DeletedRowsHash column_hash; - {// Read header values - ReadBuffer header(header_buffer_data.data(), header_buffer_data.size(), 0); - readBinary(format_version, header); - if (format_version != FORMAT_VERSION) - throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, - "Unknown deleted mask file format version {}", - static_cast(format_version)); - - readBinary(stored_rows, header); - readPODBinary(column_hash, header); - header.ignore(PADDING_SIZE); - assertEOF(header); - } - - auto data_read_buffer = std::make_unique(in); - - auto res_column = DeletedRows(ColumnUInt8::create()); - ColumnPtr res_col_ptr = res_column; - SerializationPtr serialization = DataTypeUInt8().getDefaultSerialization(); - NativeReader::readData(*serialization, res_col_ptr, *data_read_buffer, stored_rows, 0); - assertEOF(*data_read_buffer); - - // we probably don't want to check column hash here, since codec verifies data integrity. - deleted_rows = std::move(res_column); -} - -void MergeTreeDataPartDeletedMask::write(WriteBuffer & out) const -{ - {// Header - std::array header_buffer_data; - WriteBuffer header(header_buffer_data.data(), header_buffer_data.size()); - - writeBinary(FORMAT_VERSION, header); - writeBinary(static_cast(deleted_rows->size()), header); - - { - SipHash hash; - deleted_rows->updateHashFast(hash); - writePODBinary(DeletedRowsHash(hash), header); - } - - { - const char padding[PADDING_SIZE] = {'\0'}; - writePODBinary(padding, header); - } - assert(header_buffer_data.max_size() == header.count()); - - writePODBinary(header_buffer_data, out); - {// header hash - SipHash hash; - hash.update(header_buffer_data.data(), header_buffer_data.size()); - writePODBinary(DeletedRowsHash(hash), out); - } - } - assert(HEADER_SIZE == out.count()); - - const DataTypeUInt8 col_datatype; - auto codec = CompressionCodecFactory::instance().get(static_cast(DEFAULT_CODEC), &col_datatype); - auto data_write_buffer = std::make_unique(out, codec); - SerializationPtr serialization = col_datatype.getDefaultSerialization(); - - NativeWriter::writeData(*serialization, deleted_rows, *data_write_buffer, 0, deleted_rows->size()); - data_write_buffer->finalize(); -} - -} diff --git a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h b/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h deleted file mode 100644 index c8652746d98..00000000000 --- a/src/Storages/MergeTree/MergeTreeDataPartDeletedMask.h +++ /dev/null @@ -1,34 +0,0 @@ -#pragma once - -#include -#include -#include - -#include - -namespace DB -{ -class ReadBuffer; -class WriteBuffer; - -/// Per-part info about rows deleted by lightweight mutations. -struct MergeTreeDataPartDeletedMask -{ - explicit MergeTreeDataPartDeletedMask(); - using DeletedRows = ColumnUInt8::Ptr; - - static constexpr std::string_view name = "deleted_rows_mask.bin"; - - const ColumnUInt8 & getDeletedRows() const; - const DeletedRows & getDeletedRowsPtr() const { return deleted_rows; } - void setDeletedRows(DeletedRows new_rows); - void setDeletedRows(size_t rows, bool value); - - void read(ReadBuffer & in); - void write(WriteBuffer & out) const; - -private: - ColumnUInt8::Ptr deleted_rows; -}; - -}; diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index 02372011876..55848e09434 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -23,8 +23,8 @@ struct MergeTreeReaderSettings bool checksum_on_read = true; /// True if we read in order of sorting key. bool read_in_order = false; - /// Do not apply deleted mask for internal select from mutate some part columns. - bool skip_deleted_mask = false; + /// Deleted mask is applied to all reads except internal select from mutate some part columns. + bool apply_deleted_mask = true; }; struct MergeTreeWriterSettings diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 08f45e85d23..a222f2a8ad8 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -46,7 +46,7 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) } MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, - const TransactionID & tid_, const WriteSettings & settings, MutationType type_) + const TransactionID & tid_, const WriteSettings & settings) : create_time(time(nullptr)) , commands(std::move(commands_)) , disk(std::move(disk_)) @@ -54,13 +54,11 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP , file_name("tmp_mutation_" + toString(tmp_number) + ".txt") , is_temp(true) , tid(tid_) - , type(type_) { try { auto out = disk->writeFile(std::filesystem::path(path_prefix) / file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, settings); - *out << "format version: 2\n" - << "type: " << type << "\n" + *out << "format version: 1\n" << "create time: " << LocalDateTime(create_time) << "\n"; *out << "commands: "; commands.writeText(*out); @@ -123,25 +121,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat block_number = parseFileName(file_name); auto buf = disk->readFile(path_prefix + file_name); - int format_version; - *buf >> "format version: " >> format_version >> "\n"; - - /// Allow format_version = 1 for backward compatibility. - if (format_version != 1 && format_version != 2) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported format version in mutation.txt, expected '1' or '2', got '{}'", format_version); - - type = MutationType::Ordinary; - if (format_version == 2) - { - String type_str; - *buf >> "type: " >> type_str >> "\n"; - - auto type_value = magic_enum::enum_cast(type_str); - if (type_value.has_value()) - type = type_value.value(); - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported mutation type in mutation.txt, expected 'Lightweight' or 'Ordinary', got '{}'", type_str); - } + *buf >> "format version: 1\n"; LocalDateTime create_time_dt; *buf >> "create time: " >> create_time_dt >> "\n"; diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 37dbca9de7b..04297f2852a 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -11,11 +11,6 @@ namespace DB { class IBackupEntry; -/// Type of Mutate. Used to control different mutates during mutates -/// assignment. Also allows to apply special logic during mutate process -/// Stored in FutureMergedMutatedPart and MergeTreeMutationEntry. -enum class MutationType { Ordinary, Lightweight }; - /// A mutation entry for non-replicated MergeTree storage engines. /// Stores information about mutation in file mutation_*.txt. struct MergeTreeMutationEntry @@ -41,12 +36,9 @@ struct MergeTreeMutationEntry /// or UnknownCSN if it's not committed (yet) or RolledBackCSN if it's rolled back or PrehistoricCSN if there is no transaction. CSN csn = Tx::UnknownCSN; - /// Type of mutation, used for lightweight delete. - MutationType type; - /// Create a new entry and write it to a temporary file. MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number, - const TransactionID & tid_, const WriteSettings & settings, MutationType type_); + const TransactionID & tid_, const WriteSettings & settings); MergeTreeMutationEntry(const MergeTreeMutationEntry &) = delete; MergeTreeMutationEntry(MergeTreeMutationEntry &&) = default; diff --git a/src/Storages/MergeTree/MergeTreeMutationStatus.h b/src/Storages/MergeTree/MergeTreeMutationStatus.h index f0949047f6e..acda43b9254 100644 --- a/src/Storages/MergeTree/MergeTreeMutationStatus.h +++ b/src/Storages/MergeTree/MergeTreeMutationStatus.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB @@ -14,7 +13,6 @@ namespace DB struct MergeTreeMutationStatus { - MutationType type; String id; String command; time_t create_time = 0; diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index cbc409af4e8..cb2ead8a025 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -321,7 +321,6 @@ void MergeTreeRangeReader::ReadResult::clear() total_rows_per_granule = 0; filter_holder = nullptr; filter = nullptr; - deleted_mask_filter_holder = nullptr; } void MergeTreeRangeReader::ReadResult::shrink(Columns & old_columns) @@ -666,7 +665,7 @@ MergeTreeRangeReader::MergeTreeRangeReader( for (const auto & name_and_type : merge_tree_reader->getColumns()) sample_block.insert({name_and_type.type->createColumn(), name_and_type.type, name_and_type.name}); - + for (const auto & column_name : non_const_virtual_column_names_) { if (sample_block.has(column_name)) @@ -681,9 +680,6 @@ MergeTreeRangeReader::MergeTreeRangeReader( // sample_block.insert(ColumnWithTypeAndName(ColumnUInt8::create(), std::make_shared(), column_name)); } - if (merge_tree_reader->needReadDeletedMask()) - need_apply_deleted_mask = merge_tree_reader->data_part->getDeletedMask(deleted_mask); - if (prewhere_info) { const auto & step = *prewhere_info; @@ -863,15 +859,13 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar read_result = startReadingChain(max_rows, ranges); read_result.num_rows = read_result.numReadRows(); - executeDeletedRowMaskFilterColumns(read_result); - if (read_result.num_rows) { /// Physical columns go first and then some virtual columns follow size_t physical_columns_count = read_result.columns.size() - read_result.extra_columns_filled.size(); /////////// -// TODO: properly account for "virtual columns" that are overridden with real data in the part - +// TODO: properly account for "virtual columns" that are overridden with real data in the part + ///////////// Columns physical_columns(read_result.columns.begin(), read_result.columns.begin() + physical_columns_count); @@ -968,10 +962,6 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t fillPartOffsetColumn(result, leading_begin_part_offset, leading_end_part_offset); } - /// Do similar as part_offset for deleted mask. - if (need_apply_deleted_mask) - fillDeletedRowMaskColumn(result, leading_begin_part_offset, leading_end_part_offset); - return result; } @@ -1003,47 +993,6 @@ void MergeTreeRangeReader::fillPartOffsetColumn(ReadResult & result, UInt64 lead result.extra_columns_filled.push_back("_part_offset"); } -/// Fill deleted_row_mask column, referenced from fillPartOffsetColumn(). -void MergeTreeRangeReader::fillDeletedRowMaskColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset) -{ - size_t num_rows = result.numReadRows(); - - auto mask_column = ColumnUInt8::create(num_rows); - ColumnUInt8::Container & vec = mask_column->getData(); - - UInt8 * pos = vec.data(); - UInt8 * end = &vec[num_rows]; - - const auto & deleted_rows_col = deleted_mask.getDeletedRows(); - const ColumnUInt8::Container & deleted_rows_mask = deleted_rows_col.getData(); - - while (pos < end && leading_begin_part_offset < leading_end_part_offset) - { - if (deleted_rows_mask[leading_begin_part_offset++]) - *pos++ = 0; - else - *pos++ = 1; - } - - const auto start_ranges = result.startedRanges(); - - for (const auto & start_range : start_ranges) - { - UInt64 start_part_offset = index_granularity->getMarkStartingRow(start_range.range.begin); - UInt64 end_part_offset = index_granularity->getMarkStartingRow(start_range.range.end); - - while (pos < end && start_part_offset < end_part_offset) - { - if (deleted_rows_mask[start_part_offset++]) - *pos++ = 0; - else - *pos++ = 1; - } - } - - result.deleted_mask_filter_holder = std::move(mask_column); -} - Columns MergeTreeRangeReader::continueReadingChain(const ReadResult & result, size_t & num_rows) { Columns columns; @@ -1158,36 +1107,6 @@ static ColumnPtr combineFilters(ColumnPtr first, ColumnPtr second) return mut_first; } - -/// Implicitly apply deleted mask filter to columns. -/// If there is no prewhere_info, apply directly the deleted mask filter. -/// If prewhere_info exists, only apply to the first prewhere filter. -void MergeTreeRangeReader::executeDeletedRowMaskFilterColumns(ReadResult & result) -{ - if (prewhere_info || !need_apply_deleted_mask || !result.deleted_mask_filter_holder) - return; - - const ColumnUInt8 * mask_filter = typeid_cast(result.deleted_mask_filter_holder.get()); - filterColumns(result.columns, mask_filter->getData()); - - bool has_column = false; - for (auto & column : result.columns) - { - if (column) - { - has_column = true; - result.num_rows = column->size(); - break; - } - } - - /// There is only one filter column. Record the actual number. - if (!has_column) - result.num_rows = result.countBytesInResultFilter(mask_filter->getData()); - - result.need_filter = true; -} - void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result) { if (!prewhere_info) @@ -1243,7 +1162,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r else if (column_name == "_row_exists") { /// do nothing, it will be added later - /// TODO: properly implement reading non-const virtual columns or filling them with default values + /// TODO: properly implement reading non-const virtual columns or filling them with default values } else throw Exception("Unexpected non-const virtual column: " + column_name, ErrorCodes::LOGICAL_ERROR); @@ -1253,19 +1172,6 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r /// Columns might be projected out. We need to store them here so that default columns can be evaluated later. result.block_before_prewhere = block; - /// Apply deleted mask filter for the first prewhere step - if (!result.getFilter() && result.deleted_mask_filter_holder) - { - auto columns = block.getColumns(); - filterColumns(columns, result.deleted_mask_filter_holder); - if (columns.empty()) - block = block.cloneEmpty(); - else - block.setColumns(columns); - - result.setFilter(result.deleted_mask_filter_holder); - } - if (prewhere_info->actions) prewhere_info->actions->execute(block); diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index ba71f1898f6..06f3f5760fb 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -2,7 +2,6 @@ #include #include #include -#include namespace DB { @@ -242,9 +241,6 @@ public: std::map filter_bytes_map; - /// Similar as filter that you need to apply to newly-read columns - ColumnPtr deleted_mask_filter_holder; - Names extra_columns_filled; }; @@ -257,8 +253,6 @@ private: Columns continueReadingChain(const ReadResult & result, size_t & num_rows); void executePrewhereActionsAndFilterColumns(ReadResult & result); void fillPartOffsetColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset); - void fillDeletedRowMaskColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset); - void executeDeletedRowMaskFilterColumns(ReadResult & result); IMergeTreeReader * merge_tree_reader = nullptr; const MergeTreeIndexGranularity * index_granularity = nullptr; @@ -272,8 +266,6 @@ private: bool last_reader_in_chain = false; bool is_initialized = false; Names non_const_virtual_column_names; - bool need_apply_deleted_mask = false; - MergeTreeDataPartDeletedMask deleted_mask; }; } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 53a56bad97e..de48b96edd6 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -46,7 +46,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( NamesAndTypesList columns_for_reader; if (take_column_types_from_storage) { - auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects().withVirtuals(); + auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects().withVirtuals(); /// TODO: only add _rows_exists column (if it's present on disk) columns_for_reader = storage_snapshot->getColumnsByNames(options, columns_to_read); } else @@ -68,9 +68,6 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( reader = data_part->getReader(columns_for_reader, storage_snapshot->metadata, MarkRanges{MarkRange(0, data_part->getMarksCount())}, /* uncompressed_cache = */ nullptr, mark_cache.get(), reader_settings, {}, {}); - - if (data_part->hasLightweightDelete()) - need_apply_deleted_mask = data_part->getDeletedMask(deleted_mask); } Chunk MergeTreeSequentialSource::generate() @@ -78,91 +75,53 @@ try { const auto & header = getPort().getHeader(); - /// The chunk after deleted mask applied maybe empty. But the empty chunk means done of read rows. - do + if (!isCancelled() && current_row < data_part->rows_count) { - if (!isCancelled() && current_row < data_part->rows_count) + size_t rows_to_read = data_part->index_granularity.getMarkRows(current_mark); + bool continue_reading = (current_mark != 0); + + const auto & sample = reader->getColumns(); + Columns columns(sample.size()); + size_t rows_read = reader->readRows(current_mark, data_part->getMarksCount(), continue_reading, rows_to_read, columns); + + if (rows_read) { - size_t rows_to_read = data_part->index_granularity.getMarkRows(current_mark); - bool continue_reading = (current_mark != 0); + current_row += rows_read; + current_mark += (rows_to_read == rows_read); - const auto & sample = reader->getColumns(); - Columns columns(sample.size()); - size_t rows_read = reader->readRows(current_mark, data_part->getMarksCount(), continue_reading, rows_to_read, columns); + bool should_evaluate_missing_defaults = false; + reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_read); - if (rows_read) + if (should_evaluate_missing_defaults) { - current_row += rows_read; - current_mark += (rows_to_read == rows_read); - - if (need_apply_deleted_mask) - { - const auto & deleted_rows_col = deleted_mask.getDeletedRows(); - const ColumnUInt8::Container & deleted_rows_mask = deleted_rows_col.getData(); - - size_t pos = current_row - rows_read; - - /// Get deleted mask for rows_read - IColumn::Filter deleted_rows_filter(rows_read, true); - for (size_t i = 0; i < rows_read; i++) - { - if (deleted_rows_mask[pos++]) - deleted_rows_filter[i] = 0; - } - - // Filter only if some items were deleted - if (auto num_deleted_rows = std::count(deleted_rows_filter.begin(), deleted_rows_filter.end(), 0)) - { - const auto remaining_rows = deleted_rows_filter.size() - num_deleted_rows; - - /// If we return {} here, it means finished, no reading of the following rows. - /// Continue to read until remaining rows are not zero or reach the end (REAL finish). - if (!remaining_rows) - continue; - - for (auto & col : columns) - col = col->filter(deleted_rows_filter, remaining_rows); - - /// Update rows_read with actual rows in columns - rows_read = remaining_rows; - } - } - - bool should_evaluate_missing_defaults = false; - reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_read); - - if (should_evaluate_missing_defaults) - { - reader->evaluateMissingDefaults({}, columns); - } - - reader->performRequiredConversions(columns); - - /// Reorder columns and fill result block. - size_t num_columns = sample.size(); - Columns res_columns; - res_columns.reserve(num_columns); - - auto it = sample.begin(); - for (size_t i = 0; i < num_columns; ++i) - { - if (header.has(it->name)) - res_columns.emplace_back(std::move(columns[i])); - - ++it; - } - - return Chunk(std::move(res_columns), rows_read); + reader->evaluateMissingDefaults({}, columns); } - } - else - { - finish(); - } - return {}; - } while (true); + reader->performRequiredConversions(columns); + /// Reorder columns and fill result block. + size_t num_columns = sample.size(); + Columns res_columns; + res_columns.reserve(num_columns); + + auto it = sample.begin(); + for (size_t i = 0; i < num_columns; ++i) + { + if (header.has(it->name)) + res_columns.emplace_back(std::move(columns[i])); + + ++it; + } + + return Chunk(std::move(res_columns), rows_read); + } + } + else + { + finish(); + } + + return {}; } catch (...) { diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index 5a571825db5..a3e4f5fa856 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -3,7 +3,6 @@ #include #include #include -#include #include namespace DB @@ -59,9 +58,6 @@ private: /// current row at which we stop reading size_t current_row = 0; - bool need_apply_deleted_mask = false; - MergeTreeDataPartDeletedMask deleted_mask {}; - /// Closes readers and unlock part locks void finish(); }; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index ae64b08e351..0e61f499202 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -173,7 +173,7 @@ getColumnsForNewDataPart( const auto all_virtual_columns = source_part->storage.getVirtuals(); /// Preserve virtual columns that have persisted values in the source_part -/// TODO: only allow LWD mask to be overriden!!!!! +/// TODO: only allow LWD mask to be overridden! for (const auto & virtual_column : all_virtual_columns) { if (part_columns.has(virtual_column.name) && !storage_columns.contains(virtual_column.name)) @@ -188,7 +188,7 @@ getColumnsForNewDataPart( for (const auto & [column_name, _] : command.column_to_update_expression) { /// Allow to update and persist values of virtual column -/// TODO: only allow LWD mask to be overriden!!!!! +/// TODO: only allow LWD mask to be overridden! auto virtual_column = all_virtual_columns.tryGetByName(column_name); if (virtual_column && !storage_columns.contains(column_name)) storage_columns.emplace_back(column_name, virtual_column->type); @@ -1379,206 +1379,6 @@ private: std::unique_ptr part_merger_writer_task{nullptr}; }; -/// LightweightDeleteTask works for lightweight delete mutate. -/// The MutationsInterpreter returns a simple select like "select _part_offset where predicates". -/// The prepare() and execute() has special logics for LWD mutate. -class LightweightDeleteTask : public IExecutableTask -{ -public: - - explicit LightweightDeleteTask(MutationContextPtr ctx_) : ctx(ctx_) {} - - void onCompleted() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - StorageID getStorageID() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - UInt64 getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - - bool executeStep() override - { - switch (state) - { - case State::NEED_PREPARE: - { - prepare(); - - state = State::NEED_EXECUTE; - return true; - } - case State::NEED_EXECUTE: - { - execute(); - - state = State::NEED_FINALIZE; - return true; - } - case State::NEED_FINALIZE: - { - finalize(); - - state = State::SUCCESS; - return true; - } - case State::SUCCESS: - { - return false; - } - } - return false; - } - -private: - - void prepare() - { - if (ctx->execute_ttl_type != ExecuteTTLType::NONE) - ctx->files_to_skip.insert("ttl.txt"); - - ctx->data_part_storage_builder->createDirectories(); - - /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. - TransactionID tid = ctx->txn ? ctx->txn->tid : Tx::PrehistoricTID; - /// NOTE do not pass context for writing to system.transactions_info_log, - /// because part may have temporary name (with temporary block numbers). Will write it later. - ctx->new_data_part->version.setCreationTID(tid, nullptr); - ctx->new_data_part->storeVersionMetadata(); - - NameSet hardlinked_files; - /// Create hardlinks for unchanged files - for (auto it = ctx->source_part->data_part_storage->iterate(); it->isValid(); it->next()) - { - if (ctx->files_to_skip.contains(it->name())) - continue; - - String destination; - destination = it->name(); - - if (it->isFile()) - { - ctx->data_part_storage_builder->createHardLinkFrom( - *ctx->source_part->data_part_storage, it->name(), destination); - hardlinked_files.insert(it->name()); - } - else if (!endsWith(".tmp_proj", it->name())) // ignore projection tmp merge dir - { - // it's a projection part directory - ctx->data_part_storage_builder->createProjection(destination); - - auto projection_data_part_storage = ctx->source_part->data_part_storage->getProjection(destination); - auto projection_data_part_storage_builder = ctx->data_part_storage_builder->getProjection(destination); - - for (auto p_it = projection_data_part_storage->iterate(); p_it->isValid(); p_it->next()) - { - projection_data_part_storage_builder->createHardLinkFrom( - *projection_data_part_storage, p_it->name(), p_it->name()); - hardlinked_files.insert(p_it->name()); - } - } - } - - /// Tracking of hardlinked files required for zero-copy replication. - /// We don't remove them when we delete last copy of source part because - /// new part can use them. - ctx->hardlinked_files.source_table_shared_id = ctx->source_part->storage.getTableSharedID(); - ctx->hardlinked_files.source_part_name = ctx->source_part->name; - ctx->hardlinked_files.hardlinks_from_source_part = hardlinked_files; - - /// Only the _delete mask column will be written. - (*ctx->mutate_entry)->columns_written = 1; - - ctx->new_data_part->checksums = ctx->source_part->checksums; - - ctx->compression_codec = ctx->source_part->default_codec; - - if (ctx->mutating_pipeline_builder.initialized()) - { - QueryPipelineBuilder builder(std::move(ctx->mutating_pipeline_builder)); - - if (ctx->execute_ttl_type == ExecuteTTLType::NORMAL) - builder.addTransform(std::make_shared(builder.getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true)); - - if (ctx->execute_ttl_type == ExecuteTTLType::RECALCULATE) - builder.addTransform(std::make_shared(builder.getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true)); - - ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); - ctx->mutating_pipeline.setProgressCallback(ctx->progress_callback); - /// Is calculated inside MergeProgressCallback. - ctx->mutating_pipeline.disableProfileEventUpdate(); - ctx->mutating_executor = std::make_unique(ctx->mutating_pipeline); - } - } - - void execute() - { - Block block; - bool has_deleted_rows = false; - - auto new_deleted_rows = ColumnUInt8::create(); - auto & data = new_deleted_rows->getData(); - - /// If this part has already applied lightweight mutation, load the past latest bitmap to merge with current bitmap - if (ctx->source_part->hasLightweightDelete()) - { - MergeTreeDataPartDeletedMask deleted_mask {}; - if (ctx->source_part->getDeletedMask(deleted_mask)) - { - const auto & deleted_rows_col = deleted_mask.getDeletedRows(); - const auto & source_data = deleted_rows_col.getData(); - data.insert(source_data.begin(), source_data.begin() + ctx->source_part->rows_count); - - has_deleted_rows = true; - } - } - - if (!has_deleted_rows) - new_deleted_rows->insertManyDefaults(ctx->source_part->rows_count); - - /// Mark the data corresponding to the offset in the as deleted. - while (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor && ctx->mutating_executor->pull(block)) - { - size_t block_rows = block.rows(); - - if (block_rows && !has_deleted_rows) - has_deleted_rows = true; - - const auto & cols = block.getColumns(); - const auto * offset_col = typeid_cast(cols[0].get()); - const UInt64 * offset = offset_col->getData().data(); - - /// Fill 1 for rows in offset - for (size_t current_row = 0; current_row < block_rows; current_row++) - data[offset[current_row]] = 1; - } - - if (has_deleted_rows) - { - ctx->new_data_part->writeDeletedMask(ColumnUInt8::Ptr(std::move(new_deleted_rows))); - } - } - - void finalize() - { - if (ctx->mutating_executor) - { - ctx->mutating_executor.reset(); - ctx->mutating_pipeline.reset(); - } - - MutationHelpers::finalizeMutatedPart(ctx->source_part, ctx->data_part_storage_builder, ctx->new_data_part, ctx->execute_ttl_type, ctx->compression_codec, ctx->context); - } - - enum class State - { - NEED_PREPARE, - NEED_EXECUTE, - NEED_FINALIZE, - - SUCCESS - }; - - State state{State::NEED_PREPARE}; - - MutationContextPtr ctx; -}; - MutateTask::MutateTask( FutureMergedMutatedPartPtr future_part_, @@ -1611,13 +1411,6 @@ MutateTask::MutateTask( ctx->source_part = ctx->future_part->parts[0]; ctx->storage_from_source_part = std::make_shared(ctx->source_part); - /// part is checked for lightweight delete in selectPartsToMutate(). - ctx->is_lightweight_mutation = ctx->future_part->mutation_type == MutationType::Lightweight; - - /// Empty mutation commands mean that the mutation is killed. Just work as ordinary, clone the part. - if (ctx->commands->empty()) - ctx->is_lightweight_mutation = false; - auto storage_snapshot = ctx->storage_from_source_part->getStorageSnapshot(ctx->metadata_snapshot, context_); extendObjectColumns(ctx->storage_columns, storage_snapshot->object_columns, /*with_subcolumns=*/ false); } @@ -1673,7 +1466,7 @@ bool MutateTask::prepare() ctx->commands_for_part.emplace_back(command); } - if (ctx->source_part->isStoredOnDisk() && !ctx->is_lightweight_mutation && !isStorageTouchedByMutations( + if (ctx->source_part->isStoredOnDisk() && !isStorageTouchedByMutations( ctx->storage_from_source_part, ctx->metadata_snapshot, ctx->commands_for_part, Context::createCopy(context_for_reading))) { LOG_TRACE(ctx->log, "Part {} doesn't change up to mutation version {}", ctx->source_part->name, ctx->future_part->part_info.mutation); @@ -1689,25 +1482,15 @@ bool MutateTask::prepare() ctx->stage_progress = std::make_unique(1.0); - bool need_mutate_all_columns = !isWidePart(ctx->source_part); - if (!ctx->for_interpreter.empty()) { ctx->interpreter = std::make_unique( - ctx->storage_from_source_part, ctx->metadata_snapshot, ctx->for_interpreter, context_for_reading, true, ctx->is_lightweight_mutation); + ctx->storage_from_source_part, ctx->metadata_snapshot, ctx->for_interpreter, context_for_reading, true); ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices(); ctx->materialized_projections = ctx->interpreter->grabMaterializedProjections(); ctx->mutation_kind = ctx->interpreter->getMutationKind(); - - /// Skip to apply deleted mask when reading for MutateSomePartColumns. - need_mutate_all_columns = need_mutate_all_columns || (ctx->mutation_kind == MutationsInterpreter::MutationKind::MUTATE_OTHER && ctx->interpreter->isAffectingAllColumns()); - if (!need_mutate_all_columns && ctx->source_part->hasLightweightDelete() && !ctx->is_lightweight_mutation) - ctx->interpreter->setSkipDeletedMask(true); - -///// - ctx->interpreter->setSkipDeletedMask(true); -///// - + /// Always disable filtering in mutations, we want to read all rows + ctx->interpreter->setApplyDeletedMask(false); ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); ctx->progress_callback = MergeProgressCallback((*ctx->mutate_entry)->ptr(), ctx->watch_prev_elapsed, *ctx->stage_progress); @@ -1737,21 +1520,12 @@ bool MutateTask::prepare() /// It shouldn't be changed by mutation. ctx->new_data_part->index_granularity_info = ctx->source_part->index_granularity_info; - if (ctx->is_lightweight_mutation) - { - /// The metadata alter will update the metadata snapshot, we should use same as source part. - ctx->new_data_part->setColumns(ctx->source_part->getColumns()); - ctx->new_data_part->setSerializationInfos(ctx->source_part->getSerializationInfos()); - } - else - { - auto [new_columns, new_infos] = MutationHelpers::getColumnsForNewDataPart( - ctx->source_part, ctx->updated_header, ctx->storage_columns, - ctx->source_part->getSerializationInfos(), ctx->commands_for_part); + auto [new_columns, new_infos] = MutationHelpers::getColumnsForNewDataPart( + ctx->source_part, ctx->updated_header, ctx->storage_columns, + ctx->source_part->getSerializationInfos(), ctx->commands_for_part); - ctx->new_data_part->setColumns(new_columns); - ctx->new_data_part->setSerializationInfos(new_infos); - } + ctx->new_data_part->setColumns(new_columns); + ctx->new_data_part->setSerializationInfos(new_infos); ctx->new_data_part->partition.assign(ctx->source_part->partition); @@ -1768,17 +1542,11 @@ bool MutateTask::prepare() /// All columns from part are changed and may be some more that were missing before in part /// TODO We can materialize compact part without copying data - if (need_mutate_all_columns) + if (!isWidePart(ctx->source_part) + || (ctx->mutation_kind == MutationsInterpreter::MutationKind::MUTATE_OTHER && ctx->interpreter && ctx->interpreter->isAffectingAllColumns())) { task = std::make_unique(ctx); } - else if (ctx->is_lightweight_mutation) - { - ctx->files_to_skip = ctx->source_part->getFileNamesWithoutChecksums(); - - /// We will modify or create only deleted_row_mask for lightweight delete. Other columns and key values are copied as-is. - task = std::make_unique(ctx); - } else /// TODO: check that we modify only non-key columns in this case. { /// We will modify only some of the columns. Other columns and key values can be copied as-is. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index cd31d356b4b..f6c80baba05 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1920,7 +1920,6 @@ std::vector ReplicatedMergeTreeQueue::getMutationsStatu formatAST(*command.ast, buf, false, true); result.push_back(MergeTreeMutationStatus { - MutationType::Ordinary, /// TODO: ReplicatedMergeTree supports lightweight delete. entry.znode_name, buf.str(), entry.create_time, diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index b9b5874b3e6..b2f62c2bf02 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -106,6 +106,11 @@ public: return parts.front()->storage.getSettings()->materialize_ttl_recalculate_only; } + bool hasLightweightDeleteColumn() const + { + return parts.front()->getColumns().contains("_row_exists"); // TODO: fix hardcoded column name + } + private: MergeTreeData::DataPartsVector parts; const MergeTreeData & storage; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 0604bb304d0..5e161fc2e6a 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -123,7 +123,7 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) // New scope controls lifetime of pipeline. { auto storage_ptr = DatabaseCatalog::instance().getTable(getStorageID(), context); - auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context, true, false); + auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context, true); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index fc4a671c071..f3f1162287f 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -318,7 +318,7 @@ void StorageMemory::mutate(const MutationCommands & commands, ContextPtr context new_context->setSetting("max_streams_to_max_threads_ratio", 1); new_context->setSetting("max_threads", 1); - auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, new_context, true, false); + auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, new_context, true); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index bf731f09428..34fcd4c7a78 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -332,7 +332,7 @@ void StorageMergeTree::alter( DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata); if (!maybe_mutation_commands.empty()) - mutation_version = startMutation(maybe_mutation_commands, local_context, MutationType::Ordinary); + mutation_version = startMutation(maybe_mutation_commands, local_context); } /// Always execute required mutations synchronously, because alters @@ -429,7 +429,7 @@ CurrentlyMergingPartsTagger::~CurrentlyMergingPartsTagger() storage.currently_processing_in_background_condition.notify_all(); } -Int64 StorageMergeTree::startMutation(const MutationCommands & commands, ContextPtr query_context, MutationType type) +Int64 StorageMergeTree::startMutation(const MutationCommands & commands, ContextPtr query_context) { /// Choose any disk, because when we load mutations we search them at each disk /// where storage can be placed. See loadMutations(). @@ -447,7 +447,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context { std::lock_guard lock(currently_processing_in_background_mutex); - MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings(), type); + MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings()); version = increment.get(); entry.commit(version); String mutation_id = entry.file_name; @@ -554,21 +554,11 @@ void StorageMergeTree::setMutationCSN(const String & mutation_id, CSN csn) } void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr query_context) -{ - /// Make ordinary ALTER DELETE queries lightweight to check all tests. - if (query_context->getSettingsRef().lightweight_delete_mutation - && commands.size() == 1 && commands.begin()->type == MutationCommand::DELETE) - mutate(commands, query_context, MutationType::Lightweight); - else - mutate(commands, query_context, MutationType::Ordinary); -} - -void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr query_context, MutationType type) { /// Validate partition IDs (if any) before starting mutation getPartitionIdsAffectedByCommands(commands, query_context); - Int64 version = startMutation(commands, query_context, type); + Int64 version = startMutation(commands, query_context); if (query_context->getSettingsRef().mutations_sync > 0 || query_context->getCurrentTransaction()) waitForMutation(version); @@ -667,7 +657,6 @@ std::vector StorageMergeTree::getMutationsStatus() cons formatAST(*command.ast, buf, false, true); result.push_back(MergeTreeMutationStatus { - entry.type, entry.file_name, buf.str(), entry.create_time, @@ -1034,20 +1023,12 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( auto commands = std::make_shared(); size_t current_ast_elements = 0; auto last_mutation_to_apply = mutations_end_it; - - bool support_lightweight_mutate = part->supportLightweightDeleteMutate(); - MutationType first_mutation_type = support_lightweight_mutate ? mutations_begin_it->second.type : MutationType::Ordinary; for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { /// Do not squash mutations from different transactions to be able to commit/rollback them independently. if (first_mutation_tid != it->second.tid) break; - /// Do not combine mutations with different types. - /// TODO: compact part support lightweight delete. - if (support_lightweight_mutate && it->second.type != first_mutation_type) - break; - size_t commands_size = 0; MutationCommands commands_for_size_validation; for (const auto & command : it->second.commands) @@ -1073,7 +1054,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( fake_query_context->makeQueryContext(); fake_query_context->setCurrentQueryId(""); MutationsInterpreter interpreter( - shared_from_this(), metadata_snapshot, commands_for_size_validation, fake_query_context, false, false); + shared_from_this(), metadata_snapshot, commands_for_size_validation, fake_query_context, false); commands_size += interpreter.evaluateCommandsSize(); } catch (...) @@ -1132,7 +1113,6 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( future_part->part_info = new_part_info; future_part->name = part->getNewName(new_part_info); future_part->type = part->getType(); - future_part->mutation_type = first_mutation_type; tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); return std::make_shared(future_part, std::move(tagger), commands, txn); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 379e15d5bdf..632884db033 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -87,8 +87,6 @@ public: void mutate(const MutationCommands & commands, ContextPtr context) override; - /// Support lightweight delete. - void mutate(const MutationCommands & commands, ContextPtr context, MutationType type); bool hasLightweightDeletedMask() const override; /// Return introspection information about currently processing or recently processed mutations. @@ -184,7 +182,7 @@ private: /// Allocate block number for new mutation, write mutation to disk /// and into in-memory structures. Wake up merge-mutation task. - Int64 startMutation(const MutationCommands & commands, ContextPtr query_context, MutationType type); + Int64 startMutation(const MutationCommands & commands, ContextPtr query_context); /// Wait until mutation with version will finish mutation for all parts void waitForMutation(Int64 version); void waitForMutation(const String & mutation_id) override; diff --git a/src/Storages/System/StorageSystemMutations.cpp b/src/Storages/System/StorageSystemMutations.cpp index 907376a4936..fa521c632b8 100644 --- a/src/Storages/System/StorageSystemMutations.cpp +++ b/src/Storages/System/StorageSystemMutations.cpp @@ -20,7 +20,6 @@ NamesAndTypesList StorageSystemMutations::getNamesAndTypes() return { { "database", std::make_shared() }, { "table", std::make_shared() }, - { "is_lightweight", std::make_shared() }, { "mutation_id", std::make_shared() }, { "command", std::make_shared() }, { "create_time", std::make_shared() }, @@ -131,7 +130,6 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, ContextPtr c res_columns[col_num++]->insert(database); res_columns[col_num++]->insert(table); - res_columns[col_num++]->insert(status.type == MutationType::Lightweight); res_columns[col_num++]->insert(status.id); res_columns[col_num++]->insert(status.command); res_columns[col_num++]->insert(UInt64(status.create_time)); diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index acc9b08da29..6e9d9188962 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -378,7 +378,6 @@ CREATE TABLE system.mutations ( `database` String, `table` String, - `is_lightweight` UInt8, `mutation_id` String, `command` String, `create_time` DateTime, diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference index fc646843eee..31960e2ecea 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference @@ -8,11 +8,11 @@ 1 1 1 -0 UPDATE _row_exists = 0 WHERE (c % 5) = 1 1 -0 UPDATE _row_exists = 0 WHERE c = 4 1 -0 MATERIALIZE INDEX i_c 1 -0 UPDATE b = -1 WHERE a < 3 1 -0 DROP INDEX i_c 1 +UPDATE _row_exists = 0 WHERE (c % 5) = 1 1 +UPDATE _row_exists = 0 WHERE c = 4 1 +MATERIALIZE INDEX i_c 1 +UPDATE b = -1 WHERE a < 3 1 +DROP INDEX i_c 1 -----Check that select and merge with lightweight delete.----- 7 0 -1 0 diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql index 24afa5fb196..6f78e1fe464 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql @@ -5,7 +5,7 @@ CREATE TABLE merge_table_standard_delete(id Int32, name String) ENGINE = MergeTr INSERT INTO merge_table_standard_delete select number, toString(number) from numbers(100); SET mutations_sync = 1; -SET allow_experimental_lightweight_delete = 0; +--SET allow_experimental_lightweight_delete = 0; SET allow_experimental_lightweight_delete_with_row_exists = 1; DELETE FROM merge_table_standard_delete WHERE id = 10; @@ -60,7 +60,7 @@ DETACH TABLE t_light; ATTACH TABLE t_light; CHECK TABLE t_light; -SELECT is_lightweight, command, is_done FROM system.mutations WHERE database = currentDatabase() AND table = 't_light'; +SELECT command, is_done FROM system.mutations WHERE database = currentDatabase() AND table = 't_light'; SELECT '-----Check that select and merge with lightweight delete.-----'; select count(*) from t_light; diff --git a/tests/queries/0_stateless/02352_lightweight_delete.sql b/tests/queries/0_stateless/02352_lightweight_delete.sql index 46336a57c3a..4468a25448c 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete.sql +++ b/tests/queries/0_stateless/02352_lightweight_delete.sql @@ -6,7 +6,7 @@ INSERT INTO lwd_test SELECT number, randomString(10) FROM system.numbers LIMIT 1 SET mutations_sync = 1; SET allow_experimental_lightweight_delete_with_row_exists = 1; -SET allow_experimental_lightweight_delete = 0; +--SET allow_experimental_lightweight_delete = 0; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; SELECT 'Count', count() FROM lwd_test; From dc392cd4d34b25122f951418d9c7e187701cf6c7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 20 Jul 2022 21:44:51 +0200 Subject: [PATCH 487/659] Improve synchronization between hosts in distributed backup. Use ephemeral zk nodes to check other hosts for termination. --- src/Backups/BackupCoordinationLocal.cpp | 8 +- src/Backups/BackupCoordinationLocal.h | 8 +- src/Backups/BackupCoordinationRemote.cpp | 40 +- src/Backups/BackupCoordinationRemote.h | 17 +- src/Backups/BackupCoordinationStageSync.cpp | 228 ++++++ src/Backups/BackupCoordinationStageSync.h | 39 + src/Backups/BackupCoordinationStatusSync.cpp | 182 ----- src/Backups/BackupCoordinationStatusSync.h | 37 - src/Backups/BackupEntriesCollector.cpp | 70 +- src/Backups/BackupEntriesCollector.h | 7 +- src/Backups/BackupsWorker.cpp | 723 +++++++++++-------- src/Backups/BackupsWorker.h | 20 +- src/Backups/IBackupCoordination.h | 13 +- src/Backups/IRestoreCoordination.h | 13 +- src/Backups/RestoreCoordinationLocal.cpp | 8 +- src/Backups/RestoreCoordinationLocal.h | 10 +- src/Backups/RestoreCoordinationRemote.cpp | 39 +- src/Backups/RestoreCoordinationRemote.h | 21 +- src/Backups/RestorerFromBackup.cpp | 37 +- src/Backups/RestorerFromBackup.h | 5 +- src/Interpreters/InterpreterBackupQuery.cpp | 24 +- 21 files changed, 867 insertions(+), 682 deletions(-) create mode 100644 src/Backups/BackupCoordinationStageSync.cpp create mode 100644 src/Backups/BackupCoordinationStageSync.h delete mode 100644 src/Backups/BackupCoordinationStatusSync.cpp delete mode 100644 src/Backups/BackupCoordinationStatusSync.h diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index 417b84c6b5f..d4064902a40 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -13,20 +13,20 @@ using FileInfo = IBackupCoordination::FileInfo; BackupCoordinationLocal::BackupCoordinationLocal() = default; BackupCoordinationLocal::~BackupCoordinationLocal() = default; -void BackupCoordinationLocal::setStatus(const String &, const String &, const String &) +void BackupCoordinationLocal::setStage(const String &, const String &, const String &) { } -void BackupCoordinationLocal::setErrorStatus(const String &, const Exception &) +void BackupCoordinationLocal::setError(const String &, const Exception &) { } -Strings BackupCoordinationLocal::waitStatus(const Strings &, const String &) +Strings BackupCoordinationLocal::waitForStage(const Strings &, const String &) { return {}; } -Strings BackupCoordinationLocal::waitStatusFor(const Strings &, const String &, UInt64) +Strings BackupCoordinationLocal::waitForStage(const Strings &, const String &, std::chrono::milliseconds) { return {}; } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index 8cf5fba5c5c..aca7f71545b 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -20,10 +20,10 @@ public: BackupCoordinationLocal(); ~BackupCoordinationLocal() override; - void setStatus(const String & current_host, const String & new_status, const String & message) override; - void setErrorStatus(const String & current_host, const Exception & exception) override; - Strings waitStatus(const Strings & all_hosts, const String & status_to_wait) override; - Strings waitStatusFor(const Strings & all_hosts, const String & status_to_wait, UInt64 timeout_ms) override; + void setStage(const String & current_host, const String & new_stage, const String & message) override; + void setError(const String & current_host, const Exception & exception) override; + Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait) override; + Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) override; void addReplicatedPartNames(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) override; diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index a180358e088..bac99b0da2d 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -165,15 +165,28 @@ namespace constexpr size_t NUM_ATTEMPTS = 10; } -BackupCoordinationRemote::BackupCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_) +BackupCoordinationRemote::BackupCoordinationRemote( + const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool remove_zk_nodes_in_destructor_) : zookeeper_path(zookeeper_path_) , get_zookeeper(get_zookeeper_) - , status_sync(zookeeper_path_ + "/status", get_zookeeper_, &Poco::Logger::get("BackupCoordination")) + , remove_zk_nodes_in_destructor(remove_zk_nodes_in_destructor_) + , stage_sync(zookeeper_path_ + "/stage", get_zookeeper_, &Poco::Logger::get("BackupCoordination")) { createRootNodes(); } -BackupCoordinationRemote::~BackupCoordinationRemote() = default; +BackupCoordinationRemote::~BackupCoordinationRemote() +{ + try + { + if (remove_zk_nodes_in_destructor) + removeAllNodes(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} void BackupCoordinationRemote::createRootNodes() { @@ -196,24 +209,24 @@ void BackupCoordinationRemote::removeAllNodes() } -void BackupCoordinationRemote::setStatus(const String & current_host, const String & new_status, const String & message) +void BackupCoordinationRemote::setStage(const String & current_host, const String & new_stage, const String & message) { - status_sync.set(current_host, new_status, message); + stage_sync.set(current_host, new_stage, message); } -void BackupCoordinationRemote::setErrorStatus(const String & current_host, const Exception & exception) +void BackupCoordinationRemote::setError(const String & current_host, const Exception & exception) { - status_sync.setError(current_host, exception); + stage_sync.setError(current_host, exception); } -Strings BackupCoordinationRemote::waitStatus(const Strings & all_hosts, const String & status_to_wait) +Strings BackupCoordinationRemote::waitForStage(const Strings & all_hosts, const String & stage_to_wait) { - return status_sync.wait(all_hosts, status_to_wait); + return stage_sync.wait(all_hosts, stage_to_wait); } -Strings BackupCoordinationRemote::waitStatusFor(const Strings & all_hosts, const String & status_to_wait, UInt64 timeout_ms) +Strings BackupCoordinationRemote::waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) { - return status_sync.waitFor(all_hosts, status_to_wait, timeout_ms); + return stage_sync.waitFor(all_hosts, stage_to_wait, timeout); } @@ -565,9 +578,4 @@ Strings BackupCoordinationRemote::getAllArchiveSuffixes() const return node_names; } -void BackupCoordinationRemote::drop() -{ - removeAllNodes(); -} - } diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index 349d04c7d87..d1d206683fa 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB @@ -13,13 +13,13 @@ namespace DB class BackupCoordinationRemote : public IBackupCoordination { public: - BackupCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_); + BackupCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool remove_zk_nodes_in_destructor_); ~BackupCoordinationRemote() override; - void setStatus(const String & current_host, const String & new_status, const String & message) override; - void setErrorStatus(const String & current_host, const Exception & exception) override; - Strings waitStatus(const Strings & all_hosts, const String & status_to_wait) override; - Strings waitStatusFor(const Strings & all_hosts, const String & status_to_wait, UInt64 timeout_ms) override; + void setStage(const String & current_host, const String & new_stage, const String & message) override; + void setError(const String & current_host, const Exception & exception) override; + Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait) override; + Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) override; void addReplicatedPartNames( const String & table_shared_id, @@ -56,8 +56,6 @@ public: String getNextArchiveSuffix() override; Strings getAllArchiveSuffixes() const override; - void drop() override; - private: void createRootNodes(); void removeAllNodes(); @@ -66,8 +64,9 @@ private: const String zookeeper_path; const zkutil::GetZooKeeper get_zookeeper; + const bool remove_zk_nodes_in_destructor; - BackupCoordinationStatusSync status_sync; + BackupCoordinationStageSync stage_sync; mutable std::mutex mutex; mutable std::optional replicated_tables; diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp new file mode 100644 index 00000000000..5524029bbf2 --- /dev/null +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -0,0 +1,228 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FAILED_TO_SYNC_BACKUP_OR_RESTORE; +} + + +BackupCoordinationStageSync::BackupCoordinationStageSync(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, Poco::Logger * log_) + : zookeeper_path(zookeeper_path_) + , get_zookeeper(get_zookeeper_) + , log(log_) +{ + createRootNodes(); +} + +void BackupCoordinationStageSync::createRootNodes() +{ + auto zookeeper = get_zookeeper(); + zookeeper->createAncestors(zookeeper_path); + zookeeper->createIfNotExists(zookeeper_path, ""); +} + +void BackupCoordinationStageSync::set(const String & current_host, const String & new_stage, const String & message) +{ + auto zookeeper = get_zookeeper(); + + /// Make an ephemeral node so the initiator can track if the current host is still working. + String alive_node_path = zookeeper_path + "/alive|" + current_host; + auto code = zookeeper->tryCreate(alive_node_path, "", zkutil::CreateMode::Ephemeral); + if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNODEEXISTS) + throw zkutil::KeeperException(code, alive_node_path); + + zookeeper->createIfNotExists(zookeeper_path + "/started|" + current_host, ""); + zookeeper->createIfNotExists(zookeeper_path + "/current|" + current_host + "|" + new_stage, message); +} + +void BackupCoordinationStageSync::setError(const String & current_host, const Exception & exception) +{ + auto zookeeper = get_zookeeper(); + WriteBufferFromOwnString buf; + writeStringBinary(current_host, buf); + writeException(exception, buf, true); + zookeeper->createIfNotExists(zookeeper_path + "/error", buf.str()); +} + +Strings BackupCoordinationStageSync::wait(const Strings & all_hosts, const String & stage_to_wait) +{ + return waitImpl(all_hosts, stage_to_wait, {}); +} + +Strings BackupCoordinationStageSync::waitFor(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) +{ + return waitImpl(all_hosts, stage_to_wait, timeout); +} + +namespace +{ + struct UnreadyHostState + { + bool started = false; + bool alive = false; + }; +} + +struct BackupCoordinationStageSync::State +{ + Strings results; + std::map unready_hosts; + std::optional> error; + std::optional host_terminated; +}; + +BackupCoordinationStageSync::State BackupCoordinationStageSync::readCurrentState( + zkutil::ZooKeeperPtr zookeeper, const Strings & zk_nodes, const Strings & all_hosts, const String & stage_to_wait) const +{ + std::unordered_set zk_nodes_set{zk_nodes.begin(), zk_nodes.end()}; + + State state; + if (zk_nodes_set.contains("error")) + { + ReadBufferFromOwnString buf{zookeeper->get(zookeeper_path + "/error")}; + String host; + readStringBinary(host, buf); + state.error = std::make_pair(host, readException(buf, fmt::format("Got error from {}", host))); + return state; + } + + for (const auto & host : all_hosts) + { + if (!zk_nodes_set.contains("current|" + host + "|" + stage_to_wait)) + { + UnreadyHostState unready_host_state; + unready_host_state.started = zk_nodes_set.contains("started|" + host); + unready_host_state.alive = zk_nodes_set.contains("alive|" + host); + state.unready_hosts.emplace(host, unready_host_state); + if (!unready_host_state.alive && unready_host_state.started && !state.host_terminated) + state.host_terminated = host; + } + } + + if (state.host_terminated || !state.unready_hosts.empty()) + return state; + + state.results.reserve(all_hosts.size()); + for (const auto & host : all_hosts) + state.results.emplace_back(zookeeper->get(zookeeper_path + "/current|" + host + "|" + stage_to_wait)); + + return state; +} + +Strings BackupCoordinationStageSync::waitImpl(const Strings & all_hosts, const String & stage_to_wait, std::optional timeout) const +{ + if (all_hosts.empty()) + return {}; + + /// Wait until all hosts are ready or an error happens or time is out. + + auto zookeeper = get_zookeeper(); + + struct Watch + { + std::mutex mutex; + std::condition_variable event; + bool zk_nodes_changed = false; + bool watch_set = false; + }; + + /// shared_ptr because `watch_callback` can be called by ZooKeeper after leaving this function's scope. + auto watch = std::make_shared(); + + /// Called by ZooKepper when list of zk nodes have changed. + auto watch_callback = [watch](const Coordination::WatchResponse &) + { + std::lock_guard lock{watch->mutex}; + watch->zk_nodes_changed = true; + watch->watch_set = false; /// When it's triggered ZooKeeper resets the watch so we need to call getChildrenWatch() again. + watch->event.notify_all(); + }; + + auto zk_nodes_changed = [watch] { return watch->zk_nodes_changed; }; + + bool use_timeout = timeout.has_value(); + std::chrono::steady_clock::time_point end_of_timeout; + if (use_timeout) + end_of_timeout = std::chrono::steady_clock::now() + std::chrono::duration_cast(*timeout); + + State state; + + String previous_unready_host; /// Used for logging: we don't want to log the same unready host again. + + for (;;) + { + /// Get zk nodes and subscribe on their changes. + { + std::lock_guard lock{watch->mutex}; + watch->watch_set = true; + watch->zk_nodes_changed = false; + } + Strings zk_nodes = zookeeper->getChildrenWatch(zookeeper_path, nullptr, watch_callback); + + /// Read and analyze the current state of zk nodes. + state = readCurrentState(zookeeper, zk_nodes, all_hosts, stage_to_wait); + if (state.error || state.host_terminated || state.unready_hosts.empty()) + break; /// Error happened or everything is ready. + + /// Log that we will wait for another host. + const auto & unready_host = state.unready_hosts.begin()->first; + if (unready_host != previous_unready_host) + { + LOG_TRACE(log, "Waiting for host {}", unready_host); + previous_unready_host = unready_host; + } + + /// Wait until `watch_callback` is called by ZooKeeper meaning that zk nodes have changed. + { + std::unique_lock lock{watch->mutex}; + if (use_timeout) + { + auto current_time = std::chrono::steady_clock::now(); + if ((current_time > end_of_timeout) || !watch->event.wait_for(lock, end_of_timeout - current_time, zk_nodes_changed)) + break; + } + else + { + watch->event.wait(lock, zk_nodes_changed); + } + assert(watch->zk_nodes_changed); + assert(!watch->watch_set); + } + } + + /// Rethrow an error raised originally on another host. + if (state.error) + state.error->second.rethrow(); + + /// Another host terminated without errors. + if (state.host_terminated) + throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "Host {} suddenly stopped working", *state.host_terminated); + + /// Something's unready, timeout is probably not enough. + if (!state.unready_hosts.empty()) + { + const auto & [unready_host, unready_host_state] = *state.unready_hosts.begin(); + throw Exception( + ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, + "Waited for host {} too long (> {}){}", + unready_host, + to_string(*timeout), + unready_host_state.started ? "" : ": Operation didn't start"); + } + + return state.results; +} + +} diff --git a/src/Backups/BackupCoordinationStageSync.h b/src/Backups/BackupCoordinationStageSync.h new file mode 100644 index 00000000000..623b58fd9fa --- /dev/null +++ b/src/Backups/BackupCoordinationStageSync.h @@ -0,0 +1,39 @@ +#pragma once + +#include + + +namespace DB +{ + +/// Used to coordinate hosts so all hosts would come to a specific stage at around the same time. +class BackupCoordinationStageSync +{ +public: + BackupCoordinationStageSync(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, Poco::Logger * log_); + + /// Sets the stage of the current host and signal other hosts if there were other hosts waiting for that. + void set(const String & current_host, const String & new_stage, const String & message); + void setError(const String & current_host, const Exception & exception); + + /// Sets the stage of the current host and waits until all hosts come to the same stage. + /// The function returns the messages all hosts set when they come to the required stage. + Strings wait(const Strings & all_hosts, const String & stage_to_wait); + + /// Almost the same as setAndWait() but this one stops waiting and throws an exception after a specific amount of time. + Strings waitFor(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout); + +private: + void createRootNodes(); + + struct State; + State readCurrentState(zkutil::ZooKeeperPtr zookeeper, const Strings & zk_nodes, const Strings & all_hosts, const String & stage_to_wait) const; + + Strings waitImpl(const Strings & all_hosts, const String & stage_to_wait, std::optional timeout) const; + + String zookeeper_path; + zkutil::GetZooKeeper get_zookeeper; + Poco::Logger * log; +}; + +} diff --git a/src/Backups/BackupCoordinationStatusSync.cpp b/src/Backups/BackupCoordinationStatusSync.cpp deleted file mode 100644 index c0ecfdcaebe..00000000000 --- a/src/Backups/BackupCoordinationStatusSync.cpp +++ /dev/null @@ -1,182 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int FAILED_TO_SYNC_BACKUP_OR_RESTORE; -} - - -BackupCoordinationStatusSync::BackupCoordinationStatusSync(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, Poco::Logger * log_) - : zookeeper_path(zookeeper_path_) - , get_zookeeper(get_zookeeper_) - , log(log_) -{ - createRootNodes(); -} - -void BackupCoordinationStatusSync::createRootNodes() -{ - auto zookeeper = get_zookeeper(); - zookeeper->createAncestors(zookeeper_path); - zookeeper->createIfNotExists(zookeeper_path, ""); -} - -void BackupCoordinationStatusSync::set(const String & current_host, const String & new_status, const String & message) -{ - auto zookeeper = get_zookeeper(); - zookeeper->createIfNotExists(zookeeper_path + "/" + current_host + "|" + new_status, message); -} - -void BackupCoordinationStatusSync::setError(const String & current_host, const Exception & exception) -{ - auto zookeeper = get_zookeeper(); - - Exception exception2 = exception; - exception2.addMessage("Host {}", current_host); - WriteBufferFromOwnString buf; - writeException(exception2, buf, true); - - zookeeper->createIfNotExists(zookeeper_path + "/error", buf.str()); -} - -Strings BackupCoordinationStatusSync::wait(const Strings & all_hosts, const String & status_to_wait) -{ - return waitImpl(all_hosts, status_to_wait, {}); -} - -Strings BackupCoordinationStatusSync::waitFor(const Strings & all_hosts, const String & status_to_wait, UInt64 timeout_ms) -{ - return waitImpl(all_hosts, status_to_wait, timeout_ms); -} - -Strings BackupCoordinationStatusSync::waitImpl(const Strings & all_hosts, const String & status_to_wait, std::optional timeout_ms) -{ - if (all_hosts.empty()) - return {}; - - /// Wait for other hosts. - - Strings ready_hosts_results; - ready_hosts_results.resize(all_hosts.size()); - - std::map /* index in `ready_hosts_results` */> unready_hosts; - for (size_t i = 0; i != all_hosts.size(); ++i) - unready_hosts[all_hosts[i]].push_back(i); - - std::optional error; - - auto zookeeper = get_zookeeper(); - - /// Process ZooKeeper's nodes and set `all_hosts_ready` or `unready_host` or `error_message`. - auto process_zk_nodes = [&](const Strings & zk_nodes) - { - for (const String & zk_node : zk_nodes) - { - if (zk_node.starts_with("remove_watch-")) - continue; - - if (zk_node == "error") - { - ReadBufferFromOwnString buf{zookeeper->get(zookeeper_path + "/error")}; - error = readException(buf, "", true); - break; - } - - size_t separator_pos = zk_node.find('|'); - if (separator_pos == String::npos) - throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "Unexpected zk node {}", zookeeper_path + "/" + zk_node); - - String host = zk_node.substr(0, separator_pos); - String status = zk_node.substr(separator_pos + 1); - - auto it = unready_hosts.find(host); - if ((it != unready_hosts.end()) && (status == status_to_wait)) - { - String result = zookeeper->get(zookeeper_path + "/" + zk_node); - for (size_t i : it->second) - ready_hosts_results[i] = result; - unready_hosts.erase(it); - } - } - }; - - /// Wait until all hosts are ready or an error happens or time is out. - std::atomic watch_set = false; - std::condition_variable watch_triggered_event; - - auto watch_callback = [&](const Coordination::WatchResponse &) - { - watch_set = false; /// After it's triggered it's not set until we call getChildrenWatch() again. - watch_triggered_event.notify_all(); - }; - - auto watch_triggered = [&] { return !watch_set; }; - - bool use_timeout = timeout_ms.has_value(); - std::chrono::milliseconds timeout{timeout_ms.value_or(0)}; - std::chrono::steady_clock::time_point start_time = std::chrono::steady_clock::now(); - std::chrono::steady_clock::duration elapsed; - std::mutex dummy_mutex; - String previous_unready_host; - - while (!unready_hosts.empty() && !error) - { - watch_set = true; - Strings nodes = zookeeper->getChildrenWatch(zookeeper_path, nullptr, watch_callback); - process_zk_nodes(nodes); - - if (!unready_hosts.empty() && !error) - { - const auto & unready_host = unready_hosts.begin()->first; - if (unready_host != previous_unready_host) - { - LOG_TRACE(log, "Waiting for host {}", unready_host); - previous_unready_host = unready_host; - } - - std::unique_lock dummy_lock{dummy_mutex}; - if (use_timeout) - { - elapsed = std::chrono::steady_clock::now() - start_time; - if ((elapsed > timeout) || !watch_triggered_event.wait_for(dummy_lock, timeout - elapsed, watch_triggered)) - break; - } - else - watch_triggered_event.wait(dummy_lock, watch_triggered); - } - } - - if (watch_set) - { - /// Remove watch by triggering it. - zookeeper->create(zookeeper_path + "/remove_watch-", "", zkutil::CreateMode::EphemeralSequential); - std::unique_lock dummy_lock{dummy_mutex}; - watch_triggered_event.wait(dummy_lock, watch_triggered); - } - - if (error) - error->rethrow(); - - if (!unready_hosts.empty()) - { - throw Exception( - ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, - "Waited for host {} too long ({})", - unready_hosts.begin()->first, - to_string(elapsed)); - } - - return ready_hosts_results; -} - -} diff --git a/src/Backups/BackupCoordinationStatusSync.h b/src/Backups/BackupCoordinationStatusSync.h deleted file mode 100644 index fc03e8ec81c..00000000000 --- a/src/Backups/BackupCoordinationStatusSync.h +++ /dev/null @@ -1,37 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -/// Used to coordinate hosts so all hosts would come to a specific status at around the same time. -class BackupCoordinationStatusSync -{ -public: - BackupCoordinationStatusSync(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, Poco::Logger * log_); - - /// Sets the status of the current host and signal other hosts if there were other hosts waiting for that. - void set(const String & current_host, const String & new_status, const String & message); - void setError(const String & current_host, const Exception & exception); - - /// Sets the status of the current host and waits until all hosts come to the same status. - /// The function returns the messages all hosts set when they come to the required status. - Strings wait(const Strings & all_hosts, const String & status_to_wait); - - /// Almost the same as setAndWait() but this one stops waiting and throws an exception after a specific amount of time. - Strings waitFor(const Strings & all_hosts, const String & status_to_wait, UInt64 timeout_ms); - - static constexpr const char * kErrorStatus = "error"; - -private: - void createRootNodes(); - Strings waitImpl(const Strings & all_hosts, const String & status_to_wait, std::optional timeout_ms); - - String zookeeper_path; - zkutil::GetZooKeeper get_zookeeper; - Poco::Logger * log; -}; - -} diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index d2e4b1f8c4b..3cd9649de61 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -34,16 +34,21 @@ namespace ErrorCodes namespace { /// Finding all tables and databases which we're going to put to the backup and collecting their metadata. - constexpr const char * kGatheringMetadataStatus = "gathering metadata"; + constexpr const char * kGatheringMetadataStage = "gathering metadata"; + + String formatGatheringMetadataStage(size_t pass) + { + return fmt::format("{} ({})", kGatheringMetadataStage, pass); + } /// Making temporary hard links and prepare backup entries. - constexpr const char * kExtractingDataFromTablesStatus = "extracting data from tables"; + constexpr const char * kExtractingDataFromTablesStage = "extracting data from tables"; /// Running special tasks for replicated tables which can also prepare some backup entries. - constexpr const char * kRunningPostTasksStatus = "running post-tasks"; + constexpr const char * kRunningPostTasksStage = "running post-tasks"; /// Writing backup entries to the backup and removing temporary hard links. - constexpr const char * kWritingBackupStatus = "writing backup"; + constexpr const char * kWritingBackupStage = "writing backup"; /// Uppercases the first character of a passed string. String toUpperFirst(const String & str) @@ -90,7 +95,8 @@ BackupEntriesCollector::BackupEntriesCollector( , backup_settings(backup_settings_) , backup_coordination(backup_coordination_) , context(context_) - , consistent_metadata_snapshot_timeout(context->getConfigRef().getUInt64("backups.consistent_metadata_snapshot_timeout", 300000)) + , on_cluster_first_sync_timeout(context->getConfigRef().getUInt64("backups.on_cluster_first_sync_timeout", 180000)) + , consistent_metadata_snapshot_timeout(context->getConfigRef().getUInt64("backups.consistent_metadata_snapshot_timeout", 600000)) , log(&Poco::Logger::get("BackupEntriesCollector")) { } @@ -100,7 +106,7 @@ BackupEntriesCollector::~BackupEntriesCollector() = default; BackupEntries BackupEntriesCollector::run() { /// run() can be called onle once. - if (!current_status.empty()) + if (!current_stage.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Already making backup entries"); /// Find other hosts working along with us to execute this ON CLUSTER query. @@ -123,36 +129,40 @@ BackupEntries BackupEntriesCollector::run() makeBackupEntriesForTablesDefs(); /// Make backup entries for the data of the found tables. - setStatus(kExtractingDataFromTablesStatus); + setStage(kExtractingDataFromTablesStage); makeBackupEntriesForTablesData(); /// Run all the tasks added with addPostCollectingTask(). - setStatus(kRunningPostTasksStatus); + setStage(kRunningPostTasksStage); runPostTasks(); /// No more backup entries or tasks are allowed after this point. - setStatus(kWritingBackupStatus); + setStage(kWritingBackupStage); return std::move(backup_entries); } -Strings BackupEntriesCollector::setStatus(const String & new_status, const String & message) +Strings BackupEntriesCollector::setStage(const String & new_stage, const String & message) { - LOG_TRACE(log, "{}", toUpperFirst(new_status)); - current_status = new_status; + LOG_TRACE(log, "{}", toUpperFirst(new_stage)); + current_stage = new_stage; - backup_coordination->setStatus(backup_settings.host_id, new_status, message); + backup_coordination->setStage(backup_settings.host_id, new_stage, message); - if (new_status.starts_with(kGatheringMetadataStatus)) + if (new_stage == formatGatheringMetadataStage(1)) { - auto now = std::chrono::steady_clock::now(); - auto end_of_timeout = std::max(now, consistent_metadata_snapshot_start_time + consistent_metadata_snapshot_timeout); - return backup_coordination->waitStatusFor( - all_hosts, new_status, std::chrono::duration_cast(end_of_timeout - now).count()); + return backup_coordination->waitForStage(all_hosts, new_stage, on_cluster_first_sync_timeout); + } + else if (new_stage.starts_with(kGatheringMetadataStage)) + { + auto current_time = std::chrono::steady_clock::now(); + auto end_of_timeout = std::max(current_time, consistent_metadata_snapshot_end_time); + return backup_coordination->waitForStage( + all_hosts, new_stage, std::chrono::duration_cast(end_of_timeout - current_time)); } else { - return backup_coordination->waitStatus(all_hosts, new_status); + return backup_coordination->waitForStage(all_hosts, new_stage); } } @@ -173,18 +183,18 @@ void BackupEntriesCollector::calculateRootPathInBackup() /// Finds databases and tables which we will put to the backup. void BackupEntriesCollector::gatherMetadataAndCheckConsistency() { - consistent_metadata_snapshot_start_time = std::chrono::steady_clock::now(); - auto end_of_timeout = consistent_metadata_snapshot_start_time + consistent_metadata_snapshot_timeout; - setStatus(fmt::format("{} ({})", kGatheringMetadataStatus, 1)); + setStage(formatGatheringMetadataStage(1)); + + consistent_metadata_snapshot_end_time = std::chrono::steady_clock::now() + consistent_metadata_snapshot_timeout; for (size_t pass = 1;; ++pass) { - String new_status = fmt::format("{} ({})", kGatheringMetadataStatus, pass + 1); + String next_stage = formatGatheringMetadataStage(pass + 1); std::optional inconsistency_error; if (tryGatherMetadataAndCompareWithPrevious(inconsistency_error)) { /// Gathered metadata and checked consistency, cool! But we have to check that other hosts cope with that too. - auto all_hosts_results = setStatus(new_status, "consistent"); + auto all_hosts_results = setStage(next_stage, "consistent"); std::optional host_with_inconsistency; std::optional inconsistency_error_on_other_host; @@ -210,13 +220,13 @@ void BackupEntriesCollector::gatherMetadataAndCheckConsistency() else { /// Failed to gather metadata or something wasn't consistent. We'll let other hosts know that and try again. - setStatus(new_status, inconsistency_error->displayText()); + setStage(next_stage, inconsistency_error->displayText()); } /// Two passes is minimum (we need to compare with table names with previous ones to be sure we don't miss anything). if (pass >= 2) { - if (std::chrono::steady_clock::now() > end_of_timeout) + if (std::chrono::steady_clock::now() > consistent_metadata_snapshot_end_time) inconsistency_error->rethrow(); else LOG_WARNING(log, "{}", inconsistency_error->displayText()); @@ -713,7 +723,7 @@ void BackupEntriesCollector::makeBackupEntriesForTableData(const QualifiedTableN void BackupEntriesCollector::addBackupEntry(const String & file_name, BackupEntryPtr backup_entry) { - if (current_status == kWritingBackupStatus) + if (current_stage == kWritingBackupStage) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding backup entries is not allowed"); backup_entries.emplace_back(file_name, backup_entry); } @@ -725,21 +735,21 @@ void BackupEntriesCollector::addBackupEntry(const std::pair task) { - if (current_status == kWritingBackupStatus) + if (current_stage == kWritingBackupStage) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding of post tasks is not allowed"); post_tasks.push(std::move(task)); } diff --git a/src/Backups/BackupEntriesCollector.h b/src/Backups/BackupEntriesCollector.h index 03710605654..c42b5aedad4 100644 --- a/src/Backups/BackupEntriesCollector.h +++ b/src/Backups/BackupEntriesCollector.h @@ -86,12 +86,13 @@ private: void runPostTasks(); - Strings setStatus(const String & new_status, const String & message = ""); + Strings setStage(const String & new_stage, const String & message = ""); const ASTBackupQuery::Elements backup_query_elements; const BackupSettings backup_settings; std::shared_ptr backup_coordination; ContextPtr context; + std::chrono::milliseconds on_cluster_first_sync_timeout; std::chrono::milliseconds consistent_metadata_snapshot_timeout; Poco::Logger * log; @@ -129,8 +130,8 @@ private: std::optional partitions; }; - String current_status; - std::chrono::steady_clock::time_point consistent_metadata_snapshot_start_time; + String current_stage; + std::chrono::steady_clock::time_point consistent_metadata_snapshot_end_time; std::unordered_map database_infos; std::unordered_map table_infos; std::vector> previous_databases_metadata; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 09614886f06..47e1bac3200 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -18,37 +18,86 @@ #include #include #include -#include #include namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - namespace { /// Coordination status meaning that a host finished its work. - constexpr const char * kCompletedCoordinationStatus = "completed"; + constexpr const char * kCompletedStage = "completed"; - /// Sends information about the current exception to IBackupCoordination or IRestoreCoordination. - template - void sendErrorToCoordination(std::shared_ptr coordination, const String & current_host) + std::shared_ptr makeBackupCoordination(const String & coordination_zk_path, const ContextPtr & context, bool is_internal_backup) + { + if (!coordination_zk_path.empty()) + { + auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); }; + return std::make_shared(coordination_zk_path, get_zookeeper, !is_internal_backup); + } + else + { + return std::make_shared(); + } + } + + std::shared_ptr makeRestoreCoordination(const String & coordination_zk_path, const ContextPtr & context, bool is_internal_backup) + { + if (!coordination_zk_path.empty()) + { + auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); }; + return std::make_shared(coordination_zk_path, get_zookeeper, !is_internal_backup); + } + else + { + return std::make_shared(); + } + } + + /// Sends information about an exception to IBackupCoordination or IRestoreCoordination. + template + void sendExceptionToCoordination(std::shared_ptr coordination, const String & current_host, const Exception & exception) { - if (!coordination) - return; try { - coordination->setErrorStatus(current_host, Exception{getCurrentExceptionCode(), getCurrentExceptionMessage(true, true)}); + if (coordination) + coordination->setError(current_host, exception); } catch (...) { } } + + /// Sends information about the current exception to IBackupCoordination or IRestoreCoordination. + template + void sendCurrentExceptionToCoordination(std::shared_ptr coordination, const String & current_host) + { + try + { + throw; + } + catch (const Exception & e) + { + sendExceptionToCoordination(coordination, current_host, e); + } + catch (...) + { + coordination->setError(current_host, Exception{getCurrentExceptionCode(), getCurrentExceptionMessage(true, true)}); + } + } + + /// Used to change num_active_backups. + size_t getNumActiveBackupsChange(BackupStatus status) + { + return status == BackupStatus::MAKING_BACKUP; + } + + /// Used to change num_active_restores. + size_t getNumActiveRestoresChange(BackupStatus status) + { + return status == BackupStatus::RESTORING; + } } @@ -60,6 +109,7 @@ BackupsWorker::BackupsWorker(size_t num_backup_threads, size_t num_restore_threa /// We set max_free_threads = 0 because we don't want to keep any threads if there is no BACKUP or RESTORE query running right now. } + UUID BackupsWorker::start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context) { const ASTBackupQuery & backup_query = typeid_cast(*backup_or_restore_query); @@ -74,308 +124,359 @@ UUID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & c { auto backup_query = std::static_pointer_cast(query->clone()); auto backup_settings = BackupSettings::fromBackupQuery(*backup_query); - auto backup_info = BackupInfo::fromAST(*backup_query->backup_name); - bool on_cluster = !backup_query->cluster.empty(); if (!backup_settings.backup_uuid) backup_settings.backup_uuid = UUIDHelpers::generateV4(); UUID backup_uuid = *backup_settings.backup_uuid; - /// Prepare context to use. - ContextPtr context_in_use = context; - ContextMutablePtr mutable_context; - if (on_cluster || backup_settings.async) + std::shared_ptr backup_coordination; + if (!backup_settings.coordination_zk_path.empty()) + backup_coordination = makeBackupCoordination(backup_settings.coordination_zk_path, context, backup_settings.internal); + + try { - /// For ON CLUSTER queries we will need to change some settings. - /// For ASYNC queries we have to clone the context anyway. - context_in_use = mutable_context = Context::createCopy(context); + auto backup_info = BackupInfo::fromAST(*backup_query->backup_name); + addInfo(backup_uuid, backup_info.toString(), BackupStatus::MAKING_BACKUP, backup_settings.internal); + + /// Prepare context to use. + ContextPtr context_in_use = context; + ContextMutablePtr mutable_context; + bool on_cluster = !backup_query->cluster.empty(); + if (on_cluster || backup_settings.async) + { + /// For ON CLUSTER queries we will need to change some settings. + /// For ASYNC queries we have to clone the context anyway. + context_in_use = mutable_context = Context::createCopy(context); + } + + if (backup_settings.async) + { + backups_thread_pool.scheduleOrThrowOnError( + [this, backup_uuid, backup_query, backup_settings, backup_info, backup_coordination, context_in_use, mutable_context] { + doBackup( + backup_uuid, + backup_query, + backup_settings, + backup_info, + backup_coordination, + context_in_use, + mutable_context, + true); + }); + } + else + { + doBackup(backup_uuid, backup_query, backup_settings, backup_info, backup_coordination, context_in_use, mutable_context, false); + } + + return backup_uuid; } - - addInfo(backup_uuid, backup_info.toString(), BackupStatus::MAKING_BACKUP, backup_settings.internal); - - auto job = [this, - backup_uuid, - backup_query, - backup_settings, - backup_info, - on_cluster, - context_in_use, - mutable_context](bool async) mutable + catch (...) { - std::optional query_scope; - std::shared_ptr backup_coordination; - SCOPE_EXIT_SAFE(if (backup_coordination && !backup_settings.internal) backup_coordination->drop();); + /// Something bad happened, the backup has not built. + setStatus(backup_uuid, BackupStatus::FAILED_TO_BACKUP); + sendCurrentExceptionToCoordination(backup_coordination, backup_settings.host_id); + throw; + } +} - try + +void BackupsWorker::doBackup( + const UUID & backup_uuid, + const std::shared_ptr & backup_query, + BackupSettings backup_settings, + const BackupInfo & backup_info, + std::shared_ptr backup_coordination, + const ContextPtr & context, + ContextMutablePtr mutable_context, + bool called_async) +{ + std::optional query_scope; + try + { + if (called_async) { - if (async) - { - query_scope.emplace(mutable_context); - setThreadName("BackupWorker"); - } - - /// Checks access rights if this is not ON CLUSTER query. - /// (If this is ON CLUSTER query executeDDLQueryOnCluster() will check access rights later.) - auto required_access = getRequiredAccessToBackup(backup_query->elements); - if (!on_cluster) - context_in_use->checkAccess(required_access); - - ClusterPtr cluster; - if (on_cluster) - { - backup_query->cluster = context_in_use->getMacros()->expand(backup_query->cluster); - cluster = context_in_use->getCluster(backup_query->cluster); - backup_settings.cluster_host_ids = cluster->getHostIDs(); - if (backup_settings.coordination_zk_path.empty()) - { - String root_zk_path = context_in_use->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); - backup_settings.coordination_zk_path = root_zk_path + "/backup-" + toString(backup_uuid); - } - } - - /// Make a backup coordination. - if (!backup_settings.coordination_zk_path.empty()) - { - backup_coordination = std::make_shared( - backup_settings.coordination_zk_path, - [global_context = context_in_use->getGlobalContext()] { return global_context->getZooKeeper(); }); - } - else - { - backup_coordination = std::make_shared(); - } - - /// Opens a backup for writing. - BackupFactory::CreateParams backup_create_params; - backup_create_params.open_mode = IBackup::OpenMode::WRITE; - backup_create_params.context = context_in_use; - backup_create_params.backup_info = backup_info; - backup_create_params.base_backup_info = backup_settings.base_backup_info; - backup_create_params.compression_method = backup_settings.compression_method; - backup_create_params.compression_level = backup_settings.compression_level; - backup_create_params.password = backup_settings.password; - backup_create_params.is_internal_backup = backup_settings.internal; - backup_create_params.backup_coordination = backup_coordination; - backup_create_params.backup_uuid = backup_uuid; - BackupMutablePtr backup = BackupFactory::instance().createBackup(backup_create_params); - - /// Write the backup. - if (on_cluster) - { - DDLQueryOnClusterParams params; - params.cluster = cluster; - params.only_shard_num = backup_settings.shard_num; - params.only_replica_num = backup_settings.replica_num; - params.access_to_check = required_access; - backup_settings.copySettingsToQuery(*backup_query); - - // executeDDLQueryOnCluster() will return without waiting for completion - mutable_context->setSetting("distributed_ddl_task_timeout", Field{0}); - mutable_context->setSetting("distributed_ddl_output_mode", Field{"none"}); - executeDDLQueryOnCluster(backup_query, mutable_context, params); - - /// Wait until all the hosts have written their backup entries. - auto all_hosts = BackupSettings::Util::filterHostIDs( - backup_settings.cluster_host_ids, backup_settings.shard_num, backup_settings.replica_num); - backup_coordination->waitStatus(all_hosts, kCompletedCoordinationStatus); - } - else - { - backup_query->setCurrentDatabase(context_in_use->getCurrentDatabase()); - - /// Prepare backup entries. - BackupEntries backup_entries; - { - BackupEntriesCollector backup_entries_collector{backup_query->elements, backup_settings, backup_coordination, context_in_use}; - backup_entries = backup_entries_collector.run(); - } - - /// Write the backup entries to the backup. - writeBackupEntries(backup, std::move(backup_entries), backups_thread_pool); - - /// We have written our backup entries, we need to tell other hosts (they could be waiting for it). - backup_coordination->setStatus(backup_settings.host_id, kCompletedCoordinationStatus, ""); - } - - /// Finalize backup (write its metadata). - if (!backup_settings.internal) - backup->finalizeWriting(); - - /// Close the backup. - backup.reset(); - - setStatus(backup_uuid, BackupStatus::BACKUP_COMPLETE); + query_scope.emplace(mutable_context); + setThreadName("BackupWorker"); } - catch (...) + + bool on_cluster = !backup_query->cluster.empty(); + assert(mutable_context || (!on_cluster && !called_async)); + + /// Checks access rights if this is not ON CLUSTER query. + /// (If this is ON CLUSTER query executeDDLQueryOnCluster() will check access rights later.) + auto required_access = getRequiredAccessToBackup(backup_query->elements); + if (!on_cluster) + context->checkAccess(required_access); + + ClusterPtr cluster; + if (on_cluster) { - /// Something bad happened, the backup has not built. + backup_query->cluster = context->getMacros()->expand(backup_query->cluster); + cluster = context->getCluster(backup_query->cluster); + backup_settings.cluster_host_ids = cluster->getHostIDs(); + if (backup_settings.coordination_zk_path.empty()) + { + String root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); + backup_settings.coordination_zk_path = root_zk_path + "/backup-" + toString(backup_uuid); + } + } + + /// Make a backup coordination. + if (!backup_coordination) + backup_coordination = makeBackupCoordination(backup_settings.coordination_zk_path, context, backup_settings.internal); + + /// Opens a backup for writing. + BackupFactory::CreateParams backup_create_params; + backup_create_params.open_mode = IBackup::OpenMode::WRITE; + backup_create_params.context = context; + backup_create_params.backup_info = backup_info; + backup_create_params.base_backup_info = backup_settings.base_backup_info; + backup_create_params.compression_method = backup_settings.compression_method; + backup_create_params.compression_level = backup_settings.compression_level; + backup_create_params.password = backup_settings.password; + backup_create_params.is_internal_backup = backup_settings.internal; + backup_create_params.backup_coordination = backup_coordination; + backup_create_params.backup_uuid = backup_uuid; + BackupMutablePtr backup = BackupFactory::instance().createBackup(backup_create_params); + + /// Write the backup. + if (on_cluster) + { + DDLQueryOnClusterParams params; + params.cluster = cluster; + params.only_shard_num = backup_settings.shard_num; + params.only_replica_num = backup_settings.replica_num; + params.access_to_check = required_access; + backup_settings.copySettingsToQuery(*backup_query); + + // executeDDLQueryOnCluster() will return without waiting for completion + mutable_context->setSetting("distributed_ddl_task_timeout", Field{0}); + mutable_context->setSetting("distributed_ddl_output_mode", Field{"none"}); + executeDDLQueryOnCluster(backup_query, mutable_context, params); + + /// Wait until all the hosts have written their backup entries. + auto all_hosts = BackupSettings::Util::filterHostIDs( + backup_settings.cluster_host_ids, backup_settings.shard_num, backup_settings.replica_num); + backup_coordination->waitForStage(all_hosts, kCompletedStage); + } + else + { + backup_query->setCurrentDatabase(context->getCurrentDatabase()); + + /// Prepare backup entries. + BackupEntries backup_entries; + { + BackupEntriesCollector backup_entries_collector{backup_query->elements, backup_settings, backup_coordination, context}; + backup_entries = backup_entries_collector.run(); + } + + /// Write the backup entries to the backup. + writeBackupEntries(backup, std::move(backup_entries), backups_thread_pool); + + /// We have written our backup entries, we need to tell other hosts (they could be waiting for it). + backup_coordination->setStage(backup_settings.host_id, kCompletedStage, ""); + } + + /// Finalize backup (write its metadata). + if (!backup_settings.internal) + backup->finalizeWriting(); + + /// Close the backup. + backup.reset(); + + LOG_INFO(log, "{} {} was created successfully", (backup_settings.internal ? "Internal backup" : "Backup"), backup_info.toString()); + setStatus(backup_uuid, BackupStatus::BACKUP_COMPLETE); + } + catch (...) + { + /// Something bad happened, the backup has not built. + if (called_async) + { + tryLogCurrentException(log, fmt::format("Failed to make {} {}", (backup_settings.internal ? "internal backup" : "backup"), backup_info.toString())); setStatus(backup_uuid, BackupStatus::FAILED_TO_BACKUP); - sendErrorToCoordination(backup_coordination, backup_settings.host_id); - if (!async) - throw; + sendCurrentExceptionToCoordination(backup_coordination, backup_settings.host_id); } - }; - - if (backup_settings.async) - backups_thread_pool.scheduleOrThrowOnError([job]() mutable { job(true); }); - else - job(false); - - return backup_uuid; + else + { + /// setStatus() and sendCurrentExceptionToCoordination() will be called by startMakingBackup(). + throw; + } + } } UUID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr context) { - UUID restore_uuid = UUIDHelpers::generateV4(); auto restore_query = std::static_pointer_cast(query->clone()); auto restore_settings = RestoreSettings::fromRestoreQuery(*restore_query); - auto backup_info = BackupInfo::fromAST(*restore_query->backup_name); - bool on_cluster = !restore_query->cluster.empty(); + UUID restore_uuid = UUIDHelpers::generateV4(); - /// Prepare context to use. - ContextMutablePtr context_in_use = context; - if (restore_settings.async || on_cluster) + std::shared_ptr restore_coordination; + if (!restore_settings.coordination_zk_path.empty()) + restore_coordination = makeRestoreCoordination(restore_settings.coordination_zk_path, context, restore_settings.internal); + + try { - /// For ON CLUSTER queries we will need to change some settings. - /// For ASYNC queries we have to clone the context anyway. - context_in_use = Context::createCopy(context); + auto backup_info = BackupInfo::fromAST(*restore_query->backup_name); + addInfo(restore_uuid, backup_info.toString(), BackupStatus::RESTORING, restore_settings.internal); + + /// Prepare context to use. + ContextMutablePtr context_in_use = context; + bool on_cluster = !restore_query->cluster.empty(); + if (restore_settings.async || on_cluster) + { + /// For ON CLUSTER queries we will need to change some settings. + /// For ASYNC queries we have to clone the context anyway. + context_in_use = Context::createCopy(context); + } + + if (restore_settings.async) + { + backups_thread_pool.scheduleOrThrowOnError( + [this, restore_uuid, restore_query, restore_settings, backup_info, restore_coordination, context_in_use] + { doRestore(restore_uuid, restore_query, restore_settings, backup_info, restore_coordination, context_in_use, true); }); + } + else + { + doRestore(restore_uuid, restore_query, restore_settings, backup_info, restore_coordination, context_in_use, false); + } + + return restore_uuid; } - - addInfo(restore_uuid, backup_info.toString(), BackupStatus::RESTORING, restore_settings.internal); - - auto job = [this, - restore_uuid, - restore_query, - restore_settings, - backup_info, - on_cluster, - context_in_use](bool async) mutable + catch (...) { - std::optional query_scope; - std::shared_ptr restore_coordination; - SCOPE_EXIT_SAFE(if (restore_coordination && !restore_settings.internal) restore_coordination->drop();); + /// Something bad happened, the backup has not built. + setStatus(restore_uuid, BackupStatus::FAILED_TO_RESTORE); + sendCurrentExceptionToCoordination(restore_coordination, restore_settings.host_id); + throw; + } +} - try + +void BackupsWorker::doRestore( + const UUID & restore_uuid, + const std::shared_ptr & restore_query, + RestoreSettings restore_settings, + const BackupInfo & backup_info, + std::shared_ptr restore_coordination, + ContextMutablePtr context, + bool called_async) +{ + std::optional query_scope; + try + { + if (called_async) { - if (async) - { - query_scope.emplace(context_in_use); - setThreadName("RestoreWorker"); - } - - /// Open the backup for reading. - BackupFactory::CreateParams backup_open_params; - backup_open_params.open_mode = IBackup::OpenMode::READ; - backup_open_params.context = context_in_use; - backup_open_params.backup_info = backup_info; - backup_open_params.base_backup_info = restore_settings.base_backup_info; - backup_open_params.password = restore_settings.password; - BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params); - - String current_database = context_in_use->getCurrentDatabase(); - - /// Checks access rights if this is ON CLUSTER query. - /// (If this isn't ON CLUSTER query RestorerFromBackup will check access rights later.) - ClusterPtr cluster; - if (on_cluster) - { - restore_query->cluster = context_in_use->getMacros()->expand(restore_query->cluster); - cluster = context_in_use->getCluster(restore_query->cluster); - restore_settings.cluster_host_ids = cluster->getHostIDs(); - - /// We cannot just use access checking provided by the function executeDDLQueryOnCluster(): it would be incorrect - /// because different replicas can contain different set of tables and so the required access rights can differ too. - /// So the right way is pass through the entire cluster and check access for each host. - auto addresses = cluster->filterAddressesByShardOrReplica(restore_settings.shard_num, restore_settings.replica_num); - for (const auto * address : addresses) - { - restore_settings.host_id = address->toString(); - auto restore_elements = restore_query->elements; - String addr_database = address->default_database.empty() ? current_database : address->default_database; - for (auto & element : restore_elements) - element.setCurrentDatabase(addr_database); - RestorerFromBackup dummy_restorer{restore_elements, restore_settings, nullptr, backup, context_in_use}; - dummy_restorer.run(RestorerFromBackup::CHECK_ACCESS_ONLY); - } - } - - /// Make a restore coordination. - if (on_cluster && restore_settings.coordination_zk_path.empty()) - { - String root_zk_path = context_in_use->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); - restore_settings.coordination_zk_path = root_zk_path + "/restore-" + toString(restore_uuid); - } - - if (!restore_settings.coordination_zk_path.empty()) - { - restore_coordination = std::make_shared( - restore_settings.coordination_zk_path, - [global_context = context_in_use->getGlobalContext()] { return global_context->getZooKeeper(); }); - } - else - { - restore_coordination = std::make_shared(); - } - - /// Do RESTORE. - if (on_cluster) - { - - DDLQueryOnClusterParams params; - params.cluster = cluster; - params.only_shard_num = restore_settings.shard_num; - params.only_replica_num = restore_settings.replica_num; - restore_settings.copySettingsToQuery(*restore_query); - - // executeDDLQueryOnCluster() will return without waiting for completion - context_in_use->setSetting("distributed_ddl_task_timeout", Field{0}); - context_in_use->setSetting("distributed_ddl_output_mode", Field{"none"}); - - executeDDLQueryOnCluster(restore_query, context_in_use, params); - - /// Wait until all the hosts have written their backup entries. - auto all_hosts = BackupSettings::Util::filterHostIDs( - restore_settings.cluster_host_ids, restore_settings.shard_num, restore_settings.replica_num); - restore_coordination->waitStatus(all_hosts, kCompletedCoordinationStatus); - } - else - { - restore_query->setCurrentDatabase(current_database); - - /// Restore metadata and prepare data restoring tasks. - DataRestoreTasks data_restore_tasks; - { - RestorerFromBackup restorer{restore_query->elements, restore_settings, restore_coordination, - backup, context_in_use}; - data_restore_tasks = restorer.run(RestorerFromBackup::RESTORE); - } - - /// Execute the data restoring tasks. - restoreTablesData(std::move(data_restore_tasks), restores_thread_pool); - - /// We have restored everything, we need to tell other hosts (they could be waiting for it). - restore_coordination->setStatus(restore_settings.host_id, kCompletedCoordinationStatus, ""); - } - - setStatus(restore_uuid, BackupStatus::RESTORED); + query_scope.emplace(context); + setThreadName("RestoreWorker"); } - catch (...) + + /// Open the backup for reading. + BackupFactory::CreateParams backup_open_params; + backup_open_params.open_mode = IBackup::OpenMode::READ; + backup_open_params.context = context; + backup_open_params.backup_info = backup_info; + backup_open_params.base_backup_info = restore_settings.base_backup_info; + backup_open_params.password = restore_settings.password; + BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params); + + String current_database = context->getCurrentDatabase(); + + /// Checks access rights if this is ON CLUSTER query. + /// (If this isn't ON CLUSTER query RestorerFromBackup will check access rights later.) + ClusterPtr cluster; + bool on_cluster = !restore_query->cluster.empty(); + if (on_cluster) { - /// Something bad happened, the backup has not built. + restore_query->cluster = context->getMacros()->expand(restore_query->cluster); + cluster = context->getCluster(restore_query->cluster); + restore_settings.cluster_host_ids = cluster->getHostIDs(); + + /// We cannot just use access checking provided by the function executeDDLQueryOnCluster(): it would be incorrect + /// because different replicas can contain different set of tables and so the required access rights can differ too. + /// So the right way is pass through the entire cluster and check access for each host. + auto addresses = cluster->filterAddressesByShardOrReplica(restore_settings.shard_num, restore_settings.replica_num); + for (const auto * address : addresses) + { + restore_settings.host_id = address->toString(); + auto restore_elements = restore_query->elements; + String addr_database = address->default_database.empty() ? current_database : address->default_database; + for (auto & element : restore_elements) + element.setCurrentDatabase(addr_database); + RestorerFromBackup dummy_restorer{restore_elements, restore_settings, nullptr, backup, context}; + dummy_restorer.run(RestorerFromBackup::CHECK_ACCESS_ONLY); + } + } + + /// Make a restore coordination. + if (on_cluster && restore_settings.coordination_zk_path.empty()) + { + String root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); + restore_settings.coordination_zk_path = root_zk_path + "/restore-" + toString(restore_uuid); + } + + if (!restore_coordination) + restore_coordination = makeRestoreCoordination(restore_settings.coordination_zk_path, context, restore_settings.internal); + + /// Do RESTORE. + if (on_cluster) + { + + DDLQueryOnClusterParams params; + params.cluster = cluster; + params.only_shard_num = restore_settings.shard_num; + params.only_replica_num = restore_settings.replica_num; + restore_settings.copySettingsToQuery(*restore_query); + + // executeDDLQueryOnCluster() will return without waiting for completion + context->setSetting("distributed_ddl_task_timeout", Field{0}); + context->setSetting("distributed_ddl_output_mode", Field{"none"}); + + executeDDLQueryOnCluster(restore_query, context, params); + + /// Wait until all the hosts have written their backup entries. + auto all_hosts = BackupSettings::Util::filterHostIDs( + restore_settings.cluster_host_ids, restore_settings.shard_num, restore_settings.replica_num); + restore_coordination->waitForStage(all_hosts, kCompletedStage); + } + else + { + restore_query->setCurrentDatabase(current_database); + + /// Restore metadata and prepare data restoring tasks. + DataRestoreTasks data_restore_tasks; + { + RestorerFromBackup restorer{restore_query->elements, restore_settings, restore_coordination, + backup, context}; + data_restore_tasks = restorer.run(RestorerFromBackup::RESTORE); + } + + /// Execute the data restoring tasks. + restoreTablesData(std::move(data_restore_tasks), restores_thread_pool); + + /// We have restored everything, we need to tell other hosts (they could be waiting for it). + restore_coordination->setStage(restore_settings.host_id, kCompletedStage, ""); + } + + LOG_INFO(log, "Restored from {} {} successfully", (restore_settings.internal ? "internal backup" : "backup"), backup_info.toString()); + setStatus(restore_uuid, BackupStatus::RESTORED); + } + catch (...) + { + /// Something bad happened, the backup has not built. + if (called_async) + { + tryLogCurrentException(log, fmt::format("Failed to restore from {} {}", (restore_settings.internal ? "internal backup" : "backup"), backup_info.toString())); setStatus(restore_uuid, BackupStatus::FAILED_TO_RESTORE); - sendErrorToCoordination(restore_coordination, restore_settings.host_id); - if (!async) - throw; + sendCurrentExceptionToCoordination(restore_coordination, restore_settings.host_id); } - }; - - if (restore_settings.async) - backups_thread_pool.scheduleOrThrowOnError([job]() mutable { job(true); }); - else - job(false); - - return restore_uuid; + else + { + /// setStatus() and sendCurrentExceptionToCoordination() will be called by startRestoring(). + throw; + } + } } @@ -387,37 +488,28 @@ void BackupsWorker::addInfo(const UUID & uuid, const String & backup_name, Backu info.status = status; info.status_changed_time = time(nullptr); info.internal = internal; + std::lock_guard lock{infos_mutex}; infos[uuid] = std::move(info); + + num_active_backups += getNumActiveBackupsChange(status); + num_active_restores += getNumActiveRestoresChange(status); } + void BackupsWorker::setStatus(const UUID & uuid, BackupStatus status) { std::lock_guard lock{infos_mutex}; - auto & info = infos.at(uuid); + auto it = infos.find(uuid); + if (it == infos.end()) + return; + + auto & info = it->second; + auto old_status = info.status; info.status = status; info.status_changed_time = time(nullptr); - - if (status == BackupStatus::BACKUP_COMPLETE) - { - LOG_INFO(log, "{} {} was created successfully", (info.internal ? "Internal backup" : "Backup"), info.backup_name); - } - else if (status == BackupStatus::RESTORED) - { - LOG_INFO(log, "Restored from {} {} successfully", (info.internal ? "internal backup" : "backup"), info.backup_name); - } - else if ((status == BackupStatus::FAILED_TO_BACKUP) || (status == BackupStatus::FAILED_TO_RESTORE)) - { - String start_of_message; - if (status == BackupStatus::FAILED_TO_BACKUP) - start_of_message = fmt::format("Failed to create {} {}", (info.internal ? "internal backup" : "backup"), info.backup_name); - else - start_of_message = fmt::format("Failed to restore from {} {}", (info.internal ? "internal backup" : "backup"), info.backup_name); - tryLogCurrentException(log, start_of_message); - - info.error_message = getCurrentExceptionMessage(false); - info.exception = std::current_exception(); - } + num_active_backups += getNumActiveBackupsChange(status) - getNumActiveBackupsChange(old_status); + num_active_restores += getNumActiveRestoresChange(status) - getNumActiveRestoresChange(old_status); } @@ -428,7 +520,7 @@ void BackupsWorker::wait(const UUID & backup_or_restore_uuid, bool rethrow_excep { auto it = infos.find(backup_or_restore_uuid); if (it == infos.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "BackupsWorker: Unknown UUID {}", toString(backup_or_restore_uuid)); + return true; const auto & info = it->second; auto current_status = info.status; if (rethrow_exception && ((current_status == BackupStatus::FAILED_TO_BACKUP) || (current_status == BackupStatus::FAILED_TO_RESTORE))) @@ -437,12 +529,12 @@ void BackupsWorker::wait(const UUID & backup_or_restore_uuid, bool rethrow_excep }); } -BackupsWorker::Info BackupsWorker::getInfo(const UUID & backup_or_restore_uuid) const +std::optional BackupsWorker::tryGetInfo(const UUID & backup_or_restore_uuid) const { std::lock_guard lock{infos_mutex}; auto it = infos.find(backup_or_restore_uuid); if (it == infos.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "BackupsWorker: Unknown UUID {}", toString(backup_or_restore_uuid)); + return std::nullopt; return it->second; } @@ -457,14 +549,15 @@ std::vector BackupsWorker::getAllInfos() const void BackupsWorker::shutdown() { - size_t num_active_backups = backups_thread_pool.active(); - size_t num_active_restores = restores_thread_pool.active(); - if (!num_active_backups && !num_active_restores) - return; - LOG_INFO(log, "Waiting for {} backup and {} restore tasks to be finished", num_active_backups, num_active_restores); + bool has_active_backups_or_restores = (num_active_backups || num_active_restores); + if (has_active_backups_or_restores) + LOG_INFO(log, "Waiting for {} backups and {} restores to be finished", num_active_backups, num_active_restores); + backups_thread_pool.wait(); restores_thread_pool.wait(); - LOG_INFO(log, "All backup and restore tasks have finished"); + + if (has_active_backups_or_restores) + LOG_INFO(log, "All backup and restore tasks have finished"); } } diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index f546fa2497d..8db9c1367a9 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -11,6 +11,13 @@ namespace Poco::Util { class AbstractConfiguration; } namespace DB { +class ASTBackupQuery; +struct BackupSettings; +struct RestoreSettings; +struct BackupInfo; +class IBackupCoordination; +class IRestoreCoordination; + /// Manager of backups and restores: executes backups and restores' threads in the background. /// Keeps information about backups and restores started in this session. class BackupsWorker @@ -47,12 +54,21 @@ public: bool internal = false; }; - Info getInfo(const UUID & backup_or_restore_uuid) const; + std::optional tryGetInfo(const UUID & backup_or_restore_uuid) const; std::vector getAllInfos() const; private: UUID startMakingBackup(const ASTPtr & query, const ContextPtr & context); + + void doBackup(const UUID & backup_uuid, const std::shared_ptr & backup_query, BackupSettings backup_settings, + const BackupInfo & backup_info, std::shared_ptr backup_coordination, const ContextPtr & context, + ContextMutablePtr mutable_context, bool called_async); + UUID startRestoring(const ASTPtr & query, ContextMutablePtr context); + + void doRestore(const UUID & restore_uuid, const std::shared_ptr & restore_query, RestoreSettings restore_settings, + const BackupInfo & backup_info, std::shared_ptr restore_coordination, ContextMutablePtr context, + bool called_async); void addInfo(const UUID & uuid, const String & backup_name, BackupStatus status, bool internal); void setStatus(const UUID & uuid, BackupStatus status); @@ -62,6 +78,8 @@ private: std::unordered_map infos; std::condition_variable status_changed; + std::atomic num_active_backups = 0; + std::atomic num_active_restores = 0; mutable std::mutex infos_mutex; Poco::Logger * log; }; diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index 7cf43efea74..5e120218544 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -18,11 +18,11 @@ class IBackupCoordination public: virtual ~IBackupCoordination() = default; - /// Sets the current status and waits for other hosts to come to this status too. - virtual void setStatus(const String & current_host, const String & new_status, const String & message) = 0; - virtual void setErrorStatus(const String & current_host, const Exception & exception) = 0; - virtual Strings waitStatus(const Strings & all_hosts, const String & status_to_wait) = 0; - virtual Strings waitStatusFor(const Strings & all_hosts, const String & status_to_wait, UInt64 timeout_ms) = 0; + /// Sets the current stage and waits for other hosts to come to this stage too. + virtual void setStage(const String & current_host, const String & new_stage, const String & message) = 0; + virtual void setError(const String & current_host, const Exception & exception) = 0; + virtual Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait) = 0; + virtual Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; struct PartNameAndChecksum { @@ -115,9 +115,6 @@ public: /// Returns the list of all the archive suffixes which were generated. virtual Strings getAllArchiveSuffixes() const = 0; - - /// Removes remotely stored information. - virtual void drop() {} }; } diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h index e852fa3c2d4..692054ae267 100644 --- a/src/Backups/IRestoreCoordination.h +++ b/src/Backups/IRestoreCoordination.h @@ -16,11 +16,11 @@ class IRestoreCoordination public: virtual ~IRestoreCoordination() = default; - /// Sets the current status and waits for other hosts to come to this status too. - virtual void setStatus(const String & current_host, const String & new_status, const String & message) = 0; - virtual void setErrorStatus(const String & current_host, const Exception & exception) = 0; - virtual Strings waitStatus(const Strings & all_hosts, const String & status_to_wait) = 0; - virtual Strings waitStatusFor(const Strings & all_hosts, const String & status_to_wait, UInt64 timeout_ms) = 0; + /// Sets the current stage and waits for other hosts to come to this stage too. + virtual void setStage(const String & current_host, const String & new_stage, const String & message) = 0; + virtual void setError(const String & current_host, const Exception & exception) = 0; + virtual Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait) = 0; + virtual Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; static constexpr const char * kErrorStatus = "error"; @@ -34,9 +34,6 @@ public: /// Sets that this replica is going to restore a ReplicatedAccessStorage. /// The function returns false if this access storage is being already restored by another replica. virtual bool acquireReplicatedAccessStorage(const String & access_storage_zk_path) = 0; - - /// Removes remotely stored information. - virtual void drop() {} }; } diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index deab75dc7de..b2a9849c38d 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -7,20 +7,20 @@ namespace DB RestoreCoordinationLocal::RestoreCoordinationLocal() = default; RestoreCoordinationLocal::~RestoreCoordinationLocal() = default; -void RestoreCoordinationLocal::setStatus(const String &, const String &, const String &) +void RestoreCoordinationLocal::setStage(const String &, const String &, const String &) { } -void RestoreCoordinationLocal::setErrorStatus(const String &, const Exception &) +void RestoreCoordinationLocal::setError(const String &, const Exception &) { } -Strings RestoreCoordinationLocal::waitStatus(const Strings &, const String &) +Strings RestoreCoordinationLocal::waitForStage(const Strings &, const String &) { return {}; } -Strings RestoreCoordinationLocal::waitStatusFor(const Strings &, const String &, UInt64) +Strings RestoreCoordinationLocal::waitForStage(const Strings &, const String &, std::chrono::milliseconds) { return {}; } diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index d8b0052cbd2..b4e70d83b72 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -18,11 +18,11 @@ public: RestoreCoordinationLocal(); ~RestoreCoordinationLocal() override; - /// Sets the current status and waits for other hosts to come to this status too. If status starts with "error:" it'll stop waiting on all the hosts. - void setStatus(const String & current_host, const String & new_status, const String & message) override; - void setErrorStatus(const String & current_host, const Exception & exception) override; - Strings waitStatus(const Strings & all_hosts, const String & status_to_wait) override; - Strings waitStatusFor(const Strings & all_hosts, const String & status_to_wait, UInt64 timeout_ms) override; + /// Sets the current stage and waits for other hosts to come to this stage too. + void setStage(const String & current_host, const String & new_stage, const String & message) override; + void setError(const String & current_host, const Exception & exception) override; + Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait) override; + Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) override; /// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table. bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) override; diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index 86c8ca6b509..fcc6a2a24b3 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -6,15 +6,27 @@ namespace DB { -RestoreCoordinationRemote::RestoreCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_) +RestoreCoordinationRemote::RestoreCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool remove_zk_nodes_in_destructor_) : zookeeper_path(zookeeper_path_) , get_zookeeper(get_zookeeper_) - , status_sync(zookeeper_path_ + "/status", get_zookeeper_, &Poco::Logger::get("RestoreCoordination")) + , remove_zk_nodes_in_destructor(remove_zk_nodes_in_destructor_) + , stage_sync(zookeeper_path_ + "/stage", get_zookeeper_, &Poco::Logger::get("RestoreCoordination")) { createRootNodes(); } -RestoreCoordinationRemote::~RestoreCoordinationRemote() = default; +RestoreCoordinationRemote::~RestoreCoordinationRemote() +{ + try + { + if (remove_zk_nodes_in_destructor) + removeAllNodes(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} void RestoreCoordinationRemote::createRootNodes() { @@ -27,24 +39,24 @@ void RestoreCoordinationRemote::createRootNodes() } -void RestoreCoordinationRemote::setStatus(const String & current_host, const String & new_status, const String & message) +void RestoreCoordinationRemote::setStage(const String & current_host, const String & new_stage, const String & message) { - status_sync.set(current_host, new_status, message); + stage_sync.set(current_host, new_stage, message); } -void RestoreCoordinationRemote::setErrorStatus(const String & current_host, const Exception & exception) +void RestoreCoordinationRemote::setError(const String & current_host, const Exception & exception) { - status_sync.setError(current_host, exception); + stage_sync.setError(current_host, exception); } -Strings RestoreCoordinationRemote::waitStatus(const Strings & all_hosts, const String & status_to_wait) +Strings RestoreCoordinationRemote::waitForStage(const Strings & all_hosts, const String & stage_to_wait) { - return status_sync.wait(all_hosts, status_to_wait); + return stage_sync.wait(all_hosts, stage_to_wait); } -Strings RestoreCoordinationRemote::waitStatusFor(const Strings & all_hosts, const String & status_to_wait, UInt64 timeout_ms) +Strings RestoreCoordinationRemote::waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) { - return status_sync.waitFor(all_hosts, status_to_wait, timeout_ms); + return stage_sync.waitFor(all_hosts, stage_to_wait, timeout); } @@ -93,9 +105,4 @@ void RestoreCoordinationRemote::removeAllNodes() zookeeper->removeRecursive(zookeeper_path); } -void RestoreCoordinationRemote::drop() -{ - removeAllNodes(); -} - } diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index 883ea953efc..0cbbb6622ad 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -11,14 +11,14 @@ namespace DB class RestoreCoordinationRemote : public IRestoreCoordination { public: - RestoreCoordinationRemote(const String & zookeeper_path, zkutil::GetZooKeeper get_zookeeper); + RestoreCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool remove_zk_nodes_in_destructor_); ~RestoreCoordinationRemote() override; - /// Sets the current status and waits for other hosts to come to this status too. If status starts with "error:" it'll stop waiting on all the hosts. - void setStatus(const String & current_host, const String & new_status, const String & message) override; - void setErrorStatus(const String & current_host, const Exception & exception) override; - Strings waitStatus(const Strings & all_hosts, const String & status_to_wait) override; - Strings waitStatusFor(const Strings & all_hosts, const String & status_to_wait, UInt64 timeout_ms) override; + /// Sets the current stage and waits for other hosts to come to this stage too. + void setStage(const String & current_host, const String & new_stage, const String & message) override; + void setError(const String & current_host, const Exception & exception) override; + Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait) override; + Strings waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) override; /// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table. bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) override; @@ -31,9 +31,6 @@ public: /// The function returns false if this access storage is being already restored by another replica. bool acquireReplicatedAccessStorage(const String & access_storage_zk_path) override; - /// Removes remotely stored information. - void drop() override; - private: void createRootNodes(); void removeAllNodes(); @@ -42,7 +39,9 @@ private: const String zookeeper_path; const zkutil::GetZooKeeper get_zookeeper; - BackupCoordinationStatusSync status_sync; + const bool remove_zk_nodes_in_destructor; + + BackupCoordinationStageSync stage_sync; }; } diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index b67cdf9c4dd..5e43d59ae56 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -41,16 +41,16 @@ namespace ErrorCodes namespace { /// Finding databases and tables in the backup which we're going to restore. - constexpr const char * kFindingTablesInBackupStatus = "finding tables in backup"; + constexpr const char * kFindingTablesInBackupStage = "finding tables in backup"; /// Creating databases or finding them and checking their definitions. - constexpr const char * kCreatingDatabasesStatus = "creating databases"; + constexpr const char * kCreatingDatabasesStage = "creating databases"; /// Creating tables or finding them and checking their definition. - constexpr const char * kCreatingTablesStatus = "creating tables"; + constexpr const char * kCreatingTablesStage = "creating tables"; /// Inserting restored data to tables. - constexpr const char * kInsertingDataToTablesStatus = "inserting data to tables"; + constexpr const char * kInsertingDataToTablesStage = "inserting data to tables"; /// Uppercases the first character of a passed string. String toUpperFirst(const String & str) @@ -102,6 +102,7 @@ RestorerFromBackup::RestorerFromBackup( , restore_coordination(restore_coordination_) , backup(backup_) , context(context_) + , on_cluster_first_sync_timeout(context->getConfigRef().getUInt64("backups.on_cluster_first_sync_timeout", 180000)) , create_table_timeout(context->getConfigRef().getUInt64("backups.create_table_timeout", 300000)) , log(&Poco::Logger::get("RestorerFromBackup")) { @@ -112,7 +113,7 @@ RestorerFromBackup::~RestorerFromBackup() = default; RestorerFromBackup::DataRestoreTasks RestorerFromBackup::run(Mode mode) { /// run() can be called onle once. - if (!current_status.empty()) + if (!current_stage.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Already restoring"); /// Find other hosts working along with us to execute this ON CLUSTER query. @@ -126,7 +127,7 @@ RestorerFromBackup::DataRestoreTasks RestorerFromBackup::run(Mode mode) findRootPathsInBackup(); /// Find all the databases and tables which we will read from the backup. - setStatus(kFindingTablesInBackupStatus); + setStage(kFindingTablesInBackupStage); findDatabasesAndTablesInBackup(); /// Check access rights. @@ -136,27 +137,31 @@ RestorerFromBackup::DataRestoreTasks RestorerFromBackup::run(Mode mode) return {}; /// Create databases using the create queries read from the backup. - setStatus(kCreatingDatabasesStatus); + setStage(kCreatingDatabasesStage); createDatabases(); /// Create tables using the create queries read from the backup. - setStatus(kCreatingTablesStatus); + setStage(kCreatingTablesStage); createTables(); /// All what's left is to insert data to tables. /// No more data restoring tasks are allowed after this point. - setStatus(kInsertingDataToTablesStatus); + setStage(kInsertingDataToTablesStage); return getDataRestoreTasks(); } -void RestorerFromBackup::setStatus(const String & new_status, const String & message) +void RestorerFromBackup::setStage(const String & new_stage, const String & message) { - LOG_TRACE(log, "{}", toUpperFirst(new_status)); - current_status = new_status; + LOG_TRACE(log, "{}", toUpperFirst(new_stage)); + current_stage = new_stage; + if (restore_coordination) { - restore_coordination->setStatus(restore_settings.host_id, new_status, message); - restore_coordination->waitStatus(all_hosts, new_status); + restore_coordination->setStage(restore_settings.host_id, new_stage, message); + if (new_stage == kFindingTablesInBackupStage) + restore_coordination->waitForStage(all_hosts, new_stage, on_cluster_first_sync_timeout); + else + restore_coordination->waitForStage(all_hosts, new_stage); } } @@ -814,14 +819,14 @@ std::vector RestorerFromBackup::findTablesWithoutDependencie void RestorerFromBackup::addDataRestoreTask(DataRestoreTask && new_task) { - if (current_status == kInsertingDataToTablesStatus) + if (current_stage == kInsertingDataToTablesStage) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding of data-restoring tasks is not allowed"); data_restore_tasks.push_back(std::move(new_task)); } void RestorerFromBackup::addDataRestoreTasks(DataRestoreTasks && new_tasks) { - if (current_status == kInsertingDataToTablesStatus) + if (current_stage == kInsertingDataToTablesStage) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding of data-restoring tasks is not allowed"); insertAtEnd(data_restore_tasks, std::move(new_tasks)); } diff --git a/src/Backups/RestorerFromBackup.h b/src/Backups/RestorerFromBackup.h index a53477f6e6d..b081e16e2ce 100644 --- a/src/Backups/RestorerFromBackup.h +++ b/src/Backups/RestorerFromBackup.h @@ -73,6 +73,7 @@ private: std::shared_ptr restore_coordination; BackupPtr backup; ContextMutablePtr context; + std::chrono::milliseconds on_cluster_first_sync_timeout; std::chrono::milliseconds create_table_timeout; Poco::Logger * log; @@ -100,7 +101,7 @@ private: DataRestoreTasks getDataRestoreTasks(); - void setStatus(const String & new_status, const String & message = ""); + void setStage(const String & new_stage, const String & message = ""); struct DatabaseInfo { @@ -124,7 +125,7 @@ private: std::vector findTablesWithoutDependencies() const; - String current_status; + String current_stage; std::unordered_map database_infos; std::map table_infos; std::vector data_restore_tasks; diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index 246d4ba24e9..af3c8df8eef 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -17,20 +17,22 @@ namespace DB namespace { - Block getResultRow(const BackupsWorker::Info & info) + Block getResultRow(const std::optional & info) { - Block res_columns; - auto column_uuid = ColumnUUID::create(); - column_uuid->insert(info.uuid); - res_columns.insert(0, {std::move(column_uuid), std::make_shared(), "uuid"}); - auto column_backup_name = ColumnString::create(); - column_backup_name->insert(info.backup_name); - res_columns.insert(1, {std::move(column_backup_name), std::make_shared(), "backup_name"}); - auto column_status = ColumnInt8::create(); - column_status->insert(static_cast(info.status)); + + if (info) + { + column_uuid->insert(info->uuid); + column_backup_name->insert(info->backup_name); + column_status->insert(static_cast(info->status)); + } + + Block res_columns; + res_columns.insert(0, {std::move(column_uuid), std::make_shared(), "uuid"}); + res_columns.insert(1, {std::move(column_backup_name), std::make_shared(), "backup_name"}); res_columns.insert(2, {std::move(column_status), std::make_shared(getBackupStatusEnumValues()), "status"}); return res_columns; @@ -42,7 +44,7 @@ BlockIO InterpreterBackupQuery::execute() auto & backups_worker = context->getBackupsWorker(); UUID uuid = backups_worker.start(query_ptr, context); BlockIO res_io; - res_io.pipeline = QueryPipeline(std::make_shared(getResultRow(backups_worker.getInfo(uuid)))); + res_io.pipeline = QueryPipeline(std::make_shared(getResultRow(backups_worker.tryGetInfo(uuid)))); return res_io; } From 619e22fe573a3f58695efeec888b00735ba994fc Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 21 Jul 2022 11:43:42 +0200 Subject: [PATCH 488/659] Add tests for shutdown during backup. --- .../configs/lesser_timeouts.xml | 8 +++ .../test_backup_restore_on_cluster/test.py | 62 +++++++++++++++++++ 2 files changed, 70 insertions(+) create mode 100644 tests/integration/test_backup_restore_on_cluster/configs/lesser_timeouts.xml diff --git a/tests/integration/test_backup_restore_on_cluster/configs/lesser_timeouts.xml b/tests/integration/test_backup_restore_on_cluster/configs/lesser_timeouts.xml new file mode 100644 index 00000000000..9caf52fcca4 --- /dev/null +++ b/tests/integration/test_backup_restore_on_cluster/configs/lesser_timeouts.xml @@ -0,0 +1,8 @@ + + + + 1000 + 10000 + 1000 + + diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 438ab87b5c7..df03ebd320b 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -1,5 +1,6 @@ from time import sleep import pytest +import re import os.path from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV, assert_eq_with_retry @@ -11,6 +12,7 @@ main_configs = [ "configs/remote_servers.xml", "configs/replicated_access_storage.xml", "configs/backups_disk.xml", + "configs/lesser_timeouts.xml", # Default timeouts are quite big (a few minutes), the tests don't need them to be that big. ] user_configs = [ @@ -33,6 +35,7 @@ node2 = cluster.add_instance( external_dirs=["/backups/"], macros={"replica": "node2", "shard": "shard1"}, with_zookeeper=True, + stay_alive=True, # Necessary for the "test_stop_other_host_while_backup" test ) @@ -763,3 +766,62 @@ def test_mutation(): node1.query("DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY") node1.query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}") + + +def test_get_error_from_other_host(): + node1.query("CREATE TABLE tbl (`x` UInt8) ENGINE = MergeTree ORDER BY x") + node1.query("INSERT INTO tbl VALUES (3)") + + backup_name = new_backup_name() + expected_error = "Got error from node2.*Table default.tbl was not found" + assert re.search( + expected_error, + node1.query_and_get_error( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" + ), + ) + + +@pytest.mark.parametrize("kill", [False, True]) +def test_stop_other_host_while_backup(kill): + node1.query( + "CREATE TABLE tbl ON CLUSTER 'cluster' (" + "x UInt8" + ") ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}')" + "ORDER BY x" + ) + + node1.query("INSERT INTO tbl VALUES (3)") + node2.query("INSERT INTO tbl VALUES (5)") + + backup_name = new_backup_name() + + id = node1.query( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name} ASYNC" + ).split("\t")[0] + + # If kill=False the pending backup must be completed + # If kill=True the pending backup might be completed or failed + node2.stop_clickhouse(kill=kill) + + assert_eq_with_retry( + node1, + f"SELECT status FROM system.backups WHERE uuid='{id}' AND status == 'MAKING_BACKUP'", + "", + ) + + status = node1.query(f"SELECT status FROM system.backups WHERE uuid='{id}'").strip() + + if kill: + assert status in ["BACKUP_COMPLETE", "FAILED_TO_BACKUP"] + else: + assert status == "BACKUP_COMPLETE" + + node2.start_clickhouse() + + if status == "BACKUP_COMPLETE": + node1.query("DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY") + node1.query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}") + assert node1.query("SELECT * FROM tbl ORDER BY x") == TSV([3, 5]) + elif status == "FAILED_TO_BACKUP": + assert not os.path.exists(get_path_to_backup(backup_name)) From 708d0eb34cf1f6d3c248b5fb0c7f87b59bdf353a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 18 Jul 2022 20:07:37 +0200 Subject: [PATCH 489/659] Add concurrent tests for backups. --- .../test_concurrency.py | 162 ++++++++++++++++++ 1 file changed, 162 insertions(+) create mode 100644 tests/integration/test_backup_restore_on_cluster/test_concurrency.py diff --git a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py new file mode 100644 index 00000000000..95d4f27e5e8 --- /dev/null +++ b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py @@ -0,0 +1,162 @@ +import pytest +import os.path +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV, assert_eq_with_retry + + +cluster = ClickHouseCluster(__file__) + +num_nodes = 10 + + +def generate_cluster_def(): + path = os.path.join( + os.path.dirname(os.path.realpath(__file__)), + "./_gen/cluster_for_concurrency_test.xml", + ) + os.makedirs(os.path.dirname(path), exist_ok=True) + with open(path, "w") as f: + f.write("\n\t\n\t\t\n\t\t\t\n") + for i in range(num_nodes): + f.write( + f"\t\t\t\t\n\t\t\t\t\tnode{i}\n\t\t\t\t\t9000\n\t\t\t\t\n" + ) + f.write("\t\t\t\n\t\t\n\t\n") + return path + + +main_configs = ["configs/backups_disk.xml", generate_cluster_def()] + + +nodes = [] +for i in range(num_nodes): + nodes.append( + cluster.add_instance( + f"node{i}", + main_configs=main_configs, + external_dirs=["/backups/"], + macros={"replica": f"node{i}", "shard": "shard1"}, + with_zookeeper=True, + ) + ) + +node0 = nodes[0] + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def drop_after_test(): + try: + yield + finally: + node0.query("DROP TABLE IF EXISTS tbl ON CLUSTER 'cluster' NO DELAY") + + +backup_id_counter = 0 + + +def new_backup_name(): + global backup_id_counter + backup_id_counter += 1 + return f"Disk('backups', '{backup_id_counter}')" + + +def create_and_fill_table(): + node0.query( + "CREATE TABLE tbl ON CLUSTER 'cluster' (" + "x Int32" + ") ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}')" + "ORDER BY x" + ) + for i in range(num_nodes): + nodes[i].query(f"INSERT INTO tbl VALUES ({i})") + + +expected_sum = num_nodes * (num_nodes - 1) // 2 + + +def test_replicated_table(): + create_and_fill_table() + + backup_name = new_backup_name() + node0.query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}") + + node0.query(f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY") + node0.query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}") + node0.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl") + + for i in range(num_nodes): + assert nodes[i].query("SELECT sum(x) FROM tbl") == TSV([expected_sum]) + + +num_concurrent_backups = 4 + + +def test_concurrent_backups_on_same_node(): + create_and_fill_table() + + backup_names = [new_backup_name() for _ in range(num_concurrent_backups)] + + ids = [] + for backup_name in backup_names: + id = node0.query( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name} ASYNC" + ).split("\t")[0] + ids.append(id) + + ids_list = "[" + ", ".join([f"'{id}'" for id in ids]) + "]" + + assert_eq_with_retry( + node0, + f"SELECT status, error FROM system.backups WHERE status != 'BACKUP_COMPLETE' AND status != 'FAILED_TO_BACKUP' AND uuid IN {ids_list}", + "", + ) + + for backup_name in backup_names: + node0.query(f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY") + node0.query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}") + node0.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl") + for i in range(num_nodes): + assert nodes[i].query("SELECT sum(x) FROM tbl") == TSV([expected_sum]) + + +def test_concurrent_backups_on_different_nodes(): + create_and_fill_table() + + backup_names = [new_backup_name() for _ in range(num_concurrent_backups)] + + ids = [] + for i in range(num_concurrent_backups): + id = ( + nodes[i] + .query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_names[i]} ASYNC") + .split("\t")[0] + ) + ids.append(id) + + for i in range(num_concurrent_backups): + assert_eq_with_retry( + nodes[i], + f"SELECT status, error FROM system.backups WHERE status != 'BACKUP_COMPLETE' AND status != 'FAILED_TO_BACKUP' AND uuid = '{ids[i]}'", + "", + ) + + for i in range(num_concurrent_backups): + assert nodes[i].query( + f"SELECT status, error FROM system.backups WHERE uuid = '{ids[i]}'" + ) == TSV([["BACKUP_COMPLETE", ""]]) + + for i in range(num_concurrent_backups): + nodes[i].query(f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY") + nodes[i].query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_names[i]}") + nodes[i].query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl") + for j in range(num_nodes): + assert nodes[j].query("SELECT sum(x) FROM tbl") == TSV([expected_sum]) From 7acf3dd62497076887b01fa512c30148d1d28e4e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 12:27:45 +0200 Subject: [PATCH 490/659] Rename build reports statuses --- .github/workflows/backport_branches.yml | 2 +- .github/workflows/master.yml | 4 ++-- .github/workflows/pull_request.yml | 4 ++-- .github/workflows/release_branches.yml | 2 +- tests/ci/ci_config.py | 4 ++-- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 54711cea9b4..c0daa468ca6 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -394,7 +394,7 @@ jobs: - name: Set envs run: | cat >> "$GITHUB_ENV" << 'EOF' - CHECK_NAME=ClickHouse build check (actions) + CHECK_NAME=ClickHouse build check REPORTS_PATH=${{runner.temp}}/reports_dir TEMP_PATH=${{runner.temp}}/report_check NEEDS_DATA_PATH=${{runner.temp}}/needs.json diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index b76ff39f0c8..78f6a0547a4 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -971,7 +971,7 @@ jobs: - name: Set envs run: | cat >> "$GITHUB_ENV" << 'EOF' - CHECK_NAME=ClickHouse build check (actions) + CHECK_NAME=ClickHouse build check REPORTS_PATH=${{runner.temp}}/reports_dir REPORTS_PATH=${{runner.temp}}/reports_dir TEMP_PATH=${{runner.temp}}/report_check @@ -1020,7 +1020,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/report_check REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=ClickHouse special build check (actions) + CHECK_NAME=ClickHouse special build check NEEDS_DATA_PATH=${{runner.temp}}/needs.json EOF - name: Download json reports diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 05fd337e57c..625006aede3 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -1026,7 +1026,7 @@ jobs: - name: Set envs run: | cat >> "$GITHUB_ENV" << 'EOF' - CHECK_NAME=ClickHouse build check (actions) + CHECK_NAME=ClickHouse build check REPORTS_PATH=${{runner.temp}}/reports_dir TEMP_PATH=${{runner.temp}}/report_check NEEDS_DATA_PATH=${{runner.temp}}/needs.json @@ -1075,7 +1075,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/report_check REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=ClickHouse special build check (actions) + CHECK_NAME=ClickHouse special build check NEEDS_DATA_PATH=${{runner.temp}}/needs.json EOF - name: Download json reports diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index ac0be791975..bb970b43b2f 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -473,7 +473,7 @@ jobs: - name: Set envs run: | cat >> "$GITHUB_ENV" << 'EOF' - CHECK_NAME=ClickHouse build check (actions) + CHECK_NAME=ClickHouse build check REPORTS_PATH=${{runner.temp}}/reports_dir REPORTS_PATH=${{runner.temp}}/reports_dir TEMP_PATH=${{runner.temp}}/report_check diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index a530b395130..f567fe7a6ae 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -179,7 +179,7 @@ CI_CONFIG = { }, }, "builds_report_config": { - "ClickHouse build check (actions)": [ + "ClickHouse build check": [ "package_release", "coverity", "package_aarch64", @@ -190,7 +190,7 @@ CI_CONFIG = { "package_debug", "binary_release", ], - "ClickHouse special build check (actions)": [ + "ClickHouse special build check": [ "binary_tidy", "binary_splitted", "binary_darwin", From 9259e5ca7733168dd69fa2d4a296f6476996b82e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 12:32:54 +0200 Subject: [PATCH 491/659] Rename stateful tests statuses --- .github/workflows/backport_branches.yml | 2 +- .github/workflows/master.yml | 16 ++++++++-------- .github/workflows/pull_request.yml | 14 +++++++------- .github/workflows/release_branches.yml | 14 +++++++------- tests/ci/ci_config.py | 18 +++++++++--------- 5 files changed, 32 insertions(+), 32 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index c0daa468ca6..8483dcfb7b9 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -477,7 +477,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (debug, actions) + CHECK_NAME=Stateful tests (debug) REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse KILL_TIMEOUT=3600 EOF diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 78f6a0547a4..05bd6ded8ae 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -1678,7 +1678,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (release, actions) + CHECK_NAME=Stateful tests (release) REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1715,7 +1715,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_release_database_ordinary REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (release, DatabaseOrdinary, actions) + CHECK_NAME=Stateful tests (release, DatabaseOrdinary) REPO_COPY=${{runner.temp}}/stateful_release_database_ordinary/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1752,7 +1752,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (aarch64, actions) + CHECK_NAME=Stateful tests (aarch64) REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1789,7 +1789,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (address, actions) + CHECK_NAME=Stateful tests (address) REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1826,7 +1826,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (thread, actions) + CHECK_NAME=Stateful tests (thread) REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1863,7 +1863,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_msan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (memory, actions) + CHECK_NAME=Stateful tests (memory) REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1900,7 +1900,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_ubsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (ubsan, actions) + CHECK_NAME=Stateful tests (ubsan) REPO_COPY=${{runner.temp}}/stateful_ubsan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1937,7 +1937,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (debug, actions) + CHECK_NAME=Stateful tests (debug) REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse KILL_TIMEOUT=3600 EOF diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 625006aede3..391adebbea4 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -1895,7 +1895,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (release, actions) + CHECK_NAME=Stateful tests (release) REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1932,7 +1932,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (aarch64, actions) + CHECK_NAME=Stateful tests (aarch64) REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1969,7 +1969,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (address, actions) + CHECK_NAME=Stateful tests (address) REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse KILL_TIMEOUT=3600 EOF @@ -2006,7 +2006,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (thread, actions) + CHECK_NAME=Stateful tests (thread) REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -2043,7 +2043,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_msan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (memory, actions) + CHECK_NAME=Stateful tests (memory) REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -2080,7 +2080,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_ubsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (ubsan, actions) + CHECK_NAME=Stateful tests (ubsan) REPO_COPY=${{runner.temp}}/stateful_ubsan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -2117,7 +2117,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (debug, actions) + CHECK_NAME=Stateful tests (debug) REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse KILL_TIMEOUT=3600 EOF diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index bb970b43b2f..1803c4056c8 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -1060,7 +1060,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (release, actions) + CHECK_NAME=Stateful tests (release) REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1097,7 +1097,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (aarch64, actions) + CHECK_NAME=Stateful tests (aarch64) REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1134,7 +1134,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (address, actions) + CHECK_NAME=Stateful tests (address) REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1171,7 +1171,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (thread, actions) + CHECK_NAME=Stateful tests (thread) REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1208,7 +1208,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_msan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (memory, actions) + CHECK_NAME=Stateful tests (memory) REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1245,7 +1245,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_ubsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (ubsan, actions) + CHECK_NAME=Stateful tests (ubsan) REPO_COPY=${{runner.temp}}/stateful_ubsan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -1282,7 +1282,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateful_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (debug, actions) + CHECK_NAME=Stateful tests (debug) REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse KILL_TIMEOUT=3600 EOF diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index f567fe7a6ae..6b82964196a 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -203,31 +203,31 @@ CI_CONFIG = { "tests_config": { # required_build - build name for artifacts # force_tests - force success status for tests - "Stateful tests (address, actions)": { + "Stateful tests (address)": { "required_build": "package_asan", }, - "Stateful tests (thread, actions)": { + "Stateful tests (thread)": { "required_build": "package_tsan", }, - "Stateful tests (memory, actions)": { + "Stateful tests (memory)": { "required_build": "package_msan", }, - "Stateful tests (ubsan, actions)": { + "Stateful tests (ubsan)": { "required_build": "package_ubsan", }, - "Stateful tests (debug, actions)": { + "Stateful tests (debug)": { "required_build": "package_debug", }, - "Stateful tests (release, actions)": { + "Stateful tests (release)": { "required_build": "package_release", }, - "Stateful tests (aarch64, actions)": { + "Stateful tests (aarch64)": { "required_build": "package_aarch64", }, - "Stateful tests (release, DatabaseOrdinary, actions)": { + "Stateful tests (release, DatabaseOrdinary)": { "required_build": "package_release", }, - "Stateful tests (release, DatabaseReplicated, actions)": { + "Stateful tests (release, DatabaseReplicated)": { "required_build": "package_release", }, "Stateless tests (address, actions)": { From f3ba18778d7b6f812476ae3b8df4b5887cdaa2c4 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 12:36:25 +0200 Subject: [PATCH 492/659] Rename stateless tests statuses --- .github/workflows/backport_branches.yml | 2 +- .github/workflows/master.yml | 32 +++++++++++----------- .github/workflows/pull_request.yml | 36 ++++++++++++------------- .github/workflows/release_branches.yml | 28 +++++++++---------- tests/ci/ci_config.py | 22 +++++++-------- 5 files changed, 60 insertions(+), 60 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 8483dcfb7b9..64de50a589c 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -437,7 +437,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address, actions) + CHECK_NAME=Stateless tests (address) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 EOF diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 05bd6ded8ae..34c3c394791 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -1061,7 +1061,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, actions) + CHECK_NAME=Stateless tests (release) REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse KILL_TIMEOUT=10800 EOF @@ -1098,7 +1098,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_release_database_ordinary REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, DatabaseOrdinary, actions) + CHECK_NAME=Stateless tests (release, DatabaseOrdinary) REPO_COPY=${{runner.temp}}/stateless_release_database_ordinary/ClickHouse KILL_TIMEOUT=10800 EOF @@ -1135,7 +1135,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_s3_storage REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, s3 storage, actions) + CHECK_NAME=Stateless tests (release, s3 storage) REPO_COPY=${{runner.temp}}/stateless_s3_storage/ClickHouse KILL_TIMEOUT=10800 EOF @@ -1172,7 +1172,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (aarch64, actions) + CHECK_NAME=Stateless tests (aarch64) REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse KILL_TIMEOUT=10800 EOF @@ -1209,7 +1209,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address, actions) + CHECK_NAME=Stateless tests (address) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1248,7 +1248,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address, actions) + CHECK_NAME=Stateless tests (address) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1287,7 +1287,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) + CHECK_NAME=Stateless tests (thread) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1326,7 +1326,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) + CHECK_NAME=Stateless tests (thread) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1365,7 +1365,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) + CHECK_NAME=Stateless tests (thread) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 @@ -1404,7 +1404,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_ubsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (ubsan, actions) + CHECK_NAME=Stateless tests (ubsan) REPO_COPY=${{runner.temp}}/stateless_ubsan/ClickHouse KILL_TIMEOUT=10800 EOF @@ -1441,7 +1441,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) + CHECK_NAME=Stateless tests (memory) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1480,7 +1480,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) + CHECK_NAME=Stateless tests (memory) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1519,7 +1519,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) + CHECK_NAME=Stateless tests (memory) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 @@ -1558,7 +1558,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) + CHECK_NAME=Stateless tests (debug) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1597,7 +1597,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) + CHECK_NAME=Stateless tests (debug) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1636,7 +1636,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) + CHECK_NAME=Stateless tests (debug) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 391adebbea4..9f0b164ab4c 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -1116,7 +1116,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, actions) + CHECK_NAME=Stateless tests (release) REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse KILL_TIMEOUT=10800 EOF @@ -1153,7 +1153,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_database_replicated REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, DatabaseReplicated, actions) + CHECK_NAME=Stateless tests (release, DatabaseReplicated) REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1192,7 +1192,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_database_replicated REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, DatabaseReplicated, actions) + CHECK_NAME=Stateless tests (release, DatabaseReplicated) REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1231,7 +1231,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_wide_parts REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, wide parts enabled, actions) + CHECK_NAME=Stateless tests (release, wide parts enabled) REPO_COPY=${{runner.temp}}/stateless_wide_parts/ClickHouse KILL_TIMEOUT=10800 EOF @@ -1268,7 +1268,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_s3_storage REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, s3 storage, actions) + CHECK_NAME=Stateless tests (release, s3 storage) REPO_COPY=${{runner.temp}}/stateless_s3_storage/ClickHouse KILL_TIMEOUT=10800 EOF @@ -1305,7 +1305,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (aarch64, actions) + CHECK_NAME=Stateless tests (aarch64) REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse KILL_TIMEOUT=10800 EOF @@ -1342,7 +1342,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address, actions) + CHECK_NAME=Stateless tests (address) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1381,7 +1381,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address, actions) + CHECK_NAME=Stateless tests (address) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1420,7 +1420,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) + CHECK_NAME=Stateless tests (thread) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1459,7 +1459,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) + CHECK_NAME=Stateless tests (thread) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1498,7 +1498,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) + CHECK_NAME=Stateless tests (thread) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 @@ -1537,7 +1537,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_ubsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (ubsan, actions) + CHECK_NAME=Stateless tests (ubsan) REPO_COPY=${{runner.temp}}/stateless_ubsan/ClickHouse KILL_TIMEOUT=10800 EOF @@ -1574,7 +1574,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) + CHECK_NAME=Stateless tests (memory) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1613,7 +1613,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) + CHECK_NAME=Stateless tests (memory) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1652,7 +1652,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) + CHECK_NAME=Stateless tests (memory) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 @@ -1691,7 +1691,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) + CHECK_NAME=Stateless tests (debug) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -1730,7 +1730,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) + CHECK_NAME=Stateless tests (debug) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1769,7 +1769,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) + CHECK_NAME=Stateless tests (debug) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 1803c4056c8..f1142f13d55 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -517,7 +517,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, actions) + CHECK_NAME=Stateless tests (release) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 EOF @@ -554,7 +554,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (aarch64, actions) + CHECK_NAME=Stateless tests (aarch64) REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse KILL_TIMEOUT=10800 EOF @@ -591,7 +591,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address, actions) + CHECK_NAME=Stateless tests (address) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -630,7 +630,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address, actions) + CHECK_NAME=Stateless tests (address) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -669,7 +669,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) + CHECK_NAME=Stateless tests (thread) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -708,7 +708,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) + CHECK_NAME=Stateless tests (thread) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -747,7 +747,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) + CHECK_NAME=Stateless tests (thread) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 @@ -786,7 +786,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_ubsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (ubsan, actions) + CHECK_NAME=Stateless tests (ubsan) REPO_COPY=${{runner.temp}}/stateless_ubsan/ClickHouse KILL_TIMEOUT=10800 EOF @@ -823,7 +823,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) + CHECK_NAME=Stateless tests (memory) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -862,7 +862,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) + CHECK_NAME=Stateless tests (memory) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -901,7 +901,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) + CHECK_NAME=Stateless tests (memory) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 @@ -940,7 +940,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) + CHECK_NAME=Stateless tests (debug) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=0 @@ -979,7 +979,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) + CHECK_NAME=Stateless tests (debug) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=1 @@ -1018,7 +1018,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) + CHECK_NAME=Stateless tests (debug) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 RUN_BY_HASH_NUM=2 diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 6b82964196a..b06c6cd10d6 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -230,37 +230,37 @@ CI_CONFIG = { "Stateful tests (release, DatabaseReplicated)": { "required_build": "package_release", }, - "Stateless tests (address, actions)": { + "Stateless tests (address)": { "required_build": "package_asan", }, - "Stateless tests (thread, actions)": { + "Stateless tests (thread)": { "required_build": "package_tsan", }, - "Stateless tests (memory, actions)": { + "Stateless tests (memory)": { "required_build": "package_msan", }, - "Stateless tests (ubsan, actions)": { + "Stateless tests (ubsan)": { "required_build": "package_ubsan", }, - "Stateless tests (debug, actions)": { + "Stateless tests (debug)": { "required_build": "package_debug", }, - "Stateless tests (release, actions)": { + "Stateless tests (release)": { "required_build": "package_release", }, - "Stateless tests (aarch64, actions)": { + "Stateless tests (aarch64)": { "required_build": "package_aarch64", }, - "Stateless tests (release, wide parts enabled, actions)": { + "Stateless tests (release, wide parts enabled)": { "required_build": "package_release", }, - "Stateless tests (release, DatabaseOrdinary, actions)": { + "Stateless tests (release, DatabaseOrdinary)": { "required_build": "package_release", }, - "Stateless tests (release, DatabaseReplicated, actions)": { + "Stateless tests (release, DatabaseReplicated)": { "required_build": "package_release", }, - "Stateless tests (release, s3 storage, actions)": { + "Stateless tests (release, s3 storage)": { "required_build": "package_release", }, "Stress test (address, actions)": { From cc3c47c14358b12fd0989ebbbf685947f2de7aa4 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 12:41:49 +0200 Subject: [PATCH 493/659] Rename stress tests statuses --- .github/workflows/backport_branches.yml | 2 +- .github/workflows/master.yml | 10 +++++----- .github/workflows/pull_request.yml | 10 +++++----- .github/workflows/release_branches.yml | 10 +++++----- tests/ci/ci_config.py | 10 +++++----- 5 files changed, 21 insertions(+), 21 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 64de50a589c..e9f48060ae0 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -521,7 +521,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_thread REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (thread, actions) + CHECK_NAME=Stress test (thread) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 34c3c394791..cde3e2f067a 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -1977,7 +1977,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_thread REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (address, actions) + CHECK_NAME=Stress test (address) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports @@ -2017,7 +2017,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_thread REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (thread, actions) + CHECK_NAME=Stress test (thread) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports @@ -2053,7 +2053,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (memory, actions) + CHECK_NAME=Stress test (memory) REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse EOF - name: Download json reports @@ -2089,7 +2089,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_undefined REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (undefined, actions) + CHECK_NAME=Stress test (undefined) REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse EOF - name: Download json reports @@ -2125,7 +2125,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (debug, actions) + CHECK_NAME=Stress test (debug) REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse EOF - name: Download json reports diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 9f0b164ab4c..b280edc4237 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -2157,7 +2157,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_thread REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (address, actions) + CHECK_NAME=Stress test (address) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports @@ -2197,7 +2197,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_thread REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (thread, actions) + CHECK_NAME=Stress test (thread) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports @@ -2233,7 +2233,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (memory, actions) + CHECK_NAME=Stress test (memory) REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse EOF - name: Download json reports @@ -2269,7 +2269,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_undefined REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (undefined, actions) + CHECK_NAME=Stress test (undefined) REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse EOF - name: Download json reports @@ -2305,7 +2305,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (debug, actions) + CHECK_NAME=Stress test (debug) REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse EOF - name: Download json reports diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index f1142f13d55..a848ce6e819 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -1322,7 +1322,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_thread REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (address, actions) + CHECK_NAME=Stress test (address) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports @@ -1362,7 +1362,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_thread REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (thread, actions) + CHECK_NAME=Stress test (thread) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports @@ -1398,7 +1398,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_memory REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (memory, actions) + CHECK_NAME=Stress test (memory) REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse EOF - name: Download json reports @@ -1434,7 +1434,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_undefined REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (undefined, actions) + CHECK_NAME=Stress test (undefined) REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse EOF - name: Download json reports @@ -1470,7 +1470,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stress_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (debug, actions) + CHECK_NAME=Stress test (debug) REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse EOF - name: Download json reports diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index b06c6cd10d6..cc2d9119058 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -263,19 +263,19 @@ CI_CONFIG = { "Stateless tests (release, s3 storage)": { "required_build": "package_release", }, - "Stress test (address, actions)": { + "Stress test (address)": { "required_build": "package_asan", }, - "Stress test (thread, actions)": { + "Stress test (thread)": { "required_build": "package_tsan", }, - "Stress test (undefined, actions)": { + "Stress test (undefined)": { "required_build": "package_ubsan", }, - "Stress test (memory, actions)": { + "Stress test (memory)": { "required_build": "package_msan", }, - "Stress test (debug, actions)": { + "Stress test (debug)": { "required_build": "package_debug", }, "Integration tests (asan, actions)": { From 69f0cf19f408736ed6bd90fd838ef67078bd0846 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 12:43:42 +0200 Subject: [PATCH 494/659] Rename integration tests statuses --- .github/workflows/backport_branches.yml | 2 +- .github/workflows/master.yml | 18 +++++++++--------- .github/workflows/pull_request.yml | 18 +++++++++--------- .github/workflows/release_branches.yml | 18 +++++++++--------- tests/ci/ci_config.py | 8 ++++---- 5 files changed, 32 insertions(+), 32 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index e9f48060ae0..e1b2b1fad01 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -560,7 +560,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (release, actions) + CHECK_NAME=Integration tests (release) REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse EOF - name: Download json reports diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index cde3e2f067a..90fe239245a 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -2164,7 +2164,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) + CHECK_NAME=Integration tests (asan) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=0 RUN_BY_HASH_TOTAL=3 @@ -2202,7 +2202,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) + CHECK_NAME=Integration tests (asan) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=1 RUN_BY_HASH_TOTAL=3 @@ -2240,7 +2240,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) + CHECK_NAME=Integration tests (asan) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=2 RUN_BY_HASH_TOTAL=3 @@ -2278,7 +2278,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) + CHECK_NAME=Integration tests (thread) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=0 RUN_BY_HASH_TOTAL=4 @@ -2316,7 +2316,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) + CHECK_NAME=Integration tests (thread) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=1 RUN_BY_HASH_TOTAL=4 @@ -2354,7 +2354,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) + CHECK_NAME=Integration tests (thread) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=2 RUN_BY_HASH_TOTAL=4 @@ -2392,7 +2392,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) + CHECK_NAME=Integration tests (thread) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=3 RUN_BY_HASH_TOTAL=4 @@ -2430,7 +2430,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (release, actions) + CHECK_NAME=Integration tests (release) REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse RUN_BY_HASH_NUM=0 RUN_BY_HASH_TOTAL=2 @@ -2468,7 +2468,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (release, actions) + CHECK_NAME=Integration tests (release) REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse RUN_BY_HASH_NUM=1 RUN_BY_HASH_TOTAL=2 diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index b280edc4237..af1f532a984 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -2527,7 +2527,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) + CHECK_NAME=Integration tests (asan) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=0 RUN_BY_HASH_TOTAL=3 @@ -2565,7 +2565,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) + CHECK_NAME=Integration tests (asan) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=1 RUN_BY_HASH_TOTAL=3 @@ -2603,7 +2603,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) + CHECK_NAME=Integration tests (asan) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=2 RUN_BY_HASH_TOTAL=3 @@ -2641,7 +2641,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) + CHECK_NAME=Integration tests (thread) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=0 RUN_BY_HASH_TOTAL=4 @@ -2679,7 +2679,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) + CHECK_NAME=Integration tests (thread) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=1 RUN_BY_HASH_TOTAL=4 @@ -2717,7 +2717,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) + CHECK_NAME=Integration tests (thread) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=2 RUN_BY_HASH_TOTAL=4 @@ -2755,7 +2755,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) + CHECK_NAME=Integration tests (thread) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=3 RUN_BY_HASH_TOTAL=4 @@ -2793,7 +2793,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (release, actions) + CHECK_NAME=Integration tests (release) REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse RUN_BY_HASH_NUM=0 RUN_BY_HASH_TOTAL=2 @@ -2831,7 +2831,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (release, actions) + CHECK_NAME=Integration tests (release) REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse RUN_BY_HASH_NUM=1 RUN_BY_HASH_TOTAL=2 diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index a848ce6e819..fdfedc56f5d 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -1509,7 +1509,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) + CHECK_NAME=Integration tests (asan) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=0 RUN_BY_HASH_TOTAL=3 @@ -1547,7 +1547,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) + CHECK_NAME=Integration tests (asan) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=1 RUN_BY_HASH_TOTAL=3 @@ -1585,7 +1585,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) + CHECK_NAME=Integration tests (asan) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=2 RUN_BY_HASH_TOTAL=3 @@ -1623,7 +1623,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) + CHECK_NAME=Integration tests (thread) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=0 RUN_BY_HASH_TOTAL=4 @@ -1661,7 +1661,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) + CHECK_NAME=Integration tests (thread) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=1 RUN_BY_HASH_TOTAL=4 @@ -1699,7 +1699,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) + CHECK_NAME=Integration tests (thread) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=2 RUN_BY_HASH_TOTAL=4 @@ -1737,7 +1737,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) + CHECK_NAME=Integration tests (thread) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse RUN_BY_HASH_NUM=3 RUN_BY_HASH_TOTAL=4 @@ -1775,7 +1775,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (release, actions) + CHECK_NAME=Integration tests (release) REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse RUN_BY_HASH_NUM=0 RUN_BY_HASH_TOTAL=2 @@ -1813,7 +1813,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_release REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (release, actions) + CHECK_NAME=Integration tests (release) REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse RUN_BY_HASH_NUM=1 RUN_BY_HASH_TOTAL=2 diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index cc2d9119058..f0530c6b50b 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -278,16 +278,16 @@ CI_CONFIG = { "Stress test (debug)": { "required_build": "package_debug", }, - "Integration tests (asan, actions)": { + "Integration tests (asan)": { "required_build": "package_asan", }, - "Integration tests (thread, actions)": { + "Integration tests (thread)": { "required_build": "package_tsan", }, - "Integration tests (release, actions)": { + "Integration tests (release)": { "required_build": "package_release", }, - "Integration tests (memory, actions)": { + "Integration tests (memory)": { "required_build": "package_msan", }, "Integration tests flaky check (asan, actions)": { From 8be0d9287db1230fe125297026fff1b5f462e7de Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 12:45:46 +0200 Subject: [PATCH 495/659] Rename unit tests statuses --- .github/workflows/master.yml | 12 ++++++------ .github/workflows/pull_request.yml | 12 ++++++------ tests/ci/ci_config.py | 10 +++++----- 3 files changed, 17 insertions(+), 17 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 90fe239245a..118f6d8d543 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -2692,7 +2692,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/unit_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (asan, actions) + CHECK_NAME=Unit tests (asan) REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse EOF - name: Download json reports @@ -2728,7 +2728,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/unit_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (release-clang, actions) + CHECK_NAME=Unit tests (release-clang) REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse EOF - name: Download json reports @@ -2764,7 +2764,7 @@ jobs: # cat >> "$GITHUB_ENV" << 'EOF' # TEMP_PATH=${{runner.temp}}/unit_tests_asan # REPORTS_PATH=${{runner.temp}}/reports_dir - # CHECK_NAME=Unit tests (release-gcc, actions) + # CHECK_NAME=Unit tests (release-gcc) # REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse # EOF # - name: Download json reports @@ -2800,7 +2800,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/unit_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (tsan, actions) + CHECK_NAME=Unit tests (tsan) REPO_COPY=${{runner.temp}}/unit_tests_tsan/ClickHouse EOF - name: Download json reports @@ -2836,7 +2836,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/unit_tests_msan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (msan, actions) + CHECK_NAME=Unit tests (msan) REPO_COPY=${{runner.temp}}/unit_tests_msan/ClickHouse EOF - name: Download json reports @@ -2872,7 +2872,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/unit_tests_ubsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (ubsan, actions) + CHECK_NAME=Unit tests (ubsan) REPO_COPY=${{runner.temp}}/unit_tests_ubsan/ClickHouse EOF - name: Download json reports diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index af1f532a984..0235715b03a 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -2908,7 +2908,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/unit_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (asan, actions) + CHECK_NAME=Unit tests (asan) REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse EOF - name: Download json reports @@ -2944,7 +2944,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/unit_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (release-clang, actions) + CHECK_NAME=Unit tests (release-clang) REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse EOF - name: Download json reports @@ -2980,7 +2980,7 @@ jobs: # cat >> "$GITHUB_ENV" << 'EOF' # TEMP_PATH=${{runner.temp}}/unit_tests_asan # REPORTS_PATH=${{runner.temp}}/reports_dir - # CHECK_NAME=Unit tests (release-gcc, actions) + # CHECK_NAME=Unit tests (release-gcc) # REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse # EOF # - name: Download json reports @@ -3016,7 +3016,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/unit_tests_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (tsan, actions) + CHECK_NAME=Unit tests (tsan) REPO_COPY=${{runner.temp}}/unit_tests_tsan/ClickHouse EOF - name: Download json reports @@ -3052,7 +3052,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/unit_tests_msan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (msan, actions) + CHECK_NAME=Unit tests (msan) REPO_COPY=${{runner.temp}}/unit_tests_msan/ClickHouse EOF - name: Download json reports @@ -3088,7 +3088,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/unit_tests_ubsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (ubsan, actions) + CHECK_NAME=Unit tests (ubsan) REPO_COPY=${{runner.temp}}/unit_tests_ubsan/ClickHouse EOF - name: Download json reports diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index f0530c6b50b..45f329c24b0 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -302,19 +302,19 @@ CI_CONFIG = { "Testflows check (actions)": { "required_build": "package_release", }, - "Unit tests (release-clang, actions)": { + "Unit tests (release-clang)": { "required_build": "binary_release", }, - "Unit tests (asan, actions)": { + "Unit tests (asan)": { "required_build": "package_asan", }, - "Unit tests (msan, actions)": { + "Unit tests (msan)": { "required_build": "package_msan", }, - "Unit tests (tsan, actions)": { + "Unit tests (tsan)": { "required_build": "package_tsan", }, - "Unit tests (ubsan, actions)": { + "Unit tests (ubsan)": { "required_build": "package_ubsan", }, "AST fuzzer (debug, actions)": { From 3ab288b095694a61446ea464934e1ba7ebff936d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 12:47:49 +0200 Subject: [PATCH 496/659] Rename AST fuzzer statuses --- .github/workflows/master.yml | 10 +++++----- .github/workflows/pull_request.yml | 10 +++++----- tests/ci/ci_config.py | 10 +++++----- 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 118f6d8d543..6272d3460f7 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -2509,7 +2509,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (ASan, actions) + CHECK_NAME=AST fuzzer (ASan) REPO_COPY=${{runner.temp}}/ast_fuzzer_asan/ClickHouse EOF - name: Download json reports @@ -2545,7 +2545,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (TSan, actions) + CHECK_NAME=AST fuzzer (TSan) REPO_COPY=${{runner.temp}}/ast_fuzzer_tsan/ClickHouse EOF - name: Download json reports @@ -2581,7 +2581,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_ubsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (UBSan, actions) + CHECK_NAME=AST fuzzer (UBSan) REPO_COPY=${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse EOF - name: Download json reports @@ -2617,7 +2617,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_msan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (MSan, actions) + CHECK_NAME=AST fuzzer (MSan) REPO_COPY=${{runner.temp}}/ast_fuzzer_msan/ClickHouse EOF - name: Download json reports @@ -2653,7 +2653,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (debug, actions) + CHECK_NAME=AST fuzzer (debug) REPO_COPY=${{runner.temp}}/ast_fuzzer_debug/ClickHouse EOF - name: Download json reports diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 0235715b03a..545927cfb5e 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -2344,7 +2344,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (ASan, actions) + CHECK_NAME=AST fuzzer (ASan) REPO_COPY=${{runner.temp}}/ast_fuzzer_asan/ClickHouse EOF - name: Download json reports @@ -2380,7 +2380,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_tsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (TSan, actions) + CHECK_NAME=AST fuzzer (TSan) REPO_COPY=${{runner.temp}}/ast_fuzzer_tsan/ClickHouse EOF - name: Download json reports @@ -2416,7 +2416,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_ubsan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (UBSan, actions) + CHECK_NAME=AST fuzzer (UBSan) REPO_COPY=${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse EOF - name: Download json reports @@ -2452,7 +2452,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_msan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (MSan, actions) + CHECK_NAME=AST fuzzer (MSan) REPO_COPY=${{runner.temp}}/ast_fuzzer_msan/ClickHouse EOF - name: Download json reports @@ -2488,7 +2488,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/ast_fuzzer_debug REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (debug, actions) + CHECK_NAME=AST fuzzer (debug) REPO_COPY=${{runner.temp}}/ast_fuzzer_debug/ClickHouse EOF - name: Download json reports diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 45f329c24b0..61aafb250b8 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -317,19 +317,19 @@ CI_CONFIG = { "Unit tests (ubsan)": { "required_build": "package_ubsan", }, - "AST fuzzer (debug, actions)": { + "AST fuzzer (debug)": { "required_build": "package_debug", }, - "AST fuzzer (ASan, actions)": { + "AST fuzzer (ASan)": { "required_build": "package_asan", }, - "AST fuzzer (MSan, actions)": { + "AST fuzzer (MSan)": { "required_build": "package_msan", }, - "AST fuzzer (TSan, actions)": { + "AST fuzzer (TSan)": { "required_build": "package_tsan", }, - "AST fuzzer (UBSan, actions)": { + "AST fuzzer (UBSan)": { "required_build": "package_ubsan", }, "Release (actions)": { From 9679401a671b0c761b51e9ecb20ca4921d1685ad Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 12:58:36 +0200 Subject: [PATCH 497/659] Rename flaky tests statuses --- .github/workflows/pull_request.yml | 4 ++-- tests/ci/ci_config.py | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 545927cfb5e..90d5a4e2dd8 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -1808,7 +1808,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/stateless_flaky_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests flaky check (address, actions) + CHECK_NAME=Stateless tests flaky check (address) REPO_COPY=${{runner.temp}}/stateless_flaky_asan/ClickHouse KILL_TIMEOUT=3600 EOF @@ -2869,7 +2869,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan_flaky_check REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests flaky check (asan, actions) + CHECK_NAME=Integration tests flaky check (asan) REPO_COPY=${{runner.temp}}/integration_tests_asan_flaky_check/ClickHouse EOF - name: Download json reports diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 61aafb250b8..a1a83709f59 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -290,7 +290,7 @@ CI_CONFIG = { "Integration tests (memory)": { "required_build": "package_msan", }, - "Integration tests flaky check (asan, actions)": { + "Integration tests flaky check (asan)": { "required_build": "package_asan", }, "Compatibility check (actions)": { @@ -335,7 +335,7 @@ CI_CONFIG = { "Release (actions)": { "required_build": "package_release", }, - "Stateless tests flaky check (address, actions)": { + "Stateless tests flaky check (address)": { "required_build": "package_asan", }, "Stateless tests bugfix validate check (address, actions)": { From 558c0bdb7cc35f004b2819f8db8966233c057d25 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 21 Jul 2022 13:00:36 +0200 Subject: [PATCH 498/659] Continue updating the changelog --- CHANGELOG.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b3ad0d3a4e9..ce4b1c9c021 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -21,7 +21,7 @@ * Added full sorting merge join algorithm. [#35796](https://github.com/ClickHouse/ClickHouse/pull/35796) ([Vladimir C](https://github.com/vdimir)). * Implement NATS table engine, which allows to pub/sub to NATS. Closes [#32388](https://github.com/ClickHouse/ClickHouse/issues/32388). [#37171](https://github.com/ClickHouse/ClickHouse/pull/37171) ([tchepavel](https://github.com/tchepavel)). ([Kseniia Sumarokova](https://github.com/kssenii)) * Implement table function `mongodb`. Allow writes into `MongoDB` storage / table function. [#37213](https://github.com/ClickHouse/ClickHouse/pull/37213) ([aaapetrenko](https://github.com/aaapetrenko)). ([Kseniia Sumarokova](https://github.com/kssenii)) -* Add SQLInsert output format. Closes [#38441](https://github.com/ClickHouse/ClickHouse/issues/38441). [#38477](https://github.com/ClickHouse/ClickHouse/pull/38477) ([Kruglov Pavel](https://github.com/Avogar)). +* Add `SQLInsert` output format. Closes [#38441](https://github.com/ClickHouse/ClickHouse/issues/38441). [#38477](https://github.com/ClickHouse/ClickHouse/pull/38477) ([Kruglov Pavel](https://github.com/Avogar)). * Add `compatibility` setting and `system.settings_changes` system table that contains information about changes in settings through ClickHouse versions. Closes [#35972](https://github.com/ClickHouse/ClickHouse/issues/35972). [#38957](https://github.com/ClickHouse/ClickHouse/pull/38957) ([Kruglov Pavel](https://github.com/Avogar)). * Add functions `translate(string, from_string, to_string)` and `translateUTF8(string, from_string, to_string)`. It translates some characters to another. [#38935](https://github.com/ClickHouse/ClickHouse/pull/38935) ([Nikolay Degterinsky](https://github.com/evillique)). * Support `parseTimeDelta` function. It can be used like ` ;-+,:` can be used as separators, eg. `1yr-2mo`, `2m:6s`: `SELECT parseTimeDelta('1yr-2mo-4w + 12 days, 3 hours : 1 minute ; 33 seconds')`. [#39071](https://github.com/ClickHouse/ClickHouse/pull/39071) ([jiahui-97](https://github.com/jiahui-97)). @@ -47,7 +47,8 @@ #### Performance Improvement * Distinct optimization for sorted columns. Use specialized distinct transformation in case input stream is sorted by column(s) in distinct. Optimization can be applied to pre-distinct, final distinct, or both. Initial implementation by @dimarub2000. [#37803](https://github.com/ClickHouse/ClickHouse/pull/37803) ([Igor Nikonov](https://github.com/devcrafter)). * Improve performance of `ORDER BY`, `MergeTree` merges, window functions using batch version of `BinaryHeap`. [#38022](https://github.com/ClickHouse/ClickHouse/pull/38022) ([Maksim Kita](https://github.com/kitaisreal)). -* Fix significant join performance regression which was introduced in https://github.com/ClickHouse/ClickHouse/pull/35616 . It's interesting that common join queries such as ssb queries have been 10 times slower for almost 3 months while no one complains. [#38052](https://github.com/ClickHouse/ClickHouse/pull/38052) ([Amos Bird](https://github.com/amosbird)). +* More parallel execution for queries with `FINAL` [#36396](https://github.com/ClickHouse/ClickHouse/pull/36396) ([Nikita Taranov](https://github.com/nickitat)). +* Fix significant join performance regression which was introduced in [#35616](https://github.com/ClickHouse/ClickHouse/pull/35616). It's interesting that common join queries such as ssb queries have been 10 times slower for almost 3 months while no one complains. [#38052](https://github.com/ClickHouse/ClickHouse/pull/38052) ([Amos Bird](https://github.com/amosbird)). * Migrate from the Intel hyperscan library to vectorscan, this speeds up many string matching on non-x86 platforms. [#38171](https://github.com/ClickHouse/ClickHouse/pull/38171) ([Robert Schulze](https://github.com/rschu1ze)). * Increased parallelism of query plan steps executed after aggregation. [#38295](https://github.com/ClickHouse/ClickHouse/pull/38295) ([Nikita Taranov](https://github.com/nickitat)). * Improve performance of insertion to columns of type `JSON`. [#38320](https://github.com/ClickHouse/ClickHouse/pull/38320) ([Anton Popov](https://github.com/CurtizJ)). @@ -59,7 +60,6 @@ * `ORDER BY (a, b)` will use all the same benefits as `ORDER BY a, b`. [#38873](https://github.com/ClickHouse/ClickHouse/pull/38873) ([Igor Nikonov](https://github.com/devcrafter)). * Align branches within a 32B boundary to make benchmark more stable. [#38988](https://github.com/ClickHouse/ClickHouse/pull/38988) ([Guo Wangyang](https://github.com/guowangy)). It improves performance 1..2% on average for Intel. * Executable UDF, executable dictionaries, and Executable tables will avoid wasting one second during wait for subprocess termination. [#38929](https://github.com/ClickHouse/ClickHouse/pull/38929) ([Constantine Peresypkin](https://github.com/pkit)). -* TODO remove? Pushdown filter to the right side of sorting join. [#39123](https://github.com/ClickHouse/ClickHouse/pull/39123) ([Vladimir C](https://github.com/vdimir)). * Optimize accesses to `system.stack_trace` table if not all columns are selected. [#39177](https://github.com/ClickHouse/ClickHouse/pull/39177) ([Azat Khuzhin](https://github.com/azat)). * Improve isNullable/isConstant/isNull/isNotNull performance for LowCardinality argument. [#39192](https://github.com/ClickHouse/ClickHouse/pull/39192) ([Kruglov Pavel](https://github.com/Avogar)). * Optimized processing of ORDER BY in window functions. [#34632](https://github.com/ClickHouse/ClickHouse/pull/34632) ([Vladimir Chebotarev](https://github.com/excitoon)). From 31be0114306bf15c78d5accd46a55fe3c3cb89b8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 13:04:34 +0200 Subject: [PATCH 499/659] Rename bugfix checks statuses, clear unised ci config --- .github/workflows/pull_request.yml | 6 +++--- tests/ci/ci_config.py | 3 --- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 90d5a4e2dd8..9d805fd2318 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -1844,7 +1844,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/tests_bugfix_check REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Tests bugfix validate check (actions) + CHECK_NAME=tests bugfix validate check KILL_TIMEOUT=3600 REPO_COPY=${{runner.temp}}/tests_bugfix_check/ClickHouse EOF @@ -1866,12 +1866,12 @@ jobs: TEMP_PATH="${TEMP_PATH}/integration" \ REPORTS_PATH="${REPORTS_PATH}/integration" \ - python3 integration_test_check.py "Integration tests bugfix validate check" \ + python3 integration_test_check.py "Integration $CHECK_NAME" \ --validate-bugfix --post-commit-status=file || echo 'ignore exit code' TEMP_PATH="${TEMP_PATH}/stateless" \ REPORTS_PATH="${REPORTS_PATH}/stateless" \ - python3 functional_test_check.py "Stateless tests bugfix validate check" "$KILL_TIMEOUT" \ + python3 functional_test_check.py "Stateless $CHECK_NAME" "$KILL_TIMEOUT" \ --validate-bugfix --post-commit-status=file || echo 'ignore exit code' python3 bugfix_validate_check.py "${TEMP_PATH}/stateless/post_commit_status.tsv" "${TEMP_PATH}/integration/post_commit_status.tsv" diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index a1a83709f59..7f603c2bab6 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -338,9 +338,6 @@ CI_CONFIG = { "Stateless tests flaky check (address)": { "required_build": "package_asan", }, - "Stateless tests bugfix validate check (address, actions)": { - "required_build": "package_asan", - }, "ClickHouse Keeper Jepsen (actions)": { "required_build": "binary_release", }, From b0eef42b9bae163ee7c8e314eddc0d6ad7ac01fc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 13:07:14 +0200 Subject: [PATCH 500/659] Remove unused Testflows and Release ci configs --- tests/ci/ci_config.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 7f603c2bab6..0222597f234 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -299,9 +299,6 @@ CI_CONFIG = { "Split build smoke test (actions)": { "required_build": "binary_splitted", }, - "Testflows check (actions)": { - "required_build": "package_release", - }, "Unit tests (release-clang)": { "required_build": "binary_release", }, @@ -332,9 +329,6 @@ CI_CONFIG = { "AST fuzzer (UBSan)": { "required_build": "package_ubsan", }, - "Release (actions)": { - "required_build": "package_release", - }, "Stateless tests flaky check (address)": { "required_build": "package_asan", }, From d42018a2dcd44ff9837f4836b84593fa440eb482 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 13:10:22 +0200 Subject: [PATCH 501/659] Cleanup the rest of (actions) status names --- tests/ci/ci_config.py | 6 +++--- tests/ci/codebrowser_check.py | 2 +- tests/ci/compatibility_check.py | 2 +- tests/ci/docker_images_check.py | 2 +- tests/ci/docker_manifests_merge.py | 2 +- tests/ci/docker_server.py | 4 ++-- tests/ci/docs_check.py | 2 +- tests/ci/docs_release.py | 2 +- tests/ci/fast_test_check.py | 2 +- tests/ci/finish_check.py | 2 +- tests/ci/keeper_jepsen_check.py | 2 +- tests/ci/run_check.py | 2 +- tests/ci/split_build_smoke_check.py | 2 +- tests/ci/style_check.py | 2 +- 14 files changed, 17 insertions(+), 17 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 0222597f234..6b1fffce8fd 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -293,10 +293,10 @@ CI_CONFIG = { "Integration tests flaky check (asan)": { "required_build": "package_asan", }, - "Compatibility check (actions)": { + "Compatibility check": { "required_build": "package_release", }, - "Split build smoke test (actions)": { + "Split build smoke test": { "required_build": "binary_splitted", }, "Unit tests (release-clang)": { @@ -332,7 +332,7 @@ CI_CONFIG = { "Stateless tests flaky check (address)": { "required_build": "package_asan", }, - "ClickHouse Keeper Jepsen (actions)": { + "ClickHouse Keeper Jepsen": { "required_build": "binary_release", }, "Performance Comparison": { diff --git a/tests/ci/codebrowser_check.py b/tests/ci/codebrowser_check.py index 48c92e9f6ac..6e7f98bd82a 100644 --- a/tests/ci/codebrowser_check.py +++ b/tests/ci/codebrowser_check.py @@ -16,7 +16,7 @@ from commit_status_helper import post_commit_status from docker_pull_helper import get_image_with_version from tee_popen import TeePopen -NAME = "Woboq Build (actions)" +NAME = "Woboq Build" def get_run_command(repo_path, output_path, image): diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index 5490f162e42..71a959a064c 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -28,7 +28,7 @@ IMAGE_UBUNTU = "clickhouse/test-old-ubuntu" IMAGE_CENTOS = "clickhouse/test-old-centos" MAX_GLIBC_VERSION = "2.4" DOWNLOAD_RETRIES_COUNT = 5 -CHECK_NAME = "Compatibility check (actions)" +CHECK_NAME = "Compatibility check" def process_os_check(log_path): diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 2e181f678dd..5742bc6c22e 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -21,7 +21,7 @@ from s3_helper import S3Helper from stopwatch import Stopwatch from upload_result_helper import upload_results -NAME = "Push to Dockerhub (actions)" +NAME = "Push to Dockerhub" TEMP_PATH = os.path.join(RUNNER_TEMP, "docker_images_check") diff --git a/tests/ci/docker_manifests_merge.py b/tests/ci/docker_manifests_merge.py index 9371440346e..aa13bbea2fb 100644 --- a/tests/ci/docker_manifests_merge.py +++ b/tests/ci/docker_manifests_merge.py @@ -18,7 +18,7 @@ from s3_helper import S3Helper from stopwatch import Stopwatch from upload_result_helper import upload_results -NAME = "Push multi-arch images to Dockerhub (actions)" +NAME = "Push multi-arch images to Dockerhub" CHANGED_IMAGES = "changed_images_{}.json" Images = Dict[str, List[str]] diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index a54a8989565..710c18a56cb 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -303,7 +303,7 @@ def main(): image = DockerImage(args.image_path, args.image_repo, False) args.release_type = auto_release_type(args.version, args.release_type) tags = gen_tags(args.version, args.release_type) - NAME = f"Docker image {image.repo} building check (actions)" + NAME = f"Docker image {image.repo} building check" pr_info = None if CI: pr_info = PRInfo() @@ -320,7 +320,7 @@ def main(): encoding="utf-8", shell=True, ) - NAME = f"Docker image {image.repo} build and push (actions)" + NAME = f"Docker image {image.repo} build and push" logging.info("Following tags will be created: %s", ", ".join(tags)) status = "success" diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index c67e9dcc99d..d6131535ef8 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -19,7 +19,7 @@ from rerun_helper import RerunHelper from tee_popen import TeePopen -NAME = "Docs Check (actions)" +NAME = "Docs Check" if __name__ == "__main__": parser = argparse.ArgumentParser( diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py index 806db28c1b1..4a9686ec99f 100644 --- a/tests/ci/docs_release.py +++ b/tests/ci/docs_release.py @@ -18,7 +18,7 @@ from commit_status_helper import get_commit from rerun_helper import RerunHelper from tee_popen import TeePopen -NAME = "Docs Release (actions)" +NAME = "Docs Release" def parse_args() -> argparse.Namespace: diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index ce5a4195ceb..4976a6ecdc7 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -25,7 +25,7 @@ from rerun_helper import RerunHelper from tee_popen import TeePopen from ccache_utils import get_ccache_if_not_exists, upload_ccache -NAME = "Fast test (actions)" +NAME = "Fast test" def get_fasttest_cmd( diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 289e32406ef..0697f52abed 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -7,7 +7,7 @@ from pr_info import PRInfo from get_robot_token import get_best_robot_token from commit_status_helper import get_commit -NAME = "Run Check (actions)" +NAME = "Run Check" def filter_statuses(statuses): diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py index 14c31927b75..3c2f72f73d1 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/keeper_jepsen_check.py @@ -27,7 +27,7 @@ from rerun_helper import RerunHelper JEPSEN_GROUP_NAME = "jepsen_group" DESIRED_INSTANCE_COUNT = 3 IMAGE_NAME = "clickhouse/keeper-jepsen-test" -CHECK_NAME = "ClickHouse Keeper Jepsen (actions)" +CHECK_NAME = "ClickHouse Keeper Jepsen" SUCCESSFUL_TESTS_ANCHOR = "# Successful tests" diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index b6d654c7bed..ffa1b5eb2be 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -12,7 +12,7 @@ from get_robot_token import get_best_robot_token from pr_info import FORCE_TESTS_LABEL, PRInfo from workflow_approve_rerun_lambda.app import TRUSTED_CONTRIBUTORS -NAME = "Run Check (actions)" +NAME = "Run Check" TRUSTED_ORG_IDS = { 7409213, # yandex diff --git a/tests/ci/split_build_smoke_check.py b/tests/ci/split_build_smoke_check.py index 210a6f9ea86..9237df23a26 100644 --- a/tests/ci/split_build_smoke_check.py +++ b/tests/ci/split_build_smoke_check.py @@ -23,7 +23,7 @@ from rerun_helper import RerunHelper DOCKER_IMAGE = "clickhouse/split-build-smoke-test" DOWNLOAD_RETRIES_COUNT = 5 RESULT_LOG_NAME = "run.log" -CHECK_NAME = "Split build smoke test (actions)" +CHECK_NAME = "Split build smoke test" def process_result(result_folder, server_log_folder): diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 84ed9e5a124..7b5e4b45a18 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -27,7 +27,7 @@ from clickhouse_helper import ( from stopwatch import Stopwatch from rerun_helper import RerunHelper -NAME = "Style Check (actions)" +NAME = "Style Check" def process_result(result_folder): From e6969d50106231966cdc5be82f30403ba8800716 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 21 Jul 2022 11:23:09 +0000 Subject: [PATCH 502/659] Respect table alias for additional_table_filters. --- src/Interpreters/InterpreterSelectQuery.cpp | 3 ++- .../queries/0_stateless/02346_additional_filters.reference | 6 ++++++ tests/queries/0_stateless/02346_additional_filters.sql | 1 + 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 56b6cd3c136..24bbaea7dcf 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -297,7 +297,8 @@ static ASTPtr parseAdditionalFilterConditionForTable( auto & table = tuple.at(0).safeGet(); auto & filter = tuple.at(1).safeGet(); - if ((table == target.table && context.getCurrentDatabase() == target.database) || + if (table == target.alias || + (table == target.table && context.getCurrentDatabase() == target.database) || (table == target.database + '.' + target.table)) { /// Try to parse expression diff --git a/tests/queries/0_stateless/02346_additional_filters.reference b/tests/queries/0_stateless/02346_additional_filters.reference index 0cd345f71cd..22d53173e71 100644 --- a/tests/queries/0_stateless/02346_additional_filters.reference +++ b/tests/queries/0_stateless/02346_additional_filters.reference @@ -66,6 +66,12 @@ select * from system.numbers limit 5; 2 3 4 +select * from system.numbers as t limit 5 settings additional_table_filters={'t' : 'number % 2 != 0'}; +1 +3 +5 +7 +9 select * from system.numbers limit 5 settings additional_table_filters={'system.numbers' : 'number != 3'}; 0 1 diff --git a/tests/queries/0_stateless/02346_additional_filters.sql b/tests/queries/0_stateless/02346_additional_filters.sql index 24e04b9dc8b..9e0bee4549b 100644 --- a/tests/queries/0_stateless/02346_additional_filters.sql +++ b/tests/queries/0_stateless/02346_additional_filters.sql @@ -30,6 +30,7 @@ select * from remote('127.0.0.{1,2}', system.one) settings additional_table_filt select * from remote('127.0.0.{1,2}', system.one) settings additional_table_filters={'system.one' : 'dummy != 0'}; select * from system.numbers limit 5; +select * from system.numbers as t limit 5 settings additional_table_filters={'t' : 'number % 2 != 0'}; select * from system.numbers limit 5 settings additional_table_filters={'system.numbers' : 'number != 3'}; select * from system.numbers limit 5 settings additional_table_filters={'system.numbers':'number != 3','table_1':'x!=2'}; select * from (select number from system.numbers limit 5 union all select x from table_1) order by number settings additional_table_filters={'system.numbers':'number != 3','table_1':'x!=2'}; From 03bd7ce7b802c19b94eb836960a60466e5970dba Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 21 Jul 2022 08:29:40 -0300 Subject: [PATCH 503/659] Minor style fix in dns integ test --- .../test_host_regexp_multiple_ptr_records/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/test.py b/tests/integration/test_host_regexp_multiple_ptr_records/test.py index a07ea5b924d..fa2917411e4 100644 --- a/tests/integration/test_host_regexp_multiple_ptr_records/test.py +++ b/tests/integration/test_host_regexp_multiple_ptr_records/test.py @@ -39,8 +39,12 @@ def setup_dns_server(ip): def setup_ch_server(dns_server_ip): - ch_server.exec_in_container((["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"])) - ch_server.exec_in_container((["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"])) + ch_server.exec_in_container( + (["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"]) + ) + ch_server.exec_in_container( + (["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"]) + ) ch_server.query("SYSTEM DROP DNS CACHE") From ad77b84b8289fcad46df75b3b003230829846c01 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 21 Jul 2022 13:32:34 +0200 Subject: [PATCH 504/659] Update changelog --- CHANGELOG.md | 3 ++- utils/security-generator/SECURITY.md.sh | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ce4b1c9c021..56a6e27d8f1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -22,6 +22,7 @@ * Implement NATS table engine, which allows to pub/sub to NATS. Closes [#32388](https://github.com/ClickHouse/ClickHouse/issues/32388). [#37171](https://github.com/ClickHouse/ClickHouse/pull/37171) ([tchepavel](https://github.com/tchepavel)). ([Kseniia Sumarokova](https://github.com/kssenii)) * Implement table function `mongodb`. Allow writes into `MongoDB` storage / table function. [#37213](https://github.com/ClickHouse/ClickHouse/pull/37213) ([aaapetrenko](https://github.com/aaapetrenko)). ([Kseniia Sumarokova](https://github.com/kssenii)) * Add `SQLInsert` output format. Closes [#38441](https://github.com/ClickHouse/ClickHouse/issues/38441). [#38477](https://github.com/ClickHouse/ClickHouse/pull/38477) ([Kruglov Pavel](https://github.com/Avogar)). +* Introduced settings `additional_table_filters`. Using this setting, you can specify additional filtering condition for a table which will be applied directly after reading. Example: `select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x settings additional_table_filters={'system.numbers : 'number != 3', 'table_1' : 'x != 2'}`. Introduced setting `additional_result_filter` which specifies additional filtering condition for query result. Closes [#37918](https://github.com/ClickHouse/ClickHouse/issues/37918). [#38475](https://github.com/ClickHouse/ClickHouse/pull/38475) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Add `compatibility` setting and `system.settings_changes` system table that contains information about changes in settings through ClickHouse versions. Closes [#35972](https://github.com/ClickHouse/ClickHouse/issues/35972). [#38957](https://github.com/ClickHouse/ClickHouse/pull/38957) ([Kruglov Pavel](https://github.com/Avogar)). * Add functions `translate(string, from_string, to_string)` and `translateUTF8(string, from_string, to_string)`. It translates some characters to another. [#38935](https://github.com/ClickHouse/ClickHouse/pull/38935) ([Nikolay Degterinsky](https://github.com/evillique)). * Support `parseTimeDelta` function. It can be used like ` ;-+,:` can be used as separators, eg. `1yr-2mo`, `2m:6s`: `SELECT parseTimeDelta('1yr-2mo-4w + 12 days, 3 hours : 1 minute ; 33 seconds')`. [#39071](https://github.com/ClickHouse/ClickHouse/pull/39071) ([jiahui-97](https://github.com/jiahui-97)). @@ -35,7 +36,7 @@ * Add `send_logs_source_regexp` setting. Send server text logs with specified regexp to match log source name. Empty means all sources. [#39161](https://github.com/ClickHouse/ClickHouse/pull/39161) ([Amos Bird](https://github.com/amosbird)). * Support `ALTER` for `Hive` tables. [#38214](https://github.com/ClickHouse/ClickHouse/pull/38214) ([lgbo](https://github.com/lgbo-ustc)). * Support `isNullable` function. This function checks whether it's argument is nullable and return 1 or 0. Closes [#38611](https://github.com/ClickHouse/ClickHouse/issues/38611). [#38841](https://github.com/ClickHouse/ClickHouse/pull/38841) ([lokax](https://github.com/lokax)). -* Added Base58 encoding/decoding. [#38159](https://github.com/ClickHouse/ClickHouse/pull/38159) ([Andrey Zvonov](https://github.com/zvonand)). +* Added functions for base58 encoding/decoding. [#38159](https://github.com/ClickHouse/ClickHouse/pull/38159) ([Andrey Zvonov](https://github.com/zvonand)). * Add chart visualization to Play UI. [#38197](https://github.com/ClickHouse/ClickHouse/pull/38197) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Added L2 Squared distance and norm functions for both arrays and tuples. [#38545](https://github.com/ClickHouse/ClickHouse/pull/38545) ([Julian Gilyadov](https://github.com/israelg99)). * Add ability to pass HTTP headers to the `url` table function / storage via SQL. Closes [#37897](https://github.com/ClickHouse/ClickHouse/issues/37897). [#38176](https://github.com/ClickHouse/ClickHouse/pull/38176) ([Kseniia Sumarokova](https://github.com/kssenii)). diff --git a/utils/security-generator/SECURITY.md.sh b/utils/security-generator/SECURITY.md.sh index 381f5b4eaa6..15933da7942 100755 --- a/utils/security-generator/SECURITY.md.sh +++ b/utils/security-generator/SECURITY.md.sh @@ -33,7 +33,7 @@ FROM FROM ( WITH - extractGroups(version, 'v(\\d+).(\\d+)') AS v, + extractGroups(version, 'v(\\d+)\\.(\\d+)') AS v, v[1]::UInt8 AS y, v[2]::UInt8 AS m SELECT From f79f12408d8b250ea556b3f8addf6ca7ab37f3ed Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Jul 2022 13:57:24 +0200 Subject: [PATCH 505/659] fixes --- src/Storages/getStructureOfRemoteTable.cpp | 8 +++----- .../test_distributed_respect_user_timeouts/test.py | 10 +--------- 2 files changed, 4 insertions(+), 14 deletions(-) diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 82a20b8d578..4ccd8a71ede 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -123,15 +123,13 @@ ColumnsDescription getStructureOfRemoteTable( std::string fail_messages; - // use local shard as first priority, as it needs no network communication + /// Use local shard as first priority, as it needs no network communication for (const auto & shard_info : shards_info) { - if(shard_info.isLocal()) + if (shard_info.isLocal()) { const auto & res = getStructureOfRemoteTableInShard(cluster, shard_info, table_id, context, table_func_ptr); - if (res.empty()) - break; - + chassert(!res.empty()); return res; } } diff --git a/tests/integration/test_distributed_respect_user_timeouts/test.py b/tests/integration/test_distributed_respect_user_timeouts/test.py index ea79a9544d5..593843b4e4a 100644 --- a/tests/integration/test_distributed_respect_user_timeouts/test.py +++ b/tests/integration/test_distributed_respect_user_timeouts/test.py @@ -129,15 +129,7 @@ def started_cluster(request): def _check_timeout_and_exception(node, user, query_base, query): repeats = EXPECTED_BEHAVIOR[user]["times"] - extra_repeats = 1 - # Table function remote() are executed two times. - # It tries to get table structure from remote shards. - # On 'node2' it will firstly try to get structure from 'node1' (which is not available), - # so there are 1 extra connection attempts for 'node2' and 'remote' - if node.name == "node2" and query_base == "remote": - extra_repeats = 2 - - expected_timeout = EXPECTED_BEHAVIOR[user]["timeout"] * repeats * extra_repeats + expected_timeout = EXPECTED_BEHAVIOR[user]["timeout"] * repeats start = timeit.default_timer() exception = node.query_and_get_error(query, user=user) From a8da5d96fce7e72859dca8b83c8b1d509b926c69 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Jul 2022 15:05:48 +0200 Subject: [PATCH 506/659] remove some dead and commented code --- src/Access/LDAPClient.cpp | 1 - src/Client/ClientBase.cpp | 1 - src/Columns/ColumnLowCardinality.cpp | 10 --- src/Columns/ColumnNullable.cpp | 10 --- src/Columns/ColumnUnique.h | 1 - src/Common/ArrayCache.h | 2 - src/Common/DateLUTImpl.cpp | 3 - src/Common/HashTable/SmallTable.h | 67 ------------------- src/Common/RadixSort.h | 2 - .../SerializationLowCardinality.cpp | 7 -- .../SerializationLowCardinality.h | 3 - src/Dictionaries/SSDCacheDictionaryStorage.h | 2 - src/Dictionaries/XDBCDictionarySource.cpp | 2 - src/Disks/DiskWebServer.cpp | 1 - src/Functions/CustomWeekTransforms.h | 2 - src/Functions/FunctionsConversion.h | 2 - src/IO/CascadeWriteBuffer.cpp | 2 - src/Interpreters/ActionsDAG.cpp | 10 --- src/Interpreters/HashJoin.cpp | 1 - .../InterpreterSelectWithUnionQuery.cpp | 1 - src/Interpreters/JIT/CHJIT.cpp | 22 ------ src/Processors/Executors/PipelineExecutor.cpp | 1 - src/Processors/Formats/Impl/NativeFormat.cpp | 9 --- src/Processors/Sources/RemoteSource.cpp | 2 - src/QueryPipeline/QueryPipelineBuilder.cpp | 1 - src/Storages/Hive/HiveCommon.cpp | 1 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 16 ----- .../MergeTree/IMergedBlockOutputStream.h | 3 - src/Storages/MergeTree/MergeTask.cpp | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 5 -- 30 files changed, 191 deletions(-) diff --git a/src/Access/LDAPClient.cpp b/src/Access/LDAPClient.cpp index 3486be1de33..ff1ee6f3609 100644 --- a/src/Access/LDAPClient.cpp +++ b/src/Access/LDAPClient.cpp @@ -509,7 +509,6 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params) if (referrals) { SCOPE_EXIT({ -// ldap_value_free(referrals); ber_memvfree(reinterpret_cast(referrals)); referrals = nullptr; }); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index f0a8794d096..c6f14c7e865 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -152,7 +152,6 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) auto & dst_column_host_name = typeid_cast(*mutable_columns[name_pos["host_name"]]); auto & dst_array_current_time = typeid_cast(*mutable_columns[name_pos["current_time"]]).getData(); - // auto & dst_array_thread_id = typeid_cast(*mutable_columns[name_pos["thread_id"]]).getData(); auto & dst_array_type = typeid_cast(*mutable_columns[name_pos["type"]]).getData(); auto & dst_column_name = typeid_cast(*mutable_columns[name_pos["name"]]); auto & dst_array_value = typeid_cast(*mutable_columns[name_pos["value"]]).getData(); diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 62fb69a47e1..17e9bd97669 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -132,14 +132,12 @@ namespace ColumnLowCardinality::ColumnLowCardinality(MutableColumnPtr && column_unique_, MutableColumnPtr && indexes_, bool is_shared) : dictionary(std::move(column_unique_), is_shared), idx(std::move(indexes_)) { - // idx.check(getDictionary().size()); } void ColumnLowCardinality::insert(const Field & x) { compactIfSharedDictionary(); idx.insertPosition(dictionary.getColumnUnique().uniqueInsert(x)); - // idx.check(getDictionary().size()); } void ColumnLowCardinality::insertDefault() @@ -167,15 +165,12 @@ void ColumnLowCardinality::insertFrom(const IColumn & src, size_t n) const auto & nested = *low_cardinality_src->getDictionary().getNestedColumn(); idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(nested, position)); } - - // idx.check(getDictionary().size()); } void ColumnLowCardinality::insertFromFullColumn(const IColumn & src, size_t n) { compactIfSharedDictionary(); idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(src, n)); - // idx.check(getDictionary().size()); } void ColumnLowCardinality::insertRangeFrom(const IColumn & src, size_t start, size_t length) @@ -205,7 +200,6 @@ void ColumnLowCardinality::insertRangeFrom(const IColumn & src, size_t start, si auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(*used_keys, 0, used_keys->size()); idx.insertPositionsRange(*inserted_indexes->index(*sub_idx, 0), 0, length); } - // idx.check(getDictionary().size()); } void ColumnLowCardinality::insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length) @@ -213,7 +207,6 @@ void ColumnLowCardinality::insertRangeFromFullColumn(const IColumn & src, size_t compactIfSharedDictionary(); auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(src, start, length); idx.insertPositionsRange(*inserted_indexes, 0, length); - // idx.check(getDictionary().size()); } static void checkPositionsAreLimited(const IColumn & positions, UInt64 limit) @@ -254,14 +247,12 @@ void ColumnLowCardinality::insertRangeFromDictionaryEncodedColumn(const IColumn compactIfSharedDictionary(); auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(keys, 0, keys.size()); idx.insertPositionsRange(*inserted_indexes->index(positions, 0), 0, positions.size()); - // idx.check(getDictionary().size()); } void ColumnLowCardinality::insertData(const char * pos, size_t length) { compactIfSharedDictionary(); idx.insertPosition(dictionary.getColumnUnique().uniqueInsertData(pos, length)); - // idx.check(getDictionary().size()); } StringRef ColumnLowCardinality::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const @@ -276,7 +267,6 @@ const char * ColumnLowCardinality::deserializeAndInsertFromArena(const char * po const char * new_pos; idx.insertPosition(dictionary.getColumnUnique().uniqueDeserializeAndInsertFromArena(pos, new_pos)); - // idx.check(getDictionary().size()); return new_pos; } diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index d8e98ec9406..122e30d1bd8 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -273,14 +273,6 @@ llvm::Value * ColumnNullable::compileComparator(llvm::IRBuilderBase & builder, l b.CreateCondBr(lhs_or_rhs_are_null, lhs_or_rhs_are_null_block, lhs_rhs_are_not_null_block); - // if (unlikely(lval_is_null || rval_is_null)) - // { - // if (lval_is_null && rval_is_null) - // return 0; - // else - // return lval_is_null ? null_direction_hint : -null_direction_hint; - // } - b.SetInsertPoint(lhs_or_rhs_are_null_block); auto * lhs_equals_rhs_result = llvm::ConstantInt::getSigned(b.getInt8Ty(), 0); llvm::Value * lhs_and_rhs_are_null = b.CreateAnd(lhs_is_null_value, rhs_is_null_value); @@ -288,8 +280,6 @@ llvm::Value * ColumnNullable::compileComparator(llvm::IRBuilderBase & builder, l llvm::Value * lhs_or_rhs_are_null_block_result = b.CreateSelect(lhs_and_rhs_are_null, lhs_equals_rhs_result, lhs_is_null_result); b.CreateBr(join_block); - // getNestedColumn().compareAt(n, m, nested_rhs, null_direction_hint); - b.SetInsertPoint(lhs_rhs_are_not_null_block); llvm::Value * lhs_rhs_are_not_null_block_result = nested_column->compileComparator(builder, lhs_unwrapped_value, rhs_unwrapped_value, nan_direction_hint); diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 3c21a65e404..58891e30e12 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -548,7 +548,6 @@ MutableColumnPtr ColumnUnique::uniqueInsertRangeImpl( } } - // checkIndexes(*positions_column, column->size() + (overflowed_keys ? overflowed_keys->size() : 0)); return std::move(positions_column); } diff --git a/src/Common/ArrayCache.h b/src/Common/ArrayCache.h index 6efa5c92b5b..f01ff94e38b 100644 --- a/src/Common/ArrayCache.h +++ b/src/Common/ArrayCache.h @@ -514,8 +514,6 @@ private: return allocateFromFreeRegion(*free_region, size); } -// std::cerr << "Requested size: " << size << "\n"; - /// Evict something from cache and continue. while (true) { diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index 869954bb2ae..31290c53b49 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -122,9 +122,6 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) values.time_at_offset_change_value = (transition.from - cctz::civil_second(date)) / Values::OffsetChangeFactor; values.amount_of_offset_change_value = (transition.to - transition.from) / Values::OffsetChangeFactor; -// std::cerr << time_zone << ", " << date << ": change from " << transition.from << " to " << transition.to << "\n"; -// std::cerr << time_zone << ", " << date << ": change at " << values.time_at_offset_change() << " with " << values.amount_of_offset_change() << "\n"; - /// We don't support too large changes. if (values.amount_of_offset_change_value > 24 * 4) values.amount_of_offset_change_value = 24 * 4; diff --git a/src/Common/HashTable/SmallTable.h b/src/Common/HashTable/SmallTable.h index ad9537ff94a..b78901b03f6 100644 --- a/src/Common/HashTable/SmallTable.h +++ b/src/Common/HashTable/SmallTable.h @@ -74,7 +74,6 @@ public: using key_type = Key; using mapped_type = typename Cell::mapped_type; using value_type = typename Cell::value_type; - using cell_type = Cell; class Reader final : private Cell::State { @@ -247,39 +246,6 @@ public: } } - - /// Same, but return false if it's full. - bool ALWAYS_INLINE tryEmplace(Key x, iterator & it, bool & inserted) - { - Cell * res = findCell(x); - it = iteratorTo(res); - inserted = res == buf + m_size; - if (inserted) - { - if (res == buf + capacity) - return false; - - new(res) Cell(x, *this); - ++m_size; - } - return true; - } - - - /// Copy the cell from another hash table. It is assumed that there was no such key in the table yet. - void ALWAYS_INLINE insertUnique(const Cell * cell) - { - memcpy(&buf[m_size], cell, sizeof(*cell)); - ++m_size; - } - - void ALWAYS_INLINE insertUnique(Key x) - { - new(&buf[m_size]) Cell(x, *this); - ++m_size; - } - - iterator ALWAYS_INLINE find(Key x) { return iteratorTo(findCell(x)); } const_iterator ALWAYS_INLINE find(Key x) const { return iteratorTo(findCell(x)); } @@ -381,36 +347,3 @@ template > using SmallSet = SmallTable, capacity>; - -template -< - typename Key, - typename Cell, - size_t capacity -> -class SmallMapTable : public SmallTable -{ -public: - using key_type = Key; - using mapped_type = typename Cell::mapped_type; - using value_type = typename Cell::value_type; - using cell_type = Cell; - - mapped_type & ALWAYS_INLINE operator[](Key x) - { - typename SmallMapTable::iterator it; - bool inserted; - this->emplace(x, it, inserted); - new (&it->getMapped()) mapped_type(); - return it->getMapped(); - } -}; - - -template -< - typename Key, - typename Mapped, - size_t capacity -> -using SmallMap = SmallMapTable, capacity>; diff --git a/src/Common/RadixSort.h b/src/Common/RadixSort.h index 4bf975c4c7a..9ca43bee30c 100644 --- a/src/Common/RadixSort.h +++ b/src/Common/RadixSort.h @@ -355,8 +355,6 @@ private: template static inline void radixSortMSDInternal(Element * arr, size_t size, size_t limit) { -// std::cerr << PASS << ", " << size << ", " << limit << "\n"; - /// The beginning of every i-1-th bucket. 0th element will be equal to 1st. /// Last element will point to array end. std::unique_ptr prev_buckets{new Element*[HISTOGRAM_SIZE + 1]}; diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index c79f588e46c..8e19c5a740b 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -511,8 +511,6 @@ void SerializationLowCardinality::serializeBinaryBulkWithMultipleStreams( /// Insert used_keys into global dictionary and update sub_index. auto indexes_with_overflow = global_dictionary->uniqueInsertRangeWithOverflow(*keys, 0, keys->size(), settings.low_cardinality_max_dictionary_size); - // size_t max_size = settings.low_cardinality_max_dictionary_size + indexes_with_overflow.overflowed_keys->size(); - // ColumnLowCardinality::Index(indexes_with_overflow.indexes->getPtr()).check(max_size); if (global_dictionary->size() > settings.low_cardinality_max_dictionary_size) throw Exception("Got dictionary with size " + toString(global_dictionary->size()) + @@ -656,11 +654,6 @@ void SerializationLowCardinality::deserializeBinaryBulkWithMultipleStreams( { auto maps = mapIndexWithAdditionalKeys(*indexes_column, global_dictionary->size()); - // ColumnLowCardinality::Index(maps.additional_keys_map->getPtr()).check(additional_keys->size()); - - // ColumnLowCardinality::Index(indexes_column->getPtr()).check( - // maps.dictionary_map->size() + maps.additional_keys_map->size()); - auto used_keys = IColumn::mutate(global_dictionary->getNestedColumn()->index(*maps.dictionary_map, 0)); if (!maps.additional_keys_map->empty()) diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.h b/src/DataTypes/Serializations/SerializationLowCardinality.h index 0a3597e86c7..96e3a297d6a 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.h +++ b/src/DataTypes/Serializations/SerializationLowCardinality.h @@ -78,9 +78,6 @@ private: template void deserializeImpl(IColumn & column, DeserializeFunctionPtr func, Args &&... args) const; - - // template - // static MutableColumnUniquePtr createColumnUniqueImpl(const IDataType & keys_type, const Creator & creator); }; } diff --git a/src/Dictionaries/SSDCacheDictionaryStorage.h b/src/Dictionaries/SSDCacheDictionaryStorage.h index d813cf1bcc8..22a46016b2c 100644 --- a/src/Dictionaries/SSDCacheDictionaryStorage.h +++ b/src/Dictionaries/SSDCacheDictionaryStorage.h @@ -527,8 +527,6 @@ public: throw Exception(ErrorCodes::CANNOT_IO_SUBMIT, "Cannot submit request for asynchronous IO on file {}", file_path); } - // CurrentMetrics::Increment metric_increment_write{CurrentMetrics::Write}; - io_event event; while (io_getevents(aio_context.ctx, 1, 1, &event, nullptr) < 0) diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 5e9c2f7ac7a..0a097c4faef 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -275,8 +275,6 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory) bool /* created_from_ddl */) -> DictionarySourcePtr { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Dictionary source of type `jdbc` is disabled until consistent support for nullable fields."); - // BridgeHelperPtr bridge = std::make_shared>(config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".connection_string")); - // return std::make_unique(dict_struct, config, config_prefix + ".jdbc", sample_block, context, bridge); }; factory.registerSource("jdbc", create_table_source); } diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index 54dce926893..b6cda8288d7 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -74,7 +74,6 @@ void DiskWebServer::initialize(const String & uri_path) const if (file_data.type == FileType::Directory) { directories_to_load.push_back(file_path); - // file_path = fs::path(file_path) / ""; } file_path = file_path.substr(url.size()); diff --git a/src/Functions/CustomWeekTransforms.h b/src/Functions/CustomWeekTransforms.h index 8656f9da927..c296c8228b1 100644 --- a/src/Functions/CustomWeekTransforms.h +++ b/src/Functions/CustomWeekTransforms.h @@ -63,12 +63,10 @@ struct ToStartOfWeekImpl static inline UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); -// return time_zone.toFirstDayNumOfWeek(t, week_mode); } static inline UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); -// return time_zone.toFirstDayNumOfWeek(t, week_mode); } static inline UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) { diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index e0c42401207..b666602e366 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1091,8 +1091,6 @@ struct ConvertThroughParsing static constexpr bool to_datetime64 = std::is_same_v; - // using ToFieldType = typename ToDataType::FieldType; - static bool isAllRead(ReadBuffer & in) { /// In case of FixedString, skip zero bytes at end. diff --git a/src/IO/CascadeWriteBuffer.cpp b/src/IO/CascadeWriteBuffer.cpp index 616fbe9b789..ca11290c71b 100644 --- a/src/IO/CascadeWriteBuffer.cpp +++ b/src/IO/CascadeWriteBuffer.cpp @@ -50,8 +50,6 @@ void CascadeWriteBuffer::nextImpl() } set(curr_buffer->position(), curr_buffer->buffer().end() - curr_buffer->position()); -// std::cerr << "CascadeWriteBuffer a count=" << count() << " bytes=" << bytes << " offset=" << offset() -// << " bytes+size=" << bytes + buffer().size() << "\n"; } diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 2703773f464..b91fd7ac5cf 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -448,17 +448,7 @@ static ColumnWithTypeAndName executeActionForHeader(const ActionsDAG::Node * nod { case ActionsDAG::ActionType::FUNCTION: { - // bool all_args_are_const = true; - - // for (const auto & argument : arguments) - // if (typeid_cast(argument.column.get()) == nullptr) - // all_args_are_const = false; - res_column.column = node->function->execute(arguments, res_column.type, 0, true); - - // if (!all_args_are_const) - // res_column.column = res_column.column->convertToFullColumnIfConst(); - break; } diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index b54c77b385f..722ba81451a 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -2100,7 +2100,6 @@ std::shared_ptr HashJoin::getNonJoinedBlocks(const Block & left if (multiple_disjuncts) { /// ... calculate `left_columns_count` ... - // throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "TODO"); size_t left_columns_count = left_sample_block.columns(); auto non_joined = std::make_unique>(*this, max_block_size); return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 9f87a47fced..157a5e0595d 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -280,7 +280,6 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_, void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) { - // auto num_distinct_union = optimizeUnionList(); size_t num_plans = nested_interpreters.size(); const Settings & settings = context->getSettingsRef(); diff --git a/src/Interpreters/JIT/CHJIT.cpp b/src/Interpreters/JIT/CHJIT.cpp index 9eec82b4179..c2f3fc7c27d 100644 --- a/src/Interpreters/JIT/CHJIT.cpp +++ b/src/Interpreters/JIT/CHJIT.cpp @@ -244,28 +244,6 @@ private: } }; -// class AssemblyPrinter -// { -// public: - -// explicit AssemblyPrinter(llvm::TargetMachine &target_machine_) -// : target_machine(target_machine_) -// { -// } - -// void print(llvm::Module & module) -// { -// llvm::legacy::PassManager pass_manager; -// target_machine.Options.MCOptions.AsmVerbose = true; -// if (target_machine.addPassesToEmitFile(pass_manager, llvm::errs(), nullptr, llvm::CodeGenFileType::CGFT_AssemblyFile)) -// throw Exception(ErrorCodes::CANNOT_COMPILE_CODE, "MachineCode cannot be printed"); - -// pass_manager.run(module); -// } -// private: -// llvm::TargetMachine & target_machine; -// }; - /** MemoryManager for module. * Keep total allocated size during RuntimeDyld linker execution. */ diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 68225d73ff1..29c57e08573 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -205,7 +205,6 @@ void PipelineExecutor::executeStepImpl(size_t thread_num, std::atomic_bool * yie Stopwatch total_time_watch; #endif - // auto & node = tasks.getNode(thread_num); auto & context = tasks.getThreadContext(thread_num); bool yield = false; diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index 423fd483712..a8e2ddf95e4 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -74,15 +74,6 @@ protected: if (chunk) { auto block = getPort(PortKind::Main).getHeader(); - - // const auto & info = chunk.getChunkInfo(); - // const auto * agg_info = typeid_cast(info.get()); - // if (agg_info) - // { - // block.info.bucket_num = agg_info->bucket_num; - // block.info.is_overflows = agg_info->is_overflows; - // } - block.setColumns(chunk.detachColumns()); writer.write(block); } diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 9b01e048391..9f29ad9ad07 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -126,7 +126,6 @@ void RemoteSource::onCancel() { was_query_canceled = true; query_executor->cancel(&read_context); - // is_async_state = false; } void RemoteSource::onUpdatePorts() @@ -135,7 +134,6 @@ void RemoteSource::onUpdatePorts() { was_query_canceled = true; query_executor->finish(&read_context); - // is_async_state = false; } } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 88a52defa1e..340b85efae9 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -323,7 +323,6 @@ QueryPipelineBuilderPtr QueryPipelineBuilder::mergePipelines( left->pipe.processors.emplace_back(transform); left->pipe.processors.insert(left->pipe.processors.end(), right->pipe.processors.begin(), right->pipe.processors.end()); - // left->pipe.holder = std::move(right->pipe.holder); left->pipe.header = left->pipe.output_ports.front()->getHeader(); left->pipe.max_parallel_streams = std::max(left->pipe.max_parallel_streams, right->pipe.max_parallel_streams); return left; diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 7b2f04f7073..609adcf65c9 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -66,7 +66,6 @@ HiveMetastoreClient::HiveTableMetadataPtr HiveMetastoreClient::getTableMetadata( }; tryCallHiveClient(client_call); - // bool update_cache = shouldUpdateTableMetadata(db_name, table_name, partitions); String cache_key = getCacheKey(db_name, table_name); HiveTableMetadataPtr metadata = table_metadata_cache.get(cache_key); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 60941108f00..95f25aa1955 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -603,22 +603,6 @@ String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize( return *minimum_size_column; } -// String IMergeTreeDataPart::getFullPath() const -// { -// if (relative_path.empty()) -// throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); - -// return fs::path(storage.getFullPathOnDisk(volume->getDisk())) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; -// } - -// String IMergeTreeDataPart::getRelativePath() const -// { -// if (relative_path.empty()) -// throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); - -// return fs::path(storage.relative_data_path) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; -// } - void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency) { assertOnDisk(); diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index 3b94b85607a..dbcca1443b5 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -30,9 +30,6 @@ public: } protected: - // using SerializationState = ISerialization::SerializeBinaryBulkStatePtr; - - // ISerialization::OutputStreamGetter createStreamGetter(const String & name, WrittenOffsetColumns & offset_columns); /// Remove all columns marked expired in data_part. Also, clears checksums /// and columns array. Return set of removed files names. diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 7426b384394..dc468174dfa 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -596,7 +596,6 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c const auto & projections = global_ctx->metadata_snapshot->getProjections(); - // tasks_for_projections.reserve(projections.size()); for (const auto & projection : projections) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1bc4c26e40e..219093e8d75 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1584,8 +1584,6 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) return true; /// NOTE Deletion from `virtual_parts` is not done, but it is only necessary for merge. } - // bool do_fetch = false; - switch (entry.type) { case LogEntry::ATTACH_PART: @@ -1593,7 +1591,6 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) [[fallthrough]]; case LogEntry::GET_PART: return executeFetch(entry); - // do_fetch = true; case LogEntry::MERGE_PARTS: throw Exception(ErrorCodes::LOGICAL_ERROR, "Merge has to be executed by another function"); case LogEntry::MUTATE_PART: @@ -1609,8 +1606,6 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected log entry type: {}", static_cast(entry.type)); } - - // return true; } From 8bd674186a32cfb615ade336837584debc21e6fc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Jul 2022 16:15:42 +0300 Subject: [PATCH 507/659] Update getStructureOfRemoteTable.cpp --- src/Storages/getStructureOfRemoteTable.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 4ccd8a71ede..8acd7434d51 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -122,7 +122,7 @@ ColumnsDescription getStructureOfRemoteTable( const auto & shards_info = cluster.getShardsInfo(); std::string fail_messages; - + /// Use local shard as first priority, as it needs no network communication for (const auto & shard_info : shards_info) { From 9a55f84885bca5876b77ceb2875a5c58015ff37e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Jul 2022 16:24:18 +0300 Subject: [PATCH 508/659] Revert "Remove broken optimisation in Direct dictionary dictHas implementation" --- src/Dictionaries/DirectDictionary.cpp | 27 +++++--- .../02366_direct_dictionary_dicthas.reference | 62 ------------------- .../02366_direct_dictionary_dicthas.sql | 56 ----------------- 3 files changed, 17 insertions(+), 128 deletions(-) delete mode 100644 tests/queries/0_stateless/02366_direct_dictionary_dicthas.reference delete mode 100644 tests/queries/0_stateless/02366_direct_dictionary_dicthas.sql diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 20d8706ca54..6ecc216e370 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -171,6 +171,15 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys( auto requested_keys = requested_keys_extractor.extractAllKeys(); size_t requested_keys_size = requested_keys.size(); + HashMap requested_key_to_index; + requested_key_to_index.reserve(requested_keys_size); + + for (size_t i = 0; i < requested_keys.size(); ++i) + { + auto requested_key = requested_keys[i]; + requested_key_to_index[requested_key] = i; + } + auto result = ColumnUInt8::create(requested_keys_size, false); auto & result_data = result->getData(); @@ -196,17 +205,15 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys( { auto block_key = block_keys_extractor.extractCurrentKey(); - size_t index; - for (index = 0; index < requested_keys_size; ++index) - { - if (!result_data[index] && requested_keys[index] == block_key) - { - keys_found++; - result_data[index] = true; + const auto * it = requested_key_to_index.find(block_key); + assert(it); - block_keys_extractor.rollbackCurrentKey(); - } - } + size_t result_data_found_index = it->getMapped(); + /// block_keys_size cannot be used, due to duplicates. + keys_found += !result_data[result_data_found_index]; + result_data[result_data_found_index] = true; + + block_keys_extractor.rollbackCurrentKey(); } block_key_columns.clear(); diff --git a/tests/queries/0_stateless/02366_direct_dictionary_dicthas.reference b/tests/queries/0_stateless/02366_direct_dictionary_dicthas.reference deleted file mode 100644 index 49b34f828cd..00000000000 --- a/tests/queries/0_stateless/02366_direct_dictionary_dicthas.reference +++ /dev/null @@ -1,62 +0,0 @@ -0 -0 -0 -1 -0 -1 -0 -2 -1 -0 -0 0 1 -1 0 1 -2 0 1 -3 1 0 -4 0 1 -5 1 0 -6 0 1 -7 2 0 -8 1 0 -9 0 1 -1 -1 -1 -0 -1 -0 -1 -0 -0 -1 -1 -1 -1 -0 -1 -0 -1 -0 -0 -1 -1 -1 -1 -0 -1 -0 -1 -0 -0 -1 -value_0 -value_0 -value_0 -UNKNOWN -value_0 -UNKNOWN -value_0 -UNKNOWN -UNKNOWN -value_0 -4 0 -6 1 diff --git a/tests/queries/0_stateless/02366_direct_dictionary_dicthas.sql b/tests/queries/0_stateless/02366_direct_dictionary_dicthas.sql deleted file mode 100644 index b111415b56c..00000000000 --- a/tests/queries/0_stateless/02366_direct_dictionary_dicthas.sql +++ /dev/null @@ -1,56 +0,0 @@ --- Tags: no-backward-compatibility-check -DROP DATABASE IF EXISTS 02366_dictionary_db; -CREATE DATABASE 02366_dictionary_db; - -CREATE TABLE 02366_dictionary_db.dict_data -( - id UInt64, - val String -) -ENGINE = Memory; - -CREATE TABLE 02366_dictionary_db.lookup_data -( - id UInt64, - lookup_key UInt64, -) -ENGINE = Memory; - -INSERT INTO 02366_dictionary_db.dict_data VALUES(0, 'value_0'); - -INSERT INTO 02366_dictionary_db.lookup_data VALUES(0, 0); -INSERT INTO 02366_dictionary_db.lookup_data VALUES(1, 0); -INSERT INTO 02366_dictionary_db.lookup_data VALUES(2, 0); -INSERT INTO 02366_dictionary_db.lookup_data VALUES(3, 1); -INSERT INTO 02366_dictionary_db.lookup_data VALUES(4, 0); -INSERT INTO 02366_dictionary_db.lookup_data VALUES(5, 1); -INSERT INTO 02366_dictionary_db.lookup_data VALUES(6, 0); -INSERT INTO 02366_dictionary_db.lookup_data VALUES(7, 2); -INSERT INTO 02366_dictionary_db.lookup_data VALUES(8, 1); -INSERT INTO 02366_dictionary_db.lookup_data VALUES(9, 0); - -CREATE DICTIONARY 02366_dictionary_db.dict0 -( - id UInt64, - val String -) -PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data')) -LAYOUT(DIRECT()); - -SELECT lookup_key FROM 02366_dictionary_db.lookup_data ORDER BY id ASC; -SELECT id, lookup_key, dictHas(02366_dictionary_db.dict0, lookup_key) FROM 02366_dictionary_db.lookup_data ORDER BY id ASC; - --- Nesting this way seems to help it make all the lookups as a single block, although even then it isn't guaranteed -SELECT dictHas(02366_dictionary_db.dict0, lk) FROM (SELECT any(lookup_key) as lk FROM 02366_dictionary_db.lookup_data group by id ORDER BY id ASC); --- Same with this group by -SELECT dictHas(02366_dictionary_db.dict0, any(lookup_key)) FROM 02366_dictionary_db.lookup_data GROUP BY id ORDER BY id ASC; - - -SELECT dictHas(02366_dictionary_db.dict0, lookup_key) FROM 02366_dictionary_db.lookup_data ORDER BY id ASC; -SELECT dictGetOrDefault(02366_dictionary_db.dict0, 'val', lookup_key, 'UNKNOWN') FROM 02366_dictionary_db.lookup_data ORDER BY id ASC; -SELECT count(), has FROM 02366_dictionary_db.lookup_data group by dictHas(02366_dictionary_db.dict0, lookup_key) as has; - -DROP DICTIONARY 02366_dictionary_db.dict0; -DROP TABLE 02366_dictionary_db.lookup_data; -DROP TABLE 02366_dictionary_db.dict_data; From 15496c63b9455339bdd8a12e62742b13f6ac5448 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Jul 2022 10:14:53 -0400 Subject: [PATCH 509/659] __APPLE__->OS_DARWIN, __FreeBSD__->OS_FREEBSD --- utils/self-extracting-executable/compressor.cpp | 2 +- utils/self-extracting-executable/decompressor.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/self-extracting-executable/compressor.cpp b/utils/self-extracting-executable/compressor.cpp index 98cb895bb3a..78ab65cfd8c 100644 --- a/utils/self-extracting-executable/compressor.cpp +++ b/utils/self-extracting-executable/compressor.cpp @@ -10,7 +10,7 @@ #include #include -#if (defined(__APPLE__) || defined(__FreeBSD__)) && defined(__GNUC__) +#if (defined(OS_DARWIN) || defined(OS_FREEBSD)) && defined(__GNUC__) # include #elif !defined(_MSC_VER) # include diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index c02b092f9d9..74a436c5185 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -13,7 +13,7 @@ #include #include -#if (defined(__APPLE__) || defined(__FreeBSD__)) && defined(__GNUC__) +#if (defined(OS_DARWIN) || defined(OS_FREEBSD)) && defined(__GNUC__) # include #elif !defined(_MSC_VER) # include From 821f006a7ada6742ab2f3514b294a83af72f7165 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Jul 2022 14:24:11 +0000 Subject: [PATCH 510/659] Allow CREATE TEMPORARY TABLE ... (list of columns) AS ... --- src/Parsers/ParserCreateQuery.cpp | 2 +- ...02371_create_temporary_table_as_with_columns_list.reference | 2 ++ .../02371_create_temporary_table_as_with_columns_list.sql | 3 +++ 3 files changed, 6 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.reference create mode 100644 tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.sql diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index ce79ccf708a..275f3bc75cc 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -584,7 +584,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe auto storage_parse_result = storage_p.parse(pos, storage, expected); - if (storage_parse_result && need_parse_as_select()) + if ((storage_parse_result || is_temporary) && need_parse_as_select()) { if (!select_p.parse(pos, select, expected)) return false; diff --git a/tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.reference b/tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.reference new file mode 100644 index 00000000000..6fc56adcb1c --- /dev/null +++ b/tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.reference @@ -0,0 +1,2 @@ +Vasya +Petya diff --git a/tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.sql b/tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.sql new file mode 100644 index 00000000000..7d8f297b505 --- /dev/null +++ b/tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.sql @@ -0,0 +1,3 @@ +CREATE TEMPORARY TABLE test_02327 (name String) AS SELECT * FROM VALUES(('Vasya'), ('Petya')); +SELECT * FROM test_02327; +DROP TABLE test_02327; From c8ea6783dcd1d31504bd65ec0ff1cf7f3ba859a1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Jul 2022 10:27:56 -0400 Subject: [PATCH 511/659] clang tidy suggestions --- utils/self-extracting-executable/compressor.cpp | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/utils/self-extracting-executable/compressor.cpp b/utils/self-extracting-executable/compressor.cpp index 78ab65cfd8c..7afb38232f6 100644 --- a/utils/self-extracting-executable/compressor.cpp +++ b/utils/self-extracting-executable/compressor.cpp @@ -363,7 +363,14 @@ int copy_decompressor_self(const char *self, int output_fd) return 1; } - int decompressor_size = atoi(size_str); + char * end = nullptr; + int decompressor_size = strtol(size_str, &end, 10); + if (*end != 0) + { + std::cerr << "Error: unable to extract decompressor" << std::endl; + close(input_fd); + return 1; + } if (-1 == lseek(input_fd, -(decompressor_size + 15), SEEK_END)) { @@ -407,7 +414,7 @@ int copy_decompressor_file(const char *path, int output_fd) inline void usage(FILE * out, const char * name) { - fprintf(out, + (void)fprintf(out, "%s [--level=] [--decompressor=] [... ]\n" "\t--level - compression level, max is %d, negative - prefer speed over compression\n" "\t default is 5\n" From 96bcae419cad2a957c981d83b7f293dabed4b7bd Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 15 Jul 2022 14:57:58 +0000 Subject: [PATCH 512/659] Cleanup logic around join_algorithm setting --- src/Core/Settings.h | 2 +- src/Core/SettingsEnums.cpp | 3 +- src/Core/SettingsEnums.h | 3 +- src/Interpreters/ExpressionAnalyzer.cpp | 106 ++++++++++-------- src/Interpreters/FullSortingMergeJoin.h | 22 +++- src/Interpreters/HashJoin.cpp | 2 +- src/Interpreters/JoinedTables.cpp | 3 +- src/Interpreters/MergeJoin.cpp | 14 +++ src/Interpreters/MergeJoin.h | 2 + src/Interpreters/TableJoin.cpp | 16 +-- src/Interpreters/TableJoin.h | 26 ++--- src/Interpreters/TreeRewriter.cpp | 5 +- .../0_stateless/02242_join_rocksdb.sql | 8 +- 13 files changed, 120 insertions(+), 92 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9034fb924ba..3f079189491 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -355,7 +355,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).", 0) \ M(OverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ M(Bool, 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(JoinAlgorithm, join_algorithm, JoinAlgorithm::HASH, "Specify join algorithm: 'auto', 'hash', 'partial_merge', 'prefer_partial_merge', 'parallel_hash'. 'auto' tries to change HashJoin to MergeJoin on the fly to avoid out of memory.", 0) \ + M(JoinAlgorithm, join_algorithm, JoinAlgorithm::DEFAULT, "Specify join algorithm.", 0) \ M(UInt64, default_max_bytes_in_join, 1000000000, "Maximum size of right-side table if limit is required but max_bytes_in_join is not set.", 0) \ M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 3585cffb8ec..b832096c86c 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -31,7 +31,8 @@ IMPLEMENT_SETTING_ENUM(JoinStrictness, ErrorCodes::UNKNOWN_JOIN, IMPLEMENT_SETTING_MULTI_ENUM(JoinAlgorithm, ErrorCodes::UNKNOWN_JOIN, - {{"auto", JoinAlgorithm::AUTO}, + {{"default", JoinAlgorithm::DEFAULT}, + {"auto", JoinAlgorithm::AUTO}, {"hash", JoinAlgorithm::HASH}, {"partial_merge", JoinAlgorithm::PARTIAL_MERGE}, {"prefer_partial_merge", JoinAlgorithm::PREFER_PARTIAL_MERGE}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 6b9ff8277b1..b8a2bdb48b0 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -38,7 +38,8 @@ DECLARE_SETTING_ENUM(JoinStrictness) enum class JoinAlgorithm { - AUTO = 0, + DEFAULT = 0, + AUTO, HASH, PARTIAL_MERGE, PREFER_PARTIAL_MERGE, diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index a4bdc4ed252..a6e3f88b2b8 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -89,7 +89,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int UNKNOWN_IDENTIFIER; extern const int UNKNOWN_TYPE_OF_AST_NODE; - extern const int UNSUPPORTED_METHOD; } namespace @@ -1079,34 +1078,58 @@ static ActionsDAGPtr createJoinedBlockActions(ContextPtr context, const TableJoi return ExpressionAnalyzer(expression_list, syntax_result, context).getActionsDAG(true, false); } -static std::shared_ptr chooseJoinAlgorithm(std::shared_ptr analyzed_join, const Block & right_sample_block, ContextPtr context) -{ - /// HashJoin with Dictionary optimisation - if (analyzed_join->tryInitDictJoin(right_sample_block, context)) - return std::make_shared(analyzed_join, right_sample_block); +std::shared_ptr tryKeyValueJoin(std::shared_ptr analyzed_join, const Block & right_sample_block); - bool allow_merge_join = analyzed_join->allowMergeJoin(); - if (analyzed_join->forceHashJoin() || (analyzed_join->preferMergeJoin() && !allow_merge_join)) +static std::shared_ptr chooseJoinAlgorithm(std::shared_ptr analyzed_join, std::unique_ptr & joined_plan, ContextPtr context) +{ + Block right_sample_block = joined_plan->getCurrentDataStream().header; + + if (analyzed_join->isEnabledAlgorithm(JoinAlgorithm::DIRECT)) + { + if (JoinPtr kvjoin = tryKeyValueJoin(analyzed_join, right_sample_block)) + { + /// Do not need to execute plan for right part + joined_plan.reset(); + return kvjoin; + } + + /// It's not a hash join actually, that's why we check JoinAlgorithm::DIRECT + /// It's would be fixed in https://github.com/ClickHouse/ClickHouse/pull/38956 + if (analyzed_join->tryInitDictJoin(right_sample_block, context)) + { + joined_plan.reset(); + return std::make_shared(analyzed_join, right_sample_block); + } + } + + if (analyzed_join->isEnabledAlgorithm(JoinAlgorithm::PARTIAL_MERGE) || + analyzed_join->isEnabledAlgorithm(JoinAlgorithm::PREFER_PARTIAL_MERGE)) + { + if (MergeJoin::isSupported(analyzed_join)) + return std::make_shared(analyzed_join, right_sample_block); + } + + if (analyzed_join->isEnabledAlgorithm(JoinAlgorithm::HASH) || + /// partial_merge is preferred, but can't be used for specified kind of join, fallback to hash + analyzed_join->isEnabledAlgorithm(JoinAlgorithm::PREFER_PARTIAL_MERGE) || + analyzed_join->isEnabledAlgorithm(JoinAlgorithm::PARALLEL_HASH)) { if (analyzed_join->allowParallelHashJoin()) - { return std::make_shared(context, analyzed_join, context->getSettings().max_threads, right_sample_block); - } return std::make_shared(analyzed_join, right_sample_block); } - else if (analyzed_join->forceMergeJoin() || (analyzed_join->preferMergeJoin() && allow_merge_join)) + + if (analyzed_join->isEnabledAlgorithm(JoinAlgorithm::FULL_SORTING_MERGE)) { - return std::make_shared(analyzed_join, right_sample_block); + if (FullSortingMergeJoin::isSupported(analyzed_join)) + return std::make_shared(analyzed_join, right_sample_block); } - else if (analyzed_join->forceFullSortingMergeJoin()) - { - if (analyzed_join->getClauses().size() != 1) - throw Exception("Full sorting merge join is supported only for single-condition joins", ErrorCodes::NOT_IMPLEMENTED); - if (analyzed_join->isSpecialStorage()) - throw Exception("Full sorting merge join is not supported for special storage", ErrorCodes::NOT_IMPLEMENTED); - return std::make_shared(analyzed_join, right_sample_block); - } - return std::make_shared(analyzed_join, right_sample_block); + + if (analyzed_join->isEnabledAlgorithm(JoinAlgorithm::AUTO)) + return std::make_shared(analyzed_join, right_sample_block); + + throw Exception("Can't execute any of specified algorithms for specified strictness/kind and right storage type", + ErrorCodes::NOT_IMPLEMENTED); } static std::unique_ptr buildJoinedPlan( @@ -1164,27 +1187,26 @@ static std::unique_ptr buildJoinedPlan( std::shared_ptr tryKeyValueJoin(std::shared_ptr analyzed_join, const Block & right_sample_block) { - auto error_or_null = [&](const String & msg) - { - if (analyzed_join->isForcedAlgorithm(JoinAlgorithm::DIRECT)) - throw DB::Exception(ErrorCodes::UNSUPPORTED_METHOD, "Can't use '{}' join algorithm: {}", JoinAlgorithm::DIRECT, msg); - return nullptr; - }; - - if (!analyzed_join->isAllowedAlgorithm(JoinAlgorithm::DIRECT)) + if (!analyzed_join->isEnabledAlgorithm(JoinAlgorithm::DIRECT)) return nullptr; auto storage = analyzed_join->getStorageKeyValue(); if (!storage) - return error_or_null("unsupported storage"); + { + return nullptr; + } if (!isInnerOrLeft(analyzed_join->kind())) - return error_or_null("illegal kind"); + { + return nullptr; + } if (analyzed_join->strictness() != ASTTableJoin::Strictness::All && analyzed_join->strictness() != ASTTableJoin::Strictness::Any && analyzed_join->strictness() != ASTTableJoin::Strictness::RightAny) - return error_or_null("illegal strictness"); + { + return nullptr; + } const auto & clauses = analyzed_join->getClauses(); bool only_one_key = clauses.size() == 1 && @@ -1194,15 +1216,16 @@ std::shared_ptr tryKeyValueJoin(std::shared_ptr a !clauses[0].on_filter_condition_right; if (!only_one_key) - return error_or_null("multiple keys is not allowed"); + { + return nullptr; + } String key_name = clauses[0].key_names_right[0]; String original_key_name = analyzed_join->getOriginalName(key_name); const auto & storage_primary_key = storage->getPrimaryKey(); if (storage_primary_key.size() != 1 || storage_primary_key[0] != original_key_name) { - return error_or_null(fmt::format("key '{}'{} doesn't match storage '{}'", - key_name, (key_name != original_key_name ? " (aka '" + original_key_name + "')" : ""), fmt::join(storage_primary_key, ","))); + return nullptr; } return std::make_shared(analyzed_join, right_sample_block, storage); @@ -1240,18 +1263,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin( joined_plan->addStep(std::move(converting_step)); } - const Block & right_sample_block = joined_plan->getCurrentDataStream().header; - if (JoinPtr kvjoin = tryKeyValueJoin(analyzed_join, right_sample_block)) - { - joined_plan.reset(); - return kvjoin; - } - - JoinPtr join = chooseJoinAlgorithm(analyzed_join, right_sample_block, getContext()); - - /// Do not make subquery for join over dictionary. - if (analyzed_join->getDictionaryReader()) - joined_plan.reset(); + JoinPtr join = chooseJoinAlgorithm(analyzed_join, joined_plan, getContext()); return join; } diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index 3ee6ce1c1fb..14c81259159 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -34,14 +34,26 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "FullSortingMergeJoin::addJoinedBlock should not be called"); } - void checkTypesOfKeys(const Block & left_block) const override + static bool isSupported(const std::shared_ptr & table_join) { - if (table_join->getClauses().size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "FullSortingMergeJoin supports only one join key"); + if (!table_join->oneDisjunct()) + return false; + + bool support_storage = !table_join->isSpecialStorage(); + + const auto & on_expr = table_join->getOnlyClause(); + bool support_conditions = !on_expr.on_filter_condition_left && !on_expr.on_filter_condition_right; /// Key column can change nullability and it's not handled on type conversion stage, so algorithm should be aware of it - if (table_join->hasUsing() && table_join->joinUseNulls()) - throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "FullSortingMergeJoin doesn't support USING with join_use_nulls"); + bool support_using_and_nulls = !table_join->hasUsing() || !table_join->joinUseNulls(); + + return support_conditions && support_using_and_nulls && support_storage; + } + + void checkTypesOfKeys(const Block & left_block) const override + { + if (!isSupported(table_join)) + throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "FullSortingMergeJoin doesn't support specified query"); const auto & onexpr = table_join->getOnlyClause(); for (size_t i = 0; i < onexpr.key_names_left.size(); ++i) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index b54c77b385f..1691f7ec10c 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -718,7 +718,7 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) bool multiple_disjuncts = !table_join->oneDisjunct(); /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). - bool save_key_columns = !table_join->forceHashJoin() || isRightOrFull(kind) || multiple_disjuncts; + bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || isRightOrFull(kind) || multiple_disjuncts; if (save_key_columns) { saved_block_sample = right_table_keys.cloneEmpty(); diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index df47e8acdca..9e15a525cb4 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -311,7 +311,8 @@ std::shared_ptr JoinedTables::makeTableJoin(const ASTSelectQuery & se { table_join->setStorageJoin(storage_join); } - else if (auto storage_dict = std::dynamic_pointer_cast(storage); storage_dict) + else if (auto storage_dict = std::dynamic_pointer_cast(storage); + storage_dict && join_algorithm.isSet(JoinAlgorithm::DIRECT)) { table_join->setStorageJoin(storage_dict); } diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 1dea769f724..711b71a2b3d 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -1135,6 +1135,20 @@ void MergeJoin::addConditionJoinColumn(Block & block, JoinTableSide block_side) } } +bool MergeJoin::isSupported(const std::shared_ptr & table_join) +{ + auto kind = table_join->kind(); + auto strictness = table_join->strictness(); + + bool is_any = (strictness == ASTTableJoin::Strictness::Any); + bool is_all = (strictness == ASTTableJoin::Strictness::All); + bool is_semi = (strictness == ASTTableJoin::Strictness::Semi); + + bool all_join = is_all && (isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind)); + bool special_left = isInnerOrLeft(kind) && (is_any || is_semi); + + return (all_join || special_left) && table_join->oneDisjunct(); +} MergeJoin::RightBlockInfo::RightBlockInfo(std::shared_ptr block_, size_t block_number_, size_t & skip_, RowBitmaps * bitmaps_) : block(block_) diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index ab36599e6f4..3b8ad6063e3 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -37,6 +37,8 @@ public: std::shared_ptr getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; + static bool isSupported(const std::shared_ptr & table_join); + private: friend class NotJoinedMerge; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 17869e2084b..029038357c1 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -363,7 +363,7 @@ void TableJoin::addJoinedColumnsAndCorrectTypesImpl(TColumns & left_columns, boo * For `JOIN ON expr1 == expr2` we will infer common type later in makeTableJoin, * when part of plan built and types of expression will be known. */ - inferJoinKeyCommonType(left_columns, columns_from_joined_table, !isSpecialStorage(), forceFullSortingMergeJoin()); + inferJoinKeyCommonType(left_columns, columns_from_joined_table, !isSpecialStorage(), isEnabledAlgorithm(JoinAlgorithm::FULL_SORTING_MERGE)); if (auto it = left_type_map.find(col.name); it != left_type_map.end()) { @@ -409,18 +409,6 @@ bool TableJoin::oneDisjunct() const return clauses.size() == 1; } -bool TableJoin::allowMergeJoin() const -{ - bool is_any = (strictness() == ASTTableJoin::Strictness::Any); - bool is_all = (strictness() == ASTTableJoin::Strictness::All); - bool is_semi = (strictness() == ASTTableJoin::Strictness::Semi); - - bool all_join = is_all && (isInner(kind()) || isLeft(kind()) || isRight(kind()) || isFull(kind())); - bool special_left = isLeft(kind()) && (is_any || is_semi); - - return (all_join || special_left) && oneDisjunct(); -} - bool TableJoin::needStreamWithNonJoinedRows() const { if (strictness() == ASTTableJoin::Strictness::Asof || @@ -511,7 +499,7 @@ TableJoin::createConvertingActions( const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns) { - inferJoinKeyCommonType(left_sample_columns, right_sample_columns, !isSpecialStorage(), forceFullSortingMergeJoin()); + inferJoinKeyCommonType(left_sample_columns, right_sample_columns, !isSpecialStorage(), isEnabledAlgorithm(JoinAlgorithm::FULL_SORTING_MERGE)); NameToNameMap left_key_column_rename; NameToNameMap right_key_column_rename; diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 4210da6ae76..57895d6d1c1 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -193,24 +193,20 @@ public: bool sameStrictnessAndKind(ASTTableJoin::Strictness, ASTTableJoin::Kind) const; const SizeLimits & sizeLimits() const { return size_limits; } VolumePtr getTemporaryVolume() { return tmp_volume; } - bool allowMergeJoin() const; - bool isAllowedAlgorithm(JoinAlgorithm val) const { return join_algorithm.isSet(val) || join_algorithm.isSet(JoinAlgorithm::AUTO); } - bool isForcedAlgorithm(JoinAlgorithm val) const { return join_algorithm == MultiEnum(val); } - - bool preferMergeJoin() const { return join_algorithm == MultiEnum(JoinAlgorithm::PREFER_PARTIAL_MERGE); } - bool forceMergeJoin() const { return join_algorithm == MultiEnum(JoinAlgorithm::PARTIAL_MERGE); } + bool isEnabledAlgorithm(JoinAlgorithm val) const + { + /// When join_algorithm = 'default' (not specified by user) we use hash or direct algorithm. + /// It's behaviour that was initially supported by clickhouse. + bool is_enbaled_by_default = val == JoinAlgorithm::DEFAULT + || val == JoinAlgorithm::HASH + || val == JoinAlgorithm::DIRECT; + if (join_algorithm.isSet(JoinAlgorithm::DEFAULT) && is_enbaled_by_default) + return true; + return join_algorithm.isSet(val); + } bool allowParallelHashJoin() const; - bool forceFullSortingMergeJoin() const { return !isSpecialStorage() && join_algorithm.isSet(JoinAlgorithm::FULL_SORTING_MERGE); } - - bool forceHashJoin() const - { - /// HashJoin always used for DictJoin - return dictionary_reader - || join_algorithm == MultiEnum(JoinAlgorithm::HASH) - || join_algorithm == MultiEnum(JoinAlgorithm::PARALLEL_HASH); - } bool joinUseNulls() const { return join_use_nulls; } bool forceNullableRight() const { return join_use_nulls && isLeftOrFull(table_join.kind); } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index b389c3eb705..bd18984faed 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include @@ -683,7 +684,7 @@ bool tryJoinOnConst(TableJoin & analyzed_join, ASTPtr & on_expression, ContextPt else return false; - if (!analyzed_join.forceHashJoin()) + if (!analyzed_join.isEnabledAlgorithm(JoinAlgorithm::HASH)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "JOIN ON constant ({}) supported only with join algorithm 'hash'", queryToString(on_expression)); @@ -770,7 +771,7 @@ void collectJoinedColumns(TableJoin & analyzed_join, ASTTableJoin & table_join, data.asofToJoinKeys(); } - if (!analyzed_join.oneDisjunct() && !analyzed_join.forceHashJoin()) + if (!analyzed_join.oneDisjunct() && !analyzed_join.isEnabledAlgorithm(JoinAlgorithm::HASH)) throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "Only `hash` join supports multiple ORs for keys in JOIN ON section"); } } diff --git a/tests/queries/0_stateless/02242_join_rocksdb.sql b/tests/queries/0_stateless/02242_join_rocksdb.sql index 1759311163b..34b3d120eae 100644 --- a/tests/queries/0_stateless/02242_join_rocksdb.sql +++ b/tests/queries/0_stateless/02242_join_rocksdb.sql @@ -47,16 +47,16 @@ SELECT '--- totals'; SELECT rdb.key % 2, sum(k), max(value2) FROM t2 INNER JOIN rdb ON rdb.key == t2.k GROUP BY (rdb.key % 2) WITH TOTALS; SELECT '---'; -SELECT * FROM t1 RIGHT JOIN rdb ON rdb.key == t1.k; -- { serverError UNSUPPORTED_METHOD } +SELECT * FROM t1 RIGHT JOIN rdb ON rdb.key == t1.k; -- { serverError NOT_IMPLEMENTED } SELECT * FROM t1 RIGHT JOIN rdb ON rdb.key == t1.k FORMAT Null SETTINGS join_algorithm = 'direct,hash'; -SELECT * FROM t1 FULL JOIN rdb ON rdb.key == t1.k; -- { serverError UNSUPPORTED_METHOD } +SELECT * FROM t1 FULL JOIN rdb ON rdb.key == t1.k; -- { serverError NOT_IMPLEMENTED } SELECT * FROM t1 FULL JOIN rdb ON rdb.key == t1.k FORMAT Null SETTINGS join_algorithm = 'direct,hash'; -SELECT * FROM t1 INNER JOIN rdb ON rdb.key + 1 == t1.k; -- { serverError UNSUPPORTED_METHOD } +SELECT * FROM t1 INNER JOIN rdb ON rdb.key + 1 == t1.k; -- { serverError NOT_IMPLEMENTED } SELECT * FROM t1 INNER JOIN rdb ON rdb.key + 1 == t1.k FORMAT Null SETTINGS join_algorithm = 'direct,hash'; -SELECT * FROM t1 INNER JOIN (SELECT * FROM rdb) AS rdb ON rdb.key == t1.k; -- { serverError UNSUPPORTED_METHOD } +SELECT * FROM t1 INNER JOIN (SELECT * FROM rdb) AS rdb ON rdb.key == t1.k; -- { serverError NOT_IMPLEMENTED } SELECT * FROM t1 INNER JOIN (SELECT * FROM rdb) AS rdb ON rdb.key == t1.k FORMAT Null SETTINGS join_algorithm = 'direct,hash'; DROP TABLE IF EXISTS rdb; From 0deae9b4a366f617e64b14c3c1c2c3281fe2d595 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 15 Jul 2022 14:58:19 +0000 Subject: [PATCH 513/659] Update doc about join_algorithm --- docs/en/operations/settings/settings.md | 30 +++++++++++++++++++------ 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9f66d5d29a9..59ac34bd6f1 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -302,18 +302,34 @@ Default value: `ALL`. Specifies [JOIN](../../sql-reference/statements/select/join.md) algorithm. +Several algorithms can be specified, and an available one would be chosen for a particular query based on kind/strictness and table engine. + Possible values: -- `hash` — [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. -- `partial_merge` — [Sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join) is used. -- `prefer_partial_merge` — ClickHouse always tries to use `merge` join if possible. -- `auto` — ClickHouse tries to change `hash` join to `merge` join on the fly to avoid out of memory. +- `default` — `hash` or `direct`, if possible (same as `direct,hash`) -Default value: `hash`. +- `hash` — [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section. -When using `hash` algorithm the right part of `JOIN` is uploaded into RAM. +- `parallel_hash` - a variation of `hash` join that splits the data into buckets and builds several hashtables instead of one concurrently to speed up this process. + +When using the `hash` algorithm, the right part of `JOIN` is uploaded into RAM. + +- `partial_merge` — a variation of the [sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join), where only the right table is fully sorted. + +The `RIGHT JOIN` and `FULL JOIN` are supported only with `ALL` strictness (`SEMI`, `ANTI`, `ANY`, and `ASOF` are not supported). + +When using `partial_merge` algorithm, ClickHouse sorts the data and dumps it to the disk. The `partial_merge` algorithm in ClickHouse differs slightly from the classic realization. First, ClickHouse sorts the right table by joining keys in blocks and creates a min-max index for sorted blocks. Then it sorts parts of the left table by `join key` and joins them over the right table. The min-max index is also used to skip unneeded right table blocks. + +- `direct` - can be applied when the right storage supports key-value requests. + +The `direct` algorithm performs a lookup in the right table using rows from the left table as keys. It's supported only by special storage such as [Dictionary](../../engines/table-engines/special/dictionary.md#dictionary) or [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md) and only the `LEFT` and `INNER` JOINs. + +- `auto` — try `hash` join and switch on the fly to another algorithm if the memory limit is violated. + +- `full_sorting_merge` — [Sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join) with full sorting joined tables before joining. + +- `prefer_partial_merge` — ClickHouse always tries to use `partial_merge` join if possible, otherwise uses `hash`. *Deprecated*, same as `partial_merge,hash`. -When using `partial_merge` algorithm ClickHouse sorts the data and dumps it to the disk. The `merge` algorithm in ClickHouse differs a bit from the classic realization. First ClickHouse sorts the right table by [join key](../../sql-reference/statements/select/join.md#select-join) in blocks and creates min-max index for sorted blocks. Then it sorts parts of left table by `join key` and joins them over right table. The min-max index is also used to skip unneeded right table blocks. ## join_any_take_last_row {#settings-join_any_take_last_row} From e8834c5ea3598f573e27ebd4d05b3dc72aa3f00a Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 15 Jul 2022 15:03:32 +0000 Subject: [PATCH 514/659] Remove trailing whitespaces from docs/en/sql-reference/statements/select/join.md --- docs/en/sql-reference/statements/select/join.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index b029cf4bac8..a36004566a5 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -36,7 +36,7 @@ Additional join types available in ClickHouse: - `LEFT ANY JOIN`, `RIGHT ANY JOIN` and `INNER ANY JOIN`, partially (for opposite side of `LEFT` and `RIGHT`) or completely (for `INNER` and `FULL`) disables the cartesian product for standard `JOIN` types. - `ASOF JOIN` and `LEFT ASOF JOIN`, joining sequences with a non-exact match. `ASOF JOIN` usage is described below. -:::note +:::note When [join_algorithm](../../../operations/settings/settings.md#settings-join_algorithm) is set to `partial_merge`, `RIGHT JOIN` and `FULL JOIN` are supported only with `ALL` strictness (`SEMI`, `ANTI`, `ANY`, and `ASOF` are not supported). ::: @@ -64,7 +64,7 @@ Rows are joined if the whole complex condition is met. If the conditions are not The `OR` operator inside the `ON` clause works using the hash join algorithm — for each `OR` argument with join keys for `JOIN`, a separate hash table is created, so memory consumption and query execution time grow linearly with an increase in the number of expressions `OR` of the `ON` clause. -:::note +:::note If a condition refers columns from different tables, then only the equality operator (`=`) is supported so far. ::: @@ -83,7 +83,7 @@ Consider `table_1` and `table_2`: Query with one join key condition and an additional condition for `table_2`: ``` sql -SELECT name, text FROM table_1 LEFT OUTER JOIN table_2 +SELECT name, text FROM table_1 LEFT OUTER JOIN table_2 ON table_1.Id = table_2.Id AND startsWith(table_2.text, 'Text'); ``` @@ -100,7 +100,7 @@ Note that the result contains the row with the name `C` and the empty text colum Query with `INNER` type of a join and multiple conditions: ``` sql -SELECT name, text, scores FROM table_1 INNER JOIN table_2 +SELECT name, text, scores FROM table_1 INNER JOIN table_2 ON table_1.Id = table_2.Id AND table_2.scores > 10 AND startsWith(table_2.text, 'Text'); ``` @@ -199,7 +199,7 @@ For example, consider the following tables: `ASOF JOIN` can take the timestamp of a user event from `table_1` and find an event in `table_2` where the timestamp is closest to the timestamp of the event from `table_1` corresponding to the closest match condition. Equal timestamp values are the closest if available. Here, the `user_id` column can be used for joining on equality and the `ev_time` column can be used for joining on the closest match. In our example, `event_1_1` can be joined with `event_2_1` and `event_1_2` can be joined with `event_2_3`, but `event_2_2` can’t be joined. -:::note +:::note `ASOF` join is **not** supported in the [Join](../../../engines/table-engines/special/join.md) table engine. ::: From dacfc7886ceffea2dd11244a1b336edb0dd7073a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 19 Jul 2022 13:31:43 +0200 Subject: [PATCH 515/659] Improve style_check scripts style a little bit --- .../test/style/process_style_check_result.py | 11 +++--- tests/ci/style_check.py | 37 ++++++++----------- 2 files changed, 22 insertions(+), 26 deletions(-) diff --git a/docker/test/style/process_style_check_result.py b/docker/test/style/process_style_check_result.py index fd544f3e9c1..8c2110d64e5 100755 --- a/docker/test/style/process_style_check_result.py +++ b/docker/test/style/process_style_check_result.py @@ -40,10 +40,10 @@ def process_result(result_folder): def write_results(results_file, status_file, results, status): - with open(results_file, "w") as f: + with open(results_file, "w", encoding="utf-8") as f: out = csv.writer(f, delimiter="\t") out.writerows(results) - with open(status_file, "w") as f: + with open(status_file, "w", encoding="utf-8") as f: out = csv.writer(f, delimiter="\t") out.writerow(status) @@ -53,9 +53,10 @@ if __name__ == "__main__": parser = argparse.ArgumentParser( description="ClickHouse script for parsing results of style check" ) - parser.add_argument("--in-results-dir", default="/test_output/") - parser.add_argument("--out-results-file", default="/test_output/test_results.tsv") - parser.add_argument("--out-status-file", default="/test_output/check_status.tsv") + default_dir = "/test_output" + parser.add_argument("--in-results-dir", default=default_dir) + parser.add_argument("--out-results-file", default=f"{default_dir}/test_results.tsv") + parser.add_argument("--out-status-file", default=f"{default_dir}/check_status.tsv") args = parser.parse_args() state, description, test_results = process_result(args.in_results_dir) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index ce638c7fac5..f06646d83c0 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -1,32 +1,26 @@ #!/usr/bin/env python3 -import logging -import subprocess -import os import csv +import logging +import os +import subprocess import sys -from github import Github -from env_helper import ( - RUNNER_TEMP, - GITHUB_WORKSPACE, -) -from s3_helper import S3Helper -from pr_info import PRInfo -from get_robot_token import get_best_robot_token -from upload_result_helper import upload_results -from docker_pull_helper import get_image_with_version -from commit_status_helper import ( - post_commit_status, - fail_simple_check, -) from clickhouse_helper import ( ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse, ) -from stopwatch import Stopwatch +from commit_status_helper import fail_simple_check, post_commit_status +from docker_pull_helper import get_image_with_version +from env_helper import GITHUB_WORKSPACE, RUNNER_TEMP +from get_robot_token import get_best_robot_token +from github_helper import GitHub +from pr_info import PRInfo from rerun_helper import RerunHelper +from s3_helper import S3Helper +from stopwatch import Stopwatch +from upload_result_helper import upload_results NAME = "Style Check (actions)" @@ -58,7 +52,8 @@ def process_result(result_folder): try: results_path = os.path.join(result_folder, "test_results.tsv") - test_results = list(csv.reader(open(results_path, "r"), delimiter="\t")) + with open(results_path, "r", encoding="utf-8") as fd: + test_results = list(csv.reader(fd, delimiter="\t")) if len(test_results) == 0: raise Exception("Empty results") @@ -79,7 +74,7 @@ if __name__ == "__main__": pr_info = PRInfo() - gh = Github(get_best_robot_token()) + gh = GitHub(get_best_robot_token()) rerun_helper = RerunHelper(gh, pr_info, NAME) if rerun_helper.is_already_finished_by_status(): @@ -111,7 +106,7 @@ if __name__ == "__main__": report_url = upload_results( s3_helper, pr_info.number, pr_info.sha, test_results, additional_files, NAME ) - print("::notice ::Report url: {}".format(report_url)) + print(f"::notice ::Report url: {report_url}") post_commit_status(gh, pr_info.sha, NAME, description, state, report_url) prepared_events = prepare_tests_results_for_clickhouse( From 7d2cf648df84c057f91ce63f2bd7f36a6215ce3c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 19 Jul 2022 13:38:30 +0200 Subject: [PATCH 516/659] Make a docker image for style check lighter --- docker/test/style/Dockerfile | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index c2ed47a1392..0ec3f09ab7f 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -17,7 +17,9 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ python3-pip \ shellcheck \ yamllint \ - && pip3 install black boto3 codespell dohq-artifactory PyGithub unidiff pylint==2.6.2 + && pip3 install black boto3 codespell dohq-artifactory PyGithub unidiff pylint==2.6.2 \ + && apt-get clean \ + && rm -rf /root/.cache/pip # Architecture of the image when BuildKit/buildx is used ARG TARGETARCH From f60dcc30fbb8cc1968ecb15622a98df042b5b7c8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 19 Jul 2022 15:17:05 +0200 Subject: [PATCH 517/659] Push automatic fix for black format for PRs --- .github/workflows/docs_check.yml | 3 ++ .github/workflows/master.yml | 2 +- .github/workflows/pull_request.yml | 3 ++ tests/ci/style_check.py | 77 ++++++++++++++++++++++++++++++ utils/check-style/check-black | 8 +++- 5 files changed, 91 insertions(+), 2 deletions(-) diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index 0c657a245cb..b50584a2c01 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -102,6 +102,9 @@ jobs: run: | cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{ runner.temp }}/style_check + ROBOT_CLICKHOUSE_SSH_KEY<> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{ runner.temp }}/style_check + ROBOT_CLICKHOUSE_SSH_KEY<"$tmp" 2>&1; then +# Find all *.py files in the repo except the contrib directory +find_cmd=(find "$GIT_ROOT" -name '*.py' -not -path "$GIT_ROOT/contrib/*") +if ! "${find_cmd[@]}" -exec black --check --diff {} + 1>"$tmp" 2>&1; then # Show the result only if some files need formatting cat "$tmp" + # Apply formatting + "${find_cmd[@]}" -exec black {} + 1>/dev/null 2>&1 + # Automatically add changed files to stage + "${find_cmd[@]}" -exec git add -u {} + 1>/dev/null 2>&1 fi rm "$tmp" From 685a8f3949f60ecffca76f5c0a272a5a36ba650d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 20 Jul 2022 15:20:58 +0200 Subject: [PATCH 518/659] Intentionally break black formatting --- tests/ci/style_check.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index dd63909ad39..296cef0702a 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -90,9 +90,9 @@ def checkout_head(pr_info: PRInfo): remote_url = pr_info.event["pull_request"]["base"]["repo"]["ssh_url"] git_prefix = ( # All commits to remote are done as robot-clickhouse "git -c user.email=robot-clickhouse@clickhouse.com " - "-c user.name=robot-clickhouse -c commit.gpgsign=false " - "-c core.sshCommand=" - "'ssh -o UserKnownHostsFile=/dev/null -o StrictHostKeyChecking=no'" + "-c user.name=robot-clickhouse -c commit.gpgsign=false " + "-c core.sshCommand=" + "'ssh -o UserKnownHostsFile=/dev/null -o StrictHostKeyChecking=no'" ) fetch_cmd = ( f"{git_prefix} fetch --depth=1 " From 122a1123b232d2b85a76ae3684fda63d2f810db3 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 21 Jul 2022 14:57:31 +0000 Subject: [PATCH 519/659] - disable the worst case for distinct in order in perf test for now + functional test for query with the worst perfomance + debug logging in DistinctStep --- src/Processors/QueryPlan/DistinctStep.cpp | 25 +++++++++++++++++++ tests/performance/distinct_in_order.xml | 3 ++- ...tinct_in_order_optimization_long.reference | 1 + ...17_distinct_in_order_optimization_long.sql | 14 +++++++++++ 4 files changed, 42 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02317_distinct_in_order_optimization_long.reference create mode 100644 tests/queries/0_stateless/02317_distinct_in_order_optimization_long.sql diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index b9a8932b409..103f0f064a0 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -52,6 +52,27 @@ static SortDescription getSortDescription(const SortDescription & input_sort_des return distinct_sort_desc; } +static Poco::Logger * getLogger() +{ + static Poco::Logger & logger = Poco::Logger::get("DistinctStep"); + return &logger; +} + +static String dumpColumnNames(const Names & columns) +{ + WriteBufferFromOwnString wb; + bool first = true; + + for (const auto & name : columns) + { + if (!first) + wb << ", "; + first = false; + + wb << name; + } + return wb.str(); +} DistinctStep::DistinctStep( const DataStream & input_stream_, @@ -91,7 +112,11 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil if (optimize_distinct_in_order) { + LOG_DEBUG(getLogger(), "Input sort description ({}): {}", input_stream.sort_description.size(), dumpSortDescription(input_stream.sort_description)); + LOG_DEBUG(getLogger(), "Distinct columns ({}): {}", columns.size(), dumpColumnNames(columns)); SortDescription distinct_sort_desc = getSortDescription(input_stream.sort_description, columns); + LOG_DEBUG(getLogger(), "Distinct sort description ({}): {}", distinct_sort_desc.size(), dumpSortDescription(distinct_sort_desc)); + if (!distinct_sort_desc.empty()) { const bool sorted_stream = input_stream.sort_mode == DataStream::SortMode::Stream; diff --git a/tests/performance/distinct_in_order.xml b/tests/performance/distinct_in_order.xml index 834a6945622..b2c117785f9 100644 --- a/tests/performance/distinct_in_order.xml +++ b/tests/performance/distinct_in_order.xml @@ -27,7 +27,8 @@ SELECT DISTINCT low, medium FROM distinct_cardinality_low ORDER BY medium FORMAT Null SELECT DISTINCT low, medium FROM distinct_cardinality_low ORDER BY low FORMAT Null SELECT DISTINCT low, high FROM distinct_cardinality_low ORDER BY high FORMAT Null - SELECT DISTINCT low, medium, high FROM distinct_cardinality_low ORDER BY high FORMAT Null + + DROP TABLE IF EXISTS distinct_cardinality_low diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_long.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_long.reference new file mode 100644 index 00000000000..0d4d005b74c --- /dev/null +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_long.reference @@ -0,0 +1 @@ +-- check that slow query with distinct in order has the same result as ordinary distinct diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_long.sql b/tests/queries/0_stateless/02317_distinct_in_order_optimization_long.sql new file mode 100644 index 00000000000..7034538a256 --- /dev/null +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_long.sql @@ -0,0 +1,14 @@ +select '-- check that slow query with distinct in order has the same result as ordinary distinct'; +drop table if exists distinct_cardinality_low sync; +drop table if exists distinct_in_order sync; +drop table if exists ordinary_distinct sync; +CREATE TABLE distinct_cardinality_low (low UInt64, medium UInt64, high UInt64) ENGINE MergeTree() ORDER BY (low, medium); +INSERT INTO distinct_cardinality_low SELECT number % 1e2, number % 1e4, number % 1e6 FROM numbers_mt(1e8); +create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +insert into distinct_in_order select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=1; +create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +insert into ordinary_distinct select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=0; +select distinct * from distinct_in_order except select * from ordinary_distinct; +drop table if exists distinct_in_order; +drop table if exists ordinary_distinct; +drop table if exists distinct_cardinality_low; From 17de7b28768e77d842bc89b26abad44f96331c7a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 21 Jul 2022 17:14:34 +0200 Subject: [PATCH 520/659] Revert "Fix trivial count optimization with array join" --- src/Interpreters/TreeRewriter.cpp | 2 +- ...2367_optimize_trivial_count_with_array_join.reference | 1 - .../02367_optimize_trivial_count_with_array_join.sql | 9 --------- 3 files changed, 1 insertion(+), 11 deletions(-) delete mode 100644 tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.reference delete mode 100644 tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index d5a3b26ffc6..b389c3eb705 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1016,7 +1016,7 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select has_explicit_columns = !required.empty(); if (is_select && !has_explicit_columns) { - optimize_trivial_count = !columns_context.has_array_join; + optimize_trivial_count = true; /// You need to read at least one column to find the number of rows. /// We will find a column with minimum . diff --git a/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.reference b/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.reference deleted file mode 100644 index 0cfbf08886f..00000000000 --- a/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.reference +++ /dev/null @@ -1 +0,0 @@ -2 diff --git a/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql b/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql deleted file mode 100644 index 8d812c56ffb..00000000000 --- a/tests/queries/0_stateless/02367_optimize_trivial_count_with_array_join.sql +++ /dev/null @@ -1,9 +0,0 @@ -drop table if exists t; - -create table t(id UInt32) engine MergeTree order by id; - -insert into t values (1); - -select count() from t array join range(2) as a settings optimize_trivial_count_query = 1; - -drop table t; From 5071295a6a0a1beedcbae1b9795d935a7c48c536 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Jul 2022 11:43:00 -0400 Subject: [PATCH 521/659] mount.h for OS_FREEBSD --- utils/self-extracting-executable/decompressor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 74a436c5185..a8d80b76929 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -1,9 +1,9 @@ #include #include -#if defined OS_DARWIN -#include +#if defined(OS_DARWIN) || defined(OS_FREEBSD) +# include #else -#include +# include #endif #include #include From 50bd8147fd92c3e5442407b43f7c5c414598e58f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 21 Jul 2022 17:59:34 +0200 Subject: [PATCH 522/659] Make lightweight delete work with compact parts (not in optimal way yet) --- src/Interpreters/MutationsInterpreter.cpp | 10 +- ...lete_on_merge_tree_compact_parts.reference | 44 ++++++++ ...ght_delete_on_merge_tree_compact_parts.sql | 103 ++++++++++++++++++ 3 files changed, 152 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree_compact_parts.reference create mode 100644 tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree_compact_parts.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 7778e316b8b..18f8b493ad6 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -792,11 +792,11 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & if (i > 0) prepared_stages[i].output_columns = prepared_stages[i - 1].output_columns; - if (prepared_stages[i].output_columns.size() < all_columns.size()) - { - for (const auto & kv : prepared_stages[i].column_to_updated) - prepared_stages[i].output_columns.insert(kv.first); - } + /// Make sure that all updated columns are included into output_columns set. + /// This is important for a "hidden" column like _row_exists gets because it is a virtual column + /// and so it is not in the list of AllPhysical columns. + for (const auto & kv : prepared_stages[i].column_to_updated) + prepared_stages[i].output_columns.insert(kv.first); } /// Now, calculate `expressions_chain` for each stage except the first. diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree_compact_parts.reference b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree_compact_parts.reference new file mode 100644 index 00000000000..f2b9f06ebff --- /dev/null +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree_compact_parts.reference @@ -0,0 +1,44 @@ +1 Compact +1 Compact +99 +1 +1 Compact +95 +1 +1 Compact +0 +1 +5 Compact +-----lightweight mutation type----- +1 +1 +1 +UPDATE _row_exists = 0 WHERE (c % 5) = 1 1 +UPDATE _row_exists = 0 WHERE c = 4 1 +MATERIALIZE INDEX i_c 1 +UPDATE b = -1 WHERE a < 3 1 +DROP INDEX i_c 1 +-----Check that select and merge with lightweight delete.----- +7 +0 -1 0 +2 -1 2 +3 3 3 +5 5 5 +7 7 7 +8 8 8 +9 9 9 +t_light 0 0_1_1_0_10 2 +t_light 1 1_2_2_0_10 2 +t_light 2 2_3_3_0_10 2 +t_light 3 3_4_4_0_10 2 +t_light 4 4_5_5_0_10 2 +7 +t_light 0 0_1_1_1_10 2 +t_light 2 2_3_3_1_10 2 +t_light 3 3_4_4_1_10 2 +t_light 4 4_5_5_1_10 1 +-----Test lightweight delete in multi blocks----- +1 +1 +1000 -2 +1005 -2 diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree_compact_parts.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree_compact_parts.sql new file mode 100644 index 00000000000..f47560ba95b --- /dev/null +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree_compact_parts.sql @@ -0,0 +1,103 @@ +DROP TABLE IF EXISTS merge_table_standard_delete; + +CREATE TABLE merge_table_standard_delete(id Int32, name String) ENGINE = MergeTree order by id settings min_bytes_for_wide_part=10000000; + +INSERT INTO merge_table_standard_delete select number, toString(number) from numbers(100); + +SELECT COUNT(), part_type FROM system.parts WHERE database = currentDatabase() AND table = 'merge_table_standard_delete' AND active GROUP BY part_type ORDER BY part_type; + +SET mutations_sync = 1; +--SET allow_experimental_lightweight_delete = 0; +SET allow_experimental_lightweight_delete_with_row_exists = 1; + +DELETE FROM merge_table_standard_delete WHERE id = 10; +SELECT COUNT(), part_type FROM system.parts WHERE database = currentDatabase() AND table = 'merge_table_standard_delete' AND active GROUP BY part_type ORDER BY part_type; + +SELECT COUNT() FROM merge_table_standard_delete; + +DETACH TABLE merge_table_standard_delete; +ATTACH TABLE merge_table_standard_delete; +CHECK TABLE merge_table_standard_delete; + +DELETE FROM merge_table_standard_delete WHERE name IN ('1','2','3','4'); +SELECT COUNT(), part_type FROM system.parts WHERE database = currentDatabase() AND table = 'merge_table_standard_delete' AND active GROUP BY part_type ORDER BY part_type; + +SELECT COUNT() FROM merge_table_standard_delete; + +DETACH TABLE merge_table_standard_delete; +ATTACH TABLE merge_table_standard_delete; +CHECK TABLE merge_table_standard_delete; + +DELETE FROM merge_table_standard_delete WHERE 1; +SELECT COUNT(), part_type FROM system.parts WHERE database = currentDatabase() AND table = 'merge_table_standard_delete' AND active GROUP BY part_type ORDER BY part_type; + +SELECT COUNT() FROM merge_table_standard_delete; + +DETACH TABLE merge_table_standard_delete; +ATTACH TABLE merge_table_standard_delete; +CHECK TABLE merge_table_standard_delete; + +DROP TABLE merge_table_standard_delete; + +drop table if exists t_light; +create table t_light(a int, b int, c int, index i_c(b) type minmax granularity 4) engine = MergeTree order by a partition by c % 5 settings min_bytes_for_wide_part=10000000; +INSERT INTO t_light SELECT number, number, number FROM numbers(10); +SELECT COUNT(), part_type FROM system.parts WHERE database = currentDatabase() AND table = 't_light' AND active GROUP BY part_type ORDER BY part_type; + +SELECT '-----lightweight mutation type-----'; + +DELETE FROM t_light WHERE c%5=1; + +DETACH TABLE t_light; +ATTACH TABLE t_light; +CHECK TABLE t_light; + +DELETE FROM t_light WHERE c=4; + +DETACH TABLE t_light; +ATTACH TABLE t_light; +CHECK TABLE t_light; + +alter table t_light MATERIALIZE INDEX i_c; +alter table t_light update b=-1 where a<3; +alter table t_light drop index i_c; + +DETACH TABLE t_light; +ATTACH TABLE t_light; +CHECK TABLE t_light; + +SELECT command, is_done FROM system.mutations WHERE database = currentDatabase() AND table = 't_light'; + +SELECT '-----Check that select and merge with lightweight delete.-----'; +select count(*) from t_light; +select * from t_light order by a; + +select table, partition, name, rows from system.parts where database = currentDatabase() AND active and table ='t_light' order by name; + +optimize table t_light final; +select count(*) from t_light; + +select table, partition, name, rows from system.parts where database = currentDatabase() AND active and table ='t_light' and rows > 0 order by name; + +drop table t_light; + +SELECT '-----Test lightweight delete in multi blocks-----'; +CREATE TABLE t_large(a UInt32, b int) ENGINE=MergeTree order BY a settings min_bytes_for_wide_part=0; +INSERT INTO t_large SELECT number + 1, number + 1 FROM numbers(100000); + +DELETE FROM t_large WHERE a = 50000; + +DETACH TABLE t_large; +ATTACH TABLE t_large; +CHECK TABLE t_large; + +ALTER TABLE t_large UPDATE b = -2 WHERE a between 1000 and 1005; +ALTER TABLE t_large DELETE WHERE a=1; + +DETACH TABLE t_large; +ATTACH TABLE t_large; +CHECK TABLE t_large; + +SELECT * FROM t_large WHERE a in (1,1000,1005,50000) order by a; + +DROP TABLE t_large; From aa8dae5f720feb9f7e874ef1f3beca05d2d280e3 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 21 Jul 2022 12:04:11 -0400 Subject: [PATCH 523/659] add nullptr check for flushAsynchronousDecompressRequests --- src/Compression/CompressedReadBufferBase.cpp | 10 ++++------ src/Compression/CompressedReadBufferFromFile.cpp | 3 ++- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 244450414ba..38f580e2f11 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -304,16 +304,14 @@ void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_d void CompressedReadBufferBase::flushAsynchronousDecompressRequests() const { - /// The codec should be created at the latest in readHeaderAndGetCodec - assert(codec.get()); - codec->flushAsynchronousDecompressRequests(); + if (!codec) + codec->flushAsynchronousDecompressRequests(); } void CompressedReadBufferBase::setDecompressMode(ICompressionCodec::CodecMode mode) { - /// The codec should be created at the latest in readHeaderAndGetCodec - assert(codec.get()); - codec->setDecompressMode(mode); + if (!codec) + codec->setDecompressMode(mode); } /// 'compressed_in' could be initialized lazily, but before first call of 'readCompressedData'. diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index 3d2db3cdf93..68f6757e04d 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -149,6 +149,7 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) assert(size_decompressed + additional_size_at_the_end_of_buffer > 0); memory.resize(size_decompressed + additional_size_at_the_end_of_buffer); working_buffer = Buffer(memory.data(), &memory[size_decompressed]); + /// Synchronous mode must be set since we need read partial data immediately from working buffer to target buffer. setDecompressMode(ICompressionCodec::CodecMode::Synchronous); decompress(working_buffer, size_decompressed, size_compressed_without_checksum); @@ -166,9 +167,9 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) /// This is for clang static analyzer. assert(size_decompressed + additional_size_at_the_end_of_buffer > 0); - memory.resize(size_decompressed + additional_size_at_the_end_of_buffer); working_buffer = Buffer(memory.data(), &memory[size_decompressed]); + // Asynchronous mode can be set here because working_buffer wouldn't be overwritten any more since this is the last block. setDecompressMode(ICompressionCodec::CodecMode::Asynchronous); decompress(working_buffer, size_decompressed, size_compressed_without_checksum); read_tail = true; From 0d34814629bad257d7667a9459b09fc66605d940 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 21 Jul 2022 16:39:14 +0000 Subject: [PATCH 524/659] Automatic style fix --- tests/ci/style_check.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 296cef0702a..dd63909ad39 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -90,9 +90,9 @@ def checkout_head(pr_info: PRInfo): remote_url = pr_info.event["pull_request"]["base"]["repo"]["ssh_url"] git_prefix = ( # All commits to remote are done as robot-clickhouse "git -c user.email=robot-clickhouse@clickhouse.com " - "-c user.name=robot-clickhouse -c commit.gpgsign=false " - "-c core.sshCommand=" - "'ssh -o UserKnownHostsFile=/dev/null -o StrictHostKeyChecking=no'" + "-c user.name=robot-clickhouse -c commit.gpgsign=false " + "-c core.sshCommand=" + "'ssh -o UserKnownHostsFile=/dev/null -o StrictHostKeyChecking=no'" ) fetch_cmd = ( f"{git_prefix} fetch --depth=1 " From cf0b01b18942de442468fcaa2d96b86f8637aaf1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 19:21:28 +0200 Subject: [PATCH 525/659] Update version to 22.8.1.1 --- cmake/autogenerated_versions.txt | 10 +++--- .../StorageSystemContributors.generated.cpp | 34 +++++++++++++++++++ 2 files changed, 39 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index e6c60e74c36..edc7805150b 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54464) +SET(VERSION_REVISION 54465) SET(VERSION_MAJOR 22) -SET(VERSION_MINOR 7) +SET(VERSION_MINOR 8) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH 7000c4e0033bb9e69050ab8ef73e8e7465f78059) -SET(VERSION_DESCRIBE v22.7.1.1-testing) -SET(VERSION_STRING 22.7.1.1) +SET(VERSION_GITHASH f4f05ec786a8b8966dd0ea2a2d7e39a8c7db24f4) +SET(VERSION_DESCRIBE v22.8.1.1-testing) +SET(VERSION_STRING 22.8.1.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index 5f5a7887e80..d86a0d4f5df 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -131,6 +131,7 @@ const char * auto_contributors[] { "Anton Okhitin", "Anton Okulov", "Anton Patsev", + "Anton Petrov", "Anton Popov", "Anton Tihonov", "Anton Tikhonov", @@ -149,6 +150,7 @@ const char * auto_contributors[] { "Artem Zuikov", "Artemeey", "Artemkin Pavel", + "Arthur Passos", "Arthur Petukhovsky", "Arthur Tokarchuk", "Arthur Wong", @@ -193,7 +195,9 @@ const char * auto_contributors[] { "Chao Ma", "Chao Wang", "CheSema", + "Chebarykov Pavel", "Chen Yufei", + "Cheng Pan", "Chienlung Cheung", "Christian", "Christoph Wurm", @@ -248,6 +252,7 @@ const char * auto_contributors[] { "Dmitry Moskowski", "Dmitry Muzyka", "Dmitry Novik", + "Dmitry Pavlov", "Dmitry Petukhov", "Dmitry Rubashkin", "Dmitry S..ky / skype: dvska-at-skype", @@ -280,6 +285,7 @@ const char * auto_contributors[] { "Evgeniy Udodov", "Evgeny", "Evgeny Konkov", + "Evgeny Kruglov", "Evgeny Markov", "Ewout", "FArthur-cmd", @@ -323,6 +329,7 @@ const char * auto_contributors[] { "Grigory", "Grigory Buteyko", "Grigory Pervakov", + "GruffGemini", "Guillaume Tassery", "Guo Wei (William)", "Haavard Kvaalen", @@ -330,6 +337,7 @@ const char * auto_contributors[] { "HaiBo Li", "Hamoon", "Han Fei", + "Harry Lee", "Harry-Lee", "HarryLeeIBM", "Hasitha Kanchana", @@ -386,6 +394,7 @@ const char * auto_contributors[] { "Jake Liu", "Jakub Kuklis", "James Maidment", + "James Morrison", "JaosnHsieh", "Jason", "Jason Keirstead", @@ -402,6 +411,7 @@ const char * auto_contributors[] { "John Hummel", "John Skopis", "Jonatas Freitas", + "Jordi Villar", "João Figueiredo", "Julian Gilyadov", "Julian Zhou", @@ -444,6 +454,7 @@ const char * auto_contributors[] { "Larry Luo", "Lars Eidnes", "Latysheva Alexandra", + "Laurie Li", "Lemore", "Leonardo Cecchi", "Leonid Krylov", @@ -516,6 +527,7 @@ const char * auto_contributors[] { "Michael Monashev", "Michael Nutt", "Michael Razuvaev", + "Michael Schnerring", "Michael Smitasin", "Michail Safronov", "Michal Lisowski", @@ -632,6 +644,7 @@ const char * auto_contributors[] { "Pawel Rog", "Peignon Melvyn", "Peng Jian", + "Peng Liu", "Persiyanov Dmitriy Andreevich", "Pervakov Grigorii", "Pervakov Grigory", @@ -643,6 +656,7 @@ const char * auto_contributors[] { "Pxl", "Pysaoke", "Quid37", + "Rafael Acevedo", "Rafael David Tinoco", "Rajkumar", "Rajkumar Varada", @@ -670,6 +684,7 @@ const char * auto_contributors[] { "Roman Nozdrin", "Roman Peshkurov", "Roman Tsisyk", + "Roman Vasin", "Roman Zhukov", "Roy Bellingan", "Ruslan", @@ -685,6 +700,7 @@ const char * auto_contributors[] { "SaltTan", "Sami Kerola", "Samuel Chou", + "San", "Saulius Valatka", "Sean Haynes", "Sean Lafferty", @@ -760,6 +776,7 @@ const char * auto_contributors[] { "Tiaonmmn", "Tigran Khudaverdyan", "Timur Magomedov", + "Timur Solodovnikov", "TiunovNN", "Tobias Adamson", "Tobias Lins", @@ -814,6 +831,8 @@ const char * auto_contributors[] { "Vladimir C", "Vladimir Ch", "Vladimir Chebotarev", + "Vladimir Chebotaryov", + "Vladimir Galunshchikov", "Vladimir Golovchenko", "Vladimir Goncharov", "Vladimir Klimontovich", @@ -823,6 +842,7 @@ const char * auto_contributors[] { "Vladimir Smirnov", "Vladislav Rassokhin", "Vladislav Smirnov", + "Vladislav V", "Vojtech Splichal", "Volodymyr Kuznetsov", "Vsevolod Orlov", @@ -831,6 +851,7 @@ const char * auto_contributors[] { "W", "Wang Fenjin", "WangZengrui", + "Wangyang Guo", "Weiqing Xu", "William Shallum", "Winter Zhang", @@ -838,6 +859,7 @@ const char * auto_contributors[] { "Xianda Ke", "Xiang Zhou", "Xin Wang", + "Xoel Lopez Barata", "Xudong Zhang", "Y Lu", "Yakko Majuri", @@ -855,6 +877,8 @@ const char * auto_contributors[] { "Yong Wang", "Yong-Hao Zou", "Youenn Lebras", + "Yu, Peng", + "Yuko Takagi", "Yuntao Wu", "Yuri Dyachenko", "Yurii Vlasenko", @@ -871,6 +895,7 @@ const char * auto_contributors[] { "Zijie Lu", "Zoran Pandovski", "a.palagashvili", + "aaapetrenko", "abdrakhmanov", "abel-wang", "abyss7", @@ -933,6 +958,7 @@ const char * auto_contributors[] { "chang.chen", "changvvb", "chasingegg", + "chen", "chen9t", "chengy8934", "chenjian", @@ -1110,6 +1136,8 @@ const char * auto_contributors[] { "linceyou", "lincion", "lingo-xp", + "lingpeng0314", + "lirulei", "listar", "litao91", "liu-bov", @@ -1119,10 +1147,13 @@ const char * auto_contributors[] { "liuyimin", "liyang", "liyang830", + "lokax", "lomberts", "loneylee", "long2ice", + "loyispa", "lthaooo", + "ltrk2", "ltybc-coder", "luc1ph3r", "lulichao", @@ -1213,6 +1244,7 @@ const char * auto_contributors[] { "redclusive", "rfraposa", "ritaank", + "rnbondarenko", "robert", "robot-clickhouse", "robot-metrika-test", @@ -1225,6 +1257,7 @@ const char * auto_contributors[] { "ryzuo", "s-kat", "santaux", + "santrancisco", "satanson", "save-my-heart", "sdk2", @@ -1327,6 +1360,7 @@ const char * auto_contributors[] { "zhangxiao871", "zhangyifan27", "zhangyuli1", + "zhao zhou", "zhen ni", "zhifeng", "zhongyuankai", From e6ff5744e0a8fcf7b1faaab619e0e42a29657616 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 21 Jul 2022 17:47:00 +0000 Subject: [PATCH 526/659] Update version_date.tsv and changelogs after v22.7.1.2484-stable --- docs/changelogs/v22.7.1.2484-stable.md | 468 +++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 469 insertions(+) create mode 100644 docs/changelogs/v22.7.1.2484-stable.md diff --git a/docs/changelogs/v22.7.1.2484-stable.md b/docs/changelogs/v22.7.1.2484-stable.md new file mode 100644 index 00000000000..d306b1b5bd4 --- /dev/null +++ b/docs/changelogs/v22.7.1.2484-stable.md @@ -0,0 +1,468 @@ +--- +sidebar_position: 1 +sidebar_label: 2022 +--- + +# 2022 Changelog + +### ClickHouse release v22.7.1.2484-stable FIXME as compared to v22.6.1.1985-stable + +#### Backward Incompatible Change +* Enable setting `enable_positional_arguments` by default. It allows queries like `SELECT ... ORDER BY 1, 2` where 1, 2 are the references to the select clause. If you need to return the old behavior, disable this setting. [#38204](https://github.com/ClickHouse/ClickHouse/pull/38204) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* `Ordinary` database engine and old storage definition syntax for `*MergeTree` tables are deprecated. By default it's not possible to create new ones. If `system` database has `Ordinary` engine it will be automatically converted to `Atomic` on server startup. There are settings to keep old behavior (`allow_deprecated_database_ordinary` and `allow_deprecated_syntax_for_merge_tree`), but these settings may be removed in future releases. [#38335](https://github.com/ClickHouse/ClickHouse/pull/38335) ([Alexander Tokmakov](https://github.com/tavplubix)). +* * Force rewriting comma join to inner by default (set default value `cross_to_inner_join_rewrite = 2`). To have old behavior set `cross_to_inner_join_rewrite = 1`. [#39326](https://github.com/ClickHouse/ClickHouse/pull/39326) ([Vladimir C](https://github.com/vdimir)). +* Disable format_csv_allow_single_quotes by default. [#37096](https://github.com/ClickHouse/ClickHouse/issues/37096). [#39423](https://github.com/ClickHouse/ClickHouse/pull/39423) ([Kruglov Pavel](https://github.com/Avogar)). + +#### New Feature +* Add new `direct` join algorithm for RocksDB, ref [#33582](https://github.com/ClickHouse/ClickHouse/issues/33582). [#35363](https://github.com/ClickHouse/ClickHouse/pull/35363) ([Vladimir C](https://github.com/vdimir)). +* * Added full sorting merge join algorithm. [#35796](https://github.com/ClickHouse/ClickHouse/pull/35796) ([Vladimir C](https://github.com/vdimir)). +* Add a setting `zstd_window_log_max` to configure max memory usage on zstd decoding when importing external files. Closes [#35693](https://github.com/ClickHouse/ClickHouse/issues/35693). [#37015](https://github.com/ClickHouse/ClickHouse/pull/37015) ([wuxiaobai24](https://github.com/wuxiaobai24)). +* Implement NatsStorage - table engine, which allows to pub/sub to NATS. Closes [#32388](https://github.com/ClickHouse/ClickHouse/issues/32388). [#37171](https://github.com/ClickHouse/ClickHouse/pull/37171) ([tchepavel](https://github.com/tchepavel)). +* Implement table function MongoDB. Allow writes into MongoDB storage / table function. [#37213](https://github.com/ClickHouse/ClickHouse/pull/37213) ([aaapetrenko](https://github.com/aaapetrenko)). +* `clickhouse-keeper` new feature: add support for real-time digest calculation and verification. [#37555](https://github.com/ClickHouse/ClickHouse/pull/37555) ([Antonio Andelic](https://github.com/antonio2368)). +* In [#17202](https://github.com/ClickHouse/ClickHouse/issues/17202) was reported that host_regexp was being tested against only one of the possible PTR responses. This PR makes the necessary changes so that host_regexp is applied against all possible PTR responses and validate if any matches. [#37827](https://github.com/ClickHouse/ClickHouse/pull/37827) ([Arthur Passos](https://github.com/arthurpassos)). +* Support hadoop secure rpc transfer(hadoop.rpc.protection=privacy and hadoop.rpc.protection=integrity). [#37852](https://github.com/ClickHouse/ClickHouse/pull/37852) ([Peng Liu](https://github.com/michael1589)). +* Add struct type support in `StorageHive`. [#38118](https://github.com/ClickHouse/ClickHouse/pull/38118) ([lgbo](https://github.com/lgbo-ustc)). +* Added Base58 encoding/decoding. [#38159](https://github.com/ClickHouse/ClickHouse/pull/38159) ([Andrey Zvonov](https://github.com/zvonand)). +* Add chart visualization to Play UI. [#38197](https://github.com/ClickHouse/ClickHouse/pull/38197) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* support `alter` command on `StorageHive` table. [#38214](https://github.com/ClickHouse/ClickHouse/pull/38214) ([lgbo](https://github.com/lgbo-ustc)). +* Added `CREATE TABLE ... EMPTY AS SELECT` query. It automatically deduces table structure from the SELECT query, but does not fill the table after creation. Resolves [#38049](https://github.com/ClickHouse/ClickHouse/issues/38049). [#38272](https://github.com/ClickHouse/ClickHouse/pull/38272) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Adds new setting `implicit_transaction` to run standalone queries inside a transaction. It handles both creation and closing (via COMMIT if the query succeeded or ROLLBACK if it didn't) of the transaction automatically. [#38344](https://github.com/ClickHouse/ClickHouse/pull/38344) ([Raúl Marín](https://github.com/Algunenano)). +* Allow trailing comma in columns list. closes [#38425](https://github.com/ClickHouse/ClickHouse/issues/38425). [#38440](https://github.com/ClickHouse/ClickHouse/pull/38440) ([chen](https://github.com/xiedeyantu)). +* Compress clickhouse into self-extracting executable (path programs/self-extracting). New build target 'self-extracting' is added. [#38447](https://github.com/ClickHouse/ClickHouse/pull/38447) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Introduced settings `additional_table_filters`. Using this setting, you can specify additional filtering condition for a table which will be applied directly after reading. Example: `select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x settings additional_table_filters={'system.numbers : 'number != 3', 'table_1' : 'x != 2'}`. Introduced setting `additional_result_filter` which specifies additional filtering condition for query result. Closes [#37918](https://github.com/ClickHouse/ClickHouse/issues/37918). [#38475](https://github.com/ClickHouse/ClickHouse/pull/38475) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Add SQLInsert output format. Closes [#38441](https://github.com/ClickHouse/ClickHouse/issues/38441). [#38477](https://github.com/ClickHouse/ClickHouse/pull/38477) ([Kruglov Pavel](https://github.com/Avogar)). +* Downloadable clickhouse executable is compressed self-extracting. [#38653](https://github.com/ClickHouse/ClickHouse/pull/38653) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Support `isNullable` function. This function checks whether it's argument is nullable and return true(1) or false(0). Closes [#38611](https://github.com/ClickHouse/ClickHouse/issues/38611). [#38841](https://github.com/ClickHouse/ClickHouse/pull/38841) ([lokax](https://github.com/lokax)). +* Add functions `translate(string, from_string, to_string)` and `translateUTF8(string, from_string, to_string)`. [#38935](https://github.com/ClickHouse/ClickHouse/pull/38935) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add `compatibility` setting and `system.settings_changes` system table that contains information about changes in settings through ClickHouse versions. Closes [#35972](https://github.com/ClickHouse/ClickHouse/issues/35972). [#38957](https://github.com/ClickHouse/ClickHouse/pull/38957) ([Kruglov Pavel](https://github.com/Avogar)). +* Add the 3rd parameter to the tupleElement function and return it if tuple doesn't have a member. Only works if the 2nd parameter is of type String. Closes [#38872](https://github.com/ClickHouse/ClickHouse/issues/38872). [#38989](https://github.com/ClickHouse/ClickHouse/pull/38989) ([lokax](https://github.com/lokax)). +* Support parseTimedelta function. It can be used like ```sql # ' ', ';', '-', '+', ',', ':' can be used as separators, eg. "1yr-2mo", "2m:6s" SELECT parseTimeDelta('1yr-2mo-4w + 12 days, 3 hours : 1 minute ; 33 seconds');. [#39071](https://github.com/ClickHouse/ClickHouse/pull/39071) ([jiahui-97](https://github.com/jiahui-97)). +* Added options to limit IO operations with remote storage: `max_remote_read_network_bandwidth_for_server` and `max_remote_write_network_bandwidth_for_server`. [#39095](https://github.com/ClickHouse/ClickHouse/pull/39095) ([Sergei Trifonov](https://github.com/serxa)). +* Add `send_logs_source_regexp` setting. Send server text logs with specified regexp to match log source name. Empty means all sources. [#39161](https://github.com/ClickHouse/ClickHouse/pull/39161) ([Amos Bird](https://github.com/amosbird)). +* OpenTelemetry now collects traces without Processors spans by default. To enable Processors spans collection `opentelemetry_trace_processors` setting. [#39170](https://github.com/ClickHouse/ClickHouse/pull/39170) ([Ilya Yatsishin](https://github.com/qoega)). + +#### Performance Improvement +* Add new `local_filesystem_read_method` method `io_uring` based on the asynchronous Linux [io_uring](https://kernel.dk/io_uring.pdf) subsystem, improving read performance almost universally compared to the default `pread` method. [#36103](https://github.com/ClickHouse/ClickHouse/pull/36103) ([Saulius Valatka](https://github.com/sauliusvl)). +* Distinct optimization for sorted columns. Use specialized distinct transformation in case input stream is sorted by column(s) in distinct. Optimization can be applied to pre-distinct, final distinct, or both. Initial implementation by @dimarub2000. [#37803](https://github.com/ClickHouse/ClickHouse/pull/37803) ([Igor Nikonov](https://github.com/devcrafter)). +* Add VBMI optimized copyOverlap32Shuffle for LZ4 decompress. [#37891](https://github.com/ClickHouse/ClickHouse/pull/37891) ([Guo Wangyang](https://github.com/guowangy)). +* Improve performance of `ORDER BY`, `MergeTree` merges, window functions using batch version of `BinaryHeap`. [#38022](https://github.com/ClickHouse/ClickHouse/pull/38022) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix significant join performance regression which was introduced in https://github.com/ClickHouse/ClickHouse/pull/35616 . It's interesting that common join queries such as ssb queries have been 10 times slower for almost 3 months while no one complains. [#38052](https://github.com/ClickHouse/ClickHouse/pull/38052) ([Amos Bird](https://github.com/amosbird)). +* Migrate from the Intel hyperscan library to vectorscan, this speeds up many string matching on non-x86 platforms. [#38171](https://github.com/ClickHouse/ClickHouse/pull/38171) ([Robert Schulze](https://github.com/rschu1ze)). +* Increased parallelism of query plan steps executed after aggregation. [#38295](https://github.com/ClickHouse/ClickHouse/pull/38295) ([Nikita Taranov](https://github.com/nickitat)). +* Improve performance of insertion to columns of type `JSON`. [#38320](https://github.com/ClickHouse/ClickHouse/pull/38320) ([Anton Popov](https://github.com/CurtizJ)). +* Optimized insertion and lookups in the HashTable. [#38413](https://github.com/ClickHouse/ClickHouse/pull/38413) ([Nikita Taranov](https://github.com/nickitat)). +* Fix performance degradation from [#32493](https://github.com/ClickHouse/ClickHouse/issues/32493). [#38417](https://github.com/ClickHouse/ClickHouse/pull/38417) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve performance of column vector replicate using SIMD instructions. Author @zzachimed. [#38565](https://github.com/ClickHouse/ClickHouse/pull/38565) ([Maksim Kita](https://github.com/kitaisreal)). +* Norm and Distance functions for arrays speed up 1.2-2 times. [#38740](https://github.com/ClickHouse/ClickHouse/pull/38740) ([Alexander Gololobov](https://github.com/davenger)). +* A less efficient execution plan can be generated for query with ORDER BY (a, b) than for ORDER BY a, b. [#38873](https://github.com/ClickHouse/ClickHouse/pull/38873) ([Igor Nikonov](https://github.com/devcrafter)). +* Executable UDF, Executable Dictionary, Executable Storage poll subprocess fix 1 second subprocess wait during subprocess termination. [#38929](https://github.com/ClickHouse/ClickHouse/pull/38929) ([Constantine Peresypkin](https://github.com/pkit)). +* * Pushdown filter to the right side of sorting join. [#39123](https://github.com/ClickHouse/ClickHouse/pull/39123) ([Vladimir C](https://github.com/vdimir)). +* Optimize accesses to system.stack_trace. [#39177](https://github.com/ClickHouse/ClickHouse/pull/39177) ([Azat Khuzhin](https://github.com/azat)). + +#### Improvement +* Optimized processing of ORDER BY in window functions. [#34632](https://github.com/ClickHouse/ClickHouse/pull/34632) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Support SQL standard create index and drop index syntax. [#35166](https://github.com/ClickHouse/ClickHouse/pull/35166) ([Jianmei Zhang](https://github.com/zhangjmruc)). +* use simd to re-write the current column replicate funcion and got 2x performance boost in our unit benchmark test. [#37235](https://github.com/ClickHouse/ClickHouse/pull/37235) ([zzachimed](https://github.com/zzachimed)). +* Send profile events for INSERT queries (previously only SELECT was supported). [#37391](https://github.com/ClickHouse/ClickHouse/pull/37391) ([Azat Khuzhin](https://github.com/azat)). +* Implement in order aggregation (`optimize_aggregation_in_order`) for fully materialized projections. [#37469](https://github.com/ClickHouse/ClickHouse/pull/37469) ([Azat Khuzhin](https://github.com/azat)). +* * Bugfixes and performance improvements for `parallel_hash`. [#37648](https://github.com/ClickHouse/ClickHouse/pull/37648) ([Vladimir C](https://github.com/vdimir)). +* Support expressions with window functions. Closes [#19857](https://github.com/ClickHouse/ClickHouse/issues/19857). [#37848](https://github.com/ClickHouse/ClickHouse/pull/37848) ([Dmitry Novik](https://github.com/novikd)). +* S3 single objects are now removed with `RemoveObjectRequest` (sic). Fixed a bug with `S3ObjectStorage` on GCP which did not allow to use `removeFileIfExists` effectively breaking approximately half of `remove` functionality. Automatic detection for `DeleteObjects` S3 API, that is not supported by GCS. This will allow to use GCS without explicit `support_batch_delete=0` in configuration. [#37882](https://github.com/ClickHouse/ClickHouse/pull/37882) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fix refcnt for unused MergeTree parts in SELECT queries (may defer parts removal). [#37913](https://github.com/ClickHouse/ClickHouse/pull/37913) ([Azat Khuzhin](https://github.com/azat)). +* Expose basic Keeper related monitoring data (via ProfileEvents and CurrentMetrics). [#38072](https://github.com/ClickHouse/ClickHouse/pull/38072) ([lingpeng0314](https://github.com/lingpeng0314)). +* Added kerberosInit function and corresponding KerberosInit class as a replacement for kinit executable. Replaced all calls of kinit in Kafka and HDFS code by call of kerberosInit function. Added new integration test. Closes [#27651](https://github.com/ClickHouse/ClickHouse/issues/27651). [#38105](https://github.com/ClickHouse/ClickHouse/pull/38105) ([Roman Vasin](https://github.com/rvasin)). +* * Add setting `multiple_joins_try_to_keep_original_names` to not rewrite identifier name on multiple JOINs rewrite, close [#34697](https://github.com/ClickHouse/ClickHouse/issues/34697). [#38149](https://github.com/ClickHouse/ClickHouse/pull/38149) ([Vladimir C](https://github.com/vdimir)). +* improved trace-visualizer UX. [#38169](https://github.com/ClickHouse/ClickHouse/pull/38169) ([Sergei Trifonov](https://github.com/serxa)). +* Add ability to pass headers to url table function / storage via sql. Closes [#37897](https://github.com/ClickHouse/ClickHouse/issues/37897). [#38176](https://github.com/ClickHouse/ClickHouse/pull/38176) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Enable trace collection for AArch64. [#38181](https://github.com/ClickHouse/ClickHouse/pull/38181) ([Maksim Kita](https://github.com/kitaisreal)). +* Do not skip symlinks in `user_defined` directory during SQL user defined functions loading. Closes [#38042](https://github.com/ClickHouse/ClickHouse/issues/38042). [#38184](https://github.com/ClickHouse/ClickHouse/pull/38184) ([Maksim Kita](https://github.com/kitaisreal)). +* Improve the stability for hive storage integration test. Move the data prepare step into test.py. [#38260](https://github.com/ClickHouse/ClickHouse/pull/38260) ([lgbo](https://github.com/lgbo-ustc)). +* Added background cleanup of subdirectories in `store/`. In some cases clickhouse-server might left garbage subdirectories in `store/` (for example, on unsuccessful table creation) and those dirs were never been removed. Fixes [#33710](https://github.com/ClickHouse/ClickHouse/issues/33710). [#38265](https://github.com/ClickHouse/ClickHouse/pull/38265) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add `DESCRIBE CACHE` query to show cache settings from config. Add `SHOW CACHES` query to show available filesystem caches list. [#38279](https://github.com/ClickHouse/ClickHouse/pull/38279) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add access check for system drop fs cache. Support ON CLUSTER. [#38319](https://github.com/ClickHouse/ClickHouse/pull/38319) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Support `auto_close` option for postgres engine connection. Closes [#31486](https://github.com/ClickHouse/ClickHouse/issues/31486). [#38363](https://github.com/ClickHouse/ClickHouse/pull/38363) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix PostgreSQL database engine incompatibility on upgrade from 21.3 to 22.3. Closes [#36659](https://github.com/ClickHouse/ClickHouse/issues/36659). [#38369](https://github.com/ClickHouse/ClickHouse/pull/38369) ([Kseniia Sumarokova](https://github.com/kssenii)). +* `filesystemAvailable` and similar functions now work in `clickhouse-local`. This closes [#38423](https://github.com/ClickHouse/ClickHouse/issues/38423). [#38424](https://github.com/ClickHouse/ClickHouse/pull/38424) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Hardware benchmark now has support for automatic results uploading. [#38427](https://github.com/ClickHouse/ClickHouse/pull/38427) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The table `system.asynchronous_metric_log` is further optimized for storage space. This closes [#38134](https://github.com/ClickHouse/ClickHouse/issues/38134). See the [YouTube video](https://www.youtube.com/watch?v=0fSp9SF8N8A). [#38428](https://github.com/ClickHouse/ClickHouse/pull/38428) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Functions multiMatchAny(), multiMatchAnyIndex(), multiMatchAllIndices() and their fuzzy variants now accept non-const pattern array argument. [#38485](https://github.com/ClickHouse/ClickHouse/pull/38485) ([Robert Schulze](https://github.com/rschu1ze)). +* Added L2 Squared distance and norm for both arrays and tuples. [#38545](https://github.com/ClickHouse/ClickHouse/pull/38545) ([Julian Gilyadov](https://github.com/israelg99)). +* Add revision() function. [#38555](https://github.com/ClickHouse/ClickHouse/pull/38555) ([Azat Khuzhin](https://github.com/azat)). +* Add `group_by_use_nulls` setting to make aggregation key columns nullable in the case of ROLLUP, CUBE and GROUPING SETS. Closes [#37359](https://github.com/ClickHouse/ClickHouse/issues/37359). [#38642](https://github.com/ClickHouse/ClickHouse/pull/38642) ([Dmitry Novik](https://github.com/novikd)). +* Fix GCS via proxy tunnel usage. [#38726](https://github.com/ClickHouse/ClickHouse/pull/38726) ([Azat Khuzhin](https://github.com/azat)). +* Support `\i file` in clickhouse client / local (similar to psql \i). [#38813](https://github.com/ClickHouse/ClickHouse/pull/38813) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Allow null modifier in columns declaration for table functions. [#38816](https://github.com/ClickHouse/ClickHouse/pull/38816) ([Kruglov Pavel](https://github.com/Avogar)). +* - Deactivate `mutations_finalizing_task` before shutdown to avoid `TABLE_IS_READ_ONLY` errors. [#38851](https://github.com/ClickHouse/ClickHouse/pull/38851) ([Raúl Marín](https://github.com/Algunenano)). +* Fix waiting of shared lock after exclusive lock failure. [#38864](https://github.com/ClickHouse/ClickHouse/pull/38864) ([Azat Khuzhin](https://github.com/azat)). +* Add the ability to specify compression level during data export. [#38907](https://github.com/ClickHouse/ClickHouse/pull/38907) ([Nikolay Degterinsky](https://github.com/evillique)). +* New option `rewrite` in `EXPLAIN AST`. If enabled, it shows AST after it's rewritten, otherwise AST of original query. Disabled by default. [#38910](https://github.com/ClickHouse/ClickHouse/pull/38910) ([Igor Nikonov](https://github.com/devcrafter)). +* - Stop reporting Zookeeper "Node exists" exceptions in system.errors when they are expected. [#38961](https://github.com/ClickHouse/ClickHouse/pull/38961) ([Raúl Marín](https://github.com/Algunenano)). +* Allow to specify globs `* or {expr1, expr2, expr3}` inside a key for `clickhouse-extract-from-config` tool. [#38966](https://github.com/ClickHouse/ClickHouse/pull/38966) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add option enabling that SELECT from the system database requires grant. Details:. [#38970](https://github.com/ClickHouse/ClickHouse/pull/38970) ([Vitaly Baranov](https://github.com/vitlibar)). +* - clearOldLogs: Don't report KEEPER_EXCEPTION on concurrent deletes. [#39016](https://github.com/ClickHouse/ClickHouse/pull/39016) ([Raúl Marín](https://github.com/Algunenano)). +* clickhouse-keeper improvement: persist metainformation about keeper servers to disk. [#39069](https://github.com/ClickHouse/ClickHouse/pull/39069) ([Antonio Andelic](https://github.com/antonio2368)). +* Continue without exception when running out of disk space when using filesystem cache. [#39106](https://github.com/ClickHouse/ClickHouse/pull/39106) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Handling SIGTERM signals from k8s. [#39130](https://github.com/ClickHouse/ClickHouse/pull/39130) ([Timur Solodovnikov](https://github.com/tsolodov)). +* SQL function multiStringAllPositions() now accepts non-const needle arguments. [#39167](https://github.com/ClickHouse/ClickHouse/pull/39167) ([Robert Schulze](https://github.com/rschu1ze)). +* Add merge_algorithm (Undecided, Horizontal, Vertical) to system.part_log. [#39181](https://github.com/ClickHouse/ClickHouse/pull/39181) ([Azat Khuzhin](https://github.com/azat)). +* Improve isNullable/isConstant/isNull/isNotNull performance for LowCardinality argument. [#39192](https://github.com/ClickHouse/ClickHouse/pull/39192) ([Kruglov Pavel](https://github.com/Avogar)). +* - Don't report system.errors when the disk is not rotational. [#39216](https://github.com/ClickHouse/ClickHouse/pull/39216) ([Raúl Marín](https://github.com/Algunenano)). +* Metric `result_bytes` for `INSERT` queries in `system.query_log` shows number of bytes inserted. Previously value was incorrect and stored the same value as `result_rows`. [#39225](https://github.com/ClickHouse/ClickHouse/pull/39225) ([Ilya Yatsishin](https://github.com/qoega)). +* The CPU usage metric in clickhouse-client will be displayed in a better way. Fixes [#38756](https://github.com/ClickHouse/ClickHouse/issues/38756). [#39280](https://github.com/ClickHouse/ClickHouse/pull/39280) ([Sergei Trifonov](https://github.com/serxa)). +* Rethrow exception on filesystem cache initialisation on server startup, better error message. [#39386](https://github.com/ClickHouse/ClickHouse/pull/39386) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Support milliseconds, microseconds and nanoseconds in `parseTimeDelta` function. [#39447](https://github.com/ClickHouse/ClickHouse/pull/39447) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Bug Fix +* Fix crash when executing GRANT ALL ON *.* with ON CLUSTER. It was broken in https://github.com/ClickHouse/ClickHouse/pull/35767. This closes [#38618](https://github.com/ClickHouse/ClickHouse/issues/38618). [#38674](https://github.com/ClickHouse/ClickHouse/pull/38674) ([Vitaly Baranov](https://github.com/vitlibar)). +* * Fixed crash caused by IHiveFile be shared among threads. [#38887](https://github.com/ClickHouse/ClickHouse/pull/38887) ([lgbo](https://github.com/lgbo-ustc)). + +#### Build/Testing/Packaging Improvement +* - Apply Clang Thread Safety Analysis (TSA) annotations to ClickHouse. [#38068](https://github.com/ClickHouse/ClickHouse/pull/38068) ([Robert Schulze](https://github.com/rschu1ze)). +* - System table "system.licenses" is now correctly populated on Mac (Darwin). [#38294](https://github.com/ClickHouse/ClickHouse/pull/38294) ([Robert Schulze](https://github.com/rschu1ze)). +* Handle full queue exception in clickhouse-test. If it happened we need to collect debug info to understand what queries didn't finish. [#38490](https://github.com/ClickHouse/ClickHouse/pull/38490) ([Dmitry Novik](https://github.com/novikd)). +* - Change `all|noarch` packages to architecture-dependent - Fix some documentation for it - Push aarch64|arm64 packages to artifactory and release assets - Fixes [#36443](https://github.com/ClickHouse/ClickHouse/issues/36443). [#38580](https://github.com/ClickHouse/ClickHouse/pull/38580) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add `clickhouse-diagnostics` binary to the packages. [#38647](https://github.com/ClickHouse/ClickHouse/pull/38647) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Always print stacktraces if test queue is full. Follow up [#38490](https://github.com/ClickHouse/ClickHouse/issues/38490) cc @tavplubix. [#38662](https://github.com/ClickHouse/ClickHouse/pull/38662) ([Dmitry Novik](https://github.com/novikd)). +* Align branches within a 32B boundary to make benchmark more stable. [#38988](https://github.com/ClickHouse/ClickHouse/pull/38988) ([Guo Wangyang](https://github.com/guowangy)). +* Fix LSan by fixing getauxval(). [#39299](https://github.com/ClickHouse/ClickHouse/pull/39299) ([Azat Khuzhin](https://github.com/azat)). +* Adapt universal installation script for FreeBSD. [#39302](https://github.com/ClickHouse/ClickHouse/pull/39302) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Fix projection exception when aggregation keys are wrapped inside other functions. This fixes [#37151](https://github.com/ClickHouse/ClickHouse/issues/37151). [#37155](https://github.com/ClickHouse/ClickHouse/pull/37155) ([Amos Bird](https://github.com/amosbird)). +* Fix possible logical error `... with argument with type Nothing and default implementation for Nothing is expected to return result with type Nothing, got ...` in some functions. Closes: [#37610](https://github.com/ClickHouse/ClickHouse/issues/37610) Closes: [#37741](https://github.com/ClickHouse/ClickHouse/issues/37741). [#37759](https://github.com/ClickHouse/ClickHouse/pull/37759) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix incorrect columns order in subqueries of UNION (in case of duplicated columns in subselects may produce incorrect result). [#37887](https://github.com/ClickHouse/ClickHouse/pull/37887) ([Azat Khuzhin](https://github.com/azat)). +* Fix incorrect work of MODIFY ALTER Column with column names that contain dots. Closes [#37907](https://github.com/ClickHouse/ClickHouse/issues/37907). [#37971](https://github.com/ClickHouse/ClickHouse/pull/37971) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix reading of sparse columns from `MergeTree` tables that store their data in S3. [#37978](https://github.com/ClickHouse/ClickHouse/pull/37978) ([Anton Popov](https://github.com/CurtizJ)). +* Fix rounding for `Decimal128/Decimal256` with more than 19-digits long scale. [#38027](https://github.com/ClickHouse/ClickHouse/pull/38027) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix possible crash in `Distributed` async insert in case of removing a replica from config. [#38029](https://github.com/ClickHouse/ClickHouse/pull/38029) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix "Missing columns" for GLOBAL JOIN with CTE w/o alias. [#38056](https://github.com/ClickHouse/ClickHouse/pull/38056) ([Azat Khuzhin](https://github.com/azat)). +* Rewrite tuple functions as literals in backwards-compatibility mode. [#38096](https://github.com/ClickHouse/ClickHouse/pull/38096) ([Anton Kozlov](https://github.com/tonickkozlov)). +* - Fix redundant memory reservation for output block during `ORDER BY`. [#38127](https://github.com/ClickHouse/ClickHouse/pull/38127) ([iyupeng](https://github.com/iyupeng)). +* Fix possible logical error `Bad cast from type DB::IColumn* to DB::ColumnNullable*` in array mapped functions. Closes [#38006](https://github.com/ClickHouse/ClickHouse/issues/38006). [#38132](https://github.com/ClickHouse/ClickHouse/pull/38132) ([Kruglov Pavel](https://github.com/Avogar)). +* * Fix temporary name clash in partial merge join, close [#37928](https://github.com/ClickHouse/ClickHouse/issues/37928). [#38135](https://github.com/ClickHouse/ClickHouse/pull/38135) ([Vladimir C](https://github.com/vdimir)). +* With table ```SQL CREATE TABLE nested_name_tuples ( `a` Tuple(x String, y Tuple(i Int32, j String)) ) ENGINE = Memory; ```. [#38136](https://github.com/ClickHouse/ClickHouse/pull/38136) ([lgbo](https://github.com/lgbo-ustc)). +* Fix bug with nested short-circuit functions that led to execution of arguments even if condition is false. Closes [#38040](https://github.com/ClickHouse/ClickHouse/issues/38040). [#38173](https://github.com/ClickHouse/ClickHouse/pull/38173) ([Kruglov Pavel](https://github.com/Avogar)). +* (Window View is a experimental feature) Fix LOGICAL_ERROR for WINDOW VIEW with incorrect structure. [#38205](https://github.com/ClickHouse/ClickHouse/pull/38205) ([Azat Khuzhin](https://github.com/azat)). +* Update librdkafka submodule to fix crash when an OAUTHBEARER refresh callback is set. [#38225](https://github.com/ClickHouse/ClickHouse/pull/38225) ([Rafael Acevedo](https://github.com/racevedoo)). +* Do not allow recursive usage of OvercommitTracker during logging. Fixes [#37794](https://github.com/ClickHouse/ClickHouse/issues/37794) cc @tavplubix @davenger. [#38246](https://github.com/ClickHouse/ClickHouse/pull/38246) ([Dmitry Novik](https://github.com/novikd)). +* Fix INSERT into Distributed hung due to ProfileEvents. [#38307](https://github.com/ClickHouse/ClickHouse/pull/38307) ([Azat Khuzhin](https://github.com/azat)). +* Fix retries in PostgreSQL engine. [#38310](https://github.com/ClickHouse/ClickHouse/pull/38310) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix optimization in PartialSortingTransform (SIGSEGV and possible incorrect result). [#38324](https://github.com/ClickHouse/ClickHouse/pull/38324) ([Azat Khuzhin](https://github.com/azat)). +* Fix RabbitMQ with formats based on PeekableReadBuffer. Closes [#38061](https://github.com/ClickHouse/ClickHouse/issues/38061). [#38356](https://github.com/ClickHouse/ClickHouse/pull/38356) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix possible `Invalid number of rows in Chunk` in materialised pg. Closes [#37323](https://github.com/ClickHouse/ClickHouse/issues/37323). [#38360](https://github.com/ClickHouse/ClickHouse/pull/38360) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix RabbitMQ configuration with connection string setting. Closes [#36531](https://github.com/ClickHouse/ClickHouse/issues/36531). [#38365](https://github.com/ClickHouse/ClickHouse/pull/38365) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix PostgreSQL engine not using PostgreSQL schema when retrieving array dimension size. Closes [#36755](https://github.com/ClickHouse/ClickHouse/issues/36755). Closes [#36772](https://github.com/ClickHouse/ClickHouse/issues/36772). [#38366](https://github.com/ClickHouse/ClickHouse/pull/38366) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix incorrect result of distributed queries with `DISTINCT` and `LIMIT`. Fixes [#38282](https://github.com/ClickHouse/ClickHouse/issues/38282). [#38371](https://github.com/ClickHouse/ClickHouse/pull/38371) ([Anton Popov](https://github.com/CurtizJ)). +* fix: expose new CH keeper port in Dockerfile clickhouse/clickhouse-keeper fix: use correct KEEPER_CONFIG filename in clickhouse/clickhouse-keeper docker image. [#38462](https://github.com/ClickHouse/ClickHouse/pull/38462) ([Evgeny Kruglov](https://github.com/nordluf)). +* Fix parts removal (will be left forever if they had not been removed on server shutdown) after incorrect server shutdown. [#38486](https://github.com/ClickHouse/ClickHouse/pull/38486) ([Azat Khuzhin](https://github.com/azat)). +* Fixes [#38498](https://github.com/ClickHouse/ClickHouse/issues/38498) Current Implementation is similar to what shell does mentiond by @rschu1ze [here](https://github.com/ClickHouse/ClickHouse/pull/38502#issuecomment-1169057723). [#38502](https://github.com/ClickHouse/ClickHouse/pull/38502) ([Heena Bansal](https://github.com/HeenaBansal2009)). +* Fix table creation to avoid replication issues with pre-22.4 replicas. [#38541](https://github.com/ClickHouse/ClickHouse/pull/38541) ([Raúl Marín](https://github.com/Algunenano)). +* Fix crash for `mapUpdate`, `mapFilter` functions when using with constant map argument. Closes [#38547](https://github.com/ClickHouse/ClickHouse/issues/38547). [#38553](https://github.com/ClickHouse/ClickHouse/pull/38553) ([hexiaoting](https://github.com/hexiaoting)). +* Fix wrong results of countSubstrings() & position() on patterns with 0-bytes. [#38589](https://github.com/ClickHouse/ClickHouse/pull/38589) ([Robert Schulze](https://github.com/rschu1ze)). +* Now it's possible to start a clickhouse-server and attach/detach tables even for tables with the incorrect values of IPv4/IPv6 representation. Proper fix for issue [#35156](https://github.com/ClickHouse/ClickHouse/issues/35156). [#38590](https://github.com/ClickHouse/ClickHouse/pull/38590) ([alesapin](https://github.com/alesapin)). +* Adapt some more nodes to avoid issues with pre-22.4 replicas. [#38627](https://github.com/ClickHouse/ClickHouse/pull/38627) ([Raúl Marín](https://github.com/Algunenano)). +* Fix toHour() monotonicity which can lead to incorrect query result (incorrect index analysis). This fixes [#38333](https://github.com/ClickHouse/ClickHouse/issues/38333). [#38675](https://github.com/ClickHouse/ClickHouse/pull/38675) ([Amos Bird](https://github.com/amosbird)). +* `rankCorr` function will work correctly if some arguments are NaNs. This closes [#38396](https://github.com/ClickHouse/ClickHouse/issues/38396). [#38722](https://github.com/ClickHouse/ClickHouse/pull/38722) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix `parallel_view_processing=1` with `optimize_trivial_insert_select=1`. Fix `max_insert_threads` while pushing to views. [#38731](https://github.com/ClickHouse/ClickHouse/pull/38731) ([Azat Khuzhin](https://github.com/azat)). +* Fix use-after-free for Map combinator that leads to incorrect result. [#38748](https://github.com/ClickHouse/ClickHouse/pull/38748) ([Azat Khuzhin](https://github.com/azat)). +* Fix throwing exception for seekable read from s3 (exception was not thrown). [#38773](https://github.com/ClickHouse/ClickHouse/pull/38773) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix checking whether s3 storage support parallel writes. It resulted in s3 parallel writes not working. [#38792](https://github.com/ClickHouse/ClickHouse/pull/38792) ([chen](https://github.com/xiedeyantu)). +* Fix s3 seekable reads with parallel read buffer. (Affected memory usage during query). Closes [#38258](https://github.com/ClickHouse/ClickHouse/issues/38258). [#38802](https://github.com/ClickHouse/ClickHouse/pull/38802) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update `simdjson`. This fixes [#38621](https://github.com/ClickHouse/ClickHouse/issues/38621). [#38838](https://github.com/ClickHouse/ClickHouse/pull/38838) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* MergeTree fix possible logical error for Vertical merges. [#38859](https://github.com/ClickHouse/ClickHouse/pull/38859) ([Maksim Kita](https://github.com/kitaisreal)). +* - Fix settings profile with seconds unit. [#38896](https://github.com/ClickHouse/ClickHouse/pull/38896) ([Raúl Marín](https://github.com/Algunenano)). +* Fix incorrect partition pruning when there is a nullable partition. This fixes [#38941](https://github.com/ClickHouse/ClickHouse/issues/38941). [#38946](https://github.com/ClickHouse/ClickHouse/pull/38946) ([Amos Bird](https://github.com/amosbird)). +* Fix fsync_part_directory for fetches. [#38993](https://github.com/ClickHouse/ClickHouse/pull/38993) ([Azat Khuzhin](https://github.com/azat)). +* Functions multiMatch[Fuzzy](AllIndices/Any/AnyIndex)() no throw a logical error if the needle argument is empty. [#39012](https://github.com/ClickHouse/ClickHouse/pull/39012) ([Robert Schulze](https://github.com/rschu1ze)). +* Any allocations inside OvercommitTracker may lead to deadlock. Logging was not very informative so it's easier just to remove logging. Fixes [#37794](https://github.com/ClickHouse/ClickHouse/issues/37794). [#39030](https://github.com/ClickHouse/ClickHouse/pull/39030) ([Dmitry Novik](https://github.com/novikd)). +* Fix toHour() monotonicity which can lead to incorrect query result (incorrect index analysis). This fixes [#38333](https://github.com/ClickHouse/ClickHouse/issues/38333). [#39037](https://github.com/ClickHouse/ClickHouse/pull/39037) ([Amos Bird](https://github.com/amosbird)). +* Fix bug in filesystem cache that could happen in some corner case which coincided with cache capacity hitting the limit. Closes [#39066](https://github.com/ClickHouse/ClickHouse/issues/39066). [#39070](https://github.com/ClickHouse/ClickHouse/pull/39070) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix ActionsDAG construction for arguments of window expressions. Fixes [#38538](https://github.com/ClickHouse/ClickHouse/issues/38538) Allow using of higher-order functions in window expressions. [#39112](https://github.com/ClickHouse/ClickHouse/pull/39112) ([Dmitry Novik](https://github.com/novikd)). +* Keep `LowCardinality` type in `tuple()` function. Previously `LowCardinality` type was dropped and elements of created tuple had underlying type of `LowCardinality`. [#39113](https://github.com/ClickHouse/ClickHouse/pull/39113) ([Anton Popov](https://github.com/CurtizJ)). +* Fix error `Block structure mismatch` which could happen for INSERT into table with attached MATERIALIZED VIEW and enabled setting `extremes = 1`. Closes [#29759](https://github.com/ClickHouse/ClickHouse/issues/29759) and [#38729](https://github.com/ClickHouse/ClickHouse/issues/38729). [#39125](https://github.com/ClickHouse/ClickHouse/pull/39125) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix unexpected query result when both `optimize_trivial_count_query` and `empty_result_for_aggregation_by_empty_set` are set to true. This fixes [#39140](https://github.com/ClickHouse/ClickHouse/issues/39140). [#39155](https://github.com/ClickHouse/ClickHouse/pull/39155) ([Amos Bird](https://github.com/amosbird)). +* Fixed error `Not found column Type in block` in selects with `PREWHERE` and read-in-order optimizations. [#39157](https://github.com/ClickHouse/ClickHouse/pull/39157) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix extremely rare race condition in during hardnlinks for remote fs. The only way to reproduce it is concurrent run of backups. [#39190](https://github.com/ClickHouse/ClickHouse/pull/39190) ([alesapin](https://github.com/alesapin)). +* Fix fetch of in-memory part with `allow_remote_fs_zero_copy_replication`. [#39214](https://github.com/ClickHouse/ClickHouse/pull/39214) ([Azat Khuzhin](https://github.com/azat)). +* Fix NOEXCEPT_SCOPE (before it calls std::terminate and looses the exception). [#39229](https://github.com/ClickHouse/ClickHouse/pull/39229) ([Azat Khuzhin](https://github.com/azat)). +* Declare RabbitMQ queue without default arguments `x-max-length` and `x-overflow`. [#39259](https://github.com/ClickHouse/ClickHouse/pull/39259) ([rnbondarenko](https://github.com/rnbondarenko)). +* Fix segmentation fault in MaterializedPostgreSQL database engine, which could happen if some exception occurred at replication initialisation. Closes [#36939](https://github.com/ClickHouse/ClickHouse/issues/36939). [#39272](https://github.com/ClickHouse/ClickHouse/pull/39272) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix incorrect fetch postgresql tables query fro PostgreSQL database engine. Closes [#33502](https://github.com/ClickHouse/ClickHouse/issues/33502). [#39283](https://github.com/ClickHouse/ClickHouse/pull/39283) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix possible UB in MergeTreeBackgroundExecutor (leads to SIGSEGV on race with DROP/DETACH). [#39342](https://github.com/ClickHouse/ClickHouse/pull/39342) ([Azat Khuzhin](https://github.com/azat)). +* Avoid possible abort() in CapnProto on exception descruction. Closes [#30706](https://github.com/ClickHouse/ClickHouse/issues/30706). [#39365](https://github.com/ClickHouse/ClickHouse/pull/39365) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix behaviour of dictHas for direct dictionaries when multiple lookups to the same key are made in a single action. [#39385](https://github.com/ClickHouse/ClickHouse/pull/39385) ([James Morrison](https://github.com/jawm)). +* Fix crash which may happen while reading from dictionary with `DateTime64` attribute. Fixes [#38930](https://github.com/ClickHouse/ClickHouse/issues/38930). [#39391](https://github.com/ClickHouse/ClickHouse/pull/39391) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix WriteBuffer finalize in destructor when cacnel query that could lead to stuck query or even terminate. Closes [#38199](https://github.com/ClickHouse/ClickHouse/issues/38199). [#39396](https://github.com/ClickHouse/ClickHouse/pull/39396) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix UB (stack-use-after-scope) in extactAll(). [#39397](https://github.com/ClickHouse/ClickHouse/pull/39397) ([Azat Khuzhin](https://github.com/azat)). +* Fix incorrect query result when trivial count optimization is in effect with array join. This fixes [#39431](https://github.com/ClickHouse/ClickHouse/issues/39431). [#39444](https://github.com/ClickHouse/ClickHouse/pull/39444) ([Amos Bird](https://github.com/amosbird)). + +#### Bug Fix (user-visible misbehaviour in official stable or prestable release) + +* Disable send_logs_level for INSERT into Distributed to avoid possible hung. [#35075](https://github.com/ClickHouse/ClickHouse/pull/35075) ([Azat Khuzhin](https://github.com/azat)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Add a setting to use more memory for zstd decompression"'. [#38194](https://github.com/ClickHouse/ClickHouse/pull/38194) ([alesapin](https://github.com/alesapin)). +* NO CL ENTRY: 'Revert "Revert "Add a setting to use more memory for zstd decompression""'. [#38196](https://github.com/ClickHouse/ClickHouse/pull/38196) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "ClickHouse's boringssl module updated to the official version of the FIPS compliant."'. [#38201](https://github.com/ClickHouse/ClickHouse/pull/38201) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Fix optimization in PartialSortingTransform (SIGSEGV and possible incorrect result)"'. [#38361](https://github.com/ClickHouse/ClickHouse/pull/38361) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Add support for io_uring read method"'. [#38377](https://github.com/ClickHouse/ClickHouse/pull/38377) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Revert "Fix optimization in PartialSortingTransform (SIGSEGV and possible incorrect result)""'. [#38449](https://github.com/ClickHouse/ClickHouse/pull/38449) ([Maksim Kita](https://github.com/kitaisreal)). +* NO CL ENTRY: 'Don't spoil return code of integration tests runner with redundant tee'. [#38548](https://github.com/ClickHouse/ClickHouse/pull/38548) ([Vladimir Chebotarev](https://github.com/excitoon)). +* NO CL ENTRY: 'Revert "Non Negative Derivative window function"'. [#38551](https://github.com/ClickHouse/ClickHouse/pull/38551) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Upload to S3 compressed self-extracting clickhouse"'. [#38788](https://github.com/ClickHouse/ClickHouse/pull/38788) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* NO CL ENTRY: 'Revert "Smallish updates of dev guide"'. [#38848](https://github.com/ClickHouse/ClickHouse/pull/38848) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Fix toHour() monotonicity which can lead to incorrect query result (incorrect index analysis)"'. [#39001](https://github.com/ClickHouse/ClickHouse/pull/39001) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Fix WriteBuffer finalize in destructor when cacnel query"'. [#39433](https://github.com/ClickHouse/ClickHouse/pull/39433) ([Kruglov Pavel](https://github.com/Avogar)). +* NO CL ENTRY: 'Revert "[RFC] Fix LSan by fixing getauxval()"'. [#39434](https://github.com/ClickHouse/ClickHouse/pull/39434) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Remove broken optimisation in Direct dictionary dictHas implementation"'. [#39461](https://github.com/ClickHouse/ClickHouse/pull/39461) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Fix trivial count optimization with array join"'. [#39466](https://github.com/ClickHouse/ClickHouse/pull/39466) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Separate data storage abstraction for MergeTree [#36555](https://github.com/ClickHouse/ClickHouse/pull/36555) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Randomize settings related to in-order read/aggregation [#36914](https://github.com/ClickHouse/ClickHouse/pull/36914) ([Azat Khuzhin](https://github.com/azat)). +* Merge tree reader support for multiple read/filter steps: row level filter, prewhere, ... [#37165](https://github.com/ClickHouse/ClickHouse/pull/37165) ([Alexander Gololobov](https://github.com/davenger)). +* Backup Improvements 6 [#37358](https://github.com/ClickHouse/ClickHouse/pull/37358) ([Vitaly Baranov](https://github.com/vitlibar)). +* Move `updateInputStream` to `ITransformingStep` [#37393](https://github.com/ClickHouse/ClickHouse/pull/37393) ([Nikita Taranov](https://github.com/nickitat)). +* Proper wait of the clickhouse-server in tests [#37560](https://github.com/ClickHouse/ClickHouse/pull/37560) ([Azat Khuzhin](https://github.com/azat)). +* Upgrade curl to 7.83.1 [#37795](https://github.com/ClickHouse/ClickHouse/pull/37795) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* Try fix flaky tests with transactions [#37822](https://github.com/ClickHouse/ClickHouse/pull/37822) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Send perf tests results to ci database [#37841](https://github.com/ClickHouse/ClickHouse/pull/37841) ([Vladimir C](https://github.com/vdimir)). +* Remove duplicate peak mem log [#37860](https://github.com/ClickHouse/ClickHouse/pull/37860) ([Amos Bird](https://github.com/amosbird)). +* tests: fix log_comment (extra quotes) [#37932](https://github.com/ClickHouse/ClickHouse/pull/37932) ([Azat Khuzhin](https://github.com/azat)). +* Throw exception when xml user profile does not exist [#38024](https://github.com/ClickHouse/ClickHouse/pull/38024) ([nvartolomei](https://github.com/nvartolomei)). +* Add `SYNC` command to internal ZooKeeper client [#38047](https://github.com/ClickHouse/ClickHouse/pull/38047) ([Antonio Andelic](https://github.com/antonio2368)). +* Better support of GCP storage [#38069](https://github.com/ClickHouse/ClickHouse/pull/38069) ([Anton Popov](https://github.com/CurtizJ)). +* Build artifacts upload [#38086](https://github.com/ClickHouse/ClickHouse/pull/38086) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Extract some diff from pr [#36171](https://github.com/ClickHouse/ClickHouse/issues/36171) [#38088](https://github.com/ClickHouse/ClickHouse/pull/38088) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Aggregate functions added restrict into batch methods [#38090](https://github.com/ClickHouse/ClickHouse/pull/38090) ([Maksim Kita](https://github.com/kitaisreal)). +* Add perf checkers to all Jepsen tests [#38091](https://github.com/ClickHouse/ClickHouse/pull/38091) ([Antonio Andelic](https://github.com/antonio2368)). +* Some fixes for tests with tsan [#38106](https://github.com/ClickHouse/ClickHouse/pull/38106) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Bring back [#36396](https://github.com/ClickHouse/ClickHouse/issues/36396) [#38110](https://github.com/ClickHouse/ClickHouse/pull/38110) ([Nikita Taranov](https://github.com/nickitat)). +* More suppressions for backward compatibility check [#38131](https://github.com/ClickHouse/ClickHouse/pull/38131) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Cherry pick [#38137](https://github.com/ClickHouse/ClickHouse/pull/38137) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Disable parameters for non direct executable user defined functions [#38142](https://github.com/ClickHouse/ClickHouse/pull/38142) ([Maksim Kita](https://github.com/kitaisreal)). +* SortDescription compile fix typo [#38144](https://github.com/ClickHouse/ClickHouse/pull/38144) ([Maksim Kita](https://github.com/kitaisreal)). +* Update version after release [#38147](https://github.com/ClickHouse/ClickHouse/pull/38147) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* New changelog and versions updated [#38148](https://github.com/ClickHouse/ClickHouse/pull/38148) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Some fixes for clickhouse-disks [#38150](https://github.com/ClickHouse/ClickHouse/pull/38150) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove processor description from span attributes - it is not working [#38157](https://github.com/ClickHouse/ClickHouse/pull/38157) ([Ilya Yatsishin](https://github.com/qoega)). +* Bump minimum / maximum LLVM to 12 / 14 [#38170](https://github.com/ClickHouse/ClickHouse/pull/38170) ([Robert Schulze](https://github.com/rschu1ze)). +* Disk transaction [#38182](https://github.com/ClickHouse/ClickHouse/pull/38182) ([alesapin](https://github.com/alesapin)). +* Check row size to avoid out of bounds access in PostgreSQLSource [#38190](https://github.com/ClickHouse/ClickHouse/pull/38190) ([Alexander Gololobov](https://github.com/davenger)). +* tests: add no-backward-compatibility-check for 02067_lost_part_s3 [#38195](https://github.com/ClickHouse/ClickHouse/pull/38195) ([Azat Khuzhin](https://github.com/azat)). +* tests/stress: fix TSan detection (enables thread fuzzer for non-TSan builds) [#38207](https://github.com/ClickHouse/ClickHouse/pull/38207) ([Azat Khuzhin](https://github.com/azat)). +* tests: disable 01646_system_restart_replicas_smoke under stress tests [#38212](https://github.com/ClickHouse/ClickHouse/pull/38212) ([Azat Khuzhin](https://github.com/azat)). +* tests/stress: fix TSan detection [#38213](https://github.com/ClickHouse/ClickHouse/pull/38213) ([Azat Khuzhin](https://github.com/azat)). +* buffer's getFileSize small changes [#38227](https://github.com/ClickHouse/ClickHouse/pull/38227) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix test for system table count in diag tool [#38236](https://github.com/ClickHouse/ClickHouse/pull/38236) ([Dale McDiarmid](https://github.com/gingerwizard)). +* Update version_date.tsv after v22.3.7.28-lts [#38237](https://github.com/ClickHouse/ClickHouse/pull/38237) ([github-actions[bot]](https://github.com/apps/github-actions)). +* Changelog attrs [#38238](https://github.com/ClickHouse/ClickHouse/pull/38238) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix reading from s3 in some corner cases [#38239](https://github.com/ClickHouse/ClickHouse/pull/38239) ([Anton Popov](https://github.com/CurtizJ)). +* use utility methods to access x509 struct fields. [#38251](https://github.com/ClickHouse/ClickHouse/pull/38251) ([larryluogit](https://github.com/larryluogit)). +* Don't try to kill empty list of containers in `integration/runner` II [#38269](https://github.com/ClickHouse/ClickHouse/pull/38269) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Improve runners AMI and init scripts [#38273](https://github.com/ClickHouse/ClickHouse/pull/38273) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update thrift to 0.16.0 [#38280](https://github.com/ClickHouse/ClickHouse/pull/38280) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* Extract some diff from [#36171](https://github.com/ClickHouse/ClickHouse/issues/36171) [#38285](https://github.com/ClickHouse/ClickHouse/pull/38285) ([Kseniia Sumarokova](https://github.com/kssenii)). +* fix trace-viz zoom anomalies [#38287](https://github.com/ClickHouse/ClickHouse/pull/38287) ([Sergei Trifonov](https://github.com/serxa)). +* Integration tests volume [#38291](https://github.com/ClickHouse/ClickHouse/pull/38291) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* fix typo in view.md [#38292](https://github.com/ClickHouse/ClickHouse/pull/38292) ([Anton Petrov](https://github.com/gsenseless)). +* Backup improvements 7 [#38299](https://github.com/ClickHouse/ClickHouse/pull/38299) ([Vitaly Baranov](https://github.com/vitlibar)). +* Document why the submodule check does not halt the configuration [#38304](https://github.com/ClickHouse/ClickHouse/pull/38304) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix misleading error message while s3 schema inference [#38306](https://github.com/ClickHouse/ClickHouse/pull/38306) ([Kruglov Pavel](https://github.com/Avogar)). +* Update README.md [#38313](https://github.com/ClickHouse/ClickHouse/pull/38313) ([Yuko Takagi](https://github.com/yukotakagi)). +* Ban projections for zero-copy replication in a right way [#38322](https://github.com/ClickHouse/ClickHouse/pull/38322) ([alesapin](https://github.com/alesapin)). +* Checkout full repositories for performance tests [#38327](https://github.com/ClickHouse/ClickHouse/pull/38327) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fixed comments [#38331](https://github.com/ClickHouse/ClickHouse/pull/38331) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Try to fix 02305_schema_inference_with_globs [#38337](https://github.com/ClickHouse/ClickHouse/pull/38337) ([Kruglov Pavel](https://github.com/Avogar)). +* Extend ZooKeeper list request with support for filtering persistent or ephemeral nodes only [#38338](https://github.com/ClickHouse/ClickHouse/pull/38338) ([Antonio Andelic](https://github.com/antonio2368)). +* Upload logs for getting all tests command [#38343](https://github.com/ClickHouse/ClickHouse/pull/38343) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Followup test fix for ban projections [#38351](https://github.com/ClickHouse/ClickHouse/pull/38351) ([alesapin](https://github.com/alesapin)). +* Added --recursive to clickhouse-disks list [#38354](https://github.com/ClickHouse/ClickHouse/pull/38354) ([Alexander Gololobov](https://github.com/davenger)). +* Adding TLS V13 Test [#38355](https://github.com/ClickHouse/ClickHouse/pull/38355) ([larryluogit](https://github.com/larryluogit)). +* Better exception messages on wrong table engines/functions argument types [#38362](https://github.com/ClickHouse/ClickHouse/pull/38362) ([Kruglov Pavel](https://github.com/Avogar)). +* Better error message for failed odbc query [#38364](https://github.com/ClickHouse/ClickHouse/pull/38364) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Simplify parts commit methods [#38380](https://github.com/ClickHouse/ClickHouse/pull/38380) ([alesapin](https://github.com/alesapin)). +* Update docker-compose to try get rid of v1 errors [#38394](https://github.com/ClickHouse/ClickHouse/pull/38394) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Correct submodule after "base-x" commit [#38414](https://github.com/ClickHouse/ClickHouse/pull/38414) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better hardware benchmark [#38419](https://github.com/ClickHouse/ClickHouse/pull/38419) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Refactoring to enable multi-match functions with non-const needles [#38434](https://github.com/ClickHouse/ClickHouse/pull/38434) ([Robert Schulze](https://github.com/rschu1ze)). +* more consistent work with paths in object storages [#38436](https://github.com/ClickHouse/ClickHouse/pull/38436) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Distinct sorted: calculate column positions once [#38438](https://github.com/ClickHouse/ClickHouse/pull/38438) ([Igor Nikonov](https://github.com/devcrafter)). +* Small improvement of the error message to hint at possible issue [#38458](https://github.com/ClickHouse/ClickHouse/pull/38458) ([Miel Donkers](https://github.com/mdonkers)). +* Fix comment [#38465](https://github.com/ClickHouse/ClickHouse/pull/38465) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Follow up for [#38436](https://github.com/ClickHouse/ClickHouse/issues/38436) [#38466](https://github.com/ClickHouse/ClickHouse/pull/38466) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add 22.7 release webinar. [#38481](https://github.com/ClickHouse/ClickHouse/pull/38481) ([Yuko Takagi](https://github.com/yukotakagi)). +* Add some TSA annotations [#38487](https://github.com/ClickHouse/ClickHouse/pull/38487) ([Alexander Tokmakov](https://github.com/tavplubix)). +* tests: cleanup tmp data in 02335_column_ttl_expired_column_optimization [#38488](https://github.com/ClickHouse/ClickHouse/pull/38488) ([Azat Khuzhin](https://github.com/azat)). +* Cleanup: local clang-tidy warnings founded during review [#38489](https://github.com/ClickHouse/ClickHouse/pull/38489) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix some clang-tidy warnings in headers [#38491](https://github.com/ClickHouse/ClickHouse/pull/38491) ([Robert Schulze](https://github.com/rschu1ze)). +* A tiny improvement in report logging [#38507](https://github.com/ClickHouse/ClickHouse/pull/38507) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* tests: fix 02305_schema_inference_with_globs flakiness [#38511](https://github.com/ClickHouse/ClickHouse/pull/38511) ([Azat Khuzhin](https://github.com/azat)). +* Try to fix flaky test [#38516](https://github.com/ClickHouse/ClickHouse/pull/38516) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `_csv.Error: field larger than field limit` [#38518](https://github.com/ClickHouse/ClickHouse/pull/38518) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix application errors grep in stress test [#38520](https://github.com/ClickHouse/ClickHouse/pull/38520) ([Kruglov Pavel](https://github.com/Avogar)). +* Use of disk batch operations in MergeTree [#38531](https://github.com/ClickHouse/ClickHouse/pull/38531) ([alesapin](https://github.com/alesapin)). +* Backup Improvements 8 [#38537](https://github.com/ClickHouse/ClickHouse/pull/38537) ([Vitaly Baranov](https://github.com/vitlibar)). +* Update poco [#38540](https://github.com/ClickHouse/ClickHouse/pull/38540) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Don't use std::unique_lock unless we need to [#38542](https://github.com/ClickHouse/ClickHouse/pull/38542) ([Robert Schulze](https://github.com/rschu1ze)). +* Rename slightly weirdly named "BuilderBinTidy" to "BuilderBinClangTidy" [#38546](https://github.com/ClickHouse/ClickHouse/pull/38546) ([Robert Schulze](https://github.com/rschu1ze)). +* Don't rollback SessionID request in Keeper [#38556](https://github.com/ClickHouse/ClickHouse/pull/38556) ([Antonio Andelic](https://github.com/antonio2368)). +* Add logging in Epoll and TimerDescriptor in case of EINTR [#38559](https://github.com/ClickHouse/ClickHouse/pull/38559) ([Kruglov Pavel](https://github.com/Avogar)). +* SQL create drop index minor fixes [#38561](https://github.com/ClickHouse/ClickHouse/pull/38561) ([Maksim Kita](https://github.com/kitaisreal)). +* Update version_date.tsv and changelogs after v22.6.2.12-stable [#38563](https://github.com/ClickHouse/ClickHouse/pull/38563) ([github-actions[bot]](https://github.com/apps/github-actions)). +* Allow Ordinary database in Stress Tests [#38568](https://github.com/ClickHouse/ClickHouse/pull/38568) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Make postgres integration test great again [#38582](https://github.com/ClickHouse/ClickHouse/pull/38582) ([Ilya Yatsishin](https://github.com/qoega)). +* Add check for empty proccessors in AggregatingTransform::expandPipeline [#38584](https://github.com/ClickHouse/ClickHouse/pull/38584) ([filimonov](https://github.com/filimonov)). +* quick fix for 02112_with_fill_interval [#38587](https://github.com/ClickHouse/ClickHouse/pull/38587) ([Nikita Taranov](https://github.com/nickitat)). +* Remove zlib in mariadb-connector-c [#38599](https://github.com/ClickHouse/ClickHouse/pull/38599) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* Dictionaries added TSA annotations [#38601](https://github.com/ClickHouse/ClickHouse/pull/38601) ([Maksim Kita](https://github.com/kitaisreal)). +* CacheDictionary simplify update queue [#38602](https://github.com/ClickHouse/ClickHouse/pull/38602) ([Maksim Kita](https://github.com/kitaisreal)). +* Add separate option to omit symbols from heavy contrib [#38617](https://github.com/ClickHouse/ClickHouse/pull/38617) ([Azat Khuzhin](https://github.com/azat)). +* Fix exception messages in clickhouse su [#38619](https://github.com/ClickHouse/ClickHouse/pull/38619) ([filimonov](https://github.com/filimonov)). +* Added Greenplum benchmark [#38622](https://github.com/ClickHouse/ClickHouse/pull/38622) ([Dmitry Pavlov](https://github.com/kapustor)). +* Fix typo [#38623](https://github.com/ClickHouse/ClickHouse/pull/38623) ([tiegen](https://github.com/loyispa)). +* Better diagnostics in ReplicatedMergeTreeQueue [#38641](https://github.com/ClickHouse/ClickHouse/pull/38641) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Clean out randomized integration volumes each run [#38644](https://github.com/ClickHouse/ClickHouse/pull/38644) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update README.md [#38651](https://github.com/ClickHouse/ClickHouse/pull/38651) ([Yuko Takagi](https://github.com/yukotakagi)). +* Better naming for stuff related to splitted debug symbols [#38654](https://github.com/ClickHouse/ClickHouse/pull/38654) ([Robert Schulze](https://github.com/rschu1ze)). +* Add test for keeper `mntr` command [#38656](https://github.com/ClickHouse/ClickHouse/pull/38656) ([alesapin](https://github.com/alesapin)). +* Update hardware benchmark script [#38672](https://github.com/ClickHouse/ClickHouse/pull/38672) ([Filatenkov Artur](https://github.com/FArthur-cmd)). +* Fix strange backport titles issues [#38679](https://github.com/ClickHouse/ClickHouse/pull/38679) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Follow-up to [#38568](https://github.com/ClickHouse/ClickHouse/issues/38568) [#38680](https://github.com/ClickHouse/ClickHouse/pull/38680) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix possible deadlocks with `MergeTreeData::Transaction` [#38702](https://github.com/ClickHouse/ClickHouse/pull/38702) ([alesapin](https://github.com/alesapin)). +* Fix backports diff [#38703](https://github.com/ClickHouse/ClickHouse/pull/38703) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix FillingTransform [#38705](https://github.com/ClickHouse/ClickHouse/pull/38705) ([Nikita Taranov](https://github.com/nickitat)). +* Try to improve backward compatibility check [#38717](https://github.com/ClickHouse/ClickHouse/pull/38717) ([Kruglov Pavel](https://github.com/Avogar)). +* SQL create drop index fix formatting [#38720](https://github.com/ClickHouse/ClickHouse/pull/38720) ([Maksim Kita](https://github.com/kitaisreal)). +* Provide sort description for output stream in ReadFromMergeTree step [#38721](https://github.com/ClickHouse/ClickHouse/pull/38721) ([Igor Nikonov](https://github.com/devcrafter)). +* Add exp_internal for expect tests [#38728](https://github.com/ClickHouse/ClickHouse/pull/38728) ([Azat Khuzhin](https://github.com/azat)). +* Fix CLICKHOUSE_TMP in tests (fixes broken CI) [#38733](https://github.com/ClickHouse/ClickHouse/pull/38733) ([Azat Khuzhin](https://github.com/azat)). +* Add SimpleCheck [#38744](https://github.com/ClickHouse/ClickHouse/pull/38744) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Tiny tests cleanup [#38749](https://github.com/ClickHouse/ClickHouse/pull/38749) ([Azat Khuzhin](https://github.com/azat)). +* Fix replication after improper merge process [#38752](https://github.com/ClickHouse/ClickHouse/pull/38752) ([Raúl Marín](https://github.com/Algunenano)). +* tests: make aggregate_state_exception_memory_leak deterministic [#38754](https://github.com/ClickHouse/ClickHouse/pull/38754) ([Azat Khuzhin](https://github.com/azat)). +* Bump jemalloc to fix possible assertion [#38757](https://github.com/ClickHouse/ClickHouse/pull/38757) ([Azat Khuzhin](https://github.com/azat)). +* Reintroduce nonNegativeDerivative() [#38774](https://github.com/ClickHouse/ClickHouse/pull/38774) ([Andrey Zvonov](https://github.com/zvonand)). +* Temporarily disable 01710_projection_fetch_long in BC check [#38798](https://github.com/ClickHouse/ClickHouse/pull/38798) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Use native Map type for OpenTelemetry attributes [#38814](https://github.com/ClickHouse/ClickHouse/pull/38814) ([Ilya Yatsishin](https://github.com/qoega)). +* Add test for segfault in Map combinator [#38831](https://github.com/ClickHouse/ClickHouse/pull/38831) ([Kruglov Pavel](https://github.com/Avogar)). +* Update libprotobuf-mutator + fix build [#38834](https://github.com/ClickHouse/ClickHouse/pull/38834) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Get files changed in master since release is branched [#38836](https://github.com/ClickHouse/ClickHouse/pull/38836) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* update integration tests doc [#38837](https://github.com/ClickHouse/ClickHouse/pull/38837) ([Bharat Nallan](https://github.com/bharatnc)). +* Revert of revert of smallish devguide update [#38850](https://github.com/ClickHouse/ClickHouse/pull/38850) ([Robert Schulze](https://github.com/rschu1ze)). +* Do not override compiler if it had been already set [#38856](https://github.com/ClickHouse/ClickHouse/pull/38856) ([Azat Khuzhin](https://github.com/azat)). +* Move check for denied allocations [#38858](https://github.com/ClickHouse/ClickHouse/pull/38858) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Refactoring of code around object storages, added LocalObjectStorage (extracted this diff from PR [#36171](https://github.com/ClickHouse/ClickHouse/issues/36171)) [#38860](https://github.com/ClickHouse/ClickHouse/pull/38860) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backup Improvements 9 [#38861](https://github.com/ClickHouse/ClickHouse/pull/38861) ([Vitaly Baranov](https://github.com/vitlibar)). +* Simple cleanup: interpreters and parsers [#38876](https://github.com/ClickHouse/ClickHouse/pull/38876) ([Igor Nikonov](https://github.com/devcrafter)). +* Remove unnecessary log [#38892](https://github.com/ClickHouse/ClickHouse/pull/38892) ([Raúl Marín](https://github.com/Algunenano)). +* Update version_date.tsv and changelogs after v22.6.3.35-stable [#38894](https://github.com/ClickHouse/ClickHouse/pull/38894) ([github-actions[bot]](https://github.com/apps/github-actions)). +* Retry docker buildx commands with progressive sleep in between [#38898](https://github.com/ClickHouse/ClickHouse/pull/38898) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Avoid false positive red sanitizer asserts check in stress test [#38901](https://github.com/ClickHouse/ClickHouse/pull/38901) ([Kruglov Pavel](https://github.com/Avogar)). +* Interpreter cleanup: ContextPtr -> const ContextPtr & in parameters [#38902](https://github.com/ClickHouse/ClickHouse/pull/38902) ([Igor Nikonov](https://github.com/devcrafter)). +* Add a test for simdjson [#38933](https://github.com/ClickHouse/ClickHouse/pull/38933) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix a typo [#38938](https://github.com/ClickHouse/ClickHouse/pull/38938) ([Nikolay Degterinsky](https://github.com/evillique)). +* Avoid redundant join block transformation during planning. [#38943](https://github.com/ClickHouse/ClickHouse/pull/38943) ([Amos Bird](https://github.com/amosbird)). +* Rename NUMBER_OF_DIMENSIONS_MISMATHED const to NUMBER_OF_DIMENSIONS_MISMATCHED [#38947](https://github.com/ClickHouse/ClickHouse/pull/38947) ([Vladimir Galunshchikov](https://github.com/soyayaos)). +* More careful destructor in BackupImpl [#38949](https://github.com/ClickHouse/ClickHouse/pull/38949) ([Vitaly Baranov](https://github.com/vitlibar)). +* Avoid weird exception in Keeper [#38963](https://github.com/ClickHouse/ClickHouse/pull/38963) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update version_date.tsv after v22.3.8.39-lts [#38969](https://github.com/ClickHouse/ClickHouse/pull/38969) ([github-actions[bot]](https://github.com/apps/github-actions)). +* Remove tag no-backward-compatibility-check for specific versions [#38971](https://github.com/ClickHouse/ClickHouse/pull/38971) ([Kruglov Pavel](https://github.com/Avogar)). +* add Hetzner benchmark [#38974](https://github.com/ClickHouse/ClickHouse/pull/38974) ([Tyler Hannan](https://github.com/tylerhannan)). +* Update version_date.tsv after v22.4.6.53-stable [#38975](https://github.com/ClickHouse/ClickHouse/pull/38975) ([github-actions[bot]](https://github.com/apps/github-actions)). +* Disable instrumentation of sanitizer death callback [#38977](https://github.com/ClickHouse/ClickHouse/pull/38977) ([Alexander Tokmakov](https://github.com/tavplubix)). +* add ryzen 9 5950 benchmark [#38979](https://github.com/ClickHouse/ClickHouse/pull/38979) ([Tyler Hannan](https://github.com/tylerhannan)). +* EXPLAIN AST rewrite: rename to optimize [#38980](https://github.com/ClickHouse/ClickHouse/pull/38980) ([Igor Nikonov](https://github.com/devcrafter)). +* add macbook pro core i7 2014 benchmark [#38981](https://github.com/ClickHouse/ClickHouse/pull/38981) ([Tyler Hannan](https://github.com/tylerhannan)). +* add Huawei TaiShan 920 Benchmark [#38982](https://github.com/ClickHouse/ClickHouse/pull/38982) ([Tyler Hannan](https://github.com/tylerhannan)). +* tests: unique ZooKeeper path for Replicated.*MergeTree tables [#38999](https://github.com/ClickHouse/ClickHouse/pull/38999) ([Azat Khuzhin](https://github.com/azat)). +* Try another suppression for [#38629](https://github.com/ClickHouse/ClickHouse/issues/38629) [#39009](https://github.com/ClickHouse/ClickHouse/pull/39009) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add docker_server.py running to backport and release CIs [#39011](https://github.com/ClickHouse/ClickHouse/pull/39011) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix flaky `test_system_merges/test.py::test_mutation_simple` [#39013](https://github.com/ClickHouse/ClickHouse/pull/39013) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix assertion in full soring merge join [#39014](https://github.com/ClickHouse/ClickHouse/pull/39014) ([Vladimir C](https://github.com/vdimir)). +* Fix flaky 00620_optimize_on_nonleader_replica_zookeeper [#39019](https://github.com/ClickHouse/ClickHouse/pull/39019) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Suppress [#38643](https://github.com/ClickHouse/ClickHouse/issues/38643) [#39024](https://github.com/ClickHouse/ClickHouse/pull/39024) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update url.md [#39025](https://github.com/ClickHouse/ClickHouse/pull/39025) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix 'Tried to lock part ... for removal second time' [#39036](https://github.com/ClickHouse/ClickHouse/pull/39036) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add more settings for randomization [#39039](https://github.com/ClickHouse/ClickHouse/pull/39039) ([Anton Popov](https://github.com/CurtizJ)). +* add ScaleFlux CSD3000 Benchmark [#39040](https://github.com/ClickHouse/ClickHouse/pull/39040) ([Tyler Hannan](https://github.com/tylerhannan)). +* BACKUP/RESTORE ON CLUSTER use async mode on replicas now. [#39046](https://github.com/ClickHouse/ClickHouse/pull/39046) ([Vitaly Baranov](https://github.com/vitlibar)). +* More stable `test_s3_zero_copy_ttl`, weakened requirement to move data to S3 in 0-5 seconds [#39064](https://github.com/ClickHouse/ClickHouse/pull/39064) ([Vladimir Chebotaryov](https://github.com/quickhouse)). +* Parameter --decompressor added to utils/self-extracting-executable/compressor [#39065](https://github.com/ClickHouse/ClickHouse/pull/39065) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Follow-up to [#39036](https://github.com/ClickHouse/ClickHouse/issues/39036) [#39091](https://github.com/ClickHouse/ClickHouse/pull/39091) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update registerDiskS3.cpp [#39092](https://github.com/ClickHouse/ClickHouse/pull/39092) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix code in getLeastSupertype function [#39101](https://github.com/ClickHouse/ClickHouse/pull/39101) ([Kruglov Pavel](https://github.com/Avogar)). +* Remove some debug logging [#39102](https://github.com/ClickHouse/ClickHouse/pull/39102) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Prefix overridden add_executable() command with "clickhouse_" [#39108](https://github.com/ClickHouse/ClickHouse/pull/39108) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix meilisearch tests [#39110](https://github.com/ClickHouse/ClickHouse/pull/39110) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Properly remove projection from part in case it was removed from table metadata. [#39119](https://github.com/ClickHouse/ClickHouse/pull/39119) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Update cluster.py [#39120](https://github.com/ClickHouse/ClickHouse/pull/39120) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Tiny updates for tests. [#39127](https://github.com/ClickHouse/ClickHouse/pull/39127) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix data race in CompletedPipelineExecutor. [#39132](https://github.com/ClickHouse/ClickHouse/pull/39132) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix style again [#39133](https://github.com/ClickHouse/ClickHouse/pull/39133) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix path retrieval for Keeper's state [#39148](https://github.com/ClickHouse/ClickHouse/pull/39148) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Slightly better interface of waitForMutation [#39154](https://github.com/ClickHouse/ClickHouse/pull/39154) ([Amos Bird](https://github.com/amosbird)). +* ThreadPool fixes [#39160](https://github.com/ClickHouse/ClickHouse/pull/39160) ([Azat Khuzhin](https://github.com/azat)). +* Add test for [#39132](https://github.com/ClickHouse/ClickHouse/issues/39132) [#39173](https://github.com/ClickHouse/ClickHouse/pull/39173) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Suppression for BC check (`Cannot parse string 'Hello' as UInt64`) [#39176](https://github.com/ClickHouse/ClickHouse/pull/39176) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix 01961_roaring_memory_tracking test [#39187](https://github.com/ClickHouse/ClickHouse/pull/39187) ([Dmitry Novik](https://github.com/novikd)). +* Cleanup: done during [#38719](https://github.com/ClickHouse/ClickHouse/issues/38719) (SortingStep: deduce way to sort based on … [#39191](https://github.com/ClickHouse/ClickHouse/pull/39191) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix exception in AsynchronousMetrics for s390x [#39193](https://github.com/ClickHouse/ClickHouse/pull/39193) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Optimize accesses to system.stack_trace (filter by name before sending signal) [#39212](https://github.com/ClickHouse/ClickHouse/pull/39212) ([Azat Khuzhin](https://github.com/azat)). +* Enable warning "-Wdeprecated-dynamic-exception-spec" [#39213](https://github.com/ClickHouse/ClickHouse/pull/39213) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove specialization global lock/unlock from ActionLocksManager [#39215](https://github.com/ClickHouse/ClickHouse/pull/39215) ([Azat Khuzhin](https://github.com/azat)). +* Turn some warnings on [#39223](https://github.com/ClickHouse/ClickHouse/pull/39223) ([Robert Schulze](https://github.com/rschu1ze)). +* Pass const std::string_view by value, not by reference [#39224](https://github.com/ClickHouse/ClickHouse/pull/39224) ([Kruglov Pavel](https://github.com/Avogar)). +* Minor fix for BC check [#39231](https://github.com/ClickHouse/ClickHouse/pull/39231) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backport script [#39235](https://github.com/ClickHouse/ClickHouse/pull/39235) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Do not color logs on client if they are redirected to file [#39243](https://github.com/ClickHouse/ClickHouse/pull/39243) ([Anton Popov](https://github.com/CurtizJ)). +* Remove incorrect assertion [#39245](https://github.com/ClickHouse/ClickHouse/pull/39245) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add X86 prefix to x86 performance tests [#39251](https://github.com/ClickHouse/ClickHouse/pull/39251) ([Robert Schulze](https://github.com/rschu1ze)). +* Check that the destination for a backup is not in use. [#39254](https://github.com/ClickHouse/ClickHouse/pull/39254) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix stacktraces in gdb in BC check [#39256](https://github.com/ClickHouse/ClickHouse/pull/39256) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Disable flaky test `test_s3_zero_copy_on_hybrid_storage` [#39258](https://github.com/ClickHouse/ClickHouse/pull/39258) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Enabled Wc99-extensions + Wsign-conversion [#39261](https://github.com/ClickHouse/ClickHouse/pull/39261) ([Robert Schulze](https://github.com/rschu1ze)). +* Pass const StringRef by value, not by reference [#39262](https://github.com/ClickHouse/ClickHouse/pull/39262) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix assertion in transactions [#39263](https://github.com/ClickHouse/ClickHouse/pull/39263) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix macosx compilation due to endian.h [#39265](https://github.com/ClickHouse/ClickHouse/pull/39265) ([Jordi Villar](https://github.com/jrdi)). +* Another supression for BC check [#39276](https://github.com/ClickHouse/ClickHouse/pull/39276) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix builder image for releases w/o diagnostics tool [#39281](https://github.com/ClickHouse/ClickHouse/pull/39281) ([Azat Khuzhin](https://github.com/azat)). +* [RFC] Remove superior atomic from MergeTreeBackgroundExecutor and annotations for TSA [#39285](https://github.com/ClickHouse/ClickHouse/pull/39285) ([Azat Khuzhin](https://github.com/azat)). +* Fix clang tidy [#39288](https://github.com/ClickHouse/ClickHouse/pull/39288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix running cmake with predefined cache (for development only) [#39295](https://github.com/ClickHouse/ClickHouse/pull/39295) ([Azat Khuzhin](https://github.com/azat)). +* Fix googletest contrib compilation (due to GTEST_HAS_POSIX_RE=0) [#39298](https://github.com/ClickHouse/ClickHouse/pull/39298) ([Azat Khuzhin](https://github.com/azat)). +* First try at reducing the use of StringRef [#39300](https://github.com/ClickHouse/ClickHouse/pull/39300) ([Robert Schulze](https://github.com/rschu1ze)). +* Whitespaces [#39303](https://github.com/ClickHouse/ClickHouse/pull/39303) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add test for window function inside CASE [#39305](https://github.com/ClickHouse/ClickHouse/pull/39305) ([Dmitry Novik](https://github.com/novikd)). +* Simple Check should be updated on rerun [#39307](https://github.com/ClickHouse/ClickHouse/pull/39307) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix leaking of logger in clickhouse-disks [#39314](https://github.com/ClickHouse/ClickHouse/pull/39314) ([Azat Khuzhin](https://github.com/azat)). +* Update exception message [#39315](https://github.com/ClickHouse/ClickHouse/pull/39315) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix build clang-13 [#39318](https://github.com/ClickHouse/ClickHouse/pull/39318) ([alesapin](https://github.com/alesapin)). +* Auto set test name in integration tests [#39322](https://github.com/ClickHouse/ClickHouse/pull/39322) ([Vitaly Baranov](https://github.com/vitlibar)). +* Try fix flaky test_store_cleanup [#39334](https://github.com/ClickHouse/ClickHouse/pull/39334) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Do not start on unexpected Ordinary metadata [#39337](https://github.com/ClickHouse/ClickHouse/pull/39337) ([Alexander Tokmakov](https://github.com/tavplubix)). +* switch from mkdocs to Docusaurus [#39338](https://github.com/ClickHouse/ClickHouse/pull/39338) ([Dan Roscigno](https://github.com/DanRoscigno)). +* Fix flaky 01174_select_insert_isolation [#39339](https://github.com/ClickHouse/ClickHouse/pull/39339) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Better exception messages in schema inference [#39340](https://github.com/ClickHouse/ClickHouse/pull/39340) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix memory exceptions with transactions [#39341](https://github.com/ClickHouse/ClickHouse/pull/39341) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix typo [#39360](https://github.com/ClickHouse/ClickHouse/pull/39360) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix typo [#39361](https://github.com/ClickHouse/ClickHouse/pull/39361) ([Kruglov Pavel](https://github.com/Avogar)). +* Do not enqueue uneeded parts for check [#39366](https://github.com/ClickHouse/ClickHouse/pull/39366) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Avoid loading toolchain file multiple times to avoid confusing ccache [#39387](https://github.com/ClickHouse/ClickHouse/pull/39387) ([Azat Khuzhin](https://github.com/azat)). +* Fix make clean (due to crosscompile of llvm) [#39392](https://github.com/ClickHouse/ClickHouse/pull/39392) ([Azat Khuzhin](https://github.com/azat)). +* Disable real-time digest in Keeper by default [#39393](https://github.com/ClickHouse/ClickHouse/pull/39393) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix interactive client with older server [#39413](https://github.com/ClickHouse/ClickHouse/pull/39413) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix BC check [#39414](https://github.com/ClickHouse/ClickHouse/pull/39414) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix flaky test test_login_as_dropped_user_xml. [#39415](https://github.com/ClickHouse/ClickHouse/pull/39415) ([Vitaly Baranov](https://github.com/vitlibar)). +* Introduce a dependency to libuv when building NATS [#39427](https://github.com/ClickHouse/ClickHouse/pull/39427) ([ltrk2](https://github.com/ltrk2)). +* Set default value cross_to_inner_join_rewrite = 1 [#39443](https://github.com/ClickHouse/ClickHouse/pull/39443) ([Vladimir C](https://github.com/vdimir)). +* Respect table alias for additional_table_filters. [#39456](https://github.com/ClickHouse/ClickHouse/pull/39456) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### Performance optimization and Bug Fix + +* Enabled `pread_threadpool` read method by default. It will increase read performance. Bug fix: if direct IO is enabled and the number of threads is large and `pread_threadpool` is used, it may cause a logical error. [#33653](https://github.com/ClickHouse/ClickHouse/pull/33653) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index de9e4cfa0af..e4c7aae8b25 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v22.7.1.2484-stable 2022-07-21 v22.6.3.35-stable 2022-07-06 v22.6.2.12-stable 2022-06-29 v22.6.1.1985-stable 2022-06-16 From 79a76abf96167b0396b4cd93d58a0f17d480b8bc Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Jul 2022 17:47:10 +0000 Subject: [PATCH 527/659] Try fix tests --- .../02327_capnproto_protobuf_empty_messages.sh | 16 ++++++++-------- .../format_schemas/02327_schema.capnp | 2 +- .../format_schemas/02327_schema.proto | 2 +- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh index a569a6435f6..3d41c9bf721 100755 --- a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh +++ b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh @@ -15,22 +15,22 @@ mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR cp -r $CLIENT_SCHEMADIR/02327_* $SCHEMADIR/$SERVER_SCHEMADIR/ -$CLICKHOUSE_CLIENT --query="desc file(data.pb) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="desc file(data.capnp) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="desc file(data.pb) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="desc file(data.capnp) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="create table test_protobuf engine=File(Protobuf) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="create table test_capnp engine=File(CapnProto) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="create table test_protobuf engine=File(Protobuf) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="create table test_capnp engine=File(CapnProto) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="desc file(data.pb) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message', input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference=1"; -$CLICKHOUSE_CLIENT --query="desc file(data.capnp) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message', input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference=1"; +$CLICKHOUSE_CLIENT --query="desc file(data.pb) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty', input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference=1"; +$CLICKHOUSE_CLIENT --query="desc file(data.capnp) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty', input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference=1"; $CLICKHOUSE_CLIENT --query="drop table if exists test_protobuf"; -$CLICKHOUSE_CLIENT --query="create table test_protobuf engine=File(Protobuf) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message', input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference=1"; +$CLICKHOUSE_CLIENT --query="create table test_protobuf engine=File(Protobuf) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty', input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference=1"; $CLICKHOUSE_CLIENT --query="desc test_protobuf"; $CLICKHOUSE_CLIENT --query="drop table test_protobuf"; $CLICKHOUSE_CLIENT --query="drop table if exists test_capnp"; -$CLICKHOUSE_CLIENT --query="create table test_capnp engine=File(CapnProto) settings format_schema='$SERVER_SCHEMADIR/02327_schema:Message', input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference=1"; +$CLICKHOUSE_CLIENT --query="create table test_capnp engine=File(CapnProto) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty', input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference=1"; $CLICKHOUSE_CLIENT --query="desc test_capnp"; $CLICKHOUSE_CLIENT --query="drop table test_capnp"; diff --git a/tests/queries/0_stateless/format_schemas/02327_schema.capnp b/tests/queries/0_stateless/format_schemas/02327_schema.capnp index 12ccc7308c9..0d4caa99fc6 100644 --- a/tests/queries/0_stateless/format_schemas/02327_schema.capnp +++ b/tests/queries/0_stateless/format_schemas/02327_schema.capnp @@ -4,7 +4,7 @@ struct Empty { } -struct Message +struct MessageWithEmpty { tuple1 @0 : Empty; text @1 : Text; diff --git a/tests/queries/0_stateless/format_schemas/02327_schema.proto b/tests/queries/0_stateless/format_schemas/02327_schema.proto index b5067393558..d478ccf6550 100644 --- a/tests/queries/0_stateless/format_schemas/02327_schema.proto +++ b/tests/queries/0_stateless/format_schemas/02327_schema.proto @@ -3,7 +3,7 @@ syntax = "proto3"; message Empty { } -message Message { +message MessageWithEmpty { Empty empty = 1; string str = 2; }; From e96c9a1fae7a46f918710ca78e237e6c282d80e1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 19:54:45 +0200 Subject: [PATCH 528/659] Run changelog.py from `github-helpers` branch --- docs/changelogs/v22.7.1.2484-stable.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/changelogs/v22.7.1.2484-stable.md b/docs/changelogs/v22.7.1.2484-stable.md index d306b1b5bd4..0343568658b 100644 --- a/docs/changelogs/v22.7.1.2484-stable.md +++ b/docs/changelogs/v22.7.1.2484-stable.md @@ -5,7 +5,7 @@ sidebar_label: 2022 # 2022 Changelog -### ClickHouse release v22.7.1.2484-stable FIXME as compared to v22.6.1.1985-stable +### ClickHouse release v22.7.1.2484-stable (f4f05ec786a) FIXME as compared to v22.6.1.1985-stable (7000c4e0033) #### Backward Incompatible Change * Enable setting `enable_positional_arguments` by default. It allows queries like `SELECT ... ORDER BY 1, 2` where 1, 2 are the references to the select clause. If you need to return the old behavior, disable this setting. [#38204](https://github.com/ClickHouse/ClickHouse/pull/38204) ([Alexey Milovidov](https://github.com/alexey-milovidov)). From 7795b2cec3fd099d5b780ceb66a90c23a4085b54 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 21 Jul 2022 20:29:49 +0200 Subject: [PATCH 529/659] Fix system.backups: now it can show duplicate UUIDs with different flag. --- src/Backups/BackupsWorker.cpp | 42 ++++++++++----------- src/Backups/BackupsWorker.h | 16 ++++---- src/Interpreters/InterpreterBackupQuery.cpp | 4 +- 3 files changed, 31 insertions(+), 31 deletions(-) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 47e1bac3200..df501258db6 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -110,7 +110,7 @@ BackupsWorker::BackupsWorker(size_t num_backup_threads, size_t num_restore_threa } -UUID BackupsWorker::start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context) +std::pair BackupsWorker::start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context) { const ASTBackupQuery & backup_query = typeid_cast(*backup_or_restore_query); if (backup_query.kind == ASTBackupQuery::Kind::BACKUP) @@ -120,7 +120,7 @@ UUID BackupsWorker::start(const ASTPtr & backup_or_restore_query, ContextMutable } -UUID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & context) +std::pair BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & context) { auto backup_query = std::static_pointer_cast(query->clone()); auto backup_settings = BackupSettings::fromBackupQuery(*backup_query); @@ -136,7 +136,7 @@ UUID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & c try { auto backup_info = BackupInfo::fromAST(*backup_query->backup_name); - addInfo(backup_uuid, backup_info.toString(), BackupStatus::MAKING_BACKUP, backup_settings.internal); + addInfo(backup_uuid, backup_settings.internal, backup_info.toString(), BackupStatus::MAKING_BACKUP); /// Prepare context to use. ContextPtr context_in_use = context; @@ -169,12 +169,12 @@ UUID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & c doBackup(backup_uuid, backup_query, backup_settings, backup_info, backup_coordination, context_in_use, mutable_context, false); } - return backup_uuid; + return {backup_uuid, backup_settings.internal}; } catch (...) { /// Something bad happened, the backup has not built. - setStatus(backup_uuid, BackupStatus::FAILED_TO_BACKUP); + setStatus(backup_uuid, backup_settings.internal, BackupStatus::FAILED_TO_BACKUP); sendCurrentExceptionToCoordination(backup_coordination, backup_settings.host_id); throw; } @@ -286,7 +286,7 @@ void BackupsWorker::doBackup( backup.reset(); LOG_INFO(log, "{} {} was created successfully", (backup_settings.internal ? "Internal backup" : "Backup"), backup_info.toString()); - setStatus(backup_uuid, BackupStatus::BACKUP_COMPLETE); + setStatus(backup_uuid, backup_settings.internal, BackupStatus::BACKUP_COMPLETE); } catch (...) { @@ -294,7 +294,7 @@ void BackupsWorker::doBackup( if (called_async) { tryLogCurrentException(log, fmt::format("Failed to make {} {}", (backup_settings.internal ? "internal backup" : "backup"), backup_info.toString())); - setStatus(backup_uuid, BackupStatus::FAILED_TO_BACKUP); + setStatus(backup_uuid, backup_settings.internal, BackupStatus::FAILED_TO_BACKUP); sendCurrentExceptionToCoordination(backup_coordination, backup_settings.host_id); } else @@ -306,7 +306,7 @@ void BackupsWorker::doBackup( } -UUID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr context) +std::pair BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr context) { auto restore_query = std::static_pointer_cast(query->clone()); auto restore_settings = RestoreSettings::fromRestoreQuery(*restore_query); @@ -319,7 +319,7 @@ UUID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr conte try { auto backup_info = BackupInfo::fromAST(*restore_query->backup_name); - addInfo(restore_uuid, backup_info.toString(), BackupStatus::RESTORING, restore_settings.internal); + addInfo(restore_uuid, restore_settings.internal, backup_info.toString(), BackupStatus::RESTORING); /// Prepare context to use. ContextMutablePtr context_in_use = context; @@ -342,12 +342,12 @@ UUID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr conte doRestore(restore_uuid, restore_query, restore_settings, backup_info, restore_coordination, context_in_use, false); } - return restore_uuid; + return {restore_uuid, restore_settings.internal}; } catch (...) { /// Something bad happened, the backup has not built. - setStatus(restore_uuid, BackupStatus::FAILED_TO_RESTORE); + setStatus(restore_uuid, restore_settings.internal, BackupStatus::FAILED_TO_RESTORE); sendCurrentExceptionToCoordination(restore_coordination, restore_settings.host_id); throw; } @@ -460,7 +460,7 @@ void BackupsWorker::doRestore( } LOG_INFO(log, "Restored from {} {} successfully", (restore_settings.internal ? "internal backup" : "backup"), backup_info.toString()); - setStatus(restore_uuid, BackupStatus::RESTORED); + setStatus(restore_uuid, restore_settings.internal, BackupStatus::RESTORED); } catch (...) { @@ -468,7 +468,7 @@ void BackupsWorker::doRestore( if (called_async) { tryLogCurrentException(log, fmt::format("Failed to restore from {} {}", (restore_settings.internal ? "internal backup" : "backup"), backup_info.toString())); - setStatus(restore_uuid, BackupStatus::FAILED_TO_RESTORE); + setStatus(restore_uuid, restore_settings.internal, BackupStatus::FAILED_TO_RESTORE); sendCurrentExceptionToCoordination(restore_coordination, restore_settings.host_id); } else @@ -480,7 +480,7 @@ void BackupsWorker::doRestore( } -void BackupsWorker::addInfo(const UUID & uuid, const String & backup_name, BackupStatus status, bool internal) +void BackupsWorker::addInfo(const UUID & uuid, bool internal, const String & backup_name, BackupStatus status) { Info info; info.uuid = uuid; @@ -490,17 +490,17 @@ void BackupsWorker::addInfo(const UUID & uuid, const String & backup_name, Backu info.internal = internal; std::lock_guard lock{infos_mutex}; - infos[uuid] = std::move(info); + infos[{uuid, internal}] = std::move(info); num_active_backups += getNumActiveBackupsChange(status); num_active_restores += getNumActiveRestoresChange(status); } -void BackupsWorker::setStatus(const UUID & uuid, BackupStatus status) +void BackupsWorker::setStatus(const UUID & uuid, bool internal, BackupStatus status) { std::lock_guard lock{infos_mutex}; - auto it = infos.find(uuid); + auto it = infos.find({uuid, internal}); if (it == infos.end()) return; @@ -513,12 +513,12 @@ void BackupsWorker::setStatus(const UUID & uuid, BackupStatus status) } -void BackupsWorker::wait(const UUID & backup_or_restore_uuid, bool rethrow_exception) +void BackupsWorker::wait(const UUID & backup_or_restore_uuid, bool internal, bool rethrow_exception) { std::unique_lock lock{infos_mutex}; status_changed.wait(lock, [&] { - auto it = infos.find(backup_or_restore_uuid); + auto it = infos.find({backup_or_restore_uuid, internal}); if (it == infos.end()) return true; const auto & info = it->second; @@ -529,10 +529,10 @@ void BackupsWorker::wait(const UUID & backup_or_restore_uuid, bool rethrow_excep }); } -std::optional BackupsWorker::tryGetInfo(const UUID & backup_or_restore_uuid) const +std::optional BackupsWorker::tryGetInfo(const UUID & backup_or_restore_uuid, bool internal) const { std::lock_guard lock{infos_mutex}; - auto it = infos.find(backup_or_restore_uuid); + auto it = infos.find({backup_or_restore_uuid, internal}); if (it == infos.end()) return std::nullopt; return it->second; diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index 8db9c1367a9..f471dc822e0 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -29,11 +29,11 @@ public: void shutdown(); /// Starts executing a BACKUP or RESTORE query. Returns UUID of the operation. - UUID start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context); + std::pair start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context); /// Waits until a BACKUP or RESTORE query started by start() is finished. /// The function returns immediately if the operation is already finished. - void wait(const UUID & backup_or_restore_uuid, bool rethrow_exception = true); + void wait(const UUID & backup_or_restore_uuid, bool internal, bool rethrow_exception = true); /// Information about executing a BACKUP or RESTORE query started by calling start(). struct Info @@ -54,29 +54,29 @@ public: bool internal = false; }; - std::optional tryGetInfo(const UUID & backup_or_restore_uuid) const; + std::optional tryGetInfo(const UUID & backup_or_restore_uuid, bool internal) const; std::vector getAllInfos() const; private: - UUID startMakingBackup(const ASTPtr & query, const ContextPtr & context); + std::pair startMakingBackup(const ASTPtr & query, const ContextPtr & context); void doBackup(const UUID & backup_uuid, const std::shared_ptr & backup_query, BackupSettings backup_settings, const BackupInfo & backup_info, std::shared_ptr backup_coordination, const ContextPtr & context, ContextMutablePtr mutable_context, bool called_async); - UUID startRestoring(const ASTPtr & query, ContextMutablePtr context); + std::pair startRestoring(const ASTPtr & query, ContextMutablePtr context); void doRestore(const UUID & restore_uuid, const std::shared_ptr & restore_query, RestoreSettings restore_settings, const BackupInfo & backup_info, std::shared_ptr restore_coordination, ContextMutablePtr context, bool called_async); - void addInfo(const UUID & uuid, const String & backup_name, BackupStatus status, bool internal); - void setStatus(const UUID & uuid, BackupStatus status); + void addInfo(const UUID & uuid, bool internal, const String & backup_name, BackupStatus status); + void setStatus(const UUID & uuid, bool internal, BackupStatus status); ThreadPool backups_thread_pool; ThreadPool restores_thread_pool; - std::unordered_map infos; + std::map, Info> infos; std::condition_variable status_changed; std::atomic num_active_backups = 0; std::atomic num_active_restores = 0; diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index af3c8df8eef..a2bb3acf073 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -42,9 +42,9 @@ namespace BlockIO InterpreterBackupQuery::execute() { auto & backups_worker = context->getBackupsWorker(); - UUID uuid = backups_worker.start(query_ptr, context); + auto [uuid, internal] = backups_worker.start(query_ptr, context); BlockIO res_io; - res_io.pipeline = QueryPipeline(std::make_shared(getResultRow(backups_worker.tryGetInfo(uuid)))); + res_io.pipeline = QueryPipeline(std::make_shared(getResultRow(backups_worker.tryGetInfo(uuid, internal)))); return res_io; } From 36d3923622c79dab910ede3420d84d8dbcebaf51 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 21 Jul 2022 20:31:03 +0200 Subject: [PATCH 530/659] Add one more concurrency test. --- .../test_concurrency.py | 89 +++++++++++++++++++ 1 file changed, 89 insertions(+) diff --git a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py index 95d4f27e5e8..d399723c8b1 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py @@ -1,5 +1,8 @@ +from random import randint import pytest import os.path +import time +import concurrent from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV, assert_eq_with_retry @@ -27,6 +30,7 @@ def generate_cluster_def(): main_configs = ["configs/backups_disk.xml", generate_cluster_def()] +user_configs = ["configs/allow_experimental_database_replicated.xml"] nodes = [] for i in range(num_nodes): @@ -34,6 +38,7 @@ for i in range(num_nodes): cluster.add_instance( f"node{i}", main_configs=main_configs, + user_configs=user_configs, external_dirs=["/backups/"], macros={"replica": f"node{i}", "shard": "shard1"}, with_zookeeper=True, @@ -160,3 +165,87 @@ def test_concurrent_backups_on_different_nodes(): nodes[i].query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl") for j in range(num_nodes): assert nodes[j].query("SELECT sum(x) FROM tbl") == TSV([expected_sum]) + + +def test_create_or_drop_tables_during_backup(): + node0.query( + "CREATE DATABASE mydb ON CLUSTER 'cluster' ENGINE=Replicated('/clickhouse/path/','{shard}','{replica}')" + ) + + # Will do this test for 60 seconds + start_time = time.time() + end_time = start_time + 60 + + def create_table(): + while time.time() < end_time: + node = nodes[randint(0, num_nodes - 1)] + table_name = f"mydb.tbl{randint(1, num_nodes)}" + node.query( + f"CREATE TABLE IF NOT EXISTS {table_name}(x Int32) ENGINE=ReplicatedMergeTree ORDER BY x" + ) + node.query_and_get_answer_with_error( + f"INSERT INTO {table_name} SELECT rand32() FROM numbers(10)" + ) + + def drop_table(): + while time.time() < end_time: + table_name = f"mydb.tbl{randint(1, num_nodes)}" + node = nodes[randint(0, num_nodes - 1)] + node.query(f"DROP TABLE IF EXISTS {table_name} NO DELAY") + + def rename_table(): + while time.time() < end_time: + table_name1 = f"mydb.tbl{randint(1, num_nodes)}" + table_name2 = f"mydb.tbl{randint(1, num_nodes)}" + node = nodes[randint(0, num_nodes - 1)] + node.query_and_get_answer_with_error( + f"RENAME TABLE {table_name1} TO {table_name2}" + ) + + def make_backup(): + ids = [] + while time.time() < end_time: + time.sleep( + 5 + ) # 1 minute total, and around 5 seconds per each backup => around 12 backups should be created + backup_name = new_backup_name() + id = node0.query( + f"BACKUP DATABASE mydb ON CLUSTER 'cluster' TO {backup_name} ASYNC" + ).split("\t")[0] + ids.append(id) + return ids + + ids = [] + with concurrent.futures.ThreadPoolExecutor(max_workers=5) as executor: + futures = [] + ids_future = executor.submit(make_backup) + futures.append(ids_future) + futures.append(executor.submit(create_table)) + futures.append(executor.submit(drop_table)) + futures.append(executor.submit(rename_table)) + for future in futures: + future.result() + ids = ids_future.result() + + ids_list = "[" + ", ".join([f"'{id}'" for id in ids]) + "]" + for node in nodes: + assert_eq_with_retry( + node, + f"SELECT status, error from system.backups WHERE uuid IN {ids_list} AND (status == 'MAKING_BACKUP')", + "", + ) + + backup_names = {} + for node in nodes: + for id in ids: + backup_name = node.query( + f"SELECT backup_name FROM system.backups WHERE uuid='{id}' FORMAT RawBLOB" + ).strip() + if backup_name: + backup_names[id] = backup_name + + for id in ids: + node0.query("DROP DATABASE mydb ON CLUSTER 'cluster'") + node0.query( + f"RESTORE DATABASE mydb ON CLUSTER 'cluster' FROM {backup_names[id]}" + ) From d5727dfb4446e764ada829f72247e81c7e85083c Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Thu, 21 Jul 2022 13:09:45 -0700 Subject: [PATCH 531/659] Removed __BYTE_ORDER__ and fixed style issue --- src/Compression/LZ4_decompress_faster.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 54b1e4ff58e..82a86a80d8d 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -24,11 +24,11 @@ #include #endif -#if defined(__BYTE_ORDER__) && (__BYTE_ORDER__ == __ORDER_BIG_ENDIAN__) -static inline UInt16 LZ4_readLE16(const void* memPtr) +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ +static inline UInt16 LZ4_readLE16(const void* mem_ptr) { - const UInt8* p = reinterpret_cast(memPtr); - return static_cast(p[0]) + (p[1]<<8); + const UInt8* p = reinterpret_cast(mem_ptr); + return static_cast(p[0]) + (p[1] << 8); } #endif @@ -569,7 +569,7 @@ bool NO_INLINE decompressImpl( /// Get match offset. -#if defined(__BYTE_ORDER__) && (__BYTE_ORDER__ == __ORDER_BIG_ENDIAN__) +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ size_t offset = LZ4_readLE16(ip); #else size_t offset = unalignedLoad(ip); From 7db5d54820154a0287b4f1b20a699ea34862c99f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 21 Jul 2022 20:58:33 +0000 Subject: [PATCH 532/659] Adopt to the case when not all columns in distinct are part of sorting description --- src/Processors/QueryPlan/DistinctStep.cpp | 38 +++++++++++++++++-- tests/performance/distinct_in_order.xml | 3 +- ...7_distinct_in_order_optimization.reference | 1 + .../02317_distinct_in_order_optimization.sql | 15 ++++++++ ...ct_in_order_optimization_explain.reference | 18 +++++++-- ..._distinct_in_order_optimization_explain.sh | 23 ++++++----- ...tinct_in_order_optimization_long.reference | 1 - ...17_distinct_in_order_optimization_long.sql | 14 ------- 8 files changed, 78 insertions(+), 35 deletions(-) delete mode 100644 tests/queries/0_stateless/02317_distinct_in_order_optimization_long.reference delete mode 100644 tests/queries/0_stateless/02317_distinct_in_order_optimization_long.sql diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index 103f0f064a0..c268cb44267 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -119,9 +119,8 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil if (!distinct_sort_desc.empty()) { - const bool sorted_stream = input_stream.sort_mode == DataStream::SortMode::Stream; - /// pre-distinct for sorted chunks or final distinct for sorted stream (sorting inside and among chunks) - if (pre_distinct || sorted_stream) + /// pre-distinct for sorted chunks + if (pre_distinct) { pipeline.addSimpleTransform( [&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr @@ -130,10 +129,41 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil return nullptr; return std::make_shared( - header, set_size_limits, limit_hint, distinct_sort_desc, columns, sorted_stream); + header, set_size_limits, limit_hint, distinct_sort_desc, columns, false); }); return; } + /// final distinct for sorted stream (sorting inside and among chunks) + if (input_stream.sort_mode == DataStream::SortMode::Stream) + { + assert(input_stream.has_single_port); + + if (distinct_sort_desc.size() < columns.size()) + { + pipeline.addSimpleTransform( + [&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipelineBuilder::StreamType::Main) + return nullptr; + + return std::make_shared( + header, distinct_sort_desc, set_size_limits, limit_hint, columns); + }); + } + else + { + pipeline.addSimpleTransform( + [&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipelineBuilder::StreamType::Main) + return nullptr; + + return std::make_shared( + header, set_size_limits, limit_hint, distinct_sort_desc, columns, true); + }); + } + return; + } } } diff --git a/tests/performance/distinct_in_order.xml b/tests/performance/distinct_in_order.xml index b2c117785f9..834a6945622 100644 --- a/tests/performance/distinct_in_order.xml +++ b/tests/performance/distinct_in_order.xml @@ -27,8 +27,7 @@ SELECT DISTINCT low, medium FROM distinct_cardinality_low ORDER BY medium FORMAT Null SELECT DISTINCT low, medium FROM distinct_cardinality_low ORDER BY low FORMAT Null SELECT DISTINCT low, high FROM distinct_cardinality_low ORDER BY high FORMAT Null - - + SELECT DISTINCT low, medium, high FROM distinct_cardinality_low ORDER BY high FORMAT Null DROP TABLE IF EXISTS distinct_cardinality_low diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference index b53b561137e..a5ae3759d5d 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference @@ -77,3 +77,4 @@ 2 2 1 1 0 0 +-- check that distinct in order has the same result as ordinary distinct diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql index afe53a95b26..2da81b50f6c 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql @@ -44,3 +44,18 @@ select '-- distinct with non-key prefix and non-sorted column, order by non-sort select distinct b,c from distinct_in_order order by c desc; drop table if exists distinct_in_order sync; + +select '-- check that distinct in order has the same result as ordinary distinct'; +drop table if exists distinct_cardinality_low sync; +drop table if exists distinct_in_order sync; +drop table if exists ordinary_distinct sync; +CREATE TABLE distinct_cardinality_low (low UInt64, medium UInt64, high UInt64) ENGINE MergeTree() ORDER BY (low, medium); +INSERT INTO distinct_cardinality_low SELECT number % 1e2, number % 1e4, number % 1e6 FROM numbers_mt(1e8); +create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +insert into distinct_in_order select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=1; +create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +insert into ordinary_distinct select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=0; +select distinct * from distinct_in_order except select * from ordinary_distinct; +drop table if exists distinct_in_order; +drop table if exists ordinary_distinct; +drop table if exists distinct_cardinality_low; diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index f30d3fa30ea..327d0d52dc2 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -1,19 +1,29 @@ -- disable optimize_distinct_in_order -- distinct all primary key columns -> no optimizations -No optimizations +DistinctTransform +DistinctTransform -- enable optimize_distinct_in_order -- distinct with all primary key columns -> pre-distinct optimization only +DistinctTransform DistinctSortedChunkTransform -- distinct with primary key prefix -> pre-distinct optimization only +DistinctTransform DistinctSortedChunkTransform -- distinct with primary key prefix and order by on column in distinct -> pre-distinct and final distinct optimization +DistinctSortedTransform +DistinctSortedChunkTransform +-- distinct with primary key prefix and order by on the same columns -> pre-distinct and final distinct optimization DistinctSortedStreamTransform DistinctSortedChunkTransform -- distinct with primary key prefix and order by on column _not_ in distinct -> pre-distinct optimization only +DistinctTransform DistinctSortedChunkTransform -- distinct with non-primary key prefix -> no optimizations -No optimizations +DistinctTransform +DistinctTransform -- distinct with non-primary key prefix and order by on column in distinct -> final distinct optimization only -DistinctSortedStreamTransform +DistinctSortedTransform +DistinctTransform -- distinct with non-primary key prefix and order by on column _not_ in distinct -> no optimizations -No optimizations +DistinctTransform +DistinctTransform diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh index 9af0e98ecf4..e822f9695b9 100755 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh @@ -8,9 +8,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DISABLE_OPTIMIZATION="set optimize_distinct_in_order=0" ENABLE_OPTIMIZATION="set optimize_distinct_in_order=1" -GREP_OPTIMIZATIONS="grep 'DistinctSortedChunkTransform\|DistinctSortedStreamTransform'" +GREP_DISTINCT="grep 'DistinctSortedChunkTransform\|DistinctSortedStreamTransform\|DistinctSortedTransform\|DistinctTransform'" TRIM_LEADING_SPACES="sed -e 's/^[ \t]*//'" -FIND_OPTIMIZATIONS="$GREP_OPTIMIZATIONS | $TRIM_LEADING_SPACES" +FIND_DISTINCT="$GREP_DISTINCT | $TRIM_LEADING_SPACES" $CLICKHOUSE_CLIENT -q "drop table if exists distinct_in_order_explain sync" $CLICKHOUSE_CLIENT -q "create table distinct_in_order_explain (a int, b int, c int) engine=MergeTree() order by (a, b, c)" @@ -18,26 +18,29 @@ $CLICKHOUSE_CLIENT -q "insert into distinct_in_order_explain select number % num $CLICKHOUSE_CLIENT -q "select '-- disable optimize_distinct_in_order'" $CLICKHOUSE_CLIENT -q "select '-- distinct all primary key columns -> no optimizations'" -$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $GREP_OPTIMIZATIONS || echo "No optimizations" +$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- enable optimize_distinct_in_order'" $CLICKHOUSE_CLIENT -q "select '-- distinct with all primary key columns -> pre-distinct optimization only'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_OPTIMIZATIONS +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix -> pre-distinct optimization only'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_OPTIMIZATIONS +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by on column in distinct -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by c" | eval $FIND_OPTIMIZATIONS +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT + +$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by on the same columns -> pre-distinct and final distinct optimization'" +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a, b" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by on column _not_ in distinct -> pre-distinct optimization only'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by b" | eval $FIND_OPTIMIZATIONS +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix -> no optimizations'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain" | eval $GREP_OPTIMIZATIONS || echo "No optimizations" +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by on column in distinct -> final distinct optimization only'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by b" | eval $FIND_OPTIMIZATIONS +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by on column _not_ in distinct -> no optimizations'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by a" | eval $GREP_OPTIMIZATIONS || echo "No optimizations" +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by a" | eval $FIND_DISTINCT diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_long.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_long.reference deleted file mode 100644 index 0d4d005b74c..00000000000 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_long.reference +++ /dev/null @@ -1 +0,0 @@ --- check that slow query with distinct in order has the same result as ordinary distinct diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_long.sql b/tests/queries/0_stateless/02317_distinct_in_order_optimization_long.sql deleted file mode 100644 index 7034538a256..00000000000 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_long.sql +++ /dev/null @@ -1,14 +0,0 @@ -select '-- check that slow query with distinct in order has the same result as ordinary distinct'; -drop table if exists distinct_cardinality_low sync; -drop table if exists distinct_in_order sync; -drop table if exists ordinary_distinct sync; -CREATE TABLE distinct_cardinality_low (low UInt64, medium UInt64, high UInt64) ENGINE MergeTree() ORDER BY (low, medium); -INSERT INTO distinct_cardinality_low SELECT number % 1e2, number % 1e4, number % 1e6 FROM numbers_mt(1e8); -create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); -insert into distinct_in_order select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=1; -create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); -insert into ordinary_distinct select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=0; -select distinct * from distinct_in_order except select * from ordinary_distinct; -drop table if exists distinct_in_order; -drop table if exists ordinary_distinct; -drop table if exists distinct_cardinality_low; From f663c1e5e2db50a12a0ec17f665de910ebef434e Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 21 Jul 2022 17:14:20 -0400 Subject: [PATCH 533/659] fixed condition fault --- src/Compression/CompressedReadBufferBase.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 38f580e2f11..3de7b9cc688 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -304,13 +304,13 @@ void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_d void CompressedReadBufferBase::flushAsynchronousDecompressRequests() const { - if (!codec) + if (codec) codec->flushAsynchronousDecompressRequests(); } void CompressedReadBufferBase::setDecompressMode(ICompressionCodec::CodecMode mode) { - if (!codec) + if (codec) codec->setDecompressMode(mode); } From 6d289cf9a5ac7aefab855458bd51564c0d9e31b6 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 21 Jul 2022 18:18:14 -0400 Subject: [PATCH 534/659] Update README.md --- tests/integration/README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index 2d44ff70861..b0e1a4fdbb1 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -44,7 +44,8 @@ sudo -H pip install \ dict2xml \ hypothesis \ pyhdfs \ - pika + pika \ + meilisearch ``` (highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python3-pytest python3-dicttoxml python3-docker python3-pymysql python3-protobuf python3-pymongo python3-tzlocal python3-kazoo python3-psycopg2 kafka-python python3-pytest-timeout python3-minio` From 948f207e46833fda209b8832678bb643b25268d8 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 21 Jul 2022 22:47:40 -0400 Subject: [PATCH 535/659] nats-py is missing also --- tests/integration/README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index b0e1a4fdbb1..18d46908524 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -45,7 +45,8 @@ sudo -H pip install \ hypothesis \ pyhdfs \ pika \ - meilisearch + meilisearch \ + nats-py ``` (highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python3-pytest python3-dicttoxml python3-docker python3-pymysql python3-protobuf python3-pymongo python3-tzlocal python3-kazoo python3-psycopg2 kafka-python python3-pytest-timeout python3-minio` From c77117eadf900ac536b4c0e1c44811b25477872c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 22 Jul 2022 12:54:20 +0300 Subject: [PATCH 536/659] Update SSDCacheDictionaryStorage.h --- src/Dictionaries/SSDCacheDictionaryStorage.h | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Dictionaries/SSDCacheDictionaryStorage.h b/src/Dictionaries/SSDCacheDictionaryStorage.h index 22a46016b2c..459c4c44668 100644 --- a/src/Dictionaries/SSDCacheDictionaryStorage.h +++ b/src/Dictionaries/SSDCacheDictionaryStorage.h @@ -27,11 +27,6 @@ #include -namespace CurrentMetrics -{ - extern const Metric Write; -} - namespace ProfileEvents { extern const Event FileOpen; From 17b504b0d48ed5d76eb8bd958b0ad022d9a61b83 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 12:19:46 +0200 Subject: [PATCH 537/659] Build thread sanitizer binaries with clang-13 --- docker/packager/binary/Dockerfile | 15 +++++++++++++++ docker/packager/packager | 1 + tests/ci/ci_config.py | 2 +- 3 files changed, 17 insertions(+), 1 deletion(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 1dff4b1a2d4..6602e294327 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -118,5 +118,20 @@ ENV GOCACHE=/workdir/ RUN mkdir /workdir && chmod 777 /workdir WORKDIR /workdir +# FIXME: thread sanitizer is broken in clang-14, we have to build it with clang-13 +# https://github.com/ClickHouse/ClickHouse/pull/39450 +# https://github.com/google/sanitizers/issues/1540 +# https://github.com/google/sanitizers/issues/1552 + +RUN export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ + && echo "deb [trusted=yes] https://apt.llvm.org/${CODENAME}/ llvm-toolchain-${CODENAME}-13 main" >> \ + /etc/apt/sources.list.d/clang.list \ + && apt-get update \ + && apt-get install \ + clang-13 \ + clang-tidy-13 \ + --yes --no-install-recommends \ + && apt-get clean + COPY build.sh / CMD ["bash", "-c", "/build.sh 2>&1"] diff --git a/docker/packager/packager b/docker/packager/packager index 7c0f046b76c..0b00bc4e9c0 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -323,6 +323,7 @@ if __name__ == "__main__": parser.add_argument( "--compiler", choices=( + "clang-13", # For TSAN builds, see #39450 "clang-14", "clang-14-darwin", "clang-14-darwin-aarch64", diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index a530b395130..2b22ccd9c7c 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -63,7 +63,7 @@ CI_CONFIG = { "with_coverage": False, }, "package_tsan": { - "compiler": "clang-14", + "compiler": "clang-13", "build_type": "", "sanitizer": "thread", "package_type": "deb", From c6b251d0cb54693a9ca58b53312b0184cd31dacf Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 22 Jul 2022 11:58:15 +0200 Subject: [PATCH 538/659] Revert "Merge pull request #38106 from ClickHouse/tsan_less_flaky_tests" This reverts commit 0ae0cc54aa6b1fe479d68310a06d74b392d7f051, reversing changes made to 1d7cf28cabdc7ad62d0079582e763d630f3c27e1. --- docker/test/stress/run.sh | 37 +++++++++---------- .../00984_parser_stack_overflow.sh | 2 - .../01172_transaction_counters.sql | 3 +- .../01183_custom_separated_format_http.sh | 2 - .../01184_long_insert_values_huge_strings.sh | 3 +- .../0_stateless/01651_lc_insert_tiny_log.sql | 3 -- ..._long_zstd_http_compression_json_format.sh | 3 +- .../0_stateless/01926_order_by_desc_limit.sql | 3 +- .../00159_parallel_formatting_http.sh | 2 - 9 files changed, 21 insertions(+), 37 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index ffa0b12b8a3..33b60ae7ef9 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -7,29 +7,26 @@ set -x # Thread Fuzzer allows to check more permutations of possible thread scheduling # and find more potential issues. -# -# But under thread fuzzer, TSan build is too slow and this produces some flaky -# tests, so for now, as a temporary solution it had been disabled. -if ! test -f package_folder/clickhouse-server*tsan*.deb; then - export THREAD_FUZZER_CPU_TIME_PERIOD_US=1000 - export THREAD_FUZZER_SLEEP_PROBABILITY=0.1 - export THREAD_FUZZER_SLEEP_TIME_US=100000 - export THREAD_FUZZER_pthread_mutex_lock_BEFORE_MIGRATE_PROBABILITY=1 - export THREAD_FUZZER_pthread_mutex_lock_AFTER_MIGRATE_PROBABILITY=1 - export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_MIGRATE_PROBABILITY=1 - export THREAD_FUZZER_pthread_mutex_unlock_AFTER_MIGRATE_PROBABILITY=1 +export THREAD_FUZZER_CPU_TIME_PERIOD_US=1000 +export THREAD_FUZZER_SLEEP_PROBABILITY=0.1 +export THREAD_FUZZER_SLEEP_TIME_US=100000 - export THREAD_FUZZER_pthread_mutex_lock_BEFORE_SLEEP_PROBABILITY=0.001 - export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_PROBABILITY=0.001 - export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_PROBABILITY=0.001 - export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_PROBABILITY=0.001 - export THREAD_FUZZER_pthread_mutex_lock_BEFORE_SLEEP_TIME_US=10000 +export THREAD_FUZZER_pthread_mutex_lock_BEFORE_MIGRATE_PROBABILITY=1 +export THREAD_FUZZER_pthread_mutex_lock_AFTER_MIGRATE_PROBABILITY=1 +export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_MIGRATE_PROBABILITY=1 +export THREAD_FUZZER_pthread_mutex_unlock_AFTER_MIGRATE_PROBABILITY=1 + +export THREAD_FUZZER_pthread_mutex_lock_BEFORE_SLEEP_PROBABILITY=0.001 +export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_PROBABILITY=0.001 +export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_PROBABILITY=0.001 +export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_PROBABILITY=0.001 +export THREAD_FUZZER_pthread_mutex_lock_BEFORE_SLEEP_TIME_US=10000 + +export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_TIME_US=10000 +export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_TIME_US=10000 +export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_TIME_US=10000 - export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_TIME_US=10000 - export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_TIME_US=10000 - export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_TIME_US=10000 -fi function install_packages() { diff --git a/tests/queries/0_stateless/00984_parser_stack_overflow.sh b/tests/queries/0_stateless/00984_parser_stack_overflow.sh index 168ef155d9b..329e51e774a 100755 --- a/tests/queries/0_stateless/00984_parser_stack_overflow.sh +++ b/tests/queries/0_stateless/00984_parser_stack_overflow.sh @@ -1,6 +1,4 @@ #!/usr/bin/env bash -# Tags: no-tsan -# FIXME It became flaky after upgrading to llvm-14 due to obscure freezes in tsan # Such a huge timeout mostly for debug build. CLICKHOUSE_CURL_TIMEOUT=60 diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index ed40ec3f4b6..cc6212ae4c1 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -1,6 +1,5 @@ --- Tags: no-s3-storage, no-tsan, no-ordinary-database +-- Tags: no-s3-storage, no-ordinary-database -- FIXME this test fails with S3 due to a bug in DiskCacheWrapper --- FIXME It became flaky after upgrading to llvm-14 due to obscure freezes in tsan drop table if exists txn_counters; create table txn_counters (n Int64, creation_tid DEFAULT transactionID()) engine=MergeTree order by n; diff --git a/tests/queries/0_stateless/01183_custom_separated_format_http.sh b/tests/queries/0_stateless/01183_custom_separated_format_http.sh index 744cf0c08bd..8eaa22f4ecc 100755 --- a/tests/queries/0_stateless/01183_custom_separated_format_http.sh +++ b/tests/queries/0_stateless/01183_custom_separated_format_http.sh @@ -1,6 +1,4 @@ #!/usr/bin/env bash -# Tags: no-tsan -# FIXME It became flaky after upgrading to llvm-14 due to obscure freezes in tsan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01184_long_insert_values_huge_strings.sh b/tests/queries/0_stateless/01184_long_insert_values_huge_strings.sh index f4bad961f21..09a43d13a42 100755 --- a/tests/queries/0_stateless/01184_long_insert_values_huge_strings.sh +++ b/tests/queries/0_stateless/01184_long_insert_values_huge_strings.sh @@ -1,6 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-tsan -# FIXME It became flaky after upgrading to llvm-14 due to obscure freezes in tsan +# Tags: long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql b/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql index ec2a1850594..22532529812 100644 --- a/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql +++ b/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql @@ -1,6 +1,3 @@ --- Tags: no-tsan --- FIXME It became flaky after upgrading to llvm-14 due to obscure freezes in tsan - drop table if exists perf_lc_num; CREATE TABLE perf_lc_num(  num UInt8,  arr Array(LowCardinality(Int64)) default [num]  ) ENGINE = TinyLog; diff --git a/tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.sh b/tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.sh index 16f5211f012..e10032e04fd 100755 --- a/tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.sh +++ b/tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.sh @@ -1,6 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-tsan -# FIXME It became flaky after upgrading to llvm-14 due to obscure freezes in tsan +# Tags: long, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01926_order_by_desc_limit.sql b/tests/queries/0_stateless/01926_order_by_desc_limit.sql index 223dbf70fc4..86468b4fcd6 100644 --- a/tests/queries/0_stateless/01926_order_by_desc_limit.sql +++ b/tests/queries/0_stateless/01926_order_by_desc_limit.sql @@ -1,5 +1,4 @@ --- Tags: no-random-settings, no-tsan --- FIXME It became flaky after upgrading to llvm-14 due to obscure freezes in tsan +-- Tags: no-random-settings DROP TABLE IF EXISTS order_by_desc; diff --git a/tests/queries/1_stateful/00159_parallel_formatting_http.sh b/tests/queries/1_stateful/00159_parallel_formatting_http.sh index 7b949cf23e6..ea4a4d12867 100755 --- a/tests/queries/1_stateful/00159_parallel_formatting_http.sh +++ b/tests/queries/1_stateful/00159_parallel_formatting_http.sh @@ -1,6 +1,4 @@ #!/usr/bin/env bash -# Tags: no-tsan -# FIXME It became flaky after upgrading to llvm-14 due to obscure freezes in tsan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 69984b28e0320f65b833dac3d0efda5601805edf Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 22 Jul 2022 12:21:10 +0200 Subject: [PATCH 539/659] Remove unused constructor --- src/Storages/MergeTree/MergeTreeIndexGranularity.cpp | 6 ------ src/Storages/MergeTree/MergeTreeIndexGranularity.h | 2 -- 2 files changed, 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp index 8dc353714cb..ac0146edc3a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp @@ -15,12 +15,6 @@ MergeTreeIndexGranularity::MergeTreeIndexGranularity(const std::vector & { } - -MergeTreeIndexGranularity::MergeTreeIndexGranularity(size_t marks_count, size_t fixed_granularity) - : marks_rows_partial_sums(marks_count, fixed_granularity) -{ -} - /// Rows after mark to next mark size_t MergeTreeIndexGranularity::getMarkRows(size_t mark_index) const { diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.h b/src/Storages/MergeTree/MergeTreeIndexGranularity.h index 0923a496174..0d1480d68cb 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.h @@ -20,8 +20,6 @@ private: public: MergeTreeIndexGranularity() = default; explicit MergeTreeIndexGranularity(const std::vector & marks_rows_partial_sums_); - MergeTreeIndexGranularity(size_t marks_count, size_t fixed_granularity); - /// Return count of rows between marks size_t getRowsCountInRange(const MarkRange & range) const; From 0d4d1e66b58b48f1f99460f9500c976161645f2a Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Fri, 22 Jul 2022 18:53:44 +0800 Subject: [PATCH 540/659] Fix build --- src/Common/waitForPid.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/waitForPid.cpp b/src/Common/waitForPid.cpp index 38f43ae2f6a..fc0df5d7211 100644 --- a/src/Common/waitForPid.cpp +++ b/src/Common/waitForPid.cpp @@ -92,6 +92,8 @@ static int pollPid(pid_t pid, int timeout_in_ms) } #elif defined(OS_DARWIN) || defined(OS_FREEBSD) +#pragma clang diagnostic ignored "-Wreserved-identifier" + #include #include From 51f63c3a0c1885604f25eb90db8ab96de4192646 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 22 Jul 2022 10:57:57 +0000 Subject: [PATCH 541/659] Add setting optimize_distinct_in_order_memory_usage --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 3 ++- .../InterpreterSelectWithUnionQuery.cpp | 3 ++- src/Processors/QueryPlan/DistinctStep.cpp | 14 ++++++++------ src/Processors/QueryPlan/DistinctStep.h | 4 +++- .../02317_distinct_in_order_optimization.sql | 2 +- ...istinct_in_order_optimization_explain.reference | 6 ++++++ ...02317_distinct_in_order_optimization_explain.sh | 6 ++++++ 8 files changed, 29 insertions(+), 10 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8a1f47ec00e..4f5645de7b3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -621,6 +621,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, compatibility_ignore_auto_increment_in_create_table, false, "Ignore AUTO_INCREMENT keyword in column declaration if true, otherwise return error. It simplifies migration from MySQL", 0) \ M(Bool, multiple_joins_try_to_keep_original_names, false, "Do not add aliases to top level expression list on multiple joins rewrite", 0) \ M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ + M(Bool, optimize_distinct_in_order_memory_usage, false, "Try to use less memory for DISTINCT in order but can be slower", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a05d353ac73..8590889d6f8 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2622,7 +2622,8 @@ void InterpreterSelectQuery::executeDistinct(QueryPlan & query_plan, bool before limit_for_distinct, columns, pre_distinct, - settings.optimize_distinct_in_order); + settings.optimize_distinct_in_order, + settings.optimize_distinct_in_order_memory_usage); if (pre_distinct) distinct_step->setStepDescription("Preliminary DISTINCT"); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 9f87a47fced..0590ce8f467 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -335,7 +335,8 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) 0, result_header.getNames(), false, - settings.optimize_distinct_in_order); + settings.optimize_distinct_in_order, + settings.optimize_distinct_in_order_memory_usage); query_plan.addStep(std::move(distinct_step)); } diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index c268cb44267..9223218d82b 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -80,7 +80,8 @@ DistinctStep::DistinctStep( UInt64 limit_hint_, const Names & columns_, bool pre_distinct_, - bool optimize_distinct_in_order_) + bool optimize_distinct_in_order_, + bool optimize_distinct_in_order_memory_usage_) : ITransformingStep( input_stream_, input_stream_.header, @@ -90,6 +91,7 @@ DistinctStep::DistinctStep( , columns(columns_) , pre_distinct(pre_distinct_) , optimize_distinct_in_order(optimize_distinct_in_order_) + , optimize_distinct_in_order_memory_usage(optimize_distinct_in_order_memory_usage_) { if (!output_stream->distinct_columns.empty() /// Columns already distinct, do nothing && (!pre_distinct /// Main distinct @@ -138,7 +140,7 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil { assert(input_stream.has_single_port); - if (distinct_sort_desc.size() < columns.size()) + if (distinct_sort_desc.size() >= columns.size() || optimize_distinct_in_order_memory_usage) { pipeline.addSimpleTransform( [&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr @@ -146,8 +148,8 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - return std::make_shared( - header, distinct_sort_desc, set_size_limits, limit_hint, columns); + return std::make_shared( + header, set_size_limits, limit_hint, distinct_sort_desc, columns, true); }); } else @@ -158,8 +160,8 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - return std::make_shared( - header, set_size_limits, limit_hint, distinct_sort_desc, columns, true); + return std::make_shared( + header, distinct_sort_desc, set_size_limits, limit_hint, columns); }); } return; diff --git a/src/Processors/QueryPlan/DistinctStep.h b/src/Processors/QueryPlan/DistinctStep.h index dc734a58704..0d3b7e3b7e0 100644 --- a/src/Processors/QueryPlan/DistinctStep.h +++ b/src/Processors/QueryPlan/DistinctStep.h @@ -15,7 +15,8 @@ public: UInt64 limit_hint_, const Names & columns_, bool pre_distinct_, /// If is enabled, execute distinct for separate streams. Otherwise, merge streams. - bool optimize_distinct_in_order_); + bool optimize_distinct_in_order_, + bool optimize_distinct_in_order_memory_usage_); String getName() const override { return "Distinct"; } @@ -32,6 +33,7 @@ private: Names columns; bool pre_distinct; bool optimize_distinct_in_order; + bool optimize_distinct_in_order_memory_usage; }; } diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql index 2da81b50f6c..d43366a55fe 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql @@ -50,7 +50,7 @@ drop table if exists distinct_cardinality_low sync; drop table if exists distinct_in_order sync; drop table if exists ordinary_distinct sync; CREATE TABLE distinct_cardinality_low (low UInt64, medium UInt64, high UInt64) ENGINE MergeTree() ORDER BY (low, medium); -INSERT INTO distinct_cardinality_low SELECT number % 1e2, number % 1e4, number % 1e6 FROM numbers_mt(1e8); +INSERT INTO distinct_cardinality_low SELECT number % 1e1, number % 1e3, number % 1e5 FROM numbers_mt(1e6); create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); insert into distinct_in_order select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=1; create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index 327d0d52dc2..2a2ff8799f0 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -15,6 +15,12 @@ DistinctSortedChunkTransform -- distinct with primary key prefix and order by on the same columns -> pre-distinct and final distinct optimization DistinctSortedStreamTransform DistinctSortedChunkTransform +-- distinct with primary key prefix and order by column in distinct but non-primary key prefix -> pre-distinct and final distinct optimization +DistinctSortedTransform +DistinctSortedChunkTransform +-- distinct with primary key prefix and order by column in distinct but non-primary key prefix, optimize memory usage -> pre-distinct and final distinct optimization +DistinctSortedStreamTransform +DistinctSortedChunkTransform -- distinct with primary key prefix and order by on column _not_ in distinct -> pre-distinct optimization only DistinctTransform DistinctSortedChunkTransform diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh index e822f9695b9..f3f39997f0d 100755 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh @@ -33,6 +33,12 @@ $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by on the same columns -> pre-distinct and final distinct optimization'" $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a, b" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct but non-primary key prefix -> pre-distinct and final distinct optimization'" +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c settings optimize_distinct_in_order_memory_usage=0" | eval $FIND_DISTINCT + +$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct but non-primary key prefix, optimize memory usage -> pre-distinct and final distinct optimization'" +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c settings optimize_distinct_in_order_memory_usage=1" | eval $FIND_DISTINCT + $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by on column _not_ in distinct -> pre-distinct optimization only'" $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT From 151a98b18e618c0b20f0f842a181d69b9bde25e6 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 22 Jul 2022 10:08:46 -0400 Subject: [PATCH 542/659] restore aarch64 files due to a oversight --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index bbcac834526..a7d09c44652 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit bbcac834526d90d1e764164b861be426891d1743 +Subproject commit a7d09c446523b284fa4b72187bc11081adb2411f From aab7ce1aa82885ac4879c20339d6b44b581606fb Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 22 Jul 2022 10:25:14 -0400 Subject: [PATCH 543/659] Revert "restore aarch64 files due to a oversight" This reverts commit 151a98b18e618c0b20f0f842a181d69b9bde25e6. --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index a7d09c44652..bbcac834526 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit a7d09c446523b284fa4b72187bc11081adb2411f +Subproject commit bbcac834526d90d1e764164b861be426891d1743 From e7562f85fe860e6e33f0850f302e0879035f70ff Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 22 Jul 2022 12:00:53 -0400 Subject: [PATCH 544/659] Update sysroot aligned with master for missing files --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index bbcac834526..e9fb375d0a1 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit bbcac834526d90d1e764164b861be426891d1743 +Subproject commit e9fb375d0a1e5ebfd74c043f088f2342552103f8 From 4a2b7dcdb5e22f439dfde3af4f24266bda0ac158 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 21 Jul 2022 18:18:14 -0400 Subject: [PATCH 545/659] Update README.md --- tests/integration/README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index 2d44ff70861..b0e1a4fdbb1 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -44,7 +44,8 @@ sudo -H pip install \ dict2xml \ hypothesis \ pyhdfs \ - pika + pika \ + meilisearch ``` (highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python3-pytest python3-dicttoxml python3-docker python3-pymysql python3-protobuf python3-pymongo python3-tzlocal python3-kazoo python3-psycopg2 kafka-python python3-pytest-timeout python3-minio` From 121e8c4d4e5abf62ead71aeb33a2d4cf287ed08f Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 21 Jul 2022 22:47:40 -0400 Subject: [PATCH 546/659] nats-py is missing also --- tests/integration/README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index b0e1a4fdbb1..18d46908524 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -45,7 +45,8 @@ sudo -H pip install \ hypothesis \ pyhdfs \ pika \ - meilisearch + meilisearch \ + nats-py ``` (highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python3-pytest python3-dicttoxml python3-docker python3-pymysql python3-protobuf python3-pymongo python3-tzlocal python3-kazoo python3-psycopg2 kafka-python python3-pytest-timeout python3-minio` From 1889348529f01951bbdfc858e08bd2e5fe6f7402 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 21 Jul 2022 17:47:00 +0000 Subject: [PATCH 547/659] Update version_date.tsv and changelogs after v22.7.1.2484-stable --- docs/changelogs/v22.7.1.2484-stable.md | 468 +++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 469 insertions(+) create mode 100644 docs/changelogs/v22.7.1.2484-stable.md diff --git a/docs/changelogs/v22.7.1.2484-stable.md b/docs/changelogs/v22.7.1.2484-stable.md new file mode 100644 index 00000000000..d306b1b5bd4 --- /dev/null +++ b/docs/changelogs/v22.7.1.2484-stable.md @@ -0,0 +1,468 @@ +--- +sidebar_position: 1 +sidebar_label: 2022 +--- + +# 2022 Changelog + +### ClickHouse release v22.7.1.2484-stable FIXME as compared to v22.6.1.1985-stable + +#### Backward Incompatible Change +* Enable setting `enable_positional_arguments` by default. It allows queries like `SELECT ... ORDER BY 1, 2` where 1, 2 are the references to the select clause. If you need to return the old behavior, disable this setting. [#38204](https://github.com/ClickHouse/ClickHouse/pull/38204) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* `Ordinary` database engine and old storage definition syntax for `*MergeTree` tables are deprecated. By default it's not possible to create new ones. If `system` database has `Ordinary` engine it will be automatically converted to `Atomic` on server startup. There are settings to keep old behavior (`allow_deprecated_database_ordinary` and `allow_deprecated_syntax_for_merge_tree`), but these settings may be removed in future releases. [#38335](https://github.com/ClickHouse/ClickHouse/pull/38335) ([Alexander Tokmakov](https://github.com/tavplubix)). +* * Force rewriting comma join to inner by default (set default value `cross_to_inner_join_rewrite = 2`). To have old behavior set `cross_to_inner_join_rewrite = 1`. [#39326](https://github.com/ClickHouse/ClickHouse/pull/39326) ([Vladimir C](https://github.com/vdimir)). +* Disable format_csv_allow_single_quotes by default. [#37096](https://github.com/ClickHouse/ClickHouse/issues/37096). [#39423](https://github.com/ClickHouse/ClickHouse/pull/39423) ([Kruglov Pavel](https://github.com/Avogar)). + +#### New Feature +* Add new `direct` join algorithm for RocksDB, ref [#33582](https://github.com/ClickHouse/ClickHouse/issues/33582). [#35363](https://github.com/ClickHouse/ClickHouse/pull/35363) ([Vladimir C](https://github.com/vdimir)). +* * Added full sorting merge join algorithm. [#35796](https://github.com/ClickHouse/ClickHouse/pull/35796) ([Vladimir C](https://github.com/vdimir)). +* Add a setting `zstd_window_log_max` to configure max memory usage on zstd decoding when importing external files. Closes [#35693](https://github.com/ClickHouse/ClickHouse/issues/35693). [#37015](https://github.com/ClickHouse/ClickHouse/pull/37015) ([wuxiaobai24](https://github.com/wuxiaobai24)). +* Implement NatsStorage - table engine, which allows to pub/sub to NATS. Closes [#32388](https://github.com/ClickHouse/ClickHouse/issues/32388). [#37171](https://github.com/ClickHouse/ClickHouse/pull/37171) ([tchepavel](https://github.com/tchepavel)). +* Implement table function MongoDB. Allow writes into MongoDB storage / table function. [#37213](https://github.com/ClickHouse/ClickHouse/pull/37213) ([aaapetrenko](https://github.com/aaapetrenko)). +* `clickhouse-keeper` new feature: add support for real-time digest calculation and verification. [#37555](https://github.com/ClickHouse/ClickHouse/pull/37555) ([Antonio Andelic](https://github.com/antonio2368)). +* In [#17202](https://github.com/ClickHouse/ClickHouse/issues/17202) was reported that host_regexp was being tested against only one of the possible PTR responses. This PR makes the necessary changes so that host_regexp is applied against all possible PTR responses and validate if any matches. [#37827](https://github.com/ClickHouse/ClickHouse/pull/37827) ([Arthur Passos](https://github.com/arthurpassos)). +* Support hadoop secure rpc transfer(hadoop.rpc.protection=privacy and hadoop.rpc.protection=integrity). [#37852](https://github.com/ClickHouse/ClickHouse/pull/37852) ([Peng Liu](https://github.com/michael1589)). +* Add struct type support in `StorageHive`. [#38118](https://github.com/ClickHouse/ClickHouse/pull/38118) ([lgbo](https://github.com/lgbo-ustc)). +* Added Base58 encoding/decoding. [#38159](https://github.com/ClickHouse/ClickHouse/pull/38159) ([Andrey Zvonov](https://github.com/zvonand)). +* Add chart visualization to Play UI. [#38197](https://github.com/ClickHouse/ClickHouse/pull/38197) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* support `alter` command on `StorageHive` table. [#38214](https://github.com/ClickHouse/ClickHouse/pull/38214) ([lgbo](https://github.com/lgbo-ustc)). +* Added `CREATE TABLE ... EMPTY AS SELECT` query. It automatically deduces table structure from the SELECT query, but does not fill the table after creation. Resolves [#38049](https://github.com/ClickHouse/ClickHouse/issues/38049). [#38272](https://github.com/ClickHouse/ClickHouse/pull/38272) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Adds new setting `implicit_transaction` to run standalone queries inside a transaction. It handles both creation and closing (via COMMIT if the query succeeded or ROLLBACK if it didn't) of the transaction automatically. [#38344](https://github.com/ClickHouse/ClickHouse/pull/38344) ([Raúl Marín](https://github.com/Algunenano)). +* Allow trailing comma in columns list. closes [#38425](https://github.com/ClickHouse/ClickHouse/issues/38425). [#38440](https://github.com/ClickHouse/ClickHouse/pull/38440) ([chen](https://github.com/xiedeyantu)). +* Compress clickhouse into self-extracting executable (path programs/self-extracting). New build target 'self-extracting' is added. [#38447](https://github.com/ClickHouse/ClickHouse/pull/38447) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Introduced settings `additional_table_filters`. Using this setting, you can specify additional filtering condition for a table which will be applied directly after reading. Example: `select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x settings additional_table_filters={'system.numbers : 'number != 3', 'table_1' : 'x != 2'}`. Introduced setting `additional_result_filter` which specifies additional filtering condition for query result. Closes [#37918](https://github.com/ClickHouse/ClickHouse/issues/37918). [#38475](https://github.com/ClickHouse/ClickHouse/pull/38475) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Add SQLInsert output format. Closes [#38441](https://github.com/ClickHouse/ClickHouse/issues/38441). [#38477](https://github.com/ClickHouse/ClickHouse/pull/38477) ([Kruglov Pavel](https://github.com/Avogar)). +* Downloadable clickhouse executable is compressed self-extracting. [#38653](https://github.com/ClickHouse/ClickHouse/pull/38653) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Support `isNullable` function. This function checks whether it's argument is nullable and return true(1) or false(0). Closes [#38611](https://github.com/ClickHouse/ClickHouse/issues/38611). [#38841](https://github.com/ClickHouse/ClickHouse/pull/38841) ([lokax](https://github.com/lokax)). +* Add functions `translate(string, from_string, to_string)` and `translateUTF8(string, from_string, to_string)`. [#38935](https://github.com/ClickHouse/ClickHouse/pull/38935) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add `compatibility` setting and `system.settings_changes` system table that contains information about changes in settings through ClickHouse versions. Closes [#35972](https://github.com/ClickHouse/ClickHouse/issues/35972). [#38957](https://github.com/ClickHouse/ClickHouse/pull/38957) ([Kruglov Pavel](https://github.com/Avogar)). +* Add the 3rd parameter to the tupleElement function and return it if tuple doesn't have a member. Only works if the 2nd parameter is of type String. Closes [#38872](https://github.com/ClickHouse/ClickHouse/issues/38872). [#38989](https://github.com/ClickHouse/ClickHouse/pull/38989) ([lokax](https://github.com/lokax)). +* Support parseTimedelta function. It can be used like ```sql # ' ', ';', '-', '+', ',', ':' can be used as separators, eg. "1yr-2mo", "2m:6s" SELECT parseTimeDelta('1yr-2mo-4w + 12 days, 3 hours : 1 minute ; 33 seconds');. [#39071](https://github.com/ClickHouse/ClickHouse/pull/39071) ([jiahui-97](https://github.com/jiahui-97)). +* Added options to limit IO operations with remote storage: `max_remote_read_network_bandwidth_for_server` and `max_remote_write_network_bandwidth_for_server`. [#39095](https://github.com/ClickHouse/ClickHouse/pull/39095) ([Sergei Trifonov](https://github.com/serxa)). +* Add `send_logs_source_regexp` setting. Send server text logs with specified regexp to match log source name. Empty means all sources. [#39161](https://github.com/ClickHouse/ClickHouse/pull/39161) ([Amos Bird](https://github.com/amosbird)). +* OpenTelemetry now collects traces without Processors spans by default. To enable Processors spans collection `opentelemetry_trace_processors` setting. [#39170](https://github.com/ClickHouse/ClickHouse/pull/39170) ([Ilya Yatsishin](https://github.com/qoega)). + +#### Performance Improvement +* Add new `local_filesystem_read_method` method `io_uring` based on the asynchronous Linux [io_uring](https://kernel.dk/io_uring.pdf) subsystem, improving read performance almost universally compared to the default `pread` method. [#36103](https://github.com/ClickHouse/ClickHouse/pull/36103) ([Saulius Valatka](https://github.com/sauliusvl)). +* Distinct optimization for sorted columns. Use specialized distinct transformation in case input stream is sorted by column(s) in distinct. Optimization can be applied to pre-distinct, final distinct, or both. Initial implementation by @dimarub2000. [#37803](https://github.com/ClickHouse/ClickHouse/pull/37803) ([Igor Nikonov](https://github.com/devcrafter)). +* Add VBMI optimized copyOverlap32Shuffle for LZ4 decompress. [#37891](https://github.com/ClickHouse/ClickHouse/pull/37891) ([Guo Wangyang](https://github.com/guowangy)). +* Improve performance of `ORDER BY`, `MergeTree` merges, window functions using batch version of `BinaryHeap`. [#38022](https://github.com/ClickHouse/ClickHouse/pull/38022) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix significant join performance regression which was introduced in https://github.com/ClickHouse/ClickHouse/pull/35616 . It's interesting that common join queries such as ssb queries have been 10 times slower for almost 3 months while no one complains. [#38052](https://github.com/ClickHouse/ClickHouse/pull/38052) ([Amos Bird](https://github.com/amosbird)). +* Migrate from the Intel hyperscan library to vectorscan, this speeds up many string matching on non-x86 platforms. [#38171](https://github.com/ClickHouse/ClickHouse/pull/38171) ([Robert Schulze](https://github.com/rschu1ze)). +* Increased parallelism of query plan steps executed after aggregation. [#38295](https://github.com/ClickHouse/ClickHouse/pull/38295) ([Nikita Taranov](https://github.com/nickitat)). +* Improve performance of insertion to columns of type `JSON`. [#38320](https://github.com/ClickHouse/ClickHouse/pull/38320) ([Anton Popov](https://github.com/CurtizJ)). +* Optimized insertion and lookups in the HashTable. [#38413](https://github.com/ClickHouse/ClickHouse/pull/38413) ([Nikita Taranov](https://github.com/nickitat)). +* Fix performance degradation from [#32493](https://github.com/ClickHouse/ClickHouse/issues/32493). [#38417](https://github.com/ClickHouse/ClickHouse/pull/38417) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve performance of column vector replicate using SIMD instructions. Author @zzachimed. [#38565](https://github.com/ClickHouse/ClickHouse/pull/38565) ([Maksim Kita](https://github.com/kitaisreal)). +* Norm and Distance functions for arrays speed up 1.2-2 times. [#38740](https://github.com/ClickHouse/ClickHouse/pull/38740) ([Alexander Gololobov](https://github.com/davenger)). +* A less efficient execution plan can be generated for query with ORDER BY (a, b) than for ORDER BY a, b. [#38873](https://github.com/ClickHouse/ClickHouse/pull/38873) ([Igor Nikonov](https://github.com/devcrafter)). +* Executable UDF, Executable Dictionary, Executable Storage poll subprocess fix 1 second subprocess wait during subprocess termination. [#38929](https://github.com/ClickHouse/ClickHouse/pull/38929) ([Constantine Peresypkin](https://github.com/pkit)). +* * Pushdown filter to the right side of sorting join. [#39123](https://github.com/ClickHouse/ClickHouse/pull/39123) ([Vladimir C](https://github.com/vdimir)). +* Optimize accesses to system.stack_trace. [#39177](https://github.com/ClickHouse/ClickHouse/pull/39177) ([Azat Khuzhin](https://github.com/azat)). + +#### Improvement +* Optimized processing of ORDER BY in window functions. [#34632](https://github.com/ClickHouse/ClickHouse/pull/34632) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Support SQL standard create index and drop index syntax. [#35166](https://github.com/ClickHouse/ClickHouse/pull/35166) ([Jianmei Zhang](https://github.com/zhangjmruc)). +* use simd to re-write the current column replicate funcion and got 2x performance boost in our unit benchmark test. [#37235](https://github.com/ClickHouse/ClickHouse/pull/37235) ([zzachimed](https://github.com/zzachimed)). +* Send profile events for INSERT queries (previously only SELECT was supported). [#37391](https://github.com/ClickHouse/ClickHouse/pull/37391) ([Azat Khuzhin](https://github.com/azat)). +* Implement in order aggregation (`optimize_aggregation_in_order`) for fully materialized projections. [#37469](https://github.com/ClickHouse/ClickHouse/pull/37469) ([Azat Khuzhin](https://github.com/azat)). +* * Bugfixes and performance improvements for `parallel_hash`. [#37648](https://github.com/ClickHouse/ClickHouse/pull/37648) ([Vladimir C](https://github.com/vdimir)). +* Support expressions with window functions. Closes [#19857](https://github.com/ClickHouse/ClickHouse/issues/19857). [#37848](https://github.com/ClickHouse/ClickHouse/pull/37848) ([Dmitry Novik](https://github.com/novikd)). +* S3 single objects are now removed with `RemoveObjectRequest` (sic). Fixed a bug with `S3ObjectStorage` on GCP which did not allow to use `removeFileIfExists` effectively breaking approximately half of `remove` functionality. Automatic detection for `DeleteObjects` S3 API, that is not supported by GCS. This will allow to use GCS without explicit `support_batch_delete=0` in configuration. [#37882](https://github.com/ClickHouse/ClickHouse/pull/37882) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fix refcnt for unused MergeTree parts in SELECT queries (may defer parts removal). [#37913](https://github.com/ClickHouse/ClickHouse/pull/37913) ([Azat Khuzhin](https://github.com/azat)). +* Expose basic Keeper related monitoring data (via ProfileEvents and CurrentMetrics). [#38072](https://github.com/ClickHouse/ClickHouse/pull/38072) ([lingpeng0314](https://github.com/lingpeng0314)). +* Added kerberosInit function and corresponding KerberosInit class as a replacement for kinit executable. Replaced all calls of kinit in Kafka and HDFS code by call of kerberosInit function. Added new integration test. Closes [#27651](https://github.com/ClickHouse/ClickHouse/issues/27651). [#38105](https://github.com/ClickHouse/ClickHouse/pull/38105) ([Roman Vasin](https://github.com/rvasin)). +* * Add setting `multiple_joins_try_to_keep_original_names` to not rewrite identifier name on multiple JOINs rewrite, close [#34697](https://github.com/ClickHouse/ClickHouse/issues/34697). [#38149](https://github.com/ClickHouse/ClickHouse/pull/38149) ([Vladimir C](https://github.com/vdimir)). +* improved trace-visualizer UX. [#38169](https://github.com/ClickHouse/ClickHouse/pull/38169) ([Sergei Trifonov](https://github.com/serxa)). +* Add ability to pass headers to url table function / storage via sql. Closes [#37897](https://github.com/ClickHouse/ClickHouse/issues/37897). [#38176](https://github.com/ClickHouse/ClickHouse/pull/38176) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Enable trace collection for AArch64. [#38181](https://github.com/ClickHouse/ClickHouse/pull/38181) ([Maksim Kita](https://github.com/kitaisreal)). +* Do not skip symlinks in `user_defined` directory during SQL user defined functions loading. Closes [#38042](https://github.com/ClickHouse/ClickHouse/issues/38042). [#38184](https://github.com/ClickHouse/ClickHouse/pull/38184) ([Maksim Kita](https://github.com/kitaisreal)). +* Improve the stability for hive storage integration test. Move the data prepare step into test.py. [#38260](https://github.com/ClickHouse/ClickHouse/pull/38260) ([lgbo](https://github.com/lgbo-ustc)). +* Added background cleanup of subdirectories in `store/`. In some cases clickhouse-server might left garbage subdirectories in `store/` (for example, on unsuccessful table creation) and those dirs were never been removed. Fixes [#33710](https://github.com/ClickHouse/ClickHouse/issues/33710). [#38265](https://github.com/ClickHouse/ClickHouse/pull/38265) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add `DESCRIBE CACHE` query to show cache settings from config. Add `SHOW CACHES` query to show available filesystem caches list. [#38279](https://github.com/ClickHouse/ClickHouse/pull/38279) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add access check for system drop fs cache. Support ON CLUSTER. [#38319](https://github.com/ClickHouse/ClickHouse/pull/38319) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Support `auto_close` option for postgres engine connection. Closes [#31486](https://github.com/ClickHouse/ClickHouse/issues/31486). [#38363](https://github.com/ClickHouse/ClickHouse/pull/38363) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix PostgreSQL database engine incompatibility on upgrade from 21.3 to 22.3. Closes [#36659](https://github.com/ClickHouse/ClickHouse/issues/36659). [#38369](https://github.com/ClickHouse/ClickHouse/pull/38369) ([Kseniia Sumarokova](https://github.com/kssenii)). +* `filesystemAvailable` and similar functions now work in `clickhouse-local`. This closes [#38423](https://github.com/ClickHouse/ClickHouse/issues/38423). [#38424](https://github.com/ClickHouse/ClickHouse/pull/38424) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Hardware benchmark now has support for automatic results uploading. [#38427](https://github.com/ClickHouse/ClickHouse/pull/38427) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The table `system.asynchronous_metric_log` is further optimized for storage space. This closes [#38134](https://github.com/ClickHouse/ClickHouse/issues/38134). See the [YouTube video](https://www.youtube.com/watch?v=0fSp9SF8N8A). [#38428](https://github.com/ClickHouse/ClickHouse/pull/38428) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Functions multiMatchAny(), multiMatchAnyIndex(), multiMatchAllIndices() and their fuzzy variants now accept non-const pattern array argument. [#38485](https://github.com/ClickHouse/ClickHouse/pull/38485) ([Robert Schulze](https://github.com/rschu1ze)). +* Added L2 Squared distance and norm for both arrays and tuples. [#38545](https://github.com/ClickHouse/ClickHouse/pull/38545) ([Julian Gilyadov](https://github.com/israelg99)). +* Add revision() function. [#38555](https://github.com/ClickHouse/ClickHouse/pull/38555) ([Azat Khuzhin](https://github.com/azat)). +* Add `group_by_use_nulls` setting to make aggregation key columns nullable in the case of ROLLUP, CUBE and GROUPING SETS. Closes [#37359](https://github.com/ClickHouse/ClickHouse/issues/37359). [#38642](https://github.com/ClickHouse/ClickHouse/pull/38642) ([Dmitry Novik](https://github.com/novikd)). +* Fix GCS via proxy tunnel usage. [#38726](https://github.com/ClickHouse/ClickHouse/pull/38726) ([Azat Khuzhin](https://github.com/azat)). +* Support `\i file` in clickhouse client / local (similar to psql \i). [#38813](https://github.com/ClickHouse/ClickHouse/pull/38813) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Allow null modifier in columns declaration for table functions. [#38816](https://github.com/ClickHouse/ClickHouse/pull/38816) ([Kruglov Pavel](https://github.com/Avogar)). +* - Deactivate `mutations_finalizing_task` before shutdown to avoid `TABLE_IS_READ_ONLY` errors. [#38851](https://github.com/ClickHouse/ClickHouse/pull/38851) ([Raúl Marín](https://github.com/Algunenano)). +* Fix waiting of shared lock after exclusive lock failure. [#38864](https://github.com/ClickHouse/ClickHouse/pull/38864) ([Azat Khuzhin](https://github.com/azat)). +* Add the ability to specify compression level during data export. [#38907](https://github.com/ClickHouse/ClickHouse/pull/38907) ([Nikolay Degterinsky](https://github.com/evillique)). +* New option `rewrite` in `EXPLAIN AST`. If enabled, it shows AST after it's rewritten, otherwise AST of original query. Disabled by default. [#38910](https://github.com/ClickHouse/ClickHouse/pull/38910) ([Igor Nikonov](https://github.com/devcrafter)). +* - Stop reporting Zookeeper "Node exists" exceptions in system.errors when they are expected. [#38961](https://github.com/ClickHouse/ClickHouse/pull/38961) ([Raúl Marín](https://github.com/Algunenano)). +* Allow to specify globs `* or {expr1, expr2, expr3}` inside a key for `clickhouse-extract-from-config` tool. [#38966](https://github.com/ClickHouse/ClickHouse/pull/38966) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add option enabling that SELECT from the system database requires grant. Details:. [#38970](https://github.com/ClickHouse/ClickHouse/pull/38970) ([Vitaly Baranov](https://github.com/vitlibar)). +* - clearOldLogs: Don't report KEEPER_EXCEPTION on concurrent deletes. [#39016](https://github.com/ClickHouse/ClickHouse/pull/39016) ([Raúl Marín](https://github.com/Algunenano)). +* clickhouse-keeper improvement: persist metainformation about keeper servers to disk. [#39069](https://github.com/ClickHouse/ClickHouse/pull/39069) ([Antonio Andelic](https://github.com/antonio2368)). +* Continue without exception when running out of disk space when using filesystem cache. [#39106](https://github.com/ClickHouse/ClickHouse/pull/39106) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Handling SIGTERM signals from k8s. [#39130](https://github.com/ClickHouse/ClickHouse/pull/39130) ([Timur Solodovnikov](https://github.com/tsolodov)). +* SQL function multiStringAllPositions() now accepts non-const needle arguments. [#39167](https://github.com/ClickHouse/ClickHouse/pull/39167) ([Robert Schulze](https://github.com/rschu1ze)). +* Add merge_algorithm (Undecided, Horizontal, Vertical) to system.part_log. [#39181](https://github.com/ClickHouse/ClickHouse/pull/39181) ([Azat Khuzhin](https://github.com/azat)). +* Improve isNullable/isConstant/isNull/isNotNull performance for LowCardinality argument. [#39192](https://github.com/ClickHouse/ClickHouse/pull/39192) ([Kruglov Pavel](https://github.com/Avogar)). +* - Don't report system.errors when the disk is not rotational. [#39216](https://github.com/ClickHouse/ClickHouse/pull/39216) ([Raúl Marín](https://github.com/Algunenano)). +* Metric `result_bytes` for `INSERT` queries in `system.query_log` shows number of bytes inserted. Previously value was incorrect and stored the same value as `result_rows`. [#39225](https://github.com/ClickHouse/ClickHouse/pull/39225) ([Ilya Yatsishin](https://github.com/qoega)). +* The CPU usage metric in clickhouse-client will be displayed in a better way. Fixes [#38756](https://github.com/ClickHouse/ClickHouse/issues/38756). [#39280](https://github.com/ClickHouse/ClickHouse/pull/39280) ([Sergei Trifonov](https://github.com/serxa)). +* Rethrow exception on filesystem cache initialisation on server startup, better error message. [#39386](https://github.com/ClickHouse/ClickHouse/pull/39386) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Support milliseconds, microseconds and nanoseconds in `parseTimeDelta` function. [#39447](https://github.com/ClickHouse/ClickHouse/pull/39447) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Bug Fix +* Fix crash when executing GRANT ALL ON *.* with ON CLUSTER. It was broken in https://github.com/ClickHouse/ClickHouse/pull/35767. This closes [#38618](https://github.com/ClickHouse/ClickHouse/issues/38618). [#38674](https://github.com/ClickHouse/ClickHouse/pull/38674) ([Vitaly Baranov](https://github.com/vitlibar)). +* * Fixed crash caused by IHiveFile be shared among threads. [#38887](https://github.com/ClickHouse/ClickHouse/pull/38887) ([lgbo](https://github.com/lgbo-ustc)). + +#### Build/Testing/Packaging Improvement +* - Apply Clang Thread Safety Analysis (TSA) annotations to ClickHouse. [#38068](https://github.com/ClickHouse/ClickHouse/pull/38068) ([Robert Schulze](https://github.com/rschu1ze)). +* - System table "system.licenses" is now correctly populated on Mac (Darwin). [#38294](https://github.com/ClickHouse/ClickHouse/pull/38294) ([Robert Schulze](https://github.com/rschu1ze)). +* Handle full queue exception in clickhouse-test. If it happened we need to collect debug info to understand what queries didn't finish. [#38490](https://github.com/ClickHouse/ClickHouse/pull/38490) ([Dmitry Novik](https://github.com/novikd)). +* - Change `all|noarch` packages to architecture-dependent - Fix some documentation for it - Push aarch64|arm64 packages to artifactory and release assets - Fixes [#36443](https://github.com/ClickHouse/ClickHouse/issues/36443). [#38580](https://github.com/ClickHouse/ClickHouse/pull/38580) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add `clickhouse-diagnostics` binary to the packages. [#38647](https://github.com/ClickHouse/ClickHouse/pull/38647) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Always print stacktraces if test queue is full. Follow up [#38490](https://github.com/ClickHouse/ClickHouse/issues/38490) cc @tavplubix. [#38662](https://github.com/ClickHouse/ClickHouse/pull/38662) ([Dmitry Novik](https://github.com/novikd)). +* Align branches within a 32B boundary to make benchmark more stable. [#38988](https://github.com/ClickHouse/ClickHouse/pull/38988) ([Guo Wangyang](https://github.com/guowangy)). +* Fix LSan by fixing getauxval(). [#39299](https://github.com/ClickHouse/ClickHouse/pull/39299) ([Azat Khuzhin](https://github.com/azat)). +* Adapt universal installation script for FreeBSD. [#39302](https://github.com/ClickHouse/ClickHouse/pull/39302) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Fix projection exception when aggregation keys are wrapped inside other functions. This fixes [#37151](https://github.com/ClickHouse/ClickHouse/issues/37151). [#37155](https://github.com/ClickHouse/ClickHouse/pull/37155) ([Amos Bird](https://github.com/amosbird)). +* Fix possible logical error `... with argument with type Nothing and default implementation for Nothing is expected to return result with type Nothing, got ...` in some functions. Closes: [#37610](https://github.com/ClickHouse/ClickHouse/issues/37610) Closes: [#37741](https://github.com/ClickHouse/ClickHouse/issues/37741). [#37759](https://github.com/ClickHouse/ClickHouse/pull/37759) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix incorrect columns order in subqueries of UNION (in case of duplicated columns in subselects may produce incorrect result). [#37887](https://github.com/ClickHouse/ClickHouse/pull/37887) ([Azat Khuzhin](https://github.com/azat)). +* Fix incorrect work of MODIFY ALTER Column with column names that contain dots. Closes [#37907](https://github.com/ClickHouse/ClickHouse/issues/37907). [#37971](https://github.com/ClickHouse/ClickHouse/pull/37971) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix reading of sparse columns from `MergeTree` tables that store their data in S3. [#37978](https://github.com/ClickHouse/ClickHouse/pull/37978) ([Anton Popov](https://github.com/CurtizJ)). +* Fix rounding for `Decimal128/Decimal256` with more than 19-digits long scale. [#38027](https://github.com/ClickHouse/ClickHouse/pull/38027) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix possible crash in `Distributed` async insert in case of removing a replica from config. [#38029](https://github.com/ClickHouse/ClickHouse/pull/38029) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix "Missing columns" for GLOBAL JOIN with CTE w/o alias. [#38056](https://github.com/ClickHouse/ClickHouse/pull/38056) ([Azat Khuzhin](https://github.com/azat)). +* Rewrite tuple functions as literals in backwards-compatibility mode. [#38096](https://github.com/ClickHouse/ClickHouse/pull/38096) ([Anton Kozlov](https://github.com/tonickkozlov)). +* - Fix redundant memory reservation for output block during `ORDER BY`. [#38127](https://github.com/ClickHouse/ClickHouse/pull/38127) ([iyupeng](https://github.com/iyupeng)). +* Fix possible logical error `Bad cast from type DB::IColumn* to DB::ColumnNullable*` in array mapped functions. Closes [#38006](https://github.com/ClickHouse/ClickHouse/issues/38006). [#38132](https://github.com/ClickHouse/ClickHouse/pull/38132) ([Kruglov Pavel](https://github.com/Avogar)). +* * Fix temporary name clash in partial merge join, close [#37928](https://github.com/ClickHouse/ClickHouse/issues/37928). [#38135](https://github.com/ClickHouse/ClickHouse/pull/38135) ([Vladimir C](https://github.com/vdimir)). +* With table ```SQL CREATE TABLE nested_name_tuples ( `a` Tuple(x String, y Tuple(i Int32, j String)) ) ENGINE = Memory; ```. [#38136](https://github.com/ClickHouse/ClickHouse/pull/38136) ([lgbo](https://github.com/lgbo-ustc)). +* Fix bug with nested short-circuit functions that led to execution of arguments even if condition is false. Closes [#38040](https://github.com/ClickHouse/ClickHouse/issues/38040). [#38173](https://github.com/ClickHouse/ClickHouse/pull/38173) ([Kruglov Pavel](https://github.com/Avogar)). +* (Window View is a experimental feature) Fix LOGICAL_ERROR for WINDOW VIEW with incorrect structure. [#38205](https://github.com/ClickHouse/ClickHouse/pull/38205) ([Azat Khuzhin](https://github.com/azat)). +* Update librdkafka submodule to fix crash when an OAUTHBEARER refresh callback is set. [#38225](https://github.com/ClickHouse/ClickHouse/pull/38225) ([Rafael Acevedo](https://github.com/racevedoo)). +* Do not allow recursive usage of OvercommitTracker during logging. Fixes [#37794](https://github.com/ClickHouse/ClickHouse/issues/37794) cc @tavplubix @davenger. [#38246](https://github.com/ClickHouse/ClickHouse/pull/38246) ([Dmitry Novik](https://github.com/novikd)). +* Fix INSERT into Distributed hung due to ProfileEvents. [#38307](https://github.com/ClickHouse/ClickHouse/pull/38307) ([Azat Khuzhin](https://github.com/azat)). +* Fix retries in PostgreSQL engine. [#38310](https://github.com/ClickHouse/ClickHouse/pull/38310) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix optimization in PartialSortingTransform (SIGSEGV and possible incorrect result). [#38324](https://github.com/ClickHouse/ClickHouse/pull/38324) ([Azat Khuzhin](https://github.com/azat)). +* Fix RabbitMQ with formats based on PeekableReadBuffer. Closes [#38061](https://github.com/ClickHouse/ClickHouse/issues/38061). [#38356](https://github.com/ClickHouse/ClickHouse/pull/38356) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix possible `Invalid number of rows in Chunk` in materialised pg. Closes [#37323](https://github.com/ClickHouse/ClickHouse/issues/37323). [#38360](https://github.com/ClickHouse/ClickHouse/pull/38360) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix RabbitMQ configuration with connection string setting. Closes [#36531](https://github.com/ClickHouse/ClickHouse/issues/36531). [#38365](https://github.com/ClickHouse/ClickHouse/pull/38365) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix PostgreSQL engine not using PostgreSQL schema when retrieving array dimension size. Closes [#36755](https://github.com/ClickHouse/ClickHouse/issues/36755). Closes [#36772](https://github.com/ClickHouse/ClickHouse/issues/36772). [#38366](https://github.com/ClickHouse/ClickHouse/pull/38366) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix incorrect result of distributed queries with `DISTINCT` and `LIMIT`. Fixes [#38282](https://github.com/ClickHouse/ClickHouse/issues/38282). [#38371](https://github.com/ClickHouse/ClickHouse/pull/38371) ([Anton Popov](https://github.com/CurtizJ)). +* fix: expose new CH keeper port in Dockerfile clickhouse/clickhouse-keeper fix: use correct KEEPER_CONFIG filename in clickhouse/clickhouse-keeper docker image. [#38462](https://github.com/ClickHouse/ClickHouse/pull/38462) ([Evgeny Kruglov](https://github.com/nordluf)). +* Fix parts removal (will be left forever if they had not been removed on server shutdown) after incorrect server shutdown. [#38486](https://github.com/ClickHouse/ClickHouse/pull/38486) ([Azat Khuzhin](https://github.com/azat)). +* Fixes [#38498](https://github.com/ClickHouse/ClickHouse/issues/38498) Current Implementation is similar to what shell does mentiond by @rschu1ze [here](https://github.com/ClickHouse/ClickHouse/pull/38502#issuecomment-1169057723). [#38502](https://github.com/ClickHouse/ClickHouse/pull/38502) ([Heena Bansal](https://github.com/HeenaBansal2009)). +* Fix table creation to avoid replication issues with pre-22.4 replicas. [#38541](https://github.com/ClickHouse/ClickHouse/pull/38541) ([Raúl Marín](https://github.com/Algunenano)). +* Fix crash for `mapUpdate`, `mapFilter` functions when using with constant map argument. Closes [#38547](https://github.com/ClickHouse/ClickHouse/issues/38547). [#38553](https://github.com/ClickHouse/ClickHouse/pull/38553) ([hexiaoting](https://github.com/hexiaoting)). +* Fix wrong results of countSubstrings() & position() on patterns with 0-bytes. [#38589](https://github.com/ClickHouse/ClickHouse/pull/38589) ([Robert Schulze](https://github.com/rschu1ze)). +* Now it's possible to start a clickhouse-server and attach/detach tables even for tables with the incorrect values of IPv4/IPv6 representation. Proper fix for issue [#35156](https://github.com/ClickHouse/ClickHouse/issues/35156). [#38590](https://github.com/ClickHouse/ClickHouse/pull/38590) ([alesapin](https://github.com/alesapin)). +* Adapt some more nodes to avoid issues with pre-22.4 replicas. [#38627](https://github.com/ClickHouse/ClickHouse/pull/38627) ([Raúl Marín](https://github.com/Algunenano)). +* Fix toHour() monotonicity which can lead to incorrect query result (incorrect index analysis). This fixes [#38333](https://github.com/ClickHouse/ClickHouse/issues/38333). [#38675](https://github.com/ClickHouse/ClickHouse/pull/38675) ([Amos Bird](https://github.com/amosbird)). +* `rankCorr` function will work correctly if some arguments are NaNs. This closes [#38396](https://github.com/ClickHouse/ClickHouse/issues/38396). [#38722](https://github.com/ClickHouse/ClickHouse/pull/38722) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix `parallel_view_processing=1` with `optimize_trivial_insert_select=1`. Fix `max_insert_threads` while pushing to views. [#38731](https://github.com/ClickHouse/ClickHouse/pull/38731) ([Azat Khuzhin](https://github.com/azat)). +* Fix use-after-free for Map combinator that leads to incorrect result. [#38748](https://github.com/ClickHouse/ClickHouse/pull/38748) ([Azat Khuzhin](https://github.com/azat)). +* Fix throwing exception for seekable read from s3 (exception was not thrown). [#38773](https://github.com/ClickHouse/ClickHouse/pull/38773) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix checking whether s3 storage support parallel writes. It resulted in s3 parallel writes not working. [#38792](https://github.com/ClickHouse/ClickHouse/pull/38792) ([chen](https://github.com/xiedeyantu)). +* Fix s3 seekable reads with parallel read buffer. (Affected memory usage during query). Closes [#38258](https://github.com/ClickHouse/ClickHouse/issues/38258). [#38802](https://github.com/ClickHouse/ClickHouse/pull/38802) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update `simdjson`. This fixes [#38621](https://github.com/ClickHouse/ClickHouse/issues/38621). [#38838](https://github.com/ClickHouse/ClickHouse/pull/38838) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* MergeTree fix possible logical error for Vertical merges. [#38859](https://github.com/ClickHouse/ClickHouse/pull/38859) ([Maksim Kita](https://github.com/kitaisreal)). +* - Fix settings profile with seconds unit. [#38896](https://github.com/ClickHouse/ClickHouse/pull/38896) ([Raúl Marín](https://github.com/Algunenano)). +* Fix incorrect partition pruning when there is a nullable partition. This fixes [#38941](https://github.com/ClickHouse/ClickHouse/issues/38941). [#38946](https://github.com/ClickHouse/ClickHouse/pull/38946) ([Amos Bird](https://github.com/amosbird)). +* Fix fsync_part_directory for fetches. [#38993](https://github.com/ClickHouse/ClickHouse/pull/38993) ([Azat Khuzhin](https://github.com/azat)). +* Functions multiMatch[Fuzzy](AllIndices/Any/AnyIndex)() no throw a logical error if the needle argument is empty. [#39012](https://github.com/ClickHouse/ClickHouse/pull/39012) ([Robert Schulze](https://github.com/rschu1ze)). +* Any allocations inside OvercommitTracker may lead to deadlock. Logging was not very informative so it's easier just to remove logging. Fixes [#37794](https://github.com/ClickHouse/ClickHouse/issues/37794). [#39030](https://github.com/ClickHouse/ClickHouse/pull/39030) ([Dmitry Novik](https://github.com/novikd)). +* Fix toHour() monotonicity which can lead to incorrect query result (incorrect index analysis). This fixes [#38333](https://github.com/ClickHouse/ClickHouse/issues/38333). [#39037](https://github.com/ClickHouse/ClickHouse/pull/39037) ([Amos Bird](https://github.com/amosbird)). +* Fix bug in filesystem cache that could happen in some corner case which coincided with cache capacity hitting the limit. Closes [#39066](https://github.com/ClickHouse/ClickHouse/issues/39066). [#39070](https://github.com/ClickHouse/ClickHouse/pull/39070) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix ActionsDAG construction for arguments of window expressions. Fixes [#38538](https://github.com/ClickHouse/ClickHouse/issues/38538) Allow using of higher-order functions in window expressions. [#39112](https://github.com/ClickHouse/ClickHouse/pull/39112) ([Dmitry Novik](https://github.com/novikd)). +* Keep `LowCardinality` type in `tuple()` function. Previously `LowCardinality` type was dropped and elements of created tuple had underlying type of `LowCardinality`. [#39113](https://github.com/ClickHouse/ClickHouse/pull/39113) ([Anton Popov](https://github.com/CurtizJ)). +* Fix error `Block structure mismatch` which could happen for INSERT into table with attached MATERIALIZED VIEW and enabled setting `extremes = 1`. Closes [#29759](https://github.com/ClickHouse/ClickHouse/issues/29759) and [#38729](https://github.com/ClickHouse/ClickHouse/issues/38729). [#39125](https://github.com/ClickHouse/ClickHouse/pull/39125) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix unexpected query result when both `optimize_trivial_count_query` and `empty_result_for_aggregation_by_empty_set` are set to true. This fixes [#39140](https://github.com/ClickHouse/ClickHouse/issues/39140). [#39155](https://github.com/ClickHouse/ClickHouse/pull/39155) ([Amos Bird](https://github.com/amosbird)). +* Fixed error `Not found column Type in block` in selects with `PREWHERE` and read-in-order optimizations. [#39157](https://github.com/ClickHouse/ClickHouse/pull/39157) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix extremely rare race condition in during hardnlinks for remote fs. The only way to reproduce it is concurrent run of backups. [#39190](https://github.com/ClickHouse/ClickHouse/pull/39190) ([alesapin](https://github.com/alesapin)). +* Fix fetch of in-memory part with `allow_remote_fs_zero_copy_replication`. [#39214](https://github.com/ClickHouse/ClickHouse/pull/39214) ([Azat Khuzhin](https://github.com/azat)). +* Fix NOEXCEPT_SCOPE (before it calls std::terminate and looses the exception). [#39229](https://github.com/ClickHouse/ClickHouse/pull/39229) ([Azat Khuzhin](https://github.com/azat)). +* Declare RabbitMQ queue without default arguments `x-max-length` and `x-overflow`. [#39259](https://github.com/ClickHouse/ClickHouse/pull/39259) ([rnbondarenko](https://github.com/rnbondarenko)). +* Fix segmentation fault in MaterializedPostgreSQL database engine, which could happen if some exception occurred at replication initialisation. Closes [#36939](https://github.com/ClickHouse/ClickHouse/issues/36939). [#39272](https://github.com/ClickHouse/ClickHouse/pull/39272) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix incorrect fetch postgresql tables query fro PostgreSQL database engine. Closes [#33502](https://github.com/ClickHouse/ClickHouse/issues/33502). [#39283](https://github.com/ClickHouse/ClickHouse/pull/39283) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix possible UB in MergeTreeBackgroundExecutor (leads to SIGSEGV on race with DROP/DETACH). [#39342](https://github.com/ClickHouse/ClickHouse/pull/39342) ([Azat Khuzhin](https://github.com/azat)). +* Avoid possible abort() in CapnProto on exception descruction. Closes [#30706](https://github.com/ClickHouse/ClickHouse/issues/30706). [#39365](https://github.com/ClickHouse/ClickHouse/pull/39365) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix behaviour of dictHas for direct dictionaries when multiple lookups to the same key are made in a single action. [#39385](https://github.com/ClickHouse/ClickHouse/pull/39385) ([James Morrison](https://github.com/jawm)). +* Fix crash which may happen while reading from dictionary with `DateTime64` attribute. Fixes [#38930](https://github.com/ClickHouse/ClickHouse/issues/38930). [#39391](https://github.com/ClickHouse/ClickHouse/pull/39391) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix WriteBuffer finalize in destructor when cacnel query that could lead to stuck query or even terminate. Closes [#38199](https://github.com/ClickHouse/ClickHouse/issues/38199). [#39396](https://github.com/ClickHouse/ClickHouse/pull/39396) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix UB (stack-use-after-scope) in extactAll(). [#39397](https://github.com/ClickHouse/ClickHouse/pull/39397) ([Azat Khuzhin](https://github.com/azat)). +* Fix incorrect query result when trivial count optimization is in effect with array join. This fixes [#39431](https://github.com/ClickHouse/ClickHouse/issues/39431). [#39444](https://github.com/ClickHouse/ClickHouse/pull/39444) ([Amos Bird](https://github.com/amosbird)). + +#### Bug Fix (user-visible misbehaviour in official stable or prestable release) + +* Disable send_logs_level for INSERT into Distributed to avoid possible hung. [#35075](https://github.com/ClickHouse/ClickHouse/pull/35075) ([Azat Khuzhin](https://github.com/azat)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Add a setting to use more memory for zstd decompression"'. [#38194](https://github.com/ClickHouse/ClickHouse/pull/38194) ([alesapin](https://github.com/alesapin)). +* NO CL ENTRY: 'Revert "Revert "Add a setting to use more memory for zstd decompression""'. [#38196](https://github.com/ClickHouse/ClickHouse/pull/38196) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "ClickHouse's boringssl module updated to the official version of the FIPS compliant."'. [#38201](https://github.com/ClickHouse/ClickHouse/pull/38201) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Fix optimization in PartialSortingTransform (SIGSEGV and possible incorrect result)"'. [#38361](https://github.com/ClickHouse/ClickHouse/pull/38361) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Add support for io_uring read method"'. [#38377](https://github.com/ClickHouse/ClickHouse/pull/38377) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Revert "Fix optimization in PartialSortingTransform (SIGSEGV and possible incorrect result)""'. [#38449](https://github.com/ClickHouse/ClickHouse/pull/38449) ([Maksim Kita](https://github.com/kitaisreal)). +* NO CL ENTRY: 'Don't spoil return code of integration tests runner with redundant tee'. [#38548](https://github.com/ClickHouse/ClickHouse/pull/38548) ([Vladimir Chebotarev](https://github.com/excitoon)). +* NO CL ENTRY: 'Revert "Non Negative Derivative window function"'. [#38551](https://github.com/ClickHouse/ClickHouse/pull/38551) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Upload to S3 compressed self-extracting clickhouse"'. [#38788](https://github.com/ClickHouse/ClickHouse/pull/38788) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* NO CL ENTRY: 'Revert "Smallish updates of dev guide"'. [#38848](https://github.com/ClickHouse/ClickHouse/pull/38848) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Fix toHour() monotonicity which can lead to incorrect query result (incorrect index analysis)"'. [#39001](https://github.com/ClickHouse/ClickHouse/pull/39001) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Fix WriteBuffer finalize in destructor when cacnel query"'. [#39433](https://github.com/ClickHouse/ClickHouse/pull/39433) ([Kruglov Pavel](https://github.com/Avogar)). +* NO CL ENTRY: 'Revert "[RFC] Fix LSan by fixing getauxval()"'. [#39434](https://github.com/ClickHouse/ClickHouse/pull/39434) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Remove broken optimisation in Direct dictionary dictHas implementation"'. [#39461](https://github.com/ClickHouse/ClickHouse/pull/39461) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Fix trivial count optimization with array join"'. [#39466](https://github.com/ClickHouse/ClickHouse/pull/39466) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Separate data storage abstraction for MergeTree [#36555](https://github.com/ClickHouse/ClickHouse/pull/36555) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Randomize settings related to in-order read/aggregation [#36914](https://github.com/ClickHouse/ClickHouse/pull/36914) ([Azat Khuzhin](https://github.com/azat)). +* Merge tree reader support for multiple read/filter steps: row level filter, prewhere, ... [#37165](https://github.com/ClickHouse/ClickHouse/pull/37165) ([Alexander Gololobov](https://github.com/davenger)). +* Backup Improvements 6 [#37358](https://github.com/ClickHouse/ClickHouse/pull/37358) ([Vitaly Baranov](https://github.com/vitlibar)). +* Move `updateInputStream` to `ITransformingStep` [#37393](https://github.com/ClickHouse/ClickHouse/pull/37393) ([Nikita Taranov](https://github.com/nickitat)). +* Proper wait of the clickhouse-server in tests [#37560](https://github.com/ClickHouse/ClickHouse/pull/37560) ([Azat Khuzhin](https://github.com/azat)). +* Upgrade curl to 7.83.1 [#37795](https://github.com/ClickHouse/ClickHouse/pull/37795) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* Try fix flaky tests with transactions [#37822](https://github.com/ClickHouse/ClickHouse/pull/37822) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Send perf tests results to ci database [#37841](https://github.com/ClickHouse/ClickHouse/pull/37841) ([Vladimir C](https://github.com/vdimir)). +* Remove duplicate peak mem log [#37860](https://github.com/ClickHouse/ClickHouse/pull/37860) ([Amos Bird](https://github.com/amosbird)). +* tests: fix log_comment (extra quotes) [#37932](https://github.com/ClickHouse/ClickHouse/pull/37932) ([Azat Khuzhin](https://github.com/azat)). +* Throw exception when xml user profile does not exist [#38024](https://github.com/ClickHouse/ClickHouse/pull/38024) ([nvartolomei](https://github.com/nvartolomei)). +* Add `SYNC` command to internal ZooKeeper client [#38047](https://github.com/ClickHouse/ClickHouse/pull/38047) ([Antonio Andelic](https://github.com/antonio2368)). +* Better support of GCP storage [#38069](https://github.com/ClickHouse/ClickHouse/pull/38069) ([Anton Popov](https://github.com/CurtizJ)). +* Build artifacts upload [#38086](https://github.com/ClickHouse/ClickHouse/pull/38086) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Extract some diff from pr [#36171](https://github.com/ClickHouse/ClickHouse/issues/36171) [#38088](https://github.com/ClickHouse/ClickHouse/pull/38088) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Aggregate functions added restrict into batch methods [#38090](https://github.com/ClickHouse/ClickHouse/pull/38090) ([Maksim Kita](https://github.com/kitaisreal)). +* Add perf checkers to all Jepsen tests [#38091](https://github.com/ClickHouse/ClickHouse/pull/38091) ([Antonio Andelic](https://github.com/antonio2368)). +* Some fixes for tests with tsan [#38106](https://github.com/ClickHouse/ClickHouse/pull/38106) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Bring back [#36396](https://github.com/ClickHouse/ClickHouse/issues/36396) [#38110](https://github.com/ClickHouse/ClickHouse/pull/38110) ([Nikita Taranov](https://github.com/nickitat)). +* More suppressions for backward compatibility check [#38131](https://github.com/ClickHouse/ClickHouse/pull/38131) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Cherry pick [#38137](https://github.com/ClickHouse/ClickHouse/pull/38137) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Disable parameters for non direct executable user defined functions [#38142](https://github.com/ClickHouse/ClickHouse/pull/38142) ([Maksim Kita](https://github.com/kitaisreal)). +* SortDescription compile fix typo [#38144](https://github.com/ClickHouse/ClickHouse/pull/38144) ([Maksim Kita](https://github.com/kitaisreal)). +* Update version after release [#38147](https://github.com/ClickHouse/ClickHouse/pull/38147) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* New changelog and versions updated [#38148](https://github.com/ClickHouse/ClickHouse/pull/38148) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Some fixes for clickhouse-disks [#38150](https://github.com/ClickHouse/ClickHouse/pull/38150) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove processor description from span attributes - it is not working [#38157](https://github.com/ClickHouse/ClickHouse/pull/38157) ([Ilya Yatsishin](https://github.com/qoega)). +* Bump minimum / maximum LLVM to 12 / 14 [#38170](https://github.com/ClickHouse/ClickHouse/pull/38170) ([Robert Schulze](https://github.com/rschu1ze)). +* Disk transaction [#38182](https://github.com/ClickHouse/ClickHouse/pull/38182) ([alesapin](https://github.com/alesapin)). +* Check row size to avoid out of bounds access in PostgreSQLSource [#38190](https://github.com/ClickHouse/ClickHouse/pull/38190) ([Alexander Gololobov](https://github.com/davenger)). +* tests: add no-backward-compatibility-check for 02067_lost_part_s3 [#38195](https://github.com/ClickHouse/ClickHouse/pull/38195) ([Azat Khuzhin](https://github.com/azat)). +* tests/stress: fix TSan detection (enables thread fuzzer for non-TSan builds) [#38207](https://github.com/ClickHouse/ClickHouse/pull/38207) ([Azat Khuzhin](https://github.com/azat)). +* tests: disable 01646_system_restart_replicas_smoke under stress tests [#38212](https://github.com/ClickHouse/ClickHouse/pull/38212) ([Azat Khuzhin](https://github.com/azat)). +* tests/stress: fix TSan detection [#38213](https://github.com/ClickHouse/ClickHouse/pull/38213) ([Azat Khuzhin](https://github.com/azat)). +* buffer's getFileSize small changes [#38227](https://github.com/ClickHouse/ClickHouse/pull/38227) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix test for system table count in diag tool [#38236](https://github.com/ClickHouse/ClickHouse/pull/38236) ([Dale McDiarmid](https://github.com/gingerwizard)). +* Update version_date.tsv after v22.3.7.28-lts [#38237](https://github.com/ClickHouse/ClickHouse/pull/38237) ([github-actions[bot]](https://github.com/apps/github-actions)). +* Changelog attrs [#38238](https://github.com/ClickHouse/ClickHouse/pull/38238) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix reading from s3 in some corner cases [#38239](https://github.com/ClickHouse/ClickHouse/pull/38239) ([Anton Popov](https://github.com/CurtizJ)). +* use utility methods to access x509 struct fields. [#38251](https://github.com/ClickHouse/ClickHouse/pull/38251) ([larryluogit](https://github.com/larryluogit)). +* Don't try to kill empty list of containers in `integration/runner` II [#38269](https://github.com/ClickHouse/ClickHouse/pull/38269) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Improve runners AMI and init scripts [#38273](https://github.com/ClickHouse/ClickHouse/pull/38273) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update thrift to 0.16.0 [#38280](https://github.com/ClickHouse/ClickHouse/pull/38280) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* Extract some diff from [#36171](https://github.com/ClickHouse/ClickHouse/issues/36171) [#38285](https://github.com/ClickHouse/ClickHouse/pull/38285) ([Kseniia Sumarokova](https://github.com/kssenii)). +* fix trace-viz zoom anomalies [#38287](https://github.com/ClickHouse/ClickHouse/pull/38287) ([Sergei Trifonov](https://github.com/serxa)). +* Integration tests volume [#38291](https://github.com/ClickHouse/ClickHouse/pull/38291) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* fix typo in view.md [#38292](https://github.com/ClickHouse/ClickHouse/pull/38292) ([Anton Petrov](https://github.com/gsenseless)). +* Backup improvements 7 [#38299](https://github.com/ClickHouse/ClickHouse/pull/38299) ([Vitaly Baranov](https://github.com/vitlibar)). +* Document why the submodule check does not halt the configuration [#38304](https://github.com/ClickHouse/ClickHouse/pull/38304) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix misleading error message while s3 schema inference [#38306](https://github.com/ClickHouse/ClickHouse/pull/38306) ([Kruglov Pavel](https://github.com/Avogar)). +* Update README.md [#38313](https://github.com/ClickHouse/ClickHouse/pull/38313) ([Yuko Takagi](https://github.com/yukotakagi)). +* Ban projections for zero-copy replication in a right way [#38322](https://github.com/ClickHouse/ClickHouse/pull/38322) ([alesapin](https://github.com/alesapin)). +* Checkout full repositories for performance tests [#38327](https://github.com/ClickHouse/ClickHouse/pull/38327) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fixed comments [#38331](https://github.com/ClickHouse/ClickHouse/pull/38331) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Try to fix 02305_schema_inference_with_globs [#38337](https://github.com/ClickHouse/ClickHouse/pull/38337) ([Kruglov Pavel](https://github.com/Avogar)). +* Extend ZooKeeper list request with support for filtering persistent or ephemeral nodes only [#38338](https://github.com/ClickHouse/ClickHouse/pull/38338) ([Antonio Andelic](https://github.com/antonio2368)). +* Upload logs for getting all tests command [#38343](https://github.com/ClickHouse/ClickHouse/pull/38343) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Followup test fix for ban projections [#38351](https://github.com/ClickHouse/ClickHouse/pull/38351) ([alesapin](https://github.com/alesapin)). +* Added --recursive to clickhouse-disks list [#38354](https://github.com/ClickHouse/ClickHouse/pull/38354) ([Alexander Gololobov](https://github.com/davenger)). +* Adding TLS V13 Test [#38355](https://github.com/ClickHouse/ClickHouse/pull/38355) ([larryluogit](https://github.com/larryluogit)). +* Better exception messages on wrong table engines/functions argument types [#38362](https://github.com/ClickHouse/ClickHouse/pull/38362) ([Kruglov Pavel](https://github.com/Avogar)). +* Better error message for failed odbc query [#38364](https://github.com/ClickHouse/ClickHouse/pull/38364) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Simplify parts commit methods [#38380](https://github.com/ClickHouse/ClickHouse/pull/38380) ([alesapin](https://github.com/alesapin)). +* Update docker-compose to try get rid of v1 errors [#38394](https://github.com/ClickHouse/ClickHouse/pull/38394) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Correct submodule after "base-x" commit [#38414](https://github.com/ClickHouse/ClickHouse/pull/38414) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better hardware benchmark [#38419](https://github.com/ClickHouse/ClickHouse/pull/38419) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Refactoring to enable multi-match functions with non-const needles [#38434](https://github.com/ClickHouse/ClickHouse/pull/38434) ([Robert Schulze](https://github.com/rschu1ze)). +* more consistent work with paths in object storages [#38436](https://github.com/ClickHouse/ClickHouse/pull/38436) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Distinct sorted: calculate column positions once [#38438](https://github.com/ClickHouse/ClickHouse/pull/38438) ([Igor Nikonov](https://github.com/devcrafter)). +* Small improvement of the error message to hint at possible issue [#38458](https://github.com/ClickHouse/ClickHouse/pull/38458) ([Miel Donkers](https://github.com/mdonkers)). +* Fix comment [#38465](https://github.com/ClickHouse/ClickHouse/pull/38465) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Follow up for [#38436](https://github.com/ClickHouse/ClickHouse/issues/38436) [#38466](https://github.com/ClickHouse/ClickHouse/pull/38466) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add 22.7 release webinar. [#38481](https://github.com/ClickHouse/ClickHouse/pull/38481) ([Yuko Takagi](https://github.com/yukotakagi)). +* Add some TSA annotations [#38487](https://github.com/ClickHouse/ClickHouse/pull/38487) ([Alexander Tokmakov](https://github.com/tavplubix)). +* tests: cleanup tmp data in 02335_column_ttl_expired_column_optimization [#38488](https://github.com/ClickHouse/ClickHouse/pull/38488) ([Azat Khuzhin](https://github.com/azat)). +* Cleanup: local clang-tidy warnings founded during review [#38489](https://github.com/ClickHouse/ClickHouse/pull/38489) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix some clang-tidy warnings in headers [#38491](https://github.com/ClickHouse/ClickHouse/pull/38491) ([Robert Schulze](https://github.com/rschu1ze)). +* A tiny improvement in report logging [#38507](https://github.com/ClickHouse/ClickHouse/pull/38507) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* tests: fix 02305_schema_inference_with_globs flakiness [#38511](https://github.com/ClickHouse/ClickHouse/pull/38511) ([Azat Khuzhin](https://github.com/azat)). +* Try to fix flaky test [#38516](https://github.com/ClickHouse/ClickHouse/pull/38516) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `_csv.Error: field larger than field limit` [#38518](https://github.com/ClickHouse/ClickHouse/pull/38518) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix application errors grep in stress test [#38520](https://github.com/ClickHouse/ClickHouse/pull/38520) ([Kruglov Pavel](https://github.com/Avogar)). +* Use of disk batch operations in MergeTree [#38531](https://github.com/ClickHouse/ClickHouse/pull/38531) ([alesapin](https://github.com/alesapin)). +* Backup Improvements 8 [#38537](https://github.com/ClickHouse/ClickHouse/pull/38537) ([Vitaly Baranov](https://github.com/vitlibar)). +* Update poco [#38540](https://github.com/ClickHouse/ClickHouse/pull/38540) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Don't use std::unique_lock unless we need to [#38542](https://github.com/ClickHouse/ClickHouse/pull/38542) ([Robert Schulze](https://github.com/rschu1ze)). +* Rename slightly weirdly named "BuilderBinTidy" to "BuilderBinClangTidy" [#38546](https://github.com/ClickHouse/ClickHouse/pull/38546) ([Robert Schulze](https://github.com/rschu1ze)). +* Don't rollback SessionID request in Keeper [#38556](https://github.com/ClickHouse/ClickHouse/pull/38556) ([Antonio Andelic](https://github.com/antonio2368)). +* Add logging in Epoll and TimerDescriptor in case of EINTR [#38559](https://github.com/ClickHouse/ClickHouse/pull/38559) ([Kruglov Pavel](https://github.com/Avogar)). +* SQL create drop index minor fixes [#38561](https://github.com/ClickHouse/ClickHouse/pull/38561) ([Maksim Kita](https://github.com/kitaisreal)). +* Update version_date.tsv and changelogs after v22.6.2.12-stable [#38563](https://github.com/ClickHouse/ClickHouse/pull/38563) ([github-actions[bot]](https://github.com/apps/github-actions)). +* Allow Ordinary database in Stress Tests [#38568](https://github.com/ClickHouse/ClickHouse/pull/38568) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Make postgres integration test great again [#38582](https://github.com/ClickHouse/ClickHouse/pull/38582) ([Ilya Yatsishin](https://github.com/qoega)). +* Add check for empty proccessors in AggregatingTransform::expandPipeline [#38584](https://github.com/ClickHouse/ClickHouse/pull/38584) ([filimonov](https://github.com/filimonov)). +* quick fix for 02112_with_fill_interval [#38587](https://github.com/ClickHouse/ClickHouse/pull/38587) ([Nikita Taranov](https://github.com/nickitat)). +* Remove zlib in mariadb-connector-c [#38599](https://github.com/ClickHouse/ClickHouse/pull/38599) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* Dictionaries added TSA annotations [#38601](https://github.com/ClickHouse/ClickHouse/pull/38601) ([Maksim Kita](https://github.com/kitaisreal)). +* CacheDictionary simplify update queue [#38602](https://github.com/ClickHouse/ClickHouse/pull/38602) ([Maksim Kita](https://github.com/kitaisreal)). +* Add separate option to omit symbols from heavy contrib [#38617](https://github.com/ClickHouse/ClickHouse/pull/38617) ([Azat Khuzhin](https://github.com/azat)). +* Fix exception messages in clickhouse su [#38619](https://github.com/ClickHouse/ClickHouse/pull/38619) ([filimonov](https://github.com/filimonov)). +* Added Greenplum benchmark [#38622](https://github.com/ClickHouse/ClickHouse/pull/38622) ([Dmitry Pavlov](https://github.com/kapustor)). +* Fix typo [#38623](https://github.com/ClickHouse/ClickHouse/pull/38623) ([tiegen](https://github.com/loyispa)). +* Better diagnostics in ReplicatedMergeTreeQueue [#38641](https://github.com/ClickHouse/ClickHouse/pull/38641) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Clean out randomized integration volumes each run [#38644](https://github.com/ClickHouse/ClickHouse/pull/38644) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update README.md [#38651](https://github.com/ClickHouse/ClickHouse/pull/38651) ([Yuko Takagi](https://github.com/yukotakagi)). +* Better naming for stuff related to splitted debug symbols [#38654](https://github.com/ClickHouse/ClickHouse/pull/38654) ([Robert Schulze](https://github.com/rschu1ze)). +* Add test for keeper `mntr` command [#38656](https://github.com/ClickHouse/ClickHouse/pull/38656) ([alesapin](https://github.com/alesapin)). +* Update hardware benchmark script [#38672](https://github.com/ClickHouse/ClickHouse/pull/38672) ([Filatenkov Artur](https://github.com/FArthur-cmd)). +* Fix strange backport titles issues [#38679](https://github.com/ClickHouse/ClickHouse/pull/38679) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Follow-up to [#38568](https://github.com/ClickHouse/ClickHouse/issues/38568) [#38680](https://github.com/ClickHouse/ClickHouse/pull/38680) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix possible deadlocks with `MergeTreeData::Transaction` [#38702](https://github.com/ClickHouse/ClickHouse/pull/38702) ([alesapin](https://github.com/alesapin)). +* Fix backports diff [#38703](https://github.com/ClickHouse/ClickHouse/pull/38703) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix FillingTransform [#38705](https://github.com/ClickHouse/ClickHouse/pull/38705) ([Nikita Taranov](https://github.com/nickitat)). +* Try to improve backward compatibility check [#38717](https://github.com/ClickHouse/ClickHouse/pull/38717) ([Kruglov Pavel](https://github.com/Avogar)). +* SQL create drop index fix formatting [#38720](https://github.com/ClickHouse/ClickHouse/pull/38720) ([Maksim Kita](https://github.com/kitaisreal)). +* Provide sort description for output stream in ReadFromMergeTree step [#38721](https://github.com/ClickHouse/ClickHouse/pull/38721) ([Igor Nikonov](https://github.com/devcrafter)). +* Add exp_internal for expect tests [#38728](https://github.com/ClickHouse/ClickHouse/pull/38728) ([Azat Khuzhin](https://github.com/azat)). +* Fix CLICKHOUSE_TMP in tests (fixes broken CI) [#38733](https://github.com/ClickHouse/ClickHouse/pull/38733) ([Azat Khuzhin](https://github.com/azat)). +* Add SimpleCheck [#38744](https://github.com/ClickHouse/ClickHouse/pull/38744) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Tiny tests cleanup [#38749](https://github.com/ClickHouse/ClickHouse/pull/38749) ([Azat Khuzhin](https://github.com/azat)). +* Fix replication after improper merge process [#38752](https://github.com/ClickHouse/ClickHouse/pull/38752) ([Raúl Marín](https://github.com/Algunenano)). +* tests: make aggregate_state_exception_memory_leak deterministic [#38754](https://github.com/ClickHouse/ClickHouse/pull/38754) ([Azat Khuzhin](https://github.com/azat)). +* Bump jemalloc to fix possible assertion [#38757](https://github.com/ClickHouse/ClickHouse/pull/38757) ([Azat Khuzhin](https://github.com/azat)). +* Reintroduce nonNegativeDerivative() [#38774](https://github.com/ClickHouse/ClickHouse/pull/38774) ([Andrey Zvonov](https://github.com/zvonand)). +* Temporarily disable 01710_projection_fetch_long in BC check [#38798](https://github.com/ClickHouse/ClickHouse/pull/38798) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Use native Map type for OpenTelemetry attributes [#38814](https://github.com/ClickHouse/ClickHouse/pull/38814) ([Ilya Yatsishin](https://github.com/qoega)). +* Add test for segfault in Map combinator [#38831](https://github.com/ClickHouse/ClickHouse/pull/38831) ([Kruglov Pavel](https://github.com/Avogar)). +* Update libprotobuf-mutator + fix build [#38834](https://github.com/ClickHouse/ClickHouse/pull/38834) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Get files changed in master since release is branched [#38836](https://github.com/ClickHouse/ClickHouse/pull/38836) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* update integration tests doc [#38837](https://github.com/ClickHouse/ClickHouse/pull/38837) ([Bharat Nallan](https://github.com/bharatnc)). +* Revert of revert of smallish devguide update [#38850](https://github.com/ClickHouse/ClickHouse/pull/38850) ([Robert Schulze](https://github.com/rschu1ze)). +* Do not override compiler if it had been already set [#38856](https://github.com/ClickHouse/ClickHouse/pull/38856) ([Azat Khuzhin](https://github.com/azat)). +* Move check for denied allocations [#38858](https://github.com/ClickHouse/ClickHouse/pull/38858) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Refactoring of code around object storages, added LocalObjectStorage (extracted this diff from PR [#36171](https://github.com/ClickHouse/ClickHouse/issues/36171)) [#38860](https://github.com/ClickHouse/ClickHouse/pull/38860) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backup Improvements 9 [#38861](https://github.com/ClickHouse/ClickHouse/pull/38861) ([Vitaly Baranov](https://github.com/vitlibar)). +* Simple cleanup: interpreters and parsers [#38876](https://github.com/ClickHouse/ClickHouse/pull/38876) ([Igor Nikonov](https://github.com/devcrafter)). +* Remove unnecessary log [#38892](https://github.com/ClickHouse/ClickHouse/pull/38892) ([Raúl Marín](https://github.com/Algunenano)). +* Update version_date.tsv and changelogs after v22.6.3.35-stable [#38894](https://github.com/ClickHouse/ClickHouse/pull/38894) ([github-actions[bot]](https://github.com/apps/github-actions)). +* Retry docker buildx commands with progressive sleep in between [#38898](https://github.com/ClickHouse/ClickHouse/pull/38898) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Avoid false positive red sanitizer asserts check in stress test [#38901](https://github.com/ClickHouse/ClickHouse/pull/38901) ([Kruglov Pavel](https://github.com/Avogar)). +* Interpreter cleanup: ContextPtr -> const ContextPtr & in parameters [#38902](https://github.com/ClickHouse/ClickHouse/pull/38902) ([Igor Nikonov](https://github.com/devcrafter)). +* Add a test for simdjson [#38933](https://github.com/ClickHouse/ClickHouse/pull/38933) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix a typo [#38938](https://github.com/ClickHouse/ClickHouse/pull/38938) ([Nikolay Degterinsky](https://github.com/evillique)). +* Avoid redundant join block transformation during planning. [#38943](https://github.com/ClickHouse/ClickHouse/pull/38943) ([Amos Bird](https://github.com/amosbird)). +* Rename NUMBER_OF_DIMENSIONS_MISMATHED const to NUMBER_OF_DIMENSIONS_MISMATCHED [#38947](https://github.com/ClickHouse/ClickHouse/pull/38947) ([Vladimir Galunshchikov](https://github.com/soyayaos)). +* More careful destructor in BackupImpl [#38949](https://github.com/ClickHouse/ClickHouse/pull/38949) ([Vitaly Baranov](https://github.com/vitlibar)). +* Avoid weird exception in Keeper [#38963](https://github.com/ClickHouse/ClickHouse/pull/38963) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update version_date.tsv after v22.3.8.39-lts [#38969](https://github.com/ClickHouse/ClickHouse/pull/38969) ([github-actions[bot]](https://github.com/apps/github-actions)). +* Remove tag no-backward-compatibility-check for specific versions [#38971](https://github.com/ClickHouse/ClickHouse/pull/38971) ([Kruglov Pavel](https://github.com/Avogar)). +* add Hetzner benchmark [#38974](https://github.com/ClickHouse/ClickHouse/pull/38974) ([Tyler Hannan](https://github.com/tylerhannan)). +* Update version_date.tsv after v22.4.6.53-stable [#38975](https://github.com/ClickHouse/ClickHouse/pull/38975) ([github-actions[bot]](https://github.com/apps/github-actions)). +* Disable instrumentation of sanitizer death callback [#38977](https://github.com/ClickHouse/ClickHouse/pull/38977) ([Alexander Tokmakov](https://github.com/tavplubix)). +* add ryzen 9 5950 benchmark [#38979](https://github.com/ClickHouse/ClickHouse/pull/38979) ([Tyler Hannan](https://github.com/tylerhannan)). +* EXPLAIN AST rewrite: rename to optimize [#38980](https://github.com/ClickHouse/ClickHouse/pull/38980) ([Igor Nikonov](https://github.com/devcrafter)). +* add macbook pro core i7 2014 benchmark [#38981](https://github.com/ClickHouse/ClickHouse/pull/38981) ([Tyler Hannan](https://github.com/tylerhannan)). +* add Huawei TaiShan 920 Benchmark [#38982](https://github.com/ClickHouse/ClickHouse/pull/38982) ([Tyler Hannan](https://github.com/tylerhannan)). +* tests: unique ZooKeeper path for Replicated.*MergeTree tables [#38999](https://github.com/ClickHouse/ClickHouse/pull/38999) ([Azat Khuzhin](https://github.com/azat)). +* Try another suppression for [#38629](https://github.com/ClickHouse/ClickHouse/issues/38629) [#39009](https://github.com/ClickHouse/ClickHouse/pull/39009) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add docker_server.py running to backport and release CIs [#39011](https://github.com/ClickHouse/ClickHouse/pull/39011) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix flaky `test_system_merges/test.py::test_mutation_simple` [#39013](https://github.com/ClickHouse/ClickHouse/pull/39013) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix assertion in full soring merge join [#39014](https://github.com/ClickHouse/ClickHouse/pull/39014) ([Vladimir C](https://github.com/vdimir)). +* Fix flaky 00620_optimize_on_nonleader_replica_zookeeper [#39019](https://github.com/ClickHouse/ClickHouse/pull/39019) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Suppress [#38643](https://github.com/ClickHouse/ClickHouse/issues/38643) [#39024](https://github.com/ClickHouse/ClickHouse/pull/39024) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update url.md [#39025](https://github.com/ClickHouse/ClickHouse/pull/39025) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix 'Tried to lock part ... for removal second time' [#39036](https://github.com/ClickHouse/ClickHouse/pull/39036) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add more settings for randomization [#39039](https://github.com/ClickHouse/ClickHouse/pull/39039) ([Anton Popov](https://github.com/CurtizJ)). +* add ScaleFlux CSD3000 Benchmark [#39040](https://github.com/ClickHouse/ClickHouse/pull/39040) ([Tyler Hannan](https://github.com/tylerhannan)). +* BACKUP/RESTORE ON CLUSTER use async mode on replicas now. [#39046](https://github.com/ClickHouse/ClickHouse/pull/39046) ([Vitaly Baranov](https://github.com/vitlibar)). +* More stable `test_s3_zero_copy_ttl`, weakened requirement to move data to S3 in 0-5 seconds [#39064](https://github.com/ClickHouse/ClickHouse/pull/39064) ([Vladimir Chebotaryov](https://github.com/quickhouse)). +* Parameter --decompressor added to utils/self-extracting-executable/compressor [#39065](https://github.com/ClickHouse/ClickHouse/pull/39065) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Follow-up to [#39036](https://github.com/ClickHouse/ClickHouse/issues/39036) [#39091](https://github.com/ClickHouse/ClickHouse/pull/39091) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update registerDiskS3.cpp [#39092](https://github.com/ClickHouse/ClickHouse/pull/39092) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix code in getLeastSupertype function [#39101](https://github.com/ClickHouse/ClickHouse/pull/39101) ([Kruglov Pavel](https://github.com/Avogar)). +* Remove some debug logging [#39102](https://github.com/ClickHouse/ClickHouse/pull/39102) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Prefix overridden add_executable() command with "clickhouse_" [#39108](https://github.com/ClickHouse/ClickHouse/pull/39108) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix meilisearch tests [#39110](https://github.com/ClickHouse/ClickHouse/pull/39110) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Properly remove projection from part in case it was removed from table metadata. [#39119](https://github.com/ClickHouse/ClickHouse/pull/39119) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Update cluster.py [#39120](https://github.com/ClickHouse/ClickHouse/pull/39120) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Tiny updates for tests. [#39127](https://github.com/ClickHouse/ClickHouse/pull/39127) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix data race in CompletedPipelineExecutor. [#39132](https://github.com/ClickHouse/ClickHouse/pull/39132) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix style again [#39133](https://github.com/ClickHouse/ClickHouse/pull/39133) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix path retrieval for Keeper's state [#39148](https://github.com/ClickHouse/ClickHouse/pull/39148) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Slightly better interface of waitForMutation [#39154](https://github.com/ClickHouse/ClickHouse/pull/39154) ([Amos Bird](https://github.com/amosbird)). +* ThreadPool fixes [#39160](https://github.com/ClickHouse/ClickHouse/pull/39160) ([Azat Khuzhin](https://github.com/azat)). +* Add test for [#39132](https://github.com/ClickHouse/ClickHouse/issues/39132) [#39173](https://github.com/ClickHouse/ClickHouse/pull/39173) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Suppression for BC check (`Cannot parse string 'Hello' as UInt64`) [#39176](https://github.com/ClickHouse/ClickHouse/pull/39176) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix 01961_roaring_memory_tracking test [#39187](https://github.com/ClickHouse/ClickHouse/pull/39187) ([Dmitry Novik](https://github.com/novikd)). +* Cleanup: done during [#38719](https://github.com/ClickHouse/ClickHouse/issues/38719) (SortingStep: deduce way to sort based on … [#39191](https://github.com/ClickHouse/ClickHouse/pull/39191) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix exception in AsynchronousMetrics for s390x [#39193](https://github.com/ClickHouse/ClickHouse/pull/39193) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Optimize accesses to system.stack_trace (filter by name before sending signal) [#39212](https://github.com/ClickHouse/ClickHouse/pull/39212) ([Azat Khuzhin](https://github.com/azat)). +* Enable warning "-Wdeprecated-dynamic-exception-spec" [#39213](https://github.com/ClickHouse/ClickHouse/pull/39213) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove specialization global lock/unlock from ActionLocksManager [#39215](https://github.com/ClickHouse/ClickHouse/pull/39215) ([Azat Khuzhin](https://github.com/azat)). +* Turn some warnings on [#39223](https://github.com/ClickHouse/ClickHouse/pull/39223) ([Robert Schulze](https://github.com/rschu1ze)). +* Pass const std::string_view by value, not by reference [#39224](https://github.com/ClickHouse/ClickHouse/pull/39224) ([Kruglov Pavel](https://github.com/Avogar)). +* Minor fix for BC check [#39231](https://github.com/ClickHouse/ClickHouse/pull/39231) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backport script [#39235](https://github.com/ClickHouse/ClickHouse/pull/39235) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Do not color logs on client if they are redirected to file [#39243](https://github.com/ClickHouse/ClickHouse/pull/39243) ([Anton Popov](https://github.com/CurtizJ)). +* Remove incorrect assertion [#39245](https://github.com/ClickHouse/ClickHouse/pull/39245) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add X86 prefix to x86 performance tests [#39251](https://github.com/ClickHouse/ClickHouse/pull/39251) ([Robert Schulze](https://github.com/rschu1ze)). +* Check that the destination for a backup is not in use. [#39254](https://github.com/ClickHouse/ClickHouse/pull/39254) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix stacktraces in gdb in BC check [#39256](https://github.com/ClickHouse/ClickHouse/pull/39256) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Disable flaky test `test_s3_zero_copy_on_hybrid_storage` [#39258](https://github.com/ClickHouse/ClickHouse/pull/39258) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Enabled Wc99-extensions + Wsign-conversion [#39261](https://github.com/ClickHouse/ClickHouse/pull/39261) ([Robert Schulze](https://github.com/rschu1ze)). +* Pass const StringRef by value, not by reference [#39262](https://github.com/ClickHouse/ClickHouse/pull/39262) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix assertion in transactions [#39263](https://github.com/ClickHouse/ClickHouse/pull/39263) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix macosx compilation due to endian.h [#39265](https://github.com/ClickHouse/ClickHouse/pull/39265) ([Jordi Villar](https://github.com/jrdi)). +* Another supression for BC check [#39276](https://github.com/ClickHouse/ClickHouse/pull/39276) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix builder image for releases w/o diagnostics tool [#39281](https://github.com/ClickHouse/ClickHouse/pull/39281) ([Azat Khuzhin](https://github.com/azat)). +* [RFC] Remove superior atomic from MergeTreeBackgroundExecutor and annotations for TSA [#39285](https://github.com/ClickHouse/ClickHouse/pull/39285) ([Azat Khuzhin](https://github.com/azat)). +* Fix clang tidy [#39288](https://github.com/ClickHouse/ClickHouse/pull/39288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix running cmake with predefined cache (for development only) [#39295](https://github.com/ClickHouse/ClickHouse/pull/39295) ([Azat Khuzhin](https://github.com/azat)). +* Fix googletest contrib compilation (due to GTEST_HAS_POSIX_RE=0) [#39298](https://github.com/ClickHouse/ClickHouse/pull/39298) ([Azat Khuzhin](https://github.com/azat)). +* First try at reducing the use of StringRef [#39300](https://github.com/ClickHouse/ClickHouse/pull/39300) ([Robert Schulze](https://github.com/rschu1ze)). +* Whitespaces [#39303](https://github.com/ClickHouse/ClickHouse/pull/39303) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add test for window function inside CASE [#39305](https://github.com/ClickHouse/ClickHouse/pull/39305) ([Dmitry Novik](https://github.com/novikd)). +* Simple Check should be updated on rerun [#39307](https://github.com/ClickHouse/ClickHouse/pull/39307) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix leaking of logger in clickhouse-disks [#39314](https://github.com/ClickHouse/ClickHouse/pull/39314) ([Azat Khuzhin](https://github.com/azat)). +* Update exception message [#39315](https://github.com/ClickHouse/ClickHouse/pull/39315) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix build clang-13 [#39318](https://github.com/ClickHouse/ClickHouse/pull/39318) ([alesapin](https://github.com/alesapin)). +* Auto set test name in integration tests [#39322](https://github.com/ClickHouse/ClickHouse/pull/39322) ([Vitaly Baranov](https://github.com/vitlibar)). +* Try fix flaky test_store_cleanup [#39334](https://github.com/ClickHouse/ClickHouse/pull/39334) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Do not start on unexpected Ordinary metadata [#39337](https://github.com/ClickHouse/ClickHouse/pull/39337) ([Alexander Tokmakov](https://github.com/tavplubix)). +* switch from mkdocs to Docusaurus [#39338](https://github.com/ClickHouse/ClickHouse/pull/39338) ([Dan Roscigno](https://github.com/DanRoscigno)). +* Fix flaky 01174_select_insert_isolation [#39339](https://github.com/ClickHouse/ClickHouse/pull/39339) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Better exception messages in schema inference [#39340](https://github.com/ClickHouse/ClickHouse/pull/39340) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix memory exceptions with transactions [#39341](https://github.com/ClickHouse/ClickHouse/pull/39341) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix typo [#39360](https://github.com/ClickHouse/ClickHouse/pull/39360) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix typo [#39361](https://github.com/ClickHouse/ClickHouse/pull/39361) ([Kruglov Pavel](https://github.com/Avogar)). +* Do not enqueue uneeded parts for check [#39366](https://github.com/ClickHouse/ClickHouse/pull/39366) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Avoid loading toolchain file multiple times to avoid confusing ccache [#39387](https://github.com/ClickHouse/ClickHouse/pull/39387) ([Azat Khuzhin](https://github.com/azat)). +* Fix make clean (due to crosscompile of llvm) [#39392](https://github.com/ClickHouse/ClickHouse/pull/39392) ([Azat Khuzhin](https://github.com/azat)). +* Disable real-time digest in Keeper by default [#39393](https://github.com/ClickHouse/ClickHouse/pull/39393) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix interactive client with older server [#39413](https://github.com/ClickHouse/ClickHouse/pull/39413) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix BC check [#39414](https://github.com/ClickHouse/ClickHouse/pull/39414) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix flaky test test_login_as_dropped_user_xml. [#39415](https://github.com/ClickHouse/ClickHouse/pull/39415) ([Vitaly Baranov](https://github.com/vitlibar)). +* Introduce a dependency to libuv when building NATS [#39427](https://github.com/ClickHouse/ClickHouse/pull/39427) ([ltrk2](https://github.com/ltrk2)). +* Set default value cross_to_inner_join_rewrite = 1 [#39443](https://github.com/ClickHouse/ClickHouse/pull/39443) ([Vladimir C](https://github.com/vdimir)). +* Respect table alias for additional_table_filters. [#39456](https://github.com/ClickHouse/ClickHouse/pull/39456) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### Performance optimization and Bug Fix + +* Enabled `pread_threadpool` read method by default. It will increase read performance. Bug fix: if direct IO is enabled and the number of threads is large and `pread_threadpool` is used, it may cause a logical error. [#33653](https://github.com/ClickHouse/ClickHouse/pull/33653) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index de9e4cfa0af..e4c7aae8b25 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v22.7.1.2484-stable 2022-07-21 v22.6.3.35-stable 2022-07-06 v22.6.2.12-stable 2022-06-29 v22.6.1.1985-stable 2022-06-16 From 739ba294662157f41cc077ce9943b23d47babf17 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 19:54:45 +0200 Subject: [PATCH 548/659] Run changelog.py from `github-helpers` branch --- docs/changelogs/v22.7.1.2484-stable.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/changelogs/v22.7.1.2484-stable.md b/docs/changelogs/v22.7.1.2484-stable.md index d306b1b5bd4..0343568658b 100644 --- a/docs/changelogs/v22.7.1.2484-stable.md +++ b/docs/changelogs/v22.7.1.2484-stable.md @@ -5,7 +5,7 @@ sidebar_label: 2022 # 2022 Changelog -### ClickHouse release v22.7.1.2484-stable FIXME as compared to v22.6.1.1985-stable +### ClickHouse release v22.7.1.2484-stable (f4f05ec786a) FIXME as compared to v22.6.1.1985-stable (7000c4e0033) #### Backward Incompatible Change * Enable setting `enable_positional_arguments` by default. It allows queries like `SELECT ... ORDER BY 1, 2` where 1, 2 are the references to the select clause. If you need to return the old behavior, disable this setting. [#38204](https://github.com/ClickHouse/ClickHouse/pull/38204) ([Alexey Milovidov](https://github.com/alexey-milovidov)). From ecd4c3138705e2b8c01739e7a958666a4e559e9f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Jul 2022 17:31:34 +0000 Subject: [PATCH 549/659] Even less usage of StringRef --> see #39300 --- src/Common/TraceSender.cpp | 10 ++-- .../Serializations/SerializationEnum.cpp | 6 +- .../Serializations/SerializationString.cpp | 6 +- src/Functions/extractGroups.cpp | 6 +- src/Functions/formatReadableTimeDelta.cpp | 8 +-- src/Functions/geohashDecode.cpp | 4 +- src/Functions/isIPAddressContainedIn.cpp | 12 ++-- src/IO/Operators.h | 2 - src/IO/WriteHelpers.h | 57 +++---------------- src/IO/tests/gtest_manip.cpp | 5 -- src/Processors/Merges/Algorithms/Graphite.cpp | 8 +-- src/Processors/Merges/Algorithms/Graphite.h | 3 +- .../GraphiteRollupSortedAlgorithm.cpp | 4 +- .../GraphiteRollupSortedAlgorithm.h | 2 +- src/Storages/Distributed/DirectoryMonitor.cpp | 1 - .../graphite-rollup/graphite-rollup-bench.cpp | 14 ++--- 16 files changed, 50 insertions(+), 98 deletions(-) diff --git a/src/Common/TraceSender.cpp b/src/Common/TraceSender.cpp index ce8adb98740..f1287b11ac4 100644 --- a/src/Common/TraceSender.cpp +++ b/src/Common/TraceSender.cpp @@ -42,13 +42,13 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int char buffer[buf_size]; WriteBufferFromFileDescriptorDiscardOnFailure out(pipe.fds_rw[1], buf_size, buffer); - StringRef query_id; + std::string_view query_id; UInt64 thread_id; if (CurrentThread::isInitialized()) { - query_id = StringRef(CurrentThread::getQueryId()); - query_id.size = std::min(query_id.size, QUERY_ID_MAX_LEN); + query_id = CurrentThread::getQueryId(); + query_id = std::string_view(query_id.data(), std::min(query_id.size(), QUERY_ID_MAX_LEN)); thread_id = CurrentThread::get().thread_id; } @@ -59,8 +59,8 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int writeChar(false, out); /// true if requested to stop the collecting thread. - writeBinary(static_cast(query_id.size), out); - out.write(query_id.data, query_id.size); + writeBinary(static_cast(query_id.size()), out); + out.write(query_id.data(), query_id.size()); size_t stack_trace_size = stack_trace.getSize(); size_t stack_trace_offset = stack_trace.getOffset(); diff --git a/src/DataTypes/Serializations/SerializationEnum.cpp b/src/DataTypes/Serializations/SerializationEnum.cpp index 39e9885fe17..a1b9c8bf95a 100644 --- a/src/DataTypes/Serializations/SerializationEnum.cpp +++ b/src/DataTypes/Serializations/SerializationEnum.cpp @@ -18,7 +18,7 @@ void SerializationEnum::serializeText(const IColumn & column, size_t row_n template void SerializationEnum::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeEscapedString(this->getNameForValue(assert_cast(column).getData()[row_num]), ostr); + writeEscapedString(this->getNameForValue(assert_cast(column).getData()[row_num]).toView(), ostr); } template @@ -69,13 +69,13 @@ void SerializationEnum::deserializeWholeText(IColumn & column, ReadBuffer template void SerializationEnum::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - writeJSONString(this->getNameForValue(assert_cast(column).getData()[row_num]), ostr, settings); + writeJSONString(this->getNameForValue(assert_cast(column).getData()[row_num]).toView(), ostr, settings); } template void SerializationEnum::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeXMLStringForTextElement(this->getNameForValue(assert_cast(column).getData()[row_num]), ostr); + writeXMLStringForTextElement(this->getNameForValue(assert_cast(column).getData()[row_num]).toView(), ostr); } template diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 5614e970315..e07fd4f26cf 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -213,7 +213,7 @@ void SerializationString::serializeText(const IColumn & column, size_t row_num, void SerializationString::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeEscapedString(assert_cast(column).getDataAt(row_num), ostr); + writeEscapedString(assert_cast(column).getDataAt(row_num).toView(), ostr); } @@ -266,7 +266,7 @@ void SerializationString::deserializeTextQuoted(IColumn & column, ReadBuffer & i void SerializationString::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - writeJSONString(assert_cast(column).getDataAt(row_num), ostr, settings); + writeJSONString(assert_cast(column).getDataAt(row_num).toView(), ostr, settings); } @@ -278,7 +278,7 @@ void SerializationString::deserializeTextJSON(IColumn & column, ReadBuffer & ist void SerializationString::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeXMLStringForTextElement(assert_cast(column).getDataAt(row_num), ostr); + writeXMLStringForTextElement(assert_cast(column).getDataAt(row_num).toView(), ostr); } diff --git a/src/Functions/extractGroups.cpp b/src/Functions/extractGroups.cpp index 940e76df1c0..c6633732aaa 100644 --- a/src/Functions/extractGroups.cpp +++ b/src/Functions/extractGroups.cpp @@ -87,10 +87,10 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { - StringRef current_row = column_haystack->getDataAt(i); + std::string_view current_row = column_haystack->getDataAt(i).toView(); - if (re2->Match(re2_st::StringPiece(current_row.data, current_row.size), - 0, current_row.size, re2_st::RE2::UNANCHORED, matched_groups.data(), matched_groups.size())) + if (re2->Match(re2_st::StringPiece(current_row.data(), current_row.size()), + 0, current_row.size(), re2_st::RE2::UNANCHORED, matched_groups.data(), matched_groups.size())) { // 1 is to exclude group #0 which is whole re match. for (size_t group = 1; group <= groups_count; ++group) diff --git a/src/Functions/formatReadableTimeDelta.cpp b/src/Functions/formatReadableTimeDelta.cpp index d781d227c64..8efd13a734a 100644 --- a/src/Functions/formatReadableTimeDelta.cpp +++ b/src/Functions/formatReadableTimeDelta.cpp @@ -94,19 +94,19 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - StringRef maximum_unit_str; + std::string_view maximum_unit_str; if (arguments.size() == 2) { const ColumnPtr & maximum_unit_column = arguments[1].column; const ColumnConst * maximum_unit_const_col = checkAndGetColumnConstStringOrFixedString(maximum_unit_column.get()); if (maximum_unit_const_col) - maximum_unit_str = maximum_unit_const_col->getDataColumn().getDataAt(0); + maximum_unit_str = maximum_unit_const_col->getDataColumn().getDataAt(0).toView(); } Unit max_unit; /// Default means "use all available units". - if (maximum_unit_str.size == 0 || maximum_unit_str == "years") + if (maximum_unit_str.empty() || maximum_unit_str == "years") max_unit = Years; else if (maximum_unit_str == "months") max_unit = Months; @@ -122,7 +122,7 @@ public: throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected value of maximum unit argument ({}) for function {}, the only allowed values are:" " 'seconds', 'minutes', 'hours', 'days', 'months', 'years'.", - maximum_unit_str.toString(), getName()); + std::string(maximum_unit_str), getName()); auto col_to = ColumnString::create(); diff --git a/src/Functions/geohashDecode.cpp b/src/Functions/geohashDecode.cpp index 199d1a62f1d..b4e5d8e46e9 100644 --- a/src/Functions/geohashDecode.cpp +++ b/src/Functions/geohashDecode.cpp @@ -64,8 +64,8 @@ public: for (size_t i = 0; i < count; ++i) { - StringRef encoded_string = encoded->getDataAt(i); - geohashDecode(encoded_string.data, encoded_string.size, &lon_data[i], &lat_data[i]); + std::string_view encoded_string = encoded->getDataAt(i).toView(); + geohashDecode(encoded_string.data(), encoded_string.size(), &lon_data[i], &lat_data[i]); } MutableColumns result; diff --git a/src/Functions/isIPAddressContainedIn.cpp b/src/Functions/isIPAddressContainedIn.cpp index 1ba719cda63..a6f94c77ad1 100644 --- a/src/Functions/isIPAddressContainedIn.cpp +++ b/src/Functions/isIPAddressContainedIn.cpp @@ -27,7 +27,7 @@ class IPAddressVariant { public: - explicit IPAddressVariant(StringRef address_str) + explicit IPAddressVariant(std::string_view address_str) { /// IP address parser functions require that the input is /// NULL-terminated so we need to copy it. @@ -85,7 +85,7 @@ IPAddressCIDR parseIPWithCIDR(std::string_view cidr_str) throw DB::Exception("The text does not contain '/': " + std::string(cidr_str), DB::ErrorCodes::CANNOT_PARSE_TEXT); std::string_view addr_str = cidr_str.substr(0, pos_slash); - IPAddressVariant addr(StringRef{addr_str.data(), addr_str.size()}); + IPAddressVariant addr(addr_str); uint8_t prefix = 0; auto prefix_str = cidr_str.substr(pos_slash+1); @@ -188,7 +188,7 @@ namespace DB const auto & col_addr = col_addr_const.getDataColumn(); const auto & col_cidr = col_cidr_const.getDataColumn(); - const auto addr = IPAddressVariant(col_addr.getDataAt(0)); + const auto addr = IPAddressVariant(col_addr.getDataAt(0).toView()); const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(0).toView()); ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(1); @@ -204,7 +204,7 @@ namespace DB { const auto & col_addr = col_addr_const.getDataColumn(); - const auto addr = IPAddressVariant(col_addr.getDataAt (0)); + const auto addr = IPAddressVariant(col_addr.getDataAt(0).toView()); ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); ColumnUInt8::Container & vec_res = col_res->getData(); @@ -228,7 +228,7 @@ namespace DB ColumnUInt8::Container & vec_res = col_res->getData(); for (size_t i = 0; i < input_rows_count; ++i) { - const auto addr = IPAddressVariant(col_addr.getDataAt(i)); + const auto addr = IPAddressVariant(col_addr.getDataAt(i).toView()); vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; } return col_res; @@ -242,7 +242,7 @@ namespace DB for (size_t i = 0; i < input_rows_count; ++i) { - const auto addr = IPAddressVariant(col_addr.getDataAt(i)); + const auto addr = IPAddressVariant(col_addr.getDataAt(i).toView()); const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i).toView()); vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; diff --git a/src/IO/Operators.h b/src/IO/Operators.h index 114ab692dc3..93807ea39fc 100644 --- a/src/IO/Operators.h +++ b/src/IO/Operators.h @@ -54,9 +54,7 @@ template WriteBuffer & operator<< (QuoteManipWriteBuffer buf, template WriteBuffer & operator<< (DoubleQuoteManipWriteBuffer buf, const T & x) { writeDoubleQuoted(x, buf.get()); return buf; } template WriteBuffer & operator<< (BinaryManipWriteBuffer buf, const T & x) { writeBinary(x, buf.get()); return buf; } -inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const String & x) { writeEscapedString(x, buf); return buf; } inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, std::string_view x) { writeEscapedString(x, buf); return buf; } -inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, StringRef x) { writeEscapedString(x, buf); return buf; } inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const char * x) { writeEscapedString(x, strlen(x), buf); return buf; } inline WriteBuffer & operator<< (QuoteManipWriteBuffer buf, const char * x) { writeAnyQuotedString<'\''>(x, x + strlen(x), buf.get()); return buf; } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 6f35dae8300..2903a70b61a 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -360,19 +360,9 @@ void writeAnyEscapedString(const char * begin, const char * end, WriteBuffer & b } -inline void writeJSONString(StringRef s, WriteBuffer & buf, const FormatSettings & settings) -{ - writeJSONString(s.data, s.data + s.size, buf, settings); -} - inline void writeJSONString(std::string_view s, WriteBuffer & buf, const FormatSettings & settings) { - writeJSONString(StringRef{s}, buf, settings); -} - -inline void writeJSONString(const String & s, WriteBuffer & buf, const FormatSettings & settings) -{ - writeJSONString(StringRef{s}, buf, settings); + writeJSONString(s.data(), s.data() + s.size(), buf, settings); } template @@ -417,7 +407,7 @@ void writeJSONNumber(T x, WriteBuffer & ostr, const FormatSettings & settings) template -void writeAnyEscapedString(const String & s, WriteBuffer & buf) +void writeAnyEscapedString(std::string_view s, WriteBuffer & buf) { writeAnyEscapedString(s.data(), s.data() + s.size(), buf); } @@ -428,18 +418,6 @@ inline void writeEscapedString(const char * str, size_t size, WriteBuffer & buf) writeAnyEscapedString<'\''>(str, str + size, buf); } - -inline void writeEscapedString(const String & s, WriteBuffer & buf) -{ - writeEscapedString(s.data(), s.size(), buf); -} - - -inline void writeEscapedString(StringRef ref, WriteBuffer & buf) -{ - writeEscapedString(ref.data, ref.size, buf); -} - inline void writeEscapedString(std::string_view ref, WriteBuffer & buf) { writeEscapedString(ref.data(), ref.size(), buf); @@ -455,16 +433,9 @@ void writeAnyQuotedString(const char * begin, const char * end, WriteBuffer & bu template -void writeAnyQuotedString(const String & s, WriteBuffer & buf) +void writeAnyQuotedString(std::string_view ref, WriteBuffer & buf) { - writeAnyQuotedString(s.data(), s.data() + s.size(), buf); -} - - -template -void writeAnyQuotedString(StringRef ref, WriteBuffer & buf) -{ - writeAnyQuotedString(ref.data, ref.data + ref.size, buf); + writeAnyQuotedString(ref.data(), ref.data() + ref.size(), buf); } @@ -475,7 +446,7 @@ inline void writeQuotedString(const String & s, WriteBuffer & buf) inline void writeQuotedString(StringRef ref, WriteBuffer & buf) { - writeAnyQuotedString<'\''>(ref, buf); + writeAnyQuotedString<'\''>(ref.toView(), buf); } inline void writeQuotedString(std::string_view ref, WriteBuffer & buf) @@ -490,7 +461,7 @@ inline void writeDoubleQuotedString(const String & s, WriteBuffer & buf) inline void writeDoubleQuotedString(StringRef s, WriteBuffer & buf) { - writeAnyQuotedString<'"'>(s, buf); + writeAnyQuotedString<'"'>(s.toView(), buf); } inline void writeDoubleQuotedString(std::string_view s, WriteBuffer & buf) @@ -501,7 +472,7 @@ inline void writeDoubleQuotedString(std::string_view s, WriteBuffer & buf) /// Outputs a string in backquotes. inline void writeBackQuotedString(StringRef s, WriteBuffer & buf) { - writeAnyQuotedString<'`'>(s, buf); + writeAnyQuotedString<'`'>(s.toView(), buf); } /// Outputs a string in backquotes for MySQL. @@ -611,16 +582,11 @@ inline void writeXMLStringForTextElementOrAttributeValue(const char * begin, con } } -inline void writeXMLStringForTextElementOrAttributeValue(const String & s, WriteBuffer & buf) +inline void writeXMLStringForTextElementOrAttributeValue(std::string_view s, WriteBuffer & buf) { writeXMLStringForTextElementOrAttributeValue(s.data(), s.data() + s.size(), buf); } -inline void writeXMLStringForTextElementOrAttributeValue(StringRef s, WriteBuffer & buf) -{ - writeXMLStringForTextElementOrAttributeValue(s.data, s.data + s.size, buf); -} - /// Writing a string to a text node in XML (not into an attribute - otherwise you need more escaping). inline void writeXMLStringForTextElement(const char * begin, const char * end, WriteBuffer & buf) { @@ -652,16 +618,11 @@ inline void writeXMLStringForTextElement(const char * begin, const char * end, W } } -inline void writeXMLStringForTextElement(const String & s, WriteBuffer & buf) +inline void writeXMLStringForTextElement(std::string_view s, WriteBuffer & buf) { writeXMLStringForTextElement(s.data(), s.data() + s.size(), buf); } -inline void writeXMLStringForTextElement(StringRef s, WriteBuffer & buf) -{ - writeXMLStringForTextElement(s.data, s.data + s.size, buf); -} - template void formatHex(IteratorSrc src, IteratorDst dst, size_t num_bytes); void formatUUID(const UInt8 * src16, UInt8 * dst36); diff --git a/src/IO/tests/gtest_manip.cpp b/src/IO/tests/gtest_manip.cpp index afcffc7dfb9..09cc3d16619 100644 --- a/src/IO/tests/gtest_manip.cpp +++ b/src/IO/tests/gtest_manip.cpp @@ -32,11 +32,6 @@ TEST(OperatorsManipTest, EscapingTest) checkString(sv1, escape, "Hello \\'world\\'"); std::string_view sv2 = s2; checkString(sv2, escape, "Hello \\\\world\\\\"); // NOLINT - - StringRef sr1 = s1; - checkString(sr1, escape, "Hello \\'world\\'"); - StringRef sr2 = s2; - checkString(sr2, escape, "Hello \\\\world\\\\"); // NOLINT } TEST(OperatorsManipTest, QuouteTest) diff --git a/src/Processors/Merges/Algorithms/Graphite.cpp b/src/Processors/Merges/Algorithms/Graphite.cpp index 6c4ca5ef85b..2448a1e2a94 100644 --- a/src/Processors/Merges/Algorithms/Graphite.cpp +++ b/src/Processors/Merges/Algorithms/Graphite.cpp @@ -71,11 +71,11 @@ static const Graphite::Pattern undef_pattern = .type = undef_pattern.TypeUndef, }; -inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, StringRef path) +inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, std::string_view path) { if (params.patterns_typed) { - std::string_view path_view = path.toView(); + std::string_view path_view = path; if (path_view.find("?"sv) == path_view.npos) return params.patterns_plain; else @@ -89,7 +89,7 @@ inline static const Patterns & selectPatternsForMetricType(const Graphite::Param Graphite::RollupRule selectPatternForPath( const Graphite::Params & params, - StringRef path) + std::string_view path) { const Graphite::Pattern * first_match = &undef_pattern; @@ -119,7 +119,7 @@ Graphite::RollupRule selectPatternForPath( } else { - if (pattern.regexp->match(path.data, path.size)) + if (pattern.regexp->match(path.data(), path.size())) { /// General pattern with matched path if (pattern.type == pattern.TypeAll) diff --git a/src/Processors/Merges/Algorithms/Graphite.h b/src/Processors/Merges/Algorithms/Graphite.h index 05306ebe30f..46b1bbbfcad 100644 --- a/src/Processors/Merges/Algorithms/Graphite.h +++ b/src/Processors/Merges/Algorithms/Graphite.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include @@ -147,7 +146,7 @@ struct Params using RollupRule = std::pair; -Graphite::RollupRule selectPatternForPath(const Graphite::Params & params, StringRef path); +Graphite::RollupRule selectPatternForPath(const Graphite::Params & params, std::string_view path); void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params); diff --git a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp index eff62d73f50..467ded19f4d 100644 --- a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp @@ -120,7 +120,7 @@ IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge() return Status(current.impl->order); } - StringRef next_path = current->all_columns[columns_definition.path_column_num]->getDataAt(current->getRow()); + std::string_view next_path = current->all_columns[columns_definition.path_column_num]->getDataAt(current->getRow()).toView(); bool new_path = is_first || next_path != current_group_path; is_first = false; @@ -190,7 +190,7 @@ IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge() current_subgroup_newest_row.set(current, sources[current.impl->order].chunk); /// Small hack: group and subgroups have the same path, so we can set current_group_path here instead of startNextGroup - /// But since we keep in memory current_subgroup_newest_row's block, we could use StringRef for current_group_path and don't + /// But since we keep in memory current_subgroup_newest_row's block, we could use string_view for current_group_path and don't /// make deep copy of the path. current_group_path = next_path; } diff --git a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h index 4968cbfc470..d6d2f66fb82 100644 --- a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h @@ -92,7 +92,7 @@ private: */ /// Path name of current bucket - StringRef current_group_path; + std::string_view current_group_path; static constexpr size_t max_row_refs = 2; /// current_subgroup_newest_row, current_row. /// Last row with maximum version for current primary key (time bucket). diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 5e9da48fc68..bf2638f7bc3 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include diff --git a/utils/graphite-rollup/graphite-rollup-bench.cpp b/utils/graphite-rollup/graphite-rollup-bench.cpp index 4c11f90b3ff..49a3d509be6 100644 --- a/utils/graphite-rollup/graphite-rollup-bench.cpp +++ b/utils/graphite-rollup/graphite-rollup-bench.cpp @@ -20,9 +20,9 @@ using namespace DB; static SharedContextHolder shared_context = Context::createShared(); -std::vector loadMetrics(const std::string & metrics_file) +std::vector loadMetrics(const std::string & metrics_file) { - std::vector metrics; + std::vector metrics; FILE * stream; char * line = nullptr; @@ -47,7 +47,7 @@ std::vector loadMetrics(const std::string & metrics_file) } if (l > 0) { - metrics.push_back(StringRef(strdup(line), l)); + metrics.emplace_back(std::string_view(strdup(line), l)); } } } @@ -80,7 +80,7 @@ void bench(const std::string & config_path, const std::string & metrics_file, si Graphite::Params params; setGraphitePatternsFromConfig(context, "graphite_rollup", params); - std::vector metrics = loadMetrics(metrics_file); + std::vector metrics = loadMetrics(metrics_file); std::vector durations(metrics.size()); size_t j, i; @@ -99,15 +99,15 @@ void bench(const std::string & config_path, const std::string & metrics_file, si if (j == 0 && verbose) { - std::cout << metrics[i].data << ": rule with regexp '" << rule.second->regexp_str << "' found\n"; + std::cout << metrics[i].data() << ": rule with regexp '" << rule.second->regexp_str << "' found\n"; } } } for (i = 0; i < metrics.size(); i++) { - std::cout << metrics[i].data << " " << durations[i] / n << " ns\n"; - free(const_cast(static_cast(metrics[i].data))); + std::cout << metrics[i].data() << " " << durations[i] / n << " ns\n"; + free(const_cast(static_cast(metrics[i].data()))); } } From a21f45ec170ab2e4c73f7f11cae7e3dc272c0de3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 19 Jul 2022 13:17:33 +0200 Subject: [PATCH 550/659] Update src/Functions/formatReadableTimeDelta.cpp Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- src/Functions/formatReadableTimeDelta.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/formatReadableTimeDelta.cpp b/src/Functions/formatReadableTimeDelta.cpp index 8efd13a734a..219c2d95353 100644 --- a/src/Functions/formatReadableTimeDelta.cpp +++ b/src/Functions/formatReadableTimeDelta.cpp @@ -122,7 +122,7 @@ public: throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected value of maximum unit argument ({}) for function {}, the only allowed values are:" " 'seconds', 'minutes', 'hours', 'days', 'months', 'years'.", - std::string(maximum_unit_str), getName()); + maximum_unit_str, getName()); auto col_to = ColumnString::create(); From af48474297b3d6b7f4a6f65cc4c011fb1ac931ac Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 19 Jul 2022 18:44:31 +0200 Subject: [PATCH 551/659] Update src/Common/TraceSender.cpp Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- src/Common/TraceSender.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/TraceSender.cpp b/src/Common/TraceSender.cpp index f1287b11ac4..ad88e508d06 100644 --- a/src/Common/TraceSender.cpp +++ b/src/Common/TraceSender.cpp @@ -48,7 +48,8 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int if (CurrentThread::isInitialized()) { query_id = CurrentThread::getQueryId(); - query_id = std::string_view(query_id.data(), std::min(query_id.size(), QUERY_ID_MAX_LEN)); + if (query_id.size() > QUERY_ID_MAX_LEN) + query_id.remove_suffix(query_id.size() - QUERY_ID_MAX_LEN); thread_id = CurrentThread::get().thread_id; } From cae9786aa8a41f5b0ec65f47372b2458a8644425 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 20 Jul 2022 07:59:02 +0000 Subject: [PATCH 552/659] Fix test failures --- src/IO/Operators.h | 2 ++ src/IO/tests/gtest_manip.cpp | 5 +++++ 2 files changed, 7 insertions(+) diff --git a/src/IO/Operators.h b/src/IO/Operators.h index 93807ea39fc..06ff20c43e8 100644 --- a/src/IO/Operators.h +++ b/src/IO/Operators.h @@ -54,7 +54,9 @@ template WriteBuffer & operator<< (QuoteManipWriteBuffer buf, template WriteBuffer & operator<< (DoubleQuoteManipWriteBuffer buf, const T & x) { writeDoubleQuoted(x, buf.get()); return buf; } template WriteBuffer & operator<< (BinaryManipWriteBuffer buf, const T & x) { writeBinary(x, buf.get()); return buf; } +inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const String & x) { writeEscapedString(x, buf); return buf; } inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, std::string_view x) { writeEscapedString(x, buf); return buf; } +inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, StringRef x) { writeEscapedString(x.toView(), buf); return buf; } inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const char * x) { writeEscapedString(x, strlen(x), buf); return buf; } inline WriteBuffer & operator<< (QuoteManipWriteBuffer buf, const char * x) { writeAnyQuotedString<'\''>(x, x + strlen(x), buf.get()); return buf; } diff --git a/src/IO/tests/gtest_manip.cpp b/src/IO/tests/gtest_manip.cpp index 09cc3d16619..afcffc7dfb9 100644 --- a/src/IO/tests/gtest_manip.cpp +++ b/src/IO/tests/gtest_manip.cpp @@ -32,6 +32,11 @@ TEST(OperatorsManipTest, EscapingTest) checkString(sv1, escape, "Hello \\'world\\'"); std::string_view sv2 = s2; checkString(sv2, escape, "Hello \\\\world\\\\"); // NOLINT + + StringRef sr1 = s1; + checkString(sr1, escape, "Hello \\'world\\'"); + StringRef sr2 = s2; + checkString(sr2, escape, "Hello \\\\world\\\\"); // NOLINT } TEST(OperatorsManipTest, QuouteTest) From 4874b4a6e7fec4362e7d1b38b68d2dc01b5927d8 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Jul 2022 14:24:11 +0000 Subject: [PATCH 553/659] Allow CREATE TEMPORARY TABLE ... (list of columns) AS ... --- src/Parsers/ParserCreateQuery.cpp | 2 +- ...02371_create_temporary_table_as_with_columns_list.reference | 2 ++ .../02371_create_temporary_table_as_with_columns_list.sql | 3 +++ 3 files changed, 6 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.reference create mode 100644 tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.sql diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index ce79ccf708a..275f3bc75cc 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -584,7 +584,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe auto storage_parse_result = storage_p.parse(pos, storage, expected); - if (storage_parse_result && need_parse_as_select()) + if ((storage_parse_result || is_temporary) && need_parse_as_select()) { if (!select_p.parse(pos, select, expected)) return false; diff --git a/tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.reference b/tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.reference new file mode 100644 index 00000000000..6fc56adcb1c --- /dev/null +++ b/tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.reference @@ -0,0 +1,2 @@ +Vasya +Petya diff --git a/tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.sql b/tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.sql new file mode 100644 index 00000000000..7d8f297b505 --- /dev/null +++ b/tests/queries/0_stateless/02371_create_temporary_table_as_with_columns_list.sql @@ -0,0 +1,3 @@ +CREATE TEMPORARY TABLE test_02327 (name String) AS SELECT * FROM VALUES(('Vasya'), ('Petya')); +SELECT * FROM test_02327; +DROP TABLE test_02327; From 9d7a72c8bc42f1dc2d2214660f9078b46d0587c1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Jul 2022 15:05:48 +0200 Subject: [PATCH 554/659] remove some dead and commented code --- src/Access/LDAPClient.cpp | 1 - src/Client/ClientBase.cpp | 1 - src/Columns/ColumnLowCardinality.cpp | 10 --- src/Columns/ColumnNullable.cpp | 10 --- src/Columns/ColumnUnique.h | 1 - src/Common/ArrayCache.h | 2 - src/Common/DateLUTImpl.cpp | 3 - src/Common/HashTable/SmallTable.h | 67 ------------------- src/Common/RadixSort.h | 2 - .../SerializationLowCardinality.cpp | 7 -- .../SerializationLowCardinality.h | 3 - src/Dictionaries/SSDCacheDictionaryStorage.h | 2 - src/Dictionaries/XDBCDictionarySource.cpp | 2 - src/Disks/DiskWebServer.cpp | 1 - src/Functions/CustomWeekTransforms.h | 2 - src/Functions/FunctionsConversion.h | 2 - src/IO/CascadeWriteBuffer.cpp | 2 - src/Interpreters/ActionsDAG.cpp | 10 --- src/Interpreters/HashJoin.cpp | 1 - .../InterpreterSelectWithUnionQuery.cpp | 1 - src/Interpreters/JIT/CHJIT.cpp | 22 ------ src/Processors/Executors/PipelineExecutor.cpp | 1 - src/Processors/Formats/Impl/NativeFormat.cpp | 9 --- src/Processors/Sources/RemoteSource.cpp | 2 - src/QueryPipeline/QueryPipelineBuilder.cpp | 1 - src/Storages/Hive/HiveCommon.cpp | 1 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 16 ----- .../MergeTree/IMergedBlockOutputStream.h | 3 - src/Storages/MergeTree/MergeTask.cpp | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 5 -- 30 files changed, 191 deletions(-) diff --git a/src/Access/LDAPClient.cpp b/src/Access/LDAPClient.cpp index 3486be1de33..ff1ee6f3609 100644 --- a/src/Access/LDAPClient.cpp +++ b/src/Access/LDAPClient.cpp @@ -509,7 +509,6 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params) if (referrals) { SCOPE_EXIT({ -// ldap_value_free(referrals); ber_memvfree(reinterpret_cast(referrals)); referrals = nullptr; }); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index f0a8794d096..c6f14c7e865 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -152,7 +152,6 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) auto & dst_column_host_name = typeid_cast(*mutable_columns[name_pos["host_name"]]); auto & dst_array_current_time = typeid_cast(*mutable_columns[name_pos["current_time"]]).getData(); - // auto & dst_array_thread_id = typeid_cast(*mutable_columns[name_pos["thread_id"]]).getData(); auto & dst_array_type = typeid_cast(*mutable_columns[name_pos["type"]]).getData(); auto & dst_column_name = typeid_cast(*mutable_columns[name_pos["name"]]); auto & dst_array_value = typeid_cast(*mutable_columns[name_pos["value"]]).getData(); diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 62fb69a47e1..17e9bd97669 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -132,14 +132,12 @@ namespace ColumnLowCardinality::ColumnLowCardinality(MutableColumnPtr && column_unique_, MutableColumnPtr && indexes_, bool is_shared) : dictionary(std::move(column_unique_), is_shared), idx(std::move(indexes_)) { - // idx.check(getDictionary().size()); } void ColumnLowCardinality::insert(const Field & x) { compactIfSharedDictionary(); idx.insertPosition(dictionary.getColumnUnique().uniqueInsert(x)); - // idx.check(getDictionary().size()); } void ColumnLowCardinality::insertDefault() @@ -167,15 +165,12 @@ void ColumnLowCardinality::insertFrom(const IColumn & src, size_t n) const auto & nested = *low_cardinality_src->getDictionary().getNestedColumn(); idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(nested, position)); } - - // idx.check(getDictionary().size()); } void ColumnLowCardinality::insertFromFullColumn(const IColumn & src, size_t n) { compactIfSharedDictionary(); idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(src, n)); - // idx.check(getDictionary().size()); } void ColumnLowCardinality::insertRangeFrom(const IColumn & src, size_t start, size_t length) @@ -205,7 +200,6 @@ void ColumnLowCardinality::insertRangeFrom(const IColumn & src, size_t start, si auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(*used_keys, 0, used_keys->size()); idx.insertPositionsRange(*inserted_indexes->index(*sub_idx, 0), 0, length); } - // idx.check(getDictionary().size()); } void ColumnLowCardinality::insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length) @@ -213,7 +207,6 @@ void ColumnLowCardinality::insertRangeFromFullColumn(const IColumn & src, size_t compactIfSharedDictionary(); auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(src, start, length); idx.insertPositionsRange(*inserted_indexes, 0, length); - // idx.check(getDictionary().size()); } static void checkPositionsAreLimited(const IColumn & positions, UInt64 limit) @@ -254,14 +247,12 @@ void ColumnLowCardinality::insertRangeFromDictionaryEncodedColumn(const IColumn compactIfSharedDictionary(); auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(keys, 0, keys.size()); idx.insertPositionsRange(*inserted_indexes->index(positions, 0), 0, positions.size()); - // idx.check(getDictionary().size()); } void ColumnLowCardinality::insertData(const char * pos, size_t length) { compactIfSharedDictionary(); idx.insertPosition(dictionary.getColumnUnique().uniqueInsertData(pos, length)); - // idx.check(getDictionary().size()); } StringRef ColumnLowCardinality::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const @@ -276,7 +267,6 @@ const char * ColumnLowCardinality::deserializeAndInsertFromArena(const char * po const char * new_pos; idx.insertPosition(dictionary.getColumnUnique().uniqueDeserializeAndInsertFromArena(pos, new_pos)); - // idx.check(getDictionary().size()); return new_pos; } diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 8d61f6e726a..809024316bf 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -273,14 +273,6 @@ llvm::Value * ColumnNullable::compileComparator(llvm::IRBuilderBase & builder, l b.CreateCondBr(lhs_or_rhs_are_null, lhs_or_rhs_are_null_block, lhs_rhs_are_not_null_block); - // if (unlikely(lval_is_null || rval_is_null)) - // { - // if (lval_is_null && rval_is_null) - // return 0; - // else - // return lval_is_null ? null_direction_hint : -null_direction_hint; - // } - b.SetInsertPoint(lhs_or_rhs_are_null_block); auto * lhs_equals_rhs_result = llvm::ConstantInt::getSigned(b.getInt8Ty(), 0); llvm::Value * lhs_and_rhs_are_null = b.CreateAnd(lhs_is_null_value, rhs_is_null_value); @@ -288,8 +280,6 @@ llvm::Value * ColumnNullable::compileComparator(llvm::IRBuilderBase & builder, l llvm::Value * lhs_or_rhs_are_null_block_result = b.CreateSelect(lhs_and_rhs_are_null, lhs_equals_rhs_result, lhs_is_null_result); b.CreateBr(join_block); - // getNestedColumn().compareAt(n, m, nested_rhs, null_direction_hint); - b.SetInsertPoint(lhs_rhs_are_not_null_block); llvm::Value * lhs_rhs_are_not_null_block_result = nested_column->compileComparator(builder, lhs_unwrapped_value, rhs_unwrapped_value, nan_direction_hint); diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 3c21a65e404..58891e30e12 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -548,7 +548,6 @@ MutableColumnPtr ColumnUnique::uniqueInsertRangeImpl( } } - // checkIndexes(*positions_column, column->size() + (overflowed_keys ? overflowed_keys->size() : 0)); return std::move(positions_column); } diff --git a/src/Common/ArrayCache.h b/src/Common/ArrayCache.h index 6efa5c92b5b..f01ff94e38b 100644 --- a/src/Common/ArrayCache.h +++ b/src/Common/ArrayCache.h @@ -514,8 +514,6 @@ private: return allocateFromFreeRegion(*free_region, size); } -// std::cerr << "Requested size: " << size << "\n"; - /// Evict something from cache and continue. while (true) { diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index 869954bb2ae..31290c53b49 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -122,9 +122,6 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) values.time_at_offset_change_value = (transition.from - cctz::civil_second(date)) / Values::OffsetChangeFactor; values.amount_of_offset_change_value = (transition.to - transition.from) / Values::OffsetChangeFactor; -// std::cerr << time_zone << ", " << date << ": change from " << transition.from << " to " << transition.to << "\n"; -// std::cerr << time_zone << ", " << date << ": change at " << values.time_at_offset_change() << " with " << values.amount_of_offset_change() << "\n"; - /// We don't support too large changes. if (values.amount_of_offset_change_value > 24 * 4) values.amount_of_offset_change_value = 24 * 4; diff --git a/src/Common/HashTable/SmallTable.h b/src/Common/HashTable/SmallTable.h index ad9537ff94a..b78901b03f6 100644 --- a/src/Common/HashTable/SmallTable.h +++ b/src/Common/HashTable/SmallTable.h @@ -74,7 +74,6 @@ public: using key_type = Key; using mapped_type = typename Cell::mapped_type; using value_type = typename Cell::value_type; - using cell_type = Cell; class Reader final : private Cell::State { @@ -247,39 +246,6 @@ public: } } - - /// Same, but return false if it's full. - bool ALWAYS_INLINE tryEmplace(Key x, iterator & it, bool & inserted) - { - Cell * res = findCell(x); - it = iteratorTo(res); - inserted = res == buf + m_size; - if (inserted) - { - if (res == buf + capacity) - return false; - - new(res) Cell(x, *this); - ++m_size; - } - return true; - } - - - /// Copy the cell from another hash table. It is assumed that there was no such key in the table yet. - void ALWAYS_INLINE insertUnique(const Cell * cell) - { - memcpy(&buf[m_size], cell, sizeof(*cell)); - ++m_size; - } - - void ALWAYS_INLINE insertUnique(Key x) - { - new(&buf[m_size]) Cell(x, *this); - ++m_size; - } - - iterator ALWAYS_INLINE find(Key x) { return iteratorTo(findCell(x)); } const_iterator ALWAYS_INLINE find(Key x) const { return iteratorTo(findCell(x)); } @@ -381,36 +347,3 @@ template > using SmallSet = SmallTable, capacity>; - -template -< - typename Key, - typename Cell, - size_t capacity -> -class SmallMapTable : public SmallTable -{ -public: - using key_type = Key; - using mapped_type = typename Cell::mapped_type; - using value_type = typename Cell::value_type; - using cell_type = Cell; - - mapped_type & ALWAYS_INLINE operator[](Key x) - { - typename SmallMapTable::iterator it; - bool inserted; - this->emplace(x, it, inserted); - new (&it->getMapped()) mapped_type(); - return it->getMapped(); - } -}; - - -template -< - typename Key, - typename Mapped, - size_t capacity -> -using SmallMap = SmallMapTable, capacity>; diff --git a/src/Common/RadixSort.h b/src/Common/RadixSort.h index 4bf975c4c7a..9ca43bee30c 100644 --- a/src/Common/RadixSort.h +++ b/src/Common/RadixSort.h @@ -355,8 +355,6 @@ private: template static inline void radixSortMSDInternal(Element * arr, size_t size, size_t limit) { -// std::cerr << PASS << ", " << size << ", " << limit << "\n"; - /// The beginning of every i-1-th bucket. 0th element will be equal to 1st. /// Last element will point to array end. std::unique_ptr prev_buckets{new Element*[HISTOGRAM_SIZE + 1]}; diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index c79f588e46c..8e19c5a740b 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -511,8 +511,6 @@ void SerializationLowCardinality::serializeBinaryBulkWithMultipleStreams( /// Insert used_keys into global dictionary and update sub_index. auto indexes_with_overflow = global_dictionary->uniqueInsertRangeWithOverflow(*keys, 0, keys->size(), settings.low_cardinality_max_dictionary_size); - // size_t max_size = settings.low_cardinality_max_dictionary_size + indexes_with_overflow.overflowed_keys->size(); - // ColumnLowCardinality::Index(indexes_with_overflow.indexes->getPtr()).check(max_size); if (global_dictionary->size() > settings.low_cardinality_max_dictionary_size) throw Exception("Got dictionary with size " + toString(global_dictionary->size()) + @@ -656,11 +654,6 @@ void SerializationLowCardinality::deserializeBinaryBulkWithMultipleStreams( { auto maps = mapIndexWithAdditionalKeys(*indexes_column, global_dictionary->size()); - // ColumnLowCardinality::Index(maps.additional_keys_map->getPtr()).check(additional_keys->size()); - - // ColumnLowCardinality::Index(indexes_column->getPtr()).check( - // maps.dictionary_map->size() + maps.additional_keys_map->size()); - auto used_keys = IColumn::mutate(global_dictionary->getNestedColumn()->index(*maps.dictionary_map, 0)); if (!maps.additional_keys_map->empty()) diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.h b/src/DataTypes/Serializations/SerializationLowCardinality.h index 0a3597e86c7..96e3a297d6a 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.h +++ b/src/DataTypes/Serializations/SerializationLowCardinality.h @@ -78,9 +78,6 @@ private: template void deserializeImpl(IColumn & column, DeserializeFunctionPtr func, Args &&... args) const; - - // template - // static MutableColumnUniquePtr createColumnUniqueImpl(const IDataType & keys_type, const Creator & creator); }; } diff --git a/src/Dictionaries/SSDCacheDictionaryStorage.h b/src/Dictionaries/SSDCacheDictionaryStorage.h index d813cf1bcc8..22a46016b2c 100644 --- a/src/Dictionaries/SSDCacheDictionaryStorage.h +++ b/src/Dictionaries/SSDCacheDictionaryStorage.h @@ -527,8 +527,6 @@ public: throw Exception(ErrorCodes::CANNOT_IO_SUBMIT, "Cannot submit request for asynchronous IO on file {}", file_path); } - // CurrentMetrics::Increment metric_increment_write{CurrentMetrics::Write}; - io_event event; while (io_getevents(aio_context.ctx, 1, 1, &event, nullptr) < 0) diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 5e9c2f7ac7a..0a097c4faef 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -275,8 +275,6 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory) bool /* created_from_ddl */) -> DictionarySourcePtr { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Dictionary source of type `jdbc` is disabled until consistent support for nullable fields."); - // BridgeHelperPtr bridge = std::make_shared>(config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".connection_string")); - // return std::make_unique(dict_struct, config, config_prefix + ".jdbc", sample_block, context, bridge); }; factory.registerSource("jdbc", create_table_source); } diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index 54dce926893..b6cda8288d7 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -74,7 +74,6 @@ void DiskWebServer::initialize(const String & uri_path) const if (file_data.type == FileType::Directory) { directories_to_load.push_back(file_path); - // file_path = fs::path(file_path) / ""; } file_path = file_path.substr(url.size()); diff --git a/src/Functions/CustomWeekTransforms.h b/src/Functions/CustomWeekTransforms.h index 8656f9da927..c296c8228b1 100644 --- a/src/Functions/CustomWeekTransforms.h +++ b/src/Functions/CustomWeekTransforms.h @@ -63,12 +63,10 @@ struct ToStartOfWeekImpl static inline UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); -// return time_zone.toFirstDayNumOfWeek(t, week_mode); } static inline UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); -// return time_zone.toFirstDayNumOfWeek(t, week_mode); } static inline UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) { diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index e0c42401207..b666602e366 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1091,8 +1091,6 @@ struct ConvertThroughParsing static constexpr bool to_datetime64 = std::is_same_v; - // using ToFieldType = typename ToDataType::FieldType; - static bool isAllRead(ReadBuffer & in) { /// In case of FixedString, skip zero bytes at end. diff --git a/src/IO/CascadeWriteBuffer.cpp b/src/IO/CascadeWriteBuffer.cpp index 616fbe9b789..ca11290c71b 100644 --- a/src/IO/CascadeWriteBuffer.cpp +++ b/src/IO/CascadeWriteBuffer.cpp @@ -50,8 +50,6 @@ void CascadeWriteBuffer::nextImpl() } set(curr_buffer->position(), curr_buffer->buffer().end() - curr_buffer->position()); -// std::cerr << "CascadeWriteBuffer a count=" << count() << " bytes=" << bytes << " offset=" << offset() -// << " bytes+size=" << bytes + buffer().size() << "\n"; } diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 2703773f464..b91fd7ac5cf 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -448,17 +448,7 @@ static ColumnWithTypeAndName executeActionForHeader(const ActionsDAG::Node * nod { case ActionsDAG::ActionType::FUNCTION: { - // bool all_args_are_const = true; - - // for (const auto & argument : arguments) - // if (typeid_cast(argument.column.get()) == nullptr) - // all_args_are_const = false; - res_column.column = node->function->execute(arguments, res_column.type, 0, true); - - // if (!all_args_are_const) - // res_column.column = res_column.column->convertToFullColumnIfConst(); - break; } diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index b54c77b385f..722ba81451a 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -2100,7 +2100,6 @@ std::shared_ptr HashJoin::getNonJoinedBlocks(const Block & left if (multiple_disjuncts) { /// ... calculate `left_columns_count` ... - // throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "TODO"); size_t left_columns_count = left_sample_block.columns(); auto non_joined = std::make_unique>(*this, max_block_size); return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index bdec44b74f7..b2622607760 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -280,7 +280,6 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_, void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) { - // auto num_distinct_union = optimizeUnionList(); size_t num_plans = nested_interpreters.size(); const Settings & settings = context->getSettingsRef(); diff --git a/src/Interpreters/JIT/CHJIT.cpp b/src/Interpreters/JIT/CHJIT.cpp index 9eec82b4179..c2f3fc7c27d 100644 --- a/src/Interpreters/JIT/CHJIT.cpp +++ b/src/Interpreters/JIT/CHJIT.cpp @@ -244,28 +244,6 @@ private: } }; -// class AssemblyPrinter -// { -// public: - -// explicit AssemblyPrinter(llvm::TargetMachine &target_machine_) -// : target_machine(target_machine_) -// { -// } - -// void print(llvm::Module & module) -// { -// llvm::legacy::PassManager pass_manager; -// target_machine.Options.MCOptions.AsmVerbose = true; -// if (target_machine.addPassesToEmitFile(pass_manager, llvm::errs(), nullptr, llvm::CodeGenFileType::CGFT_AssemblyFile)) -// throw Exception(ErrorCodes::CANNOT_COMPILE_CODE, "MachineCode cannot be printed"); - -// pass_manager.run(module); -// } -// private: -// llvm::TargetMachine & target_machine; -// }; - /** MemoryManager for module. * Keep total allocated size during RuntimeDyld linker execution. */ diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 68225d73ff1..29c57e08573 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -205,7 +205,6 @@ void PipelineExecutor::executeStepImpl(size_t thread_num, std::atomic_bool * yie Stopwatch total_time_watch; #endif - // auto & node = tasks.getNode(thread_num); auto & context = tasks.getThreadContext(thread_num); bool yield = false; diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index 423fd483712..a8e2ddf95e4 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -74,15 +74,6 @@ protected: if (chunk) { auto block = getPort(PortKind::Main).getHeader(); - - // const auto & info = chunk.getChunkInfo(); - // const auto * agg_info = typeid_cast(info.get()); - // if (agg_info) - // { - // block.info.bucket_num = agg_info->bucket_num; - // block.info.is_overflows = agg_info->is_overflows; - // } - block.setColumns(chunk.detachColumns()); writer.write(block); } diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 9b01e048391..9f29ad9ad07 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -126,7 +126,6 @@ void RemoteSource::onCancel() { was_query_canceled = true; query_executor->cancel(&read_context); - // is_async_state = false; } void RemoteSource::onUpdatePorts() @@ -135,7 +134,6 @@ void RemoteSource::onUpdatePorts() { was_query_canceled = true; query_executor->finish(&read_context); - // is_async_state = false; } } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 88a52defa1e..340b85efae9 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -323,7 +323,6 @@ QueryPipelineBuilderPtr QueryPipelineBuilder::mergePipelines( left->pipe.processors.emplace_back(transform); left->pipe.processors.insert(left->pipe.processors.end(), right->pipe.processors.begin(), right->pipe.processors.end()); - // left->pipe.holder = std::move(right->pipe.holder); left->pipe.header = left->pipe.output_ports.front()->getHeader(); left->pipe.max_parallel_streams = std::max(left->pipe.max_parallel_streams, right->pipe.max_parallel_streams); return left; diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 7b2f04f7073..609adcf65c9 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -66,7 +66,6 @@ HiveMetastoreClient::HiveTableMetadataPtr HiveMetastoreClient::getTableMetadata( }; tryCallHiveClient(client_call); - // bool update_cache = shouldUpdateTableMetadata(db_name, table_name, partitions); String cache_key = getCacheKey(db_name, table_name); HiveTableMetadataPtr metadata = table_metadata_cache.get(cache_key); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 60941108f00..95f25aa1955 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -603,22 +603,6 @@ String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize( return *minimum_size_column; } -// String IMergeTreeDataPart::getFullPath() const -// { -// if (relative_path.empty()) -// throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); - -// return fs::path(storage.getFullPathOnDisk(volume->getDisk())) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; -// } - -// String IMergeTreeDataPart::getRelativePath() const -// { -// if (relative_path.empty()) -// throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); - -// return fs::path(storage.relative_data_path) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; -// } - void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency) { assertOnDisk(); diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index 3b94b85607a..dbcca1443b5 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -30,9 +30,6 @@ public: } protected: - // using SerializationState = ISerialization::SerializeBinaryBulkStatePtr; - - // ISerialization::OutputStreamGetter createStreamGetter(const String & name, WrittenOffsetColumns & offset_columns); /// Remove all columns marked expired in data_part. Also, clears checksums /// and columns array. Return set of removed files names. diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 7426b384394..dc468174dfa 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -596,7 +596,6 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c const auto & projections = global_ctx->metadata_snapshot->getProjections(); - // tasks_for_projections.reserve(projections.size()); for (const auto & projection : projections) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1bc4c26e40e..219093e8d75 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1584,8 +1584,6 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) return true; /// NOTE Deletion from `virtual_parts` is not done, but it is only necessary for merge. } - // bool do_fetch = false; - switch (entry.type) { case LogEntry::ATTACH_PART: @@ -1593,7 +1591,6 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) [[fallthrough]]; case LogEntry::GET_PART: return executeFetch(entry); - // do_fetch = true; case LogEntry::MERGE_PARTS: throw Exception(ErrorCodes::LOGICAL_ERROR, "Merge has to be executed by another function"); case LogEntry::MUTATE_PART: @@ -1609,8 +1606,6 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected log entry type: {}", static_cast(entry.type)); } - - // return true; } From 0cbbe7ac766160c2a296c38341c976a63e45fcd6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 22 Jul 2022 12:54:20 +0300 Subject: [PATCH 555/659] Update SSDCacheDictionaryStorage.h --- src/Dictionaries/SSDCacheDictionaryStorage.h | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Dictionaries/SSDCacheDictionaryStorage.h b/src/Dictionaries/SSDCacheDictionaryStorage.h index 22a46016b2c..459c4c44668 100644 --- a/src/Dictionaries/SSDCacheDictionaryStorage.h +++ b/src/Dictionaries/SSDCacheDictionaryStorage.h @@ -27,11 +27,6 @@ #include -namespace CurrentMetrics -{ - extern const Metric Write; -} - namespace ProfileEvents { extern const Event FileOpen; From 654047dfbf3feb3045fadd29550bfd4ca2b79773 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 22 Jul 2022 11:28:05 +0000 Subject: [PATCH 556/659] Fix compiler warning --- src/Compression/CompressedReadBufferBase.cpp | 2 +- src/Compression/CompressedReadBufferBase.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 3de7b9cc688..2c85dc6d9a9 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -308,7 +308,7 @@ void CompressedReadBufferBase::flushAsynchronousDecompressRequests() const codec->flushAsynchronousDecompressRequests(); } -void CompressedReadBufferBase::setDecompressMode(ICompressionCodec::CodecMode mode) +void CompressedReadBufferBase::setDecompressMode(ICompressionCodec::CodecMode mode) const { if (codec) codec->setDecompressMode(mode); diff --git a/src/Compression/CompressedReadBufferBase.h b/src/Compression/CompressedReadBufferBase.h index 4d76fe72fdc..baea4d2b855 100644 --- a/src/Compression/CompressedReadBufferBase.h +++ b/src/Compression/CompressedReadBufferBase.h @@ -63,7 +63,7 @@ protected: /// Set decompression mode: Synchronous/Asynchronous/SoftwareFallback. /// The mode is "Synchronous" by default. /// flushAsynchronousDecompressRequests must be called subsequently once set "Asynchronous" mode. - void setDecompressMode(ICompressionCodec::CodecMode mode); + void setDecompressMode(ICompressionCodec::CodecMode mode) const; public: /// 'compressed_in' could be initialized lazily, but before first call of 'readCompressedData'. From c6d0ca2940b04a22a5f7b34fda03e05e09e13ecd Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 22 Jul 2022 14:23:05 +0000 Subject: [PATCH 557/659] Fix data race in Avro format --- contrib/avro | 2 +- .../02372_data_race_in_avro.reference | 0 .../0_stateless/02372_data_race_in_avro.sh | 24 +++++++++++++++++++ 3 files changed, 25 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02372_data_race_in_avro.reference create mode 100755 tests/queries/0_stateless/02372_data_race_in_avro.sh diff --git a/contrib/avro b/contrib/avro index e43c46e87fd..aac4e605f07 160000 --- a/contrib/avro +++ b/contrib/avro @@ -1 +1 @@ -Subproject commit e43c46e87fd32eafdc09471e95344555454c5ef8 +Subproject commit aac4e605f070f2abd9a5f02ae70c17e4011588e8 diff --git a/tests/queries/0_stateless/02372_data_race_in_avro.reference b/tests/queries/0_stateless/02372_data_race_in_avro.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02372_data_race_in_avro.sh b/tests/queries/0_stateless/02372_data_race_in_avro.sh new file mode 100755 index 00000000000..d469e95fa78 --- /dev/null +++ b/tests/queries/0_stateless/02372_data_race_in_avro.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +for i in $(seq 1 10); +do + $CLICKHOUSE_CLIENT -q "drop table if exists t_avro_$i" + $CLICKHOUSE_CLIENT -q "create table t_avro_$i (x UInt32, s String) engine=File(Avro)" +done + +for i in $(seq 1 10); +do + $CLICKHOUSE_CLIENT -q "insert into t_avro_$i select number, 'str' from numbers(1000) settings engine_file_truncate_on_insert=1" > /dev/null & +done + +sleep 5 + +for i in $(seq 1 10); +do + $CLICKHOUSE_CLIENT-q "drop table t_avro_$i" +done + From e016f73b318f3676b1e60621200e9bdaf4742ef1 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Fri, 22 Jul 2022 11:42:14 -0400 Subject: [PATCH 558/659] update Grafana plugin --- docs/en/interfaces/third-party/gui.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/third-party/gui.md b/docs/en/interfaces/third-party/gui.md index 7f1cd3a411e..c5cc8c7c34b 100644 --- a/docs/en/interfaces/third-party/gui.md +++ b/docs/en/interfaces/third-party/gui.md @@ -67,7 +67,7 @@ Features: ### Grafana {#grafana} -[Grafana](https://grafana.com/grafana/plugins/vertamedia-clickhouse-datasource) is a platform for monitoring and visualization. +[Grafana](https://grafana.com/grafana/plugins/grafana-clickhouse-datasource/) is a platform for monitoring and visualization. "Grafana allows you to query, visualize, alert on and understand your metrics no matter where they are stored. Create, explore, and share dashboards with your team and foster a data driven culture. Trusted and loved by the community" — grafana.com. From 30a32371eba31a41c4cbcc3327d6c96024328cf6 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Fri, 22 Jul 2022 11:50:59 -0400 Subject: [PATCH 559/659] Remove experimental status from projections --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 3e5a0635339..e216a99f986 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -483,8 +483,6 @@ For example: ## Projections {#projections} Projections are like [materialized views](../../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. -Projections are an experimental feature. To enable them you must set the [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) to `1`. See also the [force_optimize_projection](../../../operations/settings/settings.md#force-optimize-projection) setting. - Projections are not supported in the `SELECT` statements with the [FINAL](../../../sql-reference/statements/select/from.md#select-from-final) modifier. ### Projection Query {#projection-query} From 6174fe1d7274e77df9540dd1e7b436738a3716c1 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 22 Jul 2022 10:08:37 +0200 Subject: [PATCH 560/659] Fix tests. --- src/Backups/BackupCoordinationStageSync.cpp | 4 +- src/Backups/BackupsWorker.cpp | 10 ++--- src/Backups/BackupsWorker.h | 4 +- src/Backups/RestorerFromBackup.cpp | 2 +- .../test_backup_restore_new/test.py | 45 +++++++++++-------- .../test_backup_restore_on_cluster/test.py | 40 ++++++++++++----- .../test_concurrency.py | 20 +++++++-- 7 files changed, 80 insertions(+), 45 deletions(-) diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp index 5524029bbf2..4b94e474345 100644 --- a/src/Backups/BackupCoordinationStageSync.cpp +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -15,7 +15,7 @@ namespace DB namespace ErrorCodes { extern const int FAILED_TO_SYNC_BACKUP_OR_RESTORE; -} +} BackupCoordinationStageSync::BackupCoordinationStageSync(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, Poco::Logger * log_) @@ -42,7 +42,7 @@ void BackupCoordinationStageSync::set(const String & current_host, const String auto code = zookeeper->tryCreate(alive_node_path, "", zkutil::CreateMode::Ephemeral); if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNODEEXISTS) throw zkutil::KeeperException(code, alive_node_path); - + zookeeper->createIfNotExists(zookeeper_path + "/started|" + current_host, ""); zookeeper->createIfNotExists(zookeeper_path + "/current|" + current_host + "|" + new_stage, message); } diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index df501258db6..b0d3fb55f2a 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -488,10 +488,10 @@ void BackupsWorker::addInfo(const UUID & uuid, bool internal, const String & bac info.status = status; info.status_changed_time = time(nullptr); info.internal = internal; - + std::lock_guard lock{infos_mutex}; infos[{uuid, internal}] = std::move(info); - + num_active_backups += getNumActiveBackupsChange(status); num_active_restores += getNumActiveRestoresChange(status); } @@ -503,7 +503,7 @@ void BackupsWorker::setStatus(const UUID & uuid, bool internal, BackupStatus sta auto it = infos.find({uuid, internal}); if (it == infos.end()) return; - + auto & info = it->second; auto old_status = info.status; info.status = status; @@ -552,10 +552,10 @@ void BackupsWorker::shutdown() bool has_active_backups_or_restores = (num_active_backups || num_active_restores); if (has_active_backups_or_restores) LOG_INFO(log, "Waiting for {} backups and {} restores to be finished", num_active_backups, num_active_restores); - + backups_thread_pool.wait(); restores_thread_pool.wait(); - + if (has_active_backups_or_restores) LOG_INFO(log, "All backup and restore tasks have finished"); } diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index f471dc822e0..7ae69271d26 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -59,13 +59,13 @@ public: private: std::pair startMakingBackup(const ASTPtr & query, const ContextPtr & context); - + void doBackup(const UUID & backup_uuid, const std::shared_ptr & backup_query, BackupSettings backup_settings, const BackupInfo & backup_info, std::shared_ptr backup_coordination, const ContextPtr & context, ContextMutablePtr mutable_context, bool called_async); std::pair startRestoring(const ASTPtr & query, ContextMutablePtr context); - + void doRestore(const UUID & restore_uuid, const std::shared_ptr & restore_query, RestoreSettings restore_settings, const BackupInfo & backup_info, std::shared_ptr restore_coordination, ContextMutablePtr context, bool called_async); diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 5e43d59ae56..3a2f5273611 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -154,7 +154,7 @@ void RestorerFromBackup::setStage(const String & new_stage, const String & messa { LOG_TRACE(log, "{}", toUpperFirst(new_stage)); current_stage = new_stage; - + if (restore_coordination) { restore_coordination->setStage(restore_settings.host_id, new_stage, message); diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index e490634e552..47f7c47d608 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -304,11 +304,13 @@ def test_async(): [id, _, status] = instance.query( f"BACKUP TABLE test.table TO {backup_name} ASYNC" ).split("\t") + assert status == "MAKING_BACKUP\n" or status == "BACKUP_COMPLETE\n" + assert_eq_with_retry( instance, - f"SELECT status FROM system.backups WHERE uuid='{id}'", - "BACKUP_COMPLETE\n", + f"SELECT status, error FROM system.backups WHERE uuid='{id}'", + TSV([["BACKUP_COMPLETE", ""]]), ) instance.query("DROP TABLE test.table") @@ -316,9 +318,13 @@ def test_async(): [id, _, status] = instance.query( f"RESTORE TABLE test.table FROM {backup_name} ASYNC" ).split("\t") + assert status == "RESTORING\n" or status == "RESTORED\n" + assert_eq_with_retry( - instance, f"SELECT status FROM system.backups WHERE uuid='{id}'", "RESTORED\n" + instance, + f"SELECT status, error FROM system.backups WHERE uuid='{id}'", + TSV([["RESTORED", ""]]), ) assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" @@ -341,14 +347,13 @@ def test_async_backups_to_same_destination(interface): assert_eq_with_retry( instance, - f"SELECT count() FROM system.backups WHERE uuid IN ['{id1}', '{id2}'] AND status != 'BACKUP_COMPLETE' AND status != 'FAILED_TO_BACKUP'", - "0\n", + f"SELECT status FROM system.backups WHERE uuid IN ['{id1}', '{id2}'] AND status == 'MAKING_BACKUP'", + "", ) - assert ( - instance.query(f"SELECT status FROM system.backups WHERE uuid='{id1}'") - == "BACKUP_COMPLETE\n" - ) + assert instance.query( + f"SELECT status, error FROM system.backups WHERE uuid='{id1}'" + ) == TSV([["BACKUP_COMPLETE", ""]]) assert ( instance.query(f"SELECT status FROM system.backups WHERE uuid='{id2}'") @@ -747,24 +752,26 @@ def test_system_users_async(): instance.query("CREATE USER u1 IDENTIFIED BY 'qwe123' SETTINGS custom_c = 3") backup_name = new_backup_name() - [id, _, status] = instance.query( + id = instance.query( f"BACKUP DATABASE default, TABLE system.users, TABLE system.roles, TABLE system.settings_profiles, TABLE system.row_policies, TABLE system.quotas TO {backup_name} ASYNC" - ).split("\t") + ).split("\t")[0] + assert_eq_with_retry( instance, - f"SELECT status FROM system.backups WHERE uuid='{id}'", - "BACKUP_COMPLETE\n", + f"SELECT status, error FROM system.backups WHERE uuid='{id}'", + TSV([["BACKUP_COMPLETE", ""]]), ) instance.query("DROP USER u1") - [id, _, status] = instance.query( + id = instance.query( f"RESTORE DATABASE default, TABLE system.users, TABLE system.roles, TABLE system.settings_profiles, TABLE system.row_policies, TABLE system.quotas FROM {backup_name} ASYNC" - ).split("\t") + ).split("\t")[0] + assert_eq_with_retry( instance, - f"SELECT status FROM system.backups WHERE uuid='{id}'", - "RESTORED\n", + f"SELECT status, error FROM system.backups WHERE uuid='{id}'", + TSV([["RESTORED", ""]]), ) assert ( @@ -889,8 +896,8 @@ def test_mutation(): ) instance.query("ALTER TABLE test.table UPDATE x=x+1 WHERE 1") - instance.query("ALTER TABLE test.table UPDATE x=x+1+sleep(1) WHERE 1") - instance.query("ALTER TABLE test.table UPDATE x=x+1+sleep(2) WHERE 1") + instance.query("ALTER TABLE test.table UPDATE x=x+1+sleep(3) WHERE 1") + instance.query("ALTER TABLE test.table UPDATE x=x+1+sleep(3) WHERE 1") backup_name = new_backup_name() instance.query(f"BACKUP TABLE test.table TO {backup_name}") diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index df03ebd320b..58fac12f041 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -404,8 +404,8 @@ def test_replicated_database_async(): assert_eq_with_retry( node1, - f"SELECT status FROM system.backups WHERE uuid='{id}'", - "BACKUP_COMPLETE\n", + f"SELECT status, error FROM system.backups WHERE uuid='{id}' AND NOT internal", + TSV([["BACKUP_COMPLETE", ""]]), ) node1.query("DROP DATABASE mydb ON CLUSTER 'cluster' NO DELAY") @@ -417,7 +417,9 @@ def test_replicated_database_async(): assert status == "RESTORING\n" or status == "RESTORED\n" assert_eq_with_retry( - node1, f"SELECT status FROM system.backups WHERE uuid='{id}'", "RESTORED\n" + node1, + f"SELECT status, error FROM system.backups WHERE uuid='{id}' AND NOT internal", + TSV([["RESTORED", ""]]), ) node1.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' mydb.tbl") @@ -460,8 +462,8 @@ def test_async_backups_to_same_destination(interface, on_cluster): for i in range(len(nodes)): assert_eq_with_retry( nodes[i], - f"SELECT count() FROM system.backups WHERE uuid='{ids[i]}' AND status != 'BACKUP_COMPLETE' AND status != 'FAILED_TO_BACKUP'", - "0\n", + f"SELECT status FROM system.backups WHERE uuid='{ids[i]}' AND status == 'MAKING_BACKUP'", + "", ) num_completed_backups = sum( @@ -469,7 +471,7 @@ def test_async_backups_to_same_destination(interface, on_cluster): int( nodes[i] .query( - f"SELECT count() FROM system.backups WHERE uuid='{ids[i]}' AND status == 'BACKUP_COMPLETE'" + f"SELECT count() FROM system.backups WHERE uuid='{ids[i]}' AND status == 'BACKUP_COMPLETE' AND NOT internal" ) .strip() ) @@ -477,7 +479,16 @@ def test_async_backups_to_same_destination(interface, on_cluster): ] ) + if num_completed_backups != 1: + for i in range(len(nodes)): + print( + nodes[i].query( + f"SELECT status, error FROM system.backups WHERE uuid='{ids[i]}' AND NOT internal" + ) + ) + assert num_completed_backups == 1 + node1.query("DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY") node1.query(f"RESTORE TABLE tbl FROM {backup_name}") assert node1.query("SELECT * FROM tbl") == "1\n" @@ -752,8 +763,8 @@ def test_mutation(): node1.query("INSERT INTO tbl SELECT number, toString(number) FROM numbers(10, 5)") node1.query("ALTER TABLE tbl UPDATE x=x+1 WHERE 1") - node1.query("ALTER TABLE tbl UPDATE x=x+1+sleep(1) WHERE 1") - node1.query("ALTER TABLE tbl UPDATE x=x+1+sleep(2) WHERE 1") + node1.query("ALTER TABLE tbl UPDATE x=x+1+sleep(3) WHERE 1") + node1.query("ALTER TABLE tbl UPDATE x=x+1+sleep(3) WHERE 1") backup_name = new_backup_name() node1.query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}") @@ -783,7 +794,7 @@ def test_get_error_from_other_host(): @pytest.mark.parametrize("kill", [False, True]) -def test_stop_other_host_while_backup(kill): +def test_stop_other_host_during_backup(kill): node1.query( "CREATE TABLE tbl ON CLUSTER 'cluster' (" "x UInt8" @@ -806,11 +817,14 @@ def test_stop_other_host_while_backup(kill): assert_eq_with_retry( node1, - f"SELECT status FROM system.backups WHERE uuid='{id}' AND status == 'MAKING_BACKUP'", + f"SELECT status FROM system.backups WHERE uuid='{id}' AND status == 'MAKING_BACKUP' AND NOT internal", "", + retry_count=100, ) - status = node1.query(f"SELECT status FROM system.backups WHERE uuid='{id}'").strip() + status = node1.query( + f"SELECT status FROM system.backups WHERE uuid='{id}' AND NOT internal" + ).strip() if kill: assert status in ["BACKUP_COMPLETE", "FAILED_TO_BACKUP"] @@ -824,4 +838,6 @@ def test_stop_other_host_while_backup(kill): node1.query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}") assert node1.query("SELECT * FROM tbl ORDER BY x") == TSV([3, 5]) elif status == "FAILED_TO_BACKUP": - assert not os.path.exists(get_path_to_backup(backup_name)) + assert not os.path.exists( + os.path.join(get_path_to_backup(backup_name), ".backup") + ) diff --git a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py index d399723c8b1..ee26f08f14e 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py @@ -121,10 +121,14 @@ def test_concurrent_backups_on_same_node(): assert_eq_with_retry( node0, - f"SELECT status, error FROM system.backups WHERE status != 'BACKUP_COMPLETE' AND status != 'FAILED_TO_BACKUP' AND uuid IN {ids_list}", + f"SELECT status FROM system.backups WHERE status == 'MAKING_BACKUP' AND uuid IN {ids_list}", "", ) + assert node0.query( + f"SELECT status, error FROM system.backups WHERE uuid IN {ids_list} AND NOT internal" + ) == TSV([["BACKUP_COMPLETE", ""]] * num_concurrent_backups) + for backup_name in backup_names: node0.query(f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY") node0.query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}") @@ -136,6 +140,7 @@ def test_concurrent_backups_on_same_node(): def test_concurrent_backups_on_different_nodes(): create_and_fill_table() + assert num_concurrent_backups <= num_nodes backup_names = [new_backup_name() for _ in range(num_concurrent_backups)] ids = [] @@ -150,13 +155,13 @@ def test_concurrent_backups_on_different_nodes(): for i in range(num_concurrent_backups): assert_eq_with_retry( nodes[i], - f"SELECT status, error FROM system.backups WHERE status != 'BACKUP_COMPLETE' AND status != 'FAILED_TO_BACKUP' AND uuid = '{ids[i]}'", + f"SELECT status FROM system.backups WHERE status == 'MAKING_BACKUP' AND uuid = '{ids[i]}'", "", ) for i in range(num_concurrent_backups): assert nodes[i].query( - f"SELECT status, error FROM system.backups WHERE uuid = '{ids[i]}'" + f"SELECT status, error FROM system.backups WHERE uuid = '{ids[i]}' AND NOT internal" ) == TSV([["BACKUP_COMPLETE", ""]]) for i in range(num_concurrent_backups): @@ -231,7 +236,14 @@ def test_create_or_drop_tables_during_backup(): for node in nodes: assert_eq_with_retry( node, - f"SELECT status, error from system.backups WHERE uuid IN {ids_list} AND (status == 'MAKING_BACKUP')", + f"SELECT status from system.backups WHERE uuid IN {ids_list} AND (status == 'MAKING_BACKUP')", + "", + ) + + for node in nodes: + assert_eq_with_retry( + node, + f"SELECT status, error from system.backups WHERE uuid IN {ids_list} AND (status == 'FAILED_TO_BACKUP')", "", ) From d23da91fe7eee79a6e89634ef782fdaea4eee365 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 22 Jul 2022 18:41:31 +0200 Subject: [PATCH 561/659] Add tag --- tests/queries/0_stateless/02372_data_race_in_avro.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02372_data_race_in_avro.sh b/tests/queries/0_stateless/02372_data_race_in_avro.sh index d469e95fa78..49c34e31923 100755 --- a/tests/queries/0_stateless/02372_data_race_in_avro.sh +++ b/tests/queries/0_stateless/02372_data_race_in_avro.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From c1a5514e46edcb40156f4a8e22f63867e8aa451a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 22 Jul 2022 16:57:45 +0000 Subject: [PATCH 562/659] fix reading from async read buffer with direct io --- .../AsynchronousReadIndirectBufferFromRemoteFS.cpp | 12 +++++++++--- src/IO/AsynchronousReadBufferFromFileDescriptor.cpp | 10 ++++++++-- .../HDFS/AsynchronousReadBufferFromHDFS.cpp | 13 +++++++++---- .../1_stateful/00024_random_counters.reference | 2 ++ tests/queries/1_stateful/00024_random_counters.sql | 9 +++++++++ 5 files changed, 37 insertions(+), 9 deletions(-) diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index 774a7ecaaaa..f58e91669c4 100644 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -168,6 +168,8 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() CurrentMetrics::Increment metric_increment{CurrentMetrics::AsynchronousReadWait}; size_t size = 0; + size_t bytes_read = 0; + if (prefetch_future.valid()) { ProfileEvents::increment(ProfileEvents::RemoteFSPrefetchedReads); @@ -181,6 +183,8 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() /// If prefetch_future is valid, size should always be greater than zero. assert(offset <= size); + bytes_read = size - offset; + ProfileEvents::increment(ProfileEvents::AsynchronousReadWaitMicroseconds, watch.elapsedMicroseconds()); } @@ -200,9 +204,11 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() auto offset = result.offset; LOG_TEST(log, "Current size: {}, offset: {}", size, offset); - assert(offset <= size); - if (size) + assert(offset <= size); + bytes_read = size - offset; + + if (bytes_read) { /// Adjust the working buffer so that it ignores `offset` bytes. internal_buffer = Buffer(memory.data(), memory.data() + memory.size()); @@ -222,7 +228,7 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() assert(file_offset_of_buffer_end <= impl->getFileSize()); prefetch_future = {}; - return size; + return bytes_read; } diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp index add18d8d12e..1bf889540eb 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp @@ -90,7 +90,10 @@ bool AsynchronousReadBufferFromFileDescriptor::nextImpl() prefetch_future = {}; file_offset_of_buffer_end += size; - if (size) + assert(offset <= size); + size_t bytes_read = size - offset; + + if (bytes_read) { prefetch_buffer.swap(memory); /// Adjust the working buffer so that it ignores `offset` bytes. @@ -109,7 +112,10 @@ bool AsynchronousReadBufferFromFileDescriptor::nextImpl() auto [size, offset] = asyncReadInto(memory.data(), memory.size()).get(); file_offset_of_buffer_end += size; - if (size) + assert(offset <= size); + size_t bytes_read = size - offset; + + if (bytes_read) { /// Adjust the working buffer so that it ignores `offset` bytes. internal_buffer = Buffer(memory.data(), memory.data() + memory.size()); diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp index 121a22e764c..eab5c2ab134 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp @@ -112,6 +112,8 @@ bool AsynchronousReadBufferFromHDFS::nextImpl() Stopwatch next_watch; Int64 wait = -1; size_t size = 0; + size_t bytes_read = 0; + if (prefetch_future.valid()) { ProfileEvents::increment(ProfileEvents::RemoteFSPrefetchedReads); @@ -126,7 +128,9 @@ bool AsynchronousReadBufferFromHDFS::nextImpl() LOG_TEST(log, "Current size: {}, offset: {}", size, offset); /// If prefetch_future is valid, size should always be greater than zero. - assert(offset < size); + assert(offset <= size); + bytes_read = size - offset; + wait = watch.elapsedMicroseconds(); ProfileEvents::increment(ProfileEvents::AsynchronousReadWaitMicroseconds, wait); } @@ -147,9 +151,10 @@ bool AsynchronousReadBufferFromHDFS::nextImpl() auto offset = result.offset; LOG_TEST(log, "Current size: {}, offset: {}", size, offset); - assert(offset < size); + assert(offset <= size); + bytes_read = size - offset; - if (size) + if (bytes_read) { /// Adjust the working buffer so that it ignores `offset` bytes. internal_buffer = Buffer(memory.data(), memory.data() + memory.size()); @@ -166,7 +171,7 @@ bool AsynchronousReadBufferFromHDFS::nextImpl() sum_duration += next_watch.elapsedMicroseconds(); sum_wait += wait; - return size; + return bytes_read; } off_t AsynchronousReadBufferFromHDFS::seek(off_t offset, int whence) diff --git a/tests/queries/1_stateful/00024_random_counters.reference b/tests/queries/1_stateful/00024_random_counters.reference index f11b66aa5b5..96ce61aeccb 100644 --- a/tests/queries/1_stateful/00024_random_counters.reference +++ b/tests/queries/1_stateful/00024_random_counters.reference @@ -998,3 +998,5 @@ 1 1 1 2 1 1 +1 1 +1 5 diff --git a/tests/queries/1_stateful/00024_random_counters.sql b/tests/queries/1_stateful/00024_random_counters.sql index 99ba9cc653b..b44f0731471 100644 --- a/tests/queries/1_stateful/00024_random_counters.sql +++ b/tests/queries/1_stateful/00024_random_counters.sql @@ -998,3 +998,12 @@ SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15094099; SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6308405; SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20762370; SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14121177; + +SYSTEM DROP UNCOMPRESSED CACHE; + +SET local_filesystem_read_method = 'pread_threadpool'; +SET min_bytes_to_use_direct_io = 1; +SET use_uncompressed_cache = 1; + +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32745436; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33436573; From 8283cc9e2cc242b2339d357134f9ccb17edc1fdb Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 22 Jul 2022 17:59:43 +0000 Subject: [PATCH 563/659] Fix possible heap-buffer-overflow in Avro --- contrib/avro | 2 +- .../0_stateless/02372_data_race_in_avro.sh | 2 +- .../02373_heap_buffer_overflow_in_avro.reference | 1 + .../02373_heap_buffer_overflow_in_avro.sh | 13 +++++++++++++ .../queries/0_stateless/data_avro/corrupted.avro | Bin 0 -> 161 bytes 5 files changed, 16 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02373_heap_buffer_overflow_in_avro.reference create mode 100755 tests/queries/0_stateless/02373_heap_buffer_overflow_in_avro.sh create mode 100644 tests/queries/0_stateless/data_avro/corrupted.avro diff --git a/contrib/avro b/contrib/avro index aac4e605f07..7832659ec98 160000 --- a/contrib/avro +++ b/contrib/avro @@ -1 +1 @@ -Subproject commit aac4e605f070f2abd9a5f02ae70c17e4011588e8 +Subproject commit 7832659ec986075d560f930c288e973c64679552 diff --git a/tests/queries/0_stateless/02372_data_race_in_avro.sh b/tests/queries/0_stateless/02372_data_race_in_avro.sh index 49c34e31923..50a7ae1e3c5 100755 --- a/tests/queries/0_stateless/02372_data_race_in_avro.sh +++ b/tests/queries/0_stateless/02372_data_race_in_avro.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02373_heap_buffer_overflow_in_avro.reference b/tests/queries/0_stateless/02373_heap_buffer_overflow_in_avro.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/02373_heap_buffer_overflow_in_avro.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/02373_heap_buffer_overflow_in_avro.sh b/tests/queries/0_stateless/02373_heap_buffer_overflow_in_avro.sh new file mode 100755 index 00000000000..23d6b722c09 --- /dev/null +++ b/tests/queries/0_stateless/02373_heap_buffer_overflow_in_avro.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +cp $CURDIR/data_avro/corrupted.avro $USER_FILES_PATH/ + +$CLICKHOUSE_CLIENT -q "select * from file(corrupted.avro)" 2>&1 | grep -F -q "Cannot read compressed data" && echo "OK" || echo "FAIL" + diff --git a/tests/queries/0_stateless/data_avro/corrupted.avro b/tests/queries/0_stateless/data_avro/corrupted.avro new file mode 100644 index 0000000000000000000000000000000000000000..0393b231d0824a7fb28d5c934c225f817c48ee82 GIT binary patch literal 161 zcmeZI%3@>@Nh~YM*GtY%NloS{&PyyPs1yT>6enk-<|g(qRx6cM7Nja!DHWwA=NF|Y z=_utT=7Rb8naW@=7Kv65AEHB7t$C<@h5TvC*om#$PBTg$++>VihV92qah>r*}6 gcX0|b$#E(O*cCndG_T#y=l9c_lh74x5xc_$07-p0x&QzG literal 0 HcmV?d00001 From 2305f282dc25670fb31d90630c275bad922f8938 Mon Sep 17 00:00:00 2001 From: Simon Liu Date: Sat, 23 Jul 2022 02:22:04 +0800 Subject: [PATCH 564/659] Removed some duplicate code. --- src/Storages/MergeTree/MergeTreeSink.cpp | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 81c5708f220..83203939903 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -155,18 +155,12 @@ void MergeTreeSink::finishDelayedChunk() { ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks); LOG_INFO(storage.log, "Block with ID {} already exists as part {}; ignoring it", block_id, res.first.getPartName()); - } - else - { - added = storage.renameTempPartAndAdd(part, transaction, partition.temp_part.builder, lock); - transaction.commit(&lock); + continue; } } - else - { - added = storage.renameTempPartAndAdd(part, transaction, partition.temp_part.builder, lock); - transaction.commit(&lock); - } + + added = storage.renameTempPartAndAdd(part, transaction, partition.temp_part.builder, lock); + transaction.commit(&lock); } /// Part can be deduplicated, so increment counters and add to part log only if it's really added From c559cac085bd7f1d79e89eccf86ad8024cd710dd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Thu, 21 Jul 2022 11:11:43 -0400 Subject: [PATCH 565/659] add to SimpleCheck --- tests/ci/build_report_check.py | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index dbf5adfe174..4bb7a619b9f 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -19,7 +19,10 @@ from report import create_build_html_report from s3_helper import S3Helper from get_robot_token import get_best_robot_token from pr_info import PRInfo -from commit_status_helper import get_commit +from commit_status_helper import ( + get_commit, + fail_simple_check, +) from ci_config import CI_CONFIG from rerun_helper import RerunHelper @@ -151,6 +154,12 @@ def main(): needs_data = json.load(file_handler) required_builds = len(needs_data) + # A report might be empty in case of `do not test` label, for example. + # We should still be able to merge such PRs. + all_skipped = needs_data is not None and all( + i["result"] == "skipped" for i in needs_data.values() + ) + logging.info("The next builds are required: %s", ", ".join(needs_data)) gh = Github(get_best_robot_token()) @@ -228,6 +237,8 @@ def main(): total_groups = len(build_results) logging.info("Totally got %s artifact groups", total_groups) if total_groups == 0: + if not all_skipped: + fail_simple_check(gh, pr_info, f"{build_check_name} failed") logging.error("No success builds, failing check") sys.exit(1) @@ -297,6 +308,8 @@ def main(): ) if summary_status == "error": + if not all_skipped: + fail_simple_check(gh, pr_info, f"{build_check_name} failed") sys.exit(1) From 739ff34c6e15f5f355937b610bd100509549c06d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 22 Jul 2022 22:46:56 +0000 Subject: [PATCH 566/659] Add some tests, still not sure about optimize_memory_usage option --- .../Transforms/DistinctSortedChunkTransform.cpp | 17 ++++++++++++----- tests/performance/distinct_in_order.xml | 6 ++++-- ...317_distinct_in_order_optimization.reference | 1 + .../02317_distinct_in_order_optimization.sql | 16 +++++++++++++--- ...inct_in_order_optimization_explain.reference | 4 ++-- ...17_distinct_in_order_optimization_explain.sh | 4 ++-- 6 files changed, 34 insertions(+), 14 deletions(-) diff --git a/src/Processors/Transforms/DistinctSortedChunkTransform.cpp b/src/Processors/Transforms/DistinctSortedChunkTransform.cpp index 8604cca5a5c..e989fad3d1f 100644 --- a/src/Processors/Transforms/DistinctSortedChunkTransform.cpp +++ b/src/Processors/Transforms/DistinctSortedChunkTransform.cpp @@ -62,7 +62,8 @@ void DistinctSortedChunkTransform::initChunkProcessing(const Columns & input_col data.init(ClearableSetVariants::chooseMethod(other_columns, other_columns_sizes)); } -size_t DistinctSortedChunkTransform::ordinaryDistinctOnRange(IColumn::Filter & filter, size_t range_begin, size_t range_end, bool clear_data) +size_t DistinctSortedChunkTransform::ordinaryDistinctOnRange( + IColumn::Filter & filter, const size_t range_begin, const size_t range_end, const bool clear_data) { size_t count = 0; switch (data.type) @@ -84,7 +85,7 @@ size_t DistinctSortedChunkTransform::ordinaryDistinctOnRange(IColumn::Filter & f template size_t DistinctSortedChunkTransform::buildFilterForRange( - Method & method, IColumn::Filter & filter, size_t range_begin, size_t range_end, bool clear_data) + Method & method, IColumn::Filter & filter, const size_t range_begin, const size_t range_end, const bool clear_data) { typename Method::State state(other_columns, other_columns_sizes, nullptr); if (clear_data) @@ -93,11 +94,11 @@ size_t DistinctSortedChunkTransform::buildFilterForRange( size_t count = 0; for (size_t i = range_begin; i < range_end; ++i) { - auto emplace_result = state.emplaceKey(method.data, i, data.string_pool); + const auto emplace_result = state.emplaceKey(method.data, i, data.string_pool); /// emit the record if there is no such key in the current set, skip otherwise filter[i] = emplace_result.isInserted(); - if (filter[i]) + if (emplace_result.isInserted()) ++count; } return count; @@ -106,7 +107,7 @@ size_t DistinctSortedChunkTransform::buildFilterForRange( void DistinctSortedChunkTransform::saveLatestKey(const size_t row_pos) { prev_chunk_latest_key.clear(); - for (auto const & col : sorted_columns) + for (const auto & col : sorted_columns) { prev_chunk_latest_key.emplace_back(col->cloneEmpty()); prev_chunk_latest_key.back()->insertFrom(*col, row_pos); @@ -224,6 +225,12 @@ void DistinctSortedChunkTransform::transform(Chunk & chunk) // set where next range start range_begin = range_end; } + /// if there is no any new rows in this chunk, just skip it + // if (output_rows) + // { + // chunk.clear(); + // return; + // } saveLatestKey(chunk_rows - 1); diff --git a/tests/performance/distinct_in_order.xml b/tests/performance/distinct_in_order.xml index 834a6945622..ee1094ed395 100644 --- a/tests/performance/distinct_in_order.xml +++ b/tests/performance/distinct_in_order.xml @@ -8,8 +8,9 @@ SELECT DISTINCT high, medium FROM distinct_cardinality_high FORMAT Null SELECT DISTINCT high, medium, low FROM distinct_cardinality_high FORMAT Null - SELECT DISTINCT high, medium FROM distinct_cardinality_high ORDER BY medium FORMAT Null + SELECT DISTINCT high, medium FROM distinct_cardinality_high ORDER BY high, medium FORMAT Null SELECT DISTINCT high, medium FROM distinct_cardinality_high ORDER BY high FORMAT Null + SELECT DISTINCT high, medium FROM distinct_cardinality_high ORDER BY medium FORMAT Null SELECT DISTINCT high, low FROM distinct_cardinality_high ORDER BY low FORMAT Null SELECT DISTINCT high, medium, low FROM distinct_cardinality_high ORDER BY low FORMAT Null @@ -24,8 +25,9 @@ SELECT DISTINCT low, high FROM distinct_cardinality_low FORMAT Null SELECT DISTINCT low, medium, high FROM distinct_cardinality_low FORMAT Null - SELECT DISTINCT low, medium FROM distinct_cardinality_low ORDER BY medium FORMAT Null + SELECT DISTINCT low, medium FROM distinct_cardinality_low ORDER BY low, medium FORMAT Null SELECT DISTINCT low, medium FROM distinct_cardinality_low ORDER BY low FORMAT Null + SELECT DISTINCT low, medium FROM distinct_cardinality_low ORDER BY medium FORMAT Null SELECT DISTINCT low, high FROM distinct_cardinality_low ORDER BY high FORMAT Null SELECT DISTINCT low, medium, high FROM distinct_cardinality_low ORDER BY high FORMAT Null diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference index a5ae3759d5d..a82e9b5cc2e 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference @@ -78,3 +78,4 @@ 1 1 0 0 -- check that distinct in order has the same result as ordinary distinct +-- check that distinct in order has the same result as ordinary distinct, optimize memory usage diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql index d43366a55fe..14618baff95 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql @@ -47,15 +47,25 @@ drop table if exists distinct_in_order sync; select '-- check that distinct in order has the same result as ordinary distinct'; drop table if exists distinct_cardinality_low sync; +CREATE TABLE distinct_cardinality_low (low UInt64, medium UInt64, high UInt64) ENGINE MergeTree() ORDER BY (low, medium); +INSERT INTO distinct_cardinality_low SELECT number % 1e1, number % 1e2, number % 1e3 FROM numbers_mt(1e4); drop table if exists distinct_in_order sync; drop table if exists ordinary_distinct sync; -CREATE TABLE distinct_cardinality_low (low UInt64, medium UInt64, high UInt64) ENGINE MergeTree() ORDER BY (low, medium); -INSERT INTO distinct_cardinality_low SELECT number % 1e1, number % 1e3, number % 1e5 FROM numbers_mt(1e6); create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); -insert into distinct_in_order select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=1; +insert into distinct_in_order select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=1, optimize_distinct_in_order_memory_usage=0; create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); insert into ordinary_distinct select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=0; select distinct * from distinct_in_order except select * from ordinary_distinct; + +select '-- check that distinct in order has the same result as ordinary distinct, optimize memory usage'; +drop table if exists distinct_in_order; +drop table if exists ordinary_distinct; +create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +insert into distinct_in_order select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=1, optimize_distinct_in_order_memory_usage=1; +create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +insert into ordinary_distinct select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=0; +select distinct * from distinct_in_order except select * from ordinary_distinct; + drop table if exists distinct_in_order; drop table if exists ordinary_distinct; drop table if exists distinct_cardinality_low; diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index 2a2ff8799f0..b41d853e3eb 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -15,10 +15,10 @@ DistinctSortedChunkTransform -- distinct with primary key prefix and order by on the same columns -> pre-distinct and final distinct optimization DistinctSortedStreamTransform DistinctSortedChunkTransform --- distinct with primary key prefix and order by column in distinct but non-primary key prefix -> pre-distinct and final distinct optimization +-- distinct with primary key prefix and order by column in distinct but non-primary key prefix -> pre-distinct and final distinct optimization DistinctSortedTransform DistinctSortedChunkTransform --- distinct with primary key prefix and order by column in distinct but non-primary key prefix, optimize memory usage -> pre-distinct and final distinct optimization +-- distinct with primary key prefix and order by column in distinct but non-primary key prefix, optimize memory usage -> pre-distinct and final distinct optimization DistinctSortedStreamTransform DistinctSortedChunkTransform -- distinct with primary key prefix and order by on column _not_ in distinct -> pre-distinct optimization only diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh index f3f39997f0d..c19bc3e6aaf 100755 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh @@ -33,10 +33,10 @@ $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by on the same columns -> pre-distinct and final distinct optimization'" $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a, b" | eval $FIND_DISTINCT -$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct but non-primary key prefix -> pre-distinct and final distinct optimization'" +$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct but non-primary key prefix -> pre-distinct and final distinct optimization'" $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c settings optimize_distinct_in_order_memory_usage=0" | eval $FIND_DISTINCT -$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct but non-primary key prefix, optimize memory usage -> pre-distinct and final distinct optimization'" +$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct but non-primary key prefix, optimize memory usage -> pre-distinct and final distinct optimization'" $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c settings optimize_distinct_in_order_memory_usage=1" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by on column _not_ in distinct -> pre-distinct optimization only'" From 95511428b303010676d3229d7d8bed0404149299 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 23 Jul 2022 00:03:26 +0000 Subject: [PATCH 567/659] Couple optimizations + do not apply filter to chunk if there is no data for output + checking clear_data flag at compile time --- .../DistinctSortedChunkTransform.cpp | 29 ++++++++++--------- .../Transforms/DistinctSortedChunkTransform.h | 5 ++-- 2 files changed, 19 insertions(+), 15 deletions(-) diff --git a/src/Processors/Transforms/DistinctSortedChunkTransform.cpp b/src/Processors/Transforms/DistinctSortedChunkTransform.cpp index e989fad3d1f..28a3260d742 100644 --- a/src/Processors/Transforms/DistinctSortedChunkTransform.cpp +++ b/src/Processors/Transforms/DistinctSortedChunkTransform.cpp @@ -62,8 +62,8 @@ void DistinctSortedChunkTransform::initChunkProcessing(const Columns & input_col data.init(ClearableSetVariants::chooseMethod(other_columns, other_columns_sizes)); } -size_t DistinctSortedChunkTransform::ordinaryDistinctOnRange( - IColumn::Filter & filter, const size_t range_begin, const size_t range_end, const bool clear_data) +template +size_t DistinctSortedChunkTransform::ordinaryDistinctOnRange(IColumn::Filter & filter, const size_t range_begin, const size_t range_end) { size_t count = 0; switch (data.type) @@ -73,7 +73,8 @@ size_t DistinctSortedChunkTransform::ordinaryDistinctOnRange( // clang-format off #define M(NAME) \ case ClearableSetVariants::Type::NAME: \ - count = buildFilterForRange(*data.NAME, filter, range_begin, range_end, clear_data); \ + if constexpr (clear_data) data.NAME->data.clear(); \ + count = buildFilterForRange(*data.NAME, filter, range_begin, range_end); \ break; APPLY_FOR_SET_VARIANTS(M) @@ -85,11 +86,9 @@ size_t DistinctSortedChunkTransform::ordinaryDistinctOnRange( template size_t DistinctSortedChunkTransform::buildFilterForRange( - Method & method, IColumn::Filter & filter, const size_t range_begin, const size_t range_end, const bool clear_data) + Method & method, IColumn::Filter & filter, const size_t range_begin, const size_t range_end) { typename Method::State state(other_columns, other_columns_sizes, nullptr); - if (clear_data) - method.data.clear(); size_t count = 0; for (size_t i = range_begin; i < range_end; ++i) @@ -180,7 +179,10 @@ std::pair DistinctSortedChunkTransform::continueWithPrevRange(co if (other_columns.empty()) std::fill(filter.begin(), filter.begin() + range_end, 0); /// skip rows already included in distinct on previous transform() else - output_rows = ordinaryDistinctOnRange(filter, 0, range_end, false); + { + constexpr bool clear_data = false; + output_rows = ordinaryDistinctOnRange(filter, 0, range_end); + } return {range_end, output_rows}; } @@ -219,18 +221,19 @@ void DistinctSortedChunkTransform::transform(Chunk & chunk) else { // ordinary distinct in range if there are "non-sorted" columns - output_rows += ordinaryDistinctOnRange(filter, range_begin, range_end, true); + constexpr bool clear_data = true; + output_rows += ordinaryDistinctOnRange(filter, range_begin, range_end); } // set where next range start range_begin = range_end; } /// if there is no any new rows in this chunk, just skip it - // if (output_rows) - // { - // chunk.clear(); - // return; - // } + if (!output_rows) + { + chunk.clear(); + return; + } saveLatestKey(chunk_rows - 1); diff --git a/src/Processors/Transforms/DistinctSortedChunkTransform.h b/src/Processors/Transforms/DistinctSortedChunkTransform.h index 0ce8addbf7e..188e3d5c4c7 100644 --- a/src/Processors/Transforms/DistinctSortedChunkTransform.h +++ b/src/Processors/Transforms/DistinctSortedChunkTransform.h @@ -43,7 +43,8 @@ protected: private: void initChunkProcessing(const Columns & input_columns); std::pair continueWithPrevRange(size_t chunk_rows, IColumn::Filter & filter); - size_t ordinaryDistinctOnRange(IColumn::Filter & filter, size_t range_begin, size_t range_end, bool clear_data); + template + size_t ordinaryDistinctOnRange(IColumn::Filter & filter, size_t range_begin, size_t range_end); inline void saveLatestKey(size_t row_pos); inline bool isLatestKeyFromPrevChunk(size_t row_pos) const; inline bool isKey(size_t key_pos, size_t row_pos) const; @@ -51,7 +52,7 @@ private: inline size_t getRangeEnd(size_t range_begin, size_t range_end, Predicate pred) const; template - size_t buildFilterForRange(Method & method, IColumn::Filter & filter, size_t range_begin, size_t range_end, bool clear_data); + size_t buildFilterForRange(Method & method, IColumn::Filter & filter, size_t range_begin, size_t range_end); ClearableSetVariants data; From d645d03e89dc41a8c084555a278a9dc92a174d6a Mon Sep 17 00:00:00 2001 From: Constantine Peresypkin Date: Sat, 23 Jul 2022 19:00:52 +0200 Subject: [PATCH 568/659] fix `-DENABLE_EXAMPLES=1` in master --- .../examples/zkutil_test_commands_new_lib.cpp | 1 + src/Common/examples/small_table.cpp | 17 ----------------- src/Interpreters/examples/hash_map_string.cpp | 3 +++ .../examples/async_read_buffer_from_hdfs.cpp | 3 ++- 4 files changed, 6 insertions(+), 18 deletions(-) diff --git a/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp b/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp index cd0f9f024fd..09b94a34b78 100644 --- a/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp +++ b/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp @@ -99,6 +99,7 @@ try std::cout << "list\n"; zk.list("/", + Coordination::ListRequestType::ALL, [&](const ListResponse & response) { if (response.error != Coordination::Error::ZOK) diff --git a/src/Common/examples/small_table.cpp b/src/Common/examples/small_table.cpp index be5e983e70a..ca38516d09a 100644 --- a/src/Common/examples/small_table.cpp +++ b/src/Common/examples/small_table.cpp @@ -1,5 +1,4 @@ #include -#include #include @@ -33,22 +32,6 @@ int main(int, char **) std::cerr << "dump: " << wb.str() << std::endl; } - { - using Cont = SmallMap; - Cont cont; - - cont.insert(Cont::value_type(1, "Hello, world!")); - cont[1] = "Goodbye."; - - for (auto x : cont) - std::cerr << x.getKey() << " -> " << x.getMapped() << std::endl; - - DB::WriteBufferFromOwnString wb; - cont.writeText(wb); - - std::cerr << "dump: " << wb.str() << std::endl; - } - { using Cont = SmallSet; Cont cont; diff --git a/src/Interpreters/examples/hash_map_string.cpp b/src/Interpreters/examples/hash_map_string.cpp index 54915ee7c10..64d0f981f50 100644 --- a/src/Interpreters/examples/hash_map_string.cpp +++ b/src/Interpreters/examples/hash_map_string.cpp @@ -30,11 +30,14 @@ struct CompactStringRef union { const char * data_mixed = nullptr; +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wnested-anon-types" struct { char dummy[6]; UInt16 size; }; +#pragma clang diagnostic pop }; CompactStringRef(const char * data_, size_t size_) diff --git a/src/Storages/examples/async_read_buffer_from_hdfs.cpp b/src/Storages/examples/async_read_buffer_from_hdfs.cpp index f9da4b9ac4f..b285857d684 100644 --- a/src/Storages/examples/async_read_buffer_from_hdfs.cpp +++ b/src/Storages/examples/async_read_buffer_from_hdfs.cpp @@ -23,7 +23,8 @@ int main() String hdfs_namenode_url = "hdfs://namenode:port/"; String path = "/path/to/hdfs/file"; - auto in = std::make_unique(hdfs_namenode_url, path, *config); + ReadSettings settings = {}; + auto in = std::make_unique(hdfs_namenode_url, path, *config, settings); auto reader = IObjectStorage::getThreadPoolReader(); AsynchronousReadBufferFromHDFS buf(reader, {}, std::move(in)); From 02a0b60e242db9449e0d2ab66d6f38c51752d69d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 23 Jul 2022 22:20:37 +0300 Subject: [PATCH 569/659] Update CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 56a6e27d8f1..70e203193c8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -12,6 +12,7 @@ #### Upgrade Notes * Enable setting `enable_positional_arguments` by default. It allows queries like `SELECT ... ORDER BY 1, 2` where 1, 2 are the references to the select clause. If you need to return the old behavior, disable this setting. [#38204](https://github.com/ClickHouse/ClickHouse/pull/38204) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable `format_csv_allow_single_quotes` by default. See [#37096](https://github.com/ClickHouse/ClickHouse/issues/37096). ([Kruglov Pavel](https://github.com/Avogar)). * `Ordinary` database engine and old storage definition syntax for `*MergeTree` tables are deprecated. By default it's not possible to create new databases with `Ordinary` engine. If `system` database has `Ordinary` engine it will be automatically converted to `Atomic` on server startup. There are settings to keep old behavior (`allow_deprecated_database_ordinary` and `allow_deprecated_syntax_for_merge_tree`), but these settings may be removed in future releases. [#38335](https://github.com/ClickHouse/ClickHouse/pull/38335) ([Alexander Tokmakov](https://github.com/tavplubix)). * Force rewriting comma join to inner by default (set default value `cross_to_inner_join_rewrite = 2`). To have old behavior set `cross_to_inner_join_rewrite = 1`. [#39326](https://github.com/ClickHouse/ClickHouse/pull/39326) ([Vladimir C](https://github.com/vdimir)). If you will face any incompatibilities, you can turn this setting back. From 594195451ee5d75ed1425c6770ed81399a39ac8e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 21 Jul 2022 21:50:19 +0200 Subject: [PATCH 570/659] Cleanups --- src/Core/Settings.h | 3 +-- src/Interpreters/InterpreterDeleteQuery.cpp | 5 ++-- src/Interpreters/MutationsInterpreter.cpp | 14 +++++----- src/Interpreters/MutationsInterpreter.h | 2 -- src/Storages/AlterCommands.cpp | 5 ++-- src/Storages/ColumnsDescription.h | 7 +++++ src/Storages/IStorage.h | 2 +- src/Storages/LightweightDeleteDescription.cpp | 9 +++++++ src/Storages/LightweightDeleteDescription.h | 13 ++++++++++ .../MergeTree/DataPartStorageOnDisk.cpp | 3 --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 --- src/Storages/MergeTree/IMergeTreeDataPart.h | 5 ++-- src/Storages/MergeTree/IMergeTreeReader.h | 2 -- src/Storages/MergeTree/MergeTask.cpp | 11 ++++---- .../MergeTreeBaseSelectProcessor.cpp | 18 ++++++------- .../MergeTree/MergeTreeBaseSelectProcessor.h | 2 +- .../MergeTree/MergeTreeBlockReadUtils.cpp | 20 +++++++------- .../MergeTree/MergeTreeBlockReadUtils.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 6 ++--- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../MergeTree/MergeTreeRangeReader.cpp | 16 +++--------- src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 +- .../MergeTree/MergeTreeSequentialSource.cpp | 4 ++- src/Storages/MergeTree/MutateTask.cpp | 7 ----- .../MergeTree/StorageFromMergeTreeDataPart.h | 4 +-- .../MergeTree/registerStorageMergeTree.cpp | 3 --- src/Storages/StorageInMemoryMetadata.cpp | 2 -- src/Storages/StorageInMemoryMetadata.h | 2 -- src/Storages/StorageSnapshot.cpp | 26 ++++++++++++++++++- src/Storages/StorageSnapshot.h | 4 +++ src/Storages/TTLDescription.h | 7 ----- tests/performance/lightweight_delete.xml | 3 +-- ...02319_lightweight_delete_on_merge_tree.sql | 3 +-- ...ght_delete_on_merge_tree_compact_parts.sql | 3 +-- .../0_stateless/02352_lightweight_delete.sql | 3 +-- 35 files changed, 120 insertions(+), 104 deletions(-) create mode 100644 src/Storages/LightweightDeleteDescription.cpp create mode 100644 src/Storages/LightweightDeleteDescription.h diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 672b8c5b1fb..4bf476befb2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -458,8 +458,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \ M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ - M(Bool, allow_experimental_lightweight_delete_with_row_exists, false, "Enable lightweight DELETE mutations using __rows_exists column for mergetree tables. Work in progress", 0) \ - M(Bool, lightweight_delete_mutation, true, "Enable to make ordinary ALTER DELETE queries lightweight for mergetree tables", 0) \ + M(Bool, allow_experimental_lightweight_delete, false, "Enable lightweight DELETE mutations for mergetree tables. Work in progress", 0) \ M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index cb4bc363d18..8c8030c6a51 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB @@ -33,7 +34,7 @@ InterpreterDeleteQuery::InterpreterDeleteQuery(const ASTPtr & query_ptr_, Contex BlockIO InterpreterDeleteQuery::execute() { - if (!getContext()->getSettingsRef().allow_experimental_lightweight_delete_with_row_exists) + if (!getContext()->getSettingsRef().allow_experimental_lightweight_delete) { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Lightweight delete mutate is experimental. Set `allow_experimental_lightweight_delete` setting to enable it"); } @@ -81,7 +82,7 @@ BlockIO InterpreterDeleteQuery::execute() command->predicate = delete_query.predicate; command->update_assignments = std::make_shared(); auto set_row_does_not_exist = std::make_shared(); - set_row_does_not_exist->column_name = metadata_snapshot->lightweight_delete_description.filter_column.name; + set_row_does_not_exist->column_name = LightweightDeleteDescription::filter_column.name; auto zero_value = std::make_shared(DB::Field(UInt8(0))); set_row_does_not_exist->children.push_back(zero_value); command->update_assignments->children.push_back(set_row_does_not_exist); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 18f8b493ad6..180e160aca6 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -28,6 +28,8 @@ #include #include #include +#include + namespace DB { @@ -349,7 +351,7 @@ static void validateUpdateColumns( } /// Allow to override value of lightweight delete filter virtual column - if (!found && column_name == metadata_snapshot->lightweight_delete_description.filter_column.name) + if (!found && column_name == LightweightDeleteDescription::filter_column.name) found = true; if (!found) @@ -508,8 +510,8 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) DataTypePtr type; if (auto physical_column = columns_desc.tryGetPhysical(column)) type = physical_column->type; - else if (column == metadata_snapshot->lightweight_delete_description.filter_column.name) - type = metadata_snapshot->lightweight_delete_description.filter_column.type; + else if (column == LightweightDeleteDescription::filter_column.name) + type = LightweightDeleteDescription::filter_column.type; else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown column {}", column); @@ -772,11 +774,11 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects(); auto all_columns = storage_snapshot->getColumns(options); - // TODO: add _row_exists column if it is present in the part??? + /// Add _row_exists column if it is present in the part if (auto part_storage = dynamic_pointer_cast(storage)) { - if (part_storage->hasLightweightDeleteColumn()) - all_columns.push_back({metadata_snapshot->lightweight_delete_description.filter_column}); + if (part_storage->hasLightweightDeletedMask()) + all_columns.push_back({LightweightDeleteDescription::filter_column}); } /// Next, for each stage calculate columns changed by this and previous stages. diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 10f764caaee..94525bf6b8c 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -86,7 +86,6 @@ private: struct Stage; ASTPtr prepareInterpreterSelectQuery(std::vector &prepared_stages, bool dry_run); - QueryPipelineBuilder addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const; std::optional getStorageSortDescriptionIfPossible(const Block & header) const; @@ -100,7 +99,6 @@ private: bool can_execute; SelectQueryOptions select_limits; - /// TODO: is it needed? bool apply_deleted_mask = true; ASTPtr mutation_ast; diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 03053eb9b18..808d634b1ea 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include @@ -786,7 +787,7 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada /// Drop alias is metadata alter, in other case mutation is required. if (type == DROP_COLUMN) return metadata.columns.hasColumnOrNested(GetColumnsOptions::AllPhysical, column_name) || - column_name == metadata.lightweight_delete_description.filter_column.name; + column_name == LightweightDeleteDescription::filter_column.name; if (type != MODIFY_COLUMN || data_type == nullptr) return false; @@ -1152,7 +1153,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const { if (all_columns.has(command.column_name) || all_columns.hasNested(command.column_name) || - (command.clear && column_name == metadata.lightweight_delete_description.filter_column.name)) + (command.clear && column_name == LightweightDeleteDescription::filter_column.name)) { if (!command.clear) /// CLEAR column is Ok even if there are dependencies. { diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 209dee885f4..eea5dc7fcbb 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -60,10 +60,17 @@ struct GetColumnsOptions return *this; } + GetColumnsOptions & withSystemColumns(bool value = true) + { + with_system_columns = value; + return *this; + } + Kind kind; bool with_subcolumns = false; bool with_virtuals = false; bool with_extended_objects = false; + bool with_system_columns = false; }; /// Description of a single table column (in CREATE TABLE for example). diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 3647941cc57..0a25a30ec6f 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -235,7 +235,7 @@ public: /// Returns true if the storage supports backup/restore for specific partitions. virtual bool supportsBackupPartition() const { return false; } - /// Return true if there are at least one part containing lightweight deleted mask. + /// Return true if there is at least one part containing lightweight deleted mask. virtual bool hasLightweightDeletedMask() const { return false; } private: diff --git a/src/Storages/LightweightDeleteDescription.cpp b/src/Storages/LightweightDeleteDescription.cpp new file mode 100644 index 00000000000..0ffb7766c80 --- /dev/null +++ b/src/Storages/LightweightDeleteDescription.cpp @@ -0,0 +1,9 @@ +#include +#include + +namespace DB +{ + +const NameAndTypePair LightweightDeleteDescription::filter_column {"_row_exists", std::make_shared()}; + +} diff --git a/src/Storages/LightweightDeleteDescription.h b/src/Storages/LightweightDeleteDescription.h new file mode 100644 index 00000000000..7177b2a5b54 --- /dev/null +++ b/src/Storages/LightweightDeleteDescription.h @@ -0,0 +1,13 @@ +#pragma once +#include +#include "Storages/TTLDescription.h" + +namespace DB +{ + +struct LightweightDeleteDescription +{ + static const NameAndTypePair filter_column; +}; + +} diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 03d24d84bb0..f3b228a0748 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -368,10 +368,7 @@ void DataPartStorageOnDisk::clearDirectory( request.emplace_back(fs::path(dir) / "delete-on-destroy.txt", true); if (!is_projection) - { request.emplace_back(fs::path(dir) / "txn_version.txt", true); - request.emplace_back(fs::path(dir) / "deleted_rows_mask.bin", true); - } disk->removeSharedFiles(request, !can_remove_shared_data, names_not_to_remove); disk->removeDirectory(dir); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 3f8000f3136..36855fe48bb 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -294,7 +294,6 @@ static void decrementTypeMetric(MergeTreeDataPartType type) } } - IMergeTreeDataPart::IMergeTreeDataPart( const MergeTreeData & storage_, const String & name_, @@ -790,9 +789,6 @@ NameSet IMergeTreeDataPart::getFileNamesWithoutChecksums() const if (data_part_storage->exists(TXN_VERSION_METADATA_FILE_NAME)) result.emplace(TXN_VERSION_METADATA_FILE_NAME); - if (data_part_storage->exists(DELETED_ROWS_MARK_FILE_NAME)) - result.emplace(DELETED_ROWS_MARK_FILE_NAME); - return result; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index a9c4590c045..b205da4d4c2 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -404,8 +405,6 @@ public: static inline constexpr auto TXN_VERSION_METADATA_FILE_NAME = "txn_version.txt"; - static inline constexpr auto DELETED_ROWS_MARK_FILE_NAME = "deleted_rows_mask.bin"; - /// One of part files which is used to check how many references (I'd like /// to say hardlinks, but it will confuse even more) we have for the part /// for zero copy replication. Sadly it's very complex. @@ -462,7 +461,7 @@ public: bool supportLightweightDeleteMutate() const; /// True if here is lightweight deleted mask file in part. - bool hasLightweightDelete() const { return data_part_storage->exists(DELETED_ROWS_MARK_FILE_NAME); } + bool hasLightweightDelete() const { return columns.contains(LightweightDeleteDescription::filter_column.name); } protected: diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 41030e522ac..b13db9c3255 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -62,8 +62,6 @@ public: MergeTreeData::DataPartPtr data_part; - bool needReadDeletedMask() { return settings.apply_deleted_mask && data_part->hasLightweightDelete(); } - protected: /// Returns actual column type in part, which can differ from table metadata. NameAndTypePair getColumnFromPart(const NameAndTypePair & required_column) const; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 65c9523f861..20992ff2a99 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -5,6 +5,7 @@ #include #include +#include #include #include @@ -814,10 +815,10 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() auto columns = global_ctx->merging_column_names; /// The part might have some rows masked by lightweight deletes - const auto lwd_filter_column = global_ctx->metadata_snapshot->lightweight_delete_description.filter_column.name; - const bool need_to_filter_deleted_rows = !lwd_filter_column.empty() && part->getColumns().contains(lwd_filter_column); + const auto lightweight_delete_filter_column = LightweightDeleteDescription::filter_column.name; + const bool need_to_filter_deleted_rows = part->hasLightweightDelete(); if (need_to_filter_deleted_rows) - columns.emplace_back(lwd_filter_column); + columns.emplace_back(lightweight_delete_filter_column); auto input = std::make_unique( *global_ctx->data, global_ctx->storage_snapshot, part, columns, ctx->read_with_direct_io, true); @@ -827,9 +828,9 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() /// Add filtering step that discards deleted rows if (need_to_filter_deleted_rows) { - pipe.addSimpleTransform([lwd_filter_column](const Block & header) + pipe.addSimpleTransform([lightweight_delete_filter_column](const Block & header) { - return std::make_shared(header, nullptr, lwd_filter_column, true); + return std::make_shared(header, nullptr, lightweight_delete_filter_column, true); }); } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 4cae54492c8..00b9959739f 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -61,7 +61,7 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( { non_const_virtual_column_names.emplace_back(*it); } - else if (*it == "_row_exists") + else if (*it == LightweightDeleteDescription::filter_column.name) { non_const_virtual_column_names.emplace_back(*it); } @@ -242,10 +242,9 @@ void MergeTreeBaseSelectProcessor::initializeMergeTreeReadersForPart( pre_reader_for_step.clear(); /// Add lightweight delete filtering step - const auto & lightweigth_delete_info = metadata_snapshot->lightweight_delete_description; - if (reader_settings.apply_deleted_mask && data_part->getColumns().contains(lightweigth_delete_info.filter_column.name)) + if (reader_settings.apply_deleted_mask && data_part->hasLightweightDelete()) { - pre_reader_for_step.push_back(data_part->getReader({lightweigth_delete_info.filter_column}, metadata_snapshot, mark_ranges, + pre_reader_for_step.push_back(data_part->getReader({LightweightDeleteDescription::filter_column}, metadata_snapshot, mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, value_size_map, profile_callback)); } @@ -268,11 +267,10 @@ void MergeTreeBaseSelectProcessor::initializeRangeReaders(MergeTreeReadTask & cu size_t pre_readers_shift = 0; /// Add filtering step with lightweight delete mask - const auto & lightweigth_delete_info = storage_snapshot->metadata->lightweight_delete_description; - if (reader_settings.apply_deleted_mask && current_task.data_part->getColumns().contains(lightweigth_delete_info.filter_column.name)) + if (reader_settings.apply_deleted_mask && current_task.data_part->hasLightweightDelete()) { current_task.pre_range_readers.push_back( - MergeTreeRangeReader(pre_reader_for_step[0].get(), prev_reader, &lwd_filter_step, last_reader, non_const_virtual_column_names)); + MergeTreeRangeReader(pre_reader_for_step[0].get(), prev_reader, &lightweight_delete_filter_step, last_reader, non_const_virtual_column_names)); prev_reader = ¤t_task.pre_range_readers.back(); pre_readers_shift++; } @@ -471,14 +469,14 @@ static void injectNonConstVirtualColumns( } } - if (virtual_column_name == "_row_exists") + if (virtual_column_name == LightweightDeleteDescription::filter_column.name) { /// If _row_exists column isn't present in the part then fill it here with 1s ColumnPtr column; if (rows) - column = DataTypeUInt8().createColumnConst(rows, 1)->convertToFullColumnIfConst(); + column = LightweightDeleteDescription::filter_column.type->createColumnConst(rows, 1)->convertToFullColumnIfConst(); else - column = DataTypeUInt8().createColumn(); + column = LightweightDeleteDescription::filter_column.type->createColumn(); inserter.insertUInt8Column(column, virtual_column_name); } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 51805fa83a2..1a04c2ef25f 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -103,7 +103,7 @@ protected: StorageSnapshotPtr storage_snapshot; /// This step is added when the part has lightweight delete mask - const PrewhereExprStep lwd_filter_step { nullptr, "_row_exists", true, true }; + const PrewhereExprStep lightweight_delete_filter_step { nullptr, LightweightDeleteDescription::filter_column.name, true, true }; PrewhereInfoPtr prewhere_info; std::unique_ptr prewhere_actions; diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index ad208f6b041..56007c5c5e7 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -101,16 +101,16 @@ NameSet injectRequiredColumns( if (!part->isProjectionPart()) alter_conversions = storage.getAlterConversionsForPart(part); - auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects(); + auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical) + .withExtendedObjects() + .withSystemColumns(); if (with_subcolumns) options.withSubcolumns(); for (size_t i = 0; i < columns.size(); ++i) { - /// We are going to fetch only physical columns - const bool is_real_column = storage_snapshot->tryGetColumn(options, columns[i]).has_value(); - const bool is_virtual_column = storage.isVirtualColumn(columns[i], storage_snapshot->getMetadataForQuery()); - if (!is_real_column && !is_virtual_column) + /// We are going to fetch only physical columns and system columns + if (!storage_snapshot->tryGetColumn(options, columns[i])) throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no physical column or subcolumn {} in table", columns[i]); have_at_least_one_physical_column |= injectRequiredColumnsRecursively( @@ -274,15 +274,15 @@ MergeTreeReadTaskColumns getReadTaskColumns( const StorageSnapshotPtr & storage_snapshot, const MergeTreeData::DataPartPtr & data_part, const Names & required_columns, - const Names & non_const_virtual_columns, + const Names & system_columns, const PrewhereInfoPtr & prewhere_info, bool with_subcolumns) { Names column_names = required_columns; Names pre_column_names; - /// read non-const virtual column from data if it exists - for (const auto & name : non_const_virtual_columns) + /// Read system columns such as lightweight delete mask "_row_exists" if it is persisted in the part + for (const auto & name : system_columns) { if (data_part->getColumns().contains(name)) column_names.push_back(name); @@ -293,7 +293,9 @@ MergeTreeReadTaskColumns getReadTaskColumns( storage, storage_snapshot, data_part, with_subcolumns, column_names); MergeTreeReadTaskColumns result; - auto options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); + auto options = GetColumnsOptions(GetColumnsOptions::All) + .withExtendedObjects() + .withSystemColumns(); if (with_subcolumns) options.withSubcolumns(); diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index d6277167555..5a36955b4d3 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -79,7 +79,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( const StorageSnapshotPtr & storage_snapshot, const MergeTreeData::DataPartPtr & data_part, const Names & required_columns, - const Names & non_const_virtual_columns, + const Names & system_columns, const PrewhereInfoPtr & prewhere_info, bool with_subcolumns); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0223561cdb6..bd52ca3dfd7 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1077,7 +1077,7 @@ void MergeTreeData::loadDataPartsFromDisk( has_adaptive_parts.store(true, std::memory_order_relaxed); /// Check if there is lightweight delete in part - if (part->hasLightweightDelete() || part->getColumns().contains("_row_exists")) // TODO: implement properly + if (part->hasLightweightDelete()) has_lightweight_in_parts.store(true, std::memory_order_relaxed); part->modification_time = part_disk_ptr->getLastModified(fs::path(relative_data_path) / part_name).epochTime(); @@ -2872,7 +2872,7 @@ bool MergeTreeData::renameTempPartAndReplaceImpl( throw Exception("MergeTreeData::Transaction for one table cannot be used with another. It is a bug.", ErrorCodes::LOGICAL_ERROR); - if (part->hasLightweightDelete() || part->getColumns().contains("_row_exists")) // TODO: implement properly + if (part->hasLightweightDelete()) has_lightweight_delete_parts.store(true); checkPartCanBeAddedToTable(part, lock); @@ -6562,7 +6562,7 @@ NamesAndTypesList MergeTreeData::getVirtuals() const NameAndTypePair("_partition_value", getPartitionValueType()), NameAndTypePair("_sample_factor", std::make_shared()), NameAndTypePair("_part_offset", std::make_shared()), - NameAndTypePair("_row_exists", std::make_shared()), + LightweightDeleteDescription::filter_column, }; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c78c187db8f..e205bb581d4 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1214,7 +1214,7 @@ static void selectColumnNames( { virt_column_names.push_back(name); } - else if (name == "_row_exists") + else if (name == LightweightDeleteDescription::filter_column.name) { virt_column_names.push_back(name); } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index cb2ead8a025..acc90fe7313 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -656,10 +656,7 @@ MergeTreeRangeReader::MergeTreeRangeReader( , prewhere_info(prewhere_info_) , last_reader_in_chain(last_reader_in_chain_) , is_initialized(true) -// , non_const_virtual_column_names() { - - if (prev_reader) sample_block = prev_reader->getSampleBlock(); @@ -675,9 +672,6 @@ MergeTreeRangeReader::MergeTreeRangeReader( if (column_name == "_part_offset") sample_block.insert(ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), column_name)); - -// if (column_name == "_row_exists") -// sample_block.insert(ColumnWithTypeAndName(ColumnUInt8::create(), std::make_shared(), column_name)); } if (prewhere_info) @@ -862,11 +856,8 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar if (read_result.num_rows) { /// Physical columns go first and then some virtual columns follow + /// TODO: is there a better way to account for virtual columns that were filled by previous readers? size_t physical_columns_count = read_result.columns.size() - read_result.extra_columns_filled.size(); -/////////// -// TODO: properly account for "virtual columns" that are overridden with real data in the part - -///////////// Columns physical_columns(read_result.columns.begin(), read_result.columns.begin() + physical_columns_count); bool should_evaluate_missing_defaults; @@ -1159,10 +1150,9 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r block.insert({result.columns[pos], std::make_shared(), column_name}); } - else if (column_name == "_row_exists") + else if (column_name == LightweightDeleteDescription::filter_column.name) { - /// do nothing, it will be added later - /// TODO: properly implement reading non-const virtual columns or filling them with default values + /// Do nothing, it will be added later } else throw Exception("Unexpected non-const virtual column: " + column_name, ErrorCodes::LOGICAL_ERROR); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 9bcc6535abb..cc2c20eda5a 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -215,7 +215,7 @@ std::vector MergeTreeReadPool::fillPerPartInfo(const RangesInDataParts & auto task_columns = getReadTaskColumns( data, storage_snapshot, part.data_part, - column_names, virtual_column_names /*TODO: fill non-const virtual columns*/, prewhere_info, /*with_subcolumns=*/ true); + column_names, virtual_column_names, prewhere_info, /*with_subcolumns=*/ true); auto size_predictor = !predict_block_size_bytes ? nullptr : MergeTreeBaseSelectProcessor::getSizePredictor(part.data_part, task_columns, sample_block); diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index de48b96edd6..c86acf11547 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -46,7 +46,9 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( NamesAndTypesList columns_for_reader; if (take_column_types_from_storage) { - auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects().withVirtuals(); /// TODO: only add _rows_exists column (if it's present on disk) + auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical) + .withExtendedObjects() + .withSystemColumns(); columns_for_reader = storage_snapshot->getColumnsByNames(options, columns_to_read); } else diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 0e61f499202..cb041775ca6 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -453,10 +453,6 @@ NameSet collectFilesToSkip( { NameSet files_to_skip = source_part->getFileNamesWithoutChecksums(); - /// Remove deleted rows mask file name to create hard link for it when mutate some columns. - if (files_to_skip.contains(IMergeTreeDataPart::DELETED_ROWS_MARK_FILE_NAME)) - files_to_skip.erase(IMergeTreeDataPart::DELETED_ROWS_MARK_FILE_NAME); - /// Skip updated files for (const auto & entry : updated_header) { @@ -650,8 +646,6 @@ struct MutationContext MergeTreeData::DataPartPtr source_part; StoragePtr storage_from_source_part; - bool is_lightweight_mutation{false}; - StorageMetadataPtr metadata_snapshot; MutationCommandsConstPtr commands; @@ -1526,7 +1520,6 @@ bool MutateTask::prepare() ctx->new_data_part->setColumns(new_columns); ctx->new_data_part->setSerializationInfos(new_infos); - ctx->new_data_part->partition.assign(ctx->source_part->partition); /// Don't change granularity type while mutating subset of columns diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index b2f62c2bf02..9249306293d 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -106,9 +106,9 @@ public: return parts.front()->storage.getSettings()->materialize_ttl_recalculate_only; } - bool hasLightweightDeleteColumn() const + bool hasLightweightDeletedMask() const override { - return parts.front()->getColumns().contains("_row_exists"); // TODO: fix hardcoded column name + return parts.front()->hasLightweightDelete(); } private: diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index beeb980a786..e52a0fed674 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -10,7 +10,6 @@ #include #include #include -#include "DataTypes/DataTypesNumber.h" #include #include @@ -678,8 +677,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (arg_num != arg_cnt) throw Exception("Wrong number of engine arguments.", ErrorCodes::BAD_ARGUMENTS); - metadata.lightweight_delete_description.filter_column = { "_row_exists", std::make_shared() }; - if (replicated) { auto storage_policy = args.getContext()->getStoragePolicy(storage_settings->storage_policy); diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index de12467bdec..66dcc938aef 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -38,7 +38,6 @@ StorageInMemoryMetadata::StorageInMemoryMetadata(const StorageInMemoryMetadata & , sampling_key(other.sampling_key) , column_ttls_by_name(other.column_ttls_by_name) , table_ttl(other.table_ttl) - , lightweight_delete_description(other.lightweight_delete_description) , settings_changes(other.settings_changes ? other.settings_changes->clone() : nullptr) , select(other.select) , comment(other.comment) @@ -64,7 +63,6 @@ StorageInMemoryMetadata & StorageInMemoryMetadata::operator=(const StorageInMemo sampling_key = other.sampling_key; column_ttls_by_name = other.column_ttls_by_name; table_ttl = other.table_ttl; - lightweight_delete_description = other.lightweight_delete_description; if (other.settings_changes) settings_changes = other.settings_changes->clone(); else diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 84a3bcb3046..a9ab96909f4 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -43,8 +43,6 @@ struct StorageInMemoryMetadata TTLColumnsDescription column_ttls_by_name; /// TTL expressions for table (Move and Rows) TTLTableDescription table_ttl; - /// Lightweight delete filter column if the storage supports it. - LightweightDeleteDescription lightweight_delete_description; /// SETTINGS expression. Supported for MergeTree, Buffer, Kafka, RabbitMQ. ASTPtr settings_changes; /// SELECT QUERY. Supported for MaterializedView and View (have to support LiveView). diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index b47623db50b..5f9857b28ef 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -19,6 +20,9 @@ void StorageSnapshot::init() { for (const auto & [name, type] : storage.getVirtuals()) virtual_columns[name] = type; + + if (storage.hasLightweightDeletedMask()) + system_columns[LightweightDeleteDescription::filter_column.name] = LightweightDeleteDescription::filter_column.type; } NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options) const @@ -28,13 +32,13 @@ NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options) if (options.with_extended_objects) extendObjectColumns(all_columns, object_columns, options.with_subcolumns); + NameSet column_names; if (options.with_virtuals) { /// Virtual columns must be appended after ordinary, /// because user can override them. if (!virtual_columns.empty()) { - NameSet column_names; for (const auto & column : all_columns) column_names.insert(column.name); @@ -44,6 +48,19 @@ NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options) } } + if (options.with_system_columns) + { + if (!system_columns.empty() && column_names.empty()) + { + for (const auto & column : all_columns) + column_names.insert(column.name); + } + + for (const auto & [name, type] : system_columns) + if (!column_names.contains(name)) + all_columns.emplace_back(name, type); + } + return all_columns; } @@ -76,6 +93,13 @@ std::optional StorageSnapshot::tryGetColumn(const GetColumnsOpt return NameAndTypePair(column_name, it->second); } + if (options.with_system_columns) + { + auto it = system_columns.find(column_name); + if (it != system_columns.end()) + return NameAndTypePair(column_name, it->second); + } + return {}; } diff --git a/src/Storages/StorageSnapshot.h b/src/Storages/StorageSnapshot.h index 6dad82dffd2..badf0d3a1e8 100644 --- a/src/Storages/StorageSnapshot.h +++ b/src/Storages/StorageSnapshot.h @@ -85,6 +85,10 @@ private: void init(); std::unordered_map virtual_columns; + + /// System columns are not visible in the schema but might be persisted in the data. + /// One example of such column is lightweight delete mask '_row_exists'. + std::unordered_map system_columns; }; using StorageSnapshotPtr = std::shared_ptr; diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index 5170b7d326c..8f60eb604b5 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -6,8 +6,6 @@ #include #include #include -#include "Core/NamesAndTypes.h" -#include "DataTypes/Serializations/ISerialization.h" namespace DB { @@ -129,9 +127,4 @@ struct TTLTableDescription static TTLTableDescription parse(const String & str, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key); }; -struct LightweightDeleteDescription -{ - NameAndTypePair filter_column; -}; - } diff --git a/tests/performance/lightweight_delete.xml b/tests/performance/lightweight_delete.xml index af7103f02d0..b29684f177f 100644 --- a/tests/performance/lightweight_delete.xml +++ b/tests/performance/lightweight_delete.xml @@ -19,8 +19,7 @@ 1 1 - 0 - 1 + 1 diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql index 6f78e1fe464..a0cdec12157 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql @@ -5,8 +5,7 @@ CREATE TABLE merge_table_standard_delete(id Int32, name String) ENGINE = MergeTr INSERT INTO merge_table_standard_delete select number, toString(number) from numbers(100); SET mutations_sync = 1; ---SET allow_experimental_lightweight_delete = 0; -SET allow_experimental_lightweight_delete_with_row_exists = 1; +SET allow_experimental_lightweight_delete = 1; DELETE FROM merge_table_standard_delete WHERE id = 10; diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree_compact_parts.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree_compact_parts.sql index f47560ba95b..c032c5bb9d1 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree_compact_parts.sql +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree_compact_parts.sql @@ -7,8 +7,7 @@ INSERT INTO merge_table_standard_delete select number, toString(number) from num SELECT COUNT(), part_type FROM system.parts WHERE database = currentDatabase() AND table = 'merge_table_standard_delete' AND active GROUP BY part_type ORDER BY part_type; SET mutations_sync = 1; ---SET allow_experimental_lightweight_delete = 0; -SET allow_experimental_lightweight_delete_with_row_exists = 1; +SET allow_experimental_lightweight_delete = 1; DELETE FROM merge_table_standard_delete WHERE id = 10; SELECT COUNT(), part_type FROM system.parts WHERE database = currentDatabase() AND table = 'merge_table_standard_delete' AND active GROUP BY part_type ORDER BY part_type; diff --git a/tests/queries/0_stateless/02352_lightweight_delete.sql b/tests/queries/0_stateless/02352_lightweight_delete.sql index 4468a25448c..4d0d53bb997 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete.sql +++ b/tests/queries/0_stateless/02352_lightweight_delete.sql @@ -5,8 +5,7 @@ CREATE TABLE lwd_test (id UInt64 , value String) ENGINE MergeTree() ORDER BY id; INSERT INTO lwd_test SELECT number, randomString(10) FROM system.numbers LIMIT 1000000; SET mutations_sync = 1; -SET allow_experimental_lightweight_delete_with_row_exists = 1; ---SET allow_experimental_lightweight_delete = 0; +SET allow_experimental_lightweight_delete = 1; SELECT 'Rows in parts', SUM(rows) FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_test' AND active; SELECT 'Count', count() FROM lwd_test; From be64b45583dc9c97dd63cb09cd0a51a003f3c0ba Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sat, 23 Jul 2022 01:14:03 +0200 Subject: [PATCH 571/659] Properly handle empty parts list --- .../MergeTree/StorageFromMergeTreeDataPart.h | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 9249306293d..79535d2b4ff 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -9,11 +9,17 @@ #include #include #include +#include namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + /// A Storage that allows reading from a single MergeTree data part. class StorageFromMergeTreeDataPart final : public IStorage { @@ -103,19 +109,21 @@ public: bool materializeTTLRecalculateOnly() const { + if (parts.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "parts must not be empty for materializeTTLRecalculateOnly"); return parts.front()->storage.getSettings()->materialize_ttl_recalculate_only; } bool hasLightweightDeletedMask() const override { - return parts.front()->hasLightweightDelete(); + return !parts.empty() && parts.front()->hasLightweightDelete(); } private: - MergeTreeData::DataPartsVector parts; + const MergeTreeData::DataPartsVector parts; const MergeTreeData & storage; - String partition_id; - MergeTreeDataSelectAnalysisResultPtr analysis_result_ptr; + const String partition_id; + const MergeTreeDataSelectAnalysisResultPtr analysis_result_ptr; static StorageID getIDFromPart(const MergeTreeData::DataPartPtr & part_) { From c8b3c574a435e31b9f32df13a4e9f46e7c8f33e0 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sat, 23 Jul 2022 13:55:43 +0200 Subject: [PATCH 572/659] Disable lightweight delete if table has projections --- src/Interpreters/MutationsInterpreter.cpp | 4 ++++ src/Storages/IStorage.h | 3 +++ src/Storages/MergeTree/IMergeTreeDataPart.cpp | 3 ++- src/Storages/MergeTree/MergeTreeData.cpp | 11 +++++++++++ src/Storages/MergeTree/MergeTreeData.h | 2 ++ .../MergeTree/StorageFromMergeTreeDataPart.h | 5 +++++ ...2319_lightweight_delete_on_merge_tree.reference | 2 ++ .../02319_lightweight_delete_on_merge_tree.sql | 14 ++++++++++++++ 8 files changed, 43 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 180e160aca6..64ea313a1af 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -352,7 +352,11 @@ static void validateUpdateColumns( /// Allow to override value of lightweight delete filter virtual column if (!found && column_name == LightweightDeleteDescription::filter_column.name) + { + if (!storage->supportsLightweightDelete()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); found = true; + } if (!found) { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 0a25a30ec6f..03e10f98b49 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -238,6 +238,9 @@ public: /// Return true if there is at least one part containing lightweight deleted mask. virtual bool hasLightweightDeletedMask() const { return false; } + /// Return true if storage can execute lightweight delete mutations. + virtual bool supportsLightweightDelete() const { return false; } + private: StorageID storage_id; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 36855fe48bb..f6b6d7bccdb 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1210,7 +1210,8 @@ void IMergeTreeDataPart::loadColumns(bool require) /// Project part / part with project parts / compact part doesn't support LWD. bool IMergeTreeDataPart::supportLightweightDeleteMutate() const { - return part_type == MergeTreeDataPartType::Wide && parent_part == nullptr && projection_parts.empty(); + return (part_type == MergeTreeDataPartType::Wide || part_type == MergeTreeDataPartType::Compact) && + parent_part == nullptr && projection_parts.empty(); } void IMergeTreeDataPart::assertHasVersionMetadata(MergeTreeTransaction * txn) const diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bd52ca3dfd7..d0f1fe75560 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4496,6 +4496,17 @@ MergeTreeData::DataPartsVector MergeTreeData::getAllDataPartsVector(MergeTreeDat return res; } +bool MergeTreeData::supportsLightweightDelete() const +{ + auto lock = lockParts(); + for (const auto & part : data_parts_by_info) + { + if (!part->supportLightweightDeleteMutate()) + return false; + } + return true; +} + MergeTreeData::ProjectionPartsVector MergeTreeData::getAllProjectionPartsVector(MergeTreeData::DataPartStateVector * out_states) const { ProjectionPartsVector res; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e18a3b20b74..ed5f6cc8b74 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -440,6 +440,8 @@ public: bool supportsDynamicSubcolumns() const override { return true; } + bool supportsLightweightDelete() const override; + NamesAndTypesList getVirtuals() const override; bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, ContextPtr, const StorageMetadataPtr & metadata_snapshot) const override; diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 79535d2b4ff..fd313a10bc8 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -119,6 +119,11 @@ public: return !parts.empty() && parts.front()->hasLightweightDelete(); } + bool supportsLightweightDelete() const override + { + return !parts.empty() && parts.front()->supportLightweightDeleteMutate(); + } + private: const MergeTreeData::DataPartsVector parts; const MergeTreeData & storage; diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference index 31960e2ecea..fec37f4306e 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.reference @@ -37,3 +37,5 @@ t_light 4 4_5_5_1_10 1 1 1000 -2 1005 -2 +----Test lighweight delete is disabled if table has projections----- +500.5 500.5 1000 diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql index a0cdec12157..67513a1cdff 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql @@ -94,3 +94,17 @@ CHECK TABLE t_large; SELECT * FROM t_large WHERE a in (1,1000,1005,50000) order by a; DROP TABLE t_large; + +SELECT '----Test lighweight delete is disabled if table has projections-----'; + +CREATE TABLE t_proj(a UInt32, b int) ENGINE=MergeTree order BY a settings min_bytes_for_wide_part=0; + +ALTER TABLE t_proj ADD PROJECTION p_1 (SELECT avg(a), avg(b), count()); + +INSERT INTO t_proj SELECT number + 1, number + 1 FROM numbers(1000); + +DELETE FROM t_proj WHERE a < 100; -- { serverError NOT_IMPLEMENTED } + +SELECT avg(a), avg(b), count() FROM t_proj; + +DROP TABLE t_proj; From 66927701139641e8826e4f10c2f2b278346238fb Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sat, 23 Jul 2022 16:42:21 +0200 Subject: [PATCH 573/659] Allow to update "system" virtual columns such as _row_exists --- src/Storages/MergeTree/MutateTask.cpp | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index cb041775ca6..59c784208c5 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -170,14 +170,15 @@ getColumnsForNewDataPart( NameToNameMap renamed_columns_to_from; NameToNameMap renamed_columns_from_to; ColumnsDescription part_columns(source_part->getColumns()); - const auto all_virtual_columns = source_part->storage.getVirtuals(); + NamesAndTypesList system_columns; + if (source_part->supportLightweightDeleteMutate()) + system_columns.push_back(LightweightDeleteDescription::filter_column); - /// Preserve virtual columns that have persisted values in the source_part -/// TODO: only allow LWD mask to be overridden! - for (const auto & virtual_column : all_virtual_columns) + /// Preserve system columns that have persisted values in the source_part + for (const auto & column : system_columns) { - if (part_columns.has(virtual_column.name) && !storage_columns.contains(virtual_column.name)) - storage_columns.emplace_back(virtual_column); + if (part_columns.has(column.name) && !storage_columns.contains(column.name)) + storage_columns.emplace_back(column); } /// All commands are validated in AlterCommand so we don't care about order @@ -187,11 +188,10 @@ getColumnsForNewDataPart( { for (const auto & [column_name, _] : command.column_to_update_expression) { - /// Allow to update and persist values of virtual column -/// TODO: only allow LWD mask to be overridden! - auto virtual_column = all_virtual_columns.tryGetByName(column_name); - if (virtual_column && !storage_columns.contains(column_name)) - storage_columns.emplace_back(column_name, virtual_column->type); + /// Allow to update and persist values of system column + auto column = system_columns.tryGetByName(column_name); + if (column && !storage_columns.contains(column_name)) + storage_columns.emplace_back(column_name, column->type); } } From 032fcab70e92347b910f5ec293442032c1cfebf3 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Sun, 24 Jul 2022 14:23:22 +0200 Subject: [PATCH 574/659] set char to be explicitely signed --- src/Common/base58.h | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Common/base58.h b/src/Common/base58.h index 5dbe48ac4b2..3d4b55a1fba 100644 --- a/src/Common/base58.h +++ b/src/Common/base58.h @@ -43,11 +43,13 @@ inline size_t encodeBase58(const char8_t * src, char8_t * dst) inline size_t decodeBase58(const char8_t * src, char8_t * dst) { - const char map_digits[128] - = {-1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 0, 1, 2, 3, 4, 5, 6, 7, 8, -1, -1, -1, -1, -1, -1, - -1, 9, 10, 11, 12, 13, 14, 15, 16, -1, 17, 18, 19, 20, 21, -1, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, -1, -1, -1, -1, -1, - -1, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, -1, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, -1, -1, -1, -1, -1}; + const signed char uint_max = UINT_MAX; + const signed char map_digits[128] + = {uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, + uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, + uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, + uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, 0, 1, 2, 3, 4, 5, 6, 7, 8, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, 9, 10, 11, 12, 13, 14, 15, 16, uint_max, 17, 18, 19, 20, 21, uint_max, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, + uint_max, uint_max, uint_max, uint_max, uint_max, uint_max, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, uint_max, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, uint_max, uint_max, uint_max, uint_max, uint_max}; size_t idx = 0; From c2954b23cba180e4abbc6282d7158a4adfa05899 Mon Sep 17 00:00:00 2001 From: Ignat Loskutov Date: Sun, 24 Jul 2022 20:55:17 +0800 Subject: [PATCH 575/659] LZ4_decompress_faster.cpp: remove endianness-dependent code Little-endian is little-endian no matter what the native endianness is. --- src/Compression/LZ4_decompress_faster.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 82a86a80d8d..269c1a681c5 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -24,13 +24,11 @@ #include #endif -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ static inline UInt16 LZ4_readLE16(const void* mem_ptr) { const UInt8* p = reinterpret_cast(mem_ptr); return static_cast(p[0]) + (p[1] << 8); } -#endif namespace LZ4 { @@ -569,11 +567,7 @@ bool NO_INLINE decompressImpl( /// Get match offset. -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ size_t offset = LZ4_readLE16(ip); -#else - size_t offset = unalignedLoad(ip); -#endif ip += 2; const UInt8 * match = op - offset; From 82f2557386b27fb70f8a56cda23c33268ecab7e8 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Sun, 24 Jul 2022 19:17:02 +0200 Subject: [PATCH 576/659] add Equinix metal N3 Xlarge --- website/benchmark/hardware/index.html | 1 + .../results/equinix_metal_n3_xlarge.json | 54 +++++++++++++++++++ 2 files changed, 55 insertions(+) create mode 100644 website/benchmark/hardware/results/equinix_metal_n3_xlarge.json diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index e4ec0f2102b..13622163ec3 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -101,6 +101,7 @@ Results for Huawei Taishan 920 are from Yu ZiChange at EioTek.
Results for Macbook Pro Intel Core i7 (2014) are from Vladislav.
Results for Hetzner EX62-NVME are from Talles Airan.
Results for AMD Ryzen 9 5950X are from Stefan.
+Results for Equinix metal n3.xlarge.x84 are from Dave Cottlehuber.

diff --git a/website/benchmark/hardware/results/equinix_metal_n3_xlarge.json b/website/benchmark/hardware/results/equinix_metal_n3_xlarge.json new file mode 100644 index 00000000000..c9fdb07c31c --- /dev/null +++ b/website/benchmark/hardware/results/equinix_metal_n3_xlarge.json @@ -0,0 +1,54 @@ +[ + { + "system": "Equinix metal n3.xlarge.x84", + "system_full": "Equinix metal n3.xlarge.x84 503 Gib local NVMe ", + "time": "2022-07-12 10:25:52", + "kind": "server", + "result": + [ + [0.010, 0.029, 0.003], + [0.387, 0.245, 0.198], + [0.089, 0.044, 0.049], + [0.128, 0.084, 0.070], + [0.210, 0.161, 0.173], + [0.321, 0.216, 0.272], + [0.011, 0.004, 0.004], + [0.109, 0.074, 0.064], + [0.353, 0.284, 0.287], + [0.437, 0.310, 0.302], + [0.194, 0.311, 0.280], + [0.194, 0.180, 0.148], + [0.351, 0.276, 0.263], + [0.381, 0.326, 0.313], + [0.353, 0.270, 0.271], + [0.324, 0.300, 0.306], + [0.839, 0.782, 0.786], + [0.608, 0.521, 0.548], + [1.387, 1.317, 1.339], + [0.090, 0.081, 0.087], + [0.623, 0.474, 0.428], + [0.581, 0.404, 0.395], + [1.137, 0.879, 0.878], + [0.794, 0.565, 0.577], + [0.211, 0.119, 0.116], + [0.162, 0.107, 0.100], + [0.230, 0.135, 0.132], + [0.606, 0.411, 0.406], + [0.724, 0.499, 0.491], + [0.638, 0.595, 0.609], + [0.289, 0.209, 0.222], + [0.469, 0.274, 0.313], + [2.042, 1.748, 1.697], + [1.484, 1.332, 1.262], + [1.347, 1.230, 1.175], + [0.516, 0.470, 0.471], + [0.142, 0.090, 0.099], + [0.078, 0.056, 0.045], + [0.084, 0.041, 0.057], + [0.248, 0.190, 0.218], + [0.057, 0.063, 0.048], + [0.059, 0.047, 0.026], + [0.018, 0.017, 0.009] + ] + } +] From cff712970eb7ad1617055bc83e5bb7ab437bca1d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Jul 2022 19:58:48 +0200 Subject: [PATCH 577/659] Add function `nowInBlock` --- src/Functions/blockNumber.cpp | 5 +- src/Functions/now.cpp | 1 - src/Functions/nowInBlock.cpp | 88 +++++++++++++++++++ src/Functions/registerFunctionsDateTime.cpp | 2 + .../0_stateless/02372_now_in_block.reference | 1 + .../0_stateless/02372_now_in_block.sql | 1 + 6 files changed, 96 insertions(+), 2 deletions(-) create mode 100644 src/Functions/nowInBlock.cpp create mode 100644 tests/queries/0_stateless/02372_now_in_block.reference create mode 100644 tests/queries/0_stateless/02372_now_in_block.sql diff --git a/src/Functions/blockNumber.cpp b/src/Functions/blockNumber.cpp index e88fc68d01f..a62a0ac873f 100644 --- a/src/Functions/blockNumber.cpp +++ b/src/Functions/blockNumber.cpp @@ -44,7 +44,10 @@ public: return 0; } - bool isDeterministic() const override { return false; } + bool isDeterministic() const override + { + return false; + } bool isDeterministicInScopeOfQuery() const override { diff --git a/src/Functions/now.cpp b/src/Functions/now.cpp index 95ab1a652a8..b2cfa7aa2b1 100644 --- a/src/Functions/now.cpp +++ b/src/Functions/now.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include diff --git a/src/Functions/nowInBlock.cpp b/src/Functions/nowInBlock.cpp new file mode 100644 index 00000000000..b657bc92085 --- /dev/null +++ b/src/Functions/nowInBlock.cpp @@ -0,0 +1,88 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +namespace +{ + +/** Returns current time at calculation of every block. + * In contrast to 'now' function, it's not a constant expression and is not a subject of constant folding. + */ +class FunctionNowInBlock : public IFunction +{ +public: + static constexpr auto name = "nowInBlock"; + static FunctionPtr create(ContextPtr) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override + { + return false; + } + + /// Optional timezone argument. + bool isVariadic() const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } + + bool isDeterministic() const override + { + return false; + } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() > 1) + { + throw Exception("Arguments size of function " + getName() + " should be 0 or 1", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + if (arguments.size() == 1 && !isStringOrFixedString(arguments[0].type)) + { + throw Exception( + "Arguments of function " + getName() + " should be String or FixedString", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + if (arguments.size() == 1) + { + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 0, 0)); + } + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + return ColumnUInt32::create(input_rows_count, time(nullptr)); + } +}; + +} + +void registerFunctionNowInBlock(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/registerFunctionsDateTime.cpp b/src/Functions/registerFunctionsDateTime.cpp index 5f733b07281..b3fcca42bed 100644 --- a/src/Functions/registerFunctionsDateTime.cpp +++ b/src/Functions/registerFunctionsDateTime.cpp @@ -43,6 +43,7 @@ void registerFunctionToRelativeMinuteNum(FunctionFactory &); void registerFunctionToRelativeSecondNum(FunctionFactory &); void registerFunctionToTime(FunctionFactory &); void registerFunctionNow(FunctionFactory &); +void registerFunctionNowInBlock(FunctionFactory &); void registerFunctionNow64(FunctionFactory &); void registerFunctionToday(FunctionFactory &); void registerFunctionYesterday(FunctionFactory &); @@ -126,6 +127,7 @@ void registerFunctionsDateTime(FunctionFactory & factory) registerFunctionToTime(factory); registerFunctionNow(factory); registerFunctionNow64(factory); + registerFunctionNowInBlock(factory); registerFunctionToday(factory); registerFunctionYesterday(factory); registerFunctionTimeSlot(factory); diff --git a/tests/queries/0_stateless/02372_now_in_block.reference b/tests/queries/0_stateless/02372_now_in_block.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/02372_now_in_block.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/02372_now_in_block.sql b/tests/queries/0_stateless/02372_now_in_block.sql new file mode 100644 index 00000000000..cc7af3da4ad --- /dev/null +++ b/tests/queries/0_stateless/02372_now_in_block.sql @@ -0,0 +1 @@ +SELECT count() FROM (SELECT DISTINCT nowInBlock() FROM system.numbers LIMIT 2); From c604df7b530d24d7474102a10b9b868de1300be0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Jul 2022 20:05:31 +0200 Subject: [PATCH 578/659] Add smoke test for timezone --- tests/queries/0_stateless/02372_now_in_block.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02372_now_in_block.sql b/tests/queries/0_stateless/02372_now_in_block.sql index cc7af3da4ad..610c6c01faa 100644 --- a/tests/queries/0_stateless/02372_now_in_block.sql +++ b/tests/queries/0_stateless/02372_now_in_block.sql @@ -1 +1 @@ -SELECT count() FROM (SELECT DISTINCT nowInBlock() FROM system.numbers LIMIT 2); +SELECT count() FROM (SELECT DISTINCT nowInBlock(), nowInBlock('Pacific/Pitcairn') FROM system.numbers LIMIT 2); From dfbfc12a10ff23143dadb7a588ce126269549329 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Jul 2022 20:07:14 +0200 Subject: [PATCH 579/659] More tests --- tests/queries/0_stateless/02372_now_in_block.reference | 1 + tests/queries/0_stateless/02372_now_in_block.sql | 3 +++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02372_now_in_block.reference b/tests/queries/0_stateless/02372_now_in_block.reference index 0cfbf08886f..5f1d0ecea5d 100644 --- a/tests/queries/0_stateless/02372_now_in_block.reference +++ b/tests/queries/0_stateless/02372_now_in_block.reference @@ -1 +1,2 @@ 2 +1 diff --git a/tests/queries/0_stateless/02372_now_in_block.sql b/tests/queries/0_stateless/02372_now_in_block.sql index 610c6c01faa..7c884c0ba7d 100644 --- a/tests/queries/0_stateless/02372_now_in_block.sql +++ b/tests/queries/0_stateless/02372_now_in_block.sql @@ -1 +1,4 @@ SELECT count() FROM (SELECT DISTINCT nowInBlock(), nowInBlock('Pacific/Pitcairn') FROM system.numbers LIMIT 2); +SELECT nowInBlock(1); -- { serverError 43 } +SELECT nowInBlock(NULL) IS NULL; +SELECT nowInBlock('UTC', 'UTC'); -- { serverError 42 } From 4333750985f289aaff4c77979e2d0bf7f574e7cc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 24 Jul 2022 18:33:52 +0000 Subject: [PATCH 580/659] Less usage of StringRef ... replaced by std::string_view, see #39262 --- base/base/JSON.cpp | 6 +++--- base/base/JSON.h | 4 ++-- src/Common/Config/AbstractConfigurationComparison.cpp | 2 +- src/Common/Volnitsky.h | 1 - src/IO/WriteBufferFromString.h | 2 +- src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp | 6 +++--- src/Storages/Distributed/DistributedSink.cpp | 6 +++--- 7 files changed, 13 insertions(+), 14 deletions(-) diff --git a/base/base/JSON.cpp b/base/base/JSON.cpp index a01063ee426..92350ea0e18 100644 --- a/base/base/JSON.cpp +++ b/base/base/JSON.cpp @@ -669,18 +669,18 @@ std::string JSON::getName() const return getString(); } -StringRef JSON::getRawString() const +std::string_view JSON::getRawString() const { Pos s = ptr_begin; if (*s != '"') throw JSONException(std::string("JSON: expected \", got ") + *s); while (++s != ptr_end && *s != '"'); if (s != ptr_end) - return StringRef(ptr_begin + 1, s - ptr_begin - 1); + return std::string_view(ptr_begin + 1, s - ptr_begin - 1); throw JSONException("JSON: incorrect syntax (expected end of string, found end of JSON)."); } -StringRef JSON::getRawName() const +std::string_view JSON::getRawName() const { return getRawString(); } diff --git a/base/base/JSON.h b/base/base/JSON.h index 3226a0d09e8..214e9f88e9b 100644 --- a/base/base/JSON.h +++ b/base/base/JSON.h @@ -136,8 +136,8 @@ public: std::string getName() const; /// Получить имя name-value пары. JSON getValue() const; /// Получить значение name-value пары. - StringRef getRawString() const; - StringRef getRawName() const; + std::string_view getRawString() const; + std::string_view getRawName() const; /// Получить значение элемента; если элемент - строка, то распарсить значение из строки; если не строка или число - то исключение. double toDouble() const; diff --git a/src/Common/Config/AbstractConfigurationComparison.cpp b/src/Common/Config/AbstractConfigurationComparison.cpp index 711c754743d..c00a1efeebe 100644 --- a/src/Common/Config/AbstractConfigurationComparison.cpp +++ b/src/Common/Config/AbstractConfigurationComparison.cpp @@ -73,7 +73,7 @@ bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const S /// Check that the right configuration has the same set of subkeys as the left configuration. Poco::Util::AbstractConfiguration::Keys right_subkeys; right.keys(right_key, right_subkeys); - std::unordered_set left_subkeys{subkeys.begin(), subkeys.end()}; + std::unordered_set left_subkeys{subkeys.begin(), subkeys.end()}; if ((left_subkeys.size() != right_subkeys.size()) || (left_subkeys.size() != subkeys.size())) return false; for (const auto & right_subkey : right_subkeys) diff --git a/src/Common/Volnitsky.h b/src/Common/Volnitsky.h index a6aef293ac1..d7ca7d35277 100644 --- a/src/Common/Volnitsky.h +++ b/src/Common/Volnitsky.h @@ -9,7 +9,6 @@ #include #include #include -#include #include /** Search for a substring in a string by Volnitsky's algorithm diff --git a/src/IO/WriteBufferFromString.h b/src/IO/WriteBufferFromString.h index f4515254e6e..1f813b1070e 100644 --- a/src/IO/WriteBufferFromString.h +++ b/src/IO/WriteBufferFromString.h @@ -30,7 +30,7 @@ class WriteBufferFromOwnString : public detail::StringHolder, public WriteBuffer public: WriteBufferFromOwnString() : WriteBufferFromString(value) {} - StringRef stringRef() const { return isFinished() ? StringRef(value) : StringRef(value.data(), pos - value.data()); } + std::string_view stringView() const { return isFinished() ? std::string_view(value) : std::string_view(value.data(), pos - value.data()); } std::string & str() { diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp index 91183ebf633..a470e193300 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp @@ -178,7 +178,7 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr { WriteBufferFromOwnString buf; writeBinary(uuid_column.getElement(row_num), buf); - std::string_view uuid_bin = buf.stringRef().toView(); + std::string_view uuid_bin = buf.stringView(); packer.pack_bin(uuid_bin.size()); packer.pack_bin_body(uuid_bin.data(), uuid_bin.size()); return; @@ -187,7 +187,7 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr { WriteBufferFromOwnString buf; writeText(uuid_column.getElement(row_num), buf); - std::string_view uuid_text = buf.stringRef().toView(); + std::string_view uuid_text = buf.stringView(); packer.pack_str(uuid_text.size()); packer.pack_bin_body(uuid_text.data(), uuid_text.size()); return; @@ -198,7 +198,7 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr UUID value = uuid_column.getElement(row_num); writeBinaryBigEndian(value.toUnderType().items[0], buf); writeBinaryBigEndian(value.toUnderType().items[1], buf); - std::string_view uuid_ext = buf.stringRef().toView(); + std::string_view uuid_ext = buf.stringView(); packer.pack_ext(sizeof(UUID), int8_t(MsgPackExtensionTypes::UUIDType)); packer.pack_ext_body(uuid_ext.data(), uuid_ext.size()); return; diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 798de060768..13c085d650b 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -764,10 +764,10 @@ void DistributedSink::writeToShard(const Block & block, const std::vector Date: Mon, 18 Jul 2022 19:02:22 +0000 Subject: [PATCH 581/659] Add a sentence on writing perf tests for SQL functions --- docs/en/development/tests.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 8ce38178020..4d85934f730 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -81,11 +81,11 @@ $ ./src/unit_tests_dbms --gtest_filter=LocalAddress* ## Performance Tests {#performance-tests} -Performance tests allow to measure and compare performance of some isolated part of ClickHouse on synthetic queries. Tests are located at `tests/performance`. Each test is represented by `.xml` file with description of test case. Tests are run with `docker/test/performance-comparison` tool . See the readme file for invocation. +Performance tests allow to measure and compare performance of some isolated part of ClickHouse on synthetic queries. Performance tests are located at `tests/performance/`. Each test is represented by an `.xml` file with a description of the test case. Tests are run with `docker/test/performance-comparison` tool . See the readme file for invocation. Each test run one or multiple queries (possibly with combinations of parameters) in a loop. -If you want to improve performance of ClickHouse in some scenario, and if improvements can be observed on simple queries, it is highly recommended to write a performance test. It always makes sense to use `perf top` or other `perf` tools during your tests. +If you want to improve performance of ClickHouse in some scenario, and if improvements can be observed on simple queries, it is highly recommended to write a performance test. Also, it is recommended to write performance tests when you add or modify SQL functions which are relatively isolated and not too obscure. It always makes sense to use `perf top` or other `perf` tools during your tests. ## Test Tools and Scripts {#test-tools-and-scripts} From b09121df11eb9a15597f0b4f239a2245ad09408c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Jul 2022 19:05:32 +0000 Subject: [PATCH 582/659] Remove comment to check for CMake version a) the build script aborts anyways if CMake is too old b) our minimally required version is >3 years old, so the chance of an abort due to outdated CMake is fairly low c) removing the hint to check the version removes the need to constantly update the version in the docs --- docs/en/development/developer-instruction.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index 77ddae6a756..dc3bb07f7c1 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -124,8 +124,6 @@ For installing CMake and Ninja on Mac OS X first install Homebrew and then insta /usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)" brew install cmake ninja -Next, check the version of CMake: `cmake --version`. If it is below 3.12, you should install a newer version from the website: https://cmake.org/download/. - ## C++ Compiler {#c-compiler} Compilers Clang starting from version 11 is supported for building ClickHouse. From cec8458429b2192e7ec8910b9435526b7480006a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Jul 2022 19:05:53 +0000 Subject: [PATCH 583/659] Remove Mac instructions ... they are given at length on the dedicated setup page for Mac --- docs/en/development/developer-instruction.md | 8 -------- 1 file changed, 8 deletions(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index dc3bb07f7c1..53443fd12bb 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -119,11 +119,6 @@ On CentOS, RedHat run `sudo yum install cmake ninja-build`. If you use Arch or Gentoo, you probably know it yourself how to install CMake. -For installing CMake and Ninja on Mac OS X first install Homebrew and then install everything else via brew: - - /usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)" - brew install cmake ninja - ## C++ Compiler {#c-compiler} Compilers Clang starting from version 11 is supported for building ClickHouse. @@ -136,9 +131,6 @@ On Ubuntu/Debian you can use the automatic installation script (check [official sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" ``` -Mac OS X build is also supported. Just run `brew install llvm` - - ## The Building Process {#the-building-process} Now that you are ready to build ClickHouse we recommend you to create a separate directory `build` inside `ClickHouse` that will contain all of the build artefacts: From 2ded3da8876ce497c365e2eb21abd8d2fa5797e4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Jul 2022 19:06:50 +0000 Subject: [PATCH 584/659] Bump Clang version to current minimum --- docs/en/development/developer-instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index 53443fd12bb..ea2ed95fd27 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -121,7 +121,7 @@ If you use Arch or Gentoo, you probably know it yourself how to install CMake. ## C++ Compiler {#c-compiler} -Compilers Clang starting from version 11 is supported for building ClickHouse. +Compilers Clang starting from version 12 is supported for building ClickHouse. Clang should be used instead of gcc. Though, our continuous integration (CI) platform runs checks for about a dozen of build combinations. From a9fb677084884ba345ae81aa2832d5306ec08e80 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Jul 2022 19:12:02 +0000 Subject: [PATCH 585/659] Bump Clang version to current minimum --- docs/en/development/style.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/style.md b/docs/en/development/style.md index 0c99426db03..25c51527b88 100644 --- a/docs/en/development/style.md +++ b/docs/en/development/style.md @@ -745,7 +745,7 @@ But other things being equal, cross-platform or portable code is preferred. **2.** Language: C++20 (see the list of available [C++20 features](https://en.cppreference.com/w/cpp/compiler_support#C.2B.2B20_features)). -**3.** Compiler: `clang`. At this time (April 2021), the code is compiled using clang version 11. (It can also be compiled using `gcc` version 10, but it's untested and not suitable for production usage). +**3.** Compiler: `clang`. At the time of writing (July 2022), the code is compiled using clang version >= 12. (It can also be compiled using `gcc`, but it's untested and not suitable for production usage). The standard library is used (`libc++`). From f3a60991f4135c516ba1036f9e9f843c62e41592 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Jul 2022 19:12:33 +0000 Subject: [PATCH 586/659] Smallish compiler warnings update --- docs/en/development/style.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/style.md b/docs/en/development/style.md index 25c51527b88..e949aba9ea2 100644 --- a/docs/en/development/style.md +++ b/docs/en/development/style.md @@ -755,7 +755,7 @@ The standard library is used (`libc++`). The CPU instruction set is the minimum supported set among our servers. Currently, it is SSE 4.2. -**6.** Use `-Wall -Wextra -Werror` compilation flags. Also `-Weverything` is used with few exceptions. +**6.** Use `-Wall -Wextra -Werror -Weverything` compilation flags with a few exception. **7.** Use static linking with all libraries except those that are difficult to connect to statically (see the output of the `ldd` command). From 68f0dcc20624a25cef903b202e304a3403b4f7e3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Jul 2022 19:13:00 +0000 Subject: [PATCH 587/659] Remove mention of C++03 exception specifiers ... such exception specifiers trigger a compiler warning these days anyways (i.e. are treated as error) --- docs/en/development/style.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/en/development/style.md b/docs/en/development/style.md index e949aba9ea2..a543c7532f8 100644 --- a/docs/en/development/style.md +++ b/docs/en/development/style.md @@ -692,9 +692,7 @@ auto s = std::string{"Hello"}; **1.** Virtual inheritance is not used. -**2.** Exception specifiers from C++03 are not used. - -**3.** Constructs which have convenient syntactic sugar in modern C++, e.g. +**2.** Constructs which have convenient syntactic sugar in modern C++, e.g. ``` // Traditional way without syntactic sugar From b6121a1bf1e13a034df2acc702935701ac981f9f Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 24 Jul 2022 17:07:47 -0300 Subject: [PATCH 588/659] Update star-schema.md --- .../getting-started/example-datasets/star-schema.md | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/docs/ru/getting-started/example-datasets/star-schema.md b/docs/ru/getting-started/example-datasets/star-schema.md index 9918ee2910d..e6a8f011cde 100644 --- a/docs/ru/getting-started/example-datasets/star-schema.md +++ b/docs/ru/getting-started/example-datasets/star-schema.md @@ -15,16 +15,15 @@ $ make Генерация данных: -:::danger "Внимание" +:::warning "Внимание" -s 100 – dbgen генерирует 600 миллионов строк (67 ГБ) -s 1000 – dbgen генерирует 6 миллиардов строк (занимает много времени) +::: ``` bash $ ./dbgen -s 1000 -T c $ ./dbgen -s 1000 -T l $ ./dbgen -s 1000 -T p -$ ./dbgen -s 1000 -T s -$ ./dbgen -s 1000 -T d ``` Создание таблиц в Кликхауз: @@ -105,11 +104,10 @@ $ clickhouse-client --query "INSERT INTO lineorder FORMAT CSV" < lineorder.tbl ``` sql SET max_memory_usage = 20000000000; + CREATE TABLE lineorder_flat -ENGINE = MergeTree -PARTITION BY toYear(LO_ORDERDATE) -ORDER BY (LO_ORDERDATE, LO_ORDERKEY) AS -SELECT +ENGINE = MergeTree ORDER BY (LO_ORDERDATE, LO_ORDERKEY) +AS SELECT l.LO_ORDERKEY AS LO_ORDERKEY, l.LO_LINENUMBER AS LO_LINENUMBER, l.LO_CUSTKEY AS LO_CUSTKEY, From 2785783defbf63ef25ed7b214baa30ce6da84e04 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 20 Jul 2022 09:05:43 +0000 Subject: [PATCH 589/659] Mark DEFLAPT_QPL as 'experimental' codec + cosmetics --- .../sql-reference/statements/create/table.md | 8 ++-- .../CompressionCodecDeflateQpl.cpp | 13 +++--- src/Compression/CompressionCodecDeflateQpl.h | 43 ++++++++----------- .../02372_qpl_is_experimental.reference | 0 .../0_stateless/02372_qpl_is_experimental.sql | 9 ++++ 5 files changed, 39 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/02372_qpl_is_experimental.reference create mode 100644 tests/queries/0_stateless/02372_qpl_is_experimental.sql diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 0c2e87fbcac..0e033456998 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -250,10 +250,12 @@ High compression levels are useful for asymmetric scenarios, like compress once, #### DEFLATE_QPL -`DEFLATE_QPL` — [Deflate compression algorithm](https://github.com/intel/qpl) implemented by Intel® Query Processing Library, which has dependency on Intel Hardware: +`DEFLATE_QPL` — [Deflate compression algorithm](https://github.com/intel/qpl) implemented by Intel® Query Processing Library. Some limitations apply: -- DEFLATE_QPL is only supported on systems with AVX2/AVX512/IAA. -- DEFLATE_QPL-compressed data can only be transferred between nodes with AVX2/AVX512/IAA. +- DEFLATE_QPL is experimental and can only be used after setting configuration parameter `allow_experimental_codecs=1`. +- DEFLATE_QPL only works if ClickHouse was compiled with support for AVX2 or AVX512 instructions +- DEFLATE_QPL works best if the system has a Intel® IAA (In-Memory Analytics Accelerator) offloading device +- DEFLATE_QPL-compressed data can only be transferred between ClickHouse nodes compiled with support for AVX2/AVX512 ### Specialized Codecs diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 81ec7ee5dca..9e165a9c913 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -28,8 +28,8 @@ DeflateQplJobHWPool & DeflateQplJobHWPool::instance() } DeflateQplJobHWPool::DeflateQplJobHWPool() - :random_engine(std::random_device()()) - ,distribution(0, MAX_HW_JOB_NUMBER-1) + : random_engine(std::random_device()()) + , distribution(0, MAX_HW_JOB_NUMBER - 1) { Poco::Logger * log = &Poco::Logger::get("DeflateQplJobHWPool"); UInt32 job_size = 0; @@ -73,7 +73,7 @@ DeflateQplJobHWPool::~DeflateQplJobHWPool() job_pool_ready = false; } -qpl_job * DeflateQplJobHWPool::acquireJob(UInt32 &job_id) +qpl_job * DeflateQplJobHWPool::acquireJob(UInt32 & job_id) { if (isJobPoolReady()) { @@ -141,7 +141,7 @@ HardwareCodecDeflateQpl::~HardwareCodecDeflateQpl() Int32 HardwareCodecDeflateQpl::doCompressData(const char * source, UInt32 source_size, char * dest, UInt32 dest_size) const { UInt32 job_id = 0; - qpl_job* job_ptr = nullptr; + qpl_job * job_ptr = nullptr; UInt32 compressed_size = 0; if (!(job_ptr = DeflateQplJobHWPool::instance().acquireJob(job_id))) { @@ -330,10 +330,9 @@ void SoftwareCodecDeflateQpl::doDecompressData(const char * source, UInt32 sourc "Execution of DeflateQpl software fallback codec failed. (Details: qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); } -//CompressionCodecDeflateQpl CompressionCodecDeflateQpl::CompressionCodecDeflateQpl() - :hw_codec(std::make_unique()) - ,sw_codec(std::make_unique()) + : hw_codec(std::make_unique()) + , sw_codec(std::make_unique()) { setCodecDescription("DEFLATE_QPL"); } diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index c15f537fd3f..1a13a1ca42c 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -1,8 +1,9 @@ #pragma once #include -#include +#include #include +#include namespace Poco { @@ -18,20 +19,16 @@ class DeflateQplJobHWPool { public: DeflateQplJobHWPool(); - ~DeflateQplJobHWPool(); - qpl_job * acquireJob(UInt32 &job_id); - - static void releaseJob(UInt32 job_id); - - static const bool & isJobPoolReady() { return job_pool_ready; } - static DeflateQplJobHWPool & instance(); + qpl_job * acquireJob(UInt32 & job_id); + static void releaseJob(UInt32 job_id); + static const bool & isJobPoolReady() { return job_pool_ready; } + private: static bool tryLockJob(UInt32 index); - static void unLockJob(UInt32 index); /// Maximum jobs running in parallel supported by IAA hardware @@ -39,9 +36,9 @@ private: /// Entire buffer for storing all job objects static std::unique_ptr hw_jobs_buffer; /// Job pool for storing all job object pointers - static std::array hw_job_ptr_pool; + static std::array hw_job_ptr_pool; /// Locks for accessing each job object pointers - static std::array hw_job_ptr_locks; + static std::array hw_job_ptr_locks; static bool job_pool_ready; std::mt19937 random_engine; std::uniform_int_distribution distribution; @@ -57,23 +54,25 @@ public: private: qpl_job * sw_job = nullptr; std::unique_ptr sw_buffer; + qpl_job * getJobCodecPtr(); }; class HardwareCodecDeflateQpl { public: - /// RET_ERROR stands for hardware codec fail,need fallback to software codec. + /// RET_ERROR stands for hardware codec fail, needs fallback to software codec. static constexpr Int32 RET_ERROR = -1; HardwareCodecDeflateQpl(); ~HardwareCodecDeflateQpl(); + Int32 doCompressData(const char * source, UInt32 source_size, char * dest, UInt32 dest_size) const; - ///Submit job request to the IAA hardware and then busy waiting till it complete. + /// Submit job request to the IAA hardware and then busy waiting till it complete. Int32 doDecompressDataSynchronous(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size); - ///Submit job request to the IAA hardware and return immediately. IAA hardware will process decompression jobs automatically. + /// Submit job request to the IAA hardware and return immediately. IAA hardware will process decompression jobs automatically. Int32 doDecompressDataAsynchronous(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size); /// Flush result for all previous requests which means busy waiting till all the jobs in "decomp_async_job_map" are finished. @@ -96,23 +95,19 @@ public: void updateHash(SipHash & hash) const override; protected: - bool isCompression() const override - { - return true; - } - - bool isGenericCompression() const override - { - return true; - } + bool isCompression() const override { return true; } + bool isGenericCompression() const override { return true; } + bool isExperimental() const override { return true; } UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; - ///Flush result for previous asynchronous decompression requests on asynchronous mode. + + /// Flush result for previous asynchronous decompression requests on asynchronous mode. void flushAsynchronousDecompressRequests() override; private: UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + std::unique_ptr hw_codec; std::unique_ptr sw_codec; }; diff --git a/tests/queries/0_stateless/02372_qpl_is_experimental.reference b/tests/queries/0_stateless/02372_qpl_is_experimental.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02372_qpl_is_experimental.sql b/tests/queries/0_stateless/02372_qpl_is_experimental.sql new file mode 100644 index 00000000000..c2bebbd8e9c --- /dev/null +++ b/tests/queries/0_stateless/02372_qpl_is_experimental.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS qpl_codec; + +CREATE TABLE qpl_codec (id Int32 CODEC(DEFLATE_QPL)) ENGINE = MergeTree() ORDER BY id; -- { serverError 36 } + +SET allow_experimental_codecs = 1; +CREATE TABLE qpl_codec (id Int32 CODEC(DEFLATE_QPL)) ENGINE = MergeTree() ORDER BY id; + +DROP TABLE IF EXISTS qpl_codec; + From a862e08fc4c4201c0bc4fcbee90c46dfc9a8fc4e Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 24 Jul 2022 17:10:36 -0300 Subject: [PATCH 590/659] Update star-schema.md --- .../getting-started/example-datasets/star-schema.md | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/docs/zh/getting-started/example-datasets/star-schema.md b/docs/zh/getting-started/example-datasets/star-schema.md index 1ae2a224ec1..47044f7b555 100644 --- a/docs/zh/getting-started/example-datasets/star-schema.md +++ b/docs/zh/getting-started/example-datasets/star-schema.md @@ -15,15 +15,14 @@ $ make 开始生成数据: -!!! warning "注意" +:::warning "注意" 使用`-s 100`dbgen 将生成 6 亿行数据(67GB), 如果使用`-s 1000`它会生成 60 亿行数据(这需要很多时间)) +::: ```bash $ ./dbgen -s 1000 -T c $ ./dbgen -s 1000 -T l $ ./dbgen -s 1000 -T p -$ ./dbgen -s 1000 -T s -$ ./dbgen -s 1000 -T d ``` 在 ClickHouse 中创建数据表: @@ -106,10 +105,8 @@ $ clickhouse-client --query "INSERT INTO lineorder FORMAT CSV" < lineorder.tbl SET max_memory_usage = 20000000000; CREATE TABLE lineorder_flat -ENGINE = MergeTree -PARTITION BY toYear(LO_ORDERDATE) -ORDER BY (LO_ORDERDATE, LO_ORDERKEY) AS -SELECT +ENGINE = MergeTree ORDER BY (LO_ORDERDATE, LO_ORDERKEY) +AS SELECT l.LO_ORDERKEY AS LO_ORDERKEY, l.LO_LINENUMBER AS LO_LINENUMBER, l.LO_CUSTKEY AS LO_CUSTKEY, From b1da2eb9e63ad521f84244e237c33376a93879b6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 24 Jul 2022 20:42:38 +0000 Subject: [PATCH 591/659] Remove tests again ... They broke because QPL is compiled only when AVX2/512 is on. This is not the case in CI. --- .../0_stateless/02372_qpl_is_experimental.reference | 0 tests/queries/0_stateless/02372_qpl_is_experimental.sql | 9 --------- 2 files changed, 9 deletions(-) delete mode 100644 tests/queries/0_stateless/02372_qpl_is_experimental.reference delete mode 100644 tests/queries/0_stateless/02372_qpl_is_experimental.sql diff --git a/tests/queries/0_stateless/02372_qpl_is_experimental.reference b/tests/queries/0_stateless/02372_qpl_is_experimental.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02372_qpl_is_experimental.sql b/tests/queries/0_stateless/02372_qpl_is_experimental.sql deleted file mode 100644 index c2bebbd8e9c..00000000000 --- a/tests/queries/0_stateless/02372_qpl_is_experimental.sql +++ /dev/null @@ -1,9 +0,0 @@ -DROP TABLE IF EXISTS qpl_codec; - -CREATE TABLE qpl_codec (id Int32 CODEC(DEFLATE_QPL)) ENGINE = MergeTree() ORDER BY id; -- { serverError 36 } - -SET allow_experimental_codecs = 1; -CREATE TABLE qpl_codec (id Int32 CODEC(DEFLATE_QPL)) ENGINE = MergeTree() ORDER BY id; - -DROP TABLE IF EXISTS qpl_codec; - From 0492f23d072b104e012230aee16219e472b48ae7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Jul 2022 23:11:54 +0200 Subject: [PATCH 592/659] Add the docs --- .../sql-reference/functions/date-time-functions.md | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 621429fb02c..d6ef387840a 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -836,7 +836,7 @@ Result: ## now -Returns the current date and time. +Returns the current date and time at the moment of query analysis. The function is a constant expression. **Syntax** @@ -884,14 +884,20 @@ Result: └──────────────────────┘ ``` +## nowInBlock + +Returns the current date and time at the moment of processing of each block of data. In contrast to the function `now`, it is not a constant expression, and the returned value will be different in different blocks for long-running queries. + +It makes sense to use this function to generate the current time in long-running INSERT SELECT queries. + ## today -Accepts zero arguments and returns the current date at one of the moments of request execution. +Accepts zero arguments and returns the current date at one of the moments of query analysis. The same as ‘toDate(now())’. ## yesterday -Accepts zero arguments and returns yesterday’s date at one of the moments of request execution. +Accepts zero arguments and returns yesterday’s date at one of the moments of query analysis. The same as ‘today() - 1’. ## timeSlot From 93f876b58afc2ad7e5eeabfbe23c590e08751557 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Jul 2022 14:52:41 +0300 Subject: [PATCH 593/659] Add a test for LSan (cherry picked from commit 51e7c41883f9e0f290c9b5c7a1853111ac52492a) v2: fix type check Signed-off-by: Azat Khuzhin --- src/Common/tests/gtest_lsan.cpp | 33 +++++++++++++++++++++++++++++++++ 1 file changed, 33 insertions(+) create mode 100644 src/Common/tests/gtest_lsan.cpp diff --git a/src/Common/tests/gtest_lsan.cpp b/src/Common/tests/gtest_lsan.cpp new file mode 100644 index 00000000000..f6e1984ec58 --- /dev/null +++ b/src/Common/tests/gtest_lsan.cpp @@ -0,0 +1,33 @@ +#include // ADDRESS_SANITIZER + +#ifdef ADDRESS_SANITIZER + +#include +#include + +#include +#include + +/// Test that ensures that LSan works. +/// +/// Regression test for the case when it may not work, +/// because of broken getauxval() [1]. +/// +/// [1]: https://github.com/ClickHouse/ClickHouse/pull/33957 +TEST(Common, LSan) +{ + int sanitizers_exit_code = 1; + + ASSERT_EXIT({ + std::thread leak_in_thread([]() + { + void * leak = malloc(4096); + ASSERT_NE(leak, nullptr); + }); + leak_in_thread.join(); + + __lsan_do_leak_check(); + }, ::testing::ExitedWithCode(sanitizers_exit_code), ".*LeakSanitizer: detected memory leaks.*"); +} + +#endif From a4f9e1da5b1c739fd0dfc2d564c57ccf2fa2cf0a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Jul 2022 17:26:00 +0300 Subject: [PATCH 594/659] Fix getauxval() by using procfs with fallback to environ getauxval() from glibc-compatibility did not work always correctly: - It does not work after setenv(), and this breaks vsyscalls, like sched_getcpu() [1] (and BaseDaemon.cpp always set TZ if timezone is defined, which is true for CI [2]). Also note, that fixing setenv() will not fix LSan, since the culprit is getauxval() [1]: https://bugzilla.redhat.com/show_bug.cgi?id=1163404 [2]: ClickHouse#32928 (comment) - Another think that is definitely broken is LSan (Leak Sanitizer), it relies on worked getauxval() but it does not work if __environ is not initialized yet (there is even a commit about this). And because of, at least, one leak had been introduced [3]: [3]: ClickHouse#33840 Fix this by using /proc/self/auxv with fallback to environ solution to make it compatible with environment that does not allow reading from auxv (or no procfs). v2: add fallback to environ solution v3: fix return value for __auxv_init_procfs() (cherry picked from commit f187c3499a9efe05af109aa8011f384f0b14297e) v4: more verbose message on errors, CI founds [1]: AUXV already has value (529267711) [1]: https://s3.amazonaws.com/clickhouse-test-reports/39103/2325f7e8442d1672ce5fb43b11039b6a8937e298/stress_test__memory__actions_.html v5: break at AT_NULL v6: ignore AT_IGNORE v7: suppress TSan and remove superior check to avoid abort() in case of race v8: proper suppressions (not inner function but itself) Refs: #33957 Signed-off-by: Azat Khuzhin --- base/glibc-compatibility/musl/getauxval.c | 174 ++++++++++++++++++---- 1 file changed, 149 insertions(+), 25 deletions(-) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index dad7aa938d7..22886013d07 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -1,68 +1,192 @@ -#include #include "atomic.h" -#include // __environ +#include +#include // open +#include // O_RDONLY +#include // read, close +#include // ssize_t +#include // perror, fprintf +#include // ElfW #include -// We don't have libc struct available here. Compute aux vector manually. -static unsigned long * __auxv = NULL; -static unsigned long __auxv_secure = 0; +#define ARRAY_SIZE(a) sizeof((a))/sizeof((a[0])) -static size_t __find_auxv(unsigned long type) +/// Suppress TSan since it is possible for this code to be called from multiple threads, +/// and initialization is safe to be done multiple times from multiple threads. +#if defined(__clang__) +# define NO_SANITIZE_THREAD __attribute__((__no_sanitize__("thread"))) +#else +# define NO_SANITIZE_THREAD +#endif + +// We don't have libc struct available here. +// Compute aux vector manually (from /proc/self/auxv). +// +// Right now there is only 51 AT_* constants, +// so 64 should be enough until this implementation will be replaced with musl. +static unsigned long __auxv_procfs[64]; +static unsigned long __auxv_secure = 0; +// Common +static unsigned long * __auxv_environ = NULL; + +static void * volatile getauxval_func; + +static unsigned long __auxv_init_environ(unsigned long type); + +// +// auxv from procfs interface +// +ssize_t __retry_read(int fd, void * buf, size_t count) +{ + for (;;) + { + ssize_t ret = read(fd, buf, count); + if (ret == -1) + { + if (errno == EINTR) + { + continue; + } + perror("Cannot read /proc/self/auxv"); + abort(); + } + return ret; + } +} +unsigned long NO_SANITIZE_THREAD __getauxval_procfs(unsigned long type) +{ + if (type == AT_SECURE) + { + return __auxv_secure; + } + + if (type >= ARRAY_SIZE(__auxv_procfs)) + { + errno = ENOENT; + return 0; + } + + return __auxv_procfs[type]; +} +static unsigned long NO_SANITIZE_THREAD __auxv_init_procfs(unsigned long type) +{ + // For debugging: + // - od -t dL /proc/self/auxv + // - LD_SHOW_AUX= ls + int fd = open("/proc/self/auxv", O_RDONLY); + // It is possible in case of: + // - no procfs mounted + // - on android you are not able to read it unless running from shell or debugging + // - some other issues + if (fd == -1) + { + // Fallback to environ. + a_cas_p(&getauxval_func, (void *)__auxv_init_procfs, (void *)__auxv_init_environ); + return __auxv_init_environ(type); + } + + ElfW(auxv_t) aux; + + /// NOTE: sizeof(aux) is very small (less then PAGE_SIZE), so partial read should not be possible. + _Static_assert(sizeof(aux) < 4096, "Unexpected sizeof(aux)"); + while (__retry_read(fd, &aux, sizeof(aux)) == sizeof(aux)) + { + if (aux.a_type == AT_NULL) + { + break; + } + if (aux.a_type == AT_IGNORE || aux.a_type == AT_IGNOREPPC) + { + continue; + } + + if (aux.a_type >= ARRAY_SIZE(__auxv_procfs)) + { + fprintf(stderr, "AT_* is out of range: %li (maximum allowed is %zu)\n", aux.a_type, ARRAY_SIZE(__auxv_procfs)); + abort(); + } + if (__auxv_procfs[aux.a_type]) + { + /// It is possible due to race on initialization. + } + __auxv_procfs[aux.a_type] = aux.a_un.a_val; + } + close(fd); + + __auxv_secure = __getauxval_procfs(AT_SECURE); + + // Now we've initialized __auxv_procfs, next time getauxval() will only call __get_auxval(). + a_cas_p(&getauxval_func, (void *)__auxv_init_procfs, (void *)__getauxval_procfs); + + return __getauxval_procfs(type); +} + +// +// auxv from environ interface +// +// NOTE: environ available only after static initializers, +// so you cannot rely on this if you need getauxval() before. +// +// Good example of such user is sanitizers, for example +// LSan will not work with __auxv_init_environ(), +// since it needs getauxval() before. +// +static size_t NO_SANITIZE_THREAD __find_auxv(unsigned long type) { size_t i; - for (i = 0; __auxv[i]; i += 2) + for (i = 0; __auxv_environ[i]; i += 2) { - if (__auxv[i] == type) + if (__auxv_environ[i] == type) + { return i + 1; + } } return (size_t) -1; } - -unsigned long __getauxval(unsigned long type) +unsigned long NO_SANITIZE_THREAD __getauxval_environ(unsigned long type) { if (type == AT_SECURE) return __auxv_secure; - if (__auxv) + if (__auxv_environ) { size_t index = __find_auxv(type); if (index != ((size_t) -1)) - return __auxv[index]; + return __auxv_environ[index]; } errno = ENOENT; return 0; } - -static void * volatile getauxval_func; - -static unsigned long __auxv_init(unsigned long type) +static unsigned long NO_SANITIZE_THREAD __auxv_init_environ(unsigned long type) { if (!__environ) { - // __environ is not initialized yet so we can't initialize __auxv right now. + // __environ is not initialized yet so we can't initialize __auxv_environ right now. // That's normally occurred only when getauxval() is called from some sanitizer's internal code. errno = ENOENT; return 0; } - // Initialize __auxv and __auxv_secure. + // Initialize __auxv_environ and __auxv_secure. size_t i; for (i = 0; __environ[i]; i++); - __auxv = (unsigned long *) (__environ + i + 1); + __auxv_environ = (unsigned long *) (__environ + i + 1); size_t secure_idx = __find_auxv(AT_SECURE); if (secure_idx != ((size_t) -1)) - __auxv_secure = __auxv[secure_idx]; + __auxv_secure = __auxv_environ[secure_idx]; - // Now we've initialized __auxv, next time getauxval() will only call __get_auxval(). - a_cas_p(&getauxval_func, (void *)__auxv_init, (void *)__getauxval); + // Now we need to switch to __getauxval_environ for all later calls, since + // everything is initialized. + a_cas_p(&getauxval_func, (void *)__auxv_init_environ, (void *)__getauxval_environ); - return __getauxval(type); + return __getauxval_environ(type); } -// First time getauxval() will call __auxv_init(). -static void * volatile getauxval_func = (void *)__auxv_init; +// Callchain: +// - __auxv_init_procfs -> __getauxval_environ +// - __auxv_init_procfs -> __auxv_init_environ -> __getauxval_environ +static void * volatile getauxval_func = (void *)__auxv_init_procfs; unsigned long getauxval(unsigned long type) { From a4ba3649f347c751af9274030734404517556fdc Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 25 Jul 2022 00:54:03 +0200 Subject: [PATCH 595/659] Fix 02286_parallel_final (#39524) --- tests/queries/0_stateless/02286_parallel_final.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02286_parallel_final.sh b/tests/queries/0_stateless/02286_parallel_final.sh index 6686a5d3e33..d31450a0482 100755 --- a/tests/queries/0_stateless/02286_parallel_final.sh +++ b/tests/queries/0_stateless/02286_parallel_final.sh @@ -9,6 +9,7 @@ test_random_values() { $CLICKHOUSE_CLIENT -n -q " create table tbl_8parts_${layers}granules_rnd (key1 UInt32, sign Int8) engine = CollapsingMergeTree(sign) order by (key1) partition by (key1 % 8); insert into tbl_8parts_${layers}granules_rnd select number, 1 from numbers_mt($((layers * 8 * 8192))); + optimize table tbl_8parts_${layers}granules_rnd final; explain pipeline select * from tbl_8parts_${layers}granules_rnd final settings max_threads = 16;" 2>&1 | grep -c "CollapsingSortedTransform" } @@ -22,6 +23,7 @@ test_sequential_values() { $CLICKHOUSE_CLIENT -n -q " create table tbl_8parts_${layers}granules_seq (key1 UInt32, sign Int8) engine = CollapsingMergeTree(sign) order by (key1) partition by (key1 / $((layers * 8192)))::UInt64; insert into tbl_8parts_${layers}granules_seq select number, 1 from numbers_mt($((layers * 8 * 8192))); + optimize table tbl_8parts_${layers}granules_seq final; explain pipeline select * from tbl_8parts_${layers}granules_seq final settings max_threads = 8;" 2>&1 | grep -c "CollapsingSortedTransform" } From a1a1d357b4c8e8c061f318c5ab5ca61aab96646d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 24 Jul 2022 20:15:48 -0300 Subject: [PATCH 596/659] Update string-replace-functions.md --- docs/en/sql-reference/functions/string-replace-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 3cd9bdf8ff5..294a7b49db5 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -96,7 +96,7 @@ For more information, see the link: [RE2](https://github.com/google/re2/blob/mas ## translate(s, from, to) -The function replaces characters in the string ‘s’ in accordance with one-to-one character mapping defined by ‘from’ and ‘to’ strings. ‘from’ and ‘to’ must be ASCII strings of the same size. Non-ASCII characters in the original string are not modified. +The function replaces characters in the string ‘s’ in accordance with one-to-one character mapping defined by ‘from’ and ‘to’ strings. ‘from’ and ‘to’ must be constant ASCII strings of the same size. Non-ASCII characters in the original string are not modified. Example: @@ -112,7 +112,7 @@ SELECT translate('Hello, World!', 'delor', 'DELOR') AS res ## translateUTF8(string, from, to) -Similar to previous function, but works with UTF-8 arguments. ‘from’ and ‘to’ must be valid UTF-8 strings of the same size. +Similar to previous function, but works with UTF-8 arguments. ‘from’ and ‘to’ must be valid constant UTF-8 strings of the same size. Example: From 31630ff38e22eba6ef4c71149d74fef6f2aeeb80 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 24 Jul 2022 20:18:49 -0300 Subject: [PATCH 597/659] Update string-replace-functions.md --- docs/ru/sql-reference/functions/string-replace-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/string-replace-functions.md b/docs/ru/sql-reference/functions/string-replace-functions.md index 1bbb68f9014..14c49c19e64 100644 --- a/docs/ru/sql-reference/functions/string-replace-functions.md +++ b/docs/ru/sql-reference/functions/string-replace-functions.md @@ -85,7 +85,7 @@ SELECT replaceRegexpAll('Hello, World!', '^', 'here: ') AS res ## translate(s, from, to) -Данная функция заменяет символы в строке ‘s’ в соответствии с поэлементным отображением определяемым строками ‘from’ и ‘to’. ‘from’ и ‘to’ должны быть корректными ASCII строками одного размера. Не ASCII символы в оригинальной строке не изменяются. +Данная функция заменяет символы в строке ‘s’ в соответствии с поэлементным отображением определяемым строками ‘from’ и ‘to’. ‘from’ и ‘to’ должны быть корректными константными ASCII строками одного размера. Не ASCII символы в оригинальной строке не изменяются. Example: @@ -101,7 +101,7 @@ SELECT translate('Hello, World!', 'delor', 'DELOR') AS res ## translateUTF8(string, from, to) -Аналогично предыдущей функции, но работает со строками, состоящими из UTF-8 символов. ‘from’ и ‘to’ должны быть корректными UTF-8 строками одного размера. +Аналогично предыдущей функции, но работает со строками, состоящими из UTF-8 символов. ‘from’ и ‘to’ должны быть корректными константными UTF-8 строками одного размера. Example: From bca33a5a0ac1939863d6594aafff350b4635edf2 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 24 Jul 2022 20:57:37 -0300 Subject: [PATCH 598/659] Update formats.md --- docs/en/interfaces/formats.md | 79 ++++++++++++++++++++++++++++++++++- 1 file changed, 78 insertions(+), 1 deletion(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index e499849426b..6678af9036b 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -194,18 +194,25 @@ Differs from the `TabSeparated` format in that the column names are written in t During parsing, the first row is expected to contain the column names. You can use column names to determine their position and to check their correctness. +:::warning If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. +::: This format is also available under the name `TSVWithNames`. ## TabSeparatedWithNamesAndTypes {#tabseparatedwithnamesandtypes} Differs from the `TabSeparated` format in that the column names are written to the first row, while the column types are in the second row. -The first row with names is processed the same way as in `TabSeparatedWithNames` format. + +:::warning +If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1, +the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1. +Otherwise, the first row will be skipped. If setting [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header) is set to 1, the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped. +::: This format is also available under the name `TSVWithNamesAndTypes`. @@ -451,10 +458,24 @@ The CSV format supports the output of totals and extremes the same way as `TabSe Also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). +:::warning +If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1, +the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1. +Otherwise, the first row will be skipped. +::: + ## CSVWithNamesAndTypes {#csvwithnamesandtypes} Also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). +:::warning +If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1, +the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1. +Otherwise, the first row will be skipped. +If setting [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header) is set to 1, +the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped. +::: + ## CustomSeparated {#format-customseparated} Similar to [Template](#format-template), but it prints or reads all names and types of columns and uses escaping rule from [format_custom_escaping_rule](../operations/settings/settings.md#format_custom_escaping_rule) setting and delimiters from [format_custom_field_delimiter](../operations/settings/settings.md#format_custom_field_delimiter), [format_custom_row_before_delimiter](../operations/settings/settings.md#format_custom_row_before_delimiter), [format_custom_row_after_delimiter](../operations/settings/settings.md#format_custom_row_after_delimiter), [format_custom_row_between_delimiter](../operations/settings/settings.md#format_custom_row_between_delimiter), [format_custom_result_before_delimiter](../operations/settings/settings.md#format_custom_result_before_delimiter) and [format_custom_result_after_delimiter](../operations/settings/settings.md#format_custom_result_after_delimiter) settings, not from format strings. @@ -465,10 +486,24 @@ There is also `CustomSeparatedIgnoreSpaces` format, which is similar to [Templat Also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). +:::warning +If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1, +the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1. +Otherwise, the first row will be skipped. +::: + ## CustomSeparatedWithNamesAndTypes {#customseparatedwithnamesandtypes} Also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). +:::warning +If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1, +the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1. +Otherwise, the first row will be skipped. +If setting [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header) is set to 1, +the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped. +::: + ## SQLInsert {#sqlinsert} Outputs data as a sequence of `INSERT INTO table (columns...) VALUES (...), (...) ...;` statements. @@ -911,18 +946,46 @@ Differs from `JSONEachRow`/`JSONStringsEachRow` in that ClickHouse will also yie Differs from `JSONCompactEachRow` format in that it also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). +:::warning +If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1, +the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1. +Otherwise, the first row will be skipped. +::: + ## JSONCompactEachRowWithNamesAndTypes {#jsoncompacteachrowwithnamesandtypes} Differs from `JSONCompactEachRow` format in that it also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). +:::warning +If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1, +the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1. +Otherwise, the first row will be skipped. +If setting [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header) is set to 1, +the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped. +::: + ## JSONCompactStringsEachRowWithNames {#jsoncompactstringseachrowwithnames} Differs from `JSONCompactStringsEachRow` in that in that it also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). +:::warning +If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1, +the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1. +Otherwise, the first row will be skipped. +::: + ## JSONCompactStringsEachRowWithNamesAndTypes {#jsoncompactstringseachrowwithnamesandtypes} Differs from `JSONCompactStringsEachRow` in that it also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). +:::warning +If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1, +the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1. +Otherwise, the first row will be skipped. +If setting [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header) is set to 1, +the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped. +::: + ```json ["num", "str", "arr"] ["Int32", "String", "Array(UInt8)"] @@ -1199,6 +1262,12 @@ Similar to [RowBinary](#rowbinary), but with added header: - [LEB128](https://en.wikipedia.org/wiki/LEB128)-encoded number of columns (N) - N `String`s specifying column names +:::warning +If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1, +the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1. +Otherwise, the first row will be skipped. +::: + ## RowBinaryWithNamesAndTypes {#rowbinarywithnamesandtypes} Similar to [RowBinary](#rowbinary), but with added header: @@ -1207,6 +1276,14 @@ Similar to [RowBinary](#rowbinary), but with added header: - N `String`s specifying column names - N `String`s specifying column types +:::warning +If setting [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header) is set to 1, +the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields) is set to 1. +Otherwise, the first row will be skipped. +If setting [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header) is set to 1, +the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped. +::: + ## Values {#data-format-values} Prints every row in brackets. Rows are separated by commas. There is no comma after the last row. The values inside the brackets are also comma-separated. Numbers are output in a decimal format without quotes. Arrays are output in square brackets. Strings, dates, and dates with times are output in quotes. Escaping rules and parsing are similar to the [TabSeparated](#tabseparated) format. During formatting, extra spaces aren’t inserted, but during parsing, they are allowed and skipped (except for spaces inside array values, which are not allowed). [NULL](../sql-reference/syntax.md) is represented as `NULL`. From 60265c4bb9e3b6238883c3b68cd1bbe21a5a20be Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 24 Jul 2022 21:06:45 -0300 Subject: [PATCH 599/659] Update settings.md --- docs/ru/operations/settings/settings.md | 36 ++++++++++++++++++++++--- 1 file changed, 33 insertions(+), 3 deletions(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 9470e3e0f9b..1d57b688217 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -527,7 +527,7 @@ SELECT * FROM table_with_enum_column_for_tsv_insert; - [Использование вложенных структур](../../interfaces/formats.md#jsoneachrow-nested) with the `JSONEachRow` format. -## input_format_with_names_use_header {#settings-input-format-with-names-use-header} +## input_format_with_names_use_header {#input_format_with_names_use_header} Включает или отключает проверку порядка столбцов при вставке данных. @@ -535,8 +535,38 @@ SELECT * FROM table_with_enum_column_for_tsv_insert; Поддерживаемые форматы: -- [CSVWithNames](../../interfaces/formats.md#csvwithnames) -- [TabSeparatedWithNames](../../interfaces/formats.md#tabseparatedwithnames) +- [CSVWithNames](../../interfaces/formats.md#csvwithnames) +- [CSVWithNamesAndTypes](../../interfaces/formats.md#csvwithnamesandtypes) +- [TabSeparatedWithNames](../../interfaces/formats.md#tabseparatedwithnames) +- [TabSeparatedWithNamesAndTypes](../../interfaces/formats.md#tabseparatedwithnamesandtypes) +- [JSONCompactEachRowWithNames](../../interfaces/formats.md#jsoncompacteachrowwithnames) +- [JSONCompactEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompacteachrowwithnamesandtypes) +- [JSONCompactStringsEachRowWithNames](../../interfaces/formats.md#jsoncompactstringseachrowwithnames) +- [JSONCompactStringsEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompactstringseachrowwithnamesandtypes) +- [RowBinaryWithNames](../../interfaces/formats.md#rowbinarywithnames) +- [RowBinaryWithNamesAndTypes](../../interfaces/formats.md#rowbinarywithnamesandtypes) +- [CustomSeparatedWithNames](../../interfaces/formats.md#customseparatedwithnames) +- [CustomSeparatedWithNamesAndTypes](../../interfaces/formats.md#customseparatedwithnamesandtypes) + +Возможные значения: + +- 0 — выключена. +- 1 — включена. + +Значение по умолчанию: 1. + +## input_format_with_types_use_header {#input_format_with_types_use_header} + +Определяет, должен ли синтаксический анализатор формата проверять, соответствуют ли типы данных из входных данных типам данных из целевой таблицы. + +Поддерживаемые форматы: + +- [CSVWithNamesAndTypes](../../interfaces/formats.md#csvwithnamesandtypes) +- [TabSeparatedWithNamesAndTypes](../../interfaces/formats.md#tabseparatedwithnamesandtypes) +- [JSONCompactEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompacteachrowwithnamesandtypes) +- [JSONCompactStringsEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompactstringseachrowwithnamesandtypes) +- [RowBinaryWithNamesAndTypes](../../interfaces/formats.md#rowbinarywithnamesandtypes-rowbinarywithnamesandtypes) +- [CustomSeparatedWithNamesAndTypes](../../interfaces/formats.md#customseparatedwithnamesandtypes) Возможные значения: From d15ee4b48bffe91e8e2044d340f3f7cd9e789c3c Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 24 Jul 2022 21:34:35 -0300 Subject: [PATCH 600/659] Update formats.md --- docs/ru/interfaces/formats.md | 78 +++++++++++++++++++++++++++++++++++ 1 file changed, 78 insertions(+) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 403abe5fdda..37a6a8688b5 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -19,6 +19,7 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT | [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | | [CSV](#csv) | ✔ | ✔ | | [CSVWithNames](#csvwithnames) | ✔ | ✔ | +| [CSVWithNamesAndTypes](#csvwithnamesandtypes) | ✔ | ✔ | | [CustomSeparated](#format-customseparated) | ✔ | ✔ | | [CustomSeparatedWithNames](#customseparatedwithnames) | ✔ | ✔ | | [CustomSeparatedWithNamesAndTypes](#customseparatedwithnamesandtypes) | ✔ | ✔ | @@ -52,6 +53,7 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT | [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ | | [ORC](#data-format-orc) | ✔ | ✔ | | [RowBinary](#rowbinary) | ✔ | ✔ | +| [RowBinaryWithNames](#rowbinarywithnames) | ✔ | ✔ | | [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ | | [Native](#native) | ✔ | ✔ | | [Null](#null) | ✗ | ✔ | @@ -171,6 +173,12 @@ SELECT * FROM nestedt FORMAT TSV При парсинге первая строка должна содержать имена столбцов. Вы можете использовать имена столбцов, чтобы указать их порядок расположения, или чтобы проверить их корректность. +:::warning +Если включен параметр [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header), +столбцы из входных данных будут сопоставлены со столбцами таблицы по их именам, столбцы с неизвестными именами будут пропущены, если включен параметр [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields). +В противном случае первая строка будет пропущена. +::: + Этот формат также доступен под именем `TSVWithNames`. ## TabSeparatedWithNamesAndTypes {#tabseparatedwithnamesandtypes} @@ -178,6 +186,14 @@ SELECT * FROM nestedt FORMAT TSV Отличается от формата `TabSeparated` тем, что в первой строке пишутся имена столбцов, а во второй - типы столбцов. При парсинге, первая и вторая строка полностью игнорируется. +:::warning +Если включен параметр [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header), +столбцы из входных данных будут сопоставлены со столбцами таблицы по их именам, столбцы с неизвестными именами будут пропущены, если включен параметр [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields). +В противном случае первая строка будет пропущена. +Если включен параметр [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header), +типы из входных данных будут сравниваться с типами соответствующих столбцов из таблицы. В противном случае вторая строка будет пропущена. +::: + Этот формат также доступен под именем `TSVWithNamesAndTypes`. ## Template {#format-template} @@ -374,6 +390,24 @@ $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FOR Выводит также заголовок, аналогично [TabSeparatedWithNames](#tabseparatedwithnames). +:::warning +Если включен параметр [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header), +столбцы из входных данных будут сопоставлены со столбцами таблицы по их именам, столбцы с неизвестными именами будут пропущены, если включен параметр [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields). +В противном случае первая строка будет пропущена. +::: + +## CSVWithNamesAndTypes {#csvwithnamesandtypes} + +В первой строке пишутся имена столбцов, а во второй - типы столбцов, аналогично [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) + +:::warning +Если включен параметр [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header), +столбцы из входных данных будут сопоставлены со столбцами таблицы по их именам, столбцы с неизвестными именами будут пропущены, если включен параметр [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields). +В противном случае первая строка будет пропущена. +Если включен параметр [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header), +типы из входных данных будут сравниваться с типами соответствующих столбцов из таблицы. В противном случае вторая строка будет пропущена. +::: + ## CustomSeparated {#format-customseparated} Аналогичен [Template](#format-template), но выводит (или считывает) все имена и типы столбцов, используя для них правило экранирования из настройки [format_custom_escaping_rule](../operations/settings/settings.md#format-custom-escaping-rule) и разделители из настроек [format_custom_field_delimiter](../operations/settings/settings.md#format-custom-field-delimiter), [format_custom_row_before_delimiter](../operations/settings/settings.md#format-custom-row-before-delimiter), [format_custom_row_after_delimiter](../operations/settings/settings.md#format-custom-row-after-delimiter), [format_custom_row_between_delimiter](../operations/settings/settings.md#format-custom-row-between-delimiter), [format_custom_result_before_delimiter](../operations/settings/settings.md#format-custom-result-before-delimiter) и [format_custom_result_after_delimiter](../operations/settings/settings.md#format-custom-result-after-delimiter), а не из форматных строк. @@ -384,10 +418,24 @@ $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FOR Выводит также заголовок с именами столбцов, аналогичен формату [TabSeparatedWithNames](#tabseparatedwithnames). +:::warning +Если включен параметр [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header), +столбцы из входных данных будут сопоставлены со столбцами таблицы по их именам, столбцы с неизвестными именами будут пропущены, если включен параметр [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields). +В противном случае первая строка будет пропущена. +::: + ## CustomSeparatedWithNamesAndTypes {#customseparatedwithnamesandtypes} Выводит также два заголовка с именами и типами столбцов, аналогичен формату [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). +:::warning +Если включен параметр [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header), +столбцы из входных данных будут сопоставлены со столбцами таблицы по их именам, столбцы с неизвестными именами будут пропущены, если включен параметр [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields). +В противном случае первая строка будет пропущена. +Если включен параметр [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header), +типы из входных данных будут сравниваться с типами соответствующих столбцов из таблицы. В противном случае вторая строка будет пропущена. +::: + ## JSON {#json} Выводит данные в формате JSON. Кроме таблицы с данными, также выводятся имена и типы столбцов, и некоторая дополнительная информация - общее количество выведенных строк, а также количество строк, которое могло бы быть выведено, если бы не было LIMIT-а. Пример: @@ -660,6 +708,14 @@ SELECT * FROM json_square_brackets; Отличается от `JSONCompactEachRow`/`JSONCompactStringsEachRow` тем, что имена и типы столбцов записываются как первые две строки. +:::warning +Если включен параметр [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header), +столбцы из входных данных будут сопоставлены со столбцами таблицы по их именам, столбцы с неизвестными именами будут пропущены, если включен параметр [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields). +В противном случае первая строка будет пропущена. +Если включен параметр [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header), +типы из входных данных будут сравниваться с типами соответствующих столбцов из таблицы. В противном случае вторая строка будет пропущена. +::: + ```json ["'hello'", "multiply(42, number)", "range(5)"] ["String", "UInt64", "Array(UInt8)"] @@ -904,6 +960,20 @@ Array представлены как длина в формате varint (unsig Для поддержки [NULL](../sql-reference/syntax.md#null-literal) перед каждым значением типа [Nullable](../sql-reference/data-types/nullable.md) следует байт содержащий 1 или 0. Если байт 1, то значение равно NULL, и этот байт интерпретируется как отдельное значение (т.е. после него следует значение следующего поля). Если байт 0, то после байта следует значение поля (не равно NULL). + +## RowBinaryWithNames {#rowbinarywithnames} + +То же самое что [RowBinary](#rowbinary), но добавляется заголовок: + +- Количество колонок - N, закодированное [LEB128](https://en.wikipedia.org/wiki/LEB128), +- N строк (`String`) с именами колонок, + +:::warning +Если включен параметр [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header), +столбцы из входных данных будут сопоставлены со столбцами таблицы по их именам, столбцы с неизвестными именами будут пропущены, если включен параметр [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields). +В противном случае первая строка будет пропущена. +::: + ## RowBinaryWithNamesAndTypes {#rowbinarywithnamesandtypes} То же самое что [RowBinary](#rowbinary), но добавляется заголовок: @@ -912,6 +982,14 @@ Array представлены как длина в формате varint (unsig - N строк (`String`) с именами колонок, - N строк (`String`) с типами колонок. +:::warning +Если включен параметр [input_format_with_names_use_header](../operations/settings/settings.md#input_format_with_names_use_header), +столбцы из входных данных будут сопоставлены со столбцами таблицы по их именам, столбцы с неизвестными именами будут пропущены, если включен параметр [input_format_skip_unknown_fields](../operations/settings/settings.md#input_format_skip_unknown_fields). +В противном случае первая строка будет пропущена. +Если включен параметр [input_format_with_types_use_header](../operations/settings/settings.md#input_format_with_types_use_header), +типы из входных данных будут сравниваться с типами соответствующих столбцов из таблицы. В противном случае вторая строка будет пропущена. +::: + ## Values {#data-format-values} Выводит каждую строку в скобках. Строки разделены запятыми. После последней строки запятой нет. Значения внутри скобок также разделены запятыми. Числа выводятся в десятичном виде без кавычек. Массивы выводятся в квадратных скобках. Строки, даты, даты-с-временем выводятся в кавычках. Правила экранирования и особенности парсинга аналогичны формату [TabSeparated](#tabseparated). При форматировании, лишние пробелы не ставятся, а при парсинге - допустимы и пропускаются (за исключением пробелов внутри значений типа массив, которые недопустимы). [NULL](../sql-reference/syntax.md) представляется как `NULL`. From da42ec7d28b8206e4271bce85298190661f34e7f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 24 Jul 2022 21:08:43 -0400 Subject: [PATCH 601/659] some refactoring --- CMakeLists.txt | 29 +++++++++++++---------------- 1 file changed, 13 insertions(+), 16 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index b462c1c7b7a..6a93449495c 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -13,16 +13,6 @@ else() set(RECONFIGURE_MESSAGE_LEVEL WARNING) endif() -# Setup NATIVE_BUILD_TARGETS list for further native build -# when cross-compiling and host architecture differs from target architecture. -# Add target to the list: -# add_native_target (...) -set_property (GLOBAL PROPERTY NATIVE_BUILD_TARGETS) -function (add_native_target) - get_property (tmp GLOBAL PROPERTY NATIVE_BUILD_TARGETS) - set_property (GLOBAL PROPERTY NATIVE_BUILD_TARGETS "${tmp}" "${ARGV}") -endfunction (add_native_target) - include (cmake/arch.cmake) include (cmake/target.cmake) include (cmake/tools.cmake) @@ -564,6 +554,16 @@ macro (clickhouse_add_executable target) endif() endmacro() +# With cross-compiling, all targets are built for the target platform which usually different from the host +# platform. This is problematic if a build artifact X (e.g. a file or an executable) is generated by running +# another executable Y previously produced in the build. This is solved by compiling and running Y for/on +# the host platform. Add target to the list: +# add_native_target( ...) +set_property (GLOBAL PROPERTY NATIVE_BUILD_TARGETS) +function (add_native_target) + set_property (GLOBAL APPEND PROPERTY NATIVE_BUILD_TARGETS ${ARGV}) +endfunction (add_native_target) + set(ConfigIncludePath ${CMAKE_CURRENT_BINARY_DIR}/includes/configs CACHE INTERNAL "Path to generated configuration files.") include_directories(${ConfigIncludePath}) @@ -582,10 +582,10 @@ include (cmake/sanitize_target_link_libraries.cmake) # Build native targets if necessary get_property(NATIVE_BUILD_TARGETS GLOBAL PROPERTY NATIVE_BUILD_TARGETS) if (NATIVE_BUILD_TARGETS - AND NOT( - CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME + AND NOT( + CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME AND CMAKE_HOST_SYSTEM_PROCESSOR STREQUAL CMAKE_SYSTEM_PROCESSOR - ) + ) ) message (STATUS "Building native targets...") @@ -597,9 +597,6 @@ if (NATIVE_BUILD_TARGETS execute_process( COMMAND ${CMAKE_COMMAND} - "-G${CMAKE_GENERATOR}" - "-DCMAKE_BUILD_TYPE=${CMAKE_BUILD_TYPE}" - "-DCMAKE_MAKE_PROGRAM=${CMAKE_MAKE_PROGRAM}" "-DCMAKE_C_COMPILER=${CMAKE_C_COMPILER}" "-DCMAKE_CXX_COMPILER=${CMAKE_CXX_COMPILER}" ${CMAKE_SOURCE_DIR} From a85818f654f8f39638a91b2ab61340d6ed0393e8 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 24 Jul 2022 21:13:00 -0400 Subject: [PATCH 602/659] some refactoring --- utils/self-extracting-executable/compressor.cpp | 2 +- utils/self-extracting-executable/decompressor.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/self-extracting-executable/compressor.cpp b/utils/self-extracting-executable/compressor.cpp index 7afb38232f6..58674818c44 100644 --- a/utils/self-extracting-executable/compressor.cpp +++ b/utils/self-extracting-executable/compressor.cpp @@ -12,7 +12,7 @@ #if (defined(OS_DARWIN) || defined(OS_FREEBSD)) && defined(__GNUC__) # include -#elif !defined(_MSC_VER) +#else # include #endif diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index a8d80b76929..3ec06e91176 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -15,7 +15,7 @@ #if (defined(OS_DARWIN) || defined(OS_FREEBSD)) && defined(__GNUC__) # include -#elif !defined(_MSC_VER) +#else # include #endif From c09413e3b9c43dcda4eef91d9776946c0372317f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Jul 2022 03:27:10 +0200 Subject: [PATCH 603/659] Add save/load capabilities to Obfuscator --- programs/obfuscator/Obfuscator.cpp | 242 ++++++++++++++++++++++++++++- 1 file changed, 239 insertions(+), 3 deletions(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index a194718db58..2021a2556cd 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -34,6 +34,10 @@ #include #include #include +#include +#include +#include +#include #include #include #include @@ -95,6 +99,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; extern const int CANNOT_SEEK_THROUGH_FILE; + extern const int UNKNOWN_FORMAT_VERSION; + extern const int INCORRECT_NUMBER_OF_COLUMNS; + extern const int TYPE_MISMATCH; } @@ -115,6 +122,12 @@ public: /// Deterministically change seed to some other value. This can be used to generate more values than were in source. virtual void updateSeed() = 0; + /// Save into file. Binary, platform-dependent, version-dependent serialization. + virtual void serialize(WriteBuffer & out) const = 0; + + /// Read from file + virtual void deserialize(ReadBuffer & in) = 0; + virtual ~IModel() = default; }; @@ -189,6 +202,8 @@ public: void train(const IColumn &) override {} void finalize() override {} + void serialize(WriteBuffer &) const override {} + void deserialize(ReadBuffer &) override {} ColumnPtr generate(const IColumn & column) override { @@ -230,6 +245,8 @@ public: void train(const IColumn &) override {} void finalize() override {} + void serialize(WriteBuffer &) const override {} + void deserialize(ReadBuffer &) override {} ColumnPtr generate(const IColumn & column) override { @@ -279,6 +296,8 @@ public: void train(const IColumn &) override {} void finalize() override {} + void serialize(WriteBuffer &) const override {} + void deserialize(ReadBuffer &) override {} ColumnPtr generate(const IColumn & column) override { @@ -311,6 +330,8 @@ class IdentityModel : public IModel public: void train(const IColumn &) override {} void finalize() override {} + void serialize(WriteBuffer &) const override {} + void deserialize(ReadBuffer &) override {} ColumnPtr generate(const IColumn & column) override { @@ -395,6 +416,8 @@ public: void train(const IColumn &) override {} void finalize() override {} + void serialize(WriteBuffer &) const override {} + void deserialize(ReadBuffer &) override {} ColumnPtr generate(const IColumn & column) override { @@ -431,6 +454,8 @@ public: void train(const IColumn &) override {} void finalize() override {} + void serialize(WriteBuffer &) const override {} + void deserialize(ReadBuffer &) override {} ColumnPtr generate(const IColumn & column) override { @@ -469,6 +494,8 @@ public: void train(const IColumn &) override {} void finalize() override {} + void serialize(WriteBuffer &) const override {} + void deserialize(ReadBuffer &) override {} ColumnPtr generate(const IColumn & column) override { @@ -512,6 +539,26 @@ struct MarkovModelParameters size_t frequency_add; double frequency_desaturate; size_t determinator_sliding_window_size; + + void serialize(WriteBuffer & out) const + { + writeBinary(order, out); + writeBinary(frequency_cutoff, out); + writeBinary(num_buckets_cutoff, out); + writeBinary(frequency_add, out); + writeBinary(frequency_desaturate, out); + writeBinary(determinator_sliding_window_size, out); + } + + void deserialize(ReadBuffer & in) + { + readBinary(order, in); + readBinary(frequency_cutoff, in); + readBinary(num_buckets_cutoff, in); + readBinary(frequency_add, in); + readBinary(frequency_desaturate, in); + readBinary(determinator_sliding_window_size, in); + } }; @@ -565,6 +612,39 @@ private: return END; } + + void serialize(WriteBuffer & out) const + { + writeBinary(total, out); + writeBinary(count_end, out); + + size_t size = buckets.size(); + writeBinary(size, out); + + for (const auto & elem : buckets) + { + writeBinary(elem.first, out); + writeBinary(elem.second, out); + } + } + + void deserialize(ReadBuffer & in) + { + readBinary(total, in); + readBinary(count_end, in); + + size_t size = 0; + readBinary(size, in); + + buckets.reserve(size); + for (size_t i = 0; i < size; ++i) + { + Buckets::value_type elem; + readBinary(elem.first, in); + readBinary(elem.second, in); + buckets.emplace(std::move(elem)); + } + } }; using Table = HashMap; @@ -621,6 +701,37 @@ public: explicit MarkovModel(MarkovModelParameters params_) : params(std::move(params_)), code_points(params.order, BEGIN) {} + void serialize(WriteBuffer & out) const + { + params.serialize(out); + + size_t size = table.size(); + writeBinary(size, out); + + for (const auto & elem : table) + { + writeBinary(elem.getKey(), out); + elem.getMapped().serialize(out); + } + } + + void deserialize(ReadBuffer & in) + { + params.deserialize(in); + + size_t size = 0; + readBinary(size, in); + + table.reserve(size); + for (size_t i = 0; i < size; ++i) + { + NGramHash key{}; + readBinary(key, in); + Histogram & histogram = table[key]; + histogram.deserialize(in); + } + } + void consume(const char * data, size_t size) { /// First 'order' number of code points are pre-filled with BEGIN. @@ -878,6 +989,16 @@ public: { seed = hash(seed); } + + void serialize(WriteBuffer & out) const override + { + markov_model.serialize(out); + } + + void deserialize(ReadBuffer & in) override + { + markov_model.deserialize(in); + } }; @@ -916,6 +1037,16 @@ public: { nested_model->updateSeed(); } + + void serialize(WriteBuffer & out) const override + { + nested_model->serialize(out); + } + + void deserialize(ReadBuffer & in) override + { + nested_model->deserialize(in); + } }; @@ -954,6 +1085,16 @@ public: { nested_model->updateSeed(); } + + void serialize(WriteBuffer & out) const override + { + nested_model->serialize(out); + } + + void deserialize(ReadBuffer & in) override + { + nested_model->deserialize(in); + } }; @@ -1046,6 +1187,18 @@ public: for (auto & model : models) model->updateSeed(); } + + void serialize(WriteBuffer & out) const + { + for (const auto & model : models) + model->serialize(out); + } + + void deserialize(ReadBuffer & in) + { + for (auto & model : models) + model->deserialize(in); + } }; } @@ -1070,6 +1223,8 @@ try ("seed", po::value(), "seed (arbitrary string), must be random string with at least 10 bytes length; note that a seed for each column is derived from this seed and a column name: you can obfuscate data for different tables and as long as you use identical seed and identical column names, the data for corresponding non-text columns for different tables will be transformed in the same way, so the data for different tables can be JOINed after obfuscation") ("limit", po::value(), "if specified - stop after generating that number of rows") ("silent", po::value()->default_value(false), "don't print information messages to stderr") + ("save", po::value(), "save the models after training to the specified file. You can use --limit 0 to skip the generation step. The file is using binary, platform-dependent, opaque serialization format. The model parameters are saved, while the seed is not.") + ("load", po::value(), "load the models instead of training from the specified file. The table structure must match the saved file. The seed should be specified separately, while other model parameters are loaded.") ("order", po::value()->default_value(5), "order of markov model to generate strings") ("frequency-cutoff", po::value()->default_value(5), "frequency cutoff for markov model: remove all buckets with count less than specified") ("num-buckets-cutoff", po::value()->default_value(0), "cutoff for number of different possible continuations for a context: remove all histograms with less than specified number of buckets") @@ -1096,12 +1251,33 @@ try return 0; } + if (options.count("save") && options.count("load")) + { + std::cerr << "The options --save and --load cannot be used together.\n"; + return 1; + } + + if (options.count("load") + && (options.count("order") + || options.count("frequency-cutoff") + || options.count("num-buckets-cutoff") + || options.count("frequency-add") + || options.count("frequency-desaturate") + || options.count("determinator-sliding-window-size"))) + { + std::cerr << "Model parameters should not be specified with the --load options, as they will be loaded from the file.\n"; + return 1; + } + UInt64 seed = sipHash64(options["seed"].as()); std::string structure = options["structure"].as(); std::string input_format = options["input-format"].as(); std::string output_format = options["output-format"].as(); + std::string load_from_file = options["load"].as(); + std::string save_into_file = options["save"].as(); + UInt64 limit = 0; if (options.count("limit")) limit = options["limit"].as(); @@ -1117,7 +1293,7 @@ try markov_model_params.frequency_desaturate = options["frequency-desaturate"].as(); markov_model_params.determinator_sliding_window_size = options["determinator-sliding-window-size"].as(); - // Create header block + /// Create the header block std::vector structure_vals; boost::split(structure_vals, structure, boost::algorithm::is_any_of(" ,"), boost::algorithm::token_compress_on); @@ -1143,6 +1319,7 @@ try ReadBufferFromFileDescriptor file_in(STDIN_FILENO); WriteBufferFromFileDescriptor file_out(STDOUT_FILENO); + if (load_from_file.empty()) { /// stdin must be seekable auto res = lseek(file_in.getFD(), 0, SEEK_SET); @@ -1156,6 +1333,9 @@ try /// Train step UInt64 source_rows = 0; + + bool rewind_needed = false; + if (load_from_file.empty()) { if (!silent) std::cerr << "Training models\n"; @@ -1173,9 +1353,63 @@ try if (!silent) std::cerr << "Processed " << source_rows << " rows\n"; } + + obfuscator.finalize(); + rewind_needed = true; + } + else + { + ReadBufferFromFile model_file_in(load_from_file); + CompressedReadBuffer model_in(model_file_in); + + UInt8 version = 0; + readBinary(version, model_in); + if (version != 0) + throw Exception("Unknown version of the model file", ErrorCodes::UNKNOWN_FORMAT_VERSION); + + readBinary(source_rows, model_in); + + Names data_types = header.getDataTypeNames(); + size_t header_size = 0; + readBinary(header_size, model_in); + if (header_size != data_types.size()) + throw Exception("The saved model was created for different number of columns", ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS); + + for (size_t i = 0; i < header_size; ++i) + { + String type; + readBinary(type, model_in); + if (type != data_types[i]) + throw Exception("The saved model was created for different types of columns", ErrorCodes::TYPE_MISMATCH); + } + + obfuscator.deserialize(model_in); } - obfuscator.finalize(); + if (!save_into_file.empty()) + { + WriteBufferFromFile model_file_out(save_into_file); + CompressedWriteBuffer model_out(model_file_out, CompressionCodecFactory::instance().get("ZSTD", 1)); + + /// You can change version on format change, it is currently set to zero. + UInt8 version = 0; + writeBinary(version, model_out); + + writeBinary(source_rows, model_out); + + /// We are writing the data types for validation, because the models serialization depends on the data types. + Names data_types = header.getDataTypeNames(); + size_t header_size = data_types.size(); + writeBinary(header_size, model_out); + for (const auto & type : data_types) + writeBinary(type, model_out); + + /// Write the models. + obfuscator.serialize(model_out); + + model_out.finalize(); + model_file_out.finalize(); + } if (!limit) limit = source_rows; @@ -1187,7 +1421,8 @@ try if (!silent) std::cerr << "Generating data\n"; - file_in.seek(0, SEEK_SET); + if (rewind_needed) + file_in.rewind(); Pipe pipe(context->getInputFormat(input_format, file_in, header, max_block_size)); @@ -1220,6 +1455,7 @@ try out_executor.finish(); obfuscator.updateSeed(); + rewind_needed = true; } return 0; From 52ad48d1ba5f18e6f14289e568a74861e4c79734 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Jul 2022 03:32:01 +0200 Subject: [PATCH 604/659] Better documentation --- programs/obfuscator/Obfuscator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 2021a2556cd..4821848ebbf 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1221,7 +1221,7 @@ try ("input-format", po::value(), "input format of the initial table data") ("output-format", po::value(), "default output format") ("seed", po::value(), "seed (arbitrary string), must be random string with at least 10 bytes length; note that a seed for each column is derived from this seed and a column name: you can obfuscate data for different tables and as long as you use identical seed and identical column names, the data for corresponding non-text columns for different tables will be transformed in the same way, so the data for different tables can be JOINed after obfuscation") - ("limit", po::value(), "if specified - stop after generating that number of rows") + ("limit", po::value(), "if specified - stop after generating that number of rows; the limit can be also greater than the number of source dataset - in this case it will process the dataset in a loop more than one time, using different seeds on every iteration, generating result as large as needed") ("silent", po::value()->default_value(false), "don't print information messages to stderr") ("save", po::value(), "save the models after training to the specified file. You can use --limit 0 to skip the generation step. The file is using binary, platform-dependent, opaque serialization format. The model parameters are saved, while the seed is not.") ("load", po::value(), "load the models instead of training from the specified file. The table structure must match the saved file. The seed should be specified separately, while other model parameters are loaded.") From 68f5a397d7ee1cf4db9896c9127025ca70fb187d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Jul 2022 03:49:09 +0200 Subject: [PATCH 605/659] Fix errors --- programs/obfuscator/Obfuscator.cpp | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 4821848ebbf..7482b34eb3d 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1275,8 +1275,13 @@ try std::string input_format = options["input-format"].as(); std::string output_format = options["output-format"].as(); - std::string load_from_file = options["load"].as(); - std::string save_into_file = options["save"].as(); + std::string load_from_file; + std::string save_into_file; + + if (options.count("load")) + load_from_file = options["load"].as(); + else if (options.count("save")) + save_into_file = options["save"].as(); UInt64 limit = 0; if (options.count("limit")) @@ -1411,7 +1416,7 @@ try model_file_out.finalize(); } - if (!limit) + if (!options.count("limit")) limit = source_rows; /// Generation step From dde89c9606ed715131f37035cdb75829c41d73fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Jul 2022 03:56:32 +0200 Subject: [PATCH 606/659] Fix errors --- programs/obfuscator/Obfuscator.cpp | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 7482b34eb3d..996d0316b4d 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1257,18 +1257,6 @@ try return 1; } - if (options.count("load") - && (options.count("order") - || options.count("frequency-cutoff") - || options.count("num-buckets-cutoff") - || options.count("frequency-add") - || options.count("frequency-desaturate") - || options.count("determinator-sliding-window-size"))) - { - std::cerr << "Model parameters should not be specified with the --load options, as they will be loaded from the file.\n"; - return 1; - } - UInt64 seed = sipHash64(options["seed"].as()); std::string structure = options["structure"].as(); @@ -1364,6 +1352,9 @@ try } else { + if (!silent) + std::cerr << "Loading models\n"; + ReadBufferFromFile model_file_in(load_from_file); CompressedReadBuffer model_in(model_file_in); @@ -1393,6 +1384,9 @@ try if (!save_into_file.empty()) { + if (!silent) + std::cerr << "Saving models\n"; + WriteBufferFromFile model_file_out(save_into_file); CompressedWriteBuffer model_out(model_file_out, CompressionCodecFactory::instance().get("ZSTD", 1)); From 4cbf4741fb787461fda6e867d9f0bf49e3ca03db Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Jul 2022 04:16:07 +0200 Subject: [PATCH 607/659] Add a test --- .../00096_obfuscator_save_load.reference | 3 +++ .../1_stateful/00096_obfuscator_save_load.sh | 18 ++++++++++++++++++ 2 files changed, 21 insertions(+) create mode 100644 tests/queries/1_stateful/00096_obfuscator_save_load.reference create mode 100755 tests/queries/1_stateful/00096_obfuscator_save_load.sh diff --git a/tests/queries/1_stateful/00096_obfuscator_save_load.reference b/tests/queries/1_stateful/00096_obfuscator_save_load.reference new file mode 100644 index 00000000000..ccab5c82eaa --- /dev/null +++ b/tests/queries/1_stateful/00096_obfuscator_save_load.reference @@ -0,0 +1,3 @@ +363464 +1000 395 195 35 +2500 650 414 36 diff --git a/tests/queries/1_stateful/00096_obfuscator_save_load.sh b/tests/queries/1_stateful/00096_obfuscator_save_load.sh new file mode 100755 index 00000000000..c90eee1d0f9 --- /dev/null +++ b/tests/queries/1_stateful/00096_obfuscator_save_load.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --max_threads 1 --query="SELECT URL, Title, SearchPhrase FROM test.hits LIMIT 1000" > "${CLICKHOUSE_TMP}"/data.tsv + +$CLICKHOUSE_OBFUSCATOR --structure "URL String, Title String, SearchPhrase String" --input-format TSV --output-format TSV --seed hello --limit 0 --save "${CLICKHOUSE_TMP}"/model.bin < "${CLICKHOUSE_TMP}"/data.tsv 2>/dev/null +wc -c < "${CLICKHOUSE_TMP}"/model.bin +$CLICKHOUSE_OBFUSCATOR --structure "URL String, Title String, SearchPhrase String" --input-format TSV --output-format TSV --seed hello --limit 2500 --load "${CLICKHOUSE_TMP}"/model.bin < "${CLICKHOUSE_TMP}"/data.tsv > "${CLICKHOUSE_TMP}"/data2500.tsv 2>/dev/null +rm "${CLICKHOUSE_TMP}"/model.bin + +$CLICKHOUSE_LOCAL --structure "URL String, Title String, SearchPhrase String" --input-format TSV --output-format TSV --query "SELECT count(), uniq(URL), uniq(Title), uniq(SearchPhrase) FROM table" < "${CLICKHOUSE_TMP}"/data.tsv +$CLICKHOUSE_LOCAL --structure "URL String, Title String, SearchPhrase String" --input-format TSV --output-format TSV --query "SELECT count(), uniq(URL), uniq(Title), uniq(SearchPhrase) FROM table" < "${CLICKHOUSE_TMP}"/data2500.tsv + +rm "${CLICKHOUSE_TMP}"/data.tsv +rm "${CLICKHOUSE_TMP}"/data2500.tsv From 1f4e7ea34c70899e3765acd3fa236e8ef330616e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Jul 2022 05:37:07 +0200 Subject: [PATCH 608/659] Add a test --- programs/obfuscator/Obfuscator.cpp | 1 - .../queries/1_stateful/00096_obfuscator_save_load.reference | 6 +++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 996d0316b4d..056ed85c670 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -766,7 +766,6 @@ public: } } - void finalize() { if (params.num_buckets_cutoff) diff --git a/tests/queries/1_stateful/00096_obfuscator_save_load.reference b/tests/queries/1_stateful/00096_obfuscator_save_load.reference index ccab5c82eaa..af0e6eade55 100644 --- a/tests/queries/1_stateful/00096_obfuscator_save_load.reference +++ b/tests/queries/1_stateful/00096_obfuscator_save_load.reference @@ -1,3 +1,3 @@ -363464 -1000 395 195 35 -2500 650 414 36 +403499 +1000 320 171 23 +2500 569 354 13 From dc96a6270d557d4c2bb65a473fffe50b36582576 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 25 Jul 2022 10:27:33 +0000 Subject: [PATCH 609/659] Remove optimize_memory_usage flag Memory usage with DistinctSortedTransform fixed in #39538 --- src/Core/Settings.h | 1 - src/Interpreters/InterpreterSelectQuery.cpp | 3 +-- .../InterpreterSelectWithUnionQuery.cpp | 3 +-- src/Processors/QueryPlan/DistinctStep.cpp | 14 ++++++-------- src/Processors/QueryPlan/DistinctStep.h | 4 +--- ...02317_distinct_in_order_optimization.reference | 1 - .../02317_distinct_in_order_optimization.sql | 11 ++--------- ...stinct_in_order_optimization_explain.reference | 9 +++------ ...2317_distinct_in_order_optimization_explain.sh | 15 +++++++-------- 9 files changed, 21 insertions(+), 40 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0a6f4bee5ba..17e4d27bbcd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -624,7 +624,6 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, compatibility_ignore_auto_increment_in_create_table, false, "Ignore AUTO_INCREMENT keyword in column declaration if true, otherwise return error. It simplifies migration from MySQL", 0) \ M(Bool, multiple_joins_try_to_keep_original_names, false, "Do not add aliases to top level expression list on multiple joins rewrite", 0) \ M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ - M(Bool, optimize_distinct_in_order_memory_usage, false, "Try to use less memory for DISTINCT in order but can be slower", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 279694f917e..24bbaea7dcf 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2684,8 +2684,7 @@ void InterpreterSelectQuery::executeDistinct(QueryPlan & query_plan, bool before limit_for_distinct, columns, pre_distinct, - settings.optimize_distinct_in_order, - settings.optimize_distinct_in_order_memory_usage); + settings.optimize_distinct_in_order); if (pre_distinct) distinct_step->setStepDescription("Preliminary DISTINCT"); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index f01a1210c3a..b2622607760 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -334,8 +334,7 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) 0, result_header.getNames(), false, - settings.optimize_distinct_in_order, - settings.optimize_distinct_in_order_memory_usage); + settings.optimize_distinct_in_order); query_plan.addStep(std::move(distinct_step)); } diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index 9223218d82b..c268cb44267 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -80,8 +80,7 @@ DistinctStep::DistinctStep( UInt64 limit_hint_, const Names & columns_, bool pre_distinct_, - bool optimize_distinct_in_order_, - bool optimize_distinct_in_order_memory_usage_) + bool optimize_distinct_in_order_) : ITransformingStep( input_stream_, input_stream_.header, @@ -91,7 +90,6 @@ DistinctStep::DistinctStep( , columns(columns_) , pre_distinct(pre_distinct_) , optimize_distinct_in_order(optimize_distinct_in_order_) - , optimize_distinct_in_order_memory_usage(optimize_distinct_in_order_memory_usage_) { if (!output_stream->distinct_columns.empty() /// Columns already distinct, do nothing && (!pre_distinct /// Main distinct @@ -140,7 +138,7 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil { assert(input_stream.has_single_port); - if (distinct_sort_desc.size() >= columns.size() || optimize_distinct_in_order_memory_usage) + if (distinct_sort_desc.size() < columns.size()) { pipeline.addSimpleTransform( [&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr @@ -148,8 +146,8 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - return std::make_shared( - header, set_size_limits, limit_hint, distinct_sort_desc, columns, true); + return std::make_shared( + header, distinct_sort_desc, set_size_limits, limit_hint, columns); }); } else @@ -160,8 +158,8 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - return std::make_shared( - header, distinct_sort_desc, set_size_limits, limit_hint, columns); + return std::make_shared( + header, set_size_limits, limit_hint, distinct_sort_desc, columns, true); }); } return; diff --git a/src/Processors/QueryPlan/DistinctStep.h b/src/Processors/QueryPlan/DistinctStep.h index 0d3b7e3b7e0..dc734a58704 100644 --- a/src/Processors/QueryPlan/DistinctStep.h +++ b/src/Processors/QueryPlan/DistinctStep.h @@ -15,8 +15,7 @@ public: UInt64 limit_hint_, const Names & columns_, bool pre_distinct_, /// If is enabled, execute distinct for separate streams. Otherwise, merge streams. - bool optimize_distinct_in_order_, - bool optimize_distinct_in_order_memory_usage_); + bool optimize_distinct_in_order_); String getName() const override { return "Distinct"; } @@ -33,7 +32,6 @@ private: Names columns; bool pre_distinct; bool optimize_distinct_in_order; - bool optimize_distinct_in_order_memory_usage; }; } diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference index a82e9b5cc2e..a5ae3759d5d 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization.reference @@ -78,4 +78,3 @@ 1 1 0 0 -- check that distinct in order has the same result as ordinary distinct --- check that distinct in order has the same result as ordinary distinct, optimize memory usage diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql index 14618baff95..f1de07e3db2 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql @@ -49,19 +49,12 @@ select '-- check that distinct in order has the same result as ordinary distinct drop table if exists distinct_cardinality_low sync; CREATE TABLE distinct_cardinality_low (low UInt64, medium UInt64, high UInt64) ENGINE MergeTree() ORDER BY (low, medium); INSERT INTO distinct_cardinality_low SELECT number % 1e1, number % 1e2, number % 1e3 FROM numbers_mt(1e4); + drop table if exists distinct_in_order sync; drop table if exists ordinary_distinct sync; -create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); -insert into distinct_in_order select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=1, optimize_distinct_in_order_memory_usage=0; -create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); -insert into ordinary_distinct select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=0; -select distinct * from distinct_in_order except select * from ordinary_distinct; -select '-- check that distinct in order has the same result as ordinary distinct, optimize memory usage'; -drop table if exists distinct_in_order; -drop table if exists ordinary_distinct; create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); -insert into distinct_in_order select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=1, optimize_distinct_in_order_memory_usage=1; +insert into distinct_in_order select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=1; create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); insert into ordinary_distinct select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=0; select distinct * from distinct_in_order except select * from ordinary_distinct; diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index b41d853e3eb..3e57d4de586 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -1,5 +1,5 @@ -- disable optimize_distinct_in_order --- distinct all primary key columns -> no optimizations +-- distinct all primary key columns -> ordinary distinct DistinctTransform DistinctTransform -- enable optimize_distinct_in_order @@ -18,18 +18,15 @@ DistinctSortedChunkTransform -- distinct with primary key prefix and order by column in distinct but non-primary key prefix -> pre-distinct and final distinct optimization DistinctSortedTransform DistinctSortedChunkTransform --- distinct with primary key prefix and order by column in distinct but non-primary key prefix, optimize memory usage -> pre-distinct and final distinct optimization -DistinctSortedStreamTransform -DistinctSortedChunkTransform -- distinct with primary key prefix and order by on column _not_ in distinct -> pre-distinct optimization only DistinctTransform DistinctSortedChunkTransform --- distinct with non-primary key prefix -> no optimizations +-- distinct with non-primary key prefix -> ordinary distinct DistinctTransform DistinctTransform -- distinct with non-primary key prefix and order by on column in distinct -> final distinct optimization only DistinctSortedTransform DistinctTransform --- distinct with non-primary key prefix and order by on column _not_ in distinct -> no optimizations +-- distinct with non-primary key prefix and order by on column _not_ in distinct -> ordinary distinct DistinctTransform DistinctTransform diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh index c19bc3e6aaf..903f3bb9e11 100755 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh @@ -13,11 +13,11 @@ TRIM_LEADING_SPACES="sed -e 's/^[ \t]*//'" FIND_DISTINCT="$GREP_DISTINCT | $TRIM_LEADING_SPACES" $CLICKHOUSE_CLIENT -q "drop table if exists distinct_in_order_explain sync" -$CLICKHOUSE_CLIENT -q "create table distinct_in_order_explain (a int, b int, c int) engine=MergeTree() order by (a, b, c)" +$CLICKHOUSE_CLIENT -q "create table distinct_in_order_explain (a int, b int, c int) engine=MergeTree() order by (a, b)" $CLICKHOUSE_CLIENT -q "insert into distinct_in_order_explain select number % number, number % 5, number % 10 from numbers(1,10)" $CLICKHOUSE_CLIENT -q "select '-- disable optimize_distinct_in_order'" -$CLICKHOUSE_CLIENT -q "select '-- distinct all primary key columns -> no optimizations'" +$CLICKHOUSE_CLIENT -q "select '-- distinct all primary key columns -> ordinary distinct'" $CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- enable optimize_distinct_in_order'" @@ -34,19 +34,18 @@ $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a, b" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct but non-primary key prefix -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c settings optimize_distinct_in_order_memory_usage=0" | eval $FIND_DISTINCT - -$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct but non-primary key prefix, optimize memory usage -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c settings optimize_distinct_in_order_memory_usage=1" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by on column _not_ in distinct -> pre-distinct optimization only'" $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT -$CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix -> no optimizations'" +$CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix -> ordinary distinct'" $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by on column in distinct -> final distinct optimization only'" $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT -$CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by on column _not_ in distinct -> no optimizations'" +$CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by on column _not_ in distinct -> ordinary distinct'" $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by a" | eval $FIND_DISTINCT + +$CLICKHOUSE_CLIENT -q "drop table if exists distinct_in_order_explain sync" From dd8667d3f995bcfa92ec546ad51cfb6ad1f7e89e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 25 Jul 2022 13:02:25 +0200 Subject: [PATCH 610/659] Update settings description --- src/Core/Settings.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index fcb23fe2d39..61ea25bcc3d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -696,11 +696,11 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, input_format_max_rows_to_read_for_schema_inference, 25000, "The maximum rows of data to read for automatic schema inference", 0) \ M(Bool, input_format_csv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in CSV format", 0) \ M(Bool, input_format_tsv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in TSV format", 0) \ - M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Allow to skip columns with unsupported types while schema inference for format Parquet", 0) \ - M(Bool, input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference, false, "Allow to skip fields with unsupported types while schema inference for format Protobuf", 0) \ - M(Bool, input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference, false, "Allow to skip columns with unsupported types while schema inference for format CapnProto", 0) \ - M(Bool, input_format_orc_skip_columns_with_unsupported_types_in_schema_inference, false, "Allow to skip columns with unsupported types while schema inference for format ORC", 0) \ - M(Bool, input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference, false, "Allow to skip columns with unsupported types while schema inference for format Arrow", 0) \ + M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ + M(Bool, input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format Protobuf", 0) \ + M(Bool, input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format CapnProto", 0) \ + M(Bool, input_format_orc_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format ORC", 0) \ + M(Bool, input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Arrow", 0) \ M(String, column_names_for_schema_inference, "", "The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'", 0) \ M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \ M(Bool, input_format_protobuf_flatten_google_wrappers, false, "Enable Google wrappers for regular non-nested columns, e.g. google.protobuf.StringValue 'str' for String column 'str'. For Nullable columns empty wrappers are recognized as defaults, and missing as nulls", 0) \ From 33ee858d18bcc4d9d80e8754d01e5d1cce507b75 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Mon, 25 Jul 2022 13:11:02 +0200 Subject: [PATCH 611/659] Fix bug with maxsplit in the splitByChar --- src/Functions/FunctionsStringArray.h | 1 + tests/queries/0_stateless/02185_split_by_char.reference | 2 ++ tests/queries/0_stateless/02185_split_by_char.sql | 4 +++- 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsStringArray.h b/src/Functions/FunctionsStringArray.h index b0f415be58a..95c06ae74ca 100644 --- a/src/Functions/FunctionsStringArray.h +++ b/src/Functions/FunctionsStringArray.h @@ -337,6 +337,7 @@ public: { pos = pos_; end = end_; + curr_split = 0; } bool get(Pos & token_begin, Pos & token_end) diff --git a/tests/queries/0_stateless/02185_split_by_char.reference b/tests/queries/0_stateless/02185_split_by_char.reference index f69d8d35006..6afd0824a73 100644 --- a/tests/queries/0_stateless/02185_split_by_char.reference +++ b/tests/queries/0_stateless/02185_split_by_char.reference @@ -3,3 +3,5 @@ ['1','2,3'] ['1','2','3'] ['1','2','3'] +['expr1','1+1=2'] +['expr2','2+2=4=1+3'] diff --git a/tests/queries/0_stateless/02185_split_by_char.sql b/tests/queries/0_stateless/02185_split_by_char.sql index 6c490654813..c45f3de07eb 100644 --- a/tests/queries/0_stateless/02185_split_by_char.sql +++ b/tests/queries/0_stateless/02185_split_by_char.sql @@ -5,4 +5,6 @@ select splitByChar(',', '1,2,3', 2); select splitByChar(',', '1,2,3', 3); select splitByChar(',', '1,2,3', -2); -- { serverError 44 } -select splitByChar(',', '1,2,3', ''); -- { serverError 43 } \ No newline at end of file +select splitByChar(',', '1,2,3', ''); -- { serverError 43 } + +SELECT splitByChar('=', s, 1) FROM values('s String', 'expr1=1+1=2', 'expr2=2+2=4=1+3') From dbcb7e5f1e03e95bd37b1327b250ad9978b1c5da Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 25 Jul 2022 13:11:36 +0200 Subject: [PATCH 612/659] Fix for empty function name on FreeBSD build --- src/Functions/array/arrayDistance.cpp | 15 +++++++-------- src/Functions/array/arrayNorm.cpp | 13 ++++++------- src/Functions/vectorFunctions.cpp | 24 ++++++++++++------------ 3 files changed, 25 insertions(+), 27 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index c3652e10644..d829c11b9de 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes struct L1Distance { - static inline String name = "L1"; + static constexpr auto name = "L1"; struct ConstParams {}; @@ -53,7 +53,7 @@ struct L1Distance struct L2Distance { - static inline String name = "L2"; + static constexpr auto name = "L2"; struct ConstParams {}; @@ -84,7 +84,7 @@ struct L2Distance struct L2SquaredDistance : L2Distance { - static inline String name = "L2Squared"; + static constexpr auto name = "L2Squared"; template static ResultType finalize(const State & state, const ConstParams &) @@ -95,7 +95,7 @@ struct L2SquaredDistance : L2Distance struct LpDistance { - static inline String name = "Lp"; + static constexpr auto name = "Lp"; struct ConstParams { @@ -130,7 +130,7 @@ struct LpDistance struct LinfDistance { - static inline String name = "Linf"; + static constexpr auto name = "Linf"; struct ConstParams {}; @@ -161,7 +161,7 @@ struct LinfDistance struct CosineDistance { - static inline String name = "Cosine"; + static constexpr auto name = "Cosine"; struct ConstParams {}; @@ -200,8 +200,7 @@ template class FunctionArrayDistance : public IFunction { public: - static inline auto name = "array" + Kernel::name + "Distance"; - String getName() const override { return name; } + String getName() const override { static auto name = String("array") + Kernel::name + "Distance"; return name; } static FunctionPtr create(ContextPtr) { return std::make_shared>(); } size_t getNumberOfArguments() const override { return 2; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; } diff --git a/src/Functions/array/arrayNorm.cpp b/src/Functions/array/arrayNorm.cpp index e1e7935fcb1..401010bee03 100644 --- a/src/Functions/array/arrayNorm.cpp +++ b/src/Functions/array/arrayNorm.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes struct L1Norm { - static inline String name = "L1"; + static constexpr auto name = "L1"; struct ConstParams {}; @@ -46,7 +46,7 @@ struct L1Norm struct L2Norm { - static inline String name = "L2"; + static constexpr auto name = "L2"; struct ConstParams {}; @@ -71,7 +71,7 @@ struct L2Norm struct L2SquaredNorm : L2Norm { - static inline String name = "L2Squared"; + static constexpr auto name = "L2Squared"; template inline static ResultType finalize(ResultType result, const ConstParams &) @@ -83,7 +83,7 @@ struct L2SquaredNorm : L2Norm struct LpNorm { - static inline String name = "Lp"; + static constexpr auto name = "Lp"; struct ConstParams { @@ -112,7 +112,7 @@ struct LpNorm struct LinfNorm { - static inline String name = "Linf"; + static constexpr auto name = "Linf"; struct ConstParams {}; @@ -140,8 +140,7 @@ template class FunctionArrayNorm : public IFunction { public: - static inline auto name = "array" + Kernel::name + "Norm"; - String getName() const override { return name; } + String getName() const override { static auto name = String("array") + Kernel::name + "Norm"; return name; } static FunctionPtr create(ContextPtr) { return std::make_shared>(); } size_t getNumberOfArguments() const override { return 1; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; } diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 7974a8bbaf4..d46d12379a7 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -1124,7 +1124,7 @@ template class TupleOrArrayFunction : public IFunction { public: - static inline String name = Traits::name; + static constexpr auto name = Traits::name; explicit TupleOrArrayFunction(ContextPtr context_) : IFunction() @@ -1173,7 +1173,7 @@ extern FunctionPtr createFunctionArrayCosineDistance(ContextPtr context_); struct L1NormTraits { - static inline String name = "L1Norm"; + static constexpr auto name = "L1Norm"; static constexpr auto CreateTupleFunction = FunctionL1Norm::create; static constexpr auto CreateArrayFunction = createFunctionArrayL1Norm; @@ -1181,7 +1181,7 @@ struct L1NormTraits struct L2NormTraits { - static inline String name = "L2Norm"; + static constexpr auto name = "L2Norm"; static constexpr auto CreateTupleFunction = FunctionL2Norm::create; static constexpr auto CreateArrayFunction = createFunctionArrayL2Norm; @@ -1189,7 +1189,7 @@ struct L2NormTraits struct L2SquaredNormTraits { - static inline String name = "L2SquaredNorm"; + static constexpr auto name = "L2SquaredNorm"; static constexpr auto CreateTupleFunction = FunctionL2SquaredNorm::create; static constexpr auto CreateArrayFunction = createFunctionArrayL2SquaredNorm; @@ -1197,7 +1197,7 @@ struct L2SquaredNormTraits struct LpNormTraits { - static inline String name = "LpNorm"; + static constexpr auto name = "LpNorm"; static constexpr auto CreateTupleFunction = FunctionLpNorm::create; static constexpr auto CreateArrayFunction = createFunctionArrayLpNorm; @@ -1205,7 +1205,7 @@ struct LpNormTraits struct LinfNormTraits { - static inline String name = "LinfNorm"; + static constexpr auto name = "LinfNorm"; static constexpr auto CreateTupleFunction = FunctionLinfNorm::create; static constexpr auto CreateArrayFunction = createFunctionArrayLinfNorm; @@ -1213,7 +1213,7 @@ struct LinfNormTraits struct L1DistanceTraits { - static inline String name = "L1Distance"; + static constexpr auto name = "L1Distance"; static constexpr auto CreateTupleFunction = FunctionL1Distance::create; static constexpr auto CreateArrayFunction = createFunctionArrayL1Distance; @@ -1221,7 +1221,7 @@ struct L1DistanceTraits struct L2DistanceTraits { - static inline String name = "L2Distance"; + static constexpr auto name = "L2Distance"; static constexpr auto CreateTupleFunction = FunctionL2Distance::create; static constexpr auto CreateArrayFunction = createFunctionArrayL2Distance; @@ -1229,7 +1229,7 @@ struct L2DistanceTraits struct L2SquaredDistanceTraits { - static inline String name = "L2SquaredDistance"; + static constexpr auto name = "L2SquaredDistance"; static constexpr auto CreateTupleFunction = FunctionL2SquaredDistance::create; static constexpr auto CreateArrayFunction = createFunctionArrayL2SquaredDistance; @@ -1237,7 +1237,7 @@ struct L2SquaredDistanceTraits struct LpDistanceTraits { - static inline String name = "LpDistance"; + static constexpr auto name = "LpDistance"; static constexpr auto CreateTupleFunction = FunctionLpDistance::create; static constexpr auto CreateArrayFunction = createFunctionArrayLpDistance; @@ -1245,7 +1245,7 @@ struct LpDistanceTraits struct LinfDistanceTraits { - static inline String name = "LinfDistance"; + static constexpr auto name = "LinfDistance"; static constexpr auto CreateTupleFunction = FunctionLinfDistance::create; static constexpr auto CreateArrayFunction = createFunctionArrayLinfDistance; @@ -1253,7 +1253,7 @@ struct LinfDistanceTraits struct CosineDistanceTraits { - static inline String name = "cosineDistance"; + static constexpr auto name = "cosineDistance"; static constexpr auto CreateTupleFunction = FunctionCosineDistance::create; static constexpr auto CreateArrayFunction = createFunctionArrayCosineDistance; From b6f640013d8d6d96b3e78e42561d5667de1535e8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 25 Jul 2022 12:40:11 +0000 Subject: [PATCH 613/659] Fix bug in ASOF JOIN with enable_optimize_predicate_expression --- src/Interpreters/JoinedTables.cpp | 11 ++++++++++- ..._asof_optimize_predicate_bug_37813.reference | 2 ++ .../02366_asof_optimize_predicate_bug_37813.sql | 17 +++++++++++++++++ 3 files changed, 29 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02366_asof_optimize_predicate_bug_37813.reference create mode 100644 tests/queries/0_stateless/02366_asof_optimize_predicate_bug_37813.sql diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index df47e8acdca..27c372ab5c7 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -59,11 +59,20 @@ void replaceJoinedTable(const ASTSelectQuery & select_query) if (!join || !join->table_expression) return; - /// TODO: Push down for CROSS JOIN is not OK [disabled] const auto & table_join = join->table_join->as(); + + /// TODO: Push down for CROSS JOIN is not OK [disabled] if (table_join.kind == ASTTableJoin::Kind::Cross) return; + /* Do not push down predicates for ASOF because it can lead to incorrect results + * (for example, if we will filter a suitable row before joining and will choose another, not the closest row). + * ANY join behavior can also be different with this optimization, + * but it's ok because we don't guarantee which row to choose for ANY, unlike ASOF, where we have to pick the closest one. + */ + if (table_join.strictness == ASTTableJoin::Strictness::Asof) + return; + auto & table_expr = join->table_expression->as(); if (table_expr.database_and_table_name) { diff --git a/tests/queries/0_stateless/02366_asof_optimize_predicate_bug_37813.reference b/tests/queries/0_stateless/02366_asof_optimize_predicate_bug_37813.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/02366_asof_optimize_predicate_bug_37813.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/02366_asof_optimize_predicate_bug_37813.sql b/tests/queries/0_stateless/02366_asof_optimize_predicate_bug_37813.sql new file mode 100644 index 00000000000..4db42a22560 --- /dev/null +++ b/tests/queries/0_stateless/02366_asof_optimize_predicate_bug_37813.sql @@ -0,0 +1,17 @@ +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (c1 Int32, c2 Int32) ENGINE MergeTree ORDER BY c1; +INSERT INTO t1 (c1, c2) VALUES (1, 10), (1, 20), (1, 30); + +DROP TABLE IF EXISTS t2; +CREATE TABLE t2 (c1 Int32, c2 Int32, c3 String) ENGINE MergeTree ORDER BY (c1, c2, c3); +INSERT INTO t2 (c1, c2, c3) VALUES (1, 5, 'a'), (1, 15, 'b'), (1, 25, 'c'); + +SET enable_optimize_predicate_expression = 1; +WITH + v1 AS (SELECT t1.c2, t2.c2, t2.c3 FROM t1 ASOF JOIN t2 USING (c1, c2)) + SELECT count() FROM v1 WHERE c3 = 'b'; + +SET enable_optimize_predicate_expression = 0; +WITH + v1 AS (SELECT t1.c2, t2.c2, t2.c3 FROM t1 ASOF JOIN t2 USING (c1, c2)) + SELECT count() FROM v1 WHERE c3 = 'b'; From 917fbb9b07765571319203b472e5e5a982755144 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 25 Jul 2022 15:49:03 +0200 Subject: [PATCH 614/659] remove c-ares from grpc --- contrib/CMakeLists.txt | 2 +- contrib/grpc | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 3bbdd061d96..08b91c1b81c 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -156,7 +156,7 @@ endif() add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) -add_contrib(c-ares-cmake c-ares) +add_contrib (c-ares-cmake c-ares) add_contrib (qpl-cmake qpl) # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. diff --git a/contrib/grpc b/contrib/grpc index 5e23e96c0c0..3f975ecab37 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 5e23e96c0c02e451dbb291cf9f66231d02b6cdb6 +Subproject commit 3f975ecab377cd5f739af780566596128f17bb74 From 7b44950d1dca791b02a6154095aea1635ec9dd2c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 25 Jul 2022 16:14:33 +0200 Subject: [PATCH 615/659] More detailed comment --- src/Storages/MergeTree/MutateTask.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 59c784208c5..e09c87311aa 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1483,7 +1483,8 @@ bool MutateTask::prepare() ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices(); ctx->materialized_projections = ctx->interpreter->grabMaterializedProjections(); ctx->mutation_kind = ctx->interpreter->getMutationKind(); - /// Always disable filtering in mutations, we want to read all rows + /// Always disable filtering in mutations: we want to read and write all rows because for updates we rewrite only some of the + /// columns and preserve the columns that are not affected, but after the update all columns must have the same number of rows. ctx->interpreter->setApplyDeletedMask(false); ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); From 48de02a7b89e868ae44b25d8abb4dd95d1710336 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 25 Jul 2022 16:15:15 +0200 Subject: [PATCH 616/659] Capitalized const name --- src/Interpreters/InterpreterDeleteQuery.cpp | 2 +- src/Interpreters/MutationsInterpreter.cpp | 8 ++++---- src/Storages/AlterCommands.cpp | 4 ++-- src/Storages/LightweightDeleteDescription.cpp | 2 +- src/Storages/LightweightDeleteDescription.h | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTask.cpp | 2 +- .../MergeTree/MergeTreeBaseSelectProcessor.cpp | 10 +++++----- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 2 +- src/Storages/StorageSnapshot.cpp | 2 +- 14 files changed, 22 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 8c8030c6a51..47b0050e46c 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -82,7 +82,7 @@ BlockIO InterpreterDeleteQuery::execute() command->predicate = delete_query.predicate; command->update_assignments = std::make_shared(); auto set_row_does_not_exist = std::make_shared(); - set_row_does_not_exist->column_name = LightweightDeleteDescription::filter_column.name; + set_row_does_not_exist->column_name = LightweightDeleteDescription::FILTER_COLUMN.name; auto zero_value = std::make_shared(DB::Field(UInt8(0))); set_row_does_not_exist->children.push_back(zero_value); command->update_assignments->children.push_back(set_row_does_not_exist); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 64ea313a1af..7b2d55911d1 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -351,7 +351,7 @@ static void validateUpdateColumns( } /// Allow to override value of lightweight delete filter virtual column - if (!found && column_name == LightweightDeleteDescription::filter_column.name) + if (!found && column_name == LightweightDeleteDescription::FILTER_COLUMN.name) { if (!storage->supportsLightweightDelete()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); @@ -514,8 +514,8 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) DataTypePtr type; if (auto physical_column = columns_desc.tryGetPhysical(column)) type = physical_column->type; - else if (column == LightweightDeleteDescription::filter_column.name) - type = LightweightDeleteDescription::filter_column.type; + else if (column == LightweightDeleteDescription::FILTER_COLUMN.name) + type = LightweightDeleteDescription::FILTER_COLUMN.type; else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown column {}", column); @@ -782,7 +782,7 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & if (auto part_storage = dynamic_pointer_cast(storage)) { if (part_storage->hasLightweightDeletedMask()) - all_columns.push_back({LightweightDeleteDescription::filter_column}); + all_columns.push_back({LightweightDeleteDescription::FILTER_COLUMN}); } /// Next, for each stage calculate columns changed by this and previous stages. diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 808d634b1ea..7f99abf31fc 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -787,7 +787,7 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada /// Drop alias is metadata alter, in other case mutation is required. if (type == DROP_COLUMN) return metadata.columns.hasColumnOrNested(GetColumnsOptions::AllPhysical, column_name) || - column_name == LightweightDeleteDescription::filter_column.name; + column_name == LightweightDeleteDescription::FILTER_COLUMN.name; if (type != MODIFY_COLUMN || data_type == nullptr) return false; @@ -1153,7 +1153,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const { if (all_columns.has(command.column_name) || all_columns.hasNested(command.column_name) || - (command.clear && column_name == LightweightDeleteDescription::filter_column.name)) + (command.clear && column_name == LightweightDeleteDescription::FILTER_COLUMN.name)) { if (!command.clear) /// CLEAR column is Ok even if there are dependencies. { diff --git a/src/Storages/LightweightDeleteDescription.cpp b/src/Storages/LightweightDeleteDescription.cpp index 0ffb7766c80..ae5e68da9c2 100644 --- a/src/Storages/LightweightDeleteDescription.cpp +++ b/src/Storages/LightweightDeleteDescription.cpp @@ -4,6 +4,6 @@ namespace DB { -const NameAndTypePair LightweightDeleteDescription::filter_column {"_row_exists", std::make_shared()}; +const NameAndTypePair LightweightDeleteDescription::FILTER_COLUMN {"_row_exists", std::make_shared()}; } diff --git a/src/Storages/LightweightDeleteDescription.h b/src/Storages/LightweightDeleteDescription.h index 7177b2a5b54..45bde59ea71 100644 --- a/src/Storages/LightweightDeleteDescription.h +++ b/src/Storages/LightweightDeleteDescription.h @@ -7,7 +7,7 @@ namespace DB struct LightweightDeleteDescription { - static const NameAndTypePair filter_column; + static const NameAndTypePair FILTER_COLUMN; }; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index b205da4d4c2..1218740daa0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -461,7 +461,7 @@ public: bool supportLightweightDeleteMutate() const; /// True if here is lightweight deleted mask file in part. - bool hasLightweightDelete() const { return columns.contains(LightweightDeleteDescription::filter_column.name); } + bool hasLightweightDelete() const { return columns.contains(LightweightDeleteDescription::FILTER_COLUMN.name); } protected: diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 77d3089d37b..e4eadf9adf7 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -814,7 +814,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() auto columns = global_ctx->merging_column_names; /// The part might have some rows masked by lightweight deletes - const auto lightweight_delete_filter_column = LightweightDeleteDescription::filter_column.name; + const auto lightweight_delete_filter_column = LightweightDeleteDescription::FILTER_COLUMN.name; const bool need_to_filter_deleted_rows = part->hasLightweightDelete(); if (need_to_filter_deleted_rows) columns.emplace_back(lightweight_delete_filter_column); diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 00b9959739f..08142bd8dd1 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -61,7 +61,7 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( { non_const_virtual_column_names.emplace_back(*it); } - else if (*it == LightweightDeleteDescription::filter_column.name) + else if (*it == LightweightDeleteDescription::FILTER_COLUMN.name) { non_const_virtual_column_names.emplace_back(*it); } @@ -244,7 +244,7 @@ void MergeTreeBaseSelectProcessor::initializeMergeTreeReadersForPart( /// Add lightweight delete filtering step if (reader_settings.apply_deleted_mask && data_part->hasLightweightDelete()) { - pre_reader_for_step.push_back(data_part->getReader({LightweightDeleteDescription::filter_column}, metadata_snapshot, mark_ranges, + pre_reader_for_step.push_back(data_part->getReader({LightweightDeleteDescription::FILTER_COLUMN}, metadata_snapshot, mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, value_size_map, profile_callback)); } @@ -469,14 +469,14 @@ static void injectNonConstVirtualColumns( } } - if (virtual_column_name == LightweightDeleteDescription::filter_column.name) + if (virtual_column_name == LightweightDeleteDescription::FILTER_COLUMN.name) { /// If _row_exists column isn't present in the part then fill it here with 1s ColumnPtr column; if (rows) - column = LightweightDeleteDescription::filter_column.type->createColumnConst(rows, 1)->convertToFullColumnIfConst(); + column = LightweightDeleteDescription::FILTER_COLUMN.type->createColumnConst(rows, 1)->convertToFullColumnIfConst(); else - column = LightweightDeleteDescription::filter_column.type->createColumn(); + column = LightweightDeleteDescription::FILTER_COLUMN.type->createColumn(); inserter.insertUInt8Column(column, virtual_column_name); } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 1a04c2ef25f..aa1b9d3541e 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -103,7 +103,7 @@ protected: StorageSnapshotPtr storage_snapshot; /// This step is added when the part has lightweight delete mask - const PrewhereExprStep lightweight_delete_filter_step { nullptr, LightweightDeleteDescription::filter_column.name, true, true }; + const PrewhereExprStep lightweight_delete_filter_step { nullptr, LightweightDeleteDescription::FILTER_COLUMN.name, true, true }; PrewhereInfoPtr prewhere_info; std::unique_ptr prewhere_actions; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f20121898ee..53425bc02f3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6575,7 +6575,7 @@ NamesAndTypesList MergeTreeData::getVirtuals() const NameAndTypePair("_partition_value", getPartitionValueType()), NameAndTypePair("_sample_factor", std::make_shared()), NameAndTypePair("_part_offset", std::make_shared()), - LightweightDeleteDescription::filter_column, + LightweightDeleteDescription::FILTER_COLUMN, }; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 0a7b3071559..a96486c79ae 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1215,7 +1215,7 @@ static void selectColumnNames( { virt_column_names.push_back(name); } - else if (name == LightweightDeleteDescription::filter_column.name) + else if (name == LightweightDeleteDescription::FILTER_COLUMN.name) { virt_column_names.push_back(name); } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index acc90fe7313..a10192c4cc1 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -1150,7 +1150,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r block.insert({result.columns[pos], std::make_shared(), column_name}); } - else if (column_name == LightweightDeleteDescription::filter_column.name) + else if (column_name == LightweightDeleteDescription::FILTER_COLUMN.name) { /// Do nothing, it will be added later } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index e09c87311aa..7d0c37051e7 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -172,7 +172,7 @@ getColumnsForNewDataPart( ColumnsDescription part_columns(source_part->getColumns()); NamesAndTypesList system_columns; if (source_part->supportLightweightDeleteMutate()) - system_columns.push_back(LightweightDeleteDescription::filter_column); + system_columns.push_back(LightweightDeleteDescription::FILTER_COLUMN); /// Preserve system columns that have persisted values in the source_part for (const auto & column : system_columns) diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index 5f9857b28ef..a99fec8c154 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -22,7 +22,7 @@ void StorageSnapshot::init() virtual_columns[name] = type; if (storage.hasLightweightDeletedMask()) - system_columns[LightweightDeleteDescription::filter_column.name] = LightweightDeleteDescription::filter_column.type; + system_columns[LightweightDeleteDescription::FILTER_COLUMN.name] = LightweightDeleteDescription::FILTER_COLUMN.type; } NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options) const From 5aae0a2e044627f67e8889f2d7b24c0b2f532d7d Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 25 Jul 2022 17:20:01 +0200 Subject: [PATCH 617/659] Fix style --- src/Formats/CapnProtoUtils.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index 84af46de5de..b8702380aa7 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -29,6 +29,7 @@ namespace ErrorCodes extern const int UNKNOWN_EXCEPTION; extern const int INCORRECT_DATA; extern const int CAPN_PROTO_BAD_TYPE; + extern const int BAD_ARGUMENTS; } capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaInfo & schema_info) From 9ffaf2fef2de730d9724615cae6235a381ff3086 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 25 Jul 2022 17:32:51 +0200 Subject: [PATCH 618/659] temporarily disable all tests with materialized postgres --- .../test_postgresql_replica_database_engine_1/test.py | 8 ++++++++ .../test_postgresql_replica_database_engine_2/test.py | 8 ++++++++ tests/integration/test_storage_postgresql_replica/test.py | 8 ++++++++ 3 files changed, 24 insertions(+) diff --git a/tests/integration/test_postgresql_replica_database_engine_1/test.py b/tests/integration/test_postgresql_replica_database_engine_1/test.py index e7b642b5028..5072c261cf7 100644 --- a/tests/integration/test_postgresql_replica_database_engine_1/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_1/test.py @@ -1,4 +1,12 @@ import pytest + +# FIXME Tests with MaterializedPostgresSQL are temporarily disabled +# https://github.com/ClickHouse/ClickHouse/issues/36898 +# https://github.com/ClickHouse/ClickHouse/issues/38677 +# https://github.com/ClickHouse/ClickHouse/pull/39272#issuecomment-1190087190 + +pytestmark = pytest.mark.skip + import time import os.path as p import random diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 796edf04f06..9b4de5356bf 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -1,4 +1,12 @@ import pytest + +# FIXME Tests with MaterializedPostgresSQL are temporarily disabled +# https://github.com/ClickHouse/ClickHouse/issues/36898 +# https://github.com/ClickHouse/ClickHouse/issues/38677 +# https://github.com/ClickHouse/ClickHouse/pull/39272#issuecomment-1190087190 + +pytestmark = pytest.mark.skip + import time import psycopg2 import os.path as p diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 64f41022aef..5df8b9029e6 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -1,4 +1,12 @@ import pytest + +# FIXME Tests with MaterializedPostgresSQL are temporarily disabled +# https://github.com/ClickHouse/ClickHouse/issues/36898 +# https://github.com/ClickHouse/ClickHouse/issues/38677 +# https://github.com/ClickHouse/ClickHouse/pull/39272#issuecomment-1190087190 + +pytestmark = pytest.mark.skip + import time import psycopg2 import os.path as p From a7f088615791fb8dfc1398a5cf88234b2665fb8b Mon Sep 17 00:00:00 2001 From: whysage Date: Mon, 25 Jul 2022 18:54:02 +0300 Subject: [PATCH 619/659] Fix second link in doc AggregateFunction --- docs/ru/sql-reference/data-types/aggregatefunction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/aggregatefunction.md b/docs/ru/sql-reference/data-types/aggregatefunction.md index 551b8eac6e2..f177f1a98b7 100644 --- a/docs/ru/sql-reference/data-types/aggregatefunction.md +++ b/docs/ru/sql-reference/data-types/aggregatefunction.md @@ -5,7 +5,7 @@ sidebar_label: AggregateFunction # AggregateFunction {#data-type-aggregatefunction} -Агрегатные функции могут обладать определяемым реализацией промежуточным состоянием, которое может быть сериализовано в тип данных, соответствующий AggregateFunction(…), и быть записано в таблицу обычно посредством [материализованного представления] (../../sql-reference/statements/create.md#create-view). Чтобы получить промежуточное состояние, обычно используются агрегатные функции с суффиксом `-State`. Чтобы в дальнейшем получить агрегированные данные необходимо использовать те же агрегатные функции с суффиксом `-Merge`. +Агрегатные функции могут обладать определяемым реализацией промежуточным состоянием, которое может быть сериализовано в тип данных, соответствующий AggregateFunction(…), и быть записано в таблицу обычно посредством [материализованного представления] (../../sql-reference/statements/create/view.md). Чтобы получить промежуточное состояние, обычно используются агрегатные функции с суффиксом `-State`. Чтобы в дальнейшем получить агрегированные данные необходимо использовать те же агрегатные функции с суффиксом `-Merge`. `AggregateFunction(name, types_of_arguments…)` — параметрический тип данных. From f1e1cff27235b97b98a1dc445a589daadf76f07c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 25 Jul 2022 18:00:54 +0200 Subject: [PATCH 620/659] fix create/drop index on cluster --- src/Interpreters/DDLWorker.cpp | 8 ++++++- ...9_sql_standard_create_drop_index.reference | 6 +++++ .../02319_sql_standard_create_drop_index.sql | 22 +++++++++++++++++++ 3 files changed, 35 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 51932ad051b..13432940c1b 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include #include #include @@ -652,7 +654,11 @@ bool DDLWorker::taskShouldBeExecutedOnLeader(const ASTPtr & ast_ddl, const Stora if (auto * query = ast_ddl->as(); query && query->kind != ASTDropQuery::Kind::Truncate) return false; - if (!ast_ddl->as() && !ast_ddl->as() && !ast_ddl->as()) + if (!ast_ddl->as() && + !ast_ddl->as() && + !ast_ddl->as() && + !ast_ddl->as() && + !ast_ddl->as()) return false; if (auto * alter = ast_ddl->as()) diff --git a/tests/queries/0_stateless/02319_sql_standard_create_drop_index.reference b/tests/queries/0_stateless/02319_sql_standard_create_drop_index.reference index 6565857f89d..a4a924fd229 100644 --- a/tests/queries/0_stateless/02319_sql_standard_create_drop_index.reference +++ b/tests/queries/0_stateless/02319_sql_standard_create_drop_index.reference @@ -2,3 +2,9 @@ CREATE TABLE default.t_index\n(\n `a` Int32,\n `b` String,\n INDEX i_a t_index i_a minmax a 4 t_index i_b bloom_filter b 2 t_index i_b bloom_filter b 2 +CREATE TABLE default.t_index\n(\n `a` Int32,\n `b` String,\n INDEX i_a a TYPE minmax GRANULARITY 4,\n INDEX i_b b TYPE bloom_filter GRANULARITY 2\n)\nENGINE = ReplicatedMergeTree(\'/test/2319/default/\', \'1\')\nORDER BY a\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t_index_replica\n(\n `a` Int32,\n `b` String,\n INDEX i_a a TYPE minmax GRANULARITY 4,\n INDEX i_b b TYPE bloom_filter GRANULARITY 2\n)\nENGINE = ReplicatedMergeTree(\'/test/2319/default/\', \'2\')\nORDER BY a\nSETTINGS index_granularity = 8192 +t_index i_a minmax a 4 +t_index i_b bloom_filter b 2 +t_index i_b bloom_filter b 2 +t_index_replica i_b bloom_filter b 2 diff --git a/tests/queries/0_stateless/02319_sql_standard_create_drop_index.sql b/tests/queries/0_stateless/02319_sql_standard_create_drop_index.sql index a33505ced3a..bb01dcf2e64 100644 --- a/tests/queries/0_stateless/02319_sql_standard_create_drop_index.sql +++ b/tests/queries/0_stateless/02319_sql_standard_create_drop_index.sql @@ -15,3 +15,25 @@ drop index if exists i_a on t_index; select table, name, type, expr, granularity from system.data_skipping_indices where database = currentDatabase() and table = 't_index'; drop table t_index; + +create table t_index(a int, b String) engine=ReplicatedMergeTree('/test/2319/{database}/', '1') order by a; +create table t_index_replica(a int, b String) engine=ReplicatedMergeTree('/test/2319/{database}/', '2') order by a; + +create index i_a on t_index(a) TYPE minmax GRANULARITY 4; +create index if not exists i_a on t_index(a) TYPE minmax GRANULARITY 2; + +create index i_b on t_index(b) TYPE bloom_filter GRANULARITY 2; + +show create table t_index; +system sync replica t_index_replica; +show create table t_index_replica; +select table, name, type, expr, granularity from system.data_skipping_indices where database = currentDatabase() and table = 't_index'; + +drop index i_a on t_index; +drop index if exists i_a on t_index; + +select table, name, type, expr, granularity from system.data_skipping_indices where database = currentDatabase() and table = 't_index'; +system sync replica t_index_replica; +select table, name, type, expr, granularity from system.data_skipping_indices where database = currentDatabase() and table = 't_index_replica'; + +drop table t_index; From 5fae9f02fef2f7cafa3ebae1e3e1f98d8c06ea15 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Mon, 25 Jul 2022 12:29:18 -0400 Subject: [PATCH 621/659] Update CHANGELOG.md - fix the link to 22.7 log in table of contents - To allow reuse of the changelog in the docs I need to change the `
` to
. With this change we can update the doc build process to ```cp $PARENT_DIR/CHANGELOG.md $PARENT_DIR/clickhouse-docs/docs/en/whats-new/changelog/index.md ``` --- CHANGELOG.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 70e203193c8..d7e2684380c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,12 +1,12 @@ ### Table of Contents -**[ClickHouse release v22.7, 2022-07-21](#226)**
-**[ClickHouse release v22.6, 2022-06-16](#226)**
-**[ClickHouse release v22.5, 2022-05-19](#225)**
-**[ClickHouse release v22.4, 2022-04-20](#224)**
-**[ClickHouse release v22.3-lts, 2022-03-17](#223)**
-**[ClickHouse release v22.2, 2022-02-17](#222)**
-**[ClickHouse release v22.1, 2022-01-18](#221)**
-**[Changelog for 2021](https://clickhouse.com/docs/en/whats-new/changelog/2021/)**
+**[ClickHouse release v22.7, 2022-07-21](#227)**
+**[ClickHouse release v22.6, 2022-06-16](#226)**
+**[ClickHouse release v22.5, 2022-05-19](#225)**
+**[ClickHouse release v22.4, 2022-04-20](#224)**
+**[ClickHouse release v22.3-lts, 2022-03-17](#223)**
+**[ClickHouse release v22.2, 2022-02-17](#222)**
+**[ClickHouse release v22.1, 2022-01-18](#221)**
+**[Changelog for 2021](https://clickhouse.com/docs/en/whats-new/changelog/2021/)**
### ClickHouse release 22.7, 2022-07-21 From 72efcc65c15b3b36ab2ec3fc9c6209894702c307 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Mon, 25 Jul 2022 12:58:57 -0400 Subject: [PATCH 622/659] Update docs/en/engines/table-engines/mergetree-family/mergetree.md --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index e216a99f986..2ca07276e63 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -482,7 +482,9 @@ For example: ## Projections {#projections} Projections are like [materialized views](../../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. - +::: note +When you are implementing projections you should also consider the [force_optimize_projection](../../../operations/settings/settings.md#force-optimize-projection) setting. +::: Projections are not supported in the `SELECT` statements with the [FINAL](../../../sql-reference/statements/select/from.md#select-from-final) modifier. ### Projection Query {#projection-query} From 813438a29d1a6a66efdafe4b24d5820f531a0db2 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 25 Jul 2022 14:48:53 -0300 Subject: [PATCH 623/659] Update settings.md --- docs/ru/operations/settings/settings.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 1d57b688217..5ad6e4ef3b6 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -656,8 +656,9 @@ ClickHouse может парсить только базовый формат `Y Изменяет поведение операций, выполняемых со строгостью `ANY`. -:::danger "Внимание" +:::warning "Внимание" Настройка применяется только для операций `JOIN`, выполняемых над таблицами с движком [Join](../../engines/table-engines/special/join.md). +::: Возможные значения: @@ -2112,8 +2113,9 @@ SELECT * FROM test_table Устанавливает приоритет ([nice](https://en.wikipedia.org/wiki/Nice_(Unix))) для потоков, исполняющих запросы. Планировщик ОС учитывает эти приоритеты при выборе следующего потока для исполнения на доступном ядре CPU. -:::danger "Предупреждение" +:::warning "Предупреждение" Для использования этой настройки необходимо установить свойство `CAP_SYS_NICE`. Пакет `clickhouse-server` устанавливает его во время инсталляции. Некоторые виртуальные окружения не позволяют установить `CAP_SYS_NICE`. В этом случае, `clickhouse-server` выводит сообщение при запуске. +::: Допустимые значения: From ddff6c86df62231469dac714c27b26f4064bcc96 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 25 Jul 2022 19:13:10 +0000 Subject: [PATCH 624/659] Update version_date.tsv after v22.3.9.19-lts --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index e4c7aae8b25..e8b2adce2b6 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -9,6 +9,7 @@ v22.4.5.9-stable 2022-05-06 v22.4.4.7-stable 2022-04-29 v22.4.3.3-stable 2022-04-26 v22.4.2.1-stable 2022-04-22 +v22.3.9.19-lts 2022-07-25 v22.3.8.39-lts 2022-07-07 v22.3.7.28-lts 2022-06-20 v22.3.6.5-lts 2022-05-06 From 250f19378dea3007b6d2e25a0d2d0545f9f36419 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 25 Jul 2022 15:22:48 -0400 Subject: [PATCH 625/659] free compression and decompression contexts --- utils/self-extracting-executable/compressor.cpp | 9 +++++++++ utils/self-extracting-executable/decompressor.cpp | 2 ++ 2 files changed, 11 insertions(+) diff --git a/utils/self-extracting-executable/compressor.cpp b/utils/self-extracting-executable/compressor.cpp index 58674818c44..f40c4725c32 100644 --- a/utils/self-extracting-executable/compressor.cpp +++ b/utils/self-extracting-executable/compressor.cpp @@ -103,12 +103,14 @@ int compress(int in_fd, int out_fd, int level, off_t & pointer, const struct sta if (ZSTD_isError(check_result)) { std::cerr << "Error (ZSTD): " << check_result << " " << ZSTD_getErrorName(check_result) << std::endl; + ZSTD_freeCCtx(cctx); return 1; } check_result = ZSTD_CCtx_setParameter(cctx, ZSTD_c_checksumFlag, 1); if (ZSTD_isError(check_result)) { std::cerr << "Error (ZSTD): " << check_result << " " << ZSTD_getErrorName(check_result) << std::endl; + ZSTD_freeCCtx(cctx); return 1; } @@ -129,11 +131,13 @@ int compress(int in_fd, int out_fd, int level, off_t & pointer, const struct sta if (output == MAP_FAILED) { perror(nullptr); + ZSTD_freeCCtx(cctx); return 1; } if (-1 == lseek(out_fd, 0, SEEK_END)) { perror(nullptr); + ZSTD_freeCCtx(cctx); return 1; } @@ -154,6 +158,7 @@ int compress(int in_fd, int out_fd, int level, off_t & pointer, const struct sta perror(nullptr); if (0 != munmap(output, 2 * max_block_size)) perror(nullptr); + ZSTD_freeCCtx(cctx); return 1; } @@ -161,6 +166,7 @@ int compress(int in_fd, int out_fd, int level, off_t & pointer, const struct sta if (current_block_size != write_data(out_fd, output, current_block_size)) { perror(nullptr); + ZSTD_freeCCtx(cctx); return 1; } pointer += current_block_size; @@ -172,8 +178,11 @@ int compress(int in_fd, int out_fd, int level, off_t & pointer, const struct sta 0 != munmap(output, 2 * max_block_size)) { perror(nullptr); + ZSTD_freeCCtx(cctx); return 1; } + + ZSTD_freeCCtx(cctx); return 0; } diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 3ec06e91176..679dc144f13 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -151,6 +151,8 @@ int decompress(char * input, char * output, off_t start, off_t end, size_t max_n --number_of_forks; } + ZSTD_freeDCtx(dctx); + /// If error happen end of processed part will not reach end if (in_pointer < end || error_happened) return 1; From f04ad30f0d49873711dafb6aca0341b12f3418e0 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 25 Jul 2022 19:25:27 +0000 Subject: [PATCH 626/659] Update version_date.tsv and changelogs after v22.6.4.35-stable --- docs/changelogs/v22.6.4.35-stable.md | 36 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 3 +++ 2 files changed, 39 insertions(+) create mode 100644 docs/changelogs/v22.6.4.35-stable.md diff --git a/docs/changelogs/v22.6.4.35-stable.md b/docs/changelogs/v22.6.4.35-stable.md new file mode 100644 index 00000000000..d70d20d6134 --- /dev/null +++ b/docs/changelogs/v22.6.4.35-stable.md @@ -0,0 +1,36 @@ +--- +sidebar_position: 1 +sidebar_label: 2022 +--- + +# 2022 Changelog + +### ClickHouse release v22.6.4.35-stable FIXME as compared to v22.6.3.35-stable + +#### Build/Testing/Packaging Improvement +* Backported in [#38822](https://github.com/ClickHouse/ClickHouse/issues/38822): - Change `all|noarch` packages to architecture-dependent - Fix some documentation for it - Push aarch64|arm64 packages to artifactory and release assets - Fixes [#36443](https://github.com/ClickHouse/ClickHouse/issues/36443). [#38580](https://github.com/ClickHouse/ClickHouse/pull/38580) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#38242](https://github.com/ClickHouse/ClickHouse/issues/38242): Fix possible crash in `Distributed` async insert in case of removing a replica from config. [#38029](https://github.com/ClickHouse/ClickHouse/pull/38029) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#38865](https://github.com/ClickHouse/ClickHouse/issues/38865): Fix s3 seekable reads with parallel read buffer. (Affected memory usage during query). Closes [#38258](https://github.com/ClickHouse/ClickHouse/issues/38258). [#38802](https://github.com/ClickHouse/ClickHouse/pull/38802) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#38853](https://github.com/ClickHouse/ClickHouse/issues/38853): Update `simdjson`. This fixes [#38621](https://github.com/ClickHouse/ClickHouse/issues/38621). [#38838](https://github.com/ClickHouse/ClickHouse/pull/38838) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#38942](https://github.com/ClickHouse/ClickHouse/issues/38942): - Fix settings profile with seconds unit. [#38896](https://github.com/ClickHouse/ClickHouse/pull/38896) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#39063](https://github.com/ClickHouse/ClickHouse/issues/39063): Any allocations inside OvercommitTracker may lead to deadlock. Logging was not very informative so it's easier just to remove logging. Fixes [#37794](https://github.com/ClickHouse/ClickHouse/issues/37794). [#39030](https://github.com/ClickHouse/ClickHouse/pull/39030) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#39077](https://github.com/ClickHouse/ClickHouse/issues/39077): Fix bug in filesystem cache that could happen in some corner case which coincided with cache capacity hitting the limit. Closes [#39066](https://github.com/ClickHouse/ClickHouse/issues/39066). [#39070](https://github.com/ClickHouse/ClickHouse/pull/39070) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#39151](https://github.com/ClickHouse/ClickHouse/issues/39151): Fix error `Block structure mismatch` which could happen for INSERT into table with attached MATERIALIZED VIEW and enabled setting `extremes = 1`. Closes [#29759](https://github.com/ClickHouse/ClickHouse/issues/29759) and [#38729](https://github.com/ClickHouse/ClickHouse/issues/38729). [#39125](https://github.com/ClickHouse/ClickHouse/pull/39125) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#39275](https://github.com/ClickHouse/ClickHouse/issues/39275): Fixed error `Not found column Type in block` in selects with `PREWHERE` and read-in-order optimizations. [#39157](https://github.com/ClickHouse/ClickHouse/pull/39157) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#39371](https://github.com/ClickHouse/ClickHouse/issues/39371): Declare RabbitMQ queue without default arguments `x-max-length` and `x-overflow`. [#39259](https://github.com/ClickHouse/ClickHouse/pull/39259) ([rnbondarenko](https://github.com/rnbondarenko)). +* Backported in [#39352](https://github.com/ClickHouse/ClickHouse/issues/39352): Fix incorrect fetch postgresql tables query fro PostgreSQL database engine. Closes [#33502](https://github.com/ClickHouse/ClickHouse/issues/33502). [#39283](https://github.com/ClickHouse/ClickHouse/pull/39283) ([Kseniia Sumarokova](https://github.com/kssenii)). + +#### NO CL CATEGORY + +* Backported in [#38685](https://github.com/ClickHouse/ClickHouse/issues/38685):. [#38449](https://github.com/ClickHouse/ClickHouse/pull/38449) ([Maksim Kita](https://github.com/kitaisreal)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Use native Map type for OpenTelemetry attributes [#38814](https://github.com/ClickHouse/ClickHouse/pull/38814) ([Ilya Yatsishin](https://github.com/qoega)). +* Retry docker buildx commands with progressive sleep in between [#38898](https://github.com/ClickHouse/ClickHouse/pull/38898) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add docker_server.py running to backport and release CIs [#39011](https://github.com/ClickHouse/ClickHouse/pull/39011) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix meilisearch tests [#39110](https://github.com/ClickHouse/ClickHouse/pull/39110) ([Kseniia Sumarokova](https://github.com/kssenii)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index e4c7aae8b25..777fd424321 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,7 +1,9 @@ v22.7.1.2484-stable 2022-07-21 +v22.6.4.35-stable 2022-07-25 v22.6.3.35-stable 2022-07-06 v22.6.2.12-stable 2022-06-29 v22.6.1.1985-stable 2022-06-16 +v22.5.3.21-stable 2022-07-25 v22.5.2.53-stable 2022-07-07 v22.5.1.2079-stable 2022-05-19 v22.4.6.53-stable 2022-07-07 @@ -9,6 +11,7 @@ v22.4.5.9-stable 2022-05-06 v22.4.4.7-stable 2022-04-29 v22.4.3.3-stable 2022-04-26 v22.4.2.1-stable 2022-04-22 +v22.3.9.19-lts 2022-07-25 v22.3.8.39-lts 2022-07-07 v22.3.7.28-lts 2022-06-20 v22.3.6.5-lts 2022-05-06 From 9abbb35cda0709408e9c3bd70a4b80caa3afee44 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 25 Jul 2022 23:40:23 +0300 Subject: [PATCH 627/659] Fix integration test --- src/IO/ReadWriteBufferFromHTTP.h | 55 +++++++++++++++----------------- 1 file changed, 25 insertions(+), 30 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index bbd1f92f0ad..d5abc4609ed 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -515,44 +515,39 @@ namespace detail for (size_t i = 0; i < settings.http_max_tries; ++i) { + exception = nullptr; + try { if (!impl) { initialize(); - switch (initialization_error) - { - case InitializeError::NON_RETRIABLE_ERROR: - { - assert(exception); - break; - } - case InitializeError::SKIP_NOT_FOUND_URL: - { - return false; - } - case InitializeError::RETRIABLE_ERROR: - { - LOG_ERROR( - log, - "HTTP request to `{}` failed at try {}/{} with bytes read: {}/{}. " - "(Current backoff wait is {}/{} ms)", - uri.toString(), i + 1, settings.http_max_tries, getOffset(), - read_range.end ? toString(*read_range.end) : "unknown", - milliseconds_to_wait, settings.http_retry_max_backoff_ms); - assert(exception); - on_retriable_error(); - continue; - } - case InitializeError::NONE: - { - break; - } + if (initialization_error == InitializeError::NON_RETRIABLE_ERROR) + { + assert(exception); + break; + } + else if (initialization_error == InitializeError::SKIP_NOT_FOUND_URL) + { + return false; + } + else if (initialization_error == InitializeError::RETRIABLE_ERROR) + { + LOG_ERROR( + log, + "HTTP request to `{}` failed at try {}/{} with bytes read: {}/{}. " + "(Current backoff wait is {}/{} ms)", + uri.toString(), i + 1, settings.http_max_tries, getOffset(), + read_range.end ? toString(*read_range.end) : "unknown", + milliseconds_to_wait, settings.http_retry_max_backoff_ms); + + assert(exception); + on_retriable_error(); + continue; } - if (exception) - break; + assert(!exception); if (use_external_buffer) { From f32d9c5539de0fe89bdf50d881b857741b1ebf25 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotaryov <108669454+quickhouse@users.noreply.github.com> Date: Mon, 25 Jul 2022 23:53:53 +0300 Subject: [PATCH 628/659] Uppercase `ROWS`, `GROUPS`, `RANGE` in queries with windows. (#39410) --- src/Client/QueryFuzzer.cpp | 8 ++++---- src/Interpreters/ExpressionAnalyzer.cpp | 4 ++-- src/Interpreters/WindowDescription.cpp | 6 +++--- src/Interpreters/WindowDescription.h | 4 ++-- src/Parsers/ASTWindowDefinition.h | 2 +- src/Parsers/ExpressionElementParsers.cpp | 6 +++--- src/Processors/Transforms/WindowTransform.cpp | 14 +++++++------- 7 files changed, 22 insertions(+), 22 deletions(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 703e6de82c6..787fad5990a 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -329,9 +329,9 @@ void QueryFuzzer::fuzzWindowFrame(ASTWindowDefinition & def) case 0: { const auto r = fuzz_rand() % 3; - def.frame_type = r == 0 ? WindowFrame::FrameType::Rows - : r == 1 ? WindowFrame::FrameType::Range - : WindowFrame::FrameType::Groups; + def.frame_type = r == 0 ? WindowFrame::FrameType::ROWS + : r == 1 ? WindowFrame::FrameType::RANGE + : WindowFrame::FrameType::GROUPS; break; } case 1: @@ -385,7 +385,7 @@ void QueryFuzzer::fuzzWindowFrame(ASTWindowDefinition & def) break; } - if (def.frame_type == WindowFrame::FrameType::Range + if (def.frame_type == WindowFrame::FrameType::RANGE && def.frame_begin_type == WindowFrame::BoundaryType::Unbounded && def.frame_begin_preceding && def.frame_end_type == WindowFrame::BoundaryType::Current) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 8a14c09819a..f16922f9772 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -828,8 +828,8 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, desc.full_sort_description.insert(desc.full_sort_description.end(), desc.order_by.begin(), desc.order_by.end()); - if (definition.frame_type != WindowFrame::FrameType::Rows - && definition.frame_type != WindowFrame::FrameType::Range) + if (definition.frame_type != WindowFrame::FrameType::ROWS + && definition.frame_type != WindowFrame::FrameType::RANGE) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Window frame '{}' is not implemented (while processing '{}')", diff --git a/src/Interpreters/WindowDescription.cpp b/src/Interpreters/WindowDescription.cpp index 4661a148d70..335610b2be9 100644 --- a/src/Interpreters/WindowDescription.cpp +++ b/src/Interpreters/WindowDescription.cpp @@ -90,8 +90,8 @@ void WindowFrame::toString(WriteBuffer & buf) const void WindowFrame::checkValid() const { // Check the validity of offsets. - if (type == WindowFrame::FrameType::Rows - || type == WindowFrame::FrameType::Groups) + if (type == WindowFrame::FrameType::ROWS + || type == WindowFrame::FrameType::GROUPS) { if (begin_type == BoundaryType::Offset && !((begin_offset.getType() == Field::Types::UInt64 @@ -197,7 +197,7 @@ void WindowDescription::checkValid() const frame.checkValid(); // RANGE OFFSET requires exactly one ORDER BY column. - if (frame.type == WindowFrame::FrameType::Range + if (frame.type == WindowFrame::FrameType::RANGE && (frame.begin_type == WindowFrame::BoundaryType::Offset || frame.end_type == WindowFrame::BoundaryType::Offset) && order_by.size() != 1) diff --git a/src/Interpreters/WindowDescription.h b/src/Interpreters/WindowDescription.h index 65c8cb9423c..e7bc0473c26 100644 --- a/src/Interpreters/WindowDescription.h +++ b/src/Interpreters/WindowDescription.h @@ -28,7 +28,7 @@ struct WindowFunctionDescription struct WindowFrame { - enum class FrameType { Rows, Groups, Range }; + enum class FrameType { ROWS, GROUPS, RANGE }; enum class BoundaryType { Unbounded, Current, Offset }; // This flag signifies that the frame properties were not set explicitly by @@ -36,7 +36,7 @@ struct WindowFrame // for the default frame of RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW. bool is_default = true; - FrameType type = FrameType::Range; + FrameType type = FrameType::RANGE; // UNBOUNDED FOLLOWING for the frame end is forbidden by the standard, but for // uniformity the begin_preceding still has to be set to true for UNBOUNDED diff --git a/src/Parsers/ASTWindowDefinition.h b/src/Parsers/ASTWindowDefinition.h index c53f8ef856e..507825f11d2 100644 --- a/src/Parsers/ASTWindowDefinition.h +++ b/src/Parsers/ASTWindowDefinition.h @@ -17,7 +17,7 @@ struct ASTWindowDefinition : public IAST ASTPtr order_by; bool frame_is_default = true; - WindowFrame::FrameType frame_type = WindowFrame::FrameType::Range; + WindowFrame::FrameType frame_type = WindowFrame::FrameType::RANGE; WindowFrame::BoundaryType frame_begin_type = WindowFrame::BoundaryType::Unbounded; ASTPtr frame_begin_offset; bool frame_begin_preceding = true; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index bd65305cc52..1de9adb834e 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1198,15 +1198,15 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p node->frame_is_default = false; if (keyword_rows.ignore(pos, expected)) { - node->frame_type = WindowFrame::FrameType::Rows; + node->frame_type = WindowFrame::FrameType::ROWS; } else if (keyword_groups.ignore(pos, expected)) { - node->frame_type = WindowFrame::FrameType::Groups; + node->frame_type = WindowFrame::FrameType::GROUPS; } else if (keyword_range.ignore(pos, expected)) { - node->frame_type = WindowFrame::FrameType::Range; + node->frame_type = WindowFrame::FrameType::RANGE; } else { diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 5c833cf8f69..5e0d896599c 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -263,7 +263,7 @@ WindowTransform::WindowTransform(const Block & input_header_, // Choose a row comparison function for RANGE OFFSET frame based on the // type of the ORDER BY column. - if (window_description.frame.type == WindowFrame::FrameType::Range + if (window_description.frame.type == WindowFrame::FrameType::RANGE && (window_description.frame.begin_type == WindowFrame::BoundaryType::Offset || window_description.frame.end_type @@ -612,10 +612,10 @@ void WindowTransform::advanceFrameStart() case WindowFrame::BoundaryType::Offset: switch (window_description.frame.type) { - case WindowFrame::FrameType::Rows: + case WindowFrame::FrameType::ROWS: advanceFrameStartRowsOffset(); break; - case WindowFrame::FrameType::Range: + case WindowFrame::FrameType::RANGE: advanceFrameStartRangeOffset(); break; default: @@ -659,14 +659,14 @@ bool WindowTransform::arePeers(const RowNumber & x, const RowNumber & y) const return true; } - if (window_description.frame.type == WindowFrame::FrameType::Rows) + if (window_description.frame.type == WindowFrame::FrameType::ROWS) { // For ROWS frame, row is only peers with itself (checked above); return false; } // For RANGE and GROUPS frames, rows that compare equal w/ORDER BY are peers. - assert(window_description.frame.type == WindowFrame::FrameType::Range); + assert(window_description.frame.type == WindowFrame::FrameType::RANGE); const size_t n = order_by_indices.size(); if (n == 0) { @@ -844,10 +844,10 @@ void WindowTransform::advanceFrameEnd() case WindowFrame::BoundaryType::Offset: switch (window_description.frame.type) { - case WindowFrame::FrameType::Rows: + case WindowFrame::FrameType::ROWS: advanceFrameEndRowsOffset(); break; - case WindowFrame::FrameType::Range: + case WindowFrame::FrameType::RANGE: advanceFrameEndRangeOffset(); break; default: From c1834d183b4a6daff298d1531301d715fec94717 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Tue, 26 Jul 2022 11:50:09 +0800 Subject: [PATCH 629/659] ReplicatedMergeTree support lightweight delete --- src/Interpreters/InterpreterDeleteQuery.cpp | 8 ++- src/Storages/StorageReplicatedMergeTree.cpp | 5 ++ src/Storages/StorageReplicatedMergeTree.h | 2 + ..._delete_on_replicated_merge_tree.reference | 30 +++++++++ ...weight_delete_on_replicated_merge_tree.sql | 64 +++++++++++++++++++ 5 files changed, 106 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.reference create mode 100644 tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.sql diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 47b0050e46c..497fae8f573 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include @@ -50,8 +51,9 @@ BlockIO InterpreterDeleteQuery::execute() /// First check table storage for validations. StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); auto storage_merge_tree = std::dynamic_pointer_cast(table); - if (!storage_merge_tree) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only MergeTree tables are supported"); + auto storage_replicated_merge_tree = std::dynamic_pointer_cast(table); + if (!storage_merge_tree && !storage_replicated_merge_tree) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only MergeTree or ReplicatedMergeTree tables are supported"); checkStorageSupportsTransactionsIfNeeded(table, getContext()); if (table->isStaticStorage()) @@ -95,7 +97,7 @@ BlockIO InterpreterDeleteQuery::execute() table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate(); - storage_merge_tree->mutate(mutation_commands, getContext()); + table->mutate(mutation_commands, getContext()); return {}; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 219093e8d75..1ec14f643d4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6015,6 +6015,11 @@ CancellationCode StorageReplicatedMergeTree::killMutation(const String & mutatio return CancellationCode::CancelSent; } +bool StorageReplicatedMergeTree::hasLightweightDeletedMask() const +{ + return has_lightweight_delete_parts.load(std::memory_order_relaxed); +} + void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() { auto table_lock = lockForShare( diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 18b9ef54777..c35e2d5cf5c 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -152,6 +152,8 @@ public: std::vector getMutationsStatus() const override; CancellationCode killMutation(const String & mutation_id) override; + bool hasLightweightDeletedMask() const override; + /** Removes a replica from ZooKeeper. If there are no other replicas, it deletes the entire table from ZooKeeper. */ void drop() override; diff --git a/tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.reference b/tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.reference new file mode 100644 index 00000000000..0153257a80b --- /dev/null +++ b/tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.reference @@ -0,0 +1,30 @@ +99 +99 +95 +0 +0 +-----Check that select and merge with lightweight delete.----- +7 +0 0 0 +2 2 2 +3 3 3 +5 5 5 +7 7 7 +8 8 8 +9 9 9 +0 0 0 +2 2 2 +3 3 3 +5 5 5 +7 7 7 +8 8 8 +9 9 9 +7 +-----Check fetch part with lightweight delete----- +0 0 0 +2 2 2 +3 3 3 +5 5 5 +6 6 6 +8 8 8 +9 9 9 diff --git a/tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.sql b/tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.sql new file mode 100644 index 00000000000..3df9acbee87 --- /dev/null +++ b/tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.sql @@ -0,0 +1,64 @@ +DROP TABLE IF EXISTS replicated_table_r1 SYNC; +DROP TABLE IF EXISTS replicated_table_r2 SYNC; + +CREATE TABLE replicated_table_r1(id Int32, name String) ENGINE = ReplicatedMergeTree('/test/02352/{database}/t_rep','1') ORDER BY id; +CREATE TABLE replicated_table_r2(id Int32, name String) ENGINE = ReplicatedMergeTree('/test/02352/{database}/t_rep','2') ORDER BY id; + +INSERT INTO replicated_table_r1 select number, toString(number) FROM numbers(100); + +SET mutations_sync = 1; +SET allow_experimental_lightweight_delete = 1; + +DELETE FROM replicated_table_r1 WHERE id = 10; + +SELECT COUNT() FROM replicated_table_r1; +SELECT COUNT() FROM replicated_table_r2; + +DELETE FROM replicated_table_r2 WHERE name IN ('1','2','3','4'); + +SELECT COUNT() FROM replicated_table_r1; + +DELETE FROM replicated_table_r1 WHERE 1; + +SELECT COUNT() FROM replicated_table_r1; +SELECT COUNT() FROM replicated_table_r2; + +DROP TABLE IF EXISTS replicated_table_r1 SYNC; +DROP TABLE IF EXISTS replicated_table_r2 SYNC; + +DROP TABLE IF EXISTS t_light_r1 SYNC; +DROP TABLE IF EXISTS t_light_r2 SYNC; + +CREATE TABLE t_light_r1(a int, b int, c int, index i_c(b) TYPE minmax granularity 4) ENGINE = ReplicatedMergeTree('/test/02352/{database}/t_light','1') ORDER BY a PARTITION BY c % 5; +CREATE TABLE t_light_r2(a int, b int, c int, index i_c(b) TYPE minmax granularity 4) ENGINE = ReplicatedMergeTree('/test/02352/{database}/t_light','2') ORDER BY a PARTITION BY c % 5; + +INSERT INTO t_light_r1 SELECT number, number, number FROM numbers(10); + +DELETE FROM t_light_r1 WHERE c%5=1; +DELETE FROM t_light_r2 WHERE c=4; + +SELECT '-----Check that select and merge with lightweight delete.-----'; +SELECT count(*) FROM t_light_r1; +SELECT * FROM t_light_r1 ORDER BY a; +SELECT * FROM t_light_r2 ORDER BY a; + +OPTIMIZE TABLE t_light_r1 FINAL; +SELECT count(*) FROM t_light_r1; + +DROP TABLE IF EXISTS t_light_r1 SYNC; +DROP TABLE IF EXISTS t_light_r2 SYNC; + +CREATE TABLE t_light_sync_r1(a int, b int, c int, index i_c(b) TYPE minmax granularity 4) ENGINE = ReplicatedMergeTree('/test/02352/{database}/t_sync','1') ORDER BY a PARTITION BY c % 5 SETTINGS min_bytes_for_wide_part=0; + +INSERT INTO t_light_sync_r1 SELECT number, number, number FROM numbers(10); + +DELETE FROM t_light_sync_r1 WHERE c%3=1; + +SELECT '-----Check fetch part with lightweight delete-----'; +CREATE TABLE t_light_sync_r2(a int, b int, c int, index i_c(b) TYPE minmax granularity 4) ENGINE = ReplicatedMergeTree('/test/02352/{database}/t_sync','2') ORDER BY a PARTITION BY c % 5 SETTINGS min_bytes_for_wide_part=0; +SYSTEM SYNC REPLICA t_light_sync_r2; + +SELECT * FROM t_light_sync_r2 ORDER BY a; + +DROP TABLE IF EXISTS t_light_sync_r1 SYNC; +DROP TABLE IF EXISTS t_light_sync_r2 SYNC; From 196b517e79e00b4d868dab379194ae3d1450623d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 22 Jul 2022 10:38:37 +0300 Subject: [PATCH 630/659] tests: add echo for 01601_custom_tld Signed-off-by: Azat Khuzhin --- .../0_stateless/01601_custom_tld.reference | 39 +++++++++++++++++++ .../queries/0_stateless/01601_custom_tld.sql | 2 + 2 files changed, 41 insertions(+) diff --git a/tests/queries/0_stateless/01601_custom_tld.reference b/tests/queries/0_stateless/01601_custom_tld.reference index ee326a77834..b4afe625dac 100644 --- a/tests/queries/0_stateless/01601_custom_tld.reference +++ b/tests/queries/0_stateless/01601_custom_tld.reference @@ -1,34 +1,73 @@ +-- { echo } + +select '-- no-tld'; -- no-tld +-- even if there is no TLD, 2-nd level by default anyway +-- FIXME: make this behavior optional (so that TLD for host never changed, either empty or something real) +select cutToFirstSignificantSubdomain('there-is-no-such-domain'); +select cutToFirstSignificantSubdomain('foo.there-is-no-such-domain'); foo.there-is-no-such-domain +select cutToFirstSignificantSubdomain('bar.foo.there-is-no-such-domain'); foo.there-is-no-such-domain +select cutToFirstSignificantSubdomainCustom('there-is-no-such-domain', 'public_suffix_list'); +select cutToFirstSignificantSubdomainCustom('foo.there-is-no-such-domain', 'public_suffix_list'); foo.there-is-no-such-domain +select cutToFirstSignificantSubdomainCustom('bar.foo.there-is-no-such-domain', 'public_suffix_list'); foo.there-is-no-such-domain +select firstSignificantSubdomainCustom('bar.foo.there-is-no-such-domain', 'public_suffix_list'); foo +select '-- generic'; -- generic +select firstSignificantSubdomainCustom('foo.kernel.biz.ss', 'public_suffix_list'); -- kernel kernel +select cutToFirstSignificantSubdomainCustom('foo.kernel.biz.ss', 'public_suffix_list'); -- kernel.biz.ss kernel.biz.ss +select '-- difference'; -- difference +-- biz.ss is not in the default TLD list, hence: +select cutToFirstSignificantSubdomain('foo.kernel.biz.ss'); -- biz.ss biz.ss +select cutToFirstSignificantSubdomainCustom('foo.kernel.biz.ss', 'public_suffix_list'); -- kernel.biz.ss kernel.biz.ss +select '-- 3+level'; -- 3+level +select cutToFirstSignificantSubdomainCustom('xx.blogspot.co.at', 'public_suffix_list'); -- xx.blogspot.co.at xx.blogspot.co.at +select firstSignificantSubdomainCustom('xx.blogspot.co.at', 'public_suffix_list'); -- blogspot blogspot +select cutToFirstSignificantSubdomainCustom('foo.bar.xx.blogspot.co.at', 'public_suffix_list'); -- xx.blogspot.co.at xx.blogspot.co.at +select firstSignificantSubdomainCustom('foo.bar.xx.blogspot.co.at', 'public_suffix_list'); -- blogspot blogspot +select '-- url'; -- url +select cutToFirstSignificantSubdomainCustom('http://foobar.com', 'public_suffix_list'); foobar.com +select cutToFirstSignificantSubdomainCustom('http://foobar.com/foo', 'public_suffix_list'); foobar.com +select cutToFirstSignificantSubdomainCustom('http://bar.foobar.com/foo', 'public_suffix_list'); foobar.com +select cutToFirstSignificantSubdomainCustom('http://xx.blogspot.co.at', 'public_suffix_list'); xx.blogspot.co.at +select '-- www'; -- www +select cutToFirstSignificantSubdomainCustomWithWWW('http://www.foo', 'public_suffix_list'); www.foo +select cutToFirstSignificantSubdomainCustom('http://www.foo', 'public_suffix_list'); foo +select '-- vector'; -- vector +select cutToFirstSignificantSubdomainCustom('http://xx.blogspot.co.at/' || toString(number), 'public_suffix_list') from numbers(1); xx.blogspot.co.at +select cutToFirstSignificantSubdomainCustom('there-is-no-such-domain' || toString(number), 'public_suffix_list') from numbers(1); +select '-- no new line'; -- no new line +select cutToFirstSignificantSubdomainCustom('foo.bar', 'no_new_line_list'); foo.bar +select cutToFirstSignificantSubdomainCustom('a.foo.bar', 'no_new_line_list'); a.foo.bar +select cutToFirstSignificantSubdomainCustom('a.foo.baz', 'no_new_line_list'); foo.baz diff --git a/tests/queries/0_stateless/01601_custom_tld.sql b/tests/queries/0_stateless/01601_custom_tld.sql index 92ce28828f8..fd55e229fbc 100644 --- a/tests/queries/0_stateless/01601_custom_tld.sql +++ b/tests/queries/0_stateless/01601_custom_tld.sql @@ -1,3 +1,5 @@ +-- { echo } + select '-- no-tld'; -- even if there is no TLD, 2-nd level by default anyway -- FIXME: make this behavior optional (so that TLD for host never changed, either empty or something real) From 1d4a7c72903a1d698954919b0ebcb107bb49aae4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 22 Jul 2022 10:36:50 +0300 Subject: [PATCH 631/659] Add support of !/* (exclamation/asterisk) in custom TLDs Public suffix list may contain special characters (you may find format here - [1]): - asterisk (*) - exclamation mark (!) [1]: https://github.com/publicsuffix/list/wiki/Format It is easier to describe how it should be interpreted with an examples. Consider the following part of the list: *.sch.uk *.kawasaki.jp !city.kawasaki.jp And here are the results for `cutToFirstSignificantSubdomainCustom()`: If you have only asterisk (*): foo.something.sheffield.sch.uk -> something.sheffield.sch.uk sheffield.sch.uk -> sheffield.sch.uk If you have exclamation mark (!) too: foo.kawasaki.jp -> foo.kawasaki.jp foo.foo.kawasaki.jp -> foo.foo.kawasaki.jp city.kawasaki.jp -> city.kawasaki.jp some.city.kawasaki.jp -> city.kawasaki.jp TLDs had been verified wit the following script [2], to match with python publicsuffix2 module. [2]: https://gist.github.com/azat/c1a7a9f1e3519793134ef4b1df5461a6 v2: fix StringHashTable padding requirements Fixes: #39468 Follow-up for: #17748 Signed-off-by: Azat Khuzhin --- src/Common/TLDListsHolder.cpp | 57 +++++++--- src/Common/TLDListsHolder.h | 31 ++++-- .../URL/ExtractFirstSignificantSubdomain.h | 104 +++++++++++------- .../URL/FirstSignificantSubdomainCustomImpl.h | 4 +- .../0_stateless/01601_custom_tld.reference | 18 +++ .../queries/0_stateless/01601_custom_tld.sql | 11 ++ 6 files changed, 159 insertions(+), 66 deletions(-) diff --git a/src/Common/TLDListsHolder.cpp b/src/Common/TLDListsHolder.cpp index a3019ac1c49..75e57d9b9d4 100644 --- a/src/Common/TLDListsHolder.cpp +++ b/src/Common/TLDListsHolder.cpp @@ -15,20 +15,31 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +constexpr size_t StringHashTablePadRequirement = 8; + /// TLDList TLDList::TLDList(size_t size) : tld_container(size) - , pool(std::make_unique(10 << 20)) -{} -bool TLDList::insert(StringRef host) + , memory_pool(std::make_unique()) { - bool inserted; - tld_container.emplace(DB::ArenaKeyHolder{host, *pool}, inserted); - return inserted; + /// StringHashTable requires padded to 8 bytes key, + /// and Arena (memory_pool here) does satisfies this, + /// since it has padding with 15 bytes at the right. + /// + /// However, StringHashTable may reference -1 byte of the key, + /// so left padding is also required: + memory_pool->alignedAlloc(StringHashTablePadRequirement, StringHashTablePadRequirement); } -bool TLDList::has(StringRef host) const +void TLDList::insert(const String & host, TLDType type) { - return tld_container.has(host); + StringRef owned_host{memory_pool->insert(host.data(), host.size()), host.size()}; + tld_container[owned_host] = type; +} +TLDType TLDList::lookup(StringRef host) const +{ + if (auto it = tld_container.find(host); it != nullptr) + return it->getMapped(); + return TLDType::TLD_NONE; } /// TLDListsHolder @@ -57,32 +68,44 @@ void TLDListsHolder::parseConfig(const std::string & top_level_domains_path, con size_t TLDListsHolder::parseAndAddTldList(const std::string & name, const std::string & path) { - std::unordered_set tld_list_tmp; + std::unordered_map tld_list_tmp; ReadBufferFromFile in(path); - String line; + String buffer; while (!in.eof()) { - readEscapedStringUntilEOL(line, in); + readEscapedStringUntilEOL(buffer, in); if (!in.eof()) ++in.position(); + std::string_view line(buffer); /// Skip comments - if (line.size() > 2 && line[0] == '/' && line[1] == '/') + if (line.starts_with("//")) continue; - line = trim(line, [](char c) { return std::isspace(c); }); + line = line.substr(0, line.rend() - std::find_if_not(line.rbegin(), line.rend(), ::isspace)); /// Skip empty line if (line.empty()) continue; - tld_list_tmp.emplace(line); + /// Validate special symbols. + if (line.starts_with("*.")) + { + line = line.substr(2); + tld_list_tmp.emplace(line, TLDType::TLD_ANY); + } + else if (line[0] == '!') + { + line = line.substr(1); + tld_list_tmp.emplace(line, TLDType::TLD_EXCLUDE); + } + else + tld_list_tmp.emplace(line, TLDType::TLD_REGULAR); } if (!in.eof()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Not all list had been read", name); TLDList tld_list(tld_list_tmp.size()); - for (const auto & host : tld_list_tmp) + for (const auto & [host, type] : tld_list_tmp) { - StringRef host_ref{host.data(), host.size()}; - tld_list.insert(host_ref); + tld_list.insert(host, type); } size_t tld_list_size = tld_list.size(); diff --git a/src/Common/TLDListsHolder.h b/src/Common/TLDListsHolder.h index e8acefb1b5e..5ea8c5afe9f 100644 --- a/src/Common/TLDListsHolder.h +++ b/src/Common/TLDListsHolder.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -12,25 +12,35 @@ namespace DB { +enum TLDType +{ + /// Does not exist marker + TLD_NONE, + /// For regular lines + TLD_REGULAR, + /// For asterisk (*) + TLD_ANY, + /// For exclamation mark (!) + TLD_EXCLUDE, +}; + /// Custom TLD List /// -/// Unlike tldLookup (which uses gperf) this one uses plain StringHashSet. +/// Unlike tldLookup (which uses gperf) this one uses plain StringHashMap. class TLDList { public: - using Container = StringHashSet<>; + using Container = StringHashMap; explicit TLDList(size_t size); - /// Return true if the tld_container does not contains such element. - bool insert(StringRef host); - /// Check is there such TLD - bool has(StringRef host) const; + void insert(const String & host, TLDType type); + TLDType lookup(StringRef host) const; size_t size() const { return tld_container.size(); } private: Container tld_container; - std::unique_ptr pool; + std::unique_ptr memory_pool; }; class TLDListsHolder @@ -48,6 +58,11 @@ public: /// - "//" -- comment, /// - empty lines will be ignored. /// + /// Treats the following special symbols: + /// - "*" + /// - "!" + /// + /// Format : https://github.com/publicsuffix/list/wiki/Format /// Example: https://publicsuffix.org/list/public_suffix_list.dat /// /// Return size of the list. diff --git a/src/Functions/URL/ExtractFirstSignificantSubdomain.h b/src/Functions/URL/ExtractFirstSignificantSubdomain.h index 70c9c25e4f3..73137da474f 100644 --- a/src/Functions/URL/ExtractFirstSignificantSubdomain.h +++ b/src/Functions/URL/ExtractFirstSignificantSubdomain.h @@ -3,15 +3,16 @@ #include #include "domain.h" #include "tldLookup.h" +#include /// TLDType namespace DB { struct FirstSignificantSubdomainDefaultLookup { - bool operator()(const char *src, size_t len) const + bool operator()(StringRef host) const { - return tldLookup::isValid(src, len); + return tldLookup::isValid(host.data, host.size); } }; @@ -51,44 +52,46 @@ struct ExtractFirstSignificantSubdomain const auto * begin = tmp; const auto * end = begin + domain_length; - const char * last_3_periods[3]{}; + std::array last_periods{}; const auto * pos = find_first_symbols<'.'>(begin, end); while (pos < end) { - last_3_periods[2] = last_3_periods[1]; - last_3_periods[1] = last_3_periods[0]; - last_3_periods[0] = pos; + last_periods[2] = last_periods[1]; + last_periods[1] = last_periods[0]; + last_periods[0] = pos; pos = find_first_symbols<'.'>(pos + 1, end); } - if (!last_3_periods[0]) + if (!last_periods[0]) return; - if (!last_3_periods[1]) + if (!last_periods[1]) { - res_size = last_3_periods[0] - begin; + res_size = last_periods[0] - begin; return; } - if (!last_3_periods[2]) - last_3_periods[2] = begin - 1; + if (!last_periods[2]) + last_periods[2] = begin - 1; - const auto * end_of_level_domain = find_first_symbols<'/'>(last_3_periods[0], end); + const auto * end_of_level_domain = find_first_symbols<'/'>(last_periods[0], end); if (!end_of_level_domain) { end_of_level_domain = end; } - if (lookup(last_3_periods[1] + 1, end_of_level_domain - last_3_periods[1] - 1)) + size_t host_len = static_cast(end_of_level_domain - last_periods[1] - 1); + StringRef host{last_periods[1] + 1, host_len}; + if (lookup(host)) { - res_data += last_3_periods[2] + 1 - begin; - res_size = last_3_periods[1] - last_3_periods[2] - 1; + res_data += last_periods[2] + 1 - begin; + res_size = last_periods[1] - last_periods[2] - 1; } else { - res_data += last_3_periods[1] + 1 - begin; - res_size = last_3_periods[0] - last_3_periods[1] - 1; + res_data += last_periods[1] + 1 - begin; + res_size = last_periods[0] - last_periods[1] - 1; } } @@ -119,40 +122,63 @@ struct ExtractFirstSignificantSubdomain const auto * begin = tmp; const auto * end = begin + domain_length; - const char * last_2_periods[2]{}; - const char * prev = begin - 1; + std::array last_periods{}; + last_periods[0] = begin - 1; + StringRef excluded_host{}; const auto * pos = find_first_symbols<'.'>(begin, end); while (pos < end) { - if (lookup(pos + 1, end - pos - 1)) + size_t host_len = static_cast(end - pos - 1); + StringRef host{pos + 1, host_len}; + TLDType tld_type = lookup(host); + switch (tld_type) { - res_data += prev + 1 - begin; - res_size = end - 1 - prev; - return; + case TLDType::TLD_NONE: + break; + case TLDType::TLD_REGULAR: + res_data += last_periods[0] + 1 - begin; + res_size = end - 1 - last_periods[0]; + return; + case TLDType::TLD_ANY: + { + StringRef regular_host{last_periods[0] + 1, static_cast(end - 1 - last_periods[0])}; + if (last_periods[1] && excluded_host != regular_host) + { + /// Return TLD_REGULAR + 1 + res_data += last_periods[1] + 1 - begin; + res_size = end - 1 - last_periods[1]; + } + else + { + /// Same as TLD_REGULAR + res_data += last_periods[0] + 1 - begin; + res_size = end - 1 - last_periods[0]; + } + return; + } + case TLDType::TLD_EXCLUDE: + excluded_host = host; + break; } - last_2_periods[1] = last_2_periods[0]; - last_2_periods[0] = pos; - prev = pos; + last_periods[1] = last_periods[0]; + last_periods[0] = pos; pos = find_first_symbols<'.'>(pos + 1, end); } - /// if there is domain of the first level (i.e. no dots in the hostname) -> return nothing - if (!last_2_periods[0]) + /// - if there is domain of the first level (i.e. no dots in the hostname) -> + /// return nothing + if (last_periods[0] == begin - 1) return; - /// if there is domain of the second level -> always return itself - if (!last_2_periods[1]) - { - res_size = last_2_periods[0] - begin; - return; - } - - /// if there is domain of the 3+ level, and zero records in TLD list -> - /// fallback to domain of the second level - res_data += last_2_periods[1] + 1 - begin; - res_size = last_2_periods[0] - last_2_periods[1] - 1; + /// - if there is domain of the second level -> + /// always return itself + /// + /// - if there is domain of the 3+ level, and zero records in TLD list -> + /// fallback to domain of the second level + res_data += last_periods[1] + 1 - begin; + res_size = last_periods[0] - last_periods[1] - 1; } }; diff --git a/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h b/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h index 5d78500c252..88aa2e72db9 100644 --- a/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h +++ b/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h @@ -25,9 +25,9 @@ struct FirstSignificantSubdomainCustomLookup { } - bool operator()(const char *pos, size_t len) const + TLDType operator()(StringRef host) const { - return tld_list.has(StringRef{pos, len}); + return tld_list.lookup(host); } }; diff --git a/tests/queries/0_stateless/01601_custom_tld.reference b/tests/queries/0_stateless/01601_custom_tld.reference index b4afe625dac..981067606a2 100644 --- a/tests/queries/0_stateless/01601_custom_tld.reference +++ b/tests/queries/0_stateless/01601_custom_tld.reference @@ -71,3 +71,21 @@ select cutToFirstSignificantSubdomainCustom('a.foo.bar', 'no_new_line_list'); a.foo.bar select cutToFirstSignificantSubdomainCustom('a.foo.baz', 'no_new_line_list'); foo.baz +select '-- asterisk'; +-- asterisk +select cutToFirstSignificantSubdomainCustom('foo.something.sheffield.sch.uk', 'public_suffix_list'); +something.sheffield.sch.uk +select cutToFirstSignificantSubdomainCustom('something.sheffield.sch.uk', 'public_suffix_list'); +something.sheffield.sch.uk +select cutToFirstSignificantSubdomainCustom('sheffield.sch.uk', 'public_suffix_list'); +sheffield.sch.uk +select '-- exclamation mark'; +-- exclamation mark +select cutToFirstSignificantSubdomainCustom('foo.kawasaki.jp', 'public_suffix_list'); +foo.kawasaki.jp +select cutToFirstSignificantSubdomainCustom('foo.foo.kawasaki.jp', 'public_suffix_list'); +foo.foo.kawasaki.jp +select cutToFirstSignificantSubdomainCustom('city.kawasaki.jp', 'public_suffix_list'); +city.kawasaki.jp +select cutToFirstSignificantSubdomainCustom('some.city.kawasaki.jp', 'public_suffix_list'); +city.kawasaki.jp diff --git a/tests/queries/0_stateless/01601_custom_tld.sql b/tests/queries/0_stateless/01601_custom_tld.sql index fd55e229fbc..69ae209af2c 100644 --- a/tests/queries/0_stateless/01601_custom_tld.sql +++ b/tests/queries/0_stateless/01601_custom_tld.sql @@ -44,3 +44,14 @@ select '-- no new line'; select cutToFirstSignificantSubdomainCustom('foo.bar', 'no_new_line_list'); select cutToFirstSignificantSubdomainCustom('a.foo.bar', 'no_new_line_list'); select cutToFirstSignificantSubdomainCustom('a.foo.baz', 'no_new_line_list'); + +select '-- asterisk'; +select cutToFirstSignificantSubdomainCustom('foo.something.sheffield.sch.uk', 'public_suffix_list'); +select cutToFirstSignificantSubdomainCustom('something.sheffield.sch.uk', 'public_suffix_list'); +select cutToFirstSignificantSubdomainCustom('sheffield.sch.uk', 'public_suffix_list'); + +select '-- exclamation mark'; +select cutToFirstSignificantSubdomainCustom('foo.kawasaki.jp', 'public_suffix_list'); +select cutToFirstSignificantSubdomainCustom('foo.foo.kawasaki.jp', 'public_suffix_list'); +select cutToFirstSignificantSubdomainCustom('city.kawasaki.jp', 'public_suffix_list'); +select cutToFirstSignificantSubdomainCustom('some.city.kawasaki.jp', 'public_suffix_list'); From bb292b6aeb4556de394a9d6e206f04e7ead88a9c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Jul 2022 07:59:58 +0200 Subject: [PATCH 632/659] Fix wrong REGEXP_REPLACE alias --- tests/queries/0_stateless/02374_regexp_replace.reference | 1 + tests/queries/0_stateless/02374_regexp_replace.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02374_regexp_replace.reference create mode 100644 tests/queries/0_stateless/02374_regexp_replace.sql diff --git a/tests/queries/0_stateless/02374_regexp_replace.reference b/tests/queries/0_stateless/02374_regexp_replace.reference new file mode 100644 index 00000000000..18915fea169 --- /dev/null +++ b/tests/queries/0_stateless/02374_regexp_replace.reference @@ -0,0 +1 @@ +https://www.clickhouse.com/ clickhouse.com diff --git a/tests/queries/0_stateless/02374_regexp_replace.sql b/tests/queries/0_stateless/02374_regexp_replace.sql new file mode 100644 index 00000000000..326adb7e618 --- /dev/null +++ b/tests/queries/0_stateless/02374_regexp_replace.sql @@ -0,0 +1 @@ +SELECT 'https://www.clickhouse.com/' AS s, REGEXP_REPLACE(s, '^https?://(?:www\.)?([^/]+)/.*$', '\1'); From a2522d6196969a71e5d47157be35f4f3a960bc1d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Jul 2022 08:00:49 +0200 Subject: [PATCH 633/659] Revert "Fix wrong REGEXP_REPLACE alias" This reverts commit bb292b6aeb4556de394a9d6e206f04e7ead88a9c. --- tests/queries/0_stateless/02374_regexp_replace.reference | 1 - tests/queries/0_stateless/02374_regexp_replace.sql | 1 - 2 files changed, 2 deletions(-) delete mode 100644 tests/queries/0_stateless/02374_regexp_replace.reference delete mode 100644 tests/queries/0_stateless/02374_regexp_replace.sql diff --git a/tests/queries/0_stateless/02374_regexp_replace.reference b/tests/queries/0_stateless/02374_regexp_replace.reference deleted file mode 100644 index 18915fea169..00000000000 --- a/tests/queries/0_stateless/02374_regexp_replace.reference +++ /dev/null @@ -1 +0,0 @@ -https://www.clickhouse.com/ clickhouse.com diff --git a/tests/queries/0_stateless/02374_regexp_replace.sql b/tests/queries/0_stateless/02374_regexp_replace.sql deleted file mode 100644 index 326adb7e618..00000000000 --- a/tests/queries/0_stateless/02374_regexp_replace.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT 'https://www.clickhouse.com/' AS s, REGEXP_REPLACE(s, '^https?://(?:www\.)?([^/]+)/.*$', '\1'); From bbb5e1306dae3533a87ce5d0eef00b1a6dadf60b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Jul 2022 07:59:58 +0200 Subject: [PATCH 634/659] Fix wrong REGEXP_REPLACE alias --- tests/queries/0_stateless/02374_regexp_replace.reference | 1 + tests/queries/0_stateless/02374_regexp_replace.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02374_regexp_replace.reference create mode 100644 tests/queries/0_stateless/02374_regexp_replace.sql diff --git a/tests/queries/0_stateless/02374_regexp_replace.reference b/tests/queries/0_stateless/02374_regexp_replace.reference new file mode 100644 index 00000000000..18915fea169 --- /dev/null +++ b/tests/queries/0_stateless/02374_regexp_replace.reference @@ -0,0 +1 @@ +https://www.clickhouse.com/ clickhouse.com diff --git a/tests/queries/0_stateless/02374_regexp_replace.sql b/tests/queries/0_stateless/02374_regexp_replace.sql new file mode 100644 index 00000000000..326adb7e618 --- /dev/null +++ b/tests/queries/0_stateless/02374_regexp_replace.sql @@ -0,0 +1 @@ +SELECT 'https://www.clickhouse.com/' AS s, REGEXP_REPLACE(s, '^https?://(?:www\.)?([^/]+)/.*$', '\1'); From 833b24b4868e7fa8e882d1638bf700d0cb409b49 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Jul 2022 08:01:49 +0200 Subject: [PATCH 635/659] Fix the wrong REGEXP_REPLACE alias --- src/Functions/replaceAll.cpp | 1 - src/Functions/replaceRegexpAll.cpp | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/replaceAll.cpp b/src/Functions/replaceAll.cpp index 25a5b33c3a0..cc29e57ea69 100644 --- a/src/Functions/replaceAll.cpp +++ b/src/Functions/replaceAll.cpp @@ -21,7 +21,6 @@ void registerFunctionReplaceAll(FunctionFactory & factory) { factory.registerFunction(); factory.registerAlias("replace", NameReplaceAll::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("REGEXP_REPLACE", NameReplaceAll::name, FunctionFactory::CaseInsensitive); } } diff --git a/src/Functions/replaceRegexpAll.cpp b/src/Functions/replaceRegexpAll.cpp index ad67efa82f4..07ffbdae792 100644 --- a/src/Functions/replaceRegexpAll.cpp +++ b/src/Functions/replaceRegexpAll.cpp @@ -20,6 +20,7 @@ using FunctionReplaceRegexpAll = FunctionStringReplace, void registerFunctionReplaceRegexpAll(FunctionFactory & factory) { factory.registerFunction(); + factory.registerAlias("REGEXP_REPLACE", NameReplaceRegexpAll::name, FunctionFactory::CaseInsensitive); } } From 4c98a7bc0f3fa9c2d65a262415a5ec303b87abdf Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 26 Jul 2022 08:46:51 +0200 Subject: [PATCH 636/659] Add test for Ordinary database. --- ...eplicated.xml => allow_database_types.xml} | 1 + .../test_backup_restore_on_cluster/test.py | 2 +- .../test_concurrency.py | 20 +++++++++++++------ 3 files changed, 16 insertions(+), 7 deletions(-) rename tests/integration/test_backup_restore_on_cluster/configs/{allow_experimental_database_replicated.xml => allow_database_types.xml} (68%) diff --git a/tests/integration/test_backup_restore_on_cluster/configs/allow_experimental_database_replicated.xml b/tests/integration/test_backup_restore_on_cluster/configs/allow_database_types.xml similarity index 68% rename from tests/integration/test_backup_restore_on_cluster/configs/allow_experimental_database_replicated.xml rename to tests/integration/test_backup_restore_on_cluster/configs/allow_database_types.xml index 0434df06457..e0e026210b1 100644 --- a/tests/integration/test_backup_restore_on_cluster/configs/allow_experimental_database_replicated.xml +++ b/tests/integration/test_backup_restore_on_cluster/configs/allow_database_types.xml @@ -2,6 +2,7 @@ 1 + 1
diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 58fac12f041..d1898213725 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -16,7 +16,7 @@ main_configs = [ ] user_configs = [ - "configs/allow_experimental_database_replicated.xml", + "configs/allow_database_types.xml", ] node1 = cluster.add_instance( diff --git a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py index ee26f08f14e..8eaed5ac486 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py @@ -30,7 +30,7 @@ def generate_cluster_def(): main_configs = ["configs/backups_disk.xml", generate_cluster_def()] -user_configs = ["configs/allow_experimental_database_replicated.xml"] +user_configs = ["configs/allow_database_types.xml"] nodes = [] for i in range(num_nodes): @@ -63,6 +63,7 @@ def drop_after_test(): yield finally: node0.query("DROP TABLE IF EXISTS tbl ON CLUSTER 'cluster' NO DELAY") + node0.query("DROP DATABASE IF EXISTS mydb ON CLUSTER 'cluster' NO DELAY") backup_id_counter = 0 @@ -172,10 +173,17 @@ def test_concurrent_backups_on_different_nodes(): assert nodes[j].query("SELECT sum(x) FROM tbl") == TSV([expected_sum]) -def test_create_or_drop_tables_during_backup(): - node0.query( - "CREATE DATABASE mydb ON CLUSTER 'cluster' ENGINE=Replicated('/clickhouse/path/','{shard}','{replica}')" - ) +@pytest.mark.parametrize( + "db_engine, table_engine", + [("Replicated", "ReplicatedMergeTree"), ("Ordinary", "MergeTree")], +) +def test_create_or_drop_tables_during_backup(db_engine, table_engine): + if db_engine == "Replicated": + db_engine = "Replicated('/clickhouse/path/','{shard}','{replica}')" + if table_engine.endswith("MergeTree"): + table_engine += " ORDER BY tuple()" + + node0.query(f"CREATE DATABASE mydb ON CLUSTER 'cluster' ENGINE={db_engine}") # Will do this test for 60 seconds start_time = time.time() @@ -186,7 +194,7 @@ def test_create_or_drop_tables_during_backup(): node = nodes[randint(0, num_nodes - 1)] table_name = f"mydb.tbl{randint(1, num_nodes)}" node.query( - f"CREATE TABLE IF NOT EXISTS {table_name}(x Int32) ENGINE=ReplicatedMergeTree ORDER BY x" + f"CREATE TABLE IF NOT EXISTS {table_name}(x Int32) ENGINE={table_engine}" ) node.query_and_get_answer_with_error( f"INSERT INTO {table_name} SELECT rand32() FROM numbers(10)" From 76599d123190701b9e73367ce71609c6bdcb043e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 26 Jul 2022 08:58:33 +0200 Subject: [PATCH 637/659] Finally fix locking storages for reading during backup. --- src/Backups/BackupEntriesCollector.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 3cd9649de61..695b5a0fbb4 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -537,12 +537,11 @@ void BackupEntriesCollector::lockTablesForReading() for (auto & [table_name, table_info] : table_infos) { auto storage = table_info.storage; - TableLockHolder table_lock; if (storage) { try { - table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); + table_info.table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); } catch (Exception & e) { From c0ec6fd9130e7bd397a6a11b38284c28a5f23e84 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 26 Jul 2022 09:53:32 +0200 Subject: [PATCH 638/659] Use Poco::Event to simplify code. --- src/Backups/BackupCoordinationStageSync.cpp | 39 ++++----------------- 1 file changed, 6 insertions(+), 33 deletions(-) diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp index 4b94e474345..5e0efbdd7b9 100644 --- a/src/Backups/BackupCoordinationStageSync.cpp +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -130,27 +129,8 @@ Strings BackupCoordinationStageSync::waitImpl(const Strings & all_hosts, const S auto zookeeper = get_zookeeper(); - struct Watch - { - std::mutex mutex; - std::condition_variable event; - bool zk_nodes_changed = false; - bool watch_set = false; - }; - - /// shared_ptr because `watch_callback` can be called by ZooKeeper after leaving this function's scope. - auto watch = std::make_shared(); - - /// Called by ZooKepper when list of zk nodes have changed. - auto watch_callback = [watch](const Coordination::WatchResponse &) - { - std::lock_guard lock{watch->mutex}; - watch->zk_nodes_changed = true; - watch->watch_set = false; /// When it's triggered ZooKeeper resets the watch so we need to call getChildrenWatch() again. - watch->event.notify_all(); - }; - - auto zk_nodes_changed = [watch] { return watch->zk_nodes_changed; }; + /// Set by ZooKepper when list of zk nodes have changed. + auto watch = std::make_shared(); bool use_timeout = timeout.has_value(); std::chrono::steady_clock::time_point end_of_timeout; @@ -164,12 +144,7 @@ Strings BackupCoordinationStageSync::waitImpl(const Strings & all_hosts, const S for (;;) { /// Get zk nodes and subscribe on their changes. - { - std::lock_guard lock{watch->mutex}; - watch->watch_set = true; - watch->zk_nodes_changed = false; - } - Strings zk_nodes = zookeeper->getChildrenWatch(zookeeper_path, nullptr, watch_callback); + Strings zk_nodes = zookeeper->getChildren(zookeeper_path, nullptr, watch); /// Read and analyze the current state of zk nodes. state = readCurrentState(zookeeper, zk_nodes, all_hosts, stage_to_wait); @@ -186,19 +161,17 @@ Strings BackupCoordinationStageSync::waitImpl(const Strings & all_hosts, const S /// Wait until `watch_callback` is called by ZooKeeper meaning that zk nodes have changed. { - std::unique_lock lock{watch->mutex}; if (use_timeout) { auto current_time = std::chrono::steady_clock::now(); - if ((current_time > end_of_timeout) || !watch->event.wait_for(lock, end_of_timeout - current_time, zk_nodes_changed)) + if ((current_time > end_of_timeout) + || !watch->tryWait(std::chrono::duration_cast(end_of_timeout - current_time).count())) break; } else { - watch->event.wait(lock, zk_nodes_changed); + watch->wait(); } - assert(watch->zk_nodes_changed); - assert(!watch->watch_set); } } From 162be5acbeff28d5947f717bc5fd9a9e9729b63a Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Tue, 26 Jul 2022 16:10:00 +0800 Subject: [PATCH 639/659] set mutations_sync to 2 to wait for mutations to complete on all replicas --- .../02352_lightweight_delete_on_replicated_merge_tree.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.sql b/tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.sql index 3df9acbee87..417dfeea094 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.sql +++ b/tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.sql @@ -6,7 +6,7 @@ CREATE TABLE replicated_table_r2(id Int32, name String) ENGINE = ReplicatedMerge INSERT INTO replicated_table_r1 select number, toString(number) FROM numbers(100); -SET mutations_sync = 1; +SET mutations_sync = 2; SET allow_experimental_lightweight_delete = 1; DELETE FROM replicated_table_r1 WHERE id = 10; From d0183de34a6142a49087cae25ccd773efd958285 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Tue, 26 Jul 2022 11:00:08 +0200 Subject: [PATCH 640/659] Update docs/en/operations/settings/settings.md Co-authored-by: Antonio Andelic --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 59ac34bd6f1..5d8ab5683c6 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -308,7 +308,7 @@ Possible values: - `default` — `hash` or `direct`, if possible (same as `direct,hash`) -- `hash` — [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section. +- `hash` — [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section. - `parallel_hash` - a variation of `hash` join that splits the data into buckets and builds several hashtables instead of one concurrently to speed up this process. From 0d65415086d0e31c7071d83a4d70f69e8f62026a Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Tue, 26 Jul 2022 11:00:12 +0200 Subject: [PATCH 641/659] Update docs/en/operations/settings/settings.md Co-authored-by: Antonio Andelic --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 5d8ab5683c6..94fc8512bc8 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -328,7 +328,7 @@ The `direct` algorithm performs a lookup in the right table using rows from the - `full_sorting_merge` — [Sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join) with full sorting joined tables before joining. -- `prefer_partial_merge` — ClickHouse always tries to use `partial_merge` join if possible, otherwise uses `hash`. *Deprecated*, same as `partial_merge,hash`. +- `prefer_partial_merge` — ClickHouse always tries to use `partial_merge` join if possible, otherwise, it uses `hash`. *Deprecated*, same as `partial_merge,hash`. ## join_any_take_last_row {#settings-join_any_take_last_row} From 683a8866ef4798e1c0695ac13afb2b16b0de28c4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 Jul 2022 09:36:39 +0000 Subject: [PATCH 642/659] Fix Chain::addSink --- src/QueryPipeline/Chain.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/QueryPipeline/Chain.cpp b/src/QueryPipeline/Chain.cpp index c33713dbe81..e5f2556a44f 100644 --- a/src/QueryPipeline/Chain.cpp +++ b/src/QueryPipeline/Chain.cpp @@ -96,7 +96,7 @@ void Chain::addSink(ProcessorPtr processor) if (!processors.empty()) connect(getOutputPort(), processor->getInputs().front()); - processors.emplace_front(std::move(processor)); + processors.emplace_back(std::move(processor)); } IProcessor & Chain::getSource() From f0cd564648c7dfdd772fbd7884e6ca86ef5a0ae5 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 26 Jul 2022 11:17:08 +0200 Subject: [PATCH 643/659] Changes after review and added comments. --- src/Backups/BackupCoordinationRemote.cpp | 6 ++ src/Backups/BackupCoordinationStage.cpp | 13 ++++ src/Backups/BackupCoordinationStage.h | 41 +++++++++++ src/Backups/BackupCoordinationStageSync.cpp | 2 +- src/Backups/BackupEntriesCollector.cpp | 43 ++++------- src/Backups/BackupsWorker.cpp | 80 ++++++++++++++++----- src/Backups/BackupsWorker.h | 2 +- src/Backups/RestoreCoordinationRemote.cpp | 7 ++ src/Backups/RestorerFromBackup.cpp | 29 +++----- src/Interpreters/InterpreterBackupQuery.cpp | 13 ++-- 10 files changed, 160 insertions(+), 76 deletions(-) create mode 100644 src/Backups/BackupCoordinationStage.cpp create mode 100644 src/Backups/BackupCoordinationStage.h diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index bac99b0da2d..8ef2db5d6f1 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -204,6 +204,12 @@ void BackupCoordinationRemote::createRootNodes() void BackupCoordinationRemote::removeAllNodes() { + /// Usually this function is called by the initiator when a backup is complete so we don't need the coordination anymore. + /// + /// However there can be a rare situation when this function is called after an error occurs on the initiator of a query + /// while some hosts are still making the backup. Removing all the nodes will remove the parent node of the backup coordination + /// at `zookeeper_path` which might cause such hosts to stop with exception "ZNONODE". Or such hosts might still do some useless part + /// of their backup work before that. Anyway in this case backup won't be finalized (because only an initiator can do that). auto zookeeper = get_zookeeper(); zookeeper->removeRecursive(zookeeper_path); } diff --git a/src/Backups/BackupCoordinationStage.cpp b/src/Backups/BackupCoordinationStage.cpp new file mode 100644 index 00000000000..bb8abdf95c4 --- /dev/null +++ b/src/Backups/BackupCoordinationStage.cpp @@ -0,0 +1,13 @@ +#include +#include + + +namespace DB +{ + +String BackupCoordinationStage::formatGatheringMetadata(size_t pass) +{ + return fmt::format("{} ({})", GATHERING_METADATA, pass); +} + +} diff --git a/src/Backups/BackupCoordinationStage.h b/src/Backups/BackupCoordinationStage.h new file mode 100644 index 00000000000..091c1f11463 --- /dev/null +++ b/src/Backups/BackupCoordinationStage.h @@ -0,0 +1,41 @@ +#pragma once + +#include + + +namespace DB +{ + +namespace BackupCoordinationStage +{ + /// Finding all tables and databases which we're going to put to the backup and collecting their metadata. + constexpr const char * GATHERING_METADATA = "gathering metadata"; + + String formatGatheringMetadata(size_t pass); + + /// Making temporary hard links and prepare backup entries. + constexpr const char * EXTRACTING_DATA_FROM_TABLES = "extracting data from tables"; + + /// Running special tasks for replicated tables which can also prepare some backup entries. + constexpr const char * RUNNING_POST_TASKS = "running post-tasks"; + + /// Writing backup entries to the backup and removing temporary hard links. + constexpr const char * WRITING_BACKUP = "writing backup"; + + /// Finding databases and tables in the backup which we're going to restore. + constexpr const char * FINDING_TABLES_IN_BACKUP = "finding tables in backup"; + + /// Creating databases or finding them and checking their definitions. + constexpr const char * CREATING_DATABASES = "creating databases"; + + /// Creating tables or finding them and checking their definition. + constexpr const char * CREATING_TABLES = "creating tables"; + + /// Inserting restored data to tables. + constexpr const char * INSERTING_DATA_TO_TABLES = "inserting data to tables"; + + /// Coordination stage meaning that a host finished its work. + constexpr const char * COMPLETED = "completed"; +} + +} diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp index 5e0efbdd7b9..e4773223075 100644 --- a/src/Backups/BackupCoordinationStageSync.cpp +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -43,7 +43,7 @@ void BackupCoordinationStageSync::set(const String & current_host, const String throw zkutil::KeeperException(code, alive_node_path); zookeeper->createIfNotExists(zookeeper_path + "/started|" + current_host, ""); - zookeeper->createIfNotExists(zookeeper_path + "/current|" + current_host + "|" + new_stage, message); + zookeeper->create(zookeeper_path + "/current|" + current_host + "|" + new_stage, message, zkutil::CreateMode::Persistent); } void BackupCoordinationStageSync::setError(const String & current_host, const Exception & exception) diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 695b5a0fbb4..22245f7056a 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -31,25 +32,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } + +namespace Stage = BackupCoordinationStage; + namespace { - /// Finding all tables and databases which we're going to put to the backup and collecting their metadata. - constexpr const char * kGatheringMetadataStage = "gathering metadata"; - - String formatGatheringMetadataStage(size_t pass) - { - return fmt::format("{} ({})", kGatheringMetadataStage, pass); - } - - /// Making temporary hard links and prepare backup entries. - constexpr const char * kExtractingDataFromTablesStage = "extracting data from tables"; - - /// Running special tasks for replicated tables which can also prepare some backup entries. - constexpr const char * kRunningPostTasksStage = "running post-tasks"; - - /// Writing backup entries to the backup and removing temporary hard links. - constexpr const char * kWritingBackupStage = "writing backup"; - /// Uppercases the first character of a passed string. String toUpperFirst(const String & str) { @@ -129,15 +116,15 @@ BackupEntries BackupEntriesCollector::run() makeBackupEntriesForTablesDefs(); /// Make backup entries for the data of the found tables. - setStage(kExtractingDataFromTablesStage); + setStage(Stage::EXTRACTING_DATA_FROM_TABLES); makeBackupEntriesForTablesData(); /// Run all the tasks added with addPostCollectingTask(). - setStage(kRunningPostTasksStage); + setStage(Stage::RUNNING_POST_TASKS); runPostTasks(); /// No more backup entries or tasks are allowed after this point. - setStage(kWritingBackupStage); + setStage(Stage::WRITING_BACKUP); return std::move(backup_entries); } @@ -149,11 +136,11 @@ Strings BackupEntriesCollector::setStage(const String & new_stage, const String backup_coordination->setStage(backup_settings.host_id, new_stage, message); - if (new_stage == formatGatheringMetadataStage(1)) + if (new_stage == Stage::formatGatheringMetadata(1)) { return backup_coordination->waitForStage(all_hosts, new_stage, on_cluster_first_sync_timeout); } - else if (new_stage.starts_with(kGatheringMetadataStage)) + else if (new_stage.starts_with(Stage::GATHERING_METADATA)) { auto current_time = std::chrono::steady_clock::now(); auto end_of_timeout = std::max(current_time, consistent_metadata_snapshot_end_time); @@ -183,13 +170,13 @@ void BackupEntriesCollector::calculateRootPathInBackup() /// Finds databases and tables which we will put to the backup. void BackupEntriesCollector::gatherMetadataAndCheckConsistency() { - setStage(formatGatheringMetadataStage(1)); + setStage(Stage::formatGatheringMetadata(1)); consistent_metadata_snapshot_end_time = std::chrono::steady_clock::now() + consistent_metadata_snapshot_timeout; for (size_t pass = 1;; ++pass) { - String next_stage = formatGatheringMetadataStage(pass + 1); + String next_stage = Stage::formatGatheringMetadata(pass + 1); std::optional inconsistency_error; if (tryGatherMetadataAndCompareWithPrevious(inconsistency_error)) { @@ -722,7 +709,7 @@ void BackupEntriesCollector::makeBackupEntriesForTableData(const QualifiedTableN void BackupEntriesCollector::addBackupEntry(const String & file_name, BackupEntryPtr backup_entry) { - if (current_stage == kWritingBackupStage) + if (current_stage == Stage::WRITING_BACKUP) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding backup entries is not allowed"); backup_entries.emplace_back(file_name, backup_entry); } @@ -734,21 +721,21 @@ void BackupEntriesCollector::addBackupEntry(const std::pair task) { - if (current_stage == kWritingBackupStage) + if (current_stage == Stage::WRITING_BACKUP) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding of post tasks is not allowed"); post_tasks.push(std::move(task)); } diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index b0d3fb55f2a..cd505ed587c 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -24,11 +25,15 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace Stage = BackupCoordinationStage; + namespace { - /// Coordination status meaning that a host finished its work. - constexpr const char * kCompletedStage = "completed"; - std::shared_ptr makeBackupCoordination(const String & coordination_zk_path, const ContextPtr & context, bool is_internal_backup) { if (!coordination_zk_path.empty()) @@ -130,8 +135,14 @@ std::pair BackupsWorker::startMakingBackup(const ASTPtr & query, con UUID backup_uuid = *backup_settings.backup_uuid; std::shared_ptr backup_coordination; - if (!backup_settings.coordination_zk_path.empty()) + + if (backup_settings.internal) + { + /// The following call of makeBackupCoordination() is not essential because doBackup() will later create a backup coordination + /// if it's not created here. However to handle errors better it's better to make a coordination here because this way + /// if an exception will be thrown in startMakingBackup() other hosts will know about that. backup_coordination = makeBackupCoordination(backup_settings.coordination_zk_path, context, backup_settings.internal); + } try { @@ -161,12 +172,20 @@ std::pair BackupsWorker::startMakingBackup(const ASTPtr & query, con backup_coordination, context_in_use, mutable_context, - true); + /* called_async= */ true); }); } else { - doBackup(backup_uuid, backup_query, backup_settings, backup_info, backup_coordination, context_in_use, mutable_context, false); + doBackup( + backup_uuid, + backup_query, + backup_settings, + backup_info, + backup_coordination, + context_in_use, + mutable_context, + /* called_async= */ false); } return {backup_uuid, backup_settings.internal}; @@ -258,7 +277,7 @@ void BackupsWorker::doBackup( /// Wait until all the hosts have written their backup entries. auto all_hosts = BackupSettings::Util::filterHostIDs( backup_settings.cluster_host_ids, backup_settings.shard_num, backup_settings.replica_num); - backup_coordination->waitForStage(all_hosts, kCompletedStage); + backup_coordination->waitForStage(all_hosts, Stage::COMPLETED); } else { @@ -275,7 +294,7 @@ void BackupsWorker::doBackup( writeBackupEntries(backup, std::move(backup_entries), backups_thread_pool); /// We have written our backup entries, we need to tell other hosts (they could be waiting for it). - backup_coordination->setStage(backup_settings.host_id, kCompletedStage, ""); + backup_coordination->setStage(backup_settings.host_id, Stage::COMPLETED, ""); } /// Finalize backup (write its metadata). @@ -313,8 +332,14 @@ std::pair BackupsWorker::startRestoring(const ASTPtr & query, Contex UUID restore_uuid = UUIDHelpers::generateV4(); std::shared_ptr restore_coordination; - if (!restore_settings.coordination_zk_path.empty()) + + if (restore_settings.internal) + { + /// The following call of makeRestoreCoordination() is not essential because doRestore() will later create a restore coordination + /// if it's not created here. However to handle errors better it's better to make a coordination here because this way + /// if an exception will be thrown in startRestoring() other hosts will know about that. restore_coordination = makeRestoreCoordination(restore_settings.coordination_zk_path, context, restore_settings.internal); + } try { @@ -334,12 +359,27 @@ std::pair BackupsWorker::startRestoring(const ASTPtr & query, Contex if (restore_settings.async) { backups_thread_pool.scheduleOrThrowOnError( - [this, restore_uuid, restore_query, restore_settings, backup_info, restore_coordination, context_in_use] - { doRestore(restore_uuid, restore_query, restore_settings, backup_info, restore_coordination, context_in_use, true); }); + [this, restore_uuid, restore_query, restore_settings, backup_info, restore_coordination, context_in_use] { + doRestore( + restore_uuid, + restore_query, + restore_settings, + backup_info, + restore_coordination, + context_in_use, + /* called_async= */ true); + }); } else { - doRestore(restore_uuid, restore_query, restore_settings, backup_info, restore_coordination, context_in_use, false); + doRestore( + restore_uuid, + restore_query, + restore_settings, + backup_info, + restore_coordination, + context_in_use, + /* called_async= */ false); } return {restore_uuid, restore_settings.internal}; @@ -438,7 +478,7 @@ void BackupsWorker::doRestore( /// Wait until all the hosts have written their backup entries. auto all_hosts = BackupSettings::Util::filterHostIDs( restore_settings.cluster_host_ids, restore_settings.shard_num, restore_settings.replica_num); - restore_coordination->waitForStage(all_hosts, kCompletedStage); + restore_coordination->waitForStage(all_hosts, Stage::COMPLETED); } else { @@ -456,7 +496,7 @@ void BackupsWorker::doRestore( restoreTablesData(std::move(data_restore_tasks), restores_thread_pool); /// We have restored everything, we need to tell other hosts (they could be waiting for it). - restore_coordination->setStage(restore_settings.host_id, kCompletedStage, ""); + restore_coordination->setStage(restore_settings.host_id, Stage::COMPLETED, ""); } LOG_INFO(log, "Restored from {} {} successfully", (restore_settings.internal ? "internal backup" : "backup"), backup_info.toString()); @@ -490,7 +530,9 @@ void BackupsWorker::addInfo(const UUID & uuid, bool internal, const String & bac info.internal = internal; std::lock_guard lock{infos_mutex}; - infos[{uuid, internal}] = std::move(info); + bool inserted = infos.try_emplace({uuid, internal}, std::move(info)).second; + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Pair of UUID={} and internal={} is already in use", uuid, internal); num_active_backups += getNumActiveBackupsChange(status); num_active_restores += getNumActiveRestoresChange(status); @@ -502,7 +544,7 @@ void BackupsWorker::setStatus(const UUID & uuid, bool internal, BackupStatus sta std::lock_guard lock{infos_mutex}; auto it = infos.find({uuid, internal}); if (it == infos.end()) - return; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown pair of UUID={} and internal={}", uuid, internal); auto & info = it->second; auto old_status = info.status; @@ -520,7 +562,7 @@ void BackupsWorker::wait(const UUID & backup_or_restore_uuid, bool internal, boo { auto it = infos.find({backup_or_restore_uuid, internal}); if (it == infos.end()) - return true; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown pair of UUID={} and internal={}", backup_or_restore_uuid, internal); const auto & info = it->second; auto current_status = info.status; if (rethrow_exception && ((current_status == BackupStatus::FAILED_TO_BACKUP) || (current_status == BackupStatus::FAILED_TO_RESTORE))) @@ -529,12 +571,12 @@ void BackupsWorker::wait(const UUID & backup_or_restore_uuid, bool internal, boo }); } -std::optional BackupsWorker::tryGetInfo(const UUID & backup_or_restore_uuid, bool internal) const +BackupsWorker::Info BackupsWorker::getInfo(const UUID & backup_or_restore_uuid, bool internal) const { std::lock_guard lock{infos_mutex}; auto it = infos.find({backup_or_restore_uuid, internal}); if (it == infos.end()) - return std::nullopt; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown pair of UUID={} and internal={}", backup_or_restore_uuid, internal); return it->second; } diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index 7ae69271d26..7db62633412 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -54,7 +54,7 @@ public: bool internal = false; }; - std::optional tryGetInfo(const UUID & backup_or_restore_uuid, bool internal) const; + Info getInfo(const UUID & backup_or_restore_uuid, bool internal) const; std::vector getAllInfos() const; private: diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index fcc6a2a24b3..e1ec8313cb5 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -101,6 +101,13 @@ bool RestoreCoordinationRemote::acquireReplicatedAccessStorage(const String & ac void RestoreCoordinationRemote::removeAllNodes() { + /// Usually this function is called by the initiator when a restore operation is complete so we don't need the coordination anymore. + /// + /// However there can be a rare situation when this function is called after an error occurs on the initiator of a query + /// while some hosts are still restoring something. Removing all the nodes will remove the parent node of the restore coordination + /// at `zookeeper_path` which might cause such hosts to stop with exception "ZNONODE". Or such hosts might still do some part + /// of their restore work before that. + auto zookeeper = get_zookeeper(); zookeeper->removeRecursive(zookeeper_path); } diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 3a2f5273611..185c23a479e 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -38,20 +39,10 @@ namespace ErrorCodes } +namespace Stage = BackupCoordinationStage; + namespace { - /// Finding databases and tables in the backup which we're going to restore. - constexpr const char * kFindingTablesInBackupStage = "finding tables in backup"; - - /// Creating databases or finding them and checking their definitions. - constexpr const char * kCreatingDatabasesStage = "creating databases"; - - /// Creating tables or finding them and checking their definition. - constexpr const char * kCreatingTablesStage = "creating tables"; - - /// Inserting restored data to tables. - constexpr const char * kInsertingDataToTablesStage = "inserting data to tables"; - /// Uppercases the first character of a passed string. String toUpperFirst(const String & str) { @@ -127,7 +118,7 @@ RestorerFromBackup::DataRestoreTasks RestorerFromBackup::run(Mode mode) findRootPathsInBackup(); /// Find all the databases and tables which we will read from the backup. - setStage(kFindingTablesInBackupStage); + setStage(Stage::FINDING_TABLES_IN_BACKUP); findDatabasesAndTablesInBackup(); /// Check access rights. @@ -137,16 +128,16 @@ RestorerFromBackup::DataRestoreTasks RestorerFromBackup::run(Mode mode) return {}; /// Create databases using the create queries read from the backup. - setStage(kCreatingDatabasesStage); + setStage(Stage::CREATING_DATABASES); createDatabases(); /// Create tables using the create queries read from the backup. - setStage(kCreatingTablesStage); + setStage(Stage::CREATING_TABLES); createTables(); /// All what's left is to insert data to tables. /// No more data restoring tasks are allowed after this point. - setStage(kInsertingDataToTablesStage); + setStage(Stage::INSERTING_DATA_TO_TABLES); return getDataRestoreTasks(); } @@ -158,7 +149,7 @@ void RestorerFromBackup::setStage(const String & new_stage, const String & messa if (restore_coordination) { restore_coordination->setStage(restore_settings.host_id, new_stage, message); - if (new_stage == kFindingTablesInBackupStage) + if (new_stage == Stage::FINDING_TABLES_IN_BACKUP) restore_coordination->waitForStage(all_hosts, new_stage, on_cluster_first_sync_timeout); else restore_coordination->waitForStage(all_hosts, new_stage); @@ -819,14 +810,14 @@ std::vector RestorerFromBackup::findTablesWithoutDependencie void RestorerFromBackup::addDataRestoreTask(DataRestoreTask && new_task) { - if (current_stage == kInsertingDataToTablesStage) + if (current_stage == Stage::INSERTING_DATA_TO_TABLES) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding of data-restoring tasks is not allowed"); data_restore_tasks.push_back(std::move(new_task)); } void RestorerFromBackup::addDataRestoreTasks(DataRestoreTasks && new_tasks) { - if (current_stage == kInsertingDataToTablesStage) + if (current_stage == Stage::INSERTING_DATA_TO_TABLES) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding of data-restoring tasks is not allowed"); insertAtEnd(data_restore_tasks, std::move(new_tasks)); } diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index a2bb3acf073..8f2060d2d02 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -17,18 +17,15 @@ namespace DB namespace { - Block getResultRow(const std::optional & info) + Block getResultRow(const BackupsWorker::Info & info) { auto column_uuid = ColumnUUID::create(); auto column_backup_name = ColumnString::create(); auto column_status = ColumnInt8::create(); - if (info) - { - column_uuid->insert(info->uuid); - column_backup_name->insert(info->backup_name); - column_status->insert(static_cast(info->status)); - } + column_uuid->insert(info.uuid); + column_backup_name->insert(info.backup_name); + column_status->insert(static_cast(info.status)); Block res_columns; res_columns.insert(0, {std::move(column_uuid), std::make_shared(), "uuid"}); @@ -44,7 +41,7 @@ BlockIO InterpreterBackupQuery::execute() auto & backups_worker = context->getBackupsWorker(); auto [uuid, internal] = backups_worker.start(query_ptr, context); BlockIO res_io; - res_io.pipeline = QueryPipeline(std::make_shared(getResultRow(backups_worker.tryGetInfo(uuid, internal)))); + res_io.pipeline = QueryPipeline(std::make_shared(getResultRow(backups_worker.getInfo(uuid, internal)))); return res_io; } From 1b1fa54db1546712491b19b1aa0985abbee979d4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 26 Jul 2022 12:42:37 +0200 Subject: [PATCH 644/659] Added 'long' tag --- .../02352_lightweight_delete_on_replicated_merge_tree.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.sql b/tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.sql index 417dfeea094..15e4d45581d 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.sql +++ b/tests/queries/0_stateless/02352_lightweight_delete_on_replicated_merge_tree.sql @@ -1,3 +1,5 @@ +-- Tags: long + DROP TABLE IF EXISTS replicated_table_r1 SYNC; DROP TABLE IF EXISTS replicated_table_r2 SYNC; From 98cb7d2dfeaddae79cdea746369dc083b784ef02 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 26 Jul 2022 14:59:07 +0300 Subject: [PATCH 645/659] Update 02319_sql_standard_create_drop_index.sql --- .../0_stateless/02319_sql_standard_create_drop_index.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02319_sql_standard_create_drop_index.sql b/tests/queries/0_stateless/02319_sql_standard_create_drop_index.sql index bb01dcf2e64..581b170ee65 100644 --- a/tests/queries/0_stateless/02319_sql_standard_create_drop_index.sql +++ b/tests/queries/0_stateless/02319_sql_standard_create_drop_index.sql @@ -16,8 +16,8 @@ select table, name, type, expr, granularity from system.data_skipping_indices wh drop table t_index; -create table t_index(a int, b String) engine=ReplicatedMergeTree('/test/2319/{database}/', '1') order by a; -create table t_index_replica(a int, b String) engine=ReplicatedMergeTree('/test/2319/{database}/', '2') order by a; +create table t_index(a int, b String) engine=ReplicatedMergeTree('/test/2319/{database}', '1') order by a; +create table t_index_replica(a int, b String) engine=ReplicatedMergeTree('/test/2319/{database}', '2') order by a; create index i_a on t_index(a) TYPE minmax GRANULARITY 4; create index if not exists i_a on t_index(a) TYPE minmax GRANULARITY 2; From e302bb8f38442e4a1ab3519bd0ddf348286f26cc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 26 Jul 2022 14:59:32 +0300 Subject: [PATCH 646/659] Update 02319_sql_standard_create_drop_index.reference --- .../02319_sql_standard_create_drop_index.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02319_sql_standard_create_drop_index.reference b/tests/queries/0_stateless/02319_sql_standard_create_drop_index.reference index a4a924fd229..bb0c387976a 100644 --- a/tests/queries/0_stateless/02319_sql_standard_create_drop_index.reference +++ b/tests/queries/0_stateless/02319_sql_standard_create_drop_index.reference @@ -2,8 +2,8 @@ CREATE TABLE default.t_index\n(\n `a` Int32,\n `b` String,\n INDEX i_a t_index i_a minmax a 4 t_index i_b bloom_filter b 2 t_index i_b bloom_filter b 2 -CREATE TABLE default.t_index\n(\n `a` Int32,\n `b` String,\n INDEX i_a a TYPE minmax GRANULARITY 4,\n INDEX i_b b TYPE bloom_filter GRANULARITY 2\n)\nENGINE = ReplicatedMergeTree(\'/test/2319/default/\', \'1\')\nORDER BY a\nSETTINGS index_granularity = 8192 -CREATE TABLE default.t_index_replica\n(\n `a` Int32,\n `b` String,\n INDEX i_a a TYPE minmax GRANULARITY 4,\n INDEX i_b b TYPE bloom_filter GRANULARITY 2\n)\nENGINE = ReplicatedMergeTree(\'/test/2319/default/\', \'2\')\nORDER BY a\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t_index\n(\n `a` Int32,\n `b` String,\n INDEX i_a a TYPE minmax GRANULARITY 4,\n INDEX i_b b TYPE bloom_filter GRANULARITY 2\n)\nENGINE = ReplicatedMergeTree(\'/test/2319/default\', \'1\')\nORDER BY a\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t_index_replica\n(\n `a` Int32,\n `b` String,\n INDEX i_a a TYPE minmax GRANULARITY 4,\n INDEX i_b b TYPE bloom_filter GRANULARITY 2\n)\nENGINE = ReplicatedMergeTree(\'/test/2319/default\', \'2\')\nORDER BY a\nSETTINGS index_granularity = 8192 t_index i_a minmax a 4 t_index i_b bloom_filter b 2 t_index i_b bloom_filter b 2 From 413024b4f44b42684fd5e706c9e441cec4b6ca32 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 26 Jul 2022 13:15:34 +0200 Subject: [PATCH 647/659] Add call ZooKeeper::sync(). --- src/Backups/BackupCoordinationRemote.cpp | 158 ++++++++++++---------- src/Backups/BackupCoordinationRemote.h | 5 +- src/Backups/RestoreCoordinationRemote.cpp | 59 +++++--- src/Backups/RestoreCoordinationRemote.h | 6 +- 4 files changed, 136 insertions(+), 92 deletions(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 8ef2db5d6f1..8d8cfc4225e 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -170,9 +170,10 @@ BackupCoordinationRemote::BackupCoordinationRemote( : zookeeper_path(zookeeper_path_) , get_zookeeper(get_zookeeper_) , remove_zk_nodes_in_destructor(remove_zk_nodes_in_destructor_) - , stage_sync(zookeeper_path_ + "/stage", get_zookeeper_, &Poco::Logger::get("BackupCoordination")) { createRootNodes(); + stage_sync.emplace( + zookeeper_path_ + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("BackupCoordination")); } BackupCoordinationRemote::~BackupCoordinationRemote() @@ -188,18 +189,37 @@ BackupCoordinationRemote::~BackupCoordinationRemote() } } +zkutil::ZooKeeperPtr BackupCoordinationRemote::getZooKeeper() const +{ + std::lock_guard lock{mutex}; + return getZooKeeperNoLock(); +} + +zkutil::ZooKeeperPtr BackupCoordinationRemote::getZooKeeperNoLock() const +{ + if (!zookeeper || zookeeper->expired()) + { + zookeeper = get_zookeeper(); + + /// It's possible that we connected to different [Zoo]Keeper instance + /// so we may read a bit stale state. + zookeeper->sync(zookeeper_path); + } + return zookeeper; +} + void BackupCoordinationRemote::createRootNodes() { - auto zookeeper = get_zookeeper(); - zookeeper->createAncestors(zookeeper_path); - zookeeper->createIfNotExists(zookeeper_path, ""); - zookeeper->createIfNotExists(zookeeper_path + "/repl_part_names", ""); - zookeeper->createIfNotExists(zookeeper_path + "/repl_mutations", ""); - zookeeper->createIfNotExists(zookeeper_path + "/repl_data_paths", ""); - zookeeper->createIfNotExists(zookeeper_path + "/repl_access", ""); - zookeeper->createIfNotExists(zookeeper_path + "/file_names", ""); - zookeeper->createIfNotExists(zookeeper_path + "/file_infos", ""); - zookeeper->createIfNotExists(zookeeper_path + "/archive_suffixes", ""); + auto zk = getZooKeeper(); + zk->createAncestors(zookeeper_path); + zk->createIfNotExists(zookeeper_path, ""); + zk->createIfNotExists(zookeeper_path + "/repl_part_names", ""); + zk->createIfNotExists(zookeeper_path + "/repl_mutations", ""); + zk->createIfNotExists(zookeeper_path + "/repl_data_paths", ""); + zk->createIfNotExists(zookeeper_path + "/repl_access", ""); + zk->createIfNotExists(zookeeper_path + "/file_names", ""); + zk->createIfNotExists(zookeeper_path + "/file_infos", ""); + zk->createIfNotExists(zookeeper_path + "/archive_suffixes", ""); } void BackupCoordinationRemote::removeAllNodes() @@ -210,29 +230,29 @@ void BackupCoordinationRemote::removeAllNodes() /// while some hosts are still making the backup. Removing all the nodes will remove the parent node of the backup coordination /// at `zookeeper_path` which might cause such hosts to stop with exception "ZNONODE". Or such hosts might still do some useless part /// of their backup work before that. Anyway in this case backup won't be finalized (because only an initiator can do that). - auto zookeeper = get_zookeeper(); - zookeeper->removeRecursive(zookeeper_path); + auto zk = getZooKeeper(); + zk->removeRecursive(zookeeper_path); } void BackupCoordinationRemote::setStage(const String & current_host, const String & new_stage, const String & message) { - stage_sync.set(current_host, new_stage, message); + stage_sync->set(current_host, new_stage, message); } void BackupCoordinationRemote::setError(const String & current_host, const Exception & exception) { - stage_sync.setError(current_host, exception); + stage_sync->setError(current_host, exception); } Strings BackupCoordinationRemote::waitForStage(const Strings & all_hosts, const String & stage_to_wait) { - return stage_sync.wait(all_hosts, stage_to_wait); + return stage_sync->wait(all_hosts, stage_to_wait); } Strings BackupCoordinationRemote::waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) { - return stage_sync.waitFor(all_hosts, stage_to_wait, timeout); + return stage_sync->waitFor(all_hosts, stage_to_wait, timeout); } @@ -248,11 +268,11 @@ void BackupCoordinationRemote::addReplicatedPartNames( throw Exception(ErrorCodes::LOGICAL_ERROR, "addReplicatedPartNames() must not be called after preparing"); } - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeper(); String path = zookeeper_path + "/repl_part_names/" + escapeForFileName(table_shared_id); - zookeeper->createIfNotExists(path, ""); + zk->createIfNotExists(path, ""); path += "/" + escapeForFileName(replica_name); - zookeeper->create(path, ReplicatedPartNames::serialize(part_names_and_checksums, table_name_for_logs), zkutil::CreateMode::Persistent); + zk->create(path, ReplicatedPartNames::serialize(part_names_and_checksums, table_name_for_logs), zkutil::CreateMode::Persistent); } Strings BackupCoordinationRemote::getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const @@ -274,11 +294,11 @@ void BackupCoordinationRemote::addReplicatedMutations( throw Exception(ErrorCodes::LOGICAL_ERROR, "addReplicatedMutations() must not be called after preparing"); } - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeper(); String path = zookeeper_path + "/repl_mutations/" + escapeForFileName(table_shared_id); - zookeeper->createIfNotExists(path, ""); + zk->createIfNotExists(path, ""); path += "/" + escapeForFileName(replica_name); - zookeeper->create(path, ReplicatedMutations::serialize(mutations, table_name_for_logs), zkutil::CreateMode::Persistent); + zk->create(path, ReplicatedMutations::serialize(mutations, table_name_for_logs), zkutil::CreateMode::Persistent); } std::vector BackupCoordinationRemote::getReplicatedMutations(const String & table_shared_id, const String & replica_name) const @@ -298,11 +318,11 @@ void BackupCoordinationRemote::addReplicatedDataPath( throw Exception(ErrorCodes::LOGICAL_ERROR, "addReplicatedDataPath() must not be called after preparing"); } - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeper(); String path = zookeeper_path + "/repl_data_paths/" + escapeForFileName(table_shared_id); - zookeeper->createIfNotExists(path, ""); + zk->createIfNotExists(path, ""); path += "/" + escapeForFileName(data_path); - zookeeper->createIfNotExists(path, ""); + zk->createIfNotExists(path, ""); } Strings BackupCoordinationRemote::getReplicatedDataPaths(const String & table_shared_id) const @@ -319,18 +339,18 @@ void BackupCoordinationRemote::prepareReplicatedTables() const return; replicated_tables.emplace(); - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeperNoLock(); { String path = zookeeper_path + "/repl_part_names"; - for (const String & escaped_table_shared_id : zookeeper->getChildren(path)) + for (const String & escaped_table_shared_id : zk->getChildren(path)) { String table_shared_id = unescapeForFileName(escaped_table_shared_id); String path2 = path + "/" + escaped_table_shared_id; - for (const String & escaped_replica_name : zookeeper->getChildren(path2)) + for (const String & escaped_replica_name : zk->getChildren(path2)) { String replica_name = unescapeForFileName(escaped_replica_name); - auto part_names = ReplicatedPartNames::deserialize(zookeeper->get(path2 + "/" + escaped_replica_name)); + auto part_names = ReplicatedPartNames::deserialize(zk->get(path2 + "/" + escaped_replica_name)); replicated_tables->addPartNames(table_shared_id, part_names.table_name_for_logs, replica_name, part_names.part_names_and_checksums); } } @@ -338,14 +358,14 @@ void BackupCoordinationRemote::prepareReplicatedTables() const { String path = zookeeper_path + "/repl_mutations"; - for (const String & escaped_table_shared_id : zookeeper->getChildren(path)) + for (const String & escaped_table_shared_id : zk->getChildren(path)) { String table_shared_id = unescapeForFileName(escaped_table_shared_id); String path2 = path + "/" + escaped_table_shared_id; - for (const String & escaped_replica_name : zookeeper->getChildren(path2)) + for (const String & escaped_replica_name : zk->getChildren(path2)) { String replica_name = unescapeForFileName(escaped_replica_name); - auto mutations = ReplicatedMutations::deserialize(zookeeper->get(path2 + "/" + escaped_replica_name)); + auto mutations = ReplicatedMutations::deserialize(zk->get(path2 + "/" + escaped_replica_name)); replicated_tables->addMutations(table_shared_id, mutations.table_name_for_logs, replica_name, mutations.mutations); } } @@ -353,11 +373,11 @@ void BackupCoordinationRemote::prepareReplicatedTables() const { String path = zookeeper_path + "/repl_data_paths"; - for (const String & escaped_table_shared_id : zookeeper->getChildren(path)) + for (const String & escaped_table_shared_id : zk->getChildren(path)) { String table_shared_id = unescapeForFileName(escaped_table_shared_id); String path2 = path + "/" + escaped_table_shared_id; - for (const String & escaped_data_path : zookeeper->getChildren(path2)) + for (const String & escaped_data_path : zk->getChildren(path2)) { String data_path = unescapeForFileName(escaped_data_path); replicated_tables->addDataPath(table_shared_id, data_path); @@ -375,13 +395,13 @@ void BackupCoordinationRemote::addReplicatedAccessFilePath(const String & access throw Exception(ErrorCodes::LOGICAL_ERROR, "addReplicatedAccessFilePath() must not be called after preparing"); } - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeper(); String path = zookeeper_path + "/repl_access/" + escapeForFileName(access_zk_path); - zookeeper->createIfNotExists(path, ""); + zk->createIfNotExists(path, ""); path += "/" + AccessEntityTypeInfo::get(access_entity_type).name; - zookeeper->createIfNotExists(path, ""); + zk->createIfNotExists(path, ""); path += "/" + host_id; - zookeeper->createIfNotExists(path, file_path); + zk->createIfNotExists(path, file_path); } Strings BackupCoordinationRemote::getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const @@ -397,20 +417,20 @@ void BackupCoordinationRemote::prepareReplicatedAccess() const return; replicated_access.emplace(); - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeperNoLock(); String path = zookeeper_path + "/repl_access"; - for (const String & escaped_access_zk_path : zookeeper->getChildren(path)) + for (const String & escaped_access_zk_path : zk->getChildren(path)) { String access_zk_path = unescapeForFileName(escaped_access_zk_path); String path2 = path + "/" + escaped_access_zk_path; - for (const String & type_str : zookeeper->getChildren(path2)) + for (const String & type_str : zk->getChildren(path2)) { AccessEntityType type = AccessEntityTypeInfo::parseType(type_str); String path3 = path2 + "/" + type_str; - for (const String & host_id : zookeeper->getChildren(path3)) + for (const String & host_id : zk->getChildren(path3)) { - String file_path = zookeeper->get(path3 + "/" + host_id); + String file_path = zk->get(path3 + "/" + host_id); replicated_access->addFilePath(access_zk_path, type, host_id, file_path); } } @@ -420,11 +440,11 @@ void BackupCoordinationRemote::prepareReplicatedAccess() const void BackupCoordinationRemote::addFileInfo(const FileInfo & file_info, bool & is_data_file_required) { - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeper(); String full_path = zookeeper_path + "/file_names/" + escapeForFileName(file_info.file_name); String size_and_checksum = serializeSizeAndChecksum(std::pair{file_info.size, file_info.checksum}); - zookeeper->create(full_path, size_and_checksum, zkutil::CreateMode::Persistent); + zk->create(full_path, size_and_checksum, zkutil::CreateMode::Persistent); if (!file_info.size) { @@ -433,7 +453,7 @@ void BackupCoordinationRemote::addFileInfo(const FileInfo & file_info, bool & is } full_path = zookeeper_path + "/file_infos/" + size_and_checksum; - auto code = zookeeper->tryCreate(full_path, serializeFileInfo(file_info), zkutil::CreateMode::Persistent); + auto code = zk->tryCreate(full_path, serializeFileInfo(file_info), zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) throw zkutil::KeeperException(code, full_path); @@ -445,15 +465,15 @@ void BackupCoordinationRemote::updateFileInfo(const FileInfo & file_info) if (!file_info.size) return; /// we don't keep FileInfos for empty files, nothing to update - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeper(); String size_and_checksum = serializeSizeAndChecksum(std::pair{file_info.size, file_info.checksum}); String full_path = zookeeper_path + "/file_infos/" + size_and_checksum; for (size_t attempt = 0; attempt < NUM_ATTEMPTS; ++attempt) { Coordination::Stat stat; - auto new_info = deserializeFileInfo(zookeeper->get(full_path, &stat)); + auto new_info = deserializeFileInfo(zk->get(full_path, &stat)); new_info.archive_suffix = file_info.archive_suffix; - auto code = zookeeper->trySet(full_path, serializeFileInfo(new_info), stat.version); + auto code = zk->trySet(full_path, serializeFileInfo(new_info), stat.version); if (code == Coordination::Error::ZOK) return; bool is_last_attempt = (attempt == NUM_ATTEMPTS - 1); @@ -464,16 +484,16 @@ void BackupCoordinationRemote::updateFileInfo(const FileInfo & file_info) std::vector BackupCoordinationRemote::getAllFileInfos() const { - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeper(); std::vector file_infos; - Strings escaped_names = zookeeper->getChildren(zookeeper_path + "/file_names"); + Strings escaped_names = zk->getChildren(zookeeper_path + "/file_names"); for (const String & escaped_name : escaped_names) { - String size_and_checksum = zookeeper->get(zookeeper_path + "/file_names/" + escaped_name); + String size_and_checksum = zk->get(zookeeper_path + "/file_names/" + escaped_name); UInt64 size = deserializeSizeAndChecksum(size_and_checksum).first; FileInfo file_info; if (size) /// we don't keep FileInfos for empty files - file_info = deserializeFileInfo(zookeeper->get(zookeeper_path + "/file_infos/" + size_and_checksum)); + file_info = deserializeFileInfo(zk->get(zookeeper_path + "/file_infos/" + size_and_checksum)); file_info.file_name = unescapeForFileName(escaped_name); file_infos.emplace_back(std::move(file_info)); } @@ -482,8 +502,8 @@ std::vector BackupCoordinationRemote::getAllFileInfos() const Strings BackupCoordinationRemote::listFiles(const String & directory, bool recursive) const { - auto zookeeper = get_zookeeper(); - Strings escaped_names = zookeeper->getChildren(zookeeper_path + "/file_names"); + auto zk = getZooKeeper(); + Strings escaped_names = zk->getChildren(zookeeper_path + "/file_names"); String prefix = directory; if (!prefix.empty() && !prefix.ends_with('/')) @@ -515,8 +535,8 @@ Strings BackupCoordinationRemote::listFiles(const String & directory, bool recur bool BackupCoordinationRemote::hasFiles(const String & directory) const { - auto zookeeper = get_zookeeper(); - Strings escaped_names = zookeeper->getChildren(zookeeper_path + "/file_names"); + auto zk = getZooKeeper(); + Strings escaped_names = zk->getChildren(zookeeper_path + "/file_names"); String prefix = directory; if (!prefix.empty() && !prefix.ends_with('/')) @@ -534,42 +554,42 @@ bool BackupCoordinationRemote::hasFiles(const String & directory) const std::optional BackupCoordinationRemote::getFileInfo(const String & file_name) const { - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeper(); String size_and_checksum; - if (!zookeeper->tryGet(zookeeper_path + "/file_names/" + escapeForFileName(file_name), size_and_checksum)) + if (!zk->tryGet(zookeeper_path + "/file_names/" + escapeForFileName(file_name), size_and_checksum)) return std::nullopt; UInt64 size = deserializeSizeAndChecksum(size_and_checksum).first; FileInfo file_info; if (size) /// we don't keep FileInfos for empty files - file_info = deserializeFileInfo(zookeeper->get(zookeeper_path + "/file_infos/" + size_and_checksum)); + file_info = deserializeFileInfo(zk->get(zookeeper_path + "/file_infos/" + size_and_checksum)); file_info.file_name = file_name; return file_info; } std::optional BackupCoordinationRemote::getFileInfo(const SizeAndChecksum & size_and_checksum) const { - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeper(); String file_info_str; - if (!zookeeper->tryGet(zookeeper_path + "/file_infos/" + serializeSizeAndChecksum(size_and_checksum), file_info_str)) + if (!zk->tryGet(zookeeper_path + "/file_infos/" + serializeSizeAndChecksum(size_and_checksum), file_info_str)) return std::nullopt; return deserializeFileInfo(file_info_str); } std::optional BackupCoordinationRemote::getFileSizeAndChecksum(const String & file_name) const { - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeper(); String size_and_checksum; - if (!zookeeper->tryGet(zookeeper_path + "/file_names/" + escapeForFileName(file_name), size_and_checksum)) + if (!zk->tryGet(zookeeper_path + "/file_names/" + escapeForFileName(file_name), size_and_checksum)) return std::nullopt; return deserializeSizeAndChecksum(size_and_checksum); } String BackupCoordinationRemote::getNextArchiveSuffix() { - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeper(); String path = zookeeper_path + "/archive_suffixes/a"; String path_created; - auto code = zookeeper->tryCreate(path, "", zkutil::CreateMode::PersistentSequential, path_created); + auto code = zk->tryCreate(path, "", zkutil::CreateMode::PersistentSequential, path_created); if (code != Coordination::Error::ZOK) throw zkutil::KeeperException(code, path); return formatArchiveSuffix(extractCounterFromSequentialNodeName(path_created)); @@ -577,8 +597,8 @@ String BackupCoordinationRemote::getNextArchiveSuffix() Strings BackupCoordinationRemote::getAllArchiveSuffixes() const { - auto zookeeper = get_zookeeper(); - Strings node_names = zookeeper->getChildren(zookeeper_path + "/archive_suffixes"); + auto zk = getZooKeeper(); + Strings node_names = zk->getChildren(zookeeper_path + "/archive_suffixes"); for (auto & node_name : node_names) node_name = formatArchiveSuffix(extractCounterFromSequentialNodeName(node_name)); return node_names; diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index d1d206683fa..83ddd7b16dc 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -57,6 +57,8 @@ public: Strings getAllArchiveSuffixes() const override; private: + zkutil::ZooKeeperPtr getZooKeeper() const; + zkutil::ZooKeeperPtr getZooKeeperNoLock() const; void createRootNodes(); void removeAllNodes(); void prepareReplicatedTables() const; @@ -66,9 +68,10 @@ private: const zkutil::GetZooKeeper get_zookeeper; const bool remove_zk_nodes_in_destructor; - BackupCoordinationStageSync stage_sync; + std::optional stage_sync; mutable std::mutex mutex; + mutable zkutil::ZooKeeperPtr zookeeper; mutable std::optional replicated_tables; mutable std::optional replicated_access; }; diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index e1ec8313cb5..89a9950aad2 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -6,13 +6,16 @@ namespace DB { -RestoreCoordinationRemote::RestoreCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool remove_zk_nodes_in_destructor_) +RestoreCoordinationRemote::RestoreCoordinationRemote( + const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool remove_zk_nodes_in_destructor_) : zookeeper_path(zookeeper_path_) , get_zookeeper(get_zookeeper_) , remove_zk_nodes_in_destructor(remove_zk_nodes_in_destructor_) - , stage_sync(zookeeper_path_ + "/stage", get_zookeeper_, &Poco::Logger::get("RestoreCoordination")) { createRootNodes(); + + stage_sync.emplace( + zookeeper_path_ + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("RestoreCoordination")); } RestoreCoordinationRemote::~RestoreCoordinationRemote() @@ -28,47 +31,61 @@ RestoreCoordinationRemote::~RestoreCoordinationRemote() } } +zkutil::ZooKeeperPtr RestoreCoordinationRemote::getZooKeeper() const +{ + std::lock_guard lock{mutex}; + if (!zookeeper || zookeeper->expired()) + { + zookeeper = get_zookeeper(); + + /// It's possible that we connected to different [Zoo]Keeper instance + /// so we may read a bit stale state. + zookeeper->sync(zookeeper_path); + } + return zookeeper; +} + void RestoreCoordinationRemote::createRootNodes() { - auto zookeeper = get_zookeeper(); - zookeeper->createAncestors(zookeeper_path); - zookeeper->createIfNotExists(zookeeper_path, ""); - zookeeper->createIfNotExists(zookeeper_path + "/repl_databases_tables_acquired", ""); - zookeeper->createIfNotExists(zookeeper_path + "/repl_tables_data_acquired", ""); - zookeeper->createIfNotExists(zookeeper_path + "/repl_access_storages_acquired", ""); + auto zk = getZooKeeper(); + zk->createAncestors(zookeeper_path); + zk->createIfNotExists(zookeeper_path, ""); + zk->createIfNotExists(zookeeper_path + "/repl_databases_tables_acquired", ""); + zk->createIfNotExists(zookeeper_path + "/repl_tables_data_acquired", ""); + zk->createIfNotExists(zookeeper_path + "/repl_access_storages_acquired", ""); } void RestoreCoordinationRemote::setStage(const String & current_host, const String & new_stage, const String & message) { - stage_sync.set(current_host, new_stage, message); + stage_sync->set(current_host, new_stage, message); } void RestoreCoordinationRemote::setError(const String & current_host, const Exception & exception) { - stage_sync.setError(current_host, exception); + stage_sync->setError(current_host, exception); } Strings RestoreCoordinationRemote::waitForStage(const Strings & all_hosts, const String & stage_to_wait) { - return stage_sync.wait(all_hosts, stage_to_wait); + return stage_sync->wait(all_hosts, stage_to_wait); } Strings RestoreCoordinationRemote::waitForStage(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) { - return stage_sync.waitFor(all_hosts, stage_to_wait, timeout); + return stage_sync->waitFor(all_hosts, stage_to_wait, timeout); } bool RestoreCoordinationRemote::acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) { - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeper(); String path = zookeeper_path + "/repl_databases_tables_acquired/" + escapeForFileName(database_zk_path); - zookeeper->createIfNotExists(path, ""); + zk->createIfNotExists(path, ""); path += "/" + escapeForFileName(table_name); - auto code = zookeeper->tryCreate(path, "", zkutil::CreateMode::Persistent); + auto code = zk->tryCreate(path, "", zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) throw zkutil::KeeperException(code, path); @@ -77,10 +94,10 @@ bool RestoreCoordinationRemote::acquireCreatingTableInReplicatedDatabase(const S bool RestoreCoordinationRemote::acquireInsertingDataIntoReplicatedTable(const String & table_zk_path) { - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeper(); String path = zookeeper_path + "/repl_tables_data_acquired/" + escapeForFileName(table_zk_path); - auto code = zookeeper->tryCreate(path, "", zkutil::CreateMode::Persistent); + auto code = zk->tryCreate(path, "", zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) throw zkutil::KeeperException(code, path); @@ -89,10 +106,10 @@ bool RestoreCoordinationRemote::acquireInsertingDataIntoReplicatedTable(const St bool RestoreCoordinationRemote::acquireReplicatedAccessStorage(const String & access_storage_zk_path) { - auto zookeeper = get_zookeeper(); + auto zk = getZooKeeper(); String path = zookeeper_path + "/repl_access_storages_acquired/" + escapeForFileName(access_storage_zk_path); - auto code = zookeeper->tryCreate(path, "", zkutil::CreateMode::Persistent); + auto code = zk->tryCreate(path, "", zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) throw zkutil::KeeperException(code, path); @@ -108,8 +125,8 @@ void RestoreCoordinationRemote::removeAllNodes() /// at `zookeeper_path` which might cause such hosts to stop with exception "ZNONODE". Or such hosts might still do some part /// of their restore work before that. - auto zookeeper = get_zookeeper(); - zookeeper->removeRecursive(zookeeper_path); + auto zk = getZooKeeper(); + zk->removeRecursive(zookeeper_path); } } diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index 0cbbb6622ad..83760a2d883 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -32,6 +32,7 @@ public: bool acquireReplicatedAccessStorage(const String & access_storage_zk_path) override; private: + zkutil::ZooKeeperPtr getZooKeeper() const; void createRootNodes(); void removeAllNodes(); @@ -41,7 +42,10 @@ private: const zkutil::GetZooKeeper get_zookeeper; const bool remove_zk_nodes_in_destructor; - BackupCoordinationStageSync stage_sync; + std::optional stage_sync; + + mutable std::mutex mutex; + mutable zkutil::ZooKeeperPtr zookeeper; }; } From 142f7d4b44b6634ec8885bbdd6b81c362ee7228b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 20 Jul 2022 19:33:35 +0200 Subject: [PATCH 648/659] Require clear style check to continue building --- .github/workflows/pull_request.yml | 32 +++++++++++++++--------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 958916b2d16..c2e816b44dc 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -254,7 +254,7 @@ jobs: #################################### ORDINARY BUILDS #################################### ######################################################################################### BuilderDebRelease: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -301,7 +301,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" BuilderBinRelease: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -346,7 +346,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" # BuilderBinGCC: - # needs: [DockerHubPush, FastTest] + # needs: [DockerHubPush, FastTest, StyleCheck] # runs-on: [self-hosted, builder] # steps: # - name: Set envs @@ -391,7 +391,7 @@ jobs: # docker rm -f $(docker ps -a -q) ||: # sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderDebAarch64: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -438,7 +438,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderDebAsan: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -483,7 +483,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderDebUBsan: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -528,7 +528,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderDebTsan: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -573,7 +573,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderDebMsan: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -618,7 +618,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderDebDebug: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -666,7 +666,7 @@ jobs: ##################################### SPECIAL BUILDS ##################################### ########################################################################################## BuilderDebSplitted: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -711,7 +711,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderBinClangTidy: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -756,7 +756,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderBinDarwin: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -801,7 +801,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderBinAarch64: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -846,7 +846,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderBinFreeBSD: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -891,7 +891,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderBinDarwinAarch64: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -936,7 +936,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderBinPPC64: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs From db0653758e612a93fca0398f0d6eb84100885300 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 18:12:07 +0200 Subject: [PATCH 649/659] Add typing to rerun and commit_status helpers --- tests/ci/commit_status_helper.py | 14 ++++++--- tests/ci/rerun_helper.py | 50 +++++++++++++++++++------------- 2 files changed, 40 insertions(+), 24 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 420ca7a0ff7..83b6203c050 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -1,10 +1,14 @@ #!/usr/bin/env python3 -import time -import os import csv -from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL +import os +import time +from typing import Optional + from ci_config import CI_CONFIG +from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL +from github import Github +from github.Commit import Commit from pr_info import SKIP_SIMPLE_CHECK_LABEL RETRY = 5 @@ -22,7 +26,9 @@ def override_status(status, check_name, invert=False): return status -def get_commit(gh, commit_sha, retry_count=RETRY): +def get_commit( + gh: Github, commit_sha: str, retry_count: int = RETRY +) -> Optional[Commit]: for i in range(retry_count): try: repo = gh.get_repo(GITHUB_REPOSITORY) diff --git a/tests/ci/rerun_helper.py b/tests/ci/rerun_helper.py index 0d523640f56..c4ae70eadb9 100644 --- a/tests/ci/rerun_helper.py +++ b/tests/ci/rerun_helper.py @@ -1,33 +1,26 @@ #!/usr/bin/env python3 +from typing import List, Optional from commit_status_helper import get_commit +from github import Github +from github.CommitStatus import CommitStatus +from pr_info import PRInfo - -def _filter_statuses(statuses): - """ - Squash statuses to latest state - 1. context="first", state="success", update_time=1 - 2. context="second", state="success", update_time=2 - 3. context="first", stat="failure", update_time=3 - =========> - 1. context="second", state="success" - 2. context="first", stat="failure" - """ - filt = {} - for status in sorted(statuses, key=lambda x: x.updated_at): - filt[status.context] = status - return filt.values() +CommitStatuses = List[CommitStatus] class RerunHelper: - def __init__(self, gh, pr_info, check_name): + def __init__(self, gh: Github, pr_info: PRInfo, check_name: str): self.gh = gh self.pr_info = pr_info self.check_name = check_name - self.pygh_commit = get_commit(gh, self.pr_info.sha) - self.statuses = _filter_statuses(self.pygh_commit.get_statuses()) + commit = get_commit(gh, self.pr_info.sha) + if commit is None: + raise ValueError(f"unable to receive commit for {pr_info.sha}") + self.pygh_commit = commit + self.statuses = self.ger_filtered_statuses() - def is_already_finished_by_status(self): + def is_already_finished_by_status(self) -> bool: # currently we agree even for failed statuses for status in self.statuses: if self.check_name in status.context and status.state in ( @@ -37,8 +30,25 @@ class RerunHelper: return True return False - def get_finished_status(self): + def get_finished_status(self) -> Optional[CommitStatus]: for status in self.statuses: if self.check_name in status.context: return status return None + + def ger_filtered_statuses(self) -> CommitStatuses: + """ + Squash statuses to latest state + 1. context="first", state="success", update_time=1 + 2. context="second", state="success", update_time=2 + 3. context="first", stat="failure", update_time=3 + =========> + 1. context="second", state="success" + 2. context="first", stat="failure" + """ + filt = {} + for status in sorted( + self.pygh_commit.get_statuses(), key=lambda x: x.updated_at + ): + filt[status.context] = status + return list(filt.values()) From d9baa0b1f7f9440f59dd7609ac1fdf772636996b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 21 Jul 2022 18:25:22 +0200 Subject: [PATCH 650/659] Finish following runs with the same exit code as the first --- tests/ci/style_check.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 0d619d28526..7ba0dc2a297 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -153,7 +153,11 @@ if __name__ == "__main__": rerun_helper = RerunHelper(gh, pr_info, NAME) if rerun_helper.is_already_finished_by_status(): logging.info("Check is already finished according to github status, exiting") - sys.exit(0) + # Finish with the same code as previous + state = rerun_helper.get_finished_status().state # type: ignore + # state == "success" -> code = 0 + code = int(state != "success") + sys.exit(code) if not os.path.exists(temp_path): os.makedirs(temp_path) From b9cda3d19283a3a46fa6e2cac34967fb49812c77 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Jul 2022 00:43:15 +0300 Subject: [PATCH 651/659] Update pull_request.yml --- .github/workflows/pull_request.yml | 81 ------------------------------ 1 file changed, 81 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index c2e816b44dc..26726302beb 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -345,51 +345,6 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - # BuilderBinGCC: - # needs: [DockerHubPush, FastTest, StyleCheck] - # runs-on: [self-hosted, builder] - # steps: - # - name: Set envs - # run: | - # cat >> "$GITHUB_ENV" << 'EOF' - # TEMP_PATH=${{runner.temp}}/build_check - # IMAGES_PATH=${{runner.temp}}/images_path - # REPO_COPY=${{runner.temp}}/build_check/ClickHouse - # CACHES_PATH=${{runner.temp}}/../ccaches - # BUILD_NAME=binary_gcc - # EOF - # - name: Download changed images - # uses: actions/download-artifact@v2 - # with: - # name: changed_images - # path: ${{ runner.temp }}/images_path - # - name: Clear repository - # run: | - # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - # - name: Check out repository code - # uses: actions/checkout@v2 - # - name: Build - # run: | - # git -C "$GITHUB_WORKSPACE" submodule sync --recursive - # git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - # sudo rm -fr "$TEMP_PATH" - # mkdir -p "$TEMP_PATH" - # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - # cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - # - name: Upload build URLs to artifacts - # if: ${{ success() || failure() }} - # uses: actions/upload-artifact@v2 - # with: - # name: ${{ env.BUILD_URLS }} - # path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - # - name: Cleanup - # if: always() - # run: | - # # shellcheck disable=SC2046 - # docker kill $(docker ps -q) ||: - # # shellcheck disable=SC2046 - # docker rm -f $(docker ps -a -q) ||: - # sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderDebAarch64: needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] @@ -2974,42 +2929,6 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" - # UnitTestsReleaseGCC: - # needs: [BuilderBinGCC] - # runs-on: [self-hosted, fuzzer-unit-tester] - # steps: - # - name: Set envs - # run: | - # cat >> "$GITHUB_ENV" << 'EOF' - # TEMP_PATH=${{runner.temp}}/unit_tests_asan - # REPORTS_PATH=${{runner.temp}}/reports_dir - # CHECK_NAME=Unit tests (release-gcc) - # REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse - # EOF - # - name: Download json reports - # uses: actions/download-artifact@v2 - # with: - # path: ${{ env.REPORTS_PATH }} - # - name: Clear repository - # run: | - # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - # - name: Check out repository code - # uses: actions/checkout@v2 - # - name: Unit test - # run: | - # sudo rm -fr "$TEMP_PATH" - # mkdir -p "$TEMP_PATH" - # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - # cd "$REPO_COPY/tests/ci" - # python3 unit_tests_check.py "$CHECK_NAME" - # - name: Cleanup - # if: always() - # run: | - # # shellcheck disable=SC2046 - # docker kill $(docker ps -q) ||: - # # shellcheck disable=SC2046 - # docker rm -f $(docker ps -a -q) ||: - # sudo rm -fr "$TEMP_PATH" UnitTestsTsan: needs: [BuilderDebTsan] runs-on: [self-hosted, fuzzer-unit-tester] From 33ce662d3ebb0b393a45155299492d7520aa34dc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 Jul 2022 15:22:00 +0000 Subject: [PATCH 652/659] Replace MemoryTrackerBlockerInThread to LockMemoryExceptionInThread in some places. Reduced MemoryTrackerBlockerInThread level to User. --- src/Common/SystemLogBase.cpp | 2 +- src/Interpreters/executeQuery.cpp | 3 ++- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 18 +----------------- src/Storages/StorageBuffer.cpp | 2 +- 4 files changed, 5 insertions(+), 20 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 67aedbd5670..791f976d5e0 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -79,7 +79,7 @@ void SystemLogBase::add(const LogElement & element) /// The size of allocation can be in order of a few megabytes. /// But this should not be accounted for query memory usage. /// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flacky. - MemoryTrackerBlockerInThread temporarily_disable_memory_tracker(VariableContext::Global); + MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; /// Should not log messages under mutex. bool queue_is_half_full = false; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index ae915aab867..7c0a95e5fcc 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -180,7 +180,8 @@ static void setExceptionStackTrace(QueryLogElement & elem) { /// Disable memory tracker for stack trace. /// Because if exception is "Memory limit (for query) exceed", then we probably can't allocate another one string. - MemoryTrackerBlockerInThread temporarily_disable_memory_tracker(VariableContext::Global); + + LockMemoryExceptionInThread lock(VariableContext::Global); try { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 60941108f00..aa7f16245a2 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -603,22 +603,6 @@ String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize( return *minimum_size_column; } -// String IMergeTreeDataPart::getFullPath() const -// { -// if (relative_path.empty()) -// throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); - -// return fs::path(storage.getFullPathOnDisk(volume->getDisk())) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; -// } - -// String IMergeTreeDataPart::getRelativePath() const -// { -// if (relative_path.empty()) -// throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); - -// return fs::path(storage.relative_data_path) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; -// } - void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency) { assertOnDisk(); @@ -626,7 +610,7 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks /// Memory should not be limited during ATTACH TABLE query. /// This is already true at the server startup but must be also ensured for manual table ATTACH. /// Motivation: memory for index is shared between queries - not belong to the query itself. - MemoryTrackerBlockerInThread temporarily_disable_memory_tracker(VariableContext::Global); + MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; try { diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 85fb20d6571..c14584a382b 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -465,7 +465,7 @@ static void appendBlock(const Block & from, Block & to) /// In case of rollback, it is better to ignore memory limits instead of abnormal server termination. /// So ignore any memory limits, even global (since memory tracking has drift). - MemoryTrackerBlockerInThread temporarily_ignore_any_memory_limits(VariableContext::Global); + LockMemoryExceptionInThread temporarily_ignore_any_memory_limits(VariableContext::Global); try { From 9f053935b639f274e0f80a2ce53dc7991418a723 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 26 Jul 2022 20:49:41 +0300 Subject: [PATCH 653/659] Update 02327_capnproto_protobuf_empty_messages.sh --- .../0_stateless/02327_capnproto_protobuf_empty_messages.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh index 3d41c9bf721..9de01dbe294 100755 --- a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh +++ b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, no-replicated-database +# Tags: no-fasttest, no-parallel, no-replicated-database, no-ordinary-database CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From fe0db1e859f76cf6378dd4ccc97c11650aa602f9 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Tue, 26 Jul 2022 20:57:49 +0200 Subject: [PATCH 654/659] add Dell PowerEdge R740XD results --- website/benchmark/hardware/index.html | 1 + .../benchmark/hardware/results/dell_r740.json | 54 +++++++++++++++++++ 2 files changed, 55 insertions(+) create mode 100644 website/benchmark/hardware/results/dell_r740.json diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index a8ab49c2c3f..1dcb7d84cfd 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -103,6 +103,7 @@ Results for Hetzner EX62-NVME are from Talles Airan.
Results for AMD Ryzen 9 5950X are from Stefan.
Results for ScaleFlux CSD 3000 are from Cliicy Luo of ScaleFlux.
Results for Equinix metal n3.xlarge.x84 are from Dave Cottlehuber. +Results for Dell PowerEdge R740xd are from Yu ZiChange at EioTek.

diff --git a/website/benchmark/hardware/results/dell_r740.json b/website/benchmark/hardware/results/dell_r740.json new file mode 100644 index 00000000000..21242e5400a --- /dev/null +++ b/website/benchmark/hardware/results/dell_r740.json @@ -0,0 +1,54 @@ +[ + { + "system": "Dell PowerEdge R740xd", + "system_full": "Dell PowerEdge R740xd, 256GB, 2 * Intel(R) Xeon(R) Silver 4214R CPU @ 2.40GHz, 48 vCPU", + "time": "2022-06-18 00:00:00", + "kind": "server", + "result": + [ + [0.004, 0.003, 0.070], + [0.086, 0.019, 0.021], + [0.220, 0.038, 0.037], + [0.596, 0.051, 0.050], + [0.189, 0.149, 0.148], + [0.991, 0.233, 0.230], + [0.004, 0.004, 0.004], + [0.022, 0.018, 0.017], + [0.519, 0.315, 0.305], + [0.469, 0.341, 0.337], + [0.252, 0.158, 0.166], + [0.252, 0.201, 0.184], + [0.532, 0.500, 0.479], + [0.642, 0.613, 0.596], + [0.635, 0.506, 0.508], + [0.579, 0.556, 0.560], + [1.587, 1.532, 1.518], + [0.813, 0.752, 0.737], + [3.990, 3.826, 3.737], + [0.114, 0.073, 0.054], + [4.866, 0.513, 0.514], + [3.822, 0.580, 0.569], + [7.784, 1.550, 1.535], + [17.171, 1.168, 0.834], + [0.511, 0.184, 0.185], + [0.190, 0.181, 0.169], + [0.214, 0.182, 0.183], + [4.611, 0.620, 0.616], + [4.234, 0.793, 0.779], + [1.823, 1.767, 1.737], + [0.813, 0.412, 0.371], + [2.306, 0.772, 0.737], + [3.995, 4.061, 4.041], + [5.142, 2.523, 2.562], + [4.803, 2.595, 2.482], + [1.172, 0.982, 0.990], + [0.454, 0.248, 0.275], + [0.200, 0.145, 0.153], + [0.232, 0.135, 0.134], + [0.621, 0.548, 0.478], + [0.130, 0.088, 0.047], + [0.112, 0.044, 0.049], + [0.048, 0.023, 0.011] + ] + } +] From dcb86eca0baee06fc9cdca0723e9075bc4b777b2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 Jul 2022 19:03:26 +0000 Subject: [PATCH 655/659] Make only one possible public ctor in MemoryTrackerBlockerInThread --- src/Common/MemoryTrackerBlockerInThread.cpp | 6 ++++++ src/Common/MemoryTrackerBlockerInThread.h | 8 +++++++- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/src/Common/MemoryTrackerBlockerInThread.cpp b/src/Common/MemoryTrackerBlockerInThread.cpp index 8eb119b2fe5..3d763576c7b 100644 --- a/src/Common/MemoryTrackerBlockerInThread.cpp +++ b/src/Common/MemoryTrackerBlockerInThread.cpp @@ -3,12 +3,18 @@ // MemoryTrackerBlockerInThread thread_local uint64_t MemoryTrackerBlockerInThread::counter = 0; thread_local VariableContext MemoryTrackerBlockerInThread::level = VariableContext::Global; + MemoryTrackerBlockerInThread::MemoryTrackerBlockerInThread(VariableContext level_) : previous_level(level) { ++counter; level = level_; } + +MemoryTrackerBlockerInThread::MemoryTrackerBlockerInThread() : MemoryTrackerBlockerInThread(VariableContext::User) +{ +} + MemoryTrackerBlockerInThread::~MemoryTrackerBlockerInThread() { --counter; diff --git a/src/Common/MemoryTrackerBlockerInThread.h b/src/Common/MemoryTrackerBlockerInThread.h index 381eb80df0c..7711988625f 100644 --- a/src/Common/MemoryTrackerBlockerInThread.h +++ b/src/Common/MemoryTrackerBlockerInThread.h @@ -11,9 +11,13 @@ private: static thread_local VariableContext level; VariableContext previous_level; + + /// level_ - block in level and above + explicit MemoryTrackerBlockerInThread(VariableContext level_); + public: /// level_ - block in level and above - explicit MemoryTrackerBlockerInThread(VariableContext level_ = VariableContext::User); + explicit MemoryTrackerBlockerInThread(); ~MemoryTrackerBlockerInThread(); MemoryTrackerBlockerInThread(const MemoryTrackerBlockerInThread &) = delete; @@ -23,4 +27,6 @@ public: { return counter > 0 && current_level >= level; } + + friend class MemoryTracker; }; From f1818a9103408d919942fe11ae1f75da540012b5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 Jul 2022 19:04:20 +0000 Subject: [PATCH 656/659] Make only one possible public ctor in MemoryTrackerBlockerInThread --- src/Common/MemoryTrackerBlockerInThread.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/MemoryTrackerBlockerInThread.h b/src/Common/MemoryTrackerBlockerInThread.h index 7711988625f..d3882056f54 100644 --- a/src/Common/MemoryTrackerBlockerInThread.h +++ b/src/Common/MemoryTrackerBlockerInThread.h @@ -16,7 +16,6 @@ private: explicit MemoryTrackerBlockerInThread(VariableContext level_); public: - /// level_ - block in level and above explicit MemoryTrackerBlockerInThread(); ~MemoryTrackerBlockerInThread(); From d93c67e3030a9b49e499d0bd8add43ac8d949926 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 26 Jul 2022 15:28:11 -0400 Subject: [PATCH 657/659] comment and test added --- src/Processors/Transforms/FillingTransform.cpp | 1 + tests/queries/0_stateless/02366_with_fill_date.reference | 0 tests/queries/0_stateless/02366_with_fill_date.sql | 6 ++++++ 3 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/02366_with_fill_date.reference create mode 100644 tests/queries/0_stateless/02366_with_fill_date.sql diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 502fb81149c..bd4842b3361 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -54,6 +54,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & WhichDataType which(type); DataTypePtr to_type; + /// For Date/DateTime types TO/FROM type should match column type if (descr.fill_from_type) { WhichDataType which_from(descr.fill_from_type); diff --git a/tests/queries/0_stateless/02366_with_fill_date.reference b/tests/queries/0_stateless/02366_with_fill_date.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02366_with_fill_date.sql b/tests/queries/0_stateless/02366_with_fill_date.sql new file mode 100644 index 00000000000..64e23b845f8 --- /dev/null +++ b/tests/queries/0_stateless/02366_with_fill_date.sql @@ -0,0 +1,6 @@ +-- Tags: no-backward-compatibility-check + +SELECT toDate('2022-02-01') AS d1 +FROM numbers(18) AS number +ORDER BY d1 ASC WITH FILL FROM toDateTime('2022-02-01') TO toDateTime('2022-07-01') STEP toIntervalMonth(1); -- { serverError 475 } + From 0055c9307d606b3144097c78f926db40d2d62116 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Tue, 26 Jul 2022 16:08:03 -0400 Subject: [PATCH 658/659] style fix --- src/Processors/Transforms/FillingTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index bd4842b3361..311c88e46e8 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -54,7 +54,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & WhichDataType which(type); DataTypePtr to_type; - /// For Date/DateTime types TO/FROM type should match column type + /// For Date/DateTime types TO/FROM type should match column type if (descr.fill_from_type) { WhichDataType which_from(descr.fill_from_type); From 2dbb159b53cb046198f9bf975f96f05628cb2d83 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 26 Jul 2022 20:35:45 +0000 Subject: [PATCH 659/659] Add optimize_distinct_in_order in settings randomizer --- tests/clickhouse-test | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index e060535c1ae..952fc7fb0a9 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -447,6 +447,7 @@ class SettingsRandomizer: "compile_aggregate_expressions": lambda: random.randint(0, 1), "compile_sort_description": lambda: random.randint(0, 1), "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), + "optimize_distinct_in_order": lambda: random.randint(0, 1), } @staticmethod