From f91feb0dcb405df80f317f456372c7374f2c75ee Mon Sep 17 00:00:00 2001 From: Daniil Ivanik Date: Tue, 30 Jan 2024 14:17:11 +0100 Subject: [PATCH 001/334] Initial working commit --- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 210 ++++++++++-------- src/Storages/SelectQueryInfo.h | 2 +- src/Storages/System/StorageSystemNumbers.cpp | 8 +- src/Storages/System/StorageSystemNumbers.h | 36 +-- src/Storages/System/attachSystemTables.cpp | 5 +- src/TableFunctions/CMakeLists.txt | 2 +- src/TableFunctions/ITableFunction.cpp | 2 +- src/TableFunctions/TableFunctionNumbers.cpp | 2 +- .../TableFunctionsGenerateSeries.cpp | 100 +++++++++ src/TableFunctions/registerTableFunctions.cpp | 1 + src/TableFunctions/registerTableFunctions.h | 1 + 11 files changed, 234 insertions(+), 135 deletions(-) create mode 100644 src/TableFunctions/TableFunctionsGenerateSeries.cpp diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 5173b18c6bf..13a14ffb917 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -26,41 +26,59 @@ namespace class NumbersSource : public ISource { public: - NumbersSource(UInt64 block_size_, UInt64 offset_, UInt64 step_) - : ISource(createHeader()), block_size(block_size_), next(offset_), step(step_) + NumbersSource(UInt64 block_size_, UInt64 offset_, UInt64 step_, const std::string& column_name, UInt64 inner_step_) + : ISource(createHeader(column_name)), block_size(block_size_), next(offset_), step(step_), inner_step(inner_step_), inner_remainder(offset_ % inner_step_) { } String getName() const override { return "Numbers"; } - static Block createHeader() { return {ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "number")}; } + static Block createHeader(const std::string& column_name) { return {ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), column_name)}; } protected: Chunk generate() override { - auto column = ColumnUInt64::create(block_size); - ColumnUInt64::Container & vec = column->getData(); UInt64 curr = next; /// The local variable for some reason works faster (>20%) than member of class. + UInt64 first_element = (curr / inner_step) * inner_step + inner_remainder; + if (first_element < curr) { + first_element += inner_step; + } + UInt64 filtered_block_size = 0; + if (first_element - curr >= block_size) { + auto column = ColumnUInt64::create(0); + return {Columns{std::move(column)}, filtered_block_size}; + } + if (first_element - curr < block_size) { + filtered_block_size = (block_size - (first_element - curr) - 1) / inner_step + 1; + } + + auto column = ColumnUInt64::create(filtered_block_size); + ColumnUInt64::Container & vec = column->getData(); UInt64 * pos = vec.data(); /// This also accelerates the code. - UInt64 * end = &vec[block_size]; - iota(pos, static_cast(end - pos), curr); + UInt64 * end = &vec[filtered_block_size]; + iota(pos, static_cast(end - pos), UInt64{0}); + for (UInt64 p = 0; p < filtered_block_size; p += 1) { + vec[p] = vec[p] * inner_step + first_element; + } next += step; progress(column->size(), column->byteSize()); - return {Columns{std::move(column)}, block_size}; + return {Columns{std::move(column)}, filtered_block_size}; } private: UInt64 block_size; UInt64 next; UInt64 step; + UInt64 inner_step; + UInt64 inner_remainder; }; -UInt128 sizeOfRange(const Range & r) +[[maybe_unused]] UInt128 sizeOfRange(const Range & r) { UInt128 size; if (r.right.isPositiveInfinity()) @@ -77,7 +95,7 @@ UInt128 sizeOfRange(const Range & r) return size; }; -auto sizeOfRanges(const Ranges & rs) +[[maybe_unused]] auto sizeOfRanges(const Ranges & rs) { UInt128 total_size{}; for (const Range & r : rs) @@ -91,7 +109,7 @@ auto sizeOfRanges(const Ranges & rs) /// Generate numbers according to ranges. /// Numbers generated is ordered in one stream. /// Notice that we will not generate additional numbers out of ranges. -class NumbersRangedSource : public ISource +class [[maybe_unused]] NumbersRangedSource : public ISource { public: /// Represent a position in Ranges list. @@ -109,8 +127,8 @@ public: using RangesStatePtr = std::shared_ptr; - NumbersRangedSource(const Ranges & ranges_, RangesStatePtr & ranges_state_, UInt64 base_block_size_) - : ISource(NumbersSource::createHeader()), ranges(ranges_), ranges_state(ranges_state_), base_block_size(base_block_size_) + [[maybe_unused]] NumbersRangedSource(const Ranges & ranges_, RangesStatePtr & ranges_state_, UInt64 base_block_size_, const std::string& column_name) + : ISource(NumbersSource::createHeader(column_name)), ranges(ranges_), ranges_state(ranges_state_), base_block_size(base_block_size_) { } @@ -273,7 +291,7 @@ private: namespace { /// Whether we should push limit down to scan. -bool shouldPushdownLimit(SelectQueryInfo & query_info, UInt64 limit_length) +[[maybe_unused]] bool shouldPushdownLimit(SelectQueryInfo & query_info, UInt64 limit_length) { const auto & query = query_info.query->as(); /// Just ignore some minor cases, such as: @@ -286,7 +304,7 @@ bool shouldPushdownLimit(SelectQueryInfo & query_info, UInt64 limit_length) /// Shrink ranges to size. /// For example: ranges: [1, 5], [8, 100]; size: 7, we will get [1, 5], [8, 9] -void shrinkRanges(Ranges & ranges, size_t size) +[[maybe_unused]] void shrinkRanges(Ranges & ranges, size_t size) { size_t last_range_idx = 0; for (size_t i = 0; i < ranges.size(); i++) @@ -375,107 +393,107 @@ Pipe ReadFromSystemNumbersStep::makePipe() num_streams = 1; /// Build rpn of query filters - KeyCondition condition(buildFilterDAG(), context, column_names, key_expression); + // KeyCondition condition(buildFilterDAG(), context, column_names, key_expression); Pipe pipe; Ranges ranges; - if (condition.extractPlainRanges(ranges)) - { - /// Intersect ranges with table range - std::optional table_range; - std::optional overflowed_table_range; + // if (condition.extractPlainRanges(ranges)) + // { + // /// Intersect ranges with table range + // std::optional table_range; + // std::optional overflowed_table_range; - if (numbers_storage.limit.has_value()) - { - if (std::numeric_limits::max() - numbers_storage.offset >= *(numbers_storage.limit)) - { - table_range.emplace(FieldRef(numbers_storage.offset), true, FieldRef(numbers_storage.offset + *(numbers_storage.limit)), false); - } - /// UInt64 overflow, for example: SELECT number FROM numbers(18446744073709551614, 5) - else - { - table_range.emplace(FieldRef(numbers_storage.offset), true, std::numeric_limits::max(), true); - auto overflow_end = UInt128(numbers_storage.offset) + UInt128(*numbers_storage.limit); - overflowed_table_range.emplace( - FieldRef(UInt64(0)), true, FieldRef(UInt64(overflow_end - std::numeric_limits::max() - 1)), false); - } - } - else - { - table_range.emplace(FieldRef(numbers_storage.offset), true, FieldRef(std::numeric_limits::max()), true); - } + // if (numbers_storage.limit.has_value()) + // { + // if (std::numeric_limits::max() - numbers_storage.offset >= *(numbers_storage.limit)) + // { + // table_range.emplace(FieldRef(numbers_storage.offset), true, FieldRef(numbers_storage.offset + *(numbers_storage.limit)), false); + // } + // /// UInt64 overflow, for example: SELECT number FROM numbers(18446744073709551614, 5) + // else + // { + // table_range.emplace(FieldRef(numbers_storage.offset), true, std::numeric_limits::max(), true); + // auto overflow_end = UInt128(numbers_storage.offset) + UInt128(*numbers_storage.limit); + // overflowed_table_range.emplace( + // FieldRef(UInt64(0)), true, FieldRef(UInt64(overflow_end - std::numeric_limits::max() - 1)), false); + // } + // } + // else + // { + // table_range.emplace(FieldRef(numbers_storage.offset), true, FieldRef(std::numeric_limits::max()), true); + // } - Ranges intersected_ranges; - for (auto & r : ranges) - { - auto intersected_range = table_range->intersectWith(r); - if (intersected_range) - intersected_ranges.push_back(*intersected_range); - } - /// intersection with overflowed_table_range goes back. - if (overflowed_table_range.has_value()) - { - for (auto & r : ranges) - { - auto intersected_range = overflowed_table_range->intersectWith(r); - if (intersected_range) - intersected_ranges.push_back(*overflowed_table_range); - } - } + // Ranges intersected_ranges; + // for (auto & r : ranges) + // { + // auto intersected_range = table_range->intersectWith(r); + // if (intersected_range) + // intersected_ranges.push_back(*intersected_range); + // } + // /// intersection with overflowed_table_range goes back. + // if (overflowed_table_range.has_value()) + // { + // for (auto & r : ranges) + // { + // auto intersected_range = overflowed_table_range->intersectWith(r); + // if (intersected_range) + // intersected_ranges.push_back(*overflowed_table_range); + // } + // } - /// ranges is blank, return a source who has no data - if (intersected_ranges.empty()) - { - pipe.addSource(std::make_shared(NumbersSource::createHeader())); - return pipe; - } - const auto & limit_length = limit_length_and_offset.first; - const auto & limit_offset = limit_length_and_offset.second; + // /// ranges is blank, return a source who has no data + // if (intersected_ranges.empty()) + // { + // pipe.addSource(std::make_shared(NumbersSource::createHeader(numbers_storage.column_name))); + // return pipe; + // } + // const auto & limit_length = limit_length_and_offset.first; + // const auto & limit_offset = limit_length_and_offset.second; - /// If intersected ranges is limited or we can pushdown limit. - if (!intersected_ranges.rbegin()->right.isPositiveInfinity() || should_pushdown_limit) - { - UInt128 total_size = sizeOfRanges(intersected_ranges); - UInt128 query_limit = limit_length + limit_offset; + // /// If intersected ranges is limited or we can pushdown limit. + // if (!intersected_ranges.rbegin()->right.isPositiveInfinity() || should_pushdown_limit) + // { + // UInt128 total_size = sizeOfRanges(intersected_ranges); + // UInt128 query_limit = limit_length + limit_offset; - /// limit total_size by query_limit - if (should_pushdown_limit && query_limit < total_size) - { - total_size = query_limit; - /// We should shrink intersected_ranges for case: - /// intersected_ranges: [1, 4], [7, 100]; query_limit: 2 - shrinkRanges(intersected_ranges, total_size); - } + // /// limit total_size by query_limit + // if (should_pushdown_limit && query_limit < total_size) + // { + // total_size = query_limit; + // /// We should shrink intersected_ranges for case: + // /// intersected_ranges: [1, 4], [7, 100]; query_limit: 2 + // shrinkRanges(intersected_ranges, total_size); + // } - checkLimits(size_t(total_size)); + // checkLimits(size_t(total_size)); - if (total_size / max_block_size < num_streams) - num_streams = static_cast(total_size / max_block_size); + // if (total_size / max_block_size < num_streams) + // num_streams = static_cast(total_size / max_block_size); - if (num_streams == 0) - num_streams = 1; + // if (num_streams == 0) + // num_streams = 1; - /// Ranges state, all streams will share the state. - auto ranges_state = std::make_shared(); - for (size_t i = 0; i < num_streams; ++i) - { - auto source = std::make_shared(intersected_ranges, ranges_state, max_block_size); + // /// Ranges state, all streams will share the state. + // auto ranges_state = std::make_shared(); + // for (size_t i = 0; i < num_streams; ++i) + // { + // auto source = std::make_shared(intersected_ranges, ranges_state, max_block_size, numbers_storage.column_name); - if (i == 0) - source->addTotalRowsApprox(total_size); + // if (i == 0) + // source->addTotalRowsApprox(total_size); - pipe.addSource(std::move(source)); - } - return pipe; - } - } + // pipe.addSource(std::move(source)); + // } + // return pipe; + // } + // } /// Fall back to NumbersSource for (size_t i = 0; i < num_streams; ++i) { auto source - = std::make_shared(max_block_size, numbers_storage.offset + i * max_block_size, num_streams * max_block_size); + = std::make_shared(max_block_size, numbers_storage.offset + i * max_block_size, num_streams * max_block_size, numbers_storage.column_name, numbers_storage.step); if (numbers_storage.limit && i == 0) { diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 662a5c0ef5a..2b4afaa6345 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -229,4 +229,4 @@ struct SelectQueryInfo bool isFinal() const; }; -} + } diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index b100be7cdf4..cd7207917a9 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -11,15 +11,16 @@ #include #include #include +#include namespace DB { -StorageSystemNumbers::StorageSystemNumbers(const StorageID & table_id, bool multithreaded_, std::optional limit_, UInt64 offset_) - : IStorage(table_id), multithreaded(multithreaded_), limit(limit_), offset(offset_) +StorageSystemNumbers::StorageSystemNumbers(const StorageID & table_id, bool multithreaded_, const std::string& column_name_, std::optional limit_, UInt64 offset_, UInt64 step_) + : IStorage(table_id), multithreaded(multithreaded_), limit(limit_), offset(offset_), column_name(column_name_), step(step_) { StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(ColumnsDescription({{"number", std::make_shared()}})); + storage_metadata.setColumns(ColumnsDescription({{column_name_, std::make_shared()}})); setInMemoryMetadata(storage_metadata); } @@ -33,6 +34,7 @@ void StorageSystemNumbers::read( size_t max_block_size, size_t num_streams) { + // LOG_DEBUG(&Poco::Logger::get("Reading from SystemNumbers"), "Limit : {}", limit.value()); query_plan.addStep(std::make_unique( column_names, shared_from_this(), storage_snapshot, query_info, std::move(context), max_block_size, num_streams)); } diff --git a/src/Storages/System/StorageSystemNumbers.h b/src/Storages/System/StorageSystemNumbers.h index fe6227db406..ffe87b8ad14 100644 --- a/src/Storages/System/StorageSystemNumbers.h +++ b/src/Storages/System/StorageSystemNumbers.h @@ -10,39 +10,11 @@ namespace DB class Context; - -/** Implements a table engine for the system table "numbers". - * The table contains the only column number UInt64. - * From this table, you can read all natural numbers, starting from 0 (to 2^64 - 1, and then again). - * - * You could also specify a limit (how many numbers to give). - * - * How to generate numbers? - * - * 1. First try a smart fashion: - * - * In this fashion we try to push filters and limit down to scanning. - * Firstly extract plain ranges(no overlapping and ordered) by filter expressions. - * - * For example: - * where (numbers > 1 and numbers < 3) or (numbers in (4, 6)) or (numbers > 7 and numbers < 9) - * - * We will get ranges - * (1, 3), [4, 4], [6, 6], (7, 9) - * - * Then split the ranges evenly to one or multi-streams. With this way we will get result without large scanning. - * - * 2. If fail to extract plain ranges, fall back to ordinary scanning. - * - * If multithreaded is specified, numbers will be generated in several streams - * (and result could be out of order). If both multithreaded and limit are specified, - * the table could give you not exactly 1..limit range, but some arbitrary 'limit' numbers. - */ -class StorageSystemNumbers final : public IStorage +class StorageSystemNumbers final : public IStorage { public: /// Otherwise, streams concurrently increment atomic. - StorageSystemNumbers(const StorageID & table_id, bool multithreaded_, std::optional limit_ = std::nullopt, UInt64 offset_ = 0); + StorageSystemNumbers(const StorageID & table_id, bool multithreaded_, const std::string& column_name, std::optional limit_ = std::nullopt, UInt64 offset_ = 0, UInt64 step_ = 1); std::string getName() const override { return "SystemNumbers"; } @@ -67,6 +39,10 @@ private: bool multithreaded; std::optional limit; UInt64 offset; + std::string column_name; + + UInt64 step; + }; } diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index bf898f57833..ddd89709b6a 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -118,8 +118,9 @@ namespace DB void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, bool has_zookeeper) { attach(context, system_database, "one", "This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`."); - attach(context, system_database, "numbers", "Generates all natural numbers, starting from 0 (to 2^64 - 1, and then again) in sorted order.", false); - attach(context, system_database, "numbers_mt", "Multithreaded version of `system.numbers`. Numbers order is not guaranteed.", true); + attach(context, system_database, "numbers", "Generates all natural numbers, starting from 0 (to 2^64 - 1, and then again) in sorted order.", false, "number"); + attach(context, system_database, "numbers_mt", "Multithreaded version of `system.numbers`. Numbers order is not guaranteed.", true, "number"); + // attach(context, system_database, "generate_series", "Multithreaded version of `system.numbers`. Numbers order is not guaranteed.", false, "generate_series"); attach(context, system_database, "zeros", "Produces unlimited number of non-materialized zeros.", false); attach(context, system_database, "zeros_mt", "Multithreaded version of system.zeros.", true); attach(context, system_database, "databases", "Lists all databases of the current server."); diff --git a/src/TableFunctions/CMakeLists.txt b/src/TableFunctions/CMakeLists.txt index 770990cc405..c5c2a660935 100644 --- a/src/TableFunctions/CMakeLists.txt +++ b/src/TableFunctions/CMakeLists.txt @@ -29,7 +29,7 @@ if (TARGET ch_contrib::azure_sdk) target_link_libraries(clickhouse_table_functions PRIVATE ch_contrib::azure_sdk) endif () -if (TARGET ch_contrib::simdjson) +if (TARGET ch_co`trib::simdjson) target_link_libraries(clickhouse_table_functions PRIVATE ch_contrib::simdjson) endif () diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index 137e1dc27fe..c854b6b0f9c 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -5,7 +5,7 @@ #include #include #include - +#include namespace ProfileEvents { diff --git a/src/TableFunctions/TableFunctionNumbers.cpp b/src/TableFunctions/TableFunctionNumbers.cpp index 262018f108c..71a9ba097c6 100644 --- a/src/TableFunctions/TableFunctionNumbers.cpp +++ b/src/TableFunctions/TableFunctionNumbers.cpp @@ -63,7 +63,7 @@ StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_f UInt64 offset = arguments.size() == 2 ? evaluateArgument(context, arguments[0]) : 0; UInt64 length = arguments.size() == 2 ? evaluateArgument(context, arguments[1]) : evaluateArgument(context, arguments[0]); - auto res = std::make_shared(StorageID(getDatabaseName(), table_name), multithreaded, length, offset); + auto res = std::make_shared(StorageID(getDatabaseName(), table_name), multithreaded, std::string{"number"}, length, offset); res->startup(); return res; } diff --git a/src/TableFunctions/TableFunctionsGenerateSeries.cpp b/src/TableFunctions/TableFunctionsGenerateSeries.cpp new file mode 100644 index 00000000000..3941f1eadb2 --- /dev/null +++ b/src/TableFunctions/TableFunctionsGenerateSeries.cpp @@ -0,0 +1,100 @@ +#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 ILLEGAL_TYPE_OF_ARGUMENT; +} + +namespace +{ + +/* numbers(limit), numbers_mt(limit) + * - the same as SELECT number FROM system.numbers LIMIT limit. + * Used for testing purposes, as a simple example of table function. + */ +class TableFunctionGenerateSeries : public ITableFunction +{ +public: + static constexpr auto name = "generate_series"; + std::string getName() const override { return name; } + bool hasStaticStructure() const override { return true; } +private: + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; + const char * getStorageTypeName() const override { return "SystemNumbers"; } + + UInt64 evaluateArgument(ContextPtr context, ASTPtr & argument) const; + + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; +}; + +ColumnsDescription TableFunctionGenerateSeries::getActualTableStructure(ContextPtr /*context*/, bool /*is_insert_query*/) const +{ + /// NOTE: https://bugs.llvm.org/show_bug.cgi?id=47418 + return ColumnsDescription{{{"generate_series", std::make_shared()}}}; +} + +StoragePtr TableFunctionGenerateSeries::executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const +{ + if (const auto * function = ast_function->as()) + { + auto arguments = function->arguments->children; + + if (arguments.size() != 2 && arguments.size() != 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires 'length' or 'offset, length'.", getName()); + + UInt64 start = evaluateArgument(context, arguments[0]); + UInt64 stop = evaluateArgument(context, arguments[1]); + UInt64 interval = (arguments.size() == 3) ? evaluateArgument(context, arguments[2]) : UInt64{1}; + if (start > stop) { + auto res = std::make_shared(StorageID(getDatabaseName(), table_name), false, std::string{"generate_series"}, 0); + res->startup(); + return res; + } + + auto res = std::make_shared(StorageID(getDatabaseName(), table_name), false, std::string{"generate_series"}, (stop - start) / interval + 1, start, interval); + res->startup(); + return res; + } + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires 'limit' or 'offset, limit'.", getName()); +} + +UInt64 TableFunctionGenerateSeries::evaluateArgument(ContextPtr context, ASTPtr & argument) const +{ + const auto & [field, type] = evaluateConstantExpression(argument, context); + + if (!isNativeNumber(type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} expression, must be numeric type", type->getName()); + + Field converted = convertFieldToType(field, DataTypeUInt64()); + if (converted.isNull()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The value {} is not representable as UInt64", + applyVisitor(FieldVisitorToString(), field)); + + return converted.safeGet(); +} + + +} + +void registerTableFunctionGenerateSeries(TableFunctionFactory & factory) +{ + factory.registerFunction({.documentation = {}, .allow_readonly = true}); + // factory.registerFunction({.documentation = {}, .allow_readonly = true}); +} + +} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 8c18c298f45..1631fa8e879 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -11,6 +11,7 @@ void registerTableFunctions() registerTableFunctionMerge(factory); registerTableFunctionRemote(factory); registerTableFunctionNumbers(factory); + registerTableFunctionGenerateSeries(factory); registerTableFunctionNull(factory); registerTableFunctionZeros(factory); registerTableFunctionExecutable(factory); diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index fae763e7dc8..111fbe8c22f 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -8,6 +8,7 @@ class TableFunctionFactory; void registerTableFunctionMerge(TableFunctionFactory & factory); void registerTableFunctionRemote(TableFunctionFactory & factory); void registerTableFunctionNumbers(TableFunctionFactory & factory); +void registerTableFunctionGenerateSeries(TableFunctionFactory & factory); void registerTableFunctionNull(TableFunctionFactory & factory); void registerTableFunctionZeros(TableFunctionFactory & factory); void registerTableFunctionExecutable(TableFunctionFactory & factory); From 3f0cfbd8c0816b007ff85b1a3997696ce5ed3214 Mon Sep 17 00:00:00 2001 From: Daniil Ivanik Date: Sat, 3 Feb 2024 19:46:00 +0100 Subject: [PATCH 002/334] Kek --- src/Common/iota.cpp | 29 ++ src/Common/iota.h | 9 + .../QueryPlan/ReadFromSystemNumbersStep.cpp | 281 ++++++++++-------- 3 files changed, 197 insertions(+), 122 deletions(-) diff --git a/src/Common/iota.cpp b/src/Common/iota.cpp index 98f18eb195b..532c4bde76d 100644 --- a/src/Common/iota.cpp +++ b/src/Common/iota.cpp @@ -27,10 +27,39 @@ void iota(T * begin, size_t count, T first_value) return iotaImpl(begin, count, first_value); } +MULTITARGET_FUNCTION_AVX2_SSE42( + MULTITARGET_FUNCTION_HEADER(template void NO_INLINE), + iotaWithStepImpl, MULTITARGET_FUNCTION_BODY((T * begin, size_t count, T first_value, T step) /// NOLINT + { + for (size_t i = 0; i < count; i++) + *(begin + i) = static_cast(first_value + i * step); + }) +) + +template +void iota_with_step(T * begin, size_t count, T first_value, T step) +{ +#if USE_MULTITARGET_CODE + if (isArchSupported(TargetArch::AVX2)) + return iotaWithStepImplAVX2(begin, count, first_value, step); + + if (isArchSupported(TargetArch::SSE42)) + return iotaWithStepImplSSE42(begin, count, first_value, step); +#endif + return iotaWithStepImpl(begin, count, first_value, step); +} + template void iota(UInt8 * begin, size_t count, UInt8 first_value); template void iota(UInt32 * begin, size_t count, UInt32 first_value); template void iota(UInt64 * begin, size_t count, UInt64 first_value); #if defined(OS_DARWIN) template void iota(size_t * begin, size_t count, size_t first_value); #endif + +template void iota_with_step(UInt8 * begin, size_t count, UInt8 first_value, UInt8 step); +template void iota_with_step(UInt32 * begin, size_t count, UInt32 first_value, UInt32 step); +template void iota_with_step(UInt64 * begin, size_t count, UInt64 first_value, UInt64 step); +#if defined(OS_DARWIN) +extern template void iota_with_step(size_t * begin, size_t count, size_t first_value, size_t step); +#endif } diff --git a/src/Common/iota.h b/src/Common/iota.h index 7910274d15d..f40cde9d5db 100644 --- a/src/Common/iota.h +++ b/src/Common/iota.h @@ -31,4 +31,13 @@ extern template void iota(UInt64 * begin, size_t count, UInt64 first_value); #if defined(OS_DARWIN) extern template void iota(size_t * begin, size_t count, size_t first_value); #endif + +template void iota_with_step(T * begin, size_t count, T first_value, T step); + +extern template void iota_with_step(UInt8 * begin, size_t count, UInt8 first_value, UInt8 step); +extern template void iota_with_step(UInt32 * begin, size_t count, UInt32 first_value, UInt32 step); +extern template void iota_with_step(UInt64 * begin, size_t count, UInt64 first_value, UInt64 step); +#if defined(OS_DARWIN) +extern template void iota(size_t * begin, size_t count, size_t first_value, size_t step); +#endif } diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 13a14ffb917..f85473e43c3 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -12,6 +12,8 @@ #include #include +#include + namespace DB { @@ -30,9 +32,9 @@ public: : ISource(createHeader(column_name)), block_size(block_size_), next(offset_), step(step_), inner_step(inner_step_), inner_remainder(offset_ % inner_step_) { } - String getName() const override { return "Numbers"; } + static Block createHeader(const std::string& column_name) { return {ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), column_name)}; } protected: @@ -57,10 +59,7 @@ protected: ColumnUInt64::Container & vec = column->getData(); UInt64 * pos = vec.data(); /// This also accelerates the code. UInt64 * end = &vec[filtered_block_size]; - iota(pos, static_cast(end - pos), UInt64{0}); - for (UInt64 p = 0; p < filtered_block_size; p += 1) { - vec[p] = vec[p] * inner_step + first_element; - } + iota_with_step(pos, static_cast(end - pos), first_element, inner_step); next += step; @@ -77,28 +76,46 @@ private: UInt64 inner_remainder; }; - -[[maybe_unused]] UInt128 sizeOfRange(const Range & r) -{ - UInt128 size; - if (r.right.isPositiveInfinity()) - return static_cast(std::numeric_limits::max()) - r.left.get() + r.left_included; - - size = static_cast(r.right.get()) - r.left.get() + 1; - - if (!r.left_included) - size--; - - if (!r.right_included) - size--; - assert(size >= 0); - return size; +struct RangeWithStep { + Range range; + UInt64 step; }; -[[maybe_unused]] auto sizeOfRanges(const Ranges & rs) +using RangesWithStep = std::vector; + +std::optional stepped_range_from_range(const Range& r, UInt64 step, UInt64 remainder) { + UInt64 begin = (r.left.get() / step) * step; + if (begin > std::numeric_limits::max() - remainder) { + return std::nullopt; + } + begin += remainder; + while (begin <= r.left.get() - r.left_included) { + if (std::numeric_limits::max() - step < begin) { + return std::nullopt; + } + begin += step; + } + + LOG_DEBUG(&Poco::Logger::get("stepped_range_from_range"), "Begin: {}", begin); + UInt128 right_edge = (r.right.get() + r.right_included); + if (begin >= right_edge) { + return std::nullopt; + } + return std::optional{RangeWithStep{Range(begin, true, static_cast(right_edge - 1), true), step}}; +} + +[[maybe_unused]] UInt128 sizeOfRange(const RangeWithStep & r) +{ + if (r.range.right.isPositiveInfinity()) + return static_cast(std::numeric_limits::max() - r.range.left.get()) / r.step + r.range.left_included; + + return static_cast(r.range.right.get() - r.range.left.get()) / r.step + 1; +}; + +[[maybe_unused]] auto sizeOfRanges(const RangesWithStep & rs) { UInt128 total_size{}; - for (const Range & r : rs) + for (const RangeWithStep & r : rs) { /// total_size will never overflow total_size += sizeOfRange(r); @@ -127,7 +144,7 @@ public: using RangesStatePtr = std::shared_ptr; - [[maybe_unused]] NumbersRangedSource(const Ranges & ranges_, RangesStatePtr & ranges_state_, UInt64 base_block_size_, const std::string& column_name) + [[maybe_unused]] NumbersRangedSource(const RangesWithStep & ranges_, RangesStatePtr & ranges_state_, UInt64 base_block_size_, const std::string& column_name) : ISource(NumbersSource::createHeader(column_name)), ranges(ranges_), ranges_state(ranges_state_), base_block_size(base_block_size_) { } @@ -187,9 +204,9 @@ protected: if (ranges.empty()) return {}; - auto first_value = [](const Range & r) { return r.left.get() + (r.left_included ? 0 : 1); }; + auto first_value = [](const RangeWithStep & r) { return r.range.left.get() + (r.range.left_included ? 0 : 1); }; - auto last_value = [](const Range & r) { return r.right.get() - (r.right_included ? 0 : 1); }; + auto last_value = [](const RangeWithStep & r) { return r.range.right.get() - (r.range.right_included ? 0 : 1); }; /// Find the data range. /// If data left is small, shrink block size. @@ -215,31 +232,33 @@ protected: UInt128 can_provide = cursor.offset_in_ranges == end.offset_in_ranges ? end.offset_in_range - cursor.offset_in_range - : static_cast(last_value(range)) - first_value(range) + 1 - cursor.offset_in_range; + : static_cast(last_value(range) - first_value(range)) / range.step + 1 - cursor.offset_in_range; /// set value to block - auto set_value = [&pos](UInt128 & start_value, UInt128 & end_value) + auto set_value = [&pos, this](UInt128 & start_value, UInt128 & end_value) { if (end_value > std::numeric_limits::max()) { - while (start_value < end_value) - *(pos++) = start_value++; + while (start_value < end_value) { + *(pos++) = start_value; + start_value += this->step; + } } else { auto start_value_64 = static_cast(start_value); auto end_value_64 = static_cast(end_value); auto size = end_value_64 - start_value_64; - iota(pos, static_cast(size), start_value_64); + iota_with_step(pos, static_cast(size), start_value_64, step); pos += size; } }; if (can_provide > need) { - UInt64 start_value = first_value(range) + cursor.offset_in_range; + UInt64 start_value = first_value(range) + cursor.offset_in_range * step; /// end_value will never overflow - iota(pos, static_cast(need), start_value); + iota_with_step(pos, static_cast(need), start_value, step); pos += need; provided += need; @@ -248,8 +267,8 @@ protected: else if (can_provide == need) { /// to avoid UInt64 overflow - UInt128 start_value = static_cast(first_value(range)) + cursor.offset_in_range; - UInt128 end_value = start_value + need; + UInt128 start_value = static_cast(first_value(range)) + cursor.offset_in_range * step; + UInt128 end_value = start_value + need * step; set_value(start_value, end_value); provided += need; @@ -259,8 +278,8 @@ protected: else { /// to avoid UInt64 overflow - UInt128 start_value = static_cast(first_value(range)) + cursor.offset_in_range; - UInt128 end_value = start_value + can_provide; + UInt128 start_value = static_cast(first_value(range)) + cursor.offset_in_range * step; + UInt128 end_value = start_value + can_provide * step; set_value(start_value, end_value); provided += static_cast(can_provide); @@ -277,13 +296,15 @@ protected: private: /// The ranges is shared between all streams. - Ranges ranges; + RangesWithStep ranges; /// Ranges state shared between all streams, actually is the start of the ranges. RangesStatePtr ranges_state; /// Base block size, will shrink when data left is not enough. UInt64 base_block_size; + + UInt64 step; }; } @@ -304,7 +325,7 @@ namespace /// Shrink ranges to size. /// For example: ranges: [1, 5], [8, 100]; size: 7, we will get [1, 5], [8, 9] -[[maybe_unused]] void shrinkRanges(Ranges & ranges, size_t size) +[[maybe_unused]] void shrinkRanges(RangesWithStep & ranges, size_t size) { size_t last_range_idx = 0; for (size_t i = 0; i < ranges.size(); i++) @@ -323,9 +344,9 @@ namespace else { auto & range = ranges[i]; - UInt64 right = range.left.get() + static_cast(size); - range.right = Field(right); - range.right_included = !range.left_included; + UInt64 right = range.range.left.get() + static_cast(size); + range.range.right = Field(right); + range.range.right_included = !range.range.left_included; last_range_idx = i; break; } @@ -393,101 +414,117 @@ Pipe ReadFromSystemNumbersStep::makePipe() num_streams = 1; /// Build rpn of query filters - // KeyCondition condition(buildFilterDAG(), context, column_names, key_expression); + KeyCondition condition(buildFilterDAG(), context, column_names, key_expression); Pipe pipe; Ranges ranges; - // if (condition.extractPlainRanges(ranges)) - // { - // /// Intersect ranges with table range - // std::optional table_range; - // std::optional overflowed_table_range; + if (condition.extractPlainRanges(ranges)) + { + LOG_DEBUG(&Poco::Logger::get("My logger"), "Use optimization"); + /// Intersect ranges with table range + std::optional table_range; + std::optional overflowed_table_range; - // if (numbers_storage.limit.has_value()) - // { - // if (std::numeric_limits::max() - numbers_storage.offset >= *(numbers_storage.limit)) - // { - // table_range.emplace(FieldRef(numbers_storage.offset), true, FieldRef(numbers_storage.offset + *(numbers_storage.limit)), false); - // } - // /// UInt64 overflow, for example: SELECT number FROM numbers(18446744073709551614, 5) - // else - // { - // table_range.emplace(FieldRef(numbers_storage.offset), true, std::numeric_limits::max(), true); - // auto overflow_end = UInt128(numbers_storage.offset) + UInt128(*numbers_storage.limit); - // overflowed_table_range.emplace( - // FieldRef(UInt64(0)), true, FieldRef(UInt64(overflow_end - std::numeric_limits::max() - 1)), false); - // } - // } - // else - // { - // table_range.emplace(FieldRef(numbers_storage.offset), true, FieldRef(std::numeric_limits::max()), true); - // } + if (numbers_storage.limit.has_value()) + { + if (std::numeric_limits::max() - numbers_storage.offset >= *(numbers_storage.limit)) + { + table_range.emplace(FieldRef(numbers_storage.offset), true, FieldRef(numbers_storage.offset + *(numbers_storage.limit)), false); + } + /// UInt64 overflow, for example: SELECT number FROM numbers(18446744073709551614, 5) + else + { + table_range.emplace(FieldRef(numbers_storage.offset), true, std::numeric_limits::max(), true); + auto overflow_end = UInt128(numbers_storage.offset) + UInt128(*numbers_storage.limit); + overflowed_table_range.emplace( + FieldRef(UInt64(0)), true, FieldRef(UInt64(overflow_end - std::numeric_limits::max() - 1)), false); + } + } + else + { + table_range.emplace(FieldRef(numbers_storage.offset), true, FieldRef(std::numeric_limits::max()), true); + } + LOG_DEBUG(&Poco::Logger::get("My logger"), "Found table ranges"); - // Ranges intersected_ranges; - // for (auto & r : ranges) - // { - // auto intersected_range = table_range->intersectWith(r); - // if (intersected_range) - // intersected_ranges.push_back(*intersected_range); - // } - // /// intersection with overflowed_table_range goes back. - // if (overflowed_table_range.has_value()) - // { - // for (auto & r : ranges) - // { - // auto intersected_range = overflowed_table_range->intersectWith(r); - // if (intersected_range) - // intersected_ranges.push_back(*overflowed_table_range); - // } - // } + RangesWithStep intersected_ranges; + for (auto & r : ranges) + { + auto intersected_range = table_range->intersectWith(r); + if (intersected_range.has_value()) { + auto range_with_step = stepped_range_from_range(intersected_range.value(), numbers_storage.step, numbers_storage.offset % numbers_storage.step); + if (range_with_step.has_value()) { + intersected_ranges.push_back(*range_with_step); + } + } + } - // /// ranges is blank, return a source who has no data - // if (intersected_ranges.empty()) - // { - // pipe.addSource(std::make_shared(NumbersSource::createHeader(numbers_storage.column_name))); - // return pipe; - // } - // const auto & limit_length = limit_length_and_offset.first; - // const auto & limit_offset = limit_length_and_offset.second; - // /// If intersected ranges is limited or we can pushdown limit. - // if (!intersected_ranges.rbegin()->right.isPositiveInfinity() || should_pushdown_limit) - // { - // UInt128 total_size = sizeOfRanges(intersected_ranges); - // UInt128 query_limit = limit_length + limit_offset; + for (const auto& range : intersected_ranges) { + LOG_DEBUG(&Poco::Logger::get("Ranges"), "Left: {}; Right {}, LI: {}, RI: {}, Step: {}", range.range.left.get(), range.range.right.get(), range.range.left_included, range.range.right_included, range.step); + // std::cout << + } + /// intersection with overflowed_table_range goes back. + if (overflowed_table_range.has_value()) + { + for (auto & r : ranges) + { + auto intersected_range = overflowed_table_range->intersectWith(r); + if (intersected_range) { + auto range_with_step = stepped_range_from_range(intersected_range.value(), numbers_storage.step, static_cast((static_cast(numbers_storage.offset) + std::numeric_limits::max() + 1) % numbers_storage.step)); + if (range_with_step) { + intersected_ranges.push_back(*range_with_step); + } + } + } + } - // /// limit total_size by query_limit - // if (should_pushdown_limit && query_limit < total_size) - // { - // total_size = query_limit; - // /// We should shrink intersected_ranges for case: - // /// intersected_ranges: [1, 4], [7, 100]; query_limit: 2 - // shrinkRanges(intersected_ranges, total_size); - // } + /// ranges is blank, return a source who has no data + if (intersected_ranges.empty()) + { + pipe.addSource(std::make_shared(NumbersSource::createHeader(numbers_storage.column_name))); + return pipe; + } + const auto & limit_length = limit_length_and_offset.first; + const auto & limit_offset = limit_length_and_offset.second; - // checkLimits(size_t(total_size)); + /// If intersected ranges is limited or we can pushdown limit. + if (!intersected_ranges.rbegin()->range.right.isPositiveInfinity() || should_pushdown_limit) + { + UInt128 total_size = sizeOfRanges(intersected_ranges); + UInt128 query_limit = limit_length + limit_offset; - // if (total_size / max_block_size < num_streams) - // num_streams = static_cast(total_size / max_block_size); + /// limit total_size by query_limit + if (should_pushdown_limit && query_limit < total_size) + { + total_size = query_limit; + /// We should shrink intersected_ranges for case: + /// intersected_ranges: [1, 4], [7, 100]; query_limit: 2 + shrinkRanges(intersected_ranges, total_size); + } - // if (num_streams == 0) - // num_streams = 1; + checkLimits(size_t(total_size)); - // /// Ranges state, all streams will share the state. - // auto ranges_state = std::make_shared(); - // for (size_t i = 0; i < num_streams; ++i) - // { - // auto source = std::make_shared(intersected_ranges, ranges_state, max_block_size, numbers_storage.column_name); + if (total_size / max_block_size < num_streams) + num_streams = static_cast(total_size / max_block_size); - // if (i == 0) - // source->addTotalRowsApprox(total_size); + if (num_streams == 0) + num_streams = 1; - // pipe.addSource(std::move(source)); - // } - // return pipe; - // } - // } + /// Ranges state, all streams will share the state. + auto ranges_state = std::make_shared(); + for (size_t i = 0; i < num_streams; ++i) + { + auto source = std::make_shared(intersected_ranges, ranges_state, max_block_size, numbers_storage.column_name); + + if (i == 0) + source->addTotalRowsApprox(total_size); + + pipe.addSource(std::move(source)); + } + return pipe; + } + } /// Fall back to NumbersSource for (size_t i = 0; i < num_streams; ++i) From 623b42574587073845a76a5e28a502a792ee6662 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 6 Feb 2024 21:34:09 +0000 Subject: [PATCH 003/334] Add feature with the right author name --- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 175 +++++++++++++----- src/Storages/SelectQueryInfo.h | 2 +- src/Storages/System/StorageSystemNumbers.cpp | 8 +- src/Storages/System/StorageSystemNumbers.h | 15 +- src/Storages/System/attachSystemTables.cpp | 2 +- src/TableFunctions/CMakeLists.txt | 2 +- src/TableFunctions/ITableFunction.cpp | 2 +- ...es.cpp => TableFunctionGenerateSeries.cpp} | 55 ++++-- src/TableFunctions/TableFunctionNumbers.cpp | 45 +++-- .../02970_generate_series.reference | 28 +++ .../0_stateless/02970_generate_series.sql | 14 ++ 11 files changed, 254 insertions(+), 94 deletions(-) rename src/TableFunctions/{TableFunctionsGenerateSeries.cpp => TableFunctionGenerateSeries.cpp} (65%) create mode 100644 tests/queries/0_stateless/02970_generate_series.reference create mode 100644 tests/queries/0_stateless/02970_generate_series.sql diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index f85473e43c3..4b957778c43 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -28,32 +28,37 @@ namespace class NumbersSource : public ISource { public: - NumbersSource(UInt64 block_size_, UInt64 offset_, UInt64 step_, const std::string& column_name, UInt64 inner_step_) - : ISource(createHeader(column_name)), block_size(block_size_), next(offset_), step(step_), inner_step(inner_step_), inner_remainder(offset_ % inner_step_) + NumbersSource(UInt64 block_size_, UInt64 offset_, UInt64 step_, const std::string & column_name, UInt64 inner_step_) + : ISource(createHeader(column_name)) + , block_size(block_size_) + , next(offset_) + , step(step_) + , inner_step(inner_step_) + , inner_remainder(offset_ % inner_step_) { } String getName() const override { return "Numbers"; } - - static Block createHeader(const std::string& column_name) { return {ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), column_name)}; } + static Block createHeader(const std::string & column_name) + { + return {ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), column_name)}; + } protected: Chunk generate() override { - UInt64 curr = next; /// The local variable for some reason works faster (>20%) than member of class. UInt64 first_element = (curr / inner_step) * inner_step + inner_remainder; - if (first_element < curr) { + if (first_element < curr) first_element += inner_step; - } UInt64 filtered_block_size = 0; - if (first_element - curr >= block_size) { + if (first_element - curr >= block_size) + { auto column = ColumnUInt64::create(0); return {Columns{std::move(column)}, filtered_block_size}; } - if (first_element - curr < block_size) { + if (first_element - curr < block_size) filtered_block_size = (block_size - (first_element - curr) - 1) / inner_step + 1; - } auto column = ColumnUInt64::create(filtered_block_size); ColumnUInt64::Container & vec = column->getData(); @@ -76,32 +81,37 @@ private: UInt64 inner_remainder; }; -struct RangeWithStep { +struct RangeWithStep +{ Range range; UInt64 step; }; using RangesWithStep = std::vector; -std::optional stepped_range_from_range(const Range& r, UInt64 step, UInt64 remainder) { - UInt64 begin = (r.left.get() / step) * step; - if (begin > std::numeric_limits::max() - remainder) { +std::optional stepped_range_from_range(const Range & r, UInt64 step, UInt64 remainder) +{ + if ((r.right.get() == 0) && (!r.right_included)) + return std::nullopt; + UInt64 begin = (r.left.get() / step) * step; + if (begin > std::numeric_limits::max() - remainder) return std::nullopt; - } begin += remainder; - while (begin <= r.left.get() - r.left_included) { - if (std::numeric_limits::max() - step < begin) { + + // LOG_DEBUG(&Poco::Logger::get("stepped_range_from_range"), "Begin: {}", begin); + // LOG_DEBUG(&Poco::Logger::get("stepped_range_from_range"), "Begin: {}", begin); + while ((r.left_included <= r.left.get()) && (begin <= r.left.get() - r.left_included)) + { + if (std::numeric_limits::max() - step < begin) return std::nullopt; - } begin += step; } - LOG_DEBUG(&Poco::Logger::get("stepped_range_from_range"), "Begin: {}", begin); - UInt128 right_edge = (r.right.get() + r.right_included); - if (begin >= right_edge) { + // LOG_DEBUG(&Poco::Logger::get("stepped_range_from_range"), "Begin: {}", begin); + if ((begin >= r.right_included) && (begin - r.right_included >= r.right.get())) return std::nullopt; - } - return std::optional{RangeWithStep{Range(begin, true, static_cast(right_edge - 1), true), step}}; + UInt64 right_edge_included = r.right.get() - (1 - r.right_included); + return std::optional{RangeWithStep{Range(begin, true, right_edge_included, true), step}}; } [[maybe_unused]] UInt128 sizeOfRange(const RangeWithStep & r) @@ -144,8 +154,17 @@ public: using RangesStatePtr = std::shared_ptr; - [[maybe_unused]] NumbersRangedSource(const RangesWithStep & ranges_, RangesStatePtr & ranges_state_, UInt64 base_block_size_, const std::string& column_name) - : ISource(NumbersSource::createHeader(column_name)), ranges(ranges_), ranges_state(ranges_state_), base_block_size(base_block_size_) + [[maybe_unused]] NumbersRangedSource( + const RangesWithStep & ranges_, + RangesStatePtr & ranges_state_, + UInt64 base_block_size_, + UInt64 step_, + const std::string & column_name) + : ISource(NumbersSource::createHeader(column_name)) + , ranges(ranges_) + , ranges_state(ranges_state_) + , base_block_size(base_block_size_) + , step(step_) { } @@ -158,6 +177,7 @@ protected: { std::lock_guard lock(ranges_state->mutex); + UInt64 need = base_block_size_; UInt64 size = 0; /// how many item found. @@ -196,6 +216,10 @@ protected: } ranges_state->pos = end; + + LOG_DEBUG(&Poco::Logger::get("Range borders"), "Begin: {} {}", start.offset_in_ranges, static_cast(start.offset_in_range)); + LOG_DEBUG(&Poco::Logger::get("Range borders"), "End: {} {}", end.offset_in_ranges, static_cast(end.offset_in_range)); + return size; } @@ -234,12 +258,19 @@ protected: ? end.offset_in_range - cursor.offset_in_range : static_cast(last_value(range) - first_value(range)) / range.step + 1 - cursor.offset_in_range; + LOG_DEBUG( + &Poco::Logger::get("Generate"), + "Can Provide: {}, Block size: {}", + static_cast(can_provide), + static_cast(block_size)); + /// set value to block auto set_value = [&pos, this](UInt128 & start_value, UInt128 & end_value) { if (end_value > std::numeric_limits::max()) { - while (start_value < end_value) { + while (start_value < end_value) + { *(pos++) = start_value; start_value += this->step; } @@ -248,7 +279,9 @@ protected: { auto start_value_64 = static_cast(start_value); auto end_value_64 = static_cast(end_value); - auto size = end_value_64 - start_value_64; + auto size = (end_value_64 - start_value_64) / this->step; + LOG_DEBUG( + &Poco::Logger::get("Iota"), "Size: {}, Step: {}, Start: {}", static_cast(size), this->step, start_value_64); iota_with_step(pos, static_cast(size), start_value_64, step); pos += size; } @@ -374,7 +407,7 @@ ReadFromSystemNumbersStep::ReadFromSystemNumbersStep( , key_expression{KeyDescription::parse(column_names[0], storage_snapshot->metadata->columns, context).expression} , max_block_size{max_block_size_} , num_streams{num_streams_} - , limit_length_and_offset(InterpreterSelectQuery::getLimitLengthAndOffset(query_info.query->as(), context)) + , limit_length_and_offset(InterpreterSelectQuery::getLimitLengthAndOffset(query_info.query->as(), context)) , should_pushdown_limit(shouldPushdownLimit(query_info, limit_length_and_offset.first)) , limit(query_info.limit) , storage_limits(query_info.storage_limits) @@ -410,14 +443,28 @@ Pipe ReadFromSystemNumbersStep::makePipe() { auto & numbers_storage = storage->as(); + LOG_DEBUG( + &Poco::Logger::get("Parameters"), + "Parameters: Limit: {}, Offset: {} Step: {}", + numbers_storage.limit.value(), + numbers_storage.offset, + numbers_storage.step); + if (!numbers_storage.multithreaded) num_streams = 1; + Pipe pipe; + Ranges ranges; + + if (numbers_storage.limit.has_value() && (numbers_storage.limit.value() == 0)) + { + pipe.addSource(std::make_shared(NumbersSource::createHeader(numbers_storage.column_name))); + return pipe; + } + /// Build rpn of query filters KeyCondition condition(buildFilterDAG(), context, column_names, key_expression); - Pipe pipe; - Ranges ranges; if (condition.extractPlainRanges(ranges)) { @@ -430,14 +477,15 @@ Pipe ReadFromSystemNumbersStep::makePipe() { if (std::numeric_limits::max() - numbers_storage.offset >= *(numbers_storage.limit)) { - table_range.emplace(FieldRef(numbers_storage.offset), true, FieldRef(numbers_storage.offset + *(numbers_storage.limit)), false); + table_range.emplace( + FieldRef(numbers_storage.offset), true, FieldRef(numbers_storage.offset + *(numbers_storage.limit)), false); } /// UInt64 overflow, for example: SELECT number FROM numbers(18446744073709551614, 5) else { table_range.emplace(FieldRef(numbers_storage.offset), true, std::numeric_limits::max(), true); auto overflow_end = UInt128(numbers_storage.offset) + UInt128(*numbers_storage.limit); - overflowed_table_range.emplace( + overflowed_table_range.emplace( FieldRef(UInt64(0)), true, FieldRef(UInt64(overflow_end - std::numeric_limits::max() - 1)), false); } } @@ -451,34 +499,59 @@ Pipe ReadFromSystemNumbersStep::makePipe() for (auto & r : ranges) { auto intersected_range = table_range->intersectWith(r); - if (intersected_range.has_value()) { - auto range_with_step = stepped_range_from_range(intersected_range.value(), numbers_storage.step, numbers_storage.offset % numbers_storage.step); - if (range_with_step.has_value()) { + if (intersected_range.has_value()) + { + LOG_DEBUG( + &Poco::Logger::get("Ranges"), + "Ranges: {} {} {} {}", + intersected_range->left.get(), + intersected_range->right.get(), + intersected_range->left_included, + intersected_range->right_included); + auto range_with_step = stepped_range_from_range( + intersected_range.value(), numbers_storage.step, numbers_storage.offset % numbers_storage.step); + if (range_with_step.has_value()) + { + LOG_DEBUG( + &Poco::Logger::get("Ranges With Step"), + "Ranges: {} {} {} {} {}", + range_with_step->range.left.get(), + range_with_step->range.right.get(), + range_with_step->range.left_included, + range_with_step->range.right_included, + range_with_step->step); intersected_ranges.push_back(*range_with_step); } } } - for (const auto& range : intersected_ranges) { - LOG_DEBUG(&Poco::Logger::get("Ranges"), "Left: {}; Right {}, LI: {}, RI: {}, Step: {}", range.range.left.get(), range.range.right.get(), range.range.left_included, range.range.right_included, range.step); - // std::cout << - } /// intersection with overflowed_table_range goes back. if (overflowed_table_range.has_value()) { for (auto & r : ranges) { auto intersected_range = overflowed_table_range->intersectWith(r); - if (intersected_range) { - auto range_with_step = stepped_range_from_range(intersected_range.value(), numbers_storage.step, static_cast((static_cast(numbers_storage.offset) + std::numeric_limits::max() + 1) % numbers_storage.step)); - if (range_with_step) { + if (intersected_range) + { + auto range_with_step = stepped_range_from_range( + intersected_range.value(), + numbers_storage.step, + static_cast( + (static_cast(numbers_storage.offset) + std::numeric_limits::max() + 1) + % numbers_storage.step)); + if (range_with_step) intersected_ranges.push_back(*range_with_step); - } } } } + // for (const auto& range : intersected_ranges) + // { + // LOG_DEBUG(&Poco::Logger::get("Ranges with step"), "Left: {}; Right {}, LI: {}, RI: {}, Step: {}", range.range.left.get(), range.range.right.get(), range.range.left_included, range.range.right_included, range.step); + // // std::cout << + // } + /// ranges is blank, return a source who has no data if (intersected_ranges.empty()) { @@ -492,6 +565,7 @@ Pipe ReadFromSystemNumbersStep::makePipe() if (!intersected_ranges.rbegin()->range.right.isPositiveInfinity() || should_pushdown_limit) { UInt128 total_size = sizeOfRanges(intersected_ranges); + LOG_DEBUG(&Poco::Logger::get("Total_Size"), "Total Size: {}", static_cast(total_size)); UInt128 query_limit = limit_length + limit_offset; /// limit total_size by query_limit @@ -515,7 +589,8 @@ Pipe ReadFromSystemNumbersStep::makePipe() auto ranges_state = std::make_shared(); for (size_t i = 0; i < num_streams; ++i) { - auto source = std::make_shared(intersected_ranges, ranges_state, max_block_size, numbers_storage.column_name); + auto source = std::make_shared( + intersected_ranges, ranges_state, max_block_size, numbers_storage.step, numbers_storage.column_name); if (i == 0) source->addTotalRowsApprox(total_size); @@ -529,12 +604,16 @@ Pipe ReadFromSystemNumbersStep::makePipe() /// Fall back to NumbersSource for (size_t i = 0; i < num_streams; ++i) { - auto source - = std::make_shared(max_block_size, numbers_storage.offset + i * max_block_size, num_streams * max_block_size, numbers_storage.column_name, numbers_storage.step); + auto source = std::make_shared( + max_block_size, + numbers_storage.offset + i * max_block_size, + num_streams * max_block_size, + numbers_storage.column_name, + numbers_storage.step); if (numbers_storage.limit && i == 0) { - auto rows_appr = *(numbers_storage.limit); + auto rows_appr = (*numbers_storage.limit - 1) / numbers_storage.step + 1; if (limit > 0 && limit < rows_appr) rows_appr = limit; source->addTotalRowsApprox(rows_appr); @@ -546,7 +625,7 @@ Pipe ReadFromSystemNumbersStep::makePipe() if (numbers_storage.limit) { size_t i = 0; - auto storage_limit = *(numbers_storage.limit); + auto storage_limit = (*numbers_storage.limit - 1) / numbers_storage.step + 1; /// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly. pipe.addSimpleTransform( [&](const Block & header) diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 2b4afaa6345..662a5c0ef5a 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -229,4 +229,4 @@ struct SelectQueryInfo bool isFinal() const; }; - } +} diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index cd7207917a9..4c319ec7105 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -16,7 +16,13 @@ namespace DB { -StorageSystemNumbers::StorageSystemNumbers(const StorageID & table_id, bool multithreaded_, const std::string& column_name_, std::optional limit_, UInt64 offset_, UInt64 step_) +StorageSystemNumbers::StorageSystemNumbers( + const StorageID & table_id, + bool multithreaded_, + const std::string & column_name_, + std::optional limit_, + UInt64 offset_, + UInt64 step_) : IStorage(table_id), multithreaded(multithreaded_), limit(limit_), offset(offset_), column_name(column_name_), step(step_) { StorageInMemoryMetadata storage_metadata; diff --git a/src/Storages/System/StorageSystemNumbers.h b/src/Storages/System/StorageSystemNumbers.h index ffe87b8ad14..9663ee25251 100644 --- a/src/Storages/System/StorageSystemNumbers.h +++ b/src/Storages/System/StorageSystemNumbers.h @@ -10,11 +10,17 @@ namespace DB class Context; -class StorageSystemNumbers final : public IStorage +class StorageSystemNumbers final : public IStorage { public: /// Otherwise, streams concurrently increment atomic. - StorageSystemNumbers(const StorageID & table_id, bool multithreaded_, const std::string& column_name, std::optional limit_ = std::nullopt, UInt64 offset_ = 0, UInt64 step_ = 1); + StorageSystemNumbers( + const StorageID & table_id, + bool multithreaded_, + const std::string & column_name, + std::optional limit_ = std::nullopt, + UInt64 offset_ = 0, + UInt64 step_ = 1); std::string getName() const override { return "SystemNumbers"; } @@ -30,7 +36,6 @@ public: bool hasEvenlyDistributedRead() const override { return true; } bool isSystemStorage() const override { return true; } - bool supportsTransactions() const override { return true; } private: @@ -38,11 +43,9 @@ private: bool multithreaded; std::optional limit; - UInt64 offset; + UInt64 offset;` std::string column_name; - UInt64 step; - }; } diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index ddd89709b6a..9eacb07bd8d 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -120,7 +120,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "one", "This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`."); attach(context, system_database, "numbers", "Generates all natural numbers, starting from 0 (to 2^64 - 1, and then again) in sorted order.", false, "number"); attach(context, system_database, "numbers_mt", "Multithreaded version of `system.numbers`. Numbers order is not guaranteed.", true, "number"); - // attach(context, system_database, "generate_series", "Multithreaded version of `system.numbers`. Numbers order is not guaranteed.", false, "generate_series"); + attach(context, system_database, "generate_series", "Generates arithmetic progression of natural numbers in sorted order in a given segment with a given step", false, "generate_series"); attach(context, system_database, "zeros", "Produces unlimited number of non-materialized zeros.", false); attach(context, system_database, "zeros_mt", "Multithreaded version of system.zeros.", true); attach(context, system_database, "databases", "Lists all databases of the current server."); diff --git a/src/TableFunctions/CMakeLists.txt b/src/TableFunctions/CMakeLists.txt index c5c2a660935..770990cc405 100644 --- a/src/TableFunctions/CMakeLists.txt +++ b/src/TableFunctions/CMakeLists.txt @@ -29,7 +29,7 @@ if (TARGET ch_contrib::azure_sdk) target_link_libraries(clickhouse_table_functions PRIVATE ch_contrib::azure_sdk) endif () -if (TARGET ch_co`trib::simdjson) +if (TARGET ch_contrib::simdjson) target_link_libraries(clickhouse_table_functions PRIVATE ch_contrib::simdjson) endif () diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index c854b6b0f9c..137e1dc27fe 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -5,7 +5,7 @@ #include #include #include -#include + namespace ProfileEvents { diff --git a/src/TableFunctions/TableFunctionsGenerateSeries.cpp b/src/TableFunctions/TableFunctionGenerateSeries.cpp similarity index 65% rename from src/TableFunctions/TableFunctionsGenerateSeries.cpp rename to src/TableFunctions/TableFunctionGenerateSeries.cpp index 3941f1eadb2..88d7b0d1a71 100644 --- a/src/TableFunctions/TableFunctionsGenerateSeries.cpp +++ b/src/TableFunctions/TableFunctionGenerateSeries.cpp @@ -1,13 +1,13 @@ +#include +#include +#include +#include +#include +#include #include #include -#include -#include #include -#include -#include -#include -#include -#include +#include #include "registerTableFunctions.h" @@ -18,6 +18,7 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int INVALID_SETTING_VALUE; } namespace @@ -33,8 +34,14 @@ public: static constexpr auto name = "generate_series"; std::string getName() const override { return name; } bool hasStaticStructure() const override { return true; } + private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const override; const char * getStorageTypeName() const override { return "SystemNumbers"; } UInt64 evaluateArgument(ContextPtr context, ASTPtr & argument) const; @@ -48,25 +55,31 @@ ColumnsDescription TableFunctionGenerateSeries::getActualTableStructure(ContextP return ColumnsDescription{{{"generate_series", std::make_shared()}}}; } -StoragePtr TableFunctionGenerateSeries::executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const +StoragePtr TableFunctionGenerateSeries::executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription /*cached_columns*/, + bool /*is_insert_query*/) const { if (const auto * function = ast_function->as()) { auto arguments = function->arguments->children; if (arguments.size() != 2 && arguments.size() != 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires 'length' or 'offset, length'.", getName()); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires 'length' or 'offset, length'.", getName()); UInt64 start = evaluateArgument(context, arguments[0]); UInt64 stop = evaluateArgument(context, arguments[1]); - UInt64 interval = (arguments.size() == 3) ? evaluateArgument(context, arguments[2]) : UInt64{1}; - if (start > stop) { - auto res = std::make_shared(StorageID(getDatabaseName(), table_name), false, std::string{"generate_series"}, 0); - res->startup(); - return res; - } - - auto res = std::make_shared(StorageID(getDatabaseName(), table_name), false, std::string{"generate_series"}, (stop - start) / interval + 1, start, interval); + UInt64 step = (arguments.size() == 3) ? evaluateArgument(context, arguments[2]) : UInt64{1}; + if (step == UInt64{0}) + throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Table function '{}' requires step to be a positive number", getName()); + auto res = (start > stop) + ? std::make_shared( + StorageID(getDatabaseName(), table_name), false, std::string{"generate_series"}, 0, 0, 0) + : std::make_shared( + StorageID(getDatabaseName(), table_name), false, std::string{"generate_series"}, (stop - start) + 1, start, step); res->startup(); return res; } @@ -82,8 +95,10 @@ UInt64 TableFunctionGenerateSeries::evaluateArgument(ContextPtr context, ASTPtr Field converted = convertFieldToType(field, DataTypeUInt64()); if (converted.isNull()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The value {} is not representable as UInt64", - applyVisitor(FieldVisitorToString(), field)); + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The value {} is not representable as UInt64", + applyVisitor(FieldVisitorToString(), field)); return converted.safeGet(); } diff --git a/src/TableFunctions/TableFunctionNumbers.cpp b/src/TableFunctions/TableFunctionNumbers.cpp index 71a9ba097c6..bcda8dc6a5e 100644 --- a/src/TableFunctions/TableFunctionNumbers.cpp +++ b/src/TableFunctions/TableFunctionNumbers.cpp @@ -1,13 +1,13 @@ +#include +#include +#include +#include +#include +#include #include #include -#include -#include #include -#include -#include -#include -#include -#include +#include #include "registerTableFunctions.h" @@ -16,8 +16,8 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; } namespace @@ -34,8 +34,14 @@ public: static constexpr auto name = multithreaded ? "numbers_mt" : "numbers"; std::string getName() const override { return name; } bool hasStaticStructure() const override { return true; } + private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const override; const char * getStorageTypeName() const override { return "SystemNumbers"; } UInt64 evaluateArgument(ContextPtr context, ASTPtr & argument) const; @@ -51,19 +57,26 @@ ColumnsDescription TableFunctionNumbers::getActualTableStructure( } template -StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const +StoragePtr TableFunctionNumbers::executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription /*cached_columns*/, + bool /*is_insert_query*/) const { if (const auto * function = ast_function->as()) { auto arguments = function->arguments->children; if (arguments.size() != 1 && arguments.size() != 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires 'length' or 'offset, length'.", getName()); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires 'length' or 'offset, length'.", getName()); UInt64 offset = arguments.size() == 2 ? evaluateArgument(context, arguments[0]) : 0; UInt64 length = arguments.size() == 2 ? evaluateArgument(context, arguments[1]) : evaluateArgument(context, arguments[0]); - auto res = std::make_shared(StorageID(getDatabaseName(), table_name), multithreaded, std::string{"number"}, length, offset); + auto res = std::make_shared( + StorageID(getDatabaseName(), table_name), multithreaded, std::string{"number"}, length, offset); res->startup(); return res; } @@ -80,8 +93,10 @@ UInt64 TableFunctionNumbers::evaluateArgument(ContextPtr context, Field converted = convertFieldToType(field, DataTypeUInt64()); if (converted.isNull()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The value {} is not representable as UInt64", - applyVisitor(FieldVisitorToString(), field)); + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The value {} is not representable as UInt64", + applyVisitor(FieldVisitorToString(), field)); return converted.safeGet(); } diff --git a/tests/queries/0_stateless/02970_generate_series.reference b/tests/queries/0_stateless/02970_generate_series.reference new file mode 100644 index 00000000000..9e6f1db911e --- /dev/null +++ b/tests/queries/0_stateless/02970_generate_series.reference @@ -0,0 +1,28 @@ +0 +1 +4 +8 +500000001 +50000000 +100000001 +0 +10 +13 +16 +19 +7 +17 +27 +37 +47 +57 +67 +77 +17 +22 +27 +32 +37 +42 +47 +52 diff --git a/tests/queries/0_stateless/02970_generate_series.sql b/tests/queries/0_stateless/02970_generate_series.sql new file mode 100644 index 00000000000..045f584a622 --- /dev/null +++ b/tests/queries/0_stateless/02970_generate_series.sql @@ -0,0 +1,14 @@ +SELECT count() FROM generate_series(5, 4); +SELECT count() FROM generate_series(0, 0); +SELECT count() FROM generate_series(10, 20, 3); +SELECT count() FROM generate_series(7, 77, 10); +SELECT count() FROM generate_series(0, 1000000000, 2); +SELECT count() FROM generate_series(0, 999999999, 20); +SELECT count() FROM generate_series(0, 1000000000, 2) WHERE generate_series % 5 == 0; + +SELECT * FROM generate_series(5, 4); +SELECT * FROM generate_series(0, 0); +SELECT * FROM generate_series(10, 20, 3); +SELECT * FROM generate_series(7, 77, 10); +SELECT * FROM generate_series(7, 52, 5) WHERE generate_series >= 13; + From 145e425ddd5707a5852dd3c6ac2672ccbd68e2bd Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 7 Feb 2024 15:29:45 +0000 Subject: [PATCH 004/334] Added Documentation --- .../table-functions/generate_series.md | 25 +++++++++ .../QueryPlan/ReadFromSystemNumbersStep.cpp | 53 ------------------- src/Storages/System/StorageSystemNumbers.h | 2 +- 3 files changed, 26 insertions(+), 54 deletions(-) create mode 100644 docs/en/sql-reference/table-functions/generate_series.md diff --git a/docs/en/sql-reference/table-functions/generate_series.md b/docs/en/sql-reference/table-functions/generate_series.md new file mode 100644 index 00000000000..de34e10ac76 --- /dev/null +++ b/docs/en/sql-reference/table-functions/generate_series.md @@ -0,0 +1,25 @@ +--- +slug: /en/sql-reference/table-functions/generate_series +sidebar_position: ? +sidebar_label: generate_series +--- + +# generate_series + +`generate_series(START, STOP)` - Returns a table with the single ‘generate_series’ column (UInt64) that contains integers from start to stop inclusively. + +`generate_series(START, STOP, STEP)` - Returns a table with the single ‘generate_series’ column (UInt64) that contains integers from start to stop inclusively with spacing between values given by STEP. + +The following queries return tables with the same content but different column names: + +``` sql +SELECT * FROM numbers(10, 5); +SELECT * FROM generate_series(10, 14); +``` + +And the following queries return tables with the same content but different column names (but the second option is more efficient): + +``` sql +SELECT * FROM numbers(10, 11) WHERE number % 3 == (10 % 3); +SELECT * FROM generate_series(10, 20, 3) ; +``` \ No newline at end of file diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 4b957778c43..3bb2e0cd69d 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -98,8 +98,6 @@ std::optional stepped_range_from_range(const Range & r, UInt64 st return std::nullopt; begin += remainder; - // LOG_DEBUG(&Poco::Logger::get("stepped_range_from_range"), "Begin: {}", begin); - // LOG_DEBUG(&Poco::Logger::get("stepped_range_from_range"), "Begin: {}", begin); while ((r.left_included <= r.left.get()) && (begin <= r.left.get() - r.left_included)) { if (std::numeric_limits::max() - step < begin) @@ -107,7 +105,6 @@ std::optional stepped_range_from_range(const Range & r, UInt64 st begin += step; } - // LOG_DEBUG(&Poco::Logger::get("stepped_range_from_range"), "Begin: {}", begin); if ((begin >= r.right_included) && (begin - r.right_included >= r.right.get())) return std::nullopt; UInt64 right_edge_included = r.right.get() - (1 - r.right_included); @@ -217,9 +214,6 @@ protected: ranges_state->pos = end; - LOG_DEBUG(&Poco::Logger::get("Range borders"), "Begin: {} {}", start.offset_in_ranges, static_cast(start.offset_in_range)); - LOG_DEBUG(&Poco::Logger::get("Range borders"), "End: {} {}", end.offset_in_ranges, static_cast(end.offset_in_range)); - return size; } @@ -258,12 +252,6 @@ protected: ? end.offset_in_range - cursor.offset_in_range : static_cast(last_value(range) - first_value(range)) / range.step + 1 - cursor.offset_in_range; - LOG_DEBUG( - &Poco::Logger::get("Generate"), - "Can Provide: {}, Block size: {}", - static_cast(can_provide), - static_cast(block_size)); - /// set value to block auto set_value = [&pos, this](UInt128 & start_value, UInt128 & end_value) { @@ -280,8 +268,6 @@ protected: auto start_value_64 = static_cast(start_value); auto end_value_64 = static_cast(end_value); auto size = (end_value_64 - start_value_64) / this->step; - LOG_DEBUG( - &Poco::Logger::get("Iota"), "Size: {}, Step: {}, Start: {}", static_cast(size), this->step, start_value_64); iota_with_step(pos, static_cast(size), start_value_64, step); pos += size; } @@ -443,13 +429,6 @@ Pipe ReadFromSystemNumbersStep::makePipe() { auto & numbers_storage = storage->as(); - LOG_DEBUG( - &Poco::Logger::get("Parameters"), - "Parameters: Limit: {}, Offset: {} Step: {}", - numbers_storage.limit.value(), - numbers_storage.offset, - numbers_storage.step); - if (!numbers_storage.multithreaded) num_streams = 1; @@ -468,7 +447,6 @@ Pipe ReadFromSystemNumbersStep::makePipe() if (condition.extractPlainRanges(ranges)) { - LOG_DEBUG(&Poco::Logger::get("My logger"), "Use optimization"); /// Intersect ranges with table range std::optional table_range; std::optional overflowed_table_range; @@ -493,36 +471,11 @@ Pipe ReadFromSystemNumbersStep::makePipe() { table_range.emplace(FieldRef(numbers_storage.offset), true, FieldRef(std::numeric_limits::max()), true); } - LOG_DEBUG(&Poco::Logger::get("My logger"), "Found table ranges"); RangesWithStep intersected_ranges; for (auto & r : ranges) { auto intersected_range = table_range->intersectWith(r); - if (intersected_range.has_value()) - { - LOG_DEBUG( - &Poco::Logger::get("Ranges"), - "Ranges: {} {} {} {}", - intersected_range->left.get(), - intersected_range->right.get(), - intersected_range->left_included, - intersected_range->right_included); - auto range_with_step = stepped_range_from_range( - intersected_range.value(), numbers_storage.step, numbers_storage.offset % numbers_storage.step); - if (range_with_step.has_value()) - { - LOG_DEBUG( - &Poco::Logger::get("Ranges With Step"), - "Ranges: {} {} {} {} {}", - range_with_step->range.left.get(), - range_with_step->range.right.get(), - range_with_step->range.left_included, - range_with_step->range.right_included, - range_with_step->step); - intersected_ranges.push_back(*range_with_step); - } - } } @@ -546,11 +499,6 @@ Pipe ReadFromSystemNumbersStep::makePipe() } } - // for (const auto& range : intersected_ranges) - // { - // LOG_DEBUG(&Poco::Logger::get("Ranges with step"), "Left: {}; Right {}, LI: {}, RI: {}, Step: {}", range.range.left.get(), range.range.right.get(), range.range.left_included, range.range.right_included, range.step); - // // std::cout << - // } /// ranges is blank, return a source who has no data if (intersected_ranges.empty()) @@ -565,7 +513,6 @@ Pipe ReadFromSystemNumbersStep::makePipe() if (!intersected_ranges.rbegin()->range.right.isPositiveInfinity() || should_pushdown_limit) { UInt128 total_size = sizeOfRanges(intersected_ranges); - LOG_DEBUG(&Poco::Logger::get("Total_Size"), "Total Size: {}", static_cast(total_size)); UInt128 query_limit = limit_length + limit_offset; /// limit total_size by query_limit diff --git a/src/Storages/System/StorageSystemNumbers.h b/src/Storages/System/StorageSystemNumbers.h index 9663ee25251..298721984b8 100644 --- a/src/Storages/System/StorageSystemNumbers.h +++ b/src/Storages/System/StorageSystemNumbers.h @@ -43,7 +43,7 @@ private: bool multithreaded; std::optional limit; - UInt64 offset;` + UInt64 offset; std::string column_name; UInt64 step; }; From 03aaedace439f5db6d9a6aaf91a1b2f978b0f6a9 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 9 Feb 2024 12:05:01 +0000 Subject: [PATCH 005/334] Fix bug --- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 3bb2e0cd69d..bc14547889b 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -435,6 +435,8 @@ Pipe ReadFromSystemNumbersStep::makePipe() Pipe pipe; Ranges ranges; + // LOG_DEBUG(&Poco::Logger::get("parameters"), "Parameters: {} {} {}", numbers_storage.step, numbers_storage.limit.value(), numbers_storage.offset); + if (numbers_storage.limit.has_value() && (numbers_storage.limit.value() == 0)) { pipe.addSource(std::make_shared(NumbersSource::createHeader(numbers_storage.column_name))); @@ -476,6 +478,15 @@ Pipe ReadFromSystemNumbersStep::makePipe() for (auto & r : ranges) { auto intersected_range = table_range->intersectWith(r); + if (intersected_range.has_value()) + { + auto range_with_step = stepped_range_from_range( + intersected_range.value(), numbers_storage.step, numbers_storage.offset % numbers_storage.step); + if (range_with_step.has_value()) + { + intersected_ranges.push_back(*range_with_step); + } + } } From 1b2f23247b7f115ba92b9908d224d4e78e8649f4 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 9 Feb 2024 12:28:54 +0000 Subject: [PATCH 006/334] Change documentation --- docs/en/sql-reference/table-functions/generate_series.md | 2 +- src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/en/sql-reference/table-functions/generate_series.md b/docs/en/sql-reference/table-functions/generate_series.md index de34e10ac76..c5d29369627 100644 --- a/docs/en/sql-reference/table-functions/generate_series.md +++ b/docs/en/sql-reference/table-functions/generate_series.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/table-functions/generate_series -sidebar_position: ? +sidebar_position: 146 sidebar_label: generate_series --- diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index bc14547889b..ab2f726aeb5 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -483,9 +483,7 @@ Pipe ReadFromSystemNumbersStep::makePipe() auto range_with_step = stepped_range_from_range( intersected_range.value(), numbers_storage.step, numbers_storage.offset % numbers_storage.step); if (range_with_step.has_value()) - { intersected_ranges.push_back(*range_with_step); - } } } From 79f91003538a71014eb035dca024285f2fbba7d5 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 9 Feb 2024 14:17:25 +0000 Subject: [PATCH 007/334] To pull --- src/TableFunctions/TableFunctionGenerateSeries.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionGenerateSeries.cpp b/src/TableFunctions/TableFunctionGenerateSeries.cpp index 88d7b0d1a71..65c4c4915c2 100644 --- a/src/TableFunctions/TableFunctionGenerateSeries.cpp +++ b/src/TableFunctions/TableFunctionGenerateSeries.cpp @@ -77,7 +77,7 @@ StoragePtr TableFunctionGenerateSeries::executeImpl( throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Table function '{}' requires step to be a positive number", getName()); auto res = (start > stop) ? std::make_shared( - StorageID(getDatabaseName(), table_name), false, std::string{"generate_series"}, 0, 0, 0) + StorageID(getDatabaseName(), table_name), false, std::string{"generate_series"}, 0, 0, 1) : std::make_shared( StorageID(getDatabaseName(), table_name), false, std::string{"generate_series"}, (stop - start) + 1, start, step); res->startup(); From f7dbcdd7e7e00d4fb6d30a02ebcb4a3befcd3190 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 9 Feb 2024 18:12:24 +0000 Subject: [PATCH 008/334] Made refactoring --- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 41 ++++++++++++------- src/Storages/System/StorageSystemNumbers.h | 28 +++++++++++++ 2 files changed, 54 insertions(+), 15 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 3a905a56aa1..2488fa37643 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -9,10 +9,12 @@ #include #include #include +#include #include #include #include +#include "base/types.h" namespace DB { @@ -28,13 +30,13 @@ namespace class NumbersSource : public ISource { public: - NumbersSource(UInt64 block_size_, UInt64 offset_, UInt64 step_, const std::string & column_name, UInt64 inner_step_) + NumbersSource(UInt64 block_size_, UInt64 offset_, UInt64 chunk_step_, const std::string & column_name, UInt64 step_, UInt64 remainder_) : ISource(createHeader(column_name)) , block_size(block_size_) , next(offset_) + , chunk_step(chunk_step_) , step(step_) - , inner_step(inner_step_) - , inner_remainder(offset_ % inner_step_) + , remainder(remainder_) { } String getName() const override { return "Numbers"; } @@ -48,25 +50,33 @@ protected: Chunk generate() override { UInt64 curr = next; /// The local variable for some reason works faster (>20%) than member of class. - UInt64 first_element = (curr / inner_step) * inner_step + inner_remainder; - if (first_element < curr) - first_element += inner_step; - UInt64 filtered_block_size = 0; + UInt64 first_element = (curr / step) * step; + if (first_element > std::numeric_limits::max() - remainder) { + auto column = ColumnUInt64::create(0); + return {Columns{std::move(column)}, 0}; + } + first_element += remainder; + if (first_element < curr) { + if (first_element > std::numeric_limits::max() - step) { + auto column = ColumnUInt64::create(0); + return {Columns{std::move(column)}, 0}; + } + first_element += step; + } if (first_element - curr >= block_size) { auto column = ColumnUInt64::create(0); - return {Columns{std::move(column)}, filtered_block_size}; + return {Columns{std::move(column)}, 0}; } - if (first_element - curr < block_size) - filtered_block_size = (block_size - (first_element - curr) - 1) / inner_step + 1; + UInt64 filtered_block_size = (block_size - (first_element - curr) - 1) / step + 1; auto column = ColumnUInt64::create(filtered_block_size); ColumnUInt64::Container & vec = column->getData(); UInt64 * pos = vec.data(); /// This also accelerates the code. UInt64 * end = &vec[filtered_block_size]; - iota_with_step(pos, static_cast(end - pos), first_element, inner_step); + iota_with_step(pos, static_cast(end - pos), first_element, step); - next += step; + next += chunk_step; progress(column->size(), column->byteSize()); @@ -76,9 +86,9 @@ protected: private: UInt64 block_size; UInt64 next; + UInt64 chunk_step; UInt64 step; - UInt64 inner_step; - UInt64 inner_remainder; + UInt64 remainder; }; struct RangeWithStep @@ -565,7 +575,8 @@ Pipe ReadFromSystemNumbersStep::makePipe() numbers_storage.offset + i * max_block_size, num_streams * max_block_size, numbers_storage.column_name, - numbers_storage.step); + numbers_storage.step, + numbers_storage.offset % numbers_storage.step); if (numbers_storage.limit && i == 0) { diff --git a/src/Storages/System/StorageSystemNumbers.h b/src/Storages/System/StorageSystemNumbers.h index 298721984b8..c698bae4393 100644 --- a/src/Storages/System/StorageSystemNumbers.h +++ b/src/Storages/System/StorageSystemNumbers.h @@ -10,6 +10,34 @@ namespace DB class Context; +/** Implements a table engine for the system table "numbers". + * The table contains the only column number UInt64. + * From this table, you can read all natural numbers, starting from 0 (to 2^64 - 1, and then again). + * + * You could also specify a limit (how many numbers to give). + * + * How to generate numbers? + * + * 1. First try a smart fashion: + * + * In this fashion we try to push filters and limit down to scanning. + * Firstly extract plain ranges(no overlapping and ordered) by filter expressions. + * + * For example: + * where (numbers > 1 and numbers < 3) or (numbers in (4, 6)) or (numbers > 7 and numbers < 9) + * + * We will get ranges + * (1, 3), [4, 4], [6, 6], (7, 9) + * + * Then split the ranges evenly to one or multi-streams. With this way we will get result without large scanning. + * + * 2. If fail to extract plain ranges, fall back to ordinary scanning. + * + * If multithreaded is specified, numbers will be generated in several streams + * (and result could be out of order). If both multithreaded and limit are specified, + * the table could give you not exactly 1..limit range, but some arbitrary 'limit' numbers. + */ + class StorageSystemNumbers final : public IStorage { public: From 696609e7d562d15cfc7a6ffa776785444a97c2e7 Mon Sep 17 00:00:00 2001 From: divanik Date: Sat, 10 Feb 2024 19:59:57 +0000 Subject: [PATCH 009/334] Kek --- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 42 ++++++++++++++++--- 1 file changed, 36 insertions(+), 6 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 2488fa37643..dc6aebc69c1 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -14,6 +14,7 @@ #include #include +#include "Core/Types.h" #include "base/types.h" namespace DB @@ -51,13 +52,16 @@ protected: { UInt64 curr = next; /// The local variable for some reason works faster (>20%) than member of class. UInt64 first_element = (curr / step) * step; - if (first_element > std::numeric_limits::max() - remainder) { + if (first_element > std::numeric_limits::max() - remainder) + { auto column = ColumnUInt64::create(0); return {Columns{std::move(column)}, 0}; } first_element += remainder; - if (first_element < curr) { - if (first_element > std::numeric_limits::max() - step) { + if (first_element < curr) + { + if (first_element > std::numeric_limits::max() - step) + { auto column = ColumnUInt64::create(0); return {Columns{std::move(column)}, 0}; } @@ -101,6 +105,8 @@ using RangesWithStep = std::vector; std::optional stepped_range_from_range(const Range & r, UInt64 step, UInt64 remainder) { + // LOG_DEBUG(&Poco::Logger::get("Stepped from range"), + // "stepped from range"); if ((r.right.get() == 0) && (!r.right_included)) return std::nullopt; UInt64 begin = (r.left.get() / step) * step; @@ -126,7 +132,11 @@ std::optional stepped_range_from_range(const Range & r, UInt64 st if (r.range.right.isPositiveInfinity()) return static_cast(std::numeric_limits::max() - r.range.left.get()) / r.step + r.range.left_included; - return static_cast(r.range.right.get() - r.range.left.get()) / r.step + 1; + UInt128 size = static_cast(r.range.right.get() - r.range.left.get()) / r.step; + if (r.range.right_included && (r.range.right.get() % r.step == 0)) { + ++size; + } + return size; }; [[maybe_unused]] auto sizeOfRanges(const RangesWithStep & rs) @@ -173,6 +183,17 @@ public: , base_block_size(base_block_size_) , step(step_) { + // for (const auto& range_with_step : ranges_) { + // // LOG_DEBUG(&Poco::Logger::get("Ranges With Step"), + // // "Ranges: {} {} {} {} {}", + // // range_with_step.range.left.get(), + // // range_with_step.range.right.get(), + // // range_with_step.range.left_included, + // // range_with_step.range.right_included, + // // range_with_step.step); + // // LOG_DEBUG(&Poco::Logger::get("Ranges With Step"), + // // "Step: {}", step); + // } } String getName() const override { return "NumbersRange"; } @@ -241,6 +262,8 @@ protected: RangesPos start, end; auto block_size = findRanges(start, end, base_block_size); + // LOG_DEBUG(&Poco::Logger::get("Found range"), "Evth: {} {} {} {} {} {}", start.offset_in_ranges, static_cast(start.offset_in_range), end.offset_in_ranges, static_cast(end.offset_in_range), base_block_size, block_size); + if (!block_size) return {}; @@ -256,6 +279,11 @@ protected: while (block_size - provided != 0) { UInt64 need = block_size - provided; + // LOG_DEBUG(&Poco::Logger::get("Indices:"), + // "Indices: {} {}, provided: {}", + // ranges.size(), + // cursor.offset_in_ranges, + // provided); auto & range = ranges[cursor.offset_in_ranges]; UInt128 can_provide = cursor.offset_in_ranges == end.offset_in_ranges @@ -445,13 +473,15 @@ Pipe ReadFromSystemNumbersStep::makePipe() Pipe pipe; Ranges ranges; - // LOG_DEBUG(&Poco::Logger::get("parameters"), "Parameters: {} {} {}", numbers_storage.step, numbers_storage.limit.value(), numbers_storage.offset); + + // LOG_DEBUG(&Poco::Logger::get("parameters"), "Parameters: {} {} {} {}", numbers_storage.step, numbers_storage.offset, numbers_storage.limit.has_value(), numbers_storage.limit.has_value() ? numbers_storage.limit.value() : UInt64{0}); if (numbers_storage.limit.has_value() && (numbers_storage.limit.value() == 0)) { pipe.addSource(std::make_shared(NumbersSource::createHeader(numbers_storage.column_name))); return pipe; } + chassert(numbers_storage.step != UInt64{0}); /// Build rpn of query filters KeyCondition condition(buildFilterDAG(), context, column_names, key_expression); @@ -575,7 +605,7 @@ Pipe ReadFromSystemNumbersStep::makePipe() numbers_storage.offset + i * max_block_size, num_streams * max_block_size, numbers_storage.column_name, - numbers_storage.step, + numbers_storage.step, numbers_storage.offset % numbers_storage.step); if (numbers_storage.limit && i == 0) From 3ec9f3c4c89dec2f1971979d7d3ae406c1ecd938 Mon Sep 17 00:00:00 2001 From: divanik Date: Sat, 10 Feb 2024 20:06:52 +0000 Subject: [PATCH 010/334] Check foormattign --- src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index dc6aebc69c1..1e6b539ee2e 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -52,15 +52,15 @@ protected: { UInt64 curr = next; /// The local variable for some reason works faster (>20%) than member of class. UInt64 first_element = (curr / step) * step; - if (first_element > std::numeric_limits::max() - remainder) + if (first_element > std::numeric_limits::max() - remainder) { auto column = ColumnUInt64::create(0); return {Columns{std::move(column)}, 0}; } first_element += remainder; - if (first_element < curr) + if (first_element < curr) { - if (first_element > std::numeric_limits::max() - step) + if (first_element > std::numeric_limits::max() - step) { auto column = ColumnUInt64::create(0); return {Columns{std::move(column)}, 0}; @@ -133,9 +133,8 @@ std::optional stepped_range_from_range(const Range & r, UInt64 st return static_cast(std::numeric_limits::max() - r.range.left.get()) / r.step + r.range.left_included; UInt128 size = static_cast(r.range.right.get() - r.range.left.get()) / r.step; - if (r.range.right_included && (r.range.right.get() % r.step == 0)) { + if (r.range.right_included && (r.range.right.get() % r.step == 0)) ++size; - } return size; }; From d0456980991c45935fd316ca7dc2bd61cf45e5b9 Mon Sep 17 00:00:00 2001 From: divanik Date: Sat, 10 Feb 2024 23:04:52 +0000 Subject: [PATCH 011/334] It seems to work --- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 31 ++++++------------- ...ble_functions_must_be_documented.reference | 1 + 2 files changed, 11 insertions(+), 21 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 1e6b539ee2e..2217e426b02 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -97,7 +97,8 @@ private: struct RangeWithStep { - Range range; + UInt64 left; + UInt64 right; UInt64 step; }; @@ -124,18 +125,12 @@ std::optional stepped_range_from_range(const Range & r, UInt64 st if ((begin >= r.right_included) && (begin - r.right_included >= r.right.get())) return std::nullopt; UInt64 right_edge_included = r.right.get() - (1 - r.right_included); - return std::optional{RangeWithStep{Range(begin, true, right_edge_included, true), step}}; + return std::optional{RangeWithStep{begin, right_edge_included, step}}; } [[maybe_unused]] UInt128 sizeOfRange(const RangeWithStep & r) { - if (r.range.right.isPositiveInfinity()) - return static_cast(std::numeric_limits::max() - r.range.left.get()) / r.step + r.range.left_included; - - UInt128 size = static_cast(r.range.right.get() - r.range.left.get()) / r.step; - if (r.range.right_included && (r.range.right.get() % r.step == 0)) - ++size; - return size; + return static_cast(r.right - r.left) / r.step + 1; }; [[maybe_unused]] auto sizeOfRanges(const RangesWithStep & rs) @@ -252,10 +247,6 @@ protected: if (ranges.empty()) return {}; - auto first_value = [](const RangeWithStep & r) { return r.range.left.get() + (r.range.left_included ? 0 : 1); }; - - auto last_value = [](const RangeWithStep & r) { return r.range.right.get() - (r.range.right_included ? 0 : 1); }; - /// Find the data range. /// If data left is small, shrink block size. RangesPos start, end; @@ -287,7 +278,7 @@ protected: UInt128 can_provide = cursor.offset_in_ranges == end.offset_in_ranges ? end.offset_in_range - cursor.offset_in_range - : static_cast(last_value(range) - first_value(range)) / range.step + 1 - cursor.offset_in_range; + : static_cast(range.right - range.left) / range.step + 1 - cursor.offset_in_range; /// set value to block auto set_value = [&pos, this](UInt128 & start_value, UInt128 & end_value) @@ -312,7 +303,7 @@ protected: if (can_provide > need) { - UInt64 start_value = first_value(range) + cursor.offset_in_range * step; + UInt64 start_value = range.left + cursor.offset_in_range * step; /// end_value will never overflow iota_with_step(pos, static_cast(need), start_value, step); pos += need; @@ -323,7 +314,7 @@ protected: else if (can_provide == need) { /// to avoid UInt64 overflow - UInt128 start_value = static_cast(first_value(range)) + cursor.offset_in_range * step; + UInt128 start_value = static_cast(range.left) + cursor.offset_in_range * step; UInt128 end_value = start_value + need * step; set_value(start_value, end_value); @@ -334,7 +325,7 @@ protected: else { /// to avoid UInt64 overflow - UInt128 start_value = static_cast(first_value(range)) + cursor.offset_in_range * step; + UInt128 start_value = static_cast(range.left) + cursor.offset_in_range * step; UInt128 end_value = start_value + can_provide * step; set_value(start_value, end_value); @@ -400,9 +391,7 @@ namespace else { auto & range = ranges[i]; - UInt64 right = range.range.left.get() + static_cast(size); - range.range.right = Field(right); - range.range.right_included = !range.range.left_included; + range.right = range.left + static_cast(size) * range.step - 1; last_range_idx = i; break; } @@ -558,7 +547,7 @@ Pipe ReadFromSystemNumbersStep::makePipe() const auto & limit_offset = limit_length_and_offset.second; /// If intersected ranges is limited or we can pushdown limit. - if (!intersected_ranges.rbegin()->range.right.isPositiveInfinity() || should_pushdown_limit) + if (should_pushdown_limit) { UInt128 total_size = sizeOfRanges(intersected_ranges); UInt128 query_limit = limit_length + limit_offset; diff --git a/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.reference b/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.reference index e4040a2d371..1e4f21a6722 100644 --- a/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.reference @@ -4,6 +4,7 @@ dictionary executable file generateRandom +generate_series input jdbc merge From 789d3c699c77d7a39f42281d9dc0c61010471242 Mon Sep 17 00:00:00 2001 From: divanik Date: Sat, 10 Feb 2024 23:49:58 +0000 Subject: [PATCH 012/334] Remove bug for mt --- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 65 +++++++++---------- 1 file changed, 31 insertions(+), 34 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 2217e426b02..3656a6d31ee 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -547,42 +547,39 @@ Pipe ReadFromSystemNumbersStep::makePipe() const auto & limit_offset = limit_length_and_offset.second; /// If intersected ranges is limited or we can pushdown limit. - if (should_pushdown_limit) + UInt128 total_size = sizeOfRanges(intersected_ranges); + UInt128 query_limit = limit_length + limit_offset; + + /// limit total_size by query_limit + if (should_pushdown_limit && query_limit < total_size) { - UInt128 total_size = sizeOfRanges(intersected_ranges); - UInt128 query_limit = limit_length + limit_offset; - - /// limit total_size by query_limit - if (should_pushdown_limit && query_limit < total_size) - { - total_size = query_limit; - /// We should shrink intersected_ranges for case: - /// intersected_ranges: [1, 4], [7, 100]; query_limit: 2 - shrinkRanges(intersected_ranges, total_size); - } - - checkLimits(size_t(total_size)); - - if (total_size / max_block_size < num_streams) - num_streams = static_cast(total_size / max_block_size); - - if (num_streams == 0) - num_streams = 1; - - /// Ranges state, all streams will share the state. - auto ranges_state = std::make_shared(); - for (size_t i = 0; i < num_streams; ++i) - { - auto source = std::make_shared( - intersected_ranges, ranges_state, max_block_size, numbers_storage.step, numbers_storage.column_name); - - if (i == 0) - source->addTotalRowsApprox(total_size); - - pipe.addSource(std::move(source)); - } - return pipe; + total_size = query_limit; + /// We should shrink intersected_ranges for case: + /// intersected_ranges: [1, 4], [7, 100]; query_limit: 2 + shrinkRanges(intersected_ranges, total_size); } + + checkLimits(size_t(total_size)); + + if (total_size / max_block_size < num_streams) + num_streams = static_cast(total_size / max_block_size); + + if (num_streams == 0) + num_streams = 1; + + /// Ranges state, all streams will share the state. + auto ranges_state = std::make_shared(); + for (size_t i = 0; i < num_streams; ++i) + { + auto source = std::make_shared( + intersected_ranges, ranges_state, max_block_size, numbers_storage.step, numbers_storage.column_name); + + if (i == 0) + source->addTotalRowsApprox(total_size); + + pipe.addSource(std::move(source)); + } + return pipe; } /// Fall back to NumbersSource From 0f84f68da77663e2adcce800cceefff5ab019b58 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 13 Feb 2024 09:59:39 +0000 Subject: [PATCH 013/334] Simplified code --- src/Common/iota.cpp | 2 +- src/Common/iota.h | 2 +- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 65 +++++-------------- .../02970_generate_series.reference | 1 - .../0_stateless/02970_generate_series.sql | 1 - 5 files changed, 20 insertions(+), 51 deletions(-) diff --git a/src/Common/iota.cpp b/src/Common/iota.cpp index 532c4bde76d..86c9e04bb06 100644 --- a/src/Common/iota.cpp +++ b/src/Common/iota.cpp @@ -60,6 +60,6 @@ template void iota_with_step(UInt8 * begin, size_t count, UInt8 first_value, UIn template void iota_with_step(UInt32 * begin, size_t count, UInt32 first_value, UInt32 step); template void iota_with_step(UInt64 * begin, size_t count, UInt64 first_value, UInt64 step); #if defined(OS_DARWIN) -extern template void iota_with_step(size_t * begin, size_t count, size_t first_value, size_t step); +template void iota_with_step(size_t * begin, size_t count, size_t first_value, size_t step); #endif } diff --git a/src/Common/iota.h b/src/Common/iota.h index f40cde9d5db..8fa18be9769 100644 --- a/src/Common/iota.h +++ b/src/Common/iota.h @@ -38,6 +38,6 @@ extern template void iota_with_step(UInt8 * begin, size_t count, UInt8 first_val extern template void iota_with_step(UInt32 * begin, size_t count, UInt32 first_value, UInt32 step); extern template void iota_with_step(UInt64 * begin, size_t count, UInt64 first_value, UInt64 step); #if defined(OS_DARWIN) -extern template void iota(size_t * begin, size_t count, size_t first_value, size_t step); +extern template void iota_with_step(size_t * begin, size_t count, size_t first_value, size_t step); #endif } diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 3656a6d31ee..d69e2b6ca5a 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -15,6 +15,7 @@ #include #include "Core/Types.h" +#include "base/Decimal_fwd.h" #include "base/types.h" namespace DB @@ -31,13 +32,12 @@ namespace class NumbersSource : public ISource { public: - NumbersSource(UInt64 block_size_, UInt64 offset_, UInt64 chunk_step_, const std::string & column_name, UInt64 step_, UInt64 remainder_) + NumbersSource(UInt64 block_size_, UInt64 offset_, UInt64 chunk_step_, const std::string & column_name, UInt64 step_) : ISource(createHeader(column_name)) , block_size(block_size_) , next(offset_) , chunk_step(chunk_step_) , step(step_) - , remainder(remainder_) { } String getName() const override { return "Numbers"; } @@ -50,41 +50,19 @@ public: protected: Chunk generate() override { - UInt64 curr = next; /// The local variable for some reason works faster (>20%) than member of class. - UInt64 first_element = (curr / step) * step; - if (first_element > std::numeric_limits::max() - remainder) - { - auto column = ColumnUInt64::create(0); - return {Columns{std::move(column)}, 0}; - } - first_element += remainder; - if (first_element < curr) - { - if (first_element > std::numeric_limits::max() - step) - { - auto column = ColumnUInt64::create(0); - return {Columns{std::move(column)}, 0}; - } - first_element += step; - } - if (first_element - curr >= block_size) - { - auto column = ColumnUInt64::create(0); - return {Columns{std::move(column)}, 0}; - } - UInt64 filtered_block_size = (block_size - (first_element - curr) - 1) / step + 1; - - auto column = ColumnUInt64::create(filtered_block_size); + auto column = ColumnUInt64::create(block_size); ColumnUInt64::Container & vec = column->getData(); + + UInt64 curr = next; /// The local variable for some reason works faster (>20%) than member of class. UInt64 * pos = vec.data(); /// This also accelerates the code. - UInt64 * end = &vec[filtered_block_size]; - iota_with_step(pos, static_cast(end - pos), first_element, step); + UInt64 * end = &vec[block_size]; + iota_with_step(pos, static_cast(end - pos), curr, step); next += chunk_step; progress(column->size(), column->byteSize()); - return {Columns{std::move(column)}, filtered_block_size}; + return {Columns{std::move(column)}, block_size}; } private: @@ -92,14 +70,13 @@ private: UInt64 next; UInt64 chunk_step; UInt64 step; - UInt64 remainder; }; struct RangeWithStep { UInt64 left; - UInt64 right; UInt64 step; + UInt128 size; }; using RangesWithStep = std::vector; @@ -125,21 +102,16 @@ std::optional stepped_range_from_range(const Range & r, UInt64 st if ((begin >= r.right_included) && (begin - r.right_included >= r.right.get())) return std::nullopt; UInt64 right_edge_included = r.right.get() - (1 - r.right_included); - return std::optional{RangeWithStep{begin, right_edge_included, step}}; + return std::optional{RangeWithStep{begin, step, static_cast(right_edge_included - begin) / step + 1}}; } -[[maybe_unused]] UInt128 sizeOfRange(const RangeWithStep & r) -{ - return static_cast(r.right - r.left) / r.step + 1; -}; - [[maybe_unused]] auto sizeOfRanges(const RangesWithStep & rs) { UInt128 total_size{}; for (const RangeWithStep & r : rs) { /// total_size will never overflow - total_size += sizeOfRange(r); + total_size += r.size; } return total_size; }; @@ -211,7 +183,7 @@ protected: while (need != 0) { UInt128 can_provide = end.offset_in_ranges == ranges.size() ? static_cast(0) - : sizeOfRange(ranges[end.offset_in_ranges]) - end.offset_in_range; + : ranges[end.offset_in_ranges].size - end.offset_in_range; if (can_provide == 0) break; @@ -278,7 +250,7 @@ protected: UInt128 can_provide = cursor.offset_in_ranges == end.offset_in_ranges ? end.offset_in_range - cursor.offset_in_range - : static_cast(range.right - range.left) / range.step + 1 - cursor.offset_in_range; + : range.size - cursor.offset_in_range; /// set value to block auto set_value = [&pos, this](UInt128 & start_value, UInt128 & end_value) @@ -377,7 +349,7 @@ namespace size_t last_range_idx = 0; for (size_t i = 0; i < ranges.size(); i++) { - auto range_size = sizeOfRange(ranges[i]); + auto range_size = ranges[i].size; if (range_size < size) { size -= static_cast(range_size); @@ -391,7 +363,7 @@ namespace else { auto & range = ranges[i]; - range.right = range.left + static_cast(size) * range.step - 1; + range.size = static_cast(size); last_range_idx = i; break; } @@ -587,11 +559,10 @@ Pipe ReadFromSystemNumbersStep::makePipe() { auto source = std::make_shared( max_block_size, - numbers_storage.offset + i * max_block_size, - num_streams * max_block_size, + numbers_storage.offset + i * max_block_size * numbers_storage.step, + num_streams * max_block_size * numbers_storage.step, numbers_storage.column_name, - numbers_storage.step, - numbers_storage.offset % numbers_storage.step); + numbers_storage.step); if (numbers_storage.limit && i == 0) { diff --git a/tests/queries/0_stateless/02970_generate_series.reference b/tests/queries/0_stateless/02970_generate_series.reference index 9e6f1db911e..4e4f556a39b 100644 --- a/tests/queries/0_stateless/02970_generate_series.reference +++ b/tests/queries/0_stateless/02970_generate_series.reference @@ -4,7 +4,6 @@ 8 500000001 50000000 -100000001 0 10 13 diff --git a/tests/queries/0_stateless/02970_generate_series.sql b/tests/queries/0_stateless/02970_generate_series.sql index 045f584a622..a7f89e1bd3f 100644 --- a/tests/queries/0_stateless/02970_generate_series.sql +++ b/tests/queries/0_stateless/02970_generate_series.sql @@ -4,7 +4,6 @@ SELECT count() FROM generate_series(10, 20, 3); SELECT count() FROM generate_series(7, 77, 10); SELECT count() FROM generate_series(0, 1000000000, 2); SELECT count() FROM generate_series(0, 999999999, 20); -SELECT count() FROM generate_series(0, 1000000000, 2) WHERE generate_series % 5 == 0; SELECT * FROM generate_series(5, 4); SELECT * FROM generate_series(0, 0); From a11e67d4aae4433dd0f3d8ee46ba40e1cd73fdd5 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 16 Feb 2024 16:41:58 +0100 Subject: [PATCH 014/334] Make max_insert_delayed_streams_for_parallel_write actually work --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 1fb2393948a..f5494e56049 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -323,6 +323,9 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) if (!temp_part.part) continue; + if (!support_parallel_write && temp_part.part->getDataPartStorage().supportParallelWrite()) + support_parallel_write = true; + BlockIDsType block_id; if constexpr (async_insert) From 458793cc50b92361848c91803d07105a91acea85 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 16 Feb 2024 17:13:37 +0100 Subject: [PATCH 015/334] Review fix --- src/Storages/MergeTree/MergeTreeSink.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 36816904a81..ebc49e22d03 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -109,9 +109,14 @@ void MergeTreeSink::consume(Chunk chunk) } } - size_t max_insert_delayed_streams_for_parallel_write = DEFAULT_DELAYED_STREAMS_FOR_PARALLEL_WRITE; - if (!support_parallel_write || settings.max_insert_delayed_streams_for_parallel_write.changed) + size_t max_insert_delayed_streams_for_parallel_write; + + if (settings.max_insert_delayed_streams_for_parallel_write.changed) max_insert_delayed_streams_for_parallel_write = settings.max_insert_delayed_streams_for_parallel_write; + else if (support_parallel_write) + max_insert_delayed_streams_for_parallel_write = DEFAULT_DELAYED_STREAMS_FOR_PARALLEL_WRITE; + else + max_insert_delayed_streams_for_parallel_write = 0; /// In case of too much columns/parts in block, flush explicitly. streams += temp_part.streams.size(); From f7b524465c60b15c85f579ca22c48d4c165bf6f2 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 16 Feb 2024 17:14:36 +0100 Subject: [PATCH 016/334] Followup --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index f5494e56049..3cbdcf5106e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -368,9 +368,13 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) profile_events_scope.reset(); UInt64 elapsed_ns = watch.elapsed(); - size_t max_insert_delayed_streams_for_parallel_write = DEFAULT_DELAYED_STREAMS_FOR_PARALLEL_WRITE; - if (!support_parallel_write || settings.max_insert_delayed_streams_for_parallel_write.changed) + size_t max_insert_delayed_streams_for_parallel_write; + if (settings.max_insert_delayed_streams_for_parallel_write.changed) max_insert_delayed_streams_for_parallel_write = settings.max_insert_delayed_streams_for_parallel_write; + else if (support_parallel_write) + max_insert_delayed_streams_for_parallel_write = DEFAULT_DELAYED_STREAMS_FOR_PARALLEL_WRITE; + else + max_insert_delayed_streams_for_parallel_write = 0; /// In case of too much columns/parts in block, flush explicitly. streams += temp_part.streams.size(); From 524a2ca72decc124ef1e38b79843c2388cceb0bb Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 27 Feb 2024 19:17:34 +0100 Subject: [PATCH 017/334] WIP on createForShard --- .../OptimizeShardingKeyRewriteInVisitor.cpp | 44 +++++++++++++++++++ 1 file changed, 44 insertions(+) diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp index 3a592c0fe55..8aca28a90ef 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp @@ -5,6 +5,12 @@ #include #include #include +#include "Analyzer/ColumnNode.h" +#include "Analyzer/ConstantNode.h" +#include "Analyzer/FunctionNode.h" +#include "Analyzer/IQueryTreeNode.h" +#include "Analyzer/InDepthQueryTreeVisitor.h" +#include "DataTypes/IDataType.h" namespace { @@ -119,4 +125,42 @@ void OptimizeShardingKeyRewriteInMatcher::visit(ASTFunction & function, Data & d } } + +class OptimizeShardingKeyRewriteIn : InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; + + void enterImpl(QueryTreeNodePtr & node) + { + auto * function_node = node->as(); + if (!function_node || function_node->getFunctionName() != "in") + return; + + auto & arguments = function_node->getArguments().getNodes(); + auto * column = arguments[0]->as(); + if (!column) + return; + + if (!data.sharding_key_expr->getRequiredColumnsWithTypes().contains(column->getColumnName())) + return; + + if (auto * constant = arguments[1]->as()) + { + if (isTuple(constant->getResultType())) + { + auto & tuple = constant->getValue().get(); + std::erase_if(tuple, [&](auto & child) + { + return tuple.size() > 1 && !shardContains(child, name, data); + }); + } + } + } + + OptimizeShardingKeyRewriteInMatcher::Data data; +}; + + } From 0de2d766fa971f54eff40641e16ed6857e1ece5f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 28 Feb 2024 15:30:06 +0100 Subject: [PATCH 018/334] WIP on different JSONs on shards --- src/Analyzer/IdentifierNode.cpp | 10 +- src/Analyzer/IdentifierNode.h | 6 ++ src/DataTypes/ObjectUtils.cpp | 75 ++++++++++++- src/DataTypes/ObjectUtils.h | 11 ++ .../ClusterProxy/SelectStreamFactory.cpp | 79 +++++++++++++- .../ClusterProxy/SelectStreamFactory.h | 26 +++++ .../ClusterProxy/executeQuery.cpp | 101 ++++++++++++------ src/Interpreters/ClusterProxy/executeQuery.h | 2 - .../OptimizeShardingKeyRewriteInVisitor.cpp | 35 ++++-- .../OptimizeShardingKeyRewriteInVisitor.h | 3 + src/Processors/QueryPlan/ReadFromRemote.cpp | 10 +- src/Storages/StorageDistributed.cpp | 32 +++--- 12 files changed, 325 insertions(+), 65 deletions(-) diff --git a/src/Analyzer/IdentifierNode.cpp b/src/Analyzer/IdentifierNode.cpp index 88b3daacb12..7e4d4c02a4c 100644 --- a/src/Analyzer/IdentifierNode.cpp +++ b/src/Analyzer/IdentifierNode.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -56,13 +57,18 @@ void IdentifierNode::updateTreeHashImpl(HashState & state) const QueryTreeNodePtr IdentifierNode::cloneImpl() const { - return std::make_shared(identifier); + auto result = std::make_shared(identifier); + result->use_parts_for_to_ast = use_parts_for_to_ast; + return result; } ASTPtr IdentifierNode::toASTImpl(const ConvertToASTOptions & /* options */) const { auto identifier_parts = identifier.getParts(); - return std::make_shared(std::move(identifier_parts)); + if (use_parts_for_to_ast) + return std::make_shared(std::move(identifier_parts)); + else + return std::make_shared(identifier.getFullName()); } } diff --git a/src/Analyzer/IdentifierNode.h b/src/Analyzer/IdentifierNode.h index 872bb14d512..3bc37b4c69d 100644 --- a/src/Analyzer/IdentifierNode.h +++ b/src/Analyzer/IdentifierNode.h @@ -52,6 +52,11 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + void useFullNameInToAST() + { + use_parts_for_to_ast = false; + } + protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; @@ -64,6 +69,7 @@ protected: private: Identifier identifier; std::optional table_expression_modifiers; + bool use_parts_for_to_ast = false; static constexpr size_t children_size = 0; }; diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 23d29136c85..01ba50d90f3 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -20,6 +21,16 @@ #include #include #include +#include "Analyzer/ConstantNode.h" +#include "Analyzer/FunctionNode.h" +#include "Analyzer/IQueryTreeNode.h" +#include "Analyzer/Identifier.h" +#include "Analyzer/IdentifierNode.h" +#include "Analyzer/QueryNode.h" +#include "Analyzer/Utils.h" +#include +#include +#include "Common/logger_useful.h" namespace DB @@ -888,10 +899,10 @@ static void addConstantToWithClause(const ASTPtr & query, const String & column_ /// @expected_columns and @available_columns contain descriptions /// of extended Object columns. -void replaceMissedSubcolumnsByConstants( +NamesAndTypes calculateMissedSubcolumns( const ColumnsDescription & expected_columns, - const ColumnsDescription & available_columns, - ASTPtr query) + const ColumnsDescription & available_columns +) { NamesAndTypes missed_names_types; @@ -928,6 +939,18 @@ void replaceMissedSubcolumnsByConstants( [](const auto & lhs, const auto & rhs) { return lhs.name < rhs.name; }); } + return missed_names_types; +} + +/// @expected_columns and @available_columns contain descriptions +/// of extended Object columns. +void replaceMissedSubcolumnsByConstants( + const ColumnsDescription & expected_columns, + const ColumnsDescription & available_columns, + ASTPtr query) +{ + NamesAndTypes missed_names_types = calculateMissedSubcolumns(expected_columns, available_columns); + if (missed_names_types.empty()) return; @@ -940,6 +963,52 @@ void replaceMissedSubcolumnsByConstants( addConstantToWithClause(query, name, type); } +/// @expected_columns and @available_columns contain descriptions +/// of extended Object columns. +void replaceMissedSubcolumnsByConstants( + const ColumnsDescription & expected_columns, + const ColumnsDescription & available_columns, + QueryTreeNodePtr & query, + const ContextPtr & context [[maybe_unused]]) +{ + NamesAndTypes missed_names_types = calculateMissedSubcolumns(expected_columns, available_columns); + + if (missed_names_types.empty()) + return; + + auto * query_node = query->as(); + if (!query_node) + return; + + auto table_expression = extractLeftTableExpression(query_node->getJoinTree()); + + auto & with_nodes = query_node->getWith().getNodes(); + + std::unordered_map column_name_to_node; + for (const auto & [name, type] : missed_names_types) + { + auto constant = std::make_shared(type->getDefault(), type); + constant->setAlias(table_expression->getAlias() + name); + // auto materialize = std::make_shared("materialize"); + + // auto function = FunctionFactory::instance().get("materialize", context); + // materialize->getArguments().getNodes() = { constant }; + // materialize->resolveAsFunction(function->build(materialize->getArgumentColumns())); + // materialize->setAlias(name); + + with_nodes.push_back(constant); + + auto id = std::make_shared(Identifier(table_expression->getAlias() + name)); + id->useFullNameInToAST(); + column_name_to_node[name] = id; + LOG_DEBUG(&Poco::Logger::get("replaceMissedSubcolumnsByConstants"), "Name {} Expression\n{}", name, column_name_to_node[name]->dumpTree()); + } + + LOG_DEBUG(&Poco::Logger::get("replaceMissedSubcolumnsByConstants"), "Table expression\n{} ", table_expression->dumpTree()); + replaceColumns(query, table_expression, column_name_to_node); + LOG_DEBUG(&Poco::Logger::get("replaceMissedSubcolumnsByConstants"), "Result:\n{} ", query->dumpTree()); +} + Field FieldVisitorReplaceScalars::operator()(const Array & x) const { if (num_dimensions_to_keep == 0) diff --git a/src/DataTypes/ObjectUtils.h b/src/DataTypes/ObjectUtils.h index 2bfcaae09ca..f4a8abe8abf 100644 --- a/src/DataTypes/ObjectUtils.h +++ b/src/DataTypes/ObjectUtils.h @@ -3,6 +3,8 @@ #include #include #include +#include "Analyzer/IQueryTreeNode.h" +#include "Interpreters/Context_fwd.h" #include #include #include @@ -14,6 +16,9 @@ namespace DB struct StorageSnapshot; using StorageSnapshotPtr = std::shared_ptr; +class IQueryTreeNode; +using QueryTreeNodePtr = std::shared_ptr; + /// Returns number of dimensions in Array type. 0 if type is not array. size_t getNumberOfDimensions(const IDataType & type); @@ -97,6 +102,12 @@ void replaceMissedSubcolumnsByConstants( const ColumnsDescription & available_columns, ASTPtr query); +void replaceMissedSubcolumnsByConstants( + const ColumnsDescription & expected_columns, + const ColumnsDescription & available_columns, + QueryTreeNodePtr & query, + const ContextPtr & context); + /// Visitor that keeps @num_dimensions_to_keep dimensions in arrays /// and replaces all scalars or nested arrays to @replacement at that level. class FieldVisitorReplaceScalars : public StaticVisitor diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index f0592735caf..5167ffc0e27 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -5,6 +5,10 @@ #include #include #include +#include "Analyzer/IQueryTreeNode.h" +#include "Interpreters/InterpreterSelectQueryAnalyzer.h" +#include "Interpreters/SelectQueryOptions.h" +#include "Planner/Utils.h" #include #include #include @@ -124,18 +128,55 @@ void SelectStreamFactory::createForShard( if (it != objects_by_shard.end()) replaceMissedSubcolumnsByConstants(storage_snapshot->object_columns, it->second, query_ast); + createForShardImpl( + shard_info, + query_ast, + main_table, + table_func_ptr, + std::move(context), + local_plans, + remote_shards, + shard_count, + parallel_replicas_enabled, + std::move(shard_filter_generator)); +} + +void SelectStreamFactory::createForShardImpl( + const Cluster::ShardInfo & shard_info, + const ASTPtr & query_ast, + const StorageID & main_table, + const ASTPtr & table_func_ptr, + ContextPtr context, + std::vector & local_plans, + Shards & remote_shards, + UInt32 shard_count, + bool parallel_replicas_enabled, + AdditionalShardFilterGenerator shard_filter_generator) +{ auto emplace_local_stream = [&]() { + Block shard_header; + if (context->getSettingsRef().allow_experimental_analyzer) + shard_header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_ast, context, SelectQueryOptions(processed_stage).analyze()); + else + shard_header = header; + local_plans.emplace_back(createLocalPlan( - query_ast, header, context, processed_stage, shard_info.shard_num, shard_count)); + query_ast, shard_header, context, processed_stage, shard_info.shard_num, shard_count)); }; auto emplace_remote_stream = [&](bool lazy = false, time_t local_delay = 0) { + Block shard_header; + if (context->getSettingsRef().allow_experimental_analyzer) + shard_header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_ast, context, SelectQueryOptions(processed_stage).analyze()); + else + shard_header = header; + remote_shards.emplace_back(Shard{ .query = query_ast, .main_table = main_table, - .header = header, + .header = shard_header, .shard_info = shard_info, .lazy = lazy, .local_delay = local_delay, @@ -243,6 +284,40 @@ void SelectStreamFactory::createForShard( emplace_remote_stream(); } +void SelectStreamFactory::createForShard( + const Cluster::ShardInfo & shard_info, + const QueryTreeNodePtr & query_tree, + const StorageID & main_table, + const ASTPtr & table_func_ptr, + ContextPtr context, + std::vector & local_plans, + Shards & remote_shards, + UInt32 shard_count, + bool parallel_replicas_enabled, + AdditionalShardFilterGenerator shard_filter_generator) +{ + + auto it = objects_by_shard.find(shard_info.shard_num); + QueryTreeNodePtr modified_query = query_tree; + if (it != objects_by_shard.end()) + replaceMissedSubcolumnsByConstants(storage_snapshot->object_columns, it->second, modified_query, context); + + auto query_ast = queryNodeToDistributedSelectQuery(modified_query); + + createForShardImpl( + shard_info, + query_ast, + main_table, + table_func_ptr, + std::move(context), + local_plans, + remote_shards, + shard_count, + parallel_replicas_enabled, + std::move(shard_filter_generator)); + +} + } } diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index 9993ea7028d..45d6ea14c01 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -7,6 +7,7 @@ #include #include #include +#include "Analyzer/IQueryTreeNode.h" namespace DB { @@ -83,10 +84,35 @@ public: bool parallel_replicas_enabled, AdditionalShardFilterGenerator shard_filter_generator); + void createForShard( + const Cluster::ShardInfo & shard_info, + const QueryTreeNodePtr & query_tree, + const StorageID & main_table, + const ASTPtr & table_func_ptr, + ContextPtr context, + std::vector & local_plans, + Shards & remote_shards, + UInt32 shard_count, + bool parallel_replicas_enabled, + AdditionalShardFilterGenerator shard_filter_generator); + const Block header; const ColumnsDescriptionByShardNum objects_by_shard; const StorageSnapshotPtr storage_snapshot; QueryProcessingStage::Enum processed_stage; + +private: + void createForShardImpl( + const Cluster::ShardInfo & shard_info, + const ASTPtr & query_ast, + const StorageID & main_table, + const ASTPtr & table_func_ptr, + ContextPtr context, + std::vector & local_plans, + Shards & remote_shards, + UInt32 shard_count, + bool parallel_replicas_enabled, + AdditionalShardFilterGenerator shard_filter_generator); }; } diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 6cdff939af1..07ef7aa6c96 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -204,12 +204,10 @@ void executeQuery( const ASTPtr & table_func_ptr, SelectStreamFactory & stream_factory, LoggerPtr log, - const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info, const ExpressionActionsPtr & sharding_key_expr, const std::string & sharding_key_column_name, - const ClusterPtr & not_optimized_cluster, const DistributedSettings & distributed_settings, AdditionalShardFilterGenerator shard_filter_generator) { @@ -218,6 +216,8 @@ void executeQuery( if (settings.max_distributed_depth && context->getClientInfo().distributed_depth >= settings.max_distributed_depth) throw Exception(ErrorCodes::TOO_LARGE_DISTRIBUTED_DEPTH, "Maximum distributed depth exceeded"); + const ClusterPtr & not_optimized_cluster = query_info.cluster; + std::vector plans; SelectStreamFactory::Shards remote_shards; @@ -237,40 +237,81 @@ void executeQuery( new_context->increaseDistributedDepth(); const size_t shards = cluster->getShardCount(); - for (size_t i = 0, s = cluster->getShardsInfo().size(); i < s; ++i) + + if (context->getSettingsRef().allow_experimental_analyzer) { - const auto & shard_info = cluster->getShardsInfo()[i]; - - ASTPtr query_ast_for_shard = query_ast->clone(); - if (sharding_key_expr && query_info.optimized_cluster && settings.optimize_skip_unused_shards_rewrite_in && shards > 1) + for (size_t i = 0, s = cluster->getShardsInfo().size(); i < s; ++i) { - OptimizeShardingKeyRewriteInVisitor::Data visitor_data{ - sharding_key_expr, - sharding_key_expr->getSampleBlock().getByPosition(0).type, - sharding_key_column_name, + const auto & shard_info = cluster->getShardsInfo()[i]; + + auto query_for_shard = query_info.query_tree->clone(); + if (sharding_key_expr && query_info.optimized_cluster && settings.optimize_skip_unused_shards_rewrite_in && shards > 1) + { + OptimizeShardingKeyRewriteInVisitor::Data visitor_data{ + sharding_key_expr, + sharding_key_expr->getSampleBlock().getByPosition(0).type, + sharding_key_column_name, + shard_info, + not_optimized_cluster->getSlotToShard(), + }; + optimizeShardingKeyRewriteIn(query_for_shard, std::move(visitor_data), new_context); + } + + // decide for each shard if parallel reading from replicas should be enabled + // according to settings and number of replicas declared per shard + const auto & addresses = cluster->getShardsAddresses().at(i); + bool parallel_replicas_enabled = addresses.size() > 1 && context->canUseTaskBasedParallelReplicas(); + + stream_factory.createForShard( shard_info, - not_optimized_cluster->getSlotToShard(), - }; - OptimizeShardingKeyRewriteInVisitor visitor(visitor_data); - visitor.visit(query_ast_for_shard); + query_for_shard, + main_table, + table_func_ptr, + new_context, + plans, + remote_shards, + static_cast(shards), + parallel_replicas_enabled, + shard_filter_generator); } + } + else + { + for (size_t i = 0, s = cluster->getShardsInfo().size(); i < s; ++i) + { + const auto & shard_info = cluster->getShardsInfo()[i]; - // decide for each shard if parallel reading from replicas should be enabled - // according to settings and number of replicas declared per shard - const auto & addresses = cluster->getShardsAddresses().at(i); - bool parallel_replicas_enabled = addresses.size() > 1 && context->canUseTaskBasedParallelReplicas(); + ASTPtr query_ast_for_shard = query_info.query->clone(); + if (sharding_key_expr && query_info.optimized_cluster && settings.optimize_skip_unused_shards_rewrite_in && shards > 1) + { + OptimizeShardingKeyRewriteInVisitor::Data visitor_data{ + sharding_key_expr, + sharding_key_expr->getSampleBlock().getByPosition(0).type, + sharding_key_column_name, + shard_info, + not_optimized_cluster->getSlotToShard(), + }; + OptimizeShardingKeyRewriteInVisitor visitor(visitor_data); + visitor.visit(query_ast_for_shard); + } - stream_factory.createForShard( - shard_info, - query_ast_for_shard, - main_table, - table_func_ptr, - new_context, - plans, - remote_shards, - static_cast(shards), - parallel_replicas_enabled, - shard_filter_generator); + // decide for each shard if parallel reading from replicas should be enabled + // according to settings and number of replicas declared per shard + const auto & addresses = cluster->getShardsAddresses().at(i); + bool parallel_replicas_enabled = addresses.size() > 1 && context->canUseTaskBasedParallelReplicas(); + + stream_factory.createForShard( + shard_info, + query_ast_for_shard, + main_table, + table_func_ptr, + new_context, + plans, + remote_shards, + static_cast(shards), + parallel_replicas_enabled, + shard_filter_generator); + } } if (!remote_shards.empty()) diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index bbc3c6c9e49..8f6f6300c7b 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -58,12 +58,10 @@ void executeQuery( const ASTPtr & table_func_ptr, SelectStreamFactory & stream_factory, LoggerPtr log, - const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info, const ExpressionActionsPtr & sharding_key_expr, const std::string & sharding_key_column_name, - const ClusterPtr & not_optimized_cluster, const DistributedSettings & distributed_settings, AdditionalShardFilterGenerator shard_filter_generator); diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp index 8aca28a90ef..42c6e63da01 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -11,6 +12,7 @@ #include "Analyzer/IQueryTreeNode.h" #include "Analyzer/InDepthQueryTreeVisitor.h" #include "DataTypes/IDataType.h" +#include "Interpreters/Context_fwd.h" namespace { @@ -126,11 +128,15 @@ void OptimizeShardingKeyRewriteInMatcher::visit(ASTFunction & function, Data & d } -class OptimizeShardingKeyRewriteIn : InDepthQueryTreeVisitorWithContext +class OptimizeShardingKeyRewriteIn : public InDepthQueryTreeVisitorWithContext { public: using Base = InDepthQueryTreeVisitorWithContext; - using Base::Base; + + OptimizeShardingKeyRewriteIn(OptimizeShardingKeyRewriteInVisitor::Data data_, ContextPtr context) + : Base(std::move(context)) + , data(std::move(data_)) + {} void enterImpl(QueryTreeNodePtr & node) { @@ -143,6 +149,8 @@ public: if (!column) return; + auto name = column->getColumnName(); + if (!data.sharding_key_expr->getRequiredColumnsWithTypes().contains(column->getColumnName())) return; @@ -150,17 +158,30 @@ public: { if (isTuple(constant->getResultType())) { - auto & tuple = constant->getValue().get(); - std::erase_if(tuple, [&](auto & child) + const auto & tuple = constant->getValue().get(); + Tuple new_tuple; + new_tuple.reserve(tuple.size()); + + for (const auto & child : tuple) { - return tuple.size() > 1 && !shardContains(child, name, data); - }); + if (shardContains(child, name, data)) + new_tuple.push_back(child); + } + + if (new_tuple.empty()) + new_tuple.push_back(tuple.back()); + node = std::make_shared(new_tuple); } } } - OptimizeShardingKeyRewriteInMatcher::Data data; + OptimizeShardingKeyRewriteInVisitor::Data data; }; +void optimizeShardingKeyRewriteIn(QueryTreeNodePtr & node, OptimizeShardingKeyRewriteInVisitor::Data data, ContextPtr context) +{ + OptimizeShardingKeyRewriteIn visitor(std::move(data), std::move(context)); + visitor.visit(node); +} } diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.h b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.h index d546db40df7..d202609160b 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.h +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.h @@ -2,6 +2,7 @@ #include #include +#include "Analyzer/IQueryTreeNode.h" namespace DB { @@ -44,4 +45,6 @@ struct OptimizeShardingKeyRewriteInMatcher using OptimizeShardingKeyRewriteInVisitor = InDepthNodeVisitor; +void optimizeShardingKeyRewriteIn(QueryTreeNodePtr & node, OptimizeShardingKeyRewriteInVisitor::Data data, ContextPtr context); + } diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 93c73a66b78..022c4f699f2 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -216,7 +216,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream }; pipes.emplace_back(createDelayedPipe(shard.header, lazily_create_stream, add_totals, add_extremes)); - addConvertingActions(pipes.back(), output_stream->header); + addConvertingActions(pipes.back(), shard.header); } void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard) @@ -281,7 +281,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact auto remote_query_executor = std::make_shared( shard.shard_info.pool, query_string, - output_stream->header, + shard.header, context, throttler, scalars, @@ -297,7 +297,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact pipes.emplace_back( createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending)); - addConvertingActions(pipes.back(), output_stream->header); + addConvertingActions(pipes.back(), shard.header); } } else @@ -305,7 +305,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact const String query_string = formattedAST(shard.query); auto remote_query_executor = std::make_shared( - shard.shard_info.pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage); + shard.shard_info.pool, query_string, shard.header, context, throttler, scalars, external_tables, stage); remote_query_executor->setLogger(log); if (context->canUseTaskBasedParallelReplicas()) @@ -326,7 +326,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact pipes.emplace_back( createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending)); - addConvertingActions(pipes.back(), output_stream->header); + addConvertingActions(pipes.back(), shard.header); } } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 92e7dcdf4f2..34ab21a4751 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -30,6 +30,7 @@ #include #include #include +#include "Analyzer/IQueryTreeNode.h" #include #include @@ -813,7 +814,8 @@ void StorageDistributed::read( const size_t /*num_streams*/) { Block header; - ASTPtr query_ast; + + SelectQueryInfo modified_query_info = query_info; if (local_context->getSettingsRef().allow_experimental_analyzer) { @@ -821,7 +823,7 @@ void StorageDistributed::read( if (!remote_table_function_ptr) remote_storage_id = StorageID{remote_database, remote_table}; - auto query_tree_distributed = buildQueryTreeDistributed(query_info, + auto query_tree_distributed = buildQueryTreeDistributed(modified_query_info, storage_snapshot, remote_storage_id, remote_table_function_ptr); @@ -831,20 +833,24 @@ void StorageDistributed::read( */ for (auto & column : header) column.column = column.column->convertToFullColumnIfConst(); - query_ast = queryNodeToDistributedSelectQuery(query_tree_distributed); + modified_query_info.query = queryNodeToDistributedSelectQuery(query_tree_distributed); + + modified_query_info.query_tree = std::move(query_tree_distributed); } else { - header = InterpreterSelectQuery(query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - query_ast = query_info.query; + header = InterpreterSelectQuery(modified_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); } - const auto & modified_query_ast = ClusterProxy::rewriteSelectQuery( - local_context, query_ast, - remote_database, remote_table, remote_table_function_ptr); + if (!local_context->getSettingsRef().allow_experimental_analyzer) + { + modified_query_info.query = ClusterProxy::rewriteSelectQuery( + local_context, modified_query_info.query, + remote_database, remote_table, remote_table_function_ptr); + } /// Return directly (with correct header) if no shard to query. - if (query_info.getCluster()->getShardsInfo().empty()) + if (modified_query_info.getCluster()->getShardsInfo().empty()) { if (local_context->getSettingsRef().allow_experimental_analyzer) return; @@ -872,7 +878,7 @@ void StorageDistributed::read( const auto & settings = local_context->getSettingsRef(); ClusterProxy::AdditionalShardFilterGenerator additional_shard_filter_generator; - if (local_context->canUseParallelReplicasCustomKey(*query_info.getCluster())) + if (local_context->canUseParallelReplicasCustomKey(*modified_query_info.getCluster())) { if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, *local_context)) { @@ -881,7 +887,7 @@ void StorageDistributed::read( column_description = this->getInMemoryMetadataPtr()->columns, custom_key_type = settings.parallel_replicas_custom_key_filter_type.value, context = local_context, - replica_count = query_info.getCluster()->getShardsInfo().front().per_replica_pools.size()](uint64_t replica_num) -> ASTPtr + replica_count = modified_query_info.getCluster()->getShardsInfo().front().per_replica_pools.size()](uint64_t replica_num) -> ASTPtr { return getCustomKeyFilterForParallelReplica( replica_count, replica_num - 1, my_custom_key_ast, custom_key_type, column_description, context); @@ -897,12 +903,10 @@ void StorageDistributed::read( remote_table_function_ptr, select_stream_factory, log, - modified_query_ast, local_context, - query_info, + modified_query_info, sharding_key_expr, sharding_key_column_name, - query_info.cluster, distributed_settings, additional_shard_filter_generator); From 8bf7c2c5971afc22dda32f9f4ad453ac481f2359 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 28 Feb 2024 15:40:42 +0100 Subject: [PATCH 019/334] Use output header --- src/Processors/QueryPlan/ReadFromRemote.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 022c4f699f2..fde2313bc15 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -216,7 +216,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream }; pipes.emplace_back(createDelayedPipe(shard.header, lazily_create_stream, add_totals, add_extremes)); - addConvertingActions(pipes.back(), shard.header); + addConvertingActions(pipes.back(), output_stream->header); } void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard) @@ -297,7 +297,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact pipes.emplace_back( createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending)); - addConvertingActions(pipes.back(), shard.header); + addConvertingActions(pipes.back(), output_stream->header); } } else @@ -326,7 +326,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact pipes.emplace_back( createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending)); - addConvertingActions(pipes.back(), shard.header); + addConvertingActions(pipes.back(), output_stream->header); } } From 8b1a1d42daa01e946aa8102d683dbab90b447838 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Thu, 29 Feb 2024 18:07:00 +0300 Subject: [PATCH 020/334] Traverse shadow directory for system.remote_data_paths --- src/Core/Settings.h | 1 + src/Disks/IDisk.h | 5 ++++- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 10 ++++++++-- src/Disks/ObjectStorages/DiskObjectStorage.h | 5 ++++- .../System/StorageSystemRemoteDataPaths.cpp | 13 +++++++++++++ 5 files changed, 30 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d77b3a45188..7cf068d7f1f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -843,6 +843,7 @@ class IColumn; M(Bool, optimize_uniq_to_count, true, "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause.", 0) \ M(Bool, use_variant_as_common_type, false, "Use Variant as a result type for if/multiIf in case when there is no common type for arguments", 0) \ M(Bool, enable_order_by_all, true, "Enable sorting expression ORDER BY ALL.", 0) \ + M(Bool, traverse_shadow_remote_data_paths, false, "Traverse shadow directory when query system.remote_data_paths", 0) \ \ /** Experimental functions */ \ M(Bool, allow_experimental_materialized_postgresql_table, false, "Allows to use the MaterializedPostgreSQL table engine. Disabled by default, because this feature is experimental", 0) \ diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 3d228850537..62b02938d1a 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -320,7 +320,10 @@ public: {} }; - virtual void getRemotePathsRecursive(const String &, std::vector &) + virtual void getRemotePathsRecursive( + const String &, + std::vector &, + const std::function & /* skip_predicate */ = {}) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getRemotePathsRecursive() not implemented for disk: {}`", diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 2a648f28f14..460d242d5cd 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -90,11 +90,17 @@ StoredObjects DiskObjectStorage::getStorageObjects(const String & local_path) co return metadata_storage->getStorageObjects(local_path); } -void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std::vector & paths_map) +void DiskObjectStorage::getRemotePathsRecursive( + const String & local_path, + std::vector & paths_map, + const std::function & skip_predicate) { if (!metadata_storage->exists(local_path)) return; + if (skip_predicate && skip_predicate(local_path)) + return; + /// Protect against concurrent delition of files (for example because of a merge). if (metadata_storage->isFile(local_path)) { @@ -142,7 +148,7 @@ void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std:: } for (; it->isValid(); it->next()) - DiskObjectStorage::getRemotePathsRecursive(fs::path(local_path) / it->name(), paths_map); + DiskObjectStorage::getRemotePathsRecursive(fs::path(local_path) / it->name(), paths_map, skip_predicate); } } diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index e1576509713..d7af656bea3 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -48,7 +48,10 @@ public: StoredObjects getStorageObjects(const String & local_path) const override; - void getRemotePathsRecursive(const String & local_path, std::vector & paths_map) override; + void getRemotePathsRecursive( + const String & local_path, + std::vector & paths_map, + const std::function & skip_predicate = {}) override; const std::string & getCacheName() const override { return object_storage->getCacheName(); } diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 87b7a84e8ba..708c1369965 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -9,6 +9,7 @@ #include #include +namespace fs = std::filesystem; namespace DB { @@ -59,6 +60,18 @@ Pipe StorageSystemRemoteDataPaths::read( std::vector remote_paths_by_local_path; disk->getRemotePathsRecursive("store", remote_paths_by_local_path); disk->getRemotePathsRecursive("data", remote_paths_by_local_path); + if (context->getSettingsRef().traverse_shadow_remote_data_paths) + disk->getRemotePathsRecursive( + "shadow", + remote_paths_by_local_path, + [](const String & local_path) + { + // `shadow/{backup_name}/revision.txt` is not an object metadata file + const auto path = fs::path(local_path); + return path.filename() == "revision.txt" && + path.parent_path().has_parent_path() && + path.parent_path().parent_path().filename() == "shadow"; + }); FileCachePtr cache; From 55053dae4459b1d1a6c05d436d1ab421a96c3934 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 29 Feb 2024 19:18:06 +0100 Subject: [PATCH 021/334] Some progress --- src/Analyzer/IdentifierNode.cpp | 10 +--- src/Analyzer/IdentifierNode.h | 6 --- src/DataTypes/ObjectUtils.cpp | 24 +++++----- src/DataTypes/ObjectUtils.h | 4 +- .../ClusterProxy/SelectStreamFactory.cpp | 10 ++-- .../ClusterProxy/SelectStreamFactory.h | 8 +++- src/Processors/QueryPlan/ReadFromRemote.cpp | 47 +++++++++++++++++++ .../test_distributed_type_object/test.py | 2 +- 8 files changed, 80 insertions(+), 31 deletions(-) diff --git a/src/Analyzer/IdentifierNode.cpp b/src/Analyzer/IdentifierNode.cpp index 7e4d4c02a4c..88b3daacb12 100644 --- a/src/Analyzer/IdentifierNode.cpp +++ b/src/Analyzer/IdentifierNode.cpp @@ -1,4 +1,3 @@ -#include #include #include @@ -57,18 +56,13 @@ void IdentifierNode::updateTreeHashImpl(HashState & state) const QueryTreeNodePtr IdentifierNode::cloneImpl() const { - auto result = std::make_shared(identifier); - result->use_parts_for_to_ast = use_parts_for_to_ast; - return result; + return std::make_shared(identifier); } ASTPtr IdentifierNode::toASTImpl(const ConvertToASTOptions & /* options */) const { auto identifier_parts = identifier.getParts(); - if (use_parts_for_to_ast) - return std::make_shared(std::move(identifier_parts)); - else - return std::make_shared(identifier.getFullName()); + return std::make_shared(std::move(identifier_parts)); } } diff --git a/src/Analyzer/IdentifierNode.h b/src/Analyzer/IdentifierNode.h index 3bc37b4c69d..872bb14d512 100644 --- a/src/Analyzer/IdentifierNode.h +++ b/src/Analyzer/IdentifierNode.h @@ -52,11 +52,6 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; - void useFullNameInToAST() - { - use_parts_for_to_ast = false; - } - protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; @@ -69,7 +64,6 @@ protected: private: Identifier identifier; std::optional table_expression_modifiers; - bool use_parts_for_to_ast = false; static constexpr size_t children_size = 0; }; diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 01ba50d90f3..47d8c5c9113 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -965,30 +965,32 @@ void replaceMissedSubcolumnsByConstants( /// @expected_columns and @available_columns contain descriptions /// of extended Object columns. -void replaceMissedSubcolumnsByConstants( +MissingObjectList replaceMissedSubcolumnsByConstants( const ColumnsDescription & expected_columns, const ColumnsDescription & available_columns, QueryTreeNodePtr & query, const ContextPtr & context [[maybe_unused]]) { + MissingObjectList missed_list; + NamesAndTypes missed_names_types = calculateMissedSubcolumns(expected_columns, available_columns); if (missed_names_types.empty()) - return; + return missed_list; auto * query_node = query->as(); if (!query_node) - return; + return missed_list; + + missed_list.reserve(missed_names_types.size()); auto table_expression = extractLeftTableExpression(query_node->getJoinTree()); - auto & with_nodes = query_node->getWith().getNodes(); - std::unordered_map column_name_to_node; for (const auto & [name, type] : missed_names_types) { auto constant = std::make_shared(type->getDefault(), type); - constant->setAlias(table_expression->getAlias() + name); + constant->setAlias(table_expression->getAlias() + "." + name); // auto materialize = std::make_shared("materialize"); // auto function = FunctionFactory::instance().get("materialize", context); @@ -996,17 +998,17 @@ void replaceMissedSubcolumnsByConstants( // materialize->resolveAsFunction(function->build(materialize->getArgumentColumns())); // materialize->setAlias(name); - with_nodes.push_back(constant); - - auto id = std::make_shared(Identifier(table_expression->getAlias() + name)); - id->useFullNameInToAST(); - column_name_to_node[name] = id; + column_name_to_node[name] = buildCastFunction(constant, type, context); + missed_list.push_back({ constant->getValueStringRepresentation() + "_" + constant->getResultType()->getName(), table_expression->getAlias() + "." + name }); + LOG_DEBUG(&Poco::Logger::get("replaceMissedSubcolumnsByConstants"), "{} -> {}", missed_list.back().first, missed_list.back().second); LOG_DEBUG(&Poco::Logger::get("replaceMissedSubcolumnsByConstants"), "Name {} Expression\n{}", name, column_name_to_node[name]->dumpTree()); } LOG_DEBUG(&Poco::Logger::get("replaceMissedSubcolumnsByConstants"), "Table expression\n{} ", table_expression->dumpTree()); replaceColumns(query, table_expression, column_name_to_node); LOG_DEBUG(&Poco::Logger::get("replaceMissedSubcolumnsByConstants"), "Result:\n{} ", query->dumpTree()); + + return missed_list; } Field FieldVisitorReplaceScalars::operator()(const Array & x) const diff --git a/src/DataTypes/ObjectUtils.h b/src/DataTypes/ObjectUtils.h index f4a8abe8abf..013e525832e 100644 --- a/src/DataTypes/ObjectUtils.h +++ b/src/DataTypes/ObjectUtils.h @@ -102,7 +102,9 @@ void replaceMissedSubcolumnsByConstants( const ColumnsDescription & available_columns, ASTPtr query); -void replaceMissedSubcolumnsByConstants( +using MissingObjectList = std::vector>; + +MissingObjectList replaceMissedSubcolumnsByConstants( const ColumnsDescription & expected_columns, const ColumnsDescription & available_columns, QueryTreeNodePtr & query, diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 5167ffc0e27..5bcd1ce68cb 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -151,7 +151,8 @@ void SelectStreamFactory::createForShardImpl( Shards & remote_shards, UInt32 shard_count, bool parallel_replicas_enabled, - AdditionalShardFilterGenerator shard_filter_generator) + AdditionalShardFilterGenerator shard_filter_generator, + MissingObjectList missed_list) { auto emplace_local_stream = [&]() { @@ -177,6 +178,7 @@ void SelectStreamFactory::createForShardImpl( .query = query_ast, .main_table = main_table, .header = shard_header, + .missing_object_list = std::move(missed_list), .shard_info = shard_info, .lazy = lazy, .local_delay = local_delay, @@ -299,8 +301,9 @@ void SelectStreamFactory::createForShard( auto it = objects_by_shard.find(shard_info.shard_num); QueryTreeNodePtr modified_query = query_tree; + MissingObjectList missed_list; if (it != objects_by_shard.end()) - replaceMissedSubcolumnsByConstants(storage_snapshot->object_columns, it->second, modified_query, context); + missed_list = replaceMissedSubcolumnsByConstants(storage_snapshot->object_columns, it->second, modified_query, context); auto query_ast = queryNodeToDistributedSelectQuery(modified_query); @@ -314,7 +317,8 @@ void SelectStreamFactory::createForShard( remote_shards, shard_count, parallel_replicas_enabled, - std::move(shard_filter_generator)); + std::move(shard_filter_generator), + std::move(missed_list)); } diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index 45d6ea14c01..bee7edb3c19 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -43,6 +44,8 @@ ASTPtr rewriteSelectQuery( using ColumnsDescriptionByShardNum = std::unordered_map; using AdditionalShardFilterGenerator = std::function; +using MissingObjectList = std::vector>; + class SelectStreamFactory { public: @@ -55,6 +58,8 @@ public: StorageID main_table; Block header; + MissingObjectList missing_object_list; + Cluster::ShardInfo shard_info; /// If we connect to replicas lazily. @@ -112,7 +117,8 @@ private: Shards & remote_shards, UInt32 shard_count, bool parallel_replicas_enabled, - AdditionalShardFilterGenerator shard_filter_generator); + AdditionalShardFilterGenerator shard_filter_generator, + MissingObjectList missed_list = {}); }; } diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index fde2313bc15..ac507c6d555 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -14,6 +15,7 @@ #include #include #include +#include "DataTypes/ObjectUtils.h" #include #include #include @@ -31,6 +33,48 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +static void addRenamingActions(Pipe & pipe, const MissingObjectList & missed_list, const Block & output_header) +{ + if (missed_list.empty()) + return; + + const auto & output_columns = output_header.getColumnsWithTypeAndName(); + std::vector indexes; + for (size_t i = 0; i < output_columns.size(); ++i) + { + bool found = false; + for (auto const & elem : missed_list) + { + if (output_columns[i].name.contains(elem.second)) + { + found = true; + break; + } + } + if (found) + indexes.push_back(i); + } + + auto dag = std::make_shared(pipe.getHeader().getColumnsWithTypeAndName()); + + for (size_t index : indexes) + { + dag->addOrReplaceInOutputs(dag->addAlias(*dag->getOutputs()[index], output_header.getByPosition(index).name)); + } + + // dag->addAliases(rename_to_apply); + + auto convert_actions = std::make_shared(dag); + pipe.addSimpleTransform([&](const Block & cur_header, Pipe::StreamType) -> ProcessorPtr + { + return std::make_shared(cur_header, convert_actions); + }); + + LOG_DEBUG(&Poco::Logger::get("addRenamingActions"), "EXPECTED:\n{}", output_header.dumpStructure()); + + LOG_DEBUG(&Poco::Logger::get("addRenamingActions"), "{}", pipe.getHeader().dumpStructure()); +} + static void addConvertingActions(Pipe & pipe, const Block & header) { if (blocksHaveEqualStructure(pipe.getHeader(), header)) @@ -216,6 +260,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream }; pipes.emplace_back(createDelayedPipe(shard.header, lazily_create_stream, add_totals, add_extremes)); + addRenamingActions(pipes.back(), shard.missing_object_list, output_stream->header); addConvertingActions(pipes.back(), output_stream->header); } @@ -297,6 +342,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact pipes.emplace_back( createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending)); + addRenamingActions(pipes.back(), shard.missing_object_list, output_stream->header); addConvertingActions(pipes.back(), output_stream->header); } } @@ -326,6 +372,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact pipes.emplace_back( createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending)); + addRenamingActions(pipes.back(), shard.missing_object_list, output_stream->header); addConvertingActions(pipes.back(), output_stream->header); } } diff --git a/tests/integration/test_distributed_type_object/test.py b/tests/integration/test_distributed_type_object/test.py index b2179af8a3f..f77e0248f02 100644 --- a/tests/integration/test_distributed_type_object/test.py +++ b/tests/integration/test_distributed_type_object/test.py @@ -59,7 +59,7 @@ def test_distributed_type_object(started_cluster): ) expected = TSV("120\n") - assert TSV(node1.query("SELECT sum(data.k2 * id) FROM dist_table")) == expected + assert TSV(node1.query("SELECT sum(data.k2 * id) FROM dist_table SETTINGS optimize_arithmetic_operations_in_aggregate_functions = 0")) == expected node1.query("TRUNCATE TABLE local_table") node2.query("TRUNCATE TABLE local_table") From 41deadda359ca02528fa6ffe9ecfed09c36b364a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 29 Feb 2024 18:37:00 +0000 Subject: [PATCH 022/334] Automatic style fix --- tests/integration/test_distributed_type_object/test.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_distributed_type_object/test.py b/tests/integration/test_distributed_type_object/test.py index f77e0248f02..7e6c000cb8e 100644 --- a/tests/integration/test_distributed_type_object/test.py +++ b/tests/integration/test_distributed_type_object/test.py @@ -59,7 +59,14 @@ def test_distributed_type_object(started_cluster): ) expected = TSV("120\n") - assert TSV(node1.query("SELECT sum(data.k2 * id) FROM dist_table SETTINGS optimize_arithmetic_operations_in_aggregate_functions = 0")) == expected + assert ( + TSV( + node1.query( + "SELECT sum(data.k2 * id) FROM dist_table SETTINGS optimize_arithmetic_operations_in_aggregate_functions = 0" + ) + ) + == expected + ) node1.query("TRUNCATE TABLE local_table") node2.query("TRUNCATE TABLE local_table") From a6cebad52bf4f29984db99cd4d4aa1eb41c50895 Mon Sep 17 00:00:00 2001 From: Nataly Merezhuk Date: Thu, 29 Feb 2024 16:32:29 -0500 Subject: [PATCH 023/334] Adds note on supported PostgreSQL versions. --- docs/en/engines/table-engines/integrations/postgresql.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index 131df1a435b..9cc4b11243e 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -8,6 +8,10 @@ sidebar_label: PostgreSQL The PostgreSQL engine allows to perform `SELECT` and `INSERT` queries on data that is stored on a remote PostgreSQL server. +:::note +Currently, only PostgreSQL versions 12 and up are supported. +::: + ## Creating a Table {#creating-a-table} ``` sql From 6143986b6d79c0262f5f7dc3052ec2a3f4cfc490 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Fri, 1 Mar 2024 14:55:02 +0300 Subject: [PATCH 024/334] Add query test --- ...raverse_shadow_system_data_paths.reference | 3 ++ ...03000_traverse_shadow_system_data_paths.sh | 34 +++++++++++++++++++ 2 files changed, 37 insertions(+) create mode 100644 tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.reference create mode 100755 tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh diff --git a/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.reference b/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.reference new file mode 100644 index 00000000000..e8183f05f5d --- /dev/null +++ b/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.reference @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh b/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh new file mode 100755 index 00000000000..a22cb200f9a --- /dev/null +++ b/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh @@ -0,0 +1,34 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +TABLE="03000_traverse_shadow_system_data_path_table" +BACKUP="03000_traverse_shadow_system_data_path_backup" + +${CLICKHOUSE_CLIENT} --query="CREATE TABLE ${TABLE} ( + id Int64, + data String +) ENGINE=MergeTree() +ORDER BY id +SETTINGS storage_policy='s3_cache';" + +${CLICKHOUSE_CLIENT} --query="INSERT INTO ${TABLE} VALUES (0, 'data');" +${CLICKHOUSE_CLIENT} --query "SELECT count() > 0 FROM system.remote_data_paths WHERE disk_name = 's3_cache'" + +${CLICKHOUSE_CLIENT} --query="ALTER TABLE ${TABLE} FREEZE WITH NAME '${BACKUP}';" +${CLICKHOUSE_CLIENT} --query="DROP TABLE ${TABLE} SYNC;" + +${CLICKHOUSE_CLIENT} --query " + SELECT count() > 0 + FROM system.remote_data_paths + WHERE disk_name = 's3_cache' AND local_path LIKE '%shadow/${BACKUP}%' + SETTINGS traverse_shadow_remote_data_paths=1;" +${CLICKHOUSE_CLIENT} --query "SYSTEM UNFREEZE WITH NAME '${BACKUP}';" >/dev/null +${CLICKHOUSE_CLIENT} --query " + SELECT count() == 0 + FROM system.remote_data_paths + WHERE disk_name = 's3_cache' AND local_path LIKE '%shadow/${BACKUP}%' + SETTINGS traverse_shadow_remote_data_paths=1;" From 1768b4477f4ff5db238cd4cc553587b136ed015d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 3 Mar 2024 11:50:22 +0100 Subject: [PATCH 025/334] Revert "Merge pull request #60690 from ClickHouse/remove-bad-test-8" This reverts commit c77eb8b1427f98daf63f7087bbdc0530b07db825, reversing changes made to bae4783fe9bd25decc41383a1234b0e936284c21. --- ..._external_tables_memory_tracking.reference | 16 ++++++ ...52_http_external_tables_memory_tracking.sh | 51 +++++++++++++++++++ 2 files changed, 67 insertions(+) create mode 100644 tests/queries/0_stateless/02152_http_external_tables_memory_tracking.reference create mode 100755 tests/queries/0_stateless/02152_http_external_tables_memory_tracking.sh diff --git a/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.reference b/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.reference new file mode 100644 index 00000000000..1fc09c8d154 --- /dev/null +++ b/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.reference @@ -0,0 +1,16 @@ +Checking input_format_parallel_parsing=false& +1 +Checking input_format_parallel_parsing=false&cancel_http_readonly_queries_on_client_close=1&readonly=1 +1 +Checking input_format_parallel_parsing=false&send_progress_in_http_headers=true +1 +Checking input_format_parallel_parsing=false&cancel_http_readonly_queries_on_client_close=1&readonly=1&send_progress_in_http_headers=true +1 +Checking input_format_parallel_parsing=true& +1 +Checking input_format_parallel_parsing=true&cancel_http_readonly_queries_on_client_close=1&readonly=1 +1 +Checking input_format_parallel_parsing=true&send_progress_in_http_headers=true +1 +Checking input_format_parallel_parsing=true&cancel_http_readonly_queries_on_client_close=1&readonly=1&send_progress_in_http_headers=true +1 diff --git a/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.sh b/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.sh new file mode 100755 index 00000000000..5f9eb460e44 --- /dev/null +++ b/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.sh @@ -0,0 +1,51 @@ +#!/usr/bin/env bash +# Tags: no-tsan, no-cpu-aarch64, no-parallel +# TSan does not supports tracing. +# trace_log doesn't work on aarch64 + +# Regression for proper release of Context, +# via tracking memory of external tables. + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +tmp_file=$(mktemp "$CURDIR/clickhouse.XXXXXX.csv") +trap 'rm $tmp_file' EXIT + +$CLICKHOUSE_CLIENT -q "SELECT toString(number) FROM numbers(1e6) FORMAT TSV" > "$tmp_file" + +function run_and_check() +{ + local query_id + query_id="$(${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @- <<<'SELECT generateUUIDv4()')" + + echo "Checking $*" + + # Run query with external table (implicit StorageMemory user) + $CLICKHOUSE_CURL -sS -F "s=@$tmp_file;" "$CLICKHOUSE_URL&s_structure=key+Int&query=SELECT+count()+FROM+s&memory_profiler_sample_probability=1&max_untracked_memory=0&query_id=$query_id&$*" -o /dev/null + + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @- <<<'SYSTEM FLUSH LOGS' + + # Check that temporary table had been destroyed. + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&allow_introspection_functions=1" --data-binary @- <<<" + WITH arrayStringConcat(arrayMap(x -> demangle(addressToSymbol(x)), trace), '\n') AS sym + SELECT count()>0 FROM system.trace_log + WHERE + sym LIKE '%DB::StorageMemory::drop%\n%TemporaryTableHolder::~TemporaryTableHolder%' AND + query_id = '$query_id' + " +} + +for input_format_parallel_parsing in false true; do + query_args_variants=( + "" + "cancel_http_readonly_queries_on_client_close=1&readonly=1" + "send_progress_in_http_headers=true" + # nested progress callback + "cancel_http_readonly_queries_on_client_close=1&readonly=1&send_progress_in_http_headers=true" + ) + for query_args in "${query_args_variants[@]}"; do + run_and_check "input_format_parallel_parsing=$input_format_parallel_parsing&$query_args" + done +done From 048a042dc4963631a23358d3e454dcd8a9eaafa2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 3 Mar 2024 11:50:46 +0100 Subject: [PATCH 026/334] Make 02152_http_external_tables_memory_tracking less flaky Signed-off-by: Azat Khuzhin --- .../02152_http_external_tables_memory_tracking.sh | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.sh b/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.sh index 5f9eb460e44..5494f7d59cb 100755 --- a/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.sh +++ b/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-cpu-aarch64, no-parallel +# Tags: no-tsan, no-cpu-aarch64, no-parallel, no-debug # TSan does not supports tracing. # trace_log doesn't work on aarch64 @@ -30,10 +30,16 @@ function run_and_check() # Check that temporary table had been destroyed. ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&allow_introspection_functions=1" --data-binary @- <<<" WITH arrayStringConcat(arrayMap(x -> demangle(addressToSymbol(x)), trace), '\n') AS sym - SELECT count()>0 FROM system.trace_log + SELECT 1 FROM system.trace_log + PREWHERE + query_id = '$query_id' AND + trace_type = 'MemorySample' AND + /* only deallocations */ + size < 0 AND + event_date >= yesterday() WHERE - sym LIKE '%DB::StorageMemory::drop%\n%TemporaryTableHolder::~TemporaryTableHolder%' AND - query_id = '$query_id' + sym LIKE '%DB::StorageMemory::drop%\n%TemporaryTableHolder::~TemporaryTableHolder%' + LIMIT 1 " } From 81185815a48b36d344bda623dd175c30e9b87ba3 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Mon, 4 Mar 2024 14:09:31 +0300 Subject: [PATCH 027/334] Update settings_changes_history --- src/Core/SettingsChangesHistory.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 4805df46d9b..b8793f437d8 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,6 +85,9 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"24.3", { + {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, + }}, {"24.2", { {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, From fbdc5e305365e9d93b86ed47144ffb13c1ce70c1 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Mon, 4 Mar 2024 17:16:51 +0300 Subject: [PATCH 028/334] Ignore flaky fail of system unfreeze --- .../0_stateless/03000_traverse_shadow_system_data_paths.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh b/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh index a22cb200f9a..2905d7801ca 100755 --- a/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh +++ b/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh @@ -26,7 +26,7 @@ ${CLICKHOUSE_CLIENT} --query " FROM system.remote_data_paths WHERE disk_name = 's3_cache' AND local_path LIKE '%shadow/${BACKUP}%' SETTINGS traverse_shadow_remote_data_paths=1;" -${CLICKHOUSE_CLIENT} --query "SYSTEM UNFREEZE WITH NAME '${BACKUP}';" >/dev/null +${CLICKHOUSE_CLIENT} --query "SYSTEM UNFREEZE WITH NAME '${BACKUP}';" &>/dev/null || true ${CLICKHOUSE_CLIENT} --query " SELECT count() == 0 FROM system.remote_data_paths From 758a75c1b46fa27a88e3dcf6e70a18dcf41d62ef Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Tue, 5 Mar 2024 09:53:30 +0300 Subject: [PATCH 029/334] Fix flaky test. Fix clang-tidy warning --- src/Disks/IDisk.h | 7 +++---- src/Disks/ObjectStorages/DiskObjectStorage.h | 2 +- src/Storages/System/StorageSystemRemoteDataPaths.cpp | 4 ++-- .../0_stateless/03000_traverse_shadow_system_data_paths.sh | 7 +------ 4 files changed, 7 insertions(+), 13 deletions(-) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 62b02938d1a..fcc92db7b96 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -321,11 +321,10 @@ public: }; virtual void getRemotePathsRecursive( - const String &, - std::vector &, - const std::function & /* skip_predicate */ = {}) + const String &, std::vector &, const std::function & /* skip_predicate */) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Method `getRemotePathsRecursive() not implemented for disk: {}`", getDataSourceDescription().toString()); } diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index d7af656bea3..9f11c0ed02e 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -51,7 +51,7 @@ public: void getRemotePathsRecursive( const String & local_path, std::vector & paths_map, - const std::function & skip_predicate = {}) override; + const std::function & skip_predicate) override; const std::string & getCacheName() const override { return object_storage->getCacheName(); } diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 708c1369965..a6263f18492 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -58,8 +58,8 @@ Pipe StorageSystemRemoteDataPaths::read( if (disk->isRemote()) { std::vector remote_paths_by_local_path; - disk->getRemotePathsRecursive("store", remote_paths_by_local_path); - disk->getRemotePathsRecursive("data", remote_paths_by_local_path); + disk->getRemotePathsRecursive("store", remote_paths_by_local_path, /* skip_predicate = */ {}); + disk->getRemotePathsRecursive("data", remote_paths_by_local_path, /* skip_predicate = */ {}); if (context->getSettingsRef().traverse_shadow_remote_data_paths) disk->getRemotePathsRecursive( "shadow", diff --git a/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh b/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh index 2905d7801ca..a1d4b9bba46 100755 --- a/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh +++ b/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh @@ -26,9 +26,4 @@ ${CLICKHOUSE_CLIENT} --query " FROM system.remote_data_paths WHERE disk_name = 's3_cache' AND local_path LIKE '%shadow/${BACKUP}%' SETTINGS traverse_shadow_remote_data_paths=1;" -${CLICKHOUSE_CLIENT} --query "SYSTEM UNFREEZE WITH NAME '${BACKUP}';" &>/dev/null || true -${CLICKHOUSE_CLIENT} --query " - SELECT count() == 0 - FROM system.remote_data_paths - WHERE disk_name = 's3_cache' AND local_path LIKE '%shadow/${BACKUP}%' - SETTINGS traverse_shadow_remote_data_paths=1;" +${CLICKHOUSE_CLIENT} --query "SYSTEM UNFREEZE WITH NAME '${BACKUP}';" &>/dev/null From df80c8c9f6ee0939cc6e6e05f3e951511a20f476 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Tue, 5 Mar 2024 10:43:48 +0300 Subject: [PATCH 030/334] Update test reference --- .../03000_traverse_shadow_system_data_paths.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.reference b/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.reference index e8183f05f5d..6ed281c757a 100644 --- a/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.reference +++ b/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.reference @@ -1,3 +1,2 @@ 1 1 -1 From e789d15948eaec3eaa9a8604e24d2f6ed7b60db5 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 5 Mar 2024 16:06:25 +0800 Subject: [PATCH 031/334] optimize insertmanyfrom of nullable(number) or nullable(string) --- src/Columns/ColumnDecimal.h | 7 +++++++ src/Columns/ColumnNullable.cpp | 8 ++++++++ src/Columns/ColumnNullable.h | 1 + src/Columns/ColumnString.cpp | 21 +++++++++++++++++++++ src/Columns/ColumnString.h | 2 ++ 5 files changed, 39 insertions(+) diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 7ca01a8342c..e0ea26744dc 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -56,6 +56,13 @@ public: void shrinkToFit() override { data.shrink_to_fit(); } void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast(src).getData()[n]); } + + void insertManyFrom(const IColumn & src, size_t position, size_t length) override + { + ValueType v = assert_cast(src).getData()[position]; + data.resize_fill(data.size() + length, v); + } + void insertData(const char * src, size_t /*length*/) override; void insertDefault() override { data.push_back(T()); } void insertManyDefaults(size_t length) override { data.resize_fill(data.size() + length); } diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 1d11827ac97..fa5fdfb8c21 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -231,6 +231,14 @@ void ColumnNullable::insertFrom(const IColumn & src, size_t n) getNullMapData().push_back(src_concrete.getNullMapData()[n]); } + +void ColumnNullable::insertManyFrom(const IColumn & src, size_t position, size_t length) +{ + const ColumnNullable & src_concrete = assert_cast(src); + getNestedColumn().insertManyFrom(src_concrete.getNestedColumn(), position, length); + getNullMapColumn().insertManyFrom(src_concrete.getNullMapColumn(), position, length); +} + void ColumnNullable::insertFromNotNullable(const IColumn & src, size_t n) { getNestedColumn().insertFrom(src, n); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index b4aef8e08fa..ef4bf4fa41b 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -69,6 +69,7 @@ public: void insert(const Field & x) override; bool tryInsert(const Field & x) override; void insertFrom(const IColumn & src, size_t n) override; + void insertManyFrom(const IColumn & src, size_t position, size_t length) override; void insertFromNotNullable(const IColumn & src, size_t n); void insertRangeFromNotNullable(const IColumn & src, size_t start, size_t length); diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index b9128372cea..f3c7ac1bf0c 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -38,6 +38,27 @@ ColumnString::ColumnString(const ColumnString & src) last_offset, chars.size()); } +void ColumnString::insertManyFrom(const IColumn & src, size_t position, size_t length) +{ + const ColumnString & src_concrete = assert_cast(src); + const UInt8 * src_buf = &src_concrete.chars[src_concrete.offsets[position - 1]]; + const size_t src_buf_size + = src_concrete.offsets[position] - src_concrete.offsets[position - 1]; /// -1th index is Ok, see PaddedPODArray. + + const size_t old_size = chars.size(); + const size_t new_size = old_size + src_buf_size * length; + chars.resize(new_size); + + const size_t old_rows = offsets.size(); + offsets.resize(old_rows + length); + + for (size_t current_offset = old_size; current_offset < new_size; current_offset += src_buf_size) + memcpySmallAllowReadWriteOverflow15(&chars[current_offset], src_buf, src_buf_size); + + for (size_t i = 0, current_offset = old_size + src_buf_size; i < length; ++i, current_offset += src_buf_size) + offsets[old_rows + i] = current_offset; +} + MutableColumnPtr ColumnString::cloneResized(size_t to_size) const { diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 04aa1849187..2d1d69ced73 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -160,6 +160,8 @@ public: } } + void insertManyFrom(const IColumn & src, size_t position, size_t length) override; + void insertData(const char * pos, size_t length) override { const size_t old_size = chars.size(); From 580fd4ba080df6e29c59b785b1fca0eea76e649c Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Tue, 5 Mar 2024 10:43:48 +0300 Subject: [PATCH 032/334] Update test reference --- .../0_stateless/03000_traverse_shadow_system_data_paths.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh b/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh index a1d4b9bba46..98575540923 100755 --- a/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh +++ b/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sh @@ -26,4 +26,4 @@ ${CLICKHOUSE_CLIENT} --query " FROM system.remote_data_paths WHERE disk_name = 's3_cache' AND local_path LIKE '%shadow/${BACKUP}%' SETTINGS traverse_shadow_remote_data_paths=1;" -${CLICKHOUSE_CLIENT} --query "SYSTEM UNFREEZE WITH NAME '${BACKUP}';" &>/dev/null +${CLICKHOUSE_CLIENT} --query "SYSTEM UNFREEZE WITH NAME '${BACKUP}';" &>/dev/null || true From a109952960acac12790cffde030062ec60208994 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 5 Mar 2024 22:08:36 +0800 Subject: [PATCH 033/334] dev columnstring --- src/Columns/ColumnArray.cpp | 83 +++++++++++++++++++++++++++++++ src/Columns/ColumnArray.h | 9 ++++ src/Columns/ColumnConst.h | 2 + src/Columns/ColumnFixedString.cpp | 14 ++++++ src/Columns/ColumnFixedString.h | 2 + 5 files changed, 110 insertions(+) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 7b268b80116..b620da81ae8 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -347,6 +347,89 @@ void ColumnArray::insertFrom(const IColumn & src_, size_t n) getOffsets().push_back(getOffsets().back() + size); } +template +void ColumnArray::insertManyFromNumber(const ColumnArray & src, size_t position, size_t length) +{ + using ColVecType = ColumnVectorOrDecimal; + size_t src_size = src.sizeAt(position); + size_t src_offset = src.offsetAt(position); + + const typename ColVecType::Container & src_data = typeid_cast(src.getData()).getData(); + typename ColVecType::Container & data_ref = typeid_cast(getData()).getData(); + size_t old_size = data_ref.size(); + size_t new_size = old_size + src_size * length; + data_ref.resize(new_size); + for (size_t i = 0, offset = old_size; i < length; ++i, offset += src_size) + memcpy(&data_ref[offset], &src_data[src_offset], src_size * sizeof(T)); +} + +void ColumnArray::insertManyFromString(const ColumnArray & src, size_t position, size_t length) +{ + size_t src_size = src.sizeAt(position); + size_t src_offset = src.offsetAt(position); + + const auto & src_string = typeid_cast(src.getData()); + const auto & src_chars = src_string.getChars(); + const auto & src_string_offsets = src_string.getOffsets(); + auto & dst_string = typeid_cast(getData()); + auto & dst_chars = dst_string.getChars(); + auto & dst_string_offsets = dst_string.getOffsets(); + + /// Each row may have multiple strings, copy them to dst_chars and update dst_offsets + size_t old_size = dst_string_offsets.size(); + size_t new_size = old_size + src_size * length; + dst_string_offsets.resize(new_size); + size_t dst_string_offset = dst_chars.size(); + for (size_t i = 0; i < length; ++i) + { + for (size_t j = 0; j < src_size; ++j) + { + size_t nested_offset = src_string_offsets[src_offset + j - 1]; + size_t nested_length = src_string_offsets[src_offset + j] - nested_offset; + + dst_string_offset += nested_length; + dst_string_offsets[old_size + i * src_size + j] = dst_string_offset; + } + } + + size_t chars_to_copy = src_string_offsets[src_offset + src_size - 1] - src_string_offsets[src_offset - 1]; + dst_chars.resize(dst_chars.size() + chars_to_copy * length); + for (size_t dst_offset = old_size; dst_offset < new_size; dst_offset += src_size) + memcpy(&dst_chars[dst_string_offsets[dst_offset - 1]], &src_chars[src_string_offsets[src_offset - 1]], chars_to_copy); +} + +void ColumnArray::insertManyFromTuple(const ColumnArray & src, size_t position, size_t length) +{ + +} +void ColumnArray::insertManyFromNullable(const ColumnArray & src, size_t position, size_t length) +{ + +} +void ColumnArray::insertManyFromGeneric(const ColumnArray & src, size_t position, size_t length) +{ + size_t src_size = src.sizeAt(position); + size_t src_offset = src.offsetAt(position); + const auto & src_data = src.getData(); + size_t new_size = data->size() + src_size * length; + data->reserve(new_size); + for (size_t i = 0; i < length; ++i) + data->insertRangeFrom(src_data, src_offset, src_size); +} + +void ColumnArray::insertManyFrom(const IColumn & src_, size_t position, size_t length) +{ + /// First fill offsets + const ColumnArray & src = assert_cast(src_); + size_t src_size = src.sizeAt(position); + auto & offsets_ref = getOffsets(); + size_t old_rows = offsets_ref.size(); + size_t new_rows = old_rows + length; + size_t old_size = offsets_ref.back(); + offsets_ref.resize(new_rows); + for (size_t i = 0, offset = old_size + src_size; i < length; ++i, offset += src_size) + offsets_ref[old_rows + i] = offset; +} void ColumnArray::insertDefault() { diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 230d8830265..73d632a38b9 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -88,6 +88,7 @@ public: void insert(const Field & x) override; bool tryInsert(const Field & x) override; void insertFrom(const IColumn & src_, size_t n) override; + void insertManyFrom(const IColumn & src, size_t position, size_t length) override; void insertDefault() override; void popBack(size_t n) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; @@ -213,6 +214,14 @@ private: ColumnPtr filterNullable(const Filter & filt, ssize_t result_size_hint) const; ColumnPtr filterGeneric(const Filter & filt, ssize_t result_size_hint) const; + /// Specializations for insertManyFrom + template + void insertManyFromNumber(const ColumnArray & src, size_t position, size_t length); + void insertManyFromString(const ColumnArray & src, size_t position, size_t length); + void insertManyFromTuple(const ColumnArray & src, size_t position, size_t length); + void insertManyFromNullable(const ColumnArray & src, size_t position, size_t length); + void insertManyFromGeneric(const ColumnArray & src, size_t position, size_t length); + int compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint, const Collator * collator=nullptr) const; }; diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index 990b7189fa3..4a3d40ca0d2 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -150,6 +150,8 @@ public: ++s; } + void insertManyFrom(const IColumn & /*src*/, size_t /* position */, size_t length) override { s += length; } + void insertDefault() override { ++s; diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index e460c84d696..b55f68d4687 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -85,6 +85,20 @@ void ColumnFixedString::insertFrom(const IColumn & src_, size_t index) memcpySmallAllowReadWriteOverflow15(chars.data() + old_size, &src.chars[n * index], n); } +void ColumnFixedString::insertManyFrom(const IColumn & src, size_t position, size_t length) +{ + const ColumnFixedString & src_concrete = assert_cast(src); + if (n != src_concrete.getN()) + throw Exception(ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH, "Size of FixedString doesn't match"); + + const size_t old_size = chars.size(); + const size_t new_size = old_size + n * length; + chars.resize(new_size); + + for (size_t offset = old_size; offset < new_size; offset += n) + memcpySmallAllowReadWriteOverflow15(&chars[offset], &src_concrete.chars[n * position], n); +} + void ColumnFixedString::insertData(const char * pos, size_t length) { if (length > n) diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index f40e1356b27..56d42e8b34e 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -100,6 +100,8 @@ public: void insertFrom(const IColumn & src_, size_t index) override; + void insertManyFrom(const IColumn & src, size_t position, size_t length) override; + void insertData(const char * pos, size_t length) override; void insertDefault() override From 53c9d4513c4b93ed79df305bb5c36c0cfb43ef79 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 6 Mar 2024 12:16:17 +0800 Subject: [PATCH 034/334] finish dev column array --- src/Columns/ColumnArray.cpp | 132 +++++++++++++++++++++++++++++++++--- src/Columns/ColumnArray.h | 3 + 2 files changed, 125 insertions(+), 10 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index b620da81ae8..aa0d5aa3e50 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -31,6 +31,7 @@ namespace ErrorCodes extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int LOGICAL_ERROR; extern const int TOO_LARGE_ARRAY_SIZE; + extern const int ILLEGAL_COLUMN; } /** Obtaining array as Field can be slow for large arrays and consume vast amount of memory. @@ -363,6 +364,19 @@ void ColumnArray::insertManyFromNumber(const ColumnArray & src, size_t position, memcpy(&data_ref[offset], &src_data[src_offset], src_size * sizeof(T)); } +void ColumnArray::insertManyFromConst(const ColumnConst & src, size_t position, size_t length) +{ + const ColumnArray * src_array = typeid_cast(&src.getDataColumn()); + if (!src_array) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Cannot insert from const column of type {} to column of type {}", + src.getDataColumn().getName(), + getName()); + + insertManyFromImpl(*src_array, 0, length, true); +} + void ColumnArray::insertManyFromString(const ColumnArray & src, size_t position, size_t length) { size_t src_size = src.sizeAt(position); @@ -400,12 +414,53 @@ void ColumnArray::insertManyFromString(const ColumnArray & src, size_t position, void ColumnArray::insertManyFromTuple(const ColumnArray & src, size_t position, size_t length) { + ColumnTuple & tuple = assert_cast(getData()); + const ColumnTuple & src_tuple = assert_cast(src.getData()); + /// Make temporary arrays for each components of Tuple. In the same way as for Nullable. + size_t tuple_size = tuple.tupleSize(); + size_t src_tuple_size = src_tuple.tupleSize(); + if (tuple_size == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty tuple"); + if (tuple_size != src_tuple_size) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Nested tuple size mismatch: {} vs {}", tuple_size, src_tuple_size); + + Columns temporary_arrays(tuple_size); + Columns src_temporary_arrays(tuple_size); + for (size_t i = 0; i < tuple_size; ++i) + { + temporary_arrays[i] = ColumnArray::create(tuple.getColumn(i).assumeMutable(), getOffsetsPtr()->assumeMutable()); + src_temporary_arrays[i] = ColumnArray::create(src_tuple.getColumn(i).assumeMutable(), src.getOffsetsPtr()->assumeMutable()); + assert_cast(*temporary_arrays[i]) + .insertManyFromImpl(assert_cast(*src_temporary_arrays[i]), position, length, false); + } + + Columns tuple_columns(tuple_size); + for (size_t i = 0; i < tuple_size; ++i) + tuple_columns[i] = assert_cast(*temporary_arrays[i]).getDataPtr(); + + getDataPtr() = ColumnTuple::create(std::move(tuple_columns)); } + void ColumnArray::insertManyFromNullable(const ColumnArray & src, size_t position, size_t length) { + ColumnNullable & nullable = assert_cast(getData()); + const ColumnNullable & src_nullable = assert_cast(src.getData()); + /// Process nested column without updating array offsets + auto array_of_nested = ColumnArray(nullable.getNestedColumnPtr()->assumeMutable(), getOffsetsPtr()->assumeMutable()); + auto src_array_of_nested = ColumnArray(src_nullable.getNestedColumnPtr()->assumeMutable(), src.getOffsetsPtr()->assumeMutable()); + array_of_nested.insertManyFromImpl(src_array_of_nested, position, length, false); + + /// Process null map column without updating array offsets + auto array_of_null_map = ColumnArray(nullable.getNullMapColumnPtr()->assumeMutable(), getOffsetsPtr()->assumeMutable()); + auto src_array_of_null_map = ColumnArray(src_nullable.getNullMapColumnPtr()->assumeMutable(), src.getOffsetsPtr()->assumeMutable()); + array_of_null_map.insertManyFromImpl(src_array_of_null_map, position, length, false); + + /// Update array data + getDataPtr() = ColumnNullable::create(array_of_nested.getDataPtr(), array_of_null_map.getDataPtr()); } + void ColumnArray::insertManyFromGeneric(const ColumnArray & src, size_t position, size_t length) { size_t src_size = src.sizeAt(position); @@ -419,16 +474,73 @@ void ColumnArray::insertManyFromGeneric(const ColumnArray & src, size_t position void ColumnArray::insertManyFrom(const IColumn & src_, size_t position, size_t length) { - /// First fill offsets - const ColumnArray & src = assert_cast(src_); - size_t src_size = src.sizeAt(position); - auto & offsets_ref = getOffsets(); - size_t old_rows = offsets_ref.size(); - size_t new_rows = old_rows + length; - size_t old_size = offsets_ref.back(); - offsets_ref.resize(new_rows); - for (size_t i = 0, offset = old_size + src_size; i < length; ++i, offset += src_size) - offsets_ref[old_rows + i] = offset; + const ColumnConst * src_const = typeid_cast(&src_); + if (src_const) + return insertManyFromConst(*src_const, position, length); + + const ColumnArray * src_array = typeid_cast(&src_); + if (!src_array) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert from column of type {} to column of type {}", src_.getName(), getName()); + + return insertManyFromImpl(*src_array, position, length, true); +} + +void ColumnArray::insertManyFromImpl(const ColumnArray & src, size_t position, size_t length, bool update_offsets) +{ + /// First fill offsets if needed + if (update_offsets) + { + size_t src_size = src.sizeAt(position); + auto & offsets_ref = getOffsets(); + size_t old_rows = offsets_ref.size(); + size_t new_rows = old_rows + length; + size_t old_size = offsets_ref.back(); + offsets_ref.resize(new_rows); + for (size_t i = 0, offset = old_size + src_size; i < length; ++i, offset += src_size) + offsets_ref[old_rows + i] = offset; + } + + if (typeid_cast(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast *>(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast *>(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast *>(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast *>(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast *>(data.get())) + return insertManyFromNumber(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromNullable(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromTuple(src, position, length); + return insertManyFromGeneric(src, position, length); } void ColumnArray::insertDefault() diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 73d632a38b9..765f86ec552 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -215,6 +215,9 @@ private: ColumnPtr filterGeneric(const Filter & filt, ssize_t result_size_hint) const; /// Specializations for insertManyFrom + void insertManyFromConst(const ColumnConst & src, size_t position, size_t length); + void insertManyFromImpl(const ColumnArray & src, size_t position, size_t length, bool update_offsets = true); + template void insertManyFromNumber(const ColumnArray & src, size_t position, size_t length); void insertManyFromString(const ColumnArray & src, size_t position, size_t length); From 3bf3c7cc708d1a564896d649a1a804b868f89d8d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 6 Mar 2024 12:32:23 +0800 Subject: [PATCH 035/334] finish column map and tuple --- src/Columns/ColumnArray.cpp | 2 +- src/Columns/ColumnMap.cpp | 5 +++++ src/Columns/ColumnMap.h | 1 + src/Columns/ColumnTuple.cpp | 12 ++++++++++++ src/Columns/ColumnTuple.h | 1 + 5 files changed, 20 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index aa0d5aa3e50..5b0df8e9b6b 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -364,7 +364,7 @@ void ColumnArray::insertManyFromNumber(const ColumnArray & src, size_t position, memcpy(&data_ref[offset], &src_data[src_offset], src_size * sizeof(T)); } -void ColumnArray::insertManyFromConst(const ColumnConst & src, size_t position, size_t length) +void ColumnArray::insertManyFromConst(const ColumnConst & src, size_t /*position*/, size_t length) { const ColumnArray * src_array = typeid_cast(&src.getDataColumn()); if (!src_array) diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 995f3103484..57e8ba685b4 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -158,6 +158,11 @@ void ColumnMap::insertFrom(const IColumn & src, size_t n) nested->insertFrom(assert_cast(src).getNestedColumn(), n); } +void ColumnMap::insertManyFrom(const IColumn & src, size_t position, size_t length) +{ + assert_cast(*nested).insertManyFrom(assert_cast(src).getNestedColumn(), position, length); +} + void ColumnMap::insertRangeFrom(const IColumn & src, size_t start, size_t length) { nested->insertRangeFrom( diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 17cd86a3788..60aa69e7bf6 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -67,6 +67,7 @@ public: void updateWeakHash32(WeakHash32 & hash) const override; void updateHashFast(SipHash & hash) const override; void insertFrom(const IColumn & src_, size_t n) override; + void insertManyFrom(const IColumn & src, size_t position, size_t length) override; void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; void expand(const Filter & mask, bool inverted) override; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 17cc58d92f5..062bdadf9d2 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -185,6 +185,18 @@ void ColumnTuple::insertFrom(const IColumn & src_, size_t n) columns[i]->insertFrom(*src.columns[i], n); } +void ColumnTuple::insertManyFrom(const IColumn & src, size_t position, size_t length) +{ + const ColumnTuple & src_tuple = assert_cast(src); + + const size_t tuple_size = columns.size(); + if (src_tuple.columns.size() != tuple_size) + throw Exception(ErrorCodes::CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE, "Cannot insert value of different size into tuple"); + + for (size_t i = 0; i < tuple_size; ++i) + columns[i]->insertManyFrom(*src_tuple.columns[i], position, length); +} + void ColumnTuple::insertDefault() { for (auto & column : columns) diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 610416b8b11..5b626155754 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -60,6 +60,7 @@ public: void insert(const Field & x) override; bool tryInsert(const Field & x) override; void insertFrom(const IColumn & src_, size_t n) override; + void insertManyFrom(const IColumn & src, size_t position, size_t length) override; void insertDefault() override; void popBack(size_t n) override; StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; From 3005bff23100539dbb71f9623dc3aed9c34a87f6 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 6 Mar 2024 14:43:33 +0800 Subject: [PATCH 036/334] fix building --- src/Columns/ColumnArray.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 5b0df8e9b6b..389b3e97820 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -425,7 +425,7 @@ void ColumnArray::insertManyFromTuple(const ColumnArray & src, size_t position, if (tuple_size != src_tuple_size) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Nested tuple size mismatch: {} vs {}", tuple_size, src_tuple_size); - Columns temporary_arrays(tuple_size); + MutableColumns temporary_arrays(tuple_size); Columns src_temporary_arrays(tuple_size); for (size_t i = 0; i < tuple_size; ++i) { From 8e413da8f156ab03c875b9525044265cffcc5b83 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 6 Mar 2024 17:32:08 +0800 Subject: [PATCH 037/334] apply opts for string nested in array --- src/Columns/ColumnArray.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 389b3e97820..44b17c89ae1 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -536,6 +536,8 @@ void ColumnArray::insertManyFromImpl(const ColumnArray & src, size_t position, s return insertManyFromNumber(src, position, length); if (typeid_cast *>(data.get())) return insertManyFromNumber(src, position, length); + if (typeid_cast(data.get())) + return insertManyFromString(src, position, length); if (typeid_cast(data.get())) return insertManyFromNullable(src, position, length); if (typeid_cast(data.get())) From 2dc1721262c9f483917750aaa6139ff7409e02dc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Mar 2024 11:53:00 +0000 Subject: [PATCH 038/334] Refactorings for consistency --- src/Functions/array/arrayDistance.cpp | 54 +++++++-------- src/Functions/array/arrayDotProduct.cpp | 91 +++++++++++++------------ 2 files changed, 73 insertions(+), 72 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index 71564f6fa93..6b72c99d829 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -379,17 +379,17 @@ public: } -#define SUPPORTED_TYPES(action) \ - action(UInt8) \ - action(UInt16) \ - action(UInt32) \ - action(UInt64) \ - action(Int8) \ - action(Int16) \ - action(Int32) \ - action(Int64) \ - action(Float32) \ - action(Float64) +#define SUPPORTED_TYPES(ACTION) \ + ACTION(UInt8) \ + ACTION(UInt16) \ + ACTION(UInt32) \ + ACTION(UInt64) \ + ACTION(Int8) \ + ACTION(Int16) \ + ACTION(Int32) \ + ACTION(Int64) \ + ACTION(Float32) \ + ACTION(Float64) private: @@ -398,12 +398,11 @@ private: { DataTypePtr type_x = typeid_cast(arguments[0].type.get())->getNestedType(); - /// Dynamic disaptch based on the 1st argument type switch (type_x->getTypeId()) { #define ON_TYPE(type) \ case TypeIndex::type: \ - return executeWithFirstType(arguments, input_rows_count); \ + return executeWithResultTypeAndLeftType(arguments, input_rows_count); \ break; SUPPORTED_TYPES(ON_TYPE) @@ -419,17 +418,16 @@ private: } } - template - ColumnPtr executeWithFirstType(const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const + template + ColumnPtr executeWithResultTypeAndLeftType(const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const { DataTypePtr type_y = typeid_cast(arguments[1].type.get())->getNestedType(); - /// Dynamic disaptch based on the 2nd argument type switch (type_y->getTypeId()) { #define ON_TYPE(type) \ case TypeIndex::type: \ - return executeWithTypes(arguments[0].column, arguments[1].column, input_rows_count, arguments); \ + return executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column, input_rows_count, arguments); \ break; SUPPORTED_TYPES(ON_TYPE) @@ -445,16 +443,16 @@ private: } } - template - ColumnPtr executeWithTypes(ColumnPtr col_x, ColumnPtr col_y, size_t input_rows_count, const ColumnsWithTypeAndName & arguments) const + template + ColumnPtr executeWithResultTypeAndLeftTypeAndRightType(ColumnPtr col_x, ColumnPtr col_y, size_t input_rows_count, const ColumnsWithTypeAndName & arguments) const { if (typeid_cast(col_x.get())) { - return executeWithTypesFirstArgConst(col_x, col_y, input_rows_count, arguments); + return executeWithLeftArgConst(col_x, col_y, input_rows_count, arguments); } else if (typeid_cast(col_y.get())) { - return executeWithTypesFirstArgConst(col_y, col_x, input_rows_count, arguments); + return executeWithLeftArgConst(col_y, col_x, input_rows_count, arguments); } col_x = col_x->convertToFullColumnIfConst(); @@ -463,8 +461,8 @@ private: const auto & array_x = *assert_cast(col_x.get()); const auto & array_y = *assert_cast(col_y.get()); - const auto & data_x = typeid_cast &>(array_x.getData()).getData(); - const auto & data_y = typeid_cast &>(array_y.getData()).getData(); + const auto & data_x = typeid_cast &>(array_x.getData()).getData(); + const auto & data_y = typeid_cast &>(array_y.getData()).getData(); const auto & offsets_x = array_x.getOffsets(); const auto & offsets_y = array_y.getOffsets(); @@ -521,8 +519,8 @@ private: } /// Special case when the 1st parameter is Const - template - ColumnPtr executeWithTypesFirstArgConst(ColumnPtr col_x, ColumnPtr col_y, size_t input_rows_count, const ColumnsWithTypeAndName & arguments) const + template + ColumnPtr executeWithLeftArgConst(ColumnPtr col_x, ColumnPtr col_y, size_t input_rows_count, const ColumnsWithTypeAndName & arguments) const { col_x = assert_cast(col_x.get())->getDataColumnPtr(); col_y = col_y->convertToFullColumnIfConst(); @@ -530,8 +528,8 @@ private: const auto & array_x = *assert_cast(col_x.get()); const auto & array_y = *assert_cast(col_y.get()); - const auto & data_x = typeid_cast &>(array_x.getData()).getData(); - const auto & data_y = typeid_cast &>(array_y.getData()).getData(); + const auto & data_x = typeid_cast &>(array_x.getData()).getData(); + const auto & data_y = typeid_cast &>(array_y.getData()).getData(); const auto & offsets_x = array_x.getOffsets(); const auto & offsets_y = array_y.getOffsets(); @@ -574,7 +572,7 @@ private: /// - the two most common metrics L2 and cosine distance, /// - the most powerful SIMD instruction set (AVX-512F). #if USE_MULTITARGET_CODE - if constexpr (std::is_same_v && std::is_same_v) /// ResultType is Float32 or Float64 + if constexpr (std::is_same_v && std::is_same_v) /// ResultType is Float32 or Float64 { if constexpr (std::is_same_v || std::is_same_v) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 6c615a058c3..548c79c567f 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -163,26 +163,29 @@ public: return Kernel::getReturnType(nested_types[0], nested_types[1]); } +#define SUPPORTED_TYPES(ACTION) \ + ACTION(UInt8) \ + ACTION(UInt16) \ + ACTION(UInt32) \ + ACTION(UInt64) \ + ACTION(Int8) \ + ACTION(Int16) \ + ACTION(Int32) \ + ACTION(Int64) \ + ACTION(Float32) \ + ACTION(Float64) + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override { switch (result_type->getTypeId()) { - #define SUPPORTED_TYPE(type) \ + #define ON_TYPE(type) \ case TypeIndex::type: \ return executeWithResultType(arguments); \ break; - SUPPORTED_TYPE(UInt8) - SUPPORTED_TYPE(UInt16) - SUPPORTED_TYPE(UInt32) - SUPPORTED_TYPE(UInt64) - SUPPORTED_TYPE(Int8) - SUPPORTED_TYPE(Int16) - SUPPORTED_TYPE(Int32) - SUPPORTED_TYPE(Int64) - SUPPORTED_TYPE(Float32) - SUPPORTED_TYPE(Float64) - #undef SUPPORTED_TYPE + SUPPORTED_TYPES(ON_TYPE) + #undef ON_TYPE default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected result type {}", result_type->getName()); @@ -194,16 +197,16 @@ private: ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments) const { ColumnPtr res; - if (!((res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)))) + if (!((res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)))) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); @@ -211,43 +214,43 @@ private: } template - ColumnPtr executeWithResultTypeAndLeft(const ColumnsWithTypeAndName & arguments) const + ColumnPtr executeWithResultTypeAndLeftType(const ColumnsWithTypeAndName & arguments) const { ColumnPtr res; - if ( (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments))) + if ( (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column))) return res; return nullptr; } template - ColumnPtr executeWithResultTypeAndLeftAndRight(const ColumnsWithTypeAndName & arguments) const + ColumnPtr executeWithResultTypeAndLeftTypeAndRightType(ColumnPtr col_x, ColumnPtr col_y) const { - ColumnPtr col_left = arguments[0].column->convertToFullColumnIfConst(); - ColumnPtr col_right = arguments[1].column->convertToFullColumnIfConst(); - if (!col_left || !col_right) + col_x = col_x->convertToFullColumnIfConst(); + col_y = col_y->convertToFullColumnIfConst(); + if (!col_x || !col_y) return nullptr; - const ColumnArray * col_arr_left = checkAndGetColumn(col_left.get()); - const ColumnArray * cokl_arr_right = checkAndGetColumn(col_right.get()); - if (!col_arr_left || !cokl_arr_right) + const ColumnArray * array_x = checkAndGetColumn(col_x.get()); + const ColumnArray * array_y = checkAndGetColumn(col_y.get()); + if (!array_x || !array_y) return nullptr; - const ColumnVector * col_arr_nested_left = checkAndGetColumn>(col_arr_left->getData()); - const ColumnVector * col_arr_nested_right = checkAndGetColumn>(cokl_arr_right->getData()); + const ColumnVector * col_arr_nested_left = checkAndGetColumn>(array_x->getData()); + const ColumnVector * col_arr_nested_right = checkAndGetColumn>(array_y->getData()); if (!col_arr_nested_left || !col_arr_nested_right) return nullptr; - if (!col_arr_left->hasEqualOffsets(*cokl_arr_right)) + if (!array_x->hasEqualOffsets(*array_y)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName()); auto col_res = ColumnVector::create(); @@ -255,7 +258,7 @@ private: vector( col_arr_nested_left->getData(), col_arr_nested_right->getData(), - col_arr_left->getOffsets(), + array_x->getOffsets(), col_res->getData()); return col_res; From 076482e8bd503ca352695173d87e9b48228389f0 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 6 Mar 2024 14:04:09 +0100 Subject: [PATCH 039/334] Remove whitespaces --- src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp index 42c6e63da01..e0d0fda81cb 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp @@ -132,7 +132,7 @@ class OptimizeShardingKeyRewriteIn : public InDepthQueryTreeVisitorWithContext; - + OptimizeShardingKeyRewriteIn(OptimizeShardingKeyRewriteInVisitor::Data data_, ContextPtr context) : Base(std::move(context)) , data(std::move(data_)) From 7065e650e1d007be4659ddb1f070b48e19cdef55 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 6 Mar 2024 14:34:03 +0100 Subject: [PATCH 040/334] Code cleanup --- src/DataTypes/ObjectUtils.cpp | 24 ++------ src/DataTypes/ObjectUtils.h | 2 - .../ClusterProxy/SelectStreamFactory.cpp | 7 +-- .../ClusterProxy/SelectStreamFactory.h | 2 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 61 +++---------------- src/Storages/StorageDistributed.cpp | 1 - tests/analyzer_integration_broken_tests.txt | 1 - .../test_distributed_type_object/test.py | 3 +- 8 files changed, 18 insertions(+), 83 deletions(-) diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 47d8c5c9113..ccfa0a28f13 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -1,4 +1,8 @@ #include +#include +#include +#include +#include #include #include #include @@ -21,16 +25,6 @@ #include #include #include -#include "Analyzer/ConstantNode.h" -#include "Analyzer/FunctionNode.h" -#include "Analyzer/IQueryTreeNode.h" -#include "Analyzer/Identifier.h" -#include "Analyzer/IdentifierNode.h" -#include "Analyzer/QueryNode.h" -#include "Analyzer/Utils.h" -#include -#include -#include "Common/logger_useful.h" namespace DB @@ -991,22 +985,12 @@ MissingObjectList replaceMissedSubcolumnsByConstants( { auto constant = std::make_shared(type->getDefault(), type); constant->setAlias(table_expression->getAlias() + "." + name); - // auto materialize = std::make_shared("materialize"); - - // auto function = FunctionFactory::instance().get("materialize", context); - // materialize->getArguments().getNodes() = { constant }; - // materialize->resolveAsFunction(function->build(materialize->getArgumentColumns())); - // materialize->setAlias(name); column_name_to_node[name] = buildCastFunction(constant, type, context); missed_list.push_back({ constant->getValueStringRepresentation() + "_" + constant->getResultType()->getName(), table_expression->getAlias() + "." + name }); - LOG_DEBUG(&Poco::Logger::get("replaceMissedSubcolumnsByConstants"), "{} -> {}", missed_list.back().first, missed_list.back().second); - LOG_DEBUG(&Poco::Logger::get("replaceMissedSubcolumnsByConstants"), "Name {} Expression\n{}", name, column_name_to_node[name]->dumpTree()); } - LOG_DEBUG(&Poco::Logger::get("replaceMissedSubcolumnsByConstants"), "Table expression\n{} ", table_expression->dumpTree()); replaceColumns(query, table_expression, column_name_to_node); - LOG_DEBUG(&Poco::Logger::get("replaceMissedSubcolumnsByConstants"), "Result:\n{} ", query->dumpTree()); return missed_list; } diff --git a/src/DataTypes/ObjectUtils.h b/src/DataTypes/ObjectUtils.h index 013e525832e..6ef19baf5ae 100644 --- a/src/DataTypes/ObjectUtils.h +++ b/src/DataTypes/ObjectUtils.h @@ -3,8 +3,6 @@ #include #include #include -#include "Analyzer/IQueryTreeNode.h" -#include "Interpreters/Context_fwd.h" #include #include #include diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 5bcd1ce68cb..4fccd83c8c0 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -5,10 +5,9 @@ #include #include #include -#include "Analyzer/IQueryTreeNode.h" -#include "Interpreters/InterpreterSelectQueryAnalyzer.h" -#include "Interpreters/SelectQueryOptions.h" -#include "Planner/Utils.h" +#include +#include +#include #include #include #include diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index bee7edb3c19..61694830b3d 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -8,7 +9,6 @@ #include #include #include -#include "Analyzer/IQueryTreeNode.h" namespace DB { diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index ac507c6d555..72848a37f6e 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -1,4 +1,3 @@ -#include #include #include @@ -15,7 +14,6 @@ #include #include #include -#include "DataTypes/ObjectUtils.h" #include #include #include @@ -33,54 +31,14 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static void addRenamingActions(Pipe & pipe, const MissingObjectList & missed_list, const Block & output_header) -{ - if (missed_list.empty()) - return; - - const auto & output_columns = output_header.getColumnsWithTypeAndName(); - std::vector indexes; - for (size_t i = 0; i < output_columns.size(); ++i) - { - bool found = false; - for (auto const & elem : missed_list) - { - if (output_columns[i].name.contains(elem.second)) - { - found = true; - break; - } - } - if (found) - indexes.push_back(i); - } - - auto dag = std::make_shared(pipe.getHeader().getColumnsWithTypeAndName()); - - for (size_t index : indexes) - { - dag->addOrReplaceInOutputs(dag->addAlias(*dag->getOutputs()[index], output_header.getByPosition(index).name)); - } - - // dag->addAliases(rename_to_apply); - - auto convert_actions = std::make_shared(dag); - pipe.addSimpleTransform([&](const Block & cur_header, Pipe::StreamType) -> ProcessorPtr - { - return std::make_shared(cur_header, convert_actions); - }); - - LOG_DEBUG(&Poco::Logger::get("addRenamingActions"), "EXPECTED:\n{}", output_header.dumpStructure()); - - LOG_DEBUG(&Poco::Logger::get("addRenamingActions"), "{}", pipe.getHeader().dumpStructure()); -} - -static void addConvertingActions(Pipe & pipe, const Block & header) +static void addConvertingActions(Pipe & pipe, const Block & header, bool use_positions_to_match = false) { if (blocksHaveEqualStructure(pipe.getHeader(), header)) return; - auto get_converting_dag = [](const Block & block_, const Block & header_) + auto match_mode = use_positions_to_match ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name; + + auto get_converting_dag = [mode = match_mode](const Block & block_, const Block & header_) { /// Convert header structure to expected. /// Also we ignore constants from result and replace it with constants from header. @@ -88,7 +46,7 @@ static void addConvertingActions(Pipe & pipe, const Block & header) return ActionsDAG::makeConvertingActions( block_.getColumnsWithTypeAndName(), header_.getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name, + mode, true); }; @@ -260,8 +218,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream }; pipes.emplace_back(createDelayedPipe(shard.header, lazily_create_stream, add_totals, add_extremes)); - addRenamingActions(pipes.back(), shard.missing_object_list, output_stream->header); - addConvertingActions(pipes.back(), output_stream->header); + addConvertingActions(pipes.back(), output_stream->header, !shard.missing_object_list.empty()); } void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard) @@ -342,8 +299,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact pipes.emplace_back( createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending)); - addRenamingActions(pipes.back(), shard.missing_object_list, output_stream->header); - addConvertingActions(pipes.back(), output_stream->header); + addConvertingActions(pipes.back(), output_stream->header, !shard.missing_object_list.empty()); } } else @@ -372,8 +328,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact pipes.emplace_back( createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending)); - addRenamingActions(pipes.back(), shard.missing_object_list, output_stream->header); - addConvertingActions(pipes.back(), output_stream->header); + addConvertingActions(pipes.back(), output_stream->header, !shard.missing_object_list.empty()); } } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 34ab21a4751..726f1788115 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -30,7 +30,6 @@ #include #include #include -#include "Analyzer/IQueryTreeNode.h" #include #include diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 796ca6bca22..a7954f91efa 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -1,6 +1,5 @@ test_build_sets_from_multiple_threads/test.py::test_set test_concurrent_backups_s3/test.py::test_concurrent_backups -test_distributed_type_object/test.py::test_distributed_type_object test_merge_table_over_distributed/test.py::test_global_in test_merge_table_over_distributed/test.py::test_select_table_name_from_merge_over_distributed test_passing_max_partitions_to_read_remotely/test.py::test_default_database_on_cluster diff --git a/tests/integration/test_distributed_type_object/test.py b/tests/integration/test_distributed_type_object/test.py index 7e6c000cb8e..360087c9dda 100644 --- a/tests/integration/test_distributed_type_object/test.py +++ b/tests/integration/test_distributed_type_object/test.py @@ -85,10 +85,11 @@ def test_distributed_type_object(started_cluster): 3\t\t\t\tfoo""" ) + # The following query is not supported by analyzer now assert ( TSV( node1.query( - "SELECT id, data.k1, data.k2.k3, data.k2.k4, data.k5 FROM dist_table ORDER BY id" + "SELECT id, data.k1, data.k2.k3, data.k2.k4, data.k5 FROM dist_table ORDER BY id SETTINGS allow_experimental_analyzer = 0" ) ) == expected From c3909743ed95adf5efd16e69f353aab8af73978b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Mar 2024 12:36:00 +0000 Subject: [PATCH 041/334] Remove repeated unnecessary unpacking of const columns --- src/Functions/array/arrayDotProduct.cpp | 88 +++++++++++++------------ 1 file changed, 46 insertions(+), 42 deletions(-) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 548c79c567f..c27170cd35b 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -19,7 +19,6 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int LOGICAL_ERROR; } @@ -196,40 +195,51 @@ private: template ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments) const { - ColumnPtr res; - if (!((res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)))) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); + DataTypePtr type_x = typeid_cast(arguments[0].type.get())->getNestedType(); - return res; + switch (type_x->getTypeId()) + { +#define ON_TYPE(type) \ + case TypeIndex::type: \ + return executeWithResultTypeAndLeftType(arguments); \ + break; + + SUPPORTED_TYPES(ON_TYPE) +#undef ON_TYPE + + default: + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Arguments of function {} has nested type {}. " + "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + getName(), + type_x->getName()); + } } template ColumnPtr executeWithResultTypeAndLeftType(const ColumnsWithTypeAndName & arguments) const { - ColumnPtr res; - if ( (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column))) - return res; + DataTypePtr type_y = typeid_cast(arguments[1].type.get())->getNestedType(); - return nullptr; + switch (type_y->getTypeId()) + { + #define ON_TYPE(type) \ + case TypeIndex::type: \ + return executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column); \ + break; + + SUPPORTED_TYPES(ON_TYPE) + #undef ON_TYPE + + default: + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Arguments of function {} has nested type {}. " + "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + getName(), + type_y->getName()); + } } template @@ -237,28 +247,22 @@ private: { col_x = col_x->convertToFullColumnIfConst(); col_y = col_y->convertToFullColumnIfConst(); - if (!col_x || !col_y) - return nullptr; - const ColumnArray * array_x = checkAndGetColumn(col_x.get()); - const ColumnArray * array_y = checkAndGetColumn(col_y.get()); - if (!array_x || !array_y) - return nullptr; + const auto & array_x = *assert_cast(col_x.get()); + const auto & array_y = *assert_cast(col_y.get()); - const ColumnVector * col_arr_nested_left = checkAndGetColumn>(array_x->getData()); - const ColumnVector * col_arr_nested_right = checkAndGetColumn>(array_y->getData()); - if (!col_arr_nested_left || !col_arr_nested_right) - return nullptr; + const auto & data_x = typeid_cast &>(array_x.getData()).getData(); + const auto & data_y = typeid_cast &>(array_y.getData()).getData(); - if (!array_x->hasEqualOffsets(*array_y)) + if (!array_x.hasEqualOffsets(array_y)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName()); auto col_res = ColumnVector::create(); vector( - col_arr_nested_left->getData(), - col_arr_nested_right->getData(), - array_x->getOffsets(), + data_x, + data_y, + array_x.getOffsets(), col_res->getData()); return col_res; From a69bcc29f5bb0bdaca1757673dac1574c97b1e2f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Mar 2024 14:53:11 +0000 Subject: [PATCH 042/334] Refactor a sanity check --- src/Functions/array/arrayDistance.cpp | 20 +++---------------- src/Functions/array/arrayDotProduct.cpp | 8 +++++--- .../queries/0_stateless/02708_dotProduct.sql | 2 +- 3 files changed, 9 insertions(+), 21 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index 6b72c99d829..c9e6e97749f 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -18,11 +18,11 @@ namespace DB { namespace ErrorCodes { + extern const int ARGUMENT_OUT_OF_BOUND; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int LOGICAL_ERROR; extern const int SIZES_OF_ARRAYS_DONT_MATCH; - extern const int ARGUMENT_OUT_OF_BOUND; } struct L1Distance @@ -465,22 +465,9 @@ private: const auto & data_y = typeid_cast &>(array_y.getData()).getData(); const auto & offsets_x = array_x.getOffsets(); - const auto & offsets_y = array_y.getOffsets(); - /// Check that arrays in both columns are the sames size - for (size_t row = 0; row < offsets_x.size(); ++row) - { - if (offsets_x[row] != offsets_y[row]) [[unlikely]] - { - ColumnArray::Offset prev_offset = row > 0 ? offsets_x[row] : 0; - throw Exception( - ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, - "Arguments of function {} have different array sizes: {} and {}", - getName(), - offsets_x[row] - prev_offset, - offsets_y[row] - prev_offset); - } - } + if (!array_x.hasEqualOffsets(array_y)) + throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Array arguments for function {} must have equal sizes", getName()); const typename Kernel::ConstParams kernel_params = initConstParams(arguments); @@ -534,7 +521,6 @@ private: const auto & offsets_x = array_x.getOffsets(); const auto & offsets_y = array_y.getOffsets(); - /// Check that arrays in both columns are the sames size ColumnArray::Offset prev_offset = 0; for (size_t row : collections::range(0, offsets_y.size())) { diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index c27170cd35b..3abd1a6c6db 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -18,9 +18,9 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int LOGICAL_ERROR; + extern const int SIZES_OF_ARRAYS_DONT_MATCH; } @@ -254,15 +254,17 @@ private: const auto & data_x = typeid_cast &>(array_x.getData()).getData(); const auto & data_y = typeid_cast &>(array_y.getData()).getData(); + const auto & offsets_x = array_x.getOffsets(); + if (!array_x.hasEqualOffsets(array_y)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName()); + throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Array arguments for function {} must have equal sizes", getName()); auto col_res = ColumnVector::create(); vector( data_x, data_y, - array_x.getOffsets(), + offsets_x, col_res->getData()); return col_res; diff --git a/tests/queries/0_stateless/02708_dotProduct.sql b/tests/queries/0_stateless/02708_dotProduct.sql index 6ad615664e8..2035e23cf1d 100644 --- a/tests/queries/0_stateless/02708_dotProduct.sql +++ b/tests/queries/0_stateless/02708_dotProduct.sql @@ -4,7 +4,7 @@ SELECT arrayDotProduct([1, 2]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATC SELECT arrayDotProduct([1, 2], 'abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayDotProduct('abc', [1, 2]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayDotProduct([1, 2], ['abc', 'def']); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT arrayDotProduct([1, 2], [3, 4, 5]); -- { serverError BAD_ARGUMENTS } +SELECT arrayDotProduct([1, 2], [3, 4, 5]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } SELECT dotProduct([1, 2], (3, 4, 5)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT '-- Tests'; From 245ea0c186614a5a13a7e0bed79f94bc6ad46d87 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Mar 2024 15:26:11 +0000 Subject: [PATCH 043/334] Implement const/non-const shortcut --- src/Functions/array/arrayDistance.cpp | 3 +- src/Functions/array/arrayDotProduct.cpp | 110 ++++++++++++++---- tests/performance/dotProduct.xml | 1 + .../0_stateless/02708_dotProduct.reference | 7 ++ .../queries/0_stateless/02708_dotProduct.sql | 9 ++ 5 files changed, 106 insertions(+), 24 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index c9e6e97749f..03f0bc7b286 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -477,6 +477,7 @@ private: /// Do the actual computation ColumnArray::Offset prev = 0; size_t row = 0; + for (auto off : offsets_x) { /// Process chunks in vectorized manner @@ -522,7 +523,7 @@ private: const auto & offsets_y = array_y.getOffsets(); ColumnArray::Offset prev_offset = 0; - for (size_t row : collections::range(0, offsets_y.size())) + for (size_t row = 0; row < offsets_y.size(); ++row) { if (offsets_x[0] != offsets_y[row] - prev_offset) [[unlikely]] { diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 3abd1a6c6db..f9a6687e028 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -245,6 +245,15 @@ private: template ColumnPtr executeWithResultTypeAndLeftTypeAndRightType(ColumnPtr col_x, ColumnPtr col_y) const { + if (typeid_cast(col_x.get())) + { + return executeWithLeftArgConst(col_x, col_y); + } + else if (typeid_cast(col_y.get())) + { + return executeWithLeftArgConst(col_y, col_x); + } + col_x = col_x->convertToFullColumnIfConst(); col_y = col_y->convertToFullColumnIfConst(); @@ -260,30 +269,83 @@ private: throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Array arguments for function {} must have equal sizes", getName()); auto col_res = ColumnVector::create(); + auto & result = col_res->getData(); - vector( - data_x, - data_y, - offsets_x, - col_res->getData()); - - return col_res; - } - - template - static void vector( - const PaddedPODArray & left, - const PaddedPODArray & right, - const ColumnArray::Offsets & offsets, - PaddedPODArray & result) - { - size_t size = offsets.size(); + size_t size = offsets_x.size(); result.resize(size); ColumnArray::Offset current_offset = 0; for (size_t row = 0; row < size; ++row) { - size_t array_size = offsets[row] - current_offset; + size_t array_size = offsets_x[row] - current_offset; + + size_t i = 0; + + /// Process chunks in vectorized manner + static constexpr size_t VEC_SIZE = 4; + typename Kernel::template State states[VEC_SIZE]; + for (; i + VEC_SIZE < array_size; i += VEC_SIZE) + { + for (size_t j = 0; j < VEC_SIZE; ++j) + Kernel::template accumulate(states[j], static_cast(data_x[current_offset + i + j]), static_cast(data_y[current_offset + i + j])); + } + + typename Kernel::template State state; + for (const auto & other_state : states) + Kernel::template combine(state, other_state); + + /// Process the tail + for (; i < array_size; ++i) + Kernel::template accumulate(state, static_cast(data_x[current_offset + i]), static_cast(data_y[current_offset + i])); + + result[row] = Kernel::template finalize(state); + + current_offset = offsets_x[row]; + } + + return col_res; + } + + template + ColumnPtr executeWithLeftArgConst(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 & data_x = typeid_cast &>(array_x.getData()).getData(); + const auto & data_y = typeid_cast &>(array_y.getData()).getData(); + + const auto & offsets_x = array_x.getOffsets(); + const auto & offsets_y = array_y.getOffsets(); + + ColumnArray::Offset prev_offset = 0; + for (size_t row = 0; row < offsets_y.size(); ++row) + { + if (offsets_x[0] != offsets_y[row] - prev_offset) [[unlikely]] + { + throw Exception( + ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, + "Arguments of function {} have different array sizes: {} and {}", + getName(), + offsets_x[0], + offsets_y[row] - prev_offset); + } + prev_offset = offsets_y[row]; + } + + auto col_res = ColumnVector::create(); + auto & result = col_res->getData(); + + size_t size = offsets_y.size(); + result.resize(size); + + ColumnArray::Offset current_offset = 0; + for (size_t row = 0; row < size; ++row) + { + size_t array_size = offsets_x[0]; typename Kernel::template State state; size_t i = 0; @@ -292,13 +354,14 @@ private: /// To avoid combinatorial explosion of SIMD kernels, focus on /// - the two most common input/output types (Float32 x Float32) --> Float32 and (Float64 x Float64) --> Float64 instead of 10 x /// 10 input types x 8 output types, + /// - const/non-const inputs instead of non-const/non-const inputs /// - the most powerful SIMD instruction set (AVX-512F). #if USE_MULTITARGET_CODE if constexpr ((std::is_same_v || std::is_same_v) && std::is_same_v && std::is_same_v) { if (isArchSupported(TargetArch::AVX512F)) - Kernel::template accumulateCombine(&left[current_offset], &right[current_offset], array_size, i, state); + Kernel::template accumulateCombine(&data_x[0], &data_y[current_offset], array_size, i, state); } #else /// Process chunks in vectorized manner @@ -307,7 +370,7 @@ private: for (; i + VEC_SIZE < array_size; i += VEC_SIZE) { for (size_t j = 0; j < VEC_SIZE; ++j) - Kernel::template accumulate(states[j], static_cast(left[i + j]), static_cast(right[i + j])); + Kernel::template accumulate(states[j], static_cast(data_x[i + j]), static_cast(data_y[current_offset + i + j])); } for (const auto & other_state : states) @@ -316,13 +379,14 @@ private: /// Process the tail for (; i < array_size; ++i) - Kernel::template accumulate(state, static_cast(left[i]), static_cast(right[i])); + Kernel::template accumulate(state, static_cast(data_x[i]), static_cast(data_y[current_offset + i])); - /// ResultType res = Kernel::template finalize(state); result[row] = Kernel::template finalize(state); - current_offset = offsets[row]; + current_offset = offsets_y[row]; } + + return col_res; } }; diff --git a/tests/performance/dotProduct.xml b/tests/performance/dotProduct.xml index 6e056964ebb..a0ab7beea9c 100644 --- a/tests/performance/dotProduct.xml +++ b/tests/performance/dotProduct.xml @@ -56,6 +56,7 @@ SELECT sum(dp) FROM (SELECT dotProduct(v, v) AS dp FROM vecs_{element_type}) + WITH (SELECT v FROM vecs_{element_type} limit 1) AS a SELECT sum(dp) FROM (SELECT dotProduct(a, v) AS dp FROM vecs_{element_type}) DROP TABLE vecs_{element_type} diff --git a/tests/queries/0_stateless/02708_dotProduct.reference b/tests/queries/0_stateless/02708_dotProduct.reference index 5cc9a9f0502..593071a3521 100644 --- a/tests/queries/0_stateless/02708_dotProduct.reference +++ b/tests/queries/0_stateless/02708_dotProduct.reference @@ -32,3 +32,10 @@ 32 32 32 +-- Tests that trigger special paths +0 61 +1 186 +0 62 +1 187 +0 62 +1 187 diff --git a/tests/queries/0_stateless/02708_dotProduct.sql b/tests/queries/0_stateless/02708_dotProduct.sql index 2035e23cf1d..ac94ecc28d3 100644 --- a/tests/queries/0_stateless/02708_dotProduct.sql +++ b/tests/queries/0_stateless/02708_dotProduct.sql @@ -45,3 +45,12 @@ SELECT '-- Aliases'; SELECT scalarProduct([1, 2, 3], [4, 5, 6]); SELECT scalarProduct((1, 2, 3), (4, 5, 6)); SELECT arrayDotProduct([1, 2, 3], [4, 5, 6]); -- actually no alias but the internal function for arrays + +SELECT '-- Tests that trigger special paths'; +DROP TABLE IF EXISTS tab; +CREATE TABLE tab(id UInt64, vec Array(Float32)) ENGINE = MergeTree ORDER BY id; +INSERT INTO tab VALUES (0, [0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0]) (1, [5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]); +SELECT id, arrayDotProduct(vec, vec) FROM tab ORDER BY id; -- non-const / non-const +SELECT id, arrayDotProduct([5.0, 2.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]::Array(Float32), vec) FROM tab ORDER BY id; -- const / non-const +SELECT id, arrayDotProduct([5.0, 2.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]::Array(Float64), vec) FROM tab ORDER BY id; -- const / non-const +DROP TABLE tab; From 87f3c957c7dc3d16d9967723e30215e12a0b5dc0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 6 Mar 2024 21:16:22 +0100 Subject: [PATCH 044/334] Blind experiment --- base/base/itoa.h | 30 +++++++++++++++++++++++------- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/base/base/itoa.h b/base/base/itoa.h index 513070c99d9..c450090d82f 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -349,16 +349,32 @@ namespace convert template -static inline char * writeUIntText(T x, char * p) +static inline char * writeUIntText(T _x, char * p) { - static_assert(is_unsigned_v); +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wbit-int-extension" + int len = digits10(_x); + static_assert(std::is_same_v || std::is_same_v); + using T_ = std::conditional_t, unsigned __int128, unsigned _BitInt(256)>; +#pragma clang diagnostic pop - int len = digits10(x); - auto * pp = p + len; - while (x >= 100) + T_ x; + T_ hundred(100ULL); + if constexpr (std::is_same_v) { - const auto i = x % 100; - x /= 100; + x = (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); + } + else + { + x = (T_(_x.items[T::_impl::little(3)]) << 192) + (T_(_x.items[T::_impl::little(2)]) << 128) + + (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); + } + + auto * pp = p + len; + while (x >= hundred) + { + const auto i = x % hundred; + x /= hundred; pp -= 2; outTwoDigits(pp, i); } From c192d0b12532060d14934e60164df7ce771d9399 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 6 Mar 2024 22:29:41 +0100 Subject: [PATCH 045/334] impl --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 83 +++++++++++++------ src/Storages/MergeTree/IMergeTreeDataPart.h | 8 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 3 +- .../MergeTree/MergeTreeDataPartCompact.h | 12 +-- .../MergeTree/MergeTreeDataPartWide.cpp | 3 +- .../MergeTree/MergeTreeDataPartWide.h | 4 +- .../MergeTree/MergeTreeIndexGranularity.cpp | 5 +- .../MergeTree/MergeTreeIndexGranularity.h | 2 + .../MergeTreeIndexGranularityInfo.cpp | 14 +++- .../MergeTree/MergeTreeIndexGranularityInfo.h | 4 + 10 files changed, 97 insertions(+), 41 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index a9bdceacef0..3922d5018c6 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -2,38 +2,41 @@ #include #include +#include #include -#include #include +#include +#include #include -#include +#include +#include +#include +#include #include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include @@ -1966,7 +1969,39 @@ void IMergeTreeDataPart::checkConsistencyBase() const } } -void IMergeTreeDataPart::checkConsistency(bool /* require_part_metadata */) const +void IMergeTreeDataPart::checkConsistency(bool require_part_metadata) const +{ + try + { + checkConsistencyBase(); + doCheckConsistency(require_part_metadata); + } + catch (Exception & e) + { + const auto part_state = fmt::format( + "state: {}, is_unexpected_local_part: {}, is_frozen: {}, is_duplicate: {}", + stateString(), + is_unexpected_local_part, + is_frozen, + is_duplicate, + is_temp); + + const auto debug_info = fmt::format( + "columns: {}, getMarkSizeInBytes: {}, getMarksCount: {}, index_granularity_info: [{}], index_granularity: [{}], " + "part_state: [{}]", + columns.toString(), + index_granularity_info.getMarkSizeInBytes(columns.size()), + index_granularity.getMarksCount(), + index_granularity_info.describe(), + index_granularity.describe(), + part_state); + + e.addMessage(debug_info); + e.rethrow(); + } +} + +void IMergeTreeDataPart::doCheckConsistency(bool /* require_part_metadata */) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'checkConsistency' is not implemented for part with type {}", getType().toString()); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 91c559d30c8..209c2d9a256 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -498,7 +498,7 @@ public: void writeChecksums(const MergeTreeDataPartChecksums & checksums_, const WriteSettings & settings); /// Checks the consistency of this data part. - virtual void checkConsistency(bool require_part_metadata) const; + void checkConsistency(bool require_part_metadata) const; /// Checks the consistency of this data part, and check the consistency of its projections (if any) as well. void checkConsistencyWithProjections(bool require_part_metadata) const; @@ -586,8 +586,6 @@ protected: void removeIfNeeded(); - void checkConsistencyBase() const; - /// Fill each_columns_size and total_size with sizes from columns files on /// disk using columns and checksums. virtual void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const = 0; @@ -610,6 +608,8 @@ protected: void initializeIndexGranularityInfo(); + virtual void doCheckConsistency(bool require_part_metadata) const; + private: String mutable_name; mutable MergeTreeDataPartState state{MergeTreeDataPartState::Temporary}; @@ -697,6 +697,8 @@ private: void incrementStateMetric(MergeTreeDataPartState state) const; void decrementStateMetric(MergeTreeDataPartState state) const; + void checkConsistencyBase() const; + /// This ugly flag is needed for debug assertions only mutable bool part_is_probably_removed_from_disk = false; }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 0ecd7abe183..5d4b602b5b8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -157,9 +157,8 @@ std::optional MergeTreeDataPartCompact::getColumnModificationTime(const return getDataPartStorage().getFileLastModified(DATA_FILE_NAME_WITH_EXTENSION).epochTime(); } -void MergeTreeDataPartCompact::checkConsistency(bool require_part_metadata) const +void MergeTreeDataPartCompact::doCheckConsistency(bool require_part_metadata) const { - checkConsistencyBase(); String mrk_file_name = DATA_FILE_NAME + getMarksFileExtension(); if (!checksums.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 35a358b3720..f897bcb0bfd 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -67,14 +67,14 @@ protected: MergeTreeIndexGranularity & index_granularity_, const MergeTreeIndexGranularityInfo & index_granularity_info_, size_t columns_count, const IDataPartStorage & data_part_storage_); -private: - void checkConsistency(bool require_part_metadata) const override; + void doCheckConsistency(bool require_part_metadata) const override; - /// Loads marks index granularity into memory - void loadIndexGranularity() override; + private: + /// Loads marks index granularity into memory + void loadIndexGranularity() override; - /// Compact parts doesn't support per column size, only total size - void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; + /// Compact parts doesn't support per column size, only total size + void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index dc6c1f0019d..0111f1e7b40 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -182,9 +182,8 @@ MergeTreeDataPartWide::~MergeTreeDataPartWide() removeIfNeeded(); } -void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const +void MergeTreeDataPartWide::doCheckConsistency(bool require_part_metadata) const { - checkConsistencyBase(); std::string marks_file_extension = index_granularity_info.mark_type.getFileExtension(); if (!checksums.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 14147c4ad56..508ea16d2d4 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -62,9 +62,9 @@ protected: MergeTreeIndexGranularity & index_granularity_, MergeTreeIndexGranularityInfo & index_granularity_info_, const IDataPartStorage & data_part_storage_, const std::string & any_column_file_name); -private: - void checkConsistency(bool require_part_metadata) const override; + void doCheckConsistency(bool require_part_metadata) const override; +private: /// Loads marks index granularity into memory void loadIndexGranularity() override; diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp index 5fdd0555777..2a45ab1d927 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp @@ -121,5 +121,8 @@ void MergeTreeIndexGranularity::resizeWithFixedGranularity(size_t size, size_t f } } - +std::string MergeTreeIndexGranularity::describe() const +{ + return fmt::format("initialized: {}, marks_rows_partial_sums: [{}]", initialized, fmt::join(marks_rows_partial_sums, ", ")); +} } diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.h b/src/Storages/MergeTree/MergeTreeIndexGranularity.h index f5677995ae0..d67762f7293 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.h @@ -95,6 +95,8 @@ public: /// Add `size` of marks with `fixed_granularity` rows void resizeWithFixedGranularity(size_t size, size_t fixed_granularity); + + std::string describe() const; }; } diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index da89d52a9ff..1ff72a4e36d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -88,6 +88,10 @@ std::string MarkType::getFileExtension() const } } +std::string MarkType::describe() const +{ + return fmt::format("adaptive: {}, compressed: {}, part_type: {}", adaptive, compressed, part_type); +} std::optional MergeTreeIndexGranularityInfo::getMarksTypeFromFilesystem(const IDataPartStorage & data_part_storage) { @@ -132,10 +136,18 @@ size_t MergeTreeIndexGranularityInfo::getMarkSizeInBytes(size_t columns_num) con throw Exception(ErrorCodes::UNKNOWN_PART_TYPE, "Unknown part type"); } +std::string MergeTreeIndexGranularityInfo::describe() const +{ + return fmt::format( + "mark_type: [{}], index_granularity_bytes: {}, fixed_index_granularity: {}", + mark_type.describe(), + index_granularity_bytes, + fixed_index_granularity); +} + size_t getAdaptiveMrkSizeCompact(size_t columns_num) { /// Each mark contains number of rows in granule and two offsets for every column. return sizeof(UInt64) * (columns_num * 2 + 1); } - } diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h index af008866919..85006c3ffde 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h @@ -24,6 +24,8 @@ struct MarkType static bool isMarkFileExtension(std::string_view extension); std::string getFileExtension() const; + std::string describe() const; + bool adaptive = false; bool compressed = false; MergeTreeDataPartType::Value part_type = MergeTreeDataPartType::Unknown; @@ -58,6 +60,8 @@ public: size_t getMarkSizeInBytes(size_t columns_num = 1) const; static std::optional getMarksTypeFromFilesystem(const IDataPartStorage & data_part_storage); + + std::string describe() const; }; constexpr inline auto getNonAdaptiveMrkSizeWide() { return sizeof(UInt64) * 2; } From 2b52583e06056e19df97216f41b81102bca8bd9d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 6 Mar 2024 23:01:42 +0100 Subject: [PATCH 046/334] fix style --- src/Storages/MergeTree/MergeTreeDataPartCompact.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index f897bcb0bfd..8bbec2808d7 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -69,7 +69,7 @@ protected: void doCheckConsistency(bool require_part_metadata) const override; - private: +private: /// Loads marks index granularity into memory void loadIndexGranularity() override; From f2a3ffe9eb79046093e77ed39f2366754e7a8ba2 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 7 Mar 2024 17:14:12 +0800 Subject: [PATCH 047/334] Replace some headers with forward decl --- src/Backups/BackupCoordinationRemote.cpp | 1 + .../NamedCollections/NamedCollectionUtils.cpp | 1 + .../ObjectStorages/ObjectStorageFactory.cpp | 3 +++ src/Formats/ReadSchemaUtils.cpp | 1 + src/Interpreters/DatabaseCatalog.cpp | 1 - src/Interpreters/DatabaseCatalog.h | 10 +++---- src/Processors/QueryPlan/AggregatingStep.cpp | 1 + src/Processors/QueryPlan/CubeStep.cpp | 1 + src/Storages/StorageAzureBlob.cpp | 4 +++ src/Storages/StorageS3.h | 27 ++++++++++--------- 10 files changed, 29 insertions(+), 21 deletions(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 9c509858b2a..b869f890f56 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include diff --git a/src/Common/NamedCollections/NamedCollectionUtils.cpp b/src/Common/NamedCollections/NamedCollectionUtils.cpp index fe0f42467c7..e3ff50f5e3f 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.cpp +++ b/src/Common/NamedCollections/NamedCollectionUtils.cpp @@ -17,6 +17,7 @@ #include #include +#include namespace fs = std::filesystem; diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 4f198be64fe..5fae257e8d4 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -21,6 +21,9 @@ #include #include +#include + +namespace fs = std::filesystem; namespace DB { diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 5badf4301bf..736a35927c3 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index a9fd5c852ba..a5a523b658b 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -26,7 +26,6 @@ #include #include -#include "Interpreters/Context_fwd.h" #include "config.h" #if USE_MYSQL diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 4fe114cc493..6995fc51941 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -1,15 +1,14 @@ #pragma once #include +#include +#include #include #include -#include #include #include -#include "Common/NamePrompter.h" +#include #include -#include "Storages/IStorage.h" -#include "Databases/IDatabase.h" #include #include @@ -23,9 +22,6 @@ #include #include #include -#include - -namespace fs = std::filesystem; namespace DB { diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index f374a7b7b10..a76bacdd97b 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Processors/QueryPlan/CubeStep.cpp b/src/Processors/QueryPlan/CubeStep.cpp index 0c632c346c7..bf2ce148529 100644 --- a/src/Processors/QueryPlan/CubeStep.cpp +++ b/src/Processors/QueryPlan/CubeStep.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 1f0fba99f84..2d4f1db04a1 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -41,6 +41,10 @@ #include #include +#include + +namespace fs = std::filesystem; + using namespace Azure::Storage::Blobs; namespace CurrentMetrics diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 65fb3b51be2..bf81ead0599 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -4,27 +4,28 @@ #if USE_AWS_S3 -#include - #include - -#include -#include - -#include -#include -#include -#include -#include +#include #include +#include +#include #include #include -#include +#include +#include +#include #include +#include #include #include +#include #include -#include +#include +#include + +#include + +namespace fs = std::filesystem; namespace Aws::S3 { From f0a8d8843de5dffae2e1d4476fb119ad34059340 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 7 Mar 2024 00:10:06 +0100 Subject: [PATCH 048/334] Not x86_64 are lagging behind in features --- base/base/itoa.h | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/base/base/itoa.h b/base/base/itoa.h index c450090d82f..a36eecaf1e5 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -351,12 +351,20 @@ namespace convert template static inline char * writeUIntText(T _x, char * p) { -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wbit-int-extension" int len = digits10(_x); static_assert(std::is_same_v || std::is_same_v); - using T_ = std::conditional_t, unsigned __int128, unsigned _BitInt(256)>; + using T_ = std::conditional_t< + std::is_same_v, + unsigned __int128, +#if defined(__x86_64__) +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wbit-int-extension" + unsigned _BitInt(256) #pragma clang diagnostic pop +#else + T +#endif + >; T_ x; T_ hundred(100ULL); @@ -366,8 +374,12 @@ static inline char * writeUIntText(T _x, char * p) } else { +#if defined(__x86_64__) x = (T_(_x.items[T::_impl::little(3)]) << 192) + (T_(_x.items[T::_impl::little(2)]) << 128) + (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); +#else + x = _x; +#endif } auto * pp = p + len; From 66dea5111298abd4301df55b5615d158105fe78f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 7 Mar 2024 12:40:48 +0100 Subject: [PATCH 049/334] fix clang-tidy --- src/Functions/array/arrayDotProduct.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index f9a6687e028..8b7c85e05dd 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -322,18 +322,18 @@ private: const auto & offsets_y = array_y.getOffsets(); ColumnArray::Offset prev_offset = 0; - for (size_t row = 0; row < offsets_y.size(); ++row) + for (auto offset_y : offsets_y) { - if (offsets_x[0] != offsets_y[row] - prev_offset) [[unlikely]] + if (offsets_x[0] != offset_y - prev_offset) [[unlikely]] { throw Exception( ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Arguments of function {} have different array sizes: {} and {}", getName(), offsets_x[0], - offsets_y[row] - prev_offset); + offset_y - prev_offset); } - prev_offset = offsets_y[row]; + prev_offset = offset_y; } auto col_res = ColumnVector::create(); From 930deee699be05398aac334ce9e025d084c68a30 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 7 Mar 2024 22:02:10 +0800 Subject: [PATCH 050/334] fix bugs --- src/Columns/ColumnArray.cpp | 63 ++++++++++++++++++------------------- src/Columns/ColumnArray.h | 2 +- 2 files changed, 32 insertions(+), 33 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 44b17c89ae1..0214375122f 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -43,29 +43,34 @@ namespace ErrorCodes static constexpr size_t max_array_size_as_field = 1000000; -ColumnArray::ColumnArray(MutableColumnPtr && nested_column, MutableColumnPtr && offsets_column) +ColumnArray::ColumnArray(MutableColumnPtr && nested_column, MutableColumnPtr && offsets_column, bool check_offsets) : data(std::move(nested_column)), offsets(std::move(offsets_column)) { - const ColumnOffsets * offsets_concrete = typeid_cast(offsets.get()); - - if (!offsets_concrete) - throw Exception(ErrorCodes::LOGICAL_ERROR, "offsets_column must be a ColumnUInt64"); - - if (!offsets_concrete->empty() && data && !data->empty()) + if (check_offsets) { - Offset last_offset = offsets_concrete->getData().back(); + const ColumnOffsets * offsets_concrete = typeid_cast(offsets.get()); - /// This will also prevent possible overflow in offset. - if (data->size() != last_offset) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "offsets_column has data inconsistent with nested_column. Data size: {}, last offset: {}", - data->size(), last_offset); + if (!offsets_concrete) + throw Exception(ErrorCodes::LOGICAL_ERROR, "offsets_column must be a ColumnUInt64"); + + if (!offsets_concrete->empty() && data && !data->empty()) + { + Offset last_offset = offsets_concrete->getData().back(); + + /// This will also prevent possible overflow in offset. + if (data->size() != last_offset) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "offsets_column has data inconsistent with nested_column. Data size: {}, last offset: {}", + data->size(), + last_offset); + } + + /** NOTE + * Arrays with constant value are possible and used in implementation of higher order functions (see FunctionReplicate). + * But in most cases, arrays with constant value are unexpected and code will work wrong. Use with caution. + */ } - - /** NOTE - * Arrays with constant value are possible and used in implementation of higher order functions (see FunctionReplicate). - * But in most cases, arrays with constant value are unexpected and code will work wrong. Use with caution. - */ } ColumnArray::ColumnArray(MutableColumnPtr && nested_column) @@ -425,20 +430,14 @@ void ColumnArray::insertManyFromTuple(const ColumnArray & src, size_t position, if (tuple_size != src_tuple_size) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Nested tuple size mismatch: {} vs {}", tuple_size, src_tuple_size); - MutableColumns temporary_arrays(tuple_size); - Columns src_temporary_arrays(tuple_size); - for (size_t i = 0; i < tuple_size; ++i) - { - temporary_arrays[i] = ColumnArray::create(tuple.getColumn(i).assumeMutable(), getOffsetsPtr()->assumeMutable()); - src_temporary_arrays[i] = ColumnArray::create(src_tuple.getColumn(i).assumeMutable(), src.getOffsetsPtr()->assumeMutable()); - assert_cast(*temporary_arrays[i]) - .insertManyFromImpl(assert_cast(*src_temporary_arrays[i]), position, length, false); - } - Columns tuple_columns(tuple_size); for (size_t i = 0; i < tuple_size; ++i) - tuple_columns[i] = assert_cast(*temporary_arrays[i]).getDataPtr(); - + { + auto array_of_element = ColumnArray(tuple.getColumn(i).assumeMutable(), getOffsetsPtr()->assumeMutable(), false); + auto src_array_of_element = ColumnArray(src_tuple.getColumn(i).assumeMutable(), src.getOffsetsPtr()->assumeMutable()); + array_of_element.insertManyFromImpl(src_array_of_element, position, length, false); + tuple_columns[i] = array_of_element.getDataPtr(); + } getDataPtr() = ColumnTuple::create(std::move(tuple_columns)); } @@ -448,12 +447,12 @@ void ColumnArray::insertManyFromNullable(const ColumnArray & src, size_t positio const ColumnNullable & src_nullable = assert_cast(src.getData()); /// Process nested column without updating array offsets - auto array_of_nested = ColumnArray(nullable.getNestedColumnPtr()->assumeMutable(), getOffsetsPtr()->assumeMutable()); + auto array_of_nested = ColumnArray(nullable.getNestedColumnPtr()->assumeMutable(), getOffsetsPtr()->assumeMutable(), false); auto src_array_of_nested = ColumnArray(src_nullable.getNestedColumnPtr()->assumeMutable(), src.getOffsetsPtr()->assumeMutable()); array_of_nested.insertManyFromImpl(src_array_of_nested, position, length, false); /// Process null map column without updating array offsets - auto array_of_null_map = ColumnArray(nullable.getNullMapColumnPtr()->assumeMutable(), getOffsetsPtr()->assumeMutable()); + auto array_of_null_map = ColumnArray(nullable.getNullMapColumnPtr()->assumeMutable(), getOffsetsPtr()->assumeMutable(), false); auto src_array_of_null_map = ColumnArray(src_nullable.getNullMapColumnPtr()->assumeMutable(), src.getOffsetsPtr()->assumeMutable()); array_of_null_map.insertManyFromImpl(src_array_of_null_map, position, length, false); diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 765f86ec552..8c4d103e7d0 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -21,7 +21,7 @@ private: friend class COWHelper, ColumnArray>; /** Create an array column with specified values and offsets. */ - ColumnArray(MutableColumnPtr && nested_column, MutableColumnPtr && offsets_column); + ColumnArray(MutableColumnPtr && nested_column, MutableColumnPtr && offsets_column, bool check_offsets = true); /** Create an empty column of arrays with the type of values as in the column `nested_column` */ explicit ColumnArray(MutableColumnPtr && nested_column); From cd9d9018e0db8139e48cb722e9e9685d2a212c8b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 7 Mar 2024 17:15:42 +0100 Subject: [PATCH 051/334] Be able to iterate --- base/base/CMakeLists.txt | 1 + base/base/itoa.cpp | 503 +++++++++++++++++++++++++++++++++++ base/base/itoa.h | 498 +++------------------------------- src/Functions/CMakeLists.txt | 1 + 4 files changed, 540 insertions(+), 463 deletions(-) create mode 100644 base/base/itoa.cpp diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index 548ba01d86a..55d046767b8 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -19,6 +19,7 @@ set (SRCS getPageSize.cpp getThreadId.cpp int8_to_string.cpp + itoa.cpp JSON.cpp mremap.cpp phdr_cache.cpp diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp new file mode 100644 index 00000000000..9fefc9f0f07 --- /dev/null +++ b/base/base/itoa.cpp @@ -0,0 +1,503 @@ +// Based on https://github.com/amdn/itoa and combined with our optimizations +// +//=== itoa.h - Fast integer to ascii conversion --*- C++ -*-// +// +// The MIT License (MIT) +// Copyright (c) 2016 Arturo Martin-de-Nicolas +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included +// in all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +// SOFTWARE. +//===----------------------------------------------------------------------===// + +#include +#include +#include +#include +#include +#include +#include + + +template +int digits10(T x) +{ + if (x < T(10ULL)) + return 1; + if (x < T(100ULL)) + return 2; + if constexpr (sizeof(T) == 1) + return 3; + else + { + if (x < T(1000ULL)) + return 3; + + if (x < T(1000000000000ULL)) + { + if (x < T(100000000ULL)) + { + if (x < T(1000000ULL)) + { + if (x < T(10000ULL)) + return 4; + else + return 5 + (x >= T(100000ULL)); + } + + return 7 + (x >= T(10000000ULL)); + } + + if (x < T(10000000000ULL)) + return 9 + (x >= T(1000000000ULL)); + + return 11 + (x >= T(100000000000ULL)); + } + + return 12 + digits10(x / T(1000000000000ULL)); + } +} + + +namespace +{ + +template +static constexpr T pow10(size_t x) +{ + return x ? 10 * pow10(x - 1) : 1; +} + +// Division by a power of 10 is implemented using a multiplicative inverse. +// This strength reduction is also done by optimizing compilers, but +// presently the fastest results are produced by using the values +// for the multiplication and the shift as given by the algorithm +// described by Agner Fog in "Optimizing Subroutines in Assembly Language" +// +// http://www.agner.org/optimize/optimizing_assembly.pdf +// +// "Integer division by a constant (all processors) +// A floating point number can be divided by a constant by multiplying +// with the reciprocal. If we want to do the same with integers, we have +// to scale the reciprocal by 2n and then shift the product to the right +// by n. There are various algorithms for finding a suitable value of n +// and compensating for rounding errors. The algorithm described below +// was invented by Terje Mathisen, Norway, and not published elsewhere." + +/// Division by constant is performed by: +/// 1. Adding 1 if needed; +/// 2. Multiplying by another constant; +/// 3. Shifting right by another constant. +template +struct Division +{ + static constexpr bool add{add_}; + static constexpr UInt multiplier{multiplier_}; + static constexpr unsigned shift{shift_}; +}; + +/// Select a type with appropriate number of bytes from the list of types. +/// First parameter is the number of bytes requested. Then goes a list of types with 1, 2, 4, ... number of bytes. +/// Example: SelectType<4, uint8_t, uint16_t, uint32_t, uint64_t> will select uint32_t. +template +struct SelectType +{ + using Result = typename SelectType::Result; +}; + +template +struct SelectType<1, T, Ts...> +{ + using Result = T; +}; + + +/// Division by 10^N where N is the size of the type. +template +using DivisionBy10PowN = typename SelectType< + N, + Division, /// divide by 10 + Division, /// divide by 100 + Division, /// divide by 10000 + Division /// divide by 100000000 + >::Result; + +template +using UnsignedOfSize = typename SelectType::Result; + +/// Holds the result of dividing an unsigned N-byte variable by 10^N resulting in +template +struct QuotientAndRemainder +{ + UnsignedOfSize quotient; // quotient with fewer than 2*N decimal digits + UnsignedOfSize remainder; // remainder with at most N decimal digits +}; + +template +QuotientAndRemainder static inline split(UnsignedOfSize value) +{ + constexpr DivisionBy10PowN division; + + UnsignedOfSize quotient = (division.multiplier * (UnsignedOfSize<2 * N>(value) + division.add)) >> division.shift; + UnsignedOfSize remainder = static_cast>(value - quotient * pow10>(N)); + + return {quotient, remainder}; +} + + +static inline char * outDigit(char * p, uint8_t value) +{ + *p = '0' + value; + ++p; + return p; +} + +// Using a lookup table to convert binary numbers from 0 to 99 +// into ascii characters as described by Andrei Alexandrescu in +// https://www.facebook.com/notes/facebook-engineering/three-optimization-tips-for-c/10151361643253920/ + +static const char digits[201] = "00010203040506070809" + "10111213141516171819" + "20212223242526272829" + "30313233343536373839" + "40414243444546474849" + "50515253545556575859" + "60616263646566676869" + "70717273747576777879" + "80818283848586878889" + "90919293949596979899"; + +static inline char * outTwoDigits(char * p, uint8_t value) +{ + memcpy(p, &digits[value * 2], 2); + p += 2; + return p; +} + + +namespace convert +{ +template +static char * head(char * p, UInt u); +template +static char * tail(char * p, UInt u); + +//===----------------------------------------------------------===// +// head: find most significant digit, skip leading zeros +//===----------------------------------------------------------===// + +// "x" contains quotient and remainder after division by 10^N +// quotient is less than 10^N +template +static inline char * head(char * p, QuotientAndRemainder x) +{ + p = head(p, UnsignedOfSize(x.quotient)); + p = tail(p, x.remainder); + return p; +} + +// "u" is less than 10^2*N +template +static inline char * head(char * p, UInt u) +{ + return u < pow10>(N) ? head(p, UnsignedOfSize(u)) : head(p, split(u)); +} + +// recursion base case, selected when "u" is one byte +template <> +inline char * head, 1>(char * p, UnsignedOfSize<1> u) +{ + return u < 10 ? outDigit(p, u) : outTwoDigits(p, u); +} + +//===----------------------------------------------------------===// +// tail: produce all digits including leading zeros +//===----------------------------------------------------------===// + +// recursive step, "u" is less than 10^2*N +template +static inline char * tail(char * p, UInt u) +{ + QuotientAndRemainder x = split(u); + p = tail(p, UnsignedOfSize(x.quotient)); + p = tail(p, x.remainder); + return p; +} + +// recursion base case, selected when "u" is one byte +template <> +inline char * tail, 1>(char * p, UnsignedOfSize<1> u) +{ + return outTwoDigits(p, u); +} + +//===----------------------------------------------------------===// +// large values are >= 10^2*N +// where x contains quotient and remainder after division by 10^N +//===----------------------------------------------------------===// + +template +static inline char * large(char * p, QuotientAndRemainder x) +{ + QuotientAndRemainder y = split(x.quotient); + p = head(p, UnsignedOfSize(y.quotient)); + p = tail(p, y.remainder); + p = tail(p, x.remainder); + return p; +} + +//===----------------------------------------------------------===// +// handle values of "u" that might be >= 10^2*N +// where N is the size of "u" in bytes +//===----------------------------------------------------------===// + +template +static inline char * uitoa(char * p, UInt u) +{ + if (u < pow10>(N)) + return head(p, UnsignedOfSize(u)); + QuotientAndRemainder x = split(u); + + return u < pow10>(2 * N) ? head(p, x) : large(p, x); +} + +// selected when "u" is one byte +template <> +inline char * uitoa, 1>(char * p, UnsignedOfSize<1> u) +{ + if (u < 10) + return outDigit(p, u); + else if (u < 100) + return outTwoDigits(p, u); + else + { + p = outDigit(p, u / 100); + p = outTwoDigits(p, u % 100); + return p; + } +} + +//===----------------------------------------------------------===// +// handle unsigned and signed integral operands +//===----------------------------------------------------------===// + +// itoa: handle unsigned integral operands (selected by SFINAE) +template && std::is_integral_v> * = nullptr> +static inline char * itoa(U u, char * p) +{ + return convert::uitoa(p, u); +} + +// itoa: handle signed integral operands (selected by SFINAE) +template && std::is_integral_v> * = nullptr> +static inline char * itoa(I i, char * p) +{ + // Need "mask" to be filled with a copy of the sign bit. + // If "i" is a negative value, then the result of "operator >>" + // is implementation-defined, though usually it is an arithmetic + // right shift that replicates the sign bit. + // Use a conditional expression to be portable, + // a good optimizing compiler generates an arithmetic right shift + // and avoids the conditional branch. + UnsignedOfSize mask = i < 0 ? ~UnsignedOfSize(0) : 0; + // Now get the absolute value of "i" and cast to unsigned type UnsignedOfSize. + // Cannot use std::abs() because the result is undefined + // in 2's complement systems for the most-negative value. + // Want to avoid conditional branch for performance reasons since + // CPU branch prediction will be ineffective when negative values + // occur randomly. + // Let "u" be "i" cast to unsigned type UnsignedOfSize. + // Subtract "u" from 2*u if "i" is positive or 0 if "i" is negative. + // This yields the absolute value with the desired type without + // using a conditional branch and without invoking undefined or + // implementation defined behavior: + UnsignedOfSize u = ((2 * UnsignedOfSize(i)) & ~mask) - UnsignedOfSize(i); + // Unconditionally store a minus sign when producing digits + // in a forward direction and increment the pointer only if + // the value is in fact negative. + // This avoids a conditional branch and is safe because we will + // always produce at least one digit and it will overwrite the + // minus sign when the value is not negative. + *p = '-'; + p += (mask & 1); + p = convert::uitoa(p, u); + return p; +} +} + + +template +static NO_INLINE char * writeUIntText(T _x, char * p) +{ + static_assert(std::is_same_v || std::is_same_v); + using T_ = std::conditional_t< + std::is_same_v, + unsigned __int128, +#if defined(__x86_64__) +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Wbit-int-extension" + unsigned _BitInt(256) +# pragma clang diagnostic pop +#else + T +#endif + >; + + T_ x; + T_ hundred(100ULL); + if constexpr (std::is_same_v) + { + x = (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); + } + else + { +#if defined(__x86_64__) + x = (T_(_x.items[T::_impl::little(3)]) << 192) + (T_(_x.items[T::_impl::little(2)]) << 128) + + (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); +#else + x = _x; +#endif + } + + int len = digits10(x); + auto * pp = p + len; + while (x >= hundred) + { + const auto i = x % hundred; + x /= hundred; + pp -= 2; + outTwoDigits(pp, i); + } + if (x < 10) + *p = '0' + x; + else + outTwoDigits(p, x); + return p + len; +} + +static ALWAYS_INLINE inline char * writeLeadingMinus(char * pos) +{ + *pos = '-'; + return pos + 1; +} + +template +static ALWAYS_INLINE inline char * writeSIntText(T x, char * pos) +{ + static_assert(std::is_same_v || std::is_same_v); + + using UnsignedT = make_unsigned_t; + static constexpr T min_int = UnsignedT(1) << (sizeof(T) * 8 - 1); + + if (unlikely(x == min_int)) + { + if constexpr (std::is_same_v) + { + const char * res = "-170141183460469231731687303715884105728"; + memcpy(pos, res, strlen(res)); + return pos + strlen(res); + } + else if constexpr (std::is_same_v) + { + const char * res = "-57896044618658097711785492504343953926634992332820282019728792003956564819968"; + memcpy(pos, res, strlen(res)); + return pos + strlen(res); + } + } + + if (x < 0) + { + x = -x; + pos = writeLeadingMinus(pos); + } + return writeUIntText(UnsignedT(x), pos); +} +} + +template +char * itoa(T i, char * p) +{ + return convert::itoa(i, p); +} + +template <> +char * itoa(UInt8 i, char * p) +{ + return convert::itoa(uint8_t(i), p); +} + +template <> +char * itoa(Int8 i, char * p) +{ + return convert::itoa(int8_t(i), p); +} + +template <> +char * itoa(UInt128 i, char * p) +{ + return writeUIntText(i, p); +} + +template <> +char * itoa(Int128 i, char * p) +{ + return writeSIntText(i, p); +} + +template <> +char * itoa(UInt256 i, char * p) +{ + return writeUIntText(i, p); +} + +template <> +char * itoa(Int256 i, char * p) +{ + return writeSIntText(i, p); +} + +#define FOR_MISSING_INTEGER_TYPES(M) \ + M(int8_t) \ + M(uint8_t) \ + M(UInt16) \ + M(UInt32) \ + M(UInt64) \ + M(Int16) \ + M(Int32) \ + M(Int64) + +#define INSTANTIATION(T) template char * itoa(T i, char * p); +FOR_MISSING_INTEGER_TYPES(INSTANTIATION) + +#undef FOR_MISSING_INTEGER_TYPES +#undef INSTANTIATION + + +#define DIGITS_INTEGER_TYPES(M) \ + M(uint8_t) \ + M(UInt8) \ + M(UInt16) \ + M(UInt32) \ + M(UInt64) \ + M(UInt128) \ + M(UInt256) + +#define INSTANTIATION(T) template int digits10(T x); +DIGITS_INTEGER_TYPES(INSTANTIATION) + +#undef DIGITS_INTEGER_TYPES +#undef INSTANTIATION diff --git a/base/base/itoa.h b/base/base/itoa.h index a36eecaf1e5..71603cdeb88 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -1,474 +1,46 @@ #pragma once -// Based on https://github.com/amdn/itoa and combined with our optimizations -// -//=== itoa.h - Fast integer to ascii conversion --*- C++ -*-// -// -// The MIT License (MIT) -// Copyright (c) 2016 Arturo Martin-de-Nicolas -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included -// in all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE -// SOFTWARE. -//===----------------------------------------------------------------------===// - -#include -#include -#include -#include #include +template char * itoa(T i, char * p); -template -inline int digits10(T x) -{ - if (x < 10ULL) - return 1; - if (x < 100ULL) - return 2; - if (x < 1000ULL) - return 3; +template <> char * itoa(UInt8 i, char * p); +template <> char * itoa(Int8 i, char * p); +template <> char * itoa(UInt128 i, char * p); +template <> char * itoa(Int128 i, char * p); +template <> char * itoa(UInt256 i, char * p); +template <> char * itoa(Int256 i, char * p); - if (x < 1000000000000ULL) - { - if (x < 100000000ULL) - { - if (x < 1000000ULL) - { - if (x < 10000ULL) - return 4; - else - return 5 + (x >= 100000ULL); - } +#define FOR_MISSING_INTEGER_TYPES(M) \ + M(int8_t) \ + M(uint8_t) \ + M(UInt16) \ + M(UInt32) \ + M(UInt64) \ + M(Int16) \ + M(Int32) \ + M(Int64) - return 7 + (x >= 10000000ULL); - } +#define INSTANTIATION(T) \ + extern template char * itoa(T i, char * p); +FOR_MISSING_INTEGER_TYPES(INSTANTIATION) - if (x < 10000000000ULL) - return 9 + (x >= 1000000000ULL); - - return 11 + (x >= 100000000000ULL); - } - - return 12 + digits10(x / 1000000000000ULL); -} +#undef FOR_MISSING_INTEGER_TYPES +#undef INSTANTIATION -namespace impl -{ +template int digits10(T x); -template -static constexpr T pow10(size_t x) -{ - return x ? 10 * pow10(x - 1) : 1; -} - -// Division by a power of 10 is implemented using a multiplicative inverse. -// This strength reduction is also done by optimizing compilers, but -// presently the fastest results are produced by using the values -// for the multiplication and the shift as given by the algorithm -// described by Agner Fog in "Optimizing Subroutines in Assembly Language" -// -// http://www.agner.org/optimize/optimizing_assembly.pdf -// -// "Integer division by a constant (all processors) -// A floating point number can be divided by a constant by multiplying -// with the reciprocal. If we want to do the same with integers, we have -// to scale the reciprocal by 2n and then shift the product to the right -// by n. There are various algorithms for finding a suitable value of n -// and compensating for rounding errors. The algorithm described below -// was invented by Terje Mathisen, Norway, and not published elsewhere." - -/// Division by constant is performed by: -/// 1. Adding 1 if needed; -/// 2. Multiplying by another constant; -/// 3. Shifting right by another constant. -template -struct Division -{ - static constexpr bool add{add_}; - static constexpr UInt multiplier{multiplier_}; - static constexpr unsigned shift{shift_}; -}; - -/// Select a type with appropriate number of bytes from the list of types. -/// First parameter is the number of bytes requested. Then goes a list of types with 1, 2, 4, ... number of bytes. -/// Example: SelectType<4, uint8_t, uint16_t, uint32_t, uint64_t> will select uint32_t. -template -struct SelectType -{ - using Result = typename SelectType::Result; -}; - -template -struct SelectType<1, T, Ts...> -{ - using Result = T; -}; - - -/// Division by 10^N where N is the size of the type. -template -using DivisionBy10PowN = typename SelectType -< - N, - Division, /// divide by 10 - Division, /// divide by 100 - Division, /// divide by 10000 - Division /// divide by 100000000 ->::Result; - -template -using UnsignedOfSize = typename SelectType -< - N, - uint8_t, - uint16_t, - uint32_t, - uint64_t, - __uint128_t ->::Result; - -/// Holds the result of dividing an unsigned N-byte variable by 10^N resulting in -template -struct QuotientAndRemainder -{ - UnsignedOfSize quotient; // quotient with fewer than 2*N decimal digits - UnsignedOfSize remainder; // remainder with at most N decimal digits -}; - -template -QuotientAndRemainder static inline split(UnsignedOfSize value) -{ - constexpr DivisionBy10PowN division; - - UnsignedOfSize quotient = (division.multiplier * (UnsignedOfSize<2 * N>(value) + division.add)) >> division.shift; - UnsignedOfSize remainder = static_cast>(value - quotient * pow10>(N)); - - return {quotient, remainder}; -} - - -static inline char * outDigit(char * p, uint8_t value) -{ - *p = '0' + value; - ++p; - return p; -} - -// Using a lookup table to convert binary numbers from 0 to 99 -// into ascii characters as described by Andrei Alexandrescu in -// https://www.facebook.com/notes/facebook-engineering/three-optimization-tips-for-c/10151361643253920/ - -static const char digits[201] = "00010203040506070809" - "10111213141516171819" - "20212223242526272829" - "30313233343536373839" - "40414243444546474849" - "50515253545556575859" - "60616263646566676869" - "70717273747576777879" - "80818283848586878889" - "90919293949596979899"; - -static inline char * outTwoDigits(char * p, uint8_t value) -{ - memcpy(p, &digits[value * 2], 2); - p += 2; - return p; -} - - -namespace convert -{ - template static char * head(char * p, UInt u); - template static char * tail(char * p, UInt u); - - //===----------------------------------------------------------===// - // head: find most significant digit, skip leading zeros - //===----------------------------------------------------------===// - - // "x" contains quotient and remainder after division by 10^N - // quotient is less than 10^N - template - static inline char * head(char * p, QuotientAndRemainder x) - { - p = head(p, UnsignedOfSize(x.quotient)); - p = tail(p, x.remainder); - return p; - } - - // "u" is less than 10^2*N - template - static inline char * head(char * p, UInt u) - { - return u < pow10>(N) - ? head(p, UnsignedOfSize(u)) - : head(p, split(u)); - } - - // recursion base case, selected when "u" is one byte - template <> - inline char * head, 1>(char * p, UnsignedOfSize<1> u) - { - return u < 10 - ? outDigit(p, u) - : outTwoDigits(p, u); - } - - //===----------------------------------------------------------===// - // tail: produce all digits including leading zeros - //===----------------------------------------------------------===// - - // recursive step, "u" is less than 10^2*N - template - static inline char * tail(char * p, UInt u) - { - QuotientAndRemainder x = split(u); - p = tail(p, UnsignedOfSize(x.quotient)); - p = tail(p, x.remainder); - return p; - } - - // recursion base case, selected when "u" is one byte - template <> - inline char * tail, 1>(char * p, UnsignedOfSize<1> u) - { - return outTwoDigits(p, u); - } - - //===----------------------------------------------------------===// - // large values are >= 10^2*N - // where x contains quotient and remainder after division by 10^N - //===----------------------------------------------------------===// - - template - static inline char * large(char * p, QuotientAndRemainder x) - { - QuotientAndRemainder y = split(x.quotient); - p = head(p, UnsignedOfSize(y.quotient)); - p = tail(p, y.remainder); - p = tail(p, x.remainder); - return p; - } - - //===----------------------------------------------------------===// - // handle values of "u" that might be >= 10^2*N - // where N is the size of "u" in bytes - //===----------------------------------------------------------===// - - template - static inline char * uitoa(char * p, UInt u) - { - if (u < pow10>(N)) - return head(p, UnsignedOfSize(u)); - QuotientAndRemainder x = split(u); - - return u < pow10>(2 * N) - ? head(p, x) - : large(p, x); - } - - // selected when "u" is one byte - template <> - inline char * uitoa, 1>(char * p, UnsignedOfSize<1> u) - { - if (u < 10) - return outDigit(p, u); - else if (u < 100) - return outTwoDigits(p, u); - else - { - p = outDigit(p, u / 100); - p = outTwoDigits(p, u % 100); - return p; - } - } - - //===----------------------------------------------------------===// - // handle unsigned and signed integral operands - //===----------------------------------------------------------===// - - // itoa: handle unsigned integral operands (selected by SFINAE) - template && std::is_integral_v> * = nullptr> - static inline char * itoa(U u, char * p) - { - return convert::uitoa(p, u); - } - - // itoa: handle signed integral operands (selected by SFINAE) - template && std::is_integral_v> * = nullptr> - static inline char * itoa(I i, char * p) - { - // Need "mask" to be filled with a copy of the sign bit. - // If "i" is a negative value, then the result of "operator >>" - // is implementation-defined, though usually it is an arithmetic - // right shift that replicates the sign bit. - // Use a conditional expression to be portable, - // a good optimizing compiler generates an arithmetic right shift - // and avoids the conditional branch. - UnsignedOfSize mask = i < 0 ? ~UnsignedOfSize(0) : 0; - // Now get the absolute value of "i" and cast to unsigned type UnsignedOfSize. - // Cannot use std::abs() because the result is undefined - // in 2's complement systems for the most-negative value. - // Want to avoid conditional branch for performance reasons since - // CPU branch prediction will be ineffective when negative values - // occur randomly. - // Let "u" be "i" cast to unsigned type UnsignedOfSize. - // Subtract "u" from 2*u if "i" is positive or 0 if "i" is negative. - // This yields the absolute value with the desired type without - // using a conditional branch and without invoking undefined or - // implementation defined behavior: - UnsignedOfSize u = ((2 * UnsignedOfSize(i)) & ~mask) - UnsignedOfSize(i); - // Unconditionally store a minus sign when producing digits - // in a forward direction and increment the pointer only if - // the value is in fact negative. - // This avoids a conditional branch and is safe because we will - // always produce at least one digit and it will overwrite the - // minus sign when the value is not negative. - *p = '-'; - p += (mask & 1); - p = convert::uitoa(p, u); - return p; - } -} - - -template -static inline char * writeUIntText(T _x, char * p) -{ - int len = digits10(_x); - static_assert(std::is_same_v || std::is_same_v); - using T_ = std::conditional_t< - std::is_same_v, - unsigned __int128, -#if defined(__x86_64__) -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wbit-int-extension" - unsigned _BitInt(256) -#pragma clang diagnostic pop -#else - T -#endif - >; - - T_ x; - T_ hundred(100ULL); - if constexpr (std::is_same_v) - { - x = (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); - } - else - { -#if defined(__x86_64__) - x = (T_(_x.items[T::_impl::little(3)]) << 192) + (T_(_x.items[T::_impl::little(2)]) << 128) + - (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); -#else - x = _x; -#endif - } - - auto * pp = p + len; - while (x >= hundred) - { - const auto i = x % hundred; - x /= hundred; - pp -= 2; - outTwoDigits(pp, i); - } - if (x < 10) - *p = '0' + x; - else - outTwoDigits(p, x); - return p + len; -} - -static inline char * writeLeadingMinus(char * pos) -{ - *pos = '-'; - return pos + 1; -} - -template -static inline char * writeSIntText(T x, char * pos) -{ - static_assert(std::is_same_v || std::is_same_v); - - using UnsignedT = make_unsigned_t; - static constexpr T min_int = UnsignedT(1) << (sizeof(T) * 8 - 1); - - if (unlikely(x == min_int)) - { - if constexpr (std::is_same_v) - { - const char * res = "-170141183460469231731687303715884105728"; - memcpy(pos, res, strlen(res)); - return pos + strlen(res); - } - else if constexpr (std::is_same_v) - { - const char * res = "-57896044618658097711785492504343953926634992332820282019728792003956564819968"; - memcpy(pos, res, strlen(res)); - return pos + strlen(res); - } - } - - if (x < 0) - { - x = -x; - pos = writeLeadingMinus(pos); - } - return writeUIntText(UnsignedT(x), pos); -} - -} - -template -char * itoa(I i, char * p) -{ - return impl::convert::itoa(i, p); -} - -template <> -inline char * itoa(char8_t i, char * p) -{ - return impl::convert::itoa(uint8_t(i), p); -} - -template <> -inline char * itoa(UInt128 i, char * p) -{ - return impl::writeUIntText(i, p); -} - -template <> -inline char * itoa(Int128 i, char * p) -{ - return impl::writeSIntText(i, p); -} - -template <> -inline char * itoa(UInt256 i, char * p) -{ - return impl::writeUIntText(i, p); -} - -template <> -inline char * itoa(Int256 i, char * p) -{ - return impl::writeSIntText(i, p); -} +#define DIGITS_INTEGER_TYPES(M) \ + M(uint8_t) \ + M(UInt8) \ + M(UInt16) \ + M(UInt32) \ + M(UInt64) \ + M(UInt128) \ + M(UInt256) +#define INSTANTIATION(T) \ + extern template int digits10(T x); +DIGITS_INTEGER_TYPES(INSTANTIATION) +#undef DIGITS_INTEGER_TYPES +#undef INSTANTIATION diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index ac3e3671ae0..dea369a508a 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -37,6 +37,7 @@ list (APPEND PUBLIC_LIBS clickhouse_dictionaries_embedded clickhouse_parsers ch_contrib::consistent_hashing + common dbms ch_contrib::metrohash ch_contrib::murmurhash From 0f0ea422f21af8e37aa5c8ef58002d608cde5c77 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 7 Mar 2024 17:05:54 +0000 Subject: [PATCH 052/334] separate limits on number of waiting and executing queries --- .../settings.md | 16 ++- programs/server/Server.cpp | 1 + src/Common/AsyncLoader.cpp | 21 +++- src/Common/AsyncLoader.h | 65 ++++++++++- src/Common/tests/gtest_async_loader.cpp | 66 +++++++++++ src/Core/ServerSettings.h | 1 + src/Interpreters/ProcessList.cpp | 103 +++++++++++++++--- src/Interpreters/ProcessList.h | 41 ++++++- .../System/StorageSystemServerSettings.cpp | 1 + 9 files changed, 293 insertions(+), 22 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 07c9a2b88ab..63fbd9d1964 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -379,6 +379,18 @@ Type: UInt64 Default: 0 +## max_waiting_queries + +Limit on total number of concurrently waiting queries. Execution of a waiting query is blocked while required tables are loading asynchronously (see `async_load_databases`). Note that waiting queries are not counted when `max_concurrent_queries`, `max_concurrent_insert_queries`, `max_concurrent_select_queries`, `max_concurrent_queries_for_user` and `max_concurrent_queries_for_all_users` limits are checked. This correction is done to avoid hitting these limits just after server startup. Zero means unlimited. + +:::note +This setting can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. +::: + +Type: UInt64 + +Default: 0 + ## max_connections Max server connections. @@ -1725,7 +1737,7 @@ Default value: `0.5`. Asynchronous loading of databases and tables. -If `true` all non-system databases with `Ordinary`, `Atomic` and `Replicated` engine will be loaded asynchronously after the ClickHouse server start up. See `system.asynchronous_loader` table, `tables_loader_background_pool_size` and `tables_loader_foreground_pool_size` server settings. Any query that tries to access a table, that is not yet loaded, will wait for exactly this table to be started up. If load job fails, query will rethrow an error (instead of shutting down the whole server in case of `async_load_databases = false`). The table that is waited for by at least one query will be loaded with higher priority. DDL queries on a database will wait for exactly that database to be started up. +If `true` all non-system databases with `Ordinary`, `Atomic` and `Replicated` engine will be loaded asynchronously after the ClickHouse server start up. See `system.asynchronous_loader` table, `tables_loader_background_pool_size` and `tables_loader_foreground_pool_size` server settings. Any query that tries to access a table, that is not yet loaded, will wait for exactly this table to be started up. If load job fails, query will rethrow an error (instead of shutting down the whole server in case of `async_load_databases = false`). The table that is waited for by at least one query will be loaded with higher priority. DDL queries on a database will wait for exactly that database to be started up. Also consider setting a limit `max_waiting_queries` for the total number of waiting queries. If `false`, all databases are loaded when the server starts. @@ -2926,7 +2938,7 @@ Default: 0 ## ignore_empty_sql_security_in_create_view_query {#ignore_empty_sql_security_in_create_view_query} -If true, ClickHouse doesn't write defaults for empty SQL security statement in CREATE VIEW queries. +If true, ClickHouse doesn't write defaults for empty SQL security statement in CREATE VIEW queries. :::note This setting is only necessary for the migration period and will become obsolete in 24.4 diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index a10f47be0b8..336563665a2 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1429,6 +1429,7 @@ try global_context->getProcessList().setMaxSize(new_server_settings.max_concurrent_queries); global_context->getProcessList().setMaxInsertQueriesAmount(new_server_settings.max_concurrent_insert_queries); global_context->getProcessList().setMaxSelectQueriesAmount(new_server_settings.max_concurrent_select_queries); + global_context->getProcessList().setMaxWaitingQueriesAmount(new_server_settings.max_waiting_queries); if (config->has("keeper_server")) global_context->updateKeeperConfiguration(*config); diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 140194e10b4..80e4c72f1c1 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -140,6 +140,11 @@ void LoadJob::finish() finish_time = std::chrono::system_clock::now(); if (waiters > 0) finished.notify_all(); + else + { + on_waiters_increment = {}; + on_waiters_decrement = {}; + } } void LoadJob::scheduled(UInt64 job_id_) @@ -765,11 +770,25 @@ void AsyncLoader::wait(std::unique_lock & job_lock, const LoadJobPtr if (job->load_status != LoadStatus::PENDING) // Shortcut just to avoid incrementing ProfileEvents return; + if (job->on_waiters_increment) + job->on_waiters_increment(job); + + // WARNING: it is important not to throw below this point to avoid `on_waiters_increment` call w/o matching `on_waiters_decrement` call + Stopwatch watch; job->waiters++; job->finished.wait(job_lock, [&] { return job->load_status != LoadStatus::PENDING; }); job->waiters--; ProfileEvents::increment(ProfileEvents::AsyncLoaderWaitMicroseconds, watch.elapsedMicroseconds()); + + if (job->on_waiters_decrement) + job->on_waiters_decrement(job); + + if (job->waiters == 0) + { + job->on_waiters_increment = {}; + job->on_waiters_decrement = {}; + } } bool AsyncLoader::canSpawnWorker(Pool & pool, std::unique_lock &) @@ -859,7 +878,7 @@ void AsyncLoader::worker(Pool & pool) try { current_load_job = job.get(); - SCOPE_EXIT({ current_load_job = nullptr; }); // Note that recursive job execution is not supported + SCOPE_EXIT({ current_load_job = nullptr; }); // Note that recursive job execution is not supported, but jobs can wait one another job->execute(*this, pool_id, job); exception_from_job = {}; } diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index b1b336d24dc..3f81a36aa96 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -59,7 +59,8 @@ enum class LoadStatus class LoadJob : private boost::noncopyable { public: - template + // NOTE: makeLoadJob() helper should be used instead of direct ctor call + template LoadJob(LoadJobSetType && dependencies_, String name_, size_t pool_id_, DFFunc && dependency_failure_, Func && func_) : dependencies(std::forward(dependencies_)) , name(std::move(name_)) @@ -69,6 +70,19 @@ public: , func(std::forward(func_)) {} + // NOTE: makeLoadJob() helper should be used instead of direct ctor call + template + LoadJob(LoadJobSetType && dependencies_, String name_, size_t pool_id_, WIFunc && on_waiters_increment_, WDFunc && on_waiters_decrement_, DFFunc && dependency_failure_, Func && func_) + : dependencies(std::forward(dependencies_)) + , name(std::move(name_)) + , execution_pool_id(pool_id_) + , pool_id(pool_id_) + , on_waiters_increment(std::forward(on_waiters_increment_)) + , on_waiters_decrement(std::forward(on_waiters_decrement_)) + , dependency_failure(std::forward(dependency_failure_)) + , func(std::forward(func_)) + {} + // Current job status. LoadStatus status() const; std::exception_ptr exception() const; @@ -112,6 +126,13 @@ private: std::atomic execution_pool_id; std::atomic pool_id; + // Handlers that is called by every new waiting thread, just before going to sleep. + // If `on_waiters_increment` throws, then wait is canceled, and corresponding `on_waiters_decrement` will never be called. + // It can be used for counting and limits on number of waiters. + // Note that implementations are called under `LoadJob::mutex` and should be fast. + std::function on_waiters_increment; + std::function on_waiters_decrement; + // Handler for failed or canceled dependencies. // If job needs to be canceled on `dependency` failure, then function should set `cancel` to a specific reason. // Note that implementation should be fast and cannot use AsyncLoader, because it is called under `AsyncLoader::mutex`. @@ -140,8 +161,50 @@ void cancelOnDependencyFailure(const LoadJobPtr & self, const LoadJobPtr & depen void ignoreDependencyFailure(const LoadJobPtr & self, const LoadJobPtr & dependency, std::exception_ptr & cancel); template concept LoadJobDependencyFailure = std::invocable; +template concept LoadJobOnWaiters = std::invocable; template concept LoadJobFunc = std::invocable; +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, LoadJobOnWaiters auto && on_waiters_increment, LoadJobOnWaiters auto && on_waiters_decrement, LoadJobDependencyFailure auto && dependency_failure, LoadJobFunc auto && func) +{ + return std::make_shared(std::move(dependencies), std::move(name), 0, on_waiters_increment, on_waiters_decrement, std::forward(dependency_failure), std::forward(func)); +} + +LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, String name, LoadJobOnWaiters auto && on_waiters_increment, LoadJobOnWaiters auto && on_waiters_decrement, LoadJobDependencyFailure auto && dependency_failure, LoadJobFunc auto && func) +{ + return std::make_shared(dependencies, std::move(name), 0, on_waiters_increment, on_waiters_decrement, std::forward(dependency_failure), std::forward(func)); +} + +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, size_t pool_id, String name, LoadJobOnWaiters auto && on_waiters_increment, LoadJobOnWaiters auto && on_waiters_decrement, LoadJobDependencyFailure auto && dependency_failure, LoadJobFunc auto && func) +{ + return std::make_shared(std::move(dependencies), std::move(name), pool_id, on_waiters_increment, on_waiters_decrement, std::forward(dependency_failure), std::forward(func)); +} + +LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, size_t pool_id, String name, LoadJobOnWaiters auto && on_waiters_increment, LoadJobOnWaiters auto && on_waiters_decrement, LoadJobDependencyFailure auto && dependency_failure, LoadJobFunc auto && func) +{ + return std::make_shared(dependencies, std::move(name), pool_id, on_waiters_increment, on_waiters_decrement, std::forward(dependency_failure), std::forward(func)); +} + +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, LoadJobOnWaiters auto && on_waiters_increment, LoadJobOnWaiters auto && on_waiters_decrement, LoadJobFunc auto && func) +{ + return std::make_shared(std::move(dependencies), std::move(name), 0, on_waiters_increment, on_waiters_decrement, cancelOnDependencyFailure, std::forward(func)); +} + +LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, String name, LoadJobOnWaiters auto && on_waiters_increment, LoadJobOnWaiters auto && on_waiters_decrement, LoadJobFunc auto && func) +{ + return std::make_shared(dependencies, std::move(name), 0, on_waiters_increment, on_waiters_decrement, cancelOnDependencyFailure, std::forward(func)); +} + +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, size_t pool_id, String name, LoadJobOnWaiters auto && on_waiters_increment, LoadJobOnWaiters auto && on_waiters_decrement, LoadJobFunc auto && func) +{ + return std::make_shared(std::move(dependencies), std::move(name), pool_id, on_waiters_increment, on_waiters_decrement, cancelOnDependencyFailure, std::forward(func)); +} + +LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, size_t pool_id, String name, LoadJobOnWaiters auto && on_waiters_increment, LoadJobOnWaiters auto && on_waiters_decrement, LoadJobFunc auto && func) +{ + return std::make_shared(dependencies, std::move(name), pool_id, on_waiters_increment, on_waiters_decrement, cancelOnDependencyFailure, std::forward(func)); +} + + LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, LoadJobDependencyFailure auto && dependency_failure, LoadJobFunc auto && func) { return std::make_shared(std::move(dependencies), std::move(name), 0, std::forward(dependency_failure), std::forward(func)); diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index fc2537abcfc..62a27f259cc 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -643,6 +643,72 @@ TEST(AsyncLoader, CustomDependencyFailure) ASSERT_EQ(good_count.load(), 3); } +TEST(AsyncLoader, WaitersLimit) +{ + AsyncLoaderTest t(16); + + std::atomic waiters_total{0}; + int waiters_limit = 5; + auto waiters_inc = [&] (const LoadJobPtr &) { + int value = waiters_total.load(); + while (true) + { + if (value >= waiters_limit) + throw Exception(ErrorCodes::ASYNC_LOAD_FAILED, "Too many waiters: {}", value); + if (waiters_total.compare_exchange_strong(value, value + 1)) + break; + } + }; + auto waiters_dec = [&] (const LoadJobPtr &) { + waiters_total.fetch_sub(1); + }; + + std::barrier sync(2); + t.loader.start(); + + auto job_func = [&] (AsyncLoader &, const LoadJobPtr &) { + sync.arrive_and_wait(); // (A) + }; + + auto job = makeLoadJob({}, "job", waiters_inc, waiters_dec, job_func); + auto task = t.schedule({job}); + + std::atomic failure{0}; + std::atomic success{0}; + std::vector waiters; + waiters.reserve(10); + auto waiter = [&] { + try + { + t.loader.wait(job); + success.fetch_add(1); + } + catch(...) + { + failure.fetch_add(1); + } + }; + + for (int i = 0; i < 10; i++) + waiters.emplace_back(waiter); + + while (failure.load() != 5) + std::this_thread::yield(); + + ASSERT_EQ(job->waitersCount(), 5); + + sync.arrive_and_wait(); // (A) + + for (auto & thread : waiters) + thread.join(); + + ASSERT_EQ(success.load(), 5); + ASSERT_EQ(failure.load(), 5); + ASSERT_EQ(waiters_total.load(), 0); + + t.loader.wait(); +} + TEST(AsyncLoader, TestConcurrency) { AsyncLoaderTest t(10); diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index c82255ec59c..129b1016fca 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -64,6 +64,7 @@ namespace DB M(UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0) \ M(UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0) \ M(UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0) \ + M(UInt64, max_waiting_queries, 0, "Maximum number of concurrently waiting queries blocked due to `async_load_databases`. Note that waiting queries are not considered by `max_concurrent_*queries*` limits. Zero means unlimited.", 0) \ \ M(Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro RAM max ratio. Allows to lower cache size on low-memory systems.", 0) \ M(String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0) \ diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 3bd7b2d4206..f451d561e60 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -83,25 +83,31 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q IAST::QueryKind query_kind = ast->getQueryKind(); const auto queue_max_wait_ms = settings.queue_max_wait_ms.totalMilliseconds(); - if (!is_unlimited_query && max_size && processes.size() >= max_size) + UInt64 waiting_queries = waiting_queries_amount.load(); + if (!is_unlimited_query && max_size && processes.size() >= max_size + waiting_queries) { if (queue_max_wait_ms) LOG_WARNING(getLogger("ProcessList"), "Too many simultaneous queries, will wait {} ms.", queue_max_wait_ms); - if (!queue_max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(queue_max_wait_ms), [&]{ return processes.size() < max_size; })) - throw Exception(ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, "Too many simultaneous queries. Maximum: {}", max_size); + if (!queue_max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(queue_max_wait_ms), + [&]{ waiting_queries = waiting_queries_amount.load(); return processes.size() < max_size + waiting_queries; })) + throw Exception(ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, + "Too many simultaneous queries. Maximum: {}{}", + max_size, waiting_queries == 0 ? "" : fmt::format(", waiting: {}", waiting_queries)); } if (!is_unlimited_query) { QueryAmount amount = getQueryKindAmount(query_kind); - if (max_insert_queries_amount && query_kind == IAST::QueryKind::Insert && amount >= max_insert_queries_amount) + UInt64 waiting_inserts = waiting_insert_queries_amount.load(); + UInt64 waiting_selects = waiting_select_queries_amount.load(); + if (max_insert_queries_amount && query_kind == IAST::QueryKind::Insert && amount >= max_insert_queries_amount + waiting_inserts) throw Exception(ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, - "Too many simultaneous insert queries. Maximum: {}, current: {}", - max_insert_queries_amount, amount); - if (max_select_queries_amount && query_kind == IAST::QueryKind::Select && amount >= max_select_queries_amount) + "Too many simultaneous insert queries. Maximum: {}, current: {}{}", + max_insert_queries_amount, amount, waiting_inserts == 0 ? "" : fmt::format(", waiting: {}", waiting_inserts)); + if (max_select_queries_amount && query_kind == IAST::QueryKind::Select && amount >= max_select_queries_amount + waiting_selects) throw Exception(ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, - "Too many simultaneous select queries. Maximum: {}, current: {}", - max_select_queries_amount, amount); + "Too many simultaneous select queries. Maximum: {}, current: {}{}", + max_select_queries_amount, amount, waiting_selects == 0 ? "" : fmt::format(", waiting: {}", waiting_selects)); } { @@ -124,10 +130,12 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q * once is already processing 50+ concurrent queries (including analysts or any other users). */ + waiting_queries = waiting_queries_amount.load(); if (!is_unlimited_query && settings.max_concurrent_queries_for_all_users - && processes.size() >= settings.max_concurrent_queries_for_all_users) + && processes.size() >= settings.max_concurrent_queries_for_all_users + waiting_queries_amount) throw Exception(ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, "Too many simultaneous queries for all users. " - "Current: {}, maximum: {}", processes.size(), settings.max_concurrent_queries_for_all_users.toString()); + "Current: {}, maximum: {}{}", processes.size(), settings.max_concurrent_queries_for_all_users.toString(), + waiting_queries == 0 ? "" : fmt::format(", waiting: {}", waiting_queries)); } /** Why we use current user? @@ -145,13 +153,15 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q if (user_process_list != user_to_queries.end()) { + UInt64 user_waiting_queries = user_process_list->second.waiting_queries_amount.load(); if (!is_unlimited_query && settings.max_concurrent_queries_for_user - && user_process_list->second.queries.size() >= settings.max_concurrent_queries_for_user) + && user_process_list->second.queries.size() >= settings.max_concurrent_queries_for_user + user_waiting_queries) throw Exception(ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, "Too many simultaneous queries for user {}. " - "Current: {}, maximum: {}", + "Current: {}, maximum: {}{}", client_info.current_user, user_process_list->second.queries.size(), - settings.max_concurrent_queries_for_user.toString()); + settings.max_concurrent_queries_for_user.toString(), + user_waiting_queries == 0 ? "" : fmt::format(", waiting: {}", user_waiting_queries)); auto running_query = user_process_list->second.queries.find(client_info.current_query_id); @@ -745,4 +755,69 @@ ProcessList::QueryAmount ProcessList::getQueryKindAmount(const IAST::QueryKind & return found->second; } +void ProcessList::increaseWaitingQueryAmount(const QueryStatusPtr & status) +{ + UInt64 limit = max_waiting_queries_amount.load(); + UInt64 value = waiting_queries_amount.load(); + while (true) + { + if (value >= limit) + throw Exception(ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, + "Too many simultaneous waiting queries. Maximum: {}, waiting: {}", + limit, value); + if (waiting_queries_amount.compare_exchange_strong(value, value + 1)) + break; + } + + // WARNING: After this point we should not throw, otherwise corresponding `decreaseWaitingQueryAmount` will not be called. + + // Update query kind counters + if (status->query_kind == IAST::QueryKind::Insert) + waiting_insert_queries_amount.fetch_add(1); + if (status->query_kind == IAST::QueryKind::Select) + waiting_select_queries_amount.fetch_add(1); + + // Update per-user counter + status->getUserProcessList()->waiting_queries_amount.fetch_add(1); + + // We have to notify because some queries might be waiting on `have_space` + // and this query leaves its space by transitioning to waiting state + have_space.notify_all(); +} + +void ProcessList::decreaseWaitingQueryAmount(const QueryStatusPtr & status) +{ + if (status->getUserProcessList()->waiting_queries_amount.fetch_sub(1) == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong insert waiting query amount for user: decrease to negative"); + + if (status->query_kind == IAST::QueryKind::Insert && waiting_insert_queries_amount.fetch_sub(1) == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong insert waiting query amount: decrease to negative"); + + if (status->query_kind == IAST::QueryKind::Select && waiting_select_queries_amount.fetch_sub(1) == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong select waiting query amount: decrease to negative"); + + if (waiting_queries_amount.fetch_sub(1) == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong waiting query amount: decrease to negative"); +} + +void ProcessList::incrementWaiters() +{ + ContextPtr context = CurrentThread::getQueryContext(); + QueryStatusPtr status = context->getProcessListElement(); + + // Query became "waiting" with the first thread that waits + if (status->waiting_threads.fetch_add(1) == 0) + increaseWaitingQueryAmount(status); +} + +void ProcessList::decrementWaiters() +{ + ContextPtr context = CurrentThread::getQueryContext(); + QueryStatusPtr status = context->getProcessListElement(); + + // Query became "non-waiting" with the last thread that no longer waits + if (status->waiting_threads.fetch_sub(1) == 1) + decreaseWaitingQueryAmount(status); +} + } diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 1c253f562e8..75350627698 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -42,10 +42,6 @@ class ThreadStatus; class ProcessListEntry; -/** List of currently executing queries. - * Also implements limit on their number. - */ - /** Information of process list element. * To output in SHOW PROCESSLIST query. Does not contain any complex objects, that do something on copy or destructor. */ @@ -114,8 +110,13 @@ protected: /// Including EndOfStream or Exception. std::atomic is_all_data_sent { false }; + /// Number of threads for the query that are waiting for load jobs + std::atomic waiting_threads{0}; + + /// For initialization of ProcessListForUser during process insertion. void setUserProcessList(ProcessListForUser * user_process_list_); /// Be careful using it. For example, queries field of ProcessListForUser could be modified concurrently. + ProcessListForUser * getUserProcessList() { return user_process_list; } const ProcessListForUser * getUserProcessList() const { return user_process_list; } /// Sets an entry in the ProcessList associated with this QueryStatus. @@ -283,6 +284,9 @@ struct ProcessListForUser /// Count network usage for all simultaneously running queries of single user. ThrottlerPtr user_throttler; + /// Number of queries waiting on load jobs + std::atomic waiting_queries_amount{0}; + ProcessListForUserInfo getInfo(bool get_profile_events = false) const; /// Clears MemoryTracker for the user. @@ -341,6 +345,9 @@ protected: }; +/** List of currently executing queries. + * Also implements limit on their number. + */ class ProcessList : public ProcessListBase { public: @@ -399,10 +406,21 @@ protected: /// amount of queries by query kind. QueryKindAmounts query_kind_amounts; + /// limit for waiting queries. 0 means no limit. Otherwise, when limit exceeded, an exception is thrown. + std::atomic max_waiting_queries_amount{0}; + + /// amounts of waiting queries + std::atomic waiting_queries_amount{0}; + std::atomic waiting_insert_queries_amount{0}; + std::atomic waiting_select_queries_amount{0}; + void increaseQueryKindAmount(const IAST::QueryKind & query_kind); void decreaseQueryKindAmount(const IAST::QueryKind & query_kind); QueryAmount getQueryKindAmount(const IAST::QueryKind & query_kind) const; + void increaseWaitingQueryAmount(const QueryStatusPtr & status); + void decreaseWaitingQueryAmount(const QueryStatusPtr & status); + public: using EntryPtr = std::shared_ptr; @@ -458,6 +476,21 @@ public: return max_select_queries_amount; } + void setMaxWaitingQueriesAmount(UInt64 max_waiting_queries_amount_) + { + max_waiting_queries_amount.store(max_waiting_queries_amount_); + // NOTE: We cannot cancel waiting queries when limit is lowered. They have to wait anyways, but new queries will be canceled instead of waiting. + } + + size_t getMaxWaitingQueriesAmount() const + { + return max_waiting_queries_amount.load(); + } + + // Handlers for AsyncLoader waiters + void incrementWaiters(); + void decrementWaiters(); + /// Try call cancel() for input and output streams of query with specified id and user CancellationCode sendCancelToQuery(const String & current_query_id, const String & current_user, bool kill = false); CancellationCode sendCancelToQuery(QueryStatusPtr elem, bool kill = false); diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index f390985546b..bf14f757a19 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -70,6 +70,7 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context {"max_concurrent_queries", {std::to_string(context->getProcessList().getMaxSize()), ChangeableWithoutRestart::Yes}}, {"max_concurrent_insert_queries", {std::to_string(context->getProcessList().getMaxInsertQueriesAmount()), ChangeableWithoutRestart::Yes}}, {"max_concurrent_select_queries", {std::to_string(context->getProcessList().getMaxSelectQueriesAmount()), ChangeableWithoutRestart::Yes}}, + {"max_waiting_queries", {std::to_string(context->getProcessList().getMaxWaitingQueriesAmount()), ChangeableWithoutRestart::Yes}}, {"background_buffer_flush_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, {"background_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, From f4fc65449cc3ace36f33323600fd1a47fbfb9736 Mon Sep 17 00:00:00 2001 From: Peter Date: Fri, 8 Mar 2024 01:20:50 +0800 Subject: [PATCH 053/334] Add another example dataset for presenting usage --- .../example-datasets/tw-weather.md | 293 ++++++++++++++++++ 1 file changed, 293 insertions(+) create mode 100644 docs/en/getting-started/example-datasets/tw-weather.md diff --git a/docs/en/getting-started/example-datasets/tw-weather.md b/docs/en/getting-started/example-datasets/tw-weather.md new file mode 100644 index 00000000000..e5f16c403d5 --- /dev/null +++ b/docs/en/getting-started/example-datasets/tw-weather.md @@ -0,0 +1,293 @@ +--- +slug: /en/getting-started/example-datasets/tw-weather +sidebar_label: Taiwan Historical Weather Datasets +sidebar_position: 1 +description: 131 million rows of weather observation data for the last 128 yrs +--- + +# Taiwan Historical Weather Datasets + +This dataset contains historical meteorological observations measurements for the last 128 years. Each row is a measurement for a point in date time and weather station. + +The origin of this dataset is available [here](https://github.com/Raingel/historical_weather) and the list of weather station numbers can be found [here](https://github.com/Raingel/weather_station_list). + +> The sources of meteorological datasets include the meteorological stations that are established by the Central Weather Administration (station code is beginning with C0, C1, and 4) and the agricultural meteorological stations belonging to the Council of Agriculture (station code other than those mentioned above): + + - StationId + - MeasuredDate, the observation time + - StnPres, the station air pressure + - SeaPres, the sea level pressure + - Td, the dew point temperature + - RH, the relative humidity + - Other elements where available + +## Downloading the data + +- A [pre-processed version](#pre-processed-data) of the data for the ClickHouse, which has been cleaned, re-structured, and enriched. This dataset covers the years from 1896 to 2023. +- [Download the original raw data](#original-raw-data) and convert to the format required by ClickHouse. Users wanting to add their own columns may wish to explore or complete their approaches. + +### Pre-processed data + +The dataset has also been re-structured from a measurement per line to a row per weather station id and measured date, i.e. + +```csv +StationId,MeasuredDate,StnPres,Tx,RH,WS,WD,WSGust,WDGust,Precp,GloblRad,TxSoil0cm,TxSoil5cm,TxSoil20cm,TxSoil50cm,TxSoil100cm,SeaPres,Td,PrecpHour,SunShine,TxSoil10cm,EvapA,Visb,UVI,Cloud Amount,TxSoil30cm,TxSoil200cm,TxSoil300cm,TxSoil500cm,VaporPressure +C0X100,2016-01-01 01:00:00,1022.1,16.1,72,1.1,8.0,,,,,,,,,,,,,,,,,,,,,,, +C0X100,2016-01-01 02:00:00,1021.6,16.0,73,1.2,358.0,,,,,,,,,,,,,,,,,,,,,,, +C0X100,2016-01-01 03:00:00,1021.3,15.8,74,1.5,353.0,,,,,,,,,,,,,,,,,,,,,,, +C0X100,2016-01-01 04:00:00,1021.2,15.8,74,1.7,8.0,,,,,,,,,,,,,,,,,,,,,,, +``` + +It is easy to query and ensure that the resulting table has less sparse and some elements are null because they're not available to be measured in this weather station. + +This dataset is available in the following Google CloudStorage location. Either download the dataset to your local filesystem (and insert them with the ClickHouse client) or insert them directly into the ClickHouse (see [Inserting from URL](#inserting-from-url)). + +To download: + +```bash +wget https://storage.googleapis.com/taiwan-weather-observaiton-datasets/preprocessed_weather_daily_1896_2023.tar.gz + +# Option: Validate the checksum +md5sum preprocessed_weather_daily_1896_2023.tar.gz +# Checksum should be equal to: 11b484f5bd9ddafec5cfb131eb2dd008 + +tar -xzvf preprocessed_weather_daily_1896_2023.tar.gz +daily_weather_preprocessed_1896_2023.csv + +# Option: Validate the checksum +md5sum daily_weather_preprocessed_1896_2023.csv +# Checksum should be equal to: 1132248c78195c43d93f843753881754 +``` + +### Original raw data + +The following details are about the steps to download the original raw data to transform and convert as you want. + +#### Download + +To download the original raw data: + +```bash +mkdir tw_raw_weather_data && cd tw_raw_weather_data + +wget https://storage.googleapis.com/taiwan-weather-observaiton-datasets/raw_data_weather_daily_1896_2023.tar.gz + +# Option: Validate the checksum +md5sum raw_data_weather_daily_1896_2023.tar.gz +# Checksum should be equal to: b66b9f137217454d655e3004d7d1b51a + +tar -xzvf raw_data_weather_daily_1896_2023.tar.gz +466920_1928.csv +466920_1929.csv +466920_1930.csv +466920_1931.csv +... + +# Option: Validate the checksum +cat *.csv | md5sum +# Checksum should be equal to: b26db404bf84d4063fac42e576464ce1 +``` + +#### Retrieve the Taiwan weather stations + +```bash +wget -O weather_sta_list.csv https://github.com/Raingel/weather_station_list/raw/main/data/weather_sta_list.csv + +# Option: Convert the UTF-8-BOM to UTF-8 encoding +sed -i '1s/^\xEF\xBB\xBF//' weather_sta_list.csv +``` + +## Create table schema + +Create the MergeTree table in ClickHouse (from the ClickHouse client). + +```bash +CREATE TABLE tw_weather_data ( + StationId String null, + MeasuredDate DateTime64, + StnPres Float64 null, + SeaPres Float64 null, + Tx Float64 null, + Td Float64 null, + RH Float64 null, + WS Float64 null, + WD Float64 null, + WSGust Float64 null, + WDGust Float64 null, + Precp Float64 null, + PrecpHour Float64 null, + SunShine Float64 null, + GloblRad Float64 null, + TxSoil0cm Float64 null, + TxSoil5cm Float64 null, + TxSoil10cm Float64 null, + TxSoil20cm Float64 null, + TxSoil50cm Float64 null, + TxSoil100cm Float64 null, + TxSoil30cm Float64 null, + TxSoil200cm Float64 null, + TxSoil300cm Float64 null, + TxSoil500cm Float64 null, + VaporPressure Float64 null, + UVI Float64 null, + "Cloud Amount" Float64 null, + EvapA Float64 null, + Visb Float64 null +) +ENGINE = MergeTree +ORDER BY (MeasuredDate); +``` + +## Inserting into ClickHouse + +### Inserting from local file + +Data can be inserted from a local file as follows (from the ClickHouse client): + +```sql +INSERT INTO tw_weather_data FROM INFILE '/path/to/daily_weather_preprocessed_1896_2023.csv' +``` + +where `/path/to` represents the specific user path to the local file on the disk. + +And the sample response output is as follows after inserting data into the ClickHouse: + +```response +Query id: 90e4b524-6e14-4855-817c-7e6f98fbeabb + +Ok. +131985329 rows in set. Elapsed: 71.770 sec. Processed 131.99 million rows, 10.06 GB (1.84 million rows/s., 140.14 MB/s.) +Peak memory usage: 583.23 MiB. +``` + +### Inserting from URL + +```sql +INSERT INTO tw_weather_data SELECT * +FROM url('https://storage.googleapis.com/taiwan-weather-observaiton-datasets/daily_weather_preprocessed_1896_2023.csv', 'CSVWithNames') + +``` +To know how to speed this up, please see our blog post on [tuning large data loads](https://clickhouse.com/blog/supercharge-your-clickhouse-data-loads-part2). + +## Check data rows and sizes + +1. Let's see how many rows are inserted: + +```sql +SELECT formatReadableQuantity(count()) +FROM tw_weather_data; +``` + +```response +┌─formatReadableQuantity(count())─┐ +│ 131.99 million │ +└─────────────────────────────────┘ +``` + +2. Let's see how much disk space are used for this table: + +```sql +SELECT + formatReadableSize(sum(bytes)) AS disk_size, + formatReadableSize(sum(data_uncompressed_bytes)) AS uncompressed_size +FROM system.parts +WHERE (`table` = 'tw_weather_data') AND active +``` + +```response +┌─disk_size─┬─uncompressed_size─┐ +│ 2.13 GiB │ 32.94 GiB │ +└───────────┴───────────────────┘ +``` + +## Sample queries + +### Q1: Retrieve the highest dew point temperature for each weather station in the specific year + +```sql +SELECT + StationId, + max(Td) AS max_td +FROM tw_weather_data +WHERE (year(MeasuredDate) = 2023) AND (Td IS NOT NULL) +GROUP BY StationId + +┌─StationId─┬─max_td─┐ +│ 466940 │ 1 │ +│ 467300 │ 1 │ +│ 467540 │ 1 │ +│ 467490 │ 1 │ +│ 467080 │ 1 │ +│ 466910 │ 1 │ +│ 467660 │ 1 │ +│ 467270 │ 1 │ +│ 467350 │ 1 │ +│ 467571 │ 1 │ +│ 466920 │ 1 │ +│ 467650 │ 1 │ +│ 467550 │ 1 │ +│ 467480 │ 1 │ +│ 467610 │ 1 │ +│ 467050 │ 1 │ +│ 467590 │ 1 │ +│ 466990 │ 1 │ +│ 467060 │ 1 │ +│ 466950 │ 1 │ +│ 467620 │ 1 │ +│ 467990 │ 1 │ +│ 466930 │ 1 │ +│ 467110 │ 1 │ +│ 466881 │ 1 │ +│ 467410 │ 1 │ +│ 467441 │ 1 │ +│ 467420 │ 1 │ +│ 467530 │ 1 │ +│ 466900 │ 1 │ +└───────────┴────────┘ + +30 rows in set. Elapsed: 0.045 sec. Processed 6.41 million rows, 187.33 MB (143.92 million rows/s., 4.21 GB/s.) +``` + +### Q2: Raw data fetching with the specific duration time range, fields and weather station + +```sql +SELECT + StnPres, + SeaPres, + Tx, + Td, + RH, + WS, + WD, + WSGust, + WDGust, + Precp, + PrecpHour +FROM tw_weather_data +WHERE (StationId = 'C0UB10') AND (MeasuredDate >= '2023-12-23') AND (MeasuredDate < '2023-12-24') +ORDER BY MeasuredDate ASC +LIMIT 10 +``` + +```response +┌─StnPres─┬─SeaPres─┬───Tx─┬───Td─┬─RH─┬──WS─┬──WD─┬─WSGust─┬─WDGust─┬─Precp─┬─PrecpHour─┐ +│ 1029.5 │ ᴺᵁᴸᴸ │ 11.8 │ ᴺᵁᴸᴸ │ 78 │ 2.7 │ 271 │ 5.5 │ 275 │ -99.8 │ -99.8 │ +│ 1029.8 │ ᴺᵁᴸᴸ │ 12.3 │ ᴺᵁᴸᴸ │ 78 │ 2.7 │ 289 │ 5.5 │ 308 │ -99.8 │ -99.8 │ +│ 1028.6 │ ᴺᵁᴸᴸ │ 12.3 │ ᴺᵁᴸᴸ │ 79 │ 2.3 │ 251 │ 6.1 │ 289 │ -99.8 │ -99.8 │ +│ 1028.2 │ ᴺᵁᴸᴸ │ 13 │ ᴺᵁᴸᴸ │ 75 │ 4.3 │ 312 │ 7.5 │ 316 │ -99.8 │ -99.8 │ +│ 1027.8 │ ᴺᵁᴸᴸ │ 11.1 │ ᴺᵁᴸᴸ │ 89 │ 7.1 │ 310 │ 11.6 │ 322 │ -99.8 │ -99.8 │ +│ 1027.8 │ ᴺᵁᴸᴸ │ 11.6 │ ᴺᵁᴸᴸ │ 90 │ 3.1 │ 269 │ 10.7 │ 295 │ -99.8 │ -99.8 │ +│ 1027.9 │ ᴺᵁᴸᴸ │ 12.3 │ ᴺᵁᴸᴸ │ 89 │ 4.7 │ 296 │ 8.1 │ 310 │ -99.8 │ -99.8 │ +│ 1028.2 │ ᴺᵁᴸᴸ │ 12.2 │ ᴺᵁᴸᴸ │ 94 │ 2.5 │ 246 │ 7.1 │ 283 │ -99.8 │ -99.8 │ +│ 1028.4 │ ᴺᵁᴸᴸ │ 12.5 │ ᴺᵁᴸᴸ │ 94 │ 3.1 │ 265 │ 4.8 │ 297 │ -99.8 │ -99.8 │ +│ 1028.3 │ ᴺᵁᴸᴸ │ 13.6 │ ᴺᵁᴸᴸ │ 91 │ 1.2 │ 273 │ 4.4 │ 256 │ -99.8 │ -99.8 │ +└─────────┴─────────┴──────┴──────┴────┴─────┴─────┴────────┴────────┴───────┴───────────┘ + +10 rows in set. Elapsed: 0.009 sec. Processed 91.70 thousand rows, 2.33 MB (9.67 million rows/s., 245.31 MB/s.) +``` + +## Credits + +We would like to acknowledge the efforts of the Central Weather Administration and Agricultural Meteorological Observation Network (Station) of the Council of Agriculture for preparing, cleaning, and distributing this dataset. We appreciate your efforts. + +Ou, J.-H., Kuo, C.-H., Wu, Y.-F., Lin, G.-C., Lee, M.-H., Chen, R.-K., Chou, H.-P., Wu, H.-Y., Chu, S.-C., Lai, Q.-J., Tsai, Y.-C., Lin, C.-C., Kuo, C.-C., Liao, C.-T., Chen, Y.-N., Chu, Y.-W., Chen, C.-Y., 2023. Application-oriented deep learning model for early warning of rice blast in Taiwan. Ecological Informatics 73, 101950. https://doi.org/10.1016/j.ecoinf.2022.101950 [13/12/2022] From 38cbc2c6c40541cc916bc591cd68b7eef70b1162 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 7 Mar 2024 18:57:16 +0100 Subject: [PATCH 054/334] Restore digits --- base/base/itoa.cpp | 43 +++++++++++++++++++------------------------ 1 file changed, 19 insertions(+), 24 deletions(-) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index 9fefc9f0f07..ef844ff68a8 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -34,42 +34,37 @@ template -int digits10(T x) +inline int digits10(T x) { - if (x < T(10ULL)) + if (x < 10ULL) return 1; - if (x < T(100ULL)) + if (x < 100ULL) return 2; - if constexpr (sizeof(T) == 1) + if (x < 1000ULL) return 3; - else + + if (x < 1000000000000ULL) { - if (x < T(1000ULL)) - return 3; - - if (x < T(1000000000000ULL)) + if (x < 100000000ULL) { - if (x < T(100000000ULL)) + if (x < 1000000ULL) { - if (x < T(1000000ULL)) - { - if (x < T(10000ULL)) - return 4; - else - return 5 + (x >= T(100000ULL)); - } - - return 7 + (x >= T(10000000ULL)); + if (x < 10000ULL) + return 4; + else + return 5 + (x >= 100000ULL); } - if (x < T(10000000000ULL)) - return 9 + (x >= T(1000000000ULL)); - - return 11 + (x >= T(100000000000ULL)); + return 7 + (x >= 10000000ULL); } - return 12 + digits10(x / T(1000000000000ULL)); + if (x < 10000000000ULL) + return 9 + (x >= 1000000000ULL); + + return 11 + (x >= 100000000000ULL); } + + return 12 + digits10(x / 1000000000000ULL); } From 444595ac576438c9d0a259debf776187ddd3fcce Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 7 Mar 2024 20:00:07 +0100 Subject: [PATCH 055/334] More speed please Mr. compiler --- base/base/itoa.cpp | 165 +++++++++++++++++++++++++++++++-------------- 1 file changed, 114 insertions(+), 51 deletions(-) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index ef844ff68a8..08912edf3ea 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -72,7 +72,7 @@ namespace { template -static constexpr T pow10(size_t x) +ALWAYS_INLINE inline constexpr T pow10(size_t x) { return x ? 10 * pow10(x - 1) : 1; } @@ -143,7 +143,7 @@ struct QuotientAndRemainder }; template -QuotientAndRemainder static inline split(UnsignedOfSize value) +QuotientAndRemainder inline split(UnsignedOfSize value) { constexpr DivisionBy10PowN division; @@ -154,7 +154,7 @@ QuotientAndRemainder static inline split(UnsignedOfSize value) } -static inline char * outDigit(char * p, uint8_t value) +ALWAYS_INLINE inline char * outDigit(char * p, uint8_t value) { *p = '0' + value; ++p; @@ -176,7 +176,7 @@ static const char digits[201] = "00010203040506070809" "80818283848586878889" "90919293949596979899"; -static inline char * outTwoDigits(char * p, uint8_t value) +ALWAYS_INLINE inline char * outTwoDigits(char * p, uint8_t value) { memcpy(p, &digits[value * 2], 2); p += 2; @@ -187,9 +187,9 @@ static inline char * outTwoDigits(char * p, uint8_t value) namespace convert { template -static char * head(char * p, UInt u); +char * head(char * p, UInt u); template -static char * tail(char * p, UInt u); +char * tail(char * p, UInt u); //===----------------------------------------------------------===// // head: find most significant digit, skip leading zeros @@ -198,7 +198,7 @@ static char * tail(char * p, UInt u); // "x" contains quotient and remainder after division by 10^N // quotient is less than 10^N template -static inline char * head(char * p, QuotientAndRemainder x) +ALWAYS_INLINE inline char * head(char * p, QuotientAndRemainder x) { p = head(p, UnsignedOfSize(x.quotient)); p = tail(p, x.remainder); @@ -207,14 +207,14 @@ static inline char * head(char * p, QuotientAndRemainder x) // "u" is less than 10^2*N template -static inline char * head(char * p, UInt u) +ALWAYS_INLINE inline char * head(char * p, UInt u) { return u < pow10>(N) ? head(p, UnsignedOfSize(u)) : head(p, split(u)); } // recursion base case, selected when "u" is one byte template <> -inline char * head, 1>(char * p, UnsignedOfSize<1> u) +ALWAYS_INLINE inline char * head, 1>(char * p, UnsignedOfSize<1> u) { return u < 10 ? outDigit(p, u) : outTwoDigits(p, u); } @@ -225,7 +225,7 @@ inline char * head, 1>(char * p, UnsignedOfSize<1> u) // recursive step, "u" is less than 10^2*N template -static inline char * tail(char * p, UInt u) +ALWAYS_INLINE inline char * tail(char * p, UInt u) { QuotientAndRemainder x = split(u); p = tail(p, UnsignedOfSize(x.quotient)); @@ -235,7 +235,7 @@ static inline char * tail(char * p, UInt u) // recursion base case, selected when "u" is one byte template <> -inline char * tail, 1>(char * p, UnsignedOfSize<1> u) +ALWAYS_INLINE inline char * tail, 1>(char * p, UnsignedOfSize<1> u) { return outTwoDigits(p, u); } @@ -244,9 +244,8 @@ inline char * tail, 1>(char * p, UnsignedOfSize<1> u) // large values are >= 10^2*N // where x contains quotient and remainder after division by 10^N //===----------------------------------------------------------===// - template -static inline char * large(char * p, QuotientAndRemainder x) +ALWAYS_INLINE inline char * large(char * p, QuotientAndRemainder x) { QuotientAndRemainder y = split(x.quotient); p = head(p, UnsignedOfSize(y.quotient)); @@ -259,9 +258,8 @@ static inline char * large(char * p, QuotientAndRemainder x) // handle values of "u" that might be >= 10^2*N // where N is the size of "u" in bytes //===----------------------------------------------------------===// - template -static inline char * uitoa(char * p, UInt u) +ALWAYS_INLINE inline char * uitoa(char * p, UInt u) { if (u < pow10>(N)) return head(p, UnsignedOfSize(u)); @@ -272,7 +270,7 @@ static inline char * uitoa(char * p, UInt u) // selected when "u" is one byte template <> -inline char * uitoa, 1>(char * p, UnsignedOfSize<1> u) +ALWAYS_INLINE inline char * uitoa, 1>(char * p, UnsignedOfSize<1> u) { if (u < 10) return outDigit(p, u); @@ -292,14 +290,14 @@ inline char * uitoa, 1>(char * p, UnsignedOfSize<1> u) // itoa: handle unsigned integral operands (selected by SFINAE) template && std::is_integral_v> * = nullptr> -static inline char * itoa(U u, char * p) +ALWAYS_INLINE inline char * itoa(U u, char * p) { return convert::uitoa(p, u); } // itoa: handle signed integral operands (selected by SFINAE) template && std::is_integral_v> * = nullptr> -static inline char * itoa(I i, char * p) +ALWAYS_INLINE inline char * itoa(I i, char * p) { // Need "mask" to be filled with a copy of the sign bit. // If "i" is a negative value, then the result of "operator >>" @@ -335,63 +333,128 @@ static inline char * itoa(I i, char * p) } -template -static NO_INLINE char * writeUIntText(T _x, char * p) +const uint64_t max_multiple_of_hundred_that_fits_in_64_bits = 1'00'00'00'00'00'00'00'00'00ull; +constexpr int max_multiple_of_hundred_blocks = 9; +static_assert(max_multiple_of_hundred_that_fits_in_64_bits % 100 == 0); + +ALWAYS_INLINE inline char * writeUIntText(UInt128 _x, char * p) { - static_assert(std::is_same_v || std::is_same_v); - using T_ = std::conditional_t< - std::is_same_v, - unsigned __int128, + /// If we the highest 8 byte item is empty, we can print only the lowest item as i64 + if (_x.items[UInt128::_impl::little(1)] == 0) + return convert::itoa(_x.items[UInt128::_impl::little(0)], p); + + /// Doing operations using __int128 is faster, as we already rely on this feature + using T = unsigned __int128; + T x = (T(_x.items[UInt128::_impl::little(1)]) << 64) + T(_x.items[UInt128::_impl::little(0)]); + + /// We are going to accumulate blocks of 2 digits to print until the number is small enough to be printed as u64 + /// To do this we could do: x / 100, x % 100 + /// But this is too many iterations with long integers, so instead we can divide by a much longer integer + /// max_multiple_of_hundred_that_fits_in_64_bits and then get the blocks out of this (as u64) + static const T large_divisor = max_multiple_of_hundred_that_fits_in_64_bits; + static const T largest_uint64 = std::numeric_limits::max(); + uint8_t two_values[20] = {0}; // 39 Max characters / 2 + + int current_block = 0; + while (x > largest_uint64) + { + uint64_t remainder = uint64_t(x % large_divisor); + x /= large_divisor; + + int pos = current_block; + while (remainder) + { + two_values[pos] = uint8_t(remainder % 100); + pos++; + remainder /= 100; + } + current_block += max_multiple_of_hundred_blocks; + } + + char * highest_part_print = convert::itoa(uint64_t(x), p); + for (int i = 0; i < current_block; i++) + { + outTwoDigits(highest_part_print, two_values[current_block - 1 - i]); + highest_part_print += 2; + } + + return highest_part_print; +} + +ALWAYS_INLINE inline char * writeUIntText(UInt256 _x, char * p) +{ + /// If possible, treat it as a smaller integer as they are much faster to print + if (_x.items[UInt256::_impl::little(3)] == 0 && _x.items[UInt256::_impl::little(2)] == 0) + return writeUIntText(UInt128{_x.items[UInt256::_impl::little(0)], _x.items[UInt256::_impl::little(1)]}, p); + + /// If available (x86) we transform from our custom class to _BitInt(256) which has better support in the compiler + /// and produces better code + using T = #if defined(__x86_64__) # pragma clang diagnostic push # pragma clang diagnostic ignored "-Wbit-int-extension" unsigned _BitInt(256) # pragma clang diagnostic pop #else - T + UInt256 #endif - >; + ; - T_ x; - T_ hundred(100ULL); - if constexpr (std::is_same_v) - { - x = (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); - } - else - { #if defined(__x86_64__) - x = (T_(_x.items[T::_impl::little(3)]) << 192) + (T_(_x.items[T::_impl::little(2)]) << 128) - + (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); + T x = (T(_x.items[UInt256::_impl::little(3)]) << 192) + (T(_x.items[UInt256::_impl::little(2)]) << 128) + + (T(_x.items[UInt256::_impl::little(1)]) << 64) + T(_x.items[UInt256::_impl::little(0)]); #else - x = _x; + T x = _x; #endif + + /// Similar to writeUIntText(UInt128) only that in this case we will stop as soon as we reach the largest u128 + /// and switch to that function + uint8_t two_values[39] = {0}; // 78 Max characters / 2 + int current_pos = 0; + + static const T large_divisor = max_multiple_of_hundred_that_fits_in_64_bits; + static const T largest_uint128 = T(std::numeric_limits::max()) << 64 | T(std::numeric_limits::max()); + + while (x > largest_uint128) + { + uint64_t remainder = uint64_t(x % large_divisor); + x /= large_divisor; + + int pos = current_pos; + while (remainder) + { + two_values[pos] = uint8_t(remainder % 100); + pos++; + remainder /= 100; + } + current_pos += max_multiple_of_hundred_blocks; } - int len = digits10(x); - auto * pp = p + len; - while (x >= hundred) +#if defined(__x86_64__) + UInt128 pending{uint64_t(x), uint64_t(x >> 64)}; +#else + UInt128 pending{x.items[UInt256::_impl::little(0)], x.items[UInt256::_impl::little(1)]}; +#endif + + char * highest_part_print = writeUIntText(pending, p); + for (int i = 0; i < current_pos; i++) { - const auto i = x % hundred; - x /= hundred; - pp -= 2; - outTwoDigits(pp, i); + outTwoDigits(highest_part_print, two_values[current_pos - 1 - i]); + highest_part_print += 2; } - if (x < 10) - *p = '0' + x; - else - outTwoDigits(p, x); - return p + len; + + return highest_part_print; } -static ALWAYS_INLINE inline char * writeLeadingMinus(char * pos) + +ALWAYS_INLINE inline char * writeLeadingMinus(char * pos) { *pos = '-'; return pos + 1; } template -static ALWAYS_INLINE inline char * writeSIntText(T x, char * pos) +ALWAYS_INLINE inline char * writeSIntText(T x, char * pos) { static_assert(std::is_same_v || std::is_same_v); From cde811804f68656577dd1497e511a4a9295d4544 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 8 Mar 2024 12:20:34 +0800 Subject: [PATCH 056/334] prevent memcpySmallAllowReadWriteOverflow15Impl optimized to memcpy --- src/Common/memcpySmall.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/memcpySmall.h b/src/Common/memcpySmall.h index f3d26c60380..f5e9f31fc46 100644 --- a/src/Common/memcpySmall.h +++ b/src/Common/memcpySmall.h @@ -49,6 +49,9 @@ namespace detail dst += 16; src += 16; n -= 16; + + /// Avoid clang loop-idion optimization, which transforms _mm_storeu_si128 to built-in memcpy + __asm__ __volatile__("" : : : "memory"); } } } From 89ae39e598a481dbb2c610ee7dca1fc7272517b7 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 8 Mar 2024 12:22:07 +0800 Subject: [PATCH 057/334] optimize column string replicate --- src/Columns/ColumnString.cpp | 38 ++++++++++++++++++++---------------- 1 file changed, 21 insertions(+), 17 deletions(-) diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index b9128372cea..0c52a7be086 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -460,6 +460,7 @@ void ColumnString::updatePermutationWithCollation(const Collator & collator, Per DefaultPartialSort()); } + ColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const { size_t col_size = size(); @@ -471,32 +472,35 @@ ColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const if (0 == col_size) return res; - Chars & res_chars = res->chars; Offsets & res_offsets = res->offsets; - res_chars.reserve_exact(chars.size() / col_size * replicate_offsets.back()); - res_offsets.reserve_exact(replicate_offsets.back()); - - Offset prev_replicate_offset = 0; - Offset prev_string_offset = 0; - Offset current_new_offset = 0; + res_offsets.resize_exact(replicate_offsets.back()); + Chars & res_chars = res->chars; + size_t res_chars_size = 0; for (size_t i = 0; i < col_size; ++i) { - size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset; - size_t string_size = offsets[i] - prev_string_offset; + size_t size_to_replicate = replicate_offsets[i] - replicate_offsets[i - 1]; + size_t string_size = offsets[i] - offsets[i - 1]; + res_chars_size += size_to_replicate * string_size; + } + res_chars.resize_exact(res_chars_size); + size_t curr_row = 0; + size_t curr_offset = 0; + for (size_t i = 0; i < col_size; ++i) + { + const size_t size_to_replicate = replicate_offsets[i] - replicate_offsets[i - 1]; + const size_t string_size = offsets[i] - offsets[i-1]; + const UInt8 * src = &chars[offsets[i - 1]]; for (size_t j = 0; j < size_to_replicate; ++j) { - current_new_offset += string_size; - res_offsets.push_back(current_new_offset); - - res_chars.resize(res_chars.size() + string_size); memcpySmallAllowReadWriteOverflow15( - &res_chars[res_chars.size() - string_size], &chars[prev_string_offset], string_size); - } + &res_chars[curr_offset], src, string_size); - prev_replicate_offset = replicate_offsets[i]; - prev_string_offset = offsets[i]; + curr_offset += string_size; + res_offsets[curr_row] = curr_offset; + ++curr_row; + } } return res; From 391af00b64e62dc7d9e1c5726c1aa202f19dffd8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Fri, 8 Mar 2024 13:47:35 +0800 Subject: [PATCH 058/334] Update memcpySmall.h --- src/Common/memcpySmall.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/memcpySmall.h b/src/Common/memcpySmall.h index f5e9f31fc46..90648254d76 100644 --- a/src/Common/memcpySmall.h +++ b/src/Common/memcpySmall.h @@ -50,7 +50,7 @@ namespace detail src += 16; n -= 16; - /// Avoid clang loop-idion optimization, which transforms _mm_storeu_si128 to built-in memcpy + /// Avoid clang loop-idiom optimization, which transforms _mm_storeu_si128 to built-in memcpy __asm__ __volatile__("" : : : "memory"); } } From d52027c5a09f0f9619bc2f5df639f1a042b2c084 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 8 Mar 2024 12:00:34 +0100 Subject: [PATCH 059/334] Style and comments --- base/base/itoa.cpp | 47 +++++++++++++++++++++++----------------------- 1 file changed, 24 insertions(+), 23 deletions(-) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index 08912edf3ea..4475ae416b9 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -165,16 +165,16 @@ ALWAYS_INLINE inline char * outDigit(char * p, uint8_t value) // into ascii characters as described by Andrei Alexandrescu in // https://www.facebook.com/notes/facebook-engineering/three-optimization-tips-for-c/10151361643253920/ -static const char digits[201] = "00010203040506070809" - "10111213141516171819" - "20212223242526272829" - "30313233343536373839" - "40414243444546474849" - "50515253545556575859" - "60616263646566676869" - "70717273747576777879" - "80818283848586878889" - "90919293949596979899"; +const char digits[201] = "00010203040506070809" + "10111213141516171819" + "20212223242526272829" + "30313233343536373839" + "40414243444546474849" + "50515253545556575859" + "60616263646566676869" + "70717273747576777879" + "80818283848586878889" + "90919293949596979899"; ALWAYS_INLINE inline char * outTwoDigits(char * p, uint8_t value) { @@ -334,23 +334,24 @@ ALWAYS_INLINE inline char * itoa(I i, char * p) const uint64_t max_multiple_of_hundred_that_fits_in_64_bits = 1'00'00'00'00'00'00'00'00'00ull; -constexpr int max_multiple_of_hundred_blocks = 9; +const int max_multiple_of_hundred_blocks = 9; static_assert(max_multiple_of_hundred_that_fits_in_64_bits % 100 == 0); ALWAYS_INLINE inline char * writeUIntText(UInt128 _x, char * p) { - /// If we the highest 8 byte item is empty, we can print only the lowest item as i64 + /// If we the highest 64bit item is empty, we can print just the lowest item as u64 if (_x.items[UInt128::_impl::little(1)] == 0) return convert::itoa(_x.items[UInt128::_impl::little(0)], p); - /// Doing operations using __int128 is faster, as we already rely on this feature + /// Doing operations using __int128 is faster and we already rely on this feature using T = unsigned __int128; T x = (T(_x.items[UInt128::_impl::little(1)]) << 64) + T(_x.items[UInt128::_impl::little(0)]); /// We are going to accumulate blocks of 2 digits to print until the number is small enough to be printed as u64 /// To do this we could do: x / 100, x % 100 - /// But this is too many iterations with long integers, so instead we can divide by a much longer integer - /// max_multiple_of_hundred_that_fits_in_64_bits and then get the blocks out of this (as u64) + /// But these would mean doing many iterations with long integers, so instead we divide by a much longer integer + /// multiple of 100 (100^9) and then get the blocks out of it (as u64) + /// Once we reach u64::max we can stop and use the fast method to print that in the front static const T large_divisor = max_multiple_of_hundred_that_fits_in_64_bits; static const T largest_uint64 = std::numeric_limits::max(); uint8_t two_values[20] = {0}; // 39 Max characters / 2 @@ -358,15 +359,15 @@ ALWAYS_INLINE inline char * writeUIntText(UInt128 _x, char * p) int current_block = 0; while (x > largest_uint64) { - uint64_t remainder = uint64_t(x % large_divisor); + uint64_t u64_remainder = uint64_t(x % large_divisor); x /= large_divisor; int pos = current_block; - while (remainder) + while (u64_remainder) { - two_values[pos] = uint8_t(remainder % 100); + two_values[pos] = uint8_t(u64_remainder % 100); pos++; - remainder /= 100; + u64_remainder /= 100; } current_block += max_multiple_of_hundred_blocks; } @@ -417,15 +418,15 @@ ALWAYS_INLINE inline char * writeUIntText(UInt256 _x, char * p) while (x > largest_uint128) { - uint64_t remainder = uint64_t(x % large_divisor); + uint64_t u64_remainder = uint64_t(x % large_divisor); x /= large_divisor; int pos = current_pos; - while (remainder) + while (u64_remainder) { - two_values[pos] = uint8_t(remainder % 100); + two_values[pos] = uint8_t(u64_remainder % 100); pos++; - remainder /= 100; + u64_remainder /= 100; } current_pos += max_multiple_of_hundred_blocks; } From 2f3c103367de4fab57602e3fc1821608df718c77 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 8 Mar 2024 12:02:53 +0100 Subject: [PATCH 060/334] OSX quirks --- base/base/itoa.cpp | 4 ++++ base/base/itoa.h | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index 4475ae416b9..d877f15b563 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -542,6 +542,10 @@ char * itoa(Int256 i, char * p) #define INSTANTIATION(T) template char * itoa(T i, char * p); FOR_MISSING_INTEGER_TYPES(INSTANTIATION) +#if defined(OS_DARWIN) +INSTANTIATION(size_t) +#endif + #undef FOR_MISSING_INTEGER_TYPES #undef INSTANTIATION diff --git a/base/base/itoa.h b/base/base/itoa.h index 71603cdeb88..98a570b12fa 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -25,6 +25,10 @@ template <> char * itoa(Int256 i, char * p); extern template char * itoa(T i, char * p); FOR_MISSING_INTEGER_TYPES(INSTANTIATION) +#if defined(OS_DARWIN) +INSTANTIATION(size_t) +#endif + #undef FOR_MISSING_INTEGER_TYPES #undef INSTANTIATION From e7cc49212a15ca8bcf87950225e066d04c8823e4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 8 Mar 2024 12:28:14 +0100 Subject: [PATCH 061/334] fix tidy build --- src/Functions/array/arrayDistance.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index 03f0bc7b286..0045075ddef 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -523,18 +523,18 @@ private: const auto & offsets_y = array_y.getOffsets(); ColumnArray::Offset prev_offset = 0; - for (size_t row = 0; row < offsets_y.size(); ++row) + for (auto offset_y : offsets_y) { - if (offsets_x[0] != offsets_y[row] - prev_offset) [[unlikely]] + if (offsets_x[0] != offset_y - prev_offset) [[unlikely]] { throw Exception( ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Arguments of function {} have different array sizes: {} and {}", getName(), offsets_x[0], - offsets_y[row] - prev_offset); + offset_y - prev_offset); } - prev_offset = offsets_y[row]; + prev_offset = offset_y; } const typename Kernel::ConstParams kernel_params = initConstParams(arguments); From 07ce390609238318a7ab115b3d7020f03150ce76 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 8 Mar 2024 13:59:24 +0100 Subject: [PATCH 062/334] No public templates, just happiness --- base/base/itoa.cpp | 44 +++++++++++++++++++++----------------------- base/base/itoa.h | 35 ++++++++++++++--------------------- 2 files changed, 35 insertions(+), 44 deletions(-) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index d877f15b563..3c4f0bb048d 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -32,9 +32,11 @@ #include #include +namespace +{ template -inline int digits10(T x) +int digits10T(T x) { if (x < 10ULL) return 1; @@ -64,13 +66,9 @@ inline int digits10(T x) return 11 + (x >= 100000000000ULL); } - return 12 + digits10(x / 1000000000000ULL); + return 12 + digits10T(x / 1000000000000ULL); } - -namespace -{ - template ALWAYS_INLINE inline constexpr T pow10(size_t x) { @@ -487,67 +485,62 @@ ALWAYS_INLINE inline char * writeSIntText(T x, char * pos) } } -template -char * itoa(T i, char * p) -{ - return convert::itoa(i, p); -} -template <> char * itoa(UInt8 i, char * p) { return convert::itoa(uint8_t(i), p); } -template <> char * itoa(Int8 i, char * p) { return convert::itoa(int8_t(i), p); } -template <> char * itoa(UInt128 i, char * p) { return writeUIntText(i, p); } -template <> char * itoa(Int128 i, char * p) { return writeSIntText(i, p); } -template <> char * itoa(UInt256 i, char * p) { return writeUIntText(i, p); } -template <> char * itoa(Int256 i, char * p) { return writeSIntText(i, p); } +#define DEFAULT_ITOA(T) \ + char * itoa(T i, char * p) \ + { \ + return convert::itoa(i, p); \ + } + #define FOR_MISSING_INTEGER_TYPES(M) \ - M(int8_t) \ M(uint8_t) \ M(UInt16) \ M(UInt32) \ M(UInt64) \ + M(int8_t) \ M(Int16) \ M(Int32) \ M(Int64) -#define INSTANTIATION(T) template char * itoa(T i, char * p); -FOR_MISSING_INTEGER_TYPES(INSTANTIATION) +FOR_MISSING_INTEGER_TYPES(DEFAULT_ITOA) #if defined(OS_DARWIN) -INSTANTIATION(size_t) +DEFAULT_ITOA(unsigned long) +DEFAULT_ITOA(long) #endif #undef FOR_MISSING_INTEGER_TYPES -#undef INSTANTIATION +#undef DEFAULT_ITOA #define DIGITS_INTEGER_TYPES(M) \ @@ -559,7 +552,12 @@ INSTANTIATION(size_t) M(UInt128) \ M(UInt256) -#define INSTANTIATION(T) template int digits10(T x); +#define INSTANTIATION(T) \ + int digits10(T x) \ + { \ + return digits10T(x); \ + } + DIGITS_INTEGER_TYPES(INSTANTIATION) #undef DIGITS_INTEGER_TYPES diff --git a/base/base/itoa.h b/base/base/itoa.h index 98a570b12fa..9a89fa739dd 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -2,39 +2,33 @@ #include -template char * itoa(T i, char * p); - -template <> char * itoa(UInt8 i, char * p); -template <> char * itoa(Int8 i, char * p); -template <> char * itoa(UInt128 i, char * p); -template <> char * itoa(Int128 i, char * p); -template <> char * itoa(UInt256 i, char * p); -template <> char * itoa(Int256 i, char * p); - -#define FOR_MISSING_INTEGER_TYPES(M) \ - M(int8_t) \ +#define FOR_INTEGER_TYPES(M) \ M(uint8_t) \ + M(UInt8) \ M(UInt16) \ M(UInt32) \ M(UInt64) \ + M(UInt128) \ + M(UInt256) \ + M(int8_t) \ + M(Int8) \ M(Int16) \ M(Int32) \ - M(Int64) + M(Int64) \ + M(Int128) \ + M(Int256) -#define INSTANTIATION(T) \ - extern template char * itoa(T i, char * p); -FOR_MISSING_INTEGER_TYPES(INSTANTIATION) +#define INSTANTIATION(T) char * itoa(T i, char * p); +FOR_INTEGER_TYPES(INSTANTIATION) #if defined(OS_DARWIN) -INSTANTIATION(size_t) +INSTANTIATION(unsigned long) +INSTANTIATION(long) #endif #undef FOR_MISSING_INTEGER_TYPES #undef INSTANTIATION - -template int digits10(T x); - #define DIGITS_INTEGER_TYPES(M) \ M(uint8_t) \ M(UInt8) \ @@ -43,8 +37,7 @@ template int digits10(T x); M(UInt64) \ M(UInt128) \ M(UInt256) -#define INSTANTIATION(T) \ - extern template int digits10(T x); +#define INSTANTIATION(T) int digits10(T x); DIGITS_INTEGER_TYPES(INSTANTIATION) #undef DIGITS_INTEGER_TYPES #undef INSTANTIATION From 4b964979c2c831a2ddeee5a7b0c10066f520ab11 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 8 Mar 2024 14:22:33 +0100 Subject: [PATCH 063/334] Digits doesn't belong with itoa anymore --- base/base/itoa.cpp | 55 ----------------------------------- base/base/itoa.h | 13 --------- src/Functions/countDigits.cpp | 34 ++++++++++++++++++++++ 3 files changed, 34 insertions(+), 68 deletions(-) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index 3c4f0bb048d..608258c6b56 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -35,40 +35,6 @@ namespace { -template -int digits10T(T x) -{ - if (x < 10ULL) - return 1; - if (x < 100ULL) - return 2; - if (x < 1000ULL) - return 3; - - if (x < 1000000000000ULL) - { - if (x < 100000000ULL) - { - if (x < 1000000ULL) - { - if (x < 10000ULL) - return 4; - else - return 5 + (x >= 100000ULL); - } - - return 7 + (x >= 10000000ULL); - } - - if (x < 10000000000ULL) - return 9 + (x >= 1000000000ULL); - - return 11 + (x >= 100000000000ULL); - } - - return 12 + digits10T(x / 1000000000000ULL); -} - template ALWAYS_INLINE inline constexpr T pow10(size_t x) { @@ -541,24 +507,3 @@ DEFAULT_ITOA(long) #undef FOR_MISSING_INTEGER_TYPES #undef DEFAULT_ITOA - - -#define DIGITS_INTEGER_TYPES(M) \ - M(uint8_t) \ - M(UInt8) \ - M(UInt16) \ - M(UInt32) \ - M(UInt64) \ - M(UInt128) \ - M(UInt256) - -#define INSTANTIATION(T) \ - int digits10(T x) \ - { \ - return digits10T(x); \ - } - -DIGITS_INTEGER_TYPES(INSTANTIATION) - -#undef DIGITS_INTEGER_TYPES -#undef INSTANTIATION diff --git a/base/base/itoa.h b/base/base/itoa.h index 9a89fa739dd..e69ce0ef17d 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -28,16 +28,3 @@ INSTANTIATION(long) #undef FOR_MISSING_INTEGER_TYPES #undef INSTANTIATION - -#define DIGITS_INTEGER_TYPES(M) \ - M(uint8_t) \ - M(UInt8) \ - M(UInt16) \ - M(UInt32) \ - M(UInt64) \ - M(UInt128) \ - M(UInt256) -#define INSTANTIATION(T) int digits10(T x); -DIGITS_INTEGER_TYPES(INSTANTIATION) -#undef DIGITS_INTEGER_TYPES -#undef INSTANTIATION diff --git a/src/Functions/countDigits.cpp b/src/Functions/countDigits.cpp index 2ca8d944b0a..f2712b5b301 100644 --- a/src/Functions/countDigits.cpp +++ b/src/Functions/countDigits.cpp @@ -20,6 +20,40 @@ namespace ErrorCodes namespace { +template +int digits10(T x) +{ + if (x < 10ULL) + return 1; + if (x < 100ULL) + return 2; + if (x < 1000ULL) + return 3; + + if (x < 1000000000000ULL) + { + if (x < 100000000ULL) + { + if (x < 1000000ULL) + { + if (x < 10000ULL) + return 4; + else + return 5 + (x >= 100000ULL); + } + + return 7 + (x >= 10000000ULL); + } + + if (x < 10000000000ULL) + return 9 + (x >= 1000000000ULL); + + return 11 + (x >= 100000000000ULL); + } + + return 12 + digits10(x / 1000000000000ULL); +} + /// Returns number of decimal digits you need to represent the value. /// For Decimal values takes in account their scales: calculates result over underlying int type which is (value * scale). /// countDigits(42) = 2, countDigits(42.000) = 5, countDigits(0.04200) = 4. From 4f27dd8f9c887cec6273dc7960b165ee2537ad26 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 8 Mar 2024 14:27:50 +0100 Subject: [PATCH 064/334] upd test --- .../0_stateless/02864_restore_table_with_broken_part.reference | 2 +- .../queries/0_stateless/02864_restore_table_with_broken_part.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02864_restore_table_with_broken_part.reference b/tests/queries/0_stateless/02864_restore_table_with_broken_part.reference index 9a8dcda81df..9247a7d6ab6 100644 --- a/tests/queries/0_stateless/02864_restore_table_with_broken_part.reference +++ b/tests/queries/0_stateless/02864_restore_table_with_broken_part.reference @@ -1,4 +1,4 @@ -data.bin doesn't exist: while restoring part all_2_2_0 +OK RESTORED 1 3 diff --git a/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh b/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh index cf99c7e9284..d3252b29eb7 100755 --- a/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh +++ b/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh @@ -26,7 +26,7 @@ ln -s "$SRC_BACKUP_DIR/$SRC_BACKUP_FILENAME" "$BACKUPS_DIR/$BACKUP_FILENAME" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tbl" # First try to restore with the setting `restore_broken_parts_as_detached` set to false. -$CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM $BACKUP_NAME" 2>&1 | grep -o -m 1 "data.bin doesn't exist: while restoring part all_2_2_0" +$CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM $BACKUP_NAME" 2>&1 | tr -d \\n | grep "data.bin doesn't exist" | grep "while restoring part all_2_2_0" > /dev/null && echo "OK" || echo "FAILED" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tbl" From 45a1f0f8ed33fb09fee92137a391dfd25ba63f05 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 8 Mar 2024 14:02:34 +0000 Subject: [PATCH 065/334] More consistency and extended tests --- src/Functions/array/arrayDistance.cpp | 8 ++-- src/Functions/array/arrayDotProduct.cpp | 43 ++++++++----------- .../0_stateless/02708_dotProduct.reference | 12 ++++++ .../queries/0_stateless/02708_dotProduct.sql | 16 +++++-- 4 files changed, 47 insertions(+), 32 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index 0045075ddef..8b591e37ff6 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -471,10 +471,9 @@ private: const typename Kernel::ConstParams kernel_params = initConstParams(arguments); - auto result = ColumnVector::create(input_rows_count); - auto & result_data = result->getData(); + auto col_res = ColumnVector::create(input_rows_count); + auto & result_data = col_res->getData(); - /// Do the actual computation ColumnArray::Offset prev = 0; size_t row = 0; @@ -503,7 +502,7 @@ private: result_data[row] = Kernel::finalize(state, kernel_params); row++; } - return result; + return col_res; } /// Special case when the 1st parameter is Const @@ -542,7 +541,6 @@ private: auto result = ColumnVector::create(input_rows_count); auto & result_data = result->getData(); - /// Do the actual computation size_t prev = 0; size_t row = 0; diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 8b7c85e05dd..97dc9653bab 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -140,6 +140,7 @@ public: static FunctionPtr create(ContextPtr) { return std::make_shared(); } size_t getNumberOfArguments() const override { return 2; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { @@ -174,13 +175,13 @@ public: ACTION(Float32) \ ACTION(Float64) - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { switch (result_type->getTypeId()) { #define ON_TYPE(type) \ case TypeIndex::type: \ - return executeWithResultType(arguments); \ + return executeWithResultType(arguments, input_rows_count); \ break; SUPPORTED_TYPES(ON_TYPE) @@ -193,7 +194,7 @@ public: private: template - ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments) const + ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const { DataTypePtr type_x = typeid_cast(arguments[0].type.get())->getNestedType(); @@ -201,7 +202,7 @@ private: { #define ON_TYPE(type) \ case TypeIndex::type: \ - return executeWithResultTypeAndLeftType(arguments); \ + return executeWithResultTypeAndLeftType(arguments, input_rows_count); \ break; SUPPORTED_TYPES(ON_TYPE) @@ -218,7 +219,7 @@ private: } template - ColumnPtr executeWithResultTypeAndLeftType(const ColumnsWithTypeAndName & arguments) const + ColumnPtr executeWithResultTypeAndLeftType(const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const { DataTypePtr type_y = typeid_cast(arguments[1].type.get())->getNestedType(); @@ -226,7 +227,7 @@ private: { #define ON_TYPE(type) \ case TypeIndex::type: \ - return executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column); \ + return executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column, input_rows_count); \ break; SUPPORTED_TYPES(ON_TYPE) @@ -243,15 +244,15 @@ private: } template - ColumnPtr executeWithResultTypeAndLeftTypeAndRightType(ColumnPtr col_x, ColumnPtr col_y) const + ColumnPtr executeWithResultTypeAndLeftTypeAndRightType(ColumnPtr col_x, ColumnPtr col_y, size_t input_rows_count) const { if (typeid_cast(col_x.get())) { - return executeWithLeftArgConst(col_x, col_y); + return executeWithLeftArgConst(col_x, col_y, input_rows_count); } else if (typeid_cast(col_y.get())) { - return executeWithLeftArgConst(col_y, col_x); + return executeWithLeftArgConst(col_y, col_x, input_rows_count); } col_x = col_x->convertToFullColumnIfConst(); @@ -268,16 +269,13 @@ private: if (!array_x.hasEqualOffsets(array_y)) throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Array arguments for function {} must have equal sizes", getName()); - auto col_res = ColumnVector::create(); - auto & result = col_res->getData(); - - size_t size = offsets_x.size(); - result.resize(size); + auto col_res = ColumnVector::create(input_rows_count); + auto & result_data = col_res->getData(); ColumnArray::Offset current_offset = 0; - for (size_t row = 0; row < size; ++row) + for (size_t row = 0; row < input_rows_count; ++row) { - size_t array_size = offsets_x[row] - current_offset; + const size_t array_size = offsets_x[row] - current_offset; size_t i = 0; @@ -298,7 +296,7 @@ private: for (; i < array_size; ++i) Kernel::template accumulate(state, static_cast(data_x[current_offset + i]), static_cast(data_y[current_offset + i])); - result[row] = Kernel::template finalize(state); + result_data[row] = Kernel::template finalize(state); current_offset = offsets_x[row]; } @@ -307,7 +305,7 @@ private: } template - ColumnPtr executeWithLeftArgConst(ColumnPtr col_x, ColumnPtr col_y) const + ColumnPtr executeWithLeftArgConst(ColumnPtr col_x, ColumnPtr col_y, size_t input_rows_count) const { col_x = assert_cast(col_x.get())->getDataColumnPtr(); col_y = col_y->convertToFullColumnIfConst(); @@ -336,16 +334,13 @@ private: prev_offset = offset_y; } - auto col_res = ColumnVector::create(); + auto col_res = ColumnVector::create(input_rows_count); auto & result = col_res->getData(); - size_t size = offsets_y.size(); - result.resize(size); - ColumnArray::Offset current_offset = 0; - for (size_t row = 0; row < size; ++row) + for (size_t row = 0; row < input_rows_count; ++row) { - size_t array_size = offsets_x[0]; + const size_t array_size = offsets_x[0]; typename Kernel::template State state; size_t i = 0; diff --git a/tests/queries/0_stateless/02708_dotProduct.reference b/tests/queries/0_stateless/02708_dotProduct.reference index 593071a3521..93a67e4c0be 100644 --- a/tests/queries/0_stateless/02708_dotProduct.reference +++ b/tests/queries/0_stateless/02708_dotProduct.reference @@ -11,6 +11,8 @@ [-1,-2,-3] [4,5,6] -32 Int64 [1,2,3] [4,5,6] 32 Float32 [1,2,3] [4,5,6] 32 Float64 +[] [] 0 Float32 +[] [] 0 UInt16 -- Tuple (1,2,3) (4,5,6) 32 UInt64 (1,2,3) (4,5,6) 32 UInt64 @@ -24,6 +26,8 @@ (1,2,3) (4,5,6) 32 Float64 -- Non-const argument [1,2,3] [4,5,6] 32 UInt16 +[] [] 0 Float32 +[] [] 0 UInt16 -- Array with mixed element arguments types (result type is the supertype) [1,2,3] [4,5,6] 32 Float32 -- Tuple with mixed element arguments types @@ -33,8 +37,16 @@ 32 32 -- Tests that trigger special paths + -- non-const / non-const 0 61 1 186 +0 61 +1 186 +0 61 +1 186 + -- const / non-const +0 62 +1 187 0 62 1 187 0 62 diff --git a/tests/queries/0_stateless/02708_dotProduct.sql b/tests/queries/0_stateless/02708_dotProduct.sql index ac94ecc28d3..05c66777dff 100644 --- a/tests/queries/0_stateless/02708_dotProduct.sql +++ b/tests/queries/0_stateless/02708_dotProduct.sql @@ -19,6 +19,9 @@ SELECT [-1, -2, -3]::Array(Int32) AS x, [4, 5, 6]::Array(Int32) AS y, dotProduct SELECT [-1, -2, -3]::Array(Int64) AS x, [4, 5, 6]::Array(Int64) AS y, dotProduct(x, y) AS res, toTypeName(res); SELECT [1, 2, 3]::Array(Float32) AS x, [4, 5, 6]::Array(Float32) AS y, dotProduct(x, y) AS res, toTypeName(res); SELECT [1, 2, 3]::Array(Float64) AS x, [4, 5, 6]::Array(Float64) AS y, dotProduct(x, y) AS res, toTypeName(res); +-- empty arrays +SELECT []::Array(Float32) AS x, []::Array(Float32) AS y, dotProduct(x, y) AS res, toTypeName(res); +SELECT []::Array(UInt8) AS x, []::Array(UInt8) AS y, dotProduct(x, y) AS res, toTypeName(res); SELECT ' -- Tuple'; SELECT (1::UInt8, 2::UInt8, 3::UInt8) AS x, (4::UInt8, 5::UInt8, 6::UInt8) AS y, dotProduct(x, y) AS res, toTypeName(res); @@ -34,6 +37,8 @@ SELECT (1::Float64, 2::Float64, 3::Float64) AS x, (4::Float64, 5::Float64, 6::Fl SELECT '-- Non-const argument'; SELECT materialize([1::UInt8, 2::UInt8, 3::UInt8]) AS x, [4::UInt8, 5::UInt8, 6::UInt8] AS y, dotProduct(x, y) AS res, toTypeName(res); +SELECT materialize([]::Array(Float32)) AS x, []::Array(Float32) AS y, dotProduct(x, y) AS res, toTypeName(res); +SELECT materialize([]::Array(UInt8)) AS x, []::Array(UInt8) AS y, dotProduct(x, y) AS res, toTypeName(res); SELECT ' -- Array with mixed element arguments types (result type is the supertype)'; SELECT [1::UInt16, 2::UInt8, 3::Float32] AS x, [4::Int16, 5::Float32, 6::UInt8] AS y, dotProduct(x, y) AS res, toTypeName(res); @@ -50,7 +55,12 @@ SELECT '-- Tests that trigger special paths'; DROP TABLE IF EXISTS tab; CREATE TABLE tab(id UInt64, vec Array(Float32)) ENGINE = MergeTree ORDER BY id; INSERT INTO tab VALUES (0, [0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0]) (1, [5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]); -SELECT id, arrayDotProduct(vec, vec) FROM tab ORDER BY id; -- non-const / non-const -SELECT id, arrayDotProduct([5.0, 2.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]::Array(Float32), vec) FROM tab ORDER BY id; -- const / non-const -SELECT id, arrayDotProduct([5.0, 2.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]::Array(Float64), vec) FROM tab ORDER BY id; -- const / non-const +SELECT ' -- non-const / non-const'; +SELECT id, arrayDotProduct(vec, vec) FROM tab ORDER BY id; +SELECT id, arrayDotProduct(vec::Array(Float64), vec::Array(Float64)) FROM tab ORDER BY id; +SELECT id, arrayDotProduct(vec::Array(UInt32), vec::Array(UInt32)) FROM tab ORDER BY id; +SELECT ' -- const / non-const'; +SELECT id, arrayDotProduct([5.0, 2.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]::Array(Float32), vec) FROM tab ORDER BY id; +SELECT id, arrayDotProduct([5.0, 2.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]::Array(Float64), vec) FROM tab ORDER BY id; +SELECT id, arrayDotProduct([5, 2, 2, 3, 5, 1, 2, 3, 5, 1, 2, 3, 5, 1, 2, 3, 5, 1, 2]::Array(UInt32), vec) FROM tab ORDER BY id; DROP TABLE tab; From 45efa69189784ce65bffd0d84462dcb30c1e6bf4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 8 Mar 2024 15:17:59 +0100 Subject: [PATCH 066/334] Add perf tests --- tests/performance/bigint_formatting.xml | 13 +++++++++++++ 1 file changed, 13 insertions(+) create mode 100644 tests/performance/bigint_formatting.xml diff --git a/tests/performance/bigint_formatting.xml b/tests/performance/bigint_formatting.xml new file mode 100644 index 00000000000..c3454f91d1d --- /dev/null +++ b/tests/performance/bigint_formatting.xml @@ -0,0 +1,13 @@ + + + CREATE TABLE bigint ( u128 UInt128, i128 Int128, u256 UInt256, i256 Int256) ENGINE = Memory + AS + SELECT * FROM generateRandom('u128 UInt128, i128 Int128, u256 UInt256, i256 Int256', 42) LIMIT 50000; + + DROP TABLE IF EXISTS bigint + + SELECT * FROM bigint WHERE NOT ignore(toString(u128)) SETTINGS max_threads = 1 + SELECT * FROM bigint WHERE NOT ignore(toString(i128)) SETTINGS max_threads = 1 + SELECT * FROM bigint WHERE NOT ignore(toString(u256)) SETTINGS max_threads = 1 + SELECT * FROM bigint WHERE NOT ignore(toString(i256)) SETTINGS max_threads = 1 + From d6e0dd45b9cc88b9002de68138440cd24452fb17 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 8 Mar 2024 22:57:49 +0800 Subject: [PATCH 067/334] Fix build --- src/Functions/coverage.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/coverage.cpp b/src/Functions/coverage.cpp index f4cac26df78..0f4cd1940b7 100644 --- a/src/Functions/coverage.cpp +++ b/src/Functions/coverage.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include From e2317477f7b95d07407db8def968d286aa9e270d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 8 Mar 2024 17:12:31 +0100 Subject: [PATCH 068/334] fix removing is_active node after re-creation --- src/Databases/DatabaseReplicatedWorker.cpp | 2 ++ tests/integration/test_replicated_database/test.py | 2 ++ 2 files changed, 4 insertions(+) diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 2056b403ff6..0a6e8f9345e 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -75,6 +75,8 @@ void DatabaseReplicatedDDLWorker::initializeReplication() String active_path = fs::path(database->replica_path) / "active"; String active_id = toString(ServerUUID::get()); zookeeper->deleteEphemeralNodeIfContentMatches(active_path, active_id); + if (active_node_holder) + active_node_holder->setAlreadyRemoved(); zookeeper->create(active_path, active_id, zkutil::CreateMode::Ephemeral); active_node_holder.reset(); active_node_holder_zookeeper = zookeeper; diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index b47f86a843d..4f449f9a296 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1141,6 +1141,8 @@ def test_sync_replica(started_cluster): dummy_node.query("SYSTEM SYNC DATABASE REPLICA test_sync_database") + assert "2\n" == main_node.query("SELECT sum(is_active) FROM system.clusters WHERE cluster='test_sync_database'") + assert dummy_node.query( "SELECT count() FROM system.tables where database='test_sync_database'" ).strip() == str(number_of_tables) From babe00003620ca34f228009d919d5613db867dee Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 8 Mar 2024 16:43:10 +0000 Subject: [PATCH 069/334] Automatic style fix --- tests/integration/test_replicated_database/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 4f449f9a296..881659262ac 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1141,7 +1141,9 @@ def test_sync_replica(started_cluster): dummy_node.query("SYSTEM SYNC DATABASE REPLICA test_sync_database") - assert "2\n" == main_node.query("SELECT sum(is_active) FROM system.clusters WHERE cluster='test_sync_database'") + assert "2\n" == main_node.query( + "SELECT sum(is_active) FROM system.clusters WHERE cluster='test_sync_database'" + ) assert dummy_node.query( "SELECT count() FROM system.tables where database='test_sync_database'" From 68360aa522169b1c0955837e93687c6d3a124912 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 8 Mar 2024 18:37:03 +0100 Subject: [PATCH 070/334] Clang format --- base/base/itoa.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index 608258c6b56..9bd2fcd1837 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -34,7 +34,6 @@ namespace { - template ALWAYS_INLINE inline constexpr T pow10(size_t x) { @@ -117,7 +116,6 @@ QuotientAndRemainder inline split(UnsignedOfSize value) return {quotient, remainder}; } - ALWAYS_INLINE inline char * outDigit(char * p, uint8_t value) { *p = '0' + value; @@ -147,7 +145,6 @@ ALWAYS_INLINE inline char * outTwoDigits(char * p, uint8_t value) return p; } - namespace convert { template @@ -296,7 +293,6 @@ ALWAYS_INLINE inline char * itoa(I i, char * p) } } - const uint64_t max_multiple_of_hundred_that_fits_in_64_bits = 1'00'00'00'00'00'00'00'00'00ull; const int max_multiple_of_hundred_blocks = 9; static_assert(max_multiple_of_hundred_that_fits_in_64_bits % 100 == 0); @@ -411,7 +407,6 @@ ALWAYS_INLINE inline char * writeUIntText(UInt256 _x, char * p) return highest_part_print; } - ALWAYS_INLINE inline char * writeLeadingMinus(char * pos) { *pos = '-'; @@ -451,7 +446,6 @@ ALWAYS_INLINE inline char * writeSIntText(T x, char * pos) } } - char * itoa(UInt8 i, char * p) { return convert::itoa(uint8_t(i), p); From 2ce96f48f3c3958ef51c3e620b886d633436bb26 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 8 Mar 2024 21:58:01 +0100 Subject: [PATCH 071/334] Update 02962_system_sync_replica_lightweight_from_modifier.sh --- .../02962_system_sync_replica_lightweight_from_modifier.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02962_system_sync_replica_lightweight_from_modifier.sh b/tests/queries/0_stateless/02962_system_sync_replica_lightweight_from_modifier.sh index f47801abf73..b61be87411d 100755 --- a/tests/queries/0_stateless/02962_system_sync_replica_lightweight_from_modifier.sh +++ b/tests/queries/0_stateless/02962_system_sync_replica_lightweight_from_modifier.sh @@ -14,7 +14,7 @@ export REPLICAS_TO_DROP for i in $(seq $TOTAL_REPLICAS); do $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_table_$i" - $CLICKHOUSE_CLIENT --query "CREATE TABLE test_table_$i (key UInt64, value UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_table', '$i') ORDER BY key" + $CLICKHOUSE_CLIENT --query "CREATE TABLE test_table_$i (key UInt64, value UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_table', '$i') ORDER BY key SETTINGS old_parts_lifetime=1" done function insert_thread() { @@ -35,7 +35,7 @@ function sync_and_drop_replicas() { done for i in $(seq $REPLICAS_TO_DROP); do - $CLICKHOUSE_CLIENT --query "CREATE TABLE test_table_$i (key UInt64, value UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_table', '$i') ORDER BY key" + $CLICKHOUSE_CLIENT --query "CREATE TABLE test_table_$i (key UInt64, value UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_table', '$i') ORDER BY key SETTINGS old_parts_lifetime=1" done done } @@ -87,4 +87,4 @@ for i in $(seq $TOTAL_REPLICAS); do if [ $i -gt $REPLICAS_TO_DROP ]; then $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_table_$i" fi -done \ No newline at end of file +done From 23b55ecbe8521692acab507408fc70665aa16f1b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 8 Mar 2024 22:02:22 +0100 Subject: [PATCH 072/334] beautify exception --- src/Functions/array/arrayDotProduct.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 97dc9653bab..3f37e6f609f 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -212,7 +212,7 @@ private: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Arguments of function {} has nested type {}. " - "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + "Supported types: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", getName(), type_x->getName()); } From 530efbe8102023f857ec29b1f9c7f2090973f736 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 10 Mar 2024 13:08:50 +0000 Subject: [PATCH 073/334] Beautify exception, pt. II --- src/Functions/array/arrayDistance.cpp | 6 +++--- src/Functions/array/arrayDotProduct.cpp | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index 8b591e37ff6..69e5e3712dd 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -357,7 +357,7 @@ public: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Arguments of function {} has nested type {}. " - "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + "Supported types: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", getName(), common_type->getName()); } @@ -412,7 +412,7 @@ private: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Arguments of function {} has nested type {}. " - "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + "Supported types: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", getName(), type_x->getName()); } @@ -437,7 +437,7 @@ private: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Arguments of function {} has nested type {}. " - "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + "Supported types: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", getName(), type_y->getName()); } diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 3f37e6f609f..a9547ca90bb 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -237,7 +237,7 @@ private: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Arguments of function {} has nested type {}. " - "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + "Supported types: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", getName(), type_y->getName()); } From bcd705517a510feffd08e9e27c93c2e09eecd144 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 10 Mar 2024 12:53:58 +0000 Subject: [PATCH 074/334] Remove unnecessary call to convertToFullColumnIfConst() --- src/Functions/array/arrayDistance.cpp | 3 --- src/Functions/array/arrayDotProduct.cpp | 3 --- src/Functions/array/arrayNorm.cpp | 3 +-- 3 files changed, 1 insertion(+), 8 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index 69e5e3712dd..6ed4bf24f99 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -455,9 +455,6 @@ private: return executeWithLeftArgConst(col_y, col_x, input_rows_count, arguments); } - col_x = col_x->convertToFullColumnIfConst(); - col_y = col_y->convertToFullColumnIfConst(); - const auto & array_x = *assert_cast(col_x.get()); const auto & array_y = *assert_cast(col_y.get()); diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index a9547ca90bb..783843a89d5 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -255,9 +255,6 @@ private: return executeWithLeftArgConst(col_y, col_x, input_rows_count); } - col_x = col_x->convertToFullColumnIfConst(); - col_y = col_y->convertToFullColumnIfConst(); - const auto & array_x = *assert_cast(col_x.get()); const auto & array_y = *assert_cast(col_y.get()); diff --git a/src/Functions/array/arrayNorm.cpp b/src/Functions/array/arrayNorm.cpp index 027a33d094c..e87eff6add1 100644 --- a/src/Functions/array/arrayNorm.cpp +++ b/src/Functions/array/arrayNorm.cpp @@ -175,8 +175,7 @@ public: } } - ColumnPtr - executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { DataTypePtr type = typeid_cast(arguments[0].type.get())->getNestedType(); ColumnPtr column = arguments[0].column->convertToFullColumnIfConst(); From 7a8d79de234e4f83a4e6421e5de49e5800ff70bc Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Sun, 10 Mar 2024 22:13:49 +0100 Subject: [PATCH 075/334] reload CI due to OOM in build From e562d97ff59739da3ca3650a33644bf9700becd7 Mon Sep 17 00:00:00 2001 From: Nikolay Yankin <211292+kolya7k@users.noreply.github.com> Date: Mon, 11 Mar 2024 14:07:19 +0300 Subject: [PATCH 076/334] Update install.md https://packages.clickhouse.com/tgz/stable/ is multi-paged now and sorted by date asc so we can't get very last version --- docs/en/getting-started/install.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 234420de374..3b01434ecc5 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -262,7 +262,7 @@ The required version can be downloaded with `curl` or `wget` from repository htt After that downloaded archives should be unpacked and installed with installation scripts. Example for the latest stable version: ``` bash -LATEST_VERSION=$(curl -s https://packages.clickhouse.com/tgz/stable/ | \ +LATEST_VERSION=$(curl -s -L https://api.github.com/repos/ClickHouse/ClickHouse/tags | \ grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | sort -V -r | head -n 1) export LATEST_VERSION From 5f1991fbef2f959f1d55c62194d948814d199fa9 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 12 Mar 2024 15:53:28 +0800 Subject: [PATCH 077/334] too big translation unit in Aggregator --- src/Common/HashTable/FixedHashMap.h | 3 + .../HashTable/TwoLevelStringHashTable.h | 1 + src/Interpreters/AggregatedData.h | 142 +++ src/Interpreters/AggregatedDataVariants.cpp | 255 ++++ src/Interpreters/AggregatedDataVariants.h | 320 +++++ src/Interpreters/AggregationMethod.cpp | 215 ++++ src/Interpreters/AggregationMethod.h | 320 +++++ src/Interpreters/Aggregator.cpp | 512 ++++---- src/Interpreters/Aggregator.h | 1035 +---------------- 9 files changed, 1541 insertions(+), 1262 deletions(-) create mode 100644 src/Interpreters/AggregatedData.h create mode 100644 src/Interpreters/AggregatedDataVariants.cpp create mode 100644 src/Interpreters/AggregatedDataVariants.h create mode 100644 src/Interpreters/AggregationMethod.cpp create mode 100644 src/Interpreters/AggregationMethod.h diff --git a/src/Common/HashTable/FixedHashMap.h b/src/Common/HashTable/FixedHashMap.h index e835a6fba94..537f37a9e6c 100644 --- a/src/Common/HashTable/FixedHashMap.h +++ b/src/Common/HashTable/FixedHashMap.h @@ -109,6 +109,9 @@ public: using Base::Base; + FixedHashMap() = default; + FixedHashMap(size_t ) {} /// NOLINT + template void ALWAYS_INLINE mergeToViaEmplace(Self & that, Func && func) { diff --git a/src/Common/HashTable/TwoLevelStringHashTable.h b/src/Common/HashTable/TwoLevelStringHashTable.h index 54c208c5b60..1ce6b3d02e3 100644 --- a/src/Common/HashTable/TwoLevelStringHashTable.h +++ b/src/Common/HashTable/TwoLevelStringHashTable.h @@ -38,6 +38,7 @@ public: Impl impls[NUM_BUCKETS]; TwoLevelStringHashTable() = default; + TwoLevelStringHashTable(size_t ) {} /// NOLINT template explicit TwoLevelStringHashTable(const Source & src) diff --git a/src/Interpreters/AggregatedData.h b/src/Interpreters/AggregatedData.h new file mode 100644 index 00000000000..6cd6b190801 --- /dev/null +++ b/src/Interpreters/AggregatedData.h @@ -0,0 +1,142 @@ +#pragma once +#include + +#include +#include +#include +#include +namespace DB +{ +/** Different data structures that can be used for aggregation + * For efficiency, the aggregation data itself is put into the pool. + * Data and pool ownership (states of aggregate functions) + * is acquired later - in `convertToBlocks` function, by the ColumnAggregateFunction object. + * + * Most data structures exist in two versions: normal and two-level (TwoLevel). + * A two-level hash table works a little slower with a small number of different keys, + * but with a large number of different keys scales better, because it allows + * parallelize some operations (merging, post-processing) in a natural way. + * + * To ensure efficient work over a wide range of conditions, + * first single-level hash tables are used, + * and when the number of different keys is large enough, + * they are converted to two-level ones. + * + * PS. There are many different approaches to the effective implementation of parallel and distributed aggregation, + * best suited for different cases, and this approach is just one of them, chosen for a combination of reasons. + */ + +using AggregatedDataWithoutKey = AggregateDataPtr; + +using AggregatedDataWithUInt8Key = FixedImplicitZeroHashMapWithCalculatedSize; +using AggregatedDataWithUInt16Key = FixedImplicitZeroHashMap; + +using AggregatedDataWithUInt32Key = HashMap>; +using AggregatedDataWithUInt64Key = HashMap>; + +using AggregatedDataWithShortStringKey = StringHashMap; + +using AggregatedDataWithStringKey = HashMapWithSavedHash; + +using AggregatedDataWithKeys128 = HashMap; +using AggregatedDataWithKeys256 = HashMap; + +using AggregatedDataWithUInt32KeyTwoLevel = TwoLevelHashMap>; +using AggregatedDataWithUInt64KeyTwoLevel = TwoLevelHashMap>; + +using AggregatedDataWithShortStringKeyTwoLevel = TwoLevelStringHashMap; + +using AggregatedDataWithStringKeyTwoLevel = TwoLevelHashMapWithSavedHash; + +using AggregatedDataWithKeys128TwoLevel = TwoLevelHashMap; +using AggregatedDataWithKeys256TwoLevel = TwoLevelHashMap; + +/** Variants with better hash function, using more than 32 bits for hash. + * Using for merging phase of external aggregation, where number of keys may be far greater than 4 billion, + * but we keep in memory and merge only sub-partition of them simultaneously. + * TODO We need to switch for better hash function not only for external aggregation, + * but also for huge aggregation results on machines with terabytes of RAM. + */ + +using AggregatedDataWithUInt64KeyHash64 = HashMap>; +using AggregatedDataWithStringKeyHash64 = HashMapWithSavedHash; +using AggregatedDataWithKeys128Hash64 = HashMap; +using AggregatedDataWithKeys256Hash64 = HashMap; + +template +struct AggregationDataWithNullKey : public Base +{ + using Base::Base; + + bool & hasNullKeyData() { return has_null_key_data; } + AggregateDataPtr & getNullKeyData() { return null_key_data; } + bool hasNullKeyData() const { return has_null_key_data; } + const AggregateDataPtr & getNullKeyData() const { return null_key_data; } + size_t size() const { return Base::size() + (has_null_key_data ? 1 : 0); } + bool empty() const { return Base::empty() && !has_null_key_data; } + void clear() + { + Base::clear(); + has_null_key_data = false; + } + void clearAndShrink() + { + Base::clearAndShrink(); + has_null_key_data = false; + } + +private: + bool has_null_key_data = false; + AggregateDataPtr null_key_data = nullptr; +}; + +template +struct AggregationDataWithNullKeyTwoLevel : public Base +{ + using Base::Base; + using Base::impls; + + AggregationDataWithNullKeyTwoLevel() = default; + + template + explicit AggregationDataWithNullKeyTwoLevel(const Other & other) : Base(other) + { + impls[0].hasNullKeyData() = other.hasNullKeyData(); + impls[0].getNullKeyData() = other.getNullKeyData(); + } + + bool & hasNullKeyData() { return impls[0].hasNullKeyData(); } + AggregateDataPtr & getNullKeyData() { return impls[0].getNullKeyData(); } + bool hasNullKeyData() const { return impls[0].hasNullKeyData(); } + const AggregateDataPtr & getNullKeyData() const { return impls[0].getNullKeyData(); } +}; + +template +using HashTableWithNullKey = AggregationDataWithNullKey>; +template +using StringHashTableWithNullKey = AggregationDataWithNullKey>; + +using AggregatedDataWithNullableUInt8Key = AggregationDataWithNullKey; +using AggregatedDataWithNullableUInt16Key = AggregationDataWithNullKey; +using AggregatedDataWithNullableUInt32Key = AggregationDataWithNullKey; + + +using AggregatedDataWithNullableUInt64Key = AggregationDataWithNullKey; +using AggregatedDataWithNullableStringKey = AggregationDataWithNullKey; +using AggregatedDataWithNullableShortStringKey = AggregationDataWithNullKey; + + +using AggregatedDataWithNullableUInt32KeyTwoLevel = AggregationDataWithNullKeyTwoLevel< + TwoLevelHashMap, + TwoLevelHashTableGrower<>, HashTableAllocator, HashTableWithNullKey>>; +using AggregatedDataWithNullableUInt64KeyTwoLevel = AggregationDataWithNullKeyTwoLevel< + TwoLevelHashMap, + TwoLevelHashTableGrower<>, HashTableAllocator, HashTableWithNullKey>>; + +using AggregatedDataWithNullableShortStringKeyTwoLevel = AggregationDataWithNullKeyTwoLevel< + TwoLevelStringHashMap>; + +using AggregatedDataWithNullableStringKeyTwoLevel = AggregationDataWithNullKeyTwoLevel< + TwoLevelHashMapWithSavedHash, + TwoLevelHashTableGrower<>, HashTableAllocator, HashTableWithNullKey>>; +} diff --git a/src/Interpreters/AggregatedDataVariants.cpp b/src/Interpreters/AggregatedDataVariants.cpp new file mode 100644 index 00000000000..0c86c58bd3e --- /dev/null +++ b/src/Interpreters/AggregatedDataVariants.cpp @@ -0,0 +1,255 @@ +#include +#include + +namespace ProfileEvents +{ + extern const Event AggregationPreallocatedElementsInHashTables; +} + +namespace DB +{ +namespace ErrorCodes +{ + extern const int UNKNOWN_AGGREGATED_DATA_VARIANT; + extern const int LOGICAL_ERROR; + +} +using ColumnsHashing::HashMethodContext; +using ColumnsHashing::HashMethodContextPtr; +using ColumnsHashing::LastElementCacheStats; + +AggregatedDataVariants::AggregatedDataVariants() : aggregates_pools(1, std::make_shared()), aggregates_pool(aggregates_pools.back().get()) {} + +AggregatedDataVariants::~AggregatedDataVariants() +{ + if (aggregator && !aggregator->all_aggregates_has_trivial_destructor) + { + try + { + aggregator->destroyAllAggregateStates(*this); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } +} + +// The std::is_constructible trait isn't suitable here because some classes have template constructors with semantics different from providing size hints. +// Also string hash table variants are not supported due to the fact that both local perf tests and tests in CI showed slowdowns for them. +template +struct HasConstructorOfNumberOfElements : std::false_type +{ +}; + +template +struct HasConstructorOfNumberOfElements> : std::true_type +{ +}; + +template typename ImplTable> +struct HasConstructorOfNumberOfElements> : std::true_type +{ +}; + +template +struct HasConstructorOfNumberOfElements> : std::true_type +{ +}; + +template +struct HasConstructorOfNumberOfElements> : std::true_type +{ +}; + +template