From 2ce0ef9c8a656bfbbf8c763af78cb55a07f60044 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Nov 2023 00:23:20 +0100 Subject: [PATCH 01/28] Enable HTTP OPTIONS method by default --- programs/server/config.xml | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index d0bf1c7d66a..e2da8d789fb 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -104,15 +104,14 @@ - - + - 3 + 10 From 8c253b9e3e46c416623047d2df876eab0e2eb5ba Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 10 Nov 2023 05:23:50 +0100 Subject: [PATCH 04/28] Remove C++ templates --- src/Dictionaries/RangeHashedDictionary.h | 2 - src/Functions/CMakeLists.txt | 2 + src/Functions/CastOverloadResolver.cpp | 146 ++++++++++++++++++ src/Functions/CastOverloadResolver.h | 141 ++--------------- src/Functions/FunctionsConversion.h | 23 +-- src/Functions/FunctionsStringArray.h | 9 +- src/Functions/formatDateTime.cpp | 17 +- src/Functions/parseDateTime.cpp | 5 +- src/Functions/toBool.cpp | 17 +- src/Interpreters/ActionsDAG.cpp | 9 +- src/Interpreters/castColumn.cpp | 28 ++-- src/Interpreters/castColumn.h | 8 +- .../Formats/Impl/ArrowFieldIndexUtil.h | 19 ++- src/Storages/HDFS/StorageHDFS.cpp | 1 - src/Storages/KVStorageUtils.cpp | 4 + src/Storages/MergeTree/KeyCondition.cpp | 7 +- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 5 +- src/Storages/PartitionedSink.cpp | 4 +- src/Storages/StorageS3.cpp | 2 - src/Storages/System/StorageSystemBackups.cpp | 1 + 20 files changed, 247 insertions(+), 203 deletions(-) diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index 624a57d65b5..9be9fa1d0d4 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -227,9 +227,7 @@ private: struct KeyAttribute final { RangeStorageTypeContainer container; - RangeStorageTypeContainer invalid_intervals_container; - }; void createAttributes(); diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 31f7f24eb13..57904a8ca1c 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -9,6 +9,7 @@ extract_into_parent_list(clickhouse_functions_sources dbms_sources FunctionHelpers.cpp extractTimeZoneFromFunctionArguments.cpp FunctionsLogical.cpp + CastOverloadResolver.cpp ) extract_into_parent_list(clickhouse_functions_headers dbms_headers IFunction.h @@ -16,6 +17,7 @@ extract_into_parent_list(clickhouse_functions_headers dbms_headers FunctionHelpers.h extractTimeZoneFromFunctionArguments.h FunctionsLogical.h + CastOverloadResolver.h ) add_library(clickhouse_functions_obj OBJECT ${clickhouse_functions_headers} ${clickhouse_functions_sources}) diff --git a/src/Functions/CastOverloadResolver.cpp b/src/Functions/CastOverloadResolver.cpp index 20a08e3b60b..7fc46db50f1 100644 --- a/src/Functions/CastOverloadResolver.cpp +++ b/src/Functions/CastOverloadResolver.cpp @@ -1,10 +1,156 @@ #include +#include #include +#include namespace DB { +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +/** CastInternal does not preserve nullability of the data type, + * i.e. CastInternal(toNullable(toInt8(1)) as Int32) will be Int32(1). + * + * Cast preserves nullability according to setting `cast_keep_nullable`, + * i.e. Cast(toNullable(toInt8(1)) as Int32) will be Nullable(Int32(1)) if `cast_keep_nullable` == 1. + */ +template +class CastOverloadResolverImpl : public IFunctionOverloadResolver +{ +public: + using MonotonicityForRange = FunctionCastBase::MonotonicityForRange; + + static constexpr auto name = cast_type == CastType::accurate + ? CastName::accurate_cast_name + : (cast_type == CastType::accurateOrNull ? CastName::accurate_cast_or_null_name : CastName::cast_name); + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + explicit CastOverloadResolverImpl(ContextPtr context_, std::optional diagnostic_, bool keep_nullable_, const DataTypeValidationSettings & data_type_validation_settings_) + : context(context_) + , diagnostic(std::move(diagnostic_)) + , keep_nullable(keep_nullable_) + , data_type_validation_settings(data_type_validation_settings_) + { + } + + static FunctionOverloadResolverPtr create(ContextPtr context) + { + const auto & settings_ref = context->getSettingsRef(); + + if constexpr (internal) + return createImpl(context, {}, false /*keep_nullable*/); + + return createImpl(context, {}, settings_ref.cast_keep_nullable, DataTypeValidationSettings(settings_ref)); + } + + static FunctionOverloadResolverPtr createImpl(ContextPtr context, std::optional diagnostic = {}, bool keep_nullable = false, const DataTypeValidationSettings & data_type_validation_settings = {}) + { + assert(!internal || !keep_nullable); + return std::make_unique(context, std::move(diagnostic), keep_nullable, data_type_validation_settings); + } + + static FunctionOverloadResolverPtr createImpl(std::optional diagnostic = {}, bool keep_nullable = false, const DataTypeValidationSettings & data_type_validation_settings = {}) + { + assert(!internal || !keep_nullable); + return std::make_unique(ContextPtr(), std::move(diagnostic), keep_nullable, data_type_validation_settings); + } + +protected: + + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override + { + DataTypes data_types(arguments.size()); + + for (size_t i = 0; i < arguments.size(); ++i) + data_types[i] = arguments[i].type; + + auto monotonicity = MonotonicityHelper::getMonotonicityInformation(arguments.front().type, return_type.get()); + return std::make_unique>(context, name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + const auto & column = arguments.back().column; + if (!column) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument to {} must be a constant string describing type. " + "Instead there is non-constant column of type {}", getName(), arguments.back().type->getName()); + + const auto * type_col = checkAndGetColumnConst(column.get()); + if (!type_col) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument to {} must be a constant string describing type. " + "Instead there is a column with the following structure: {}", getName(), column->dumpStructure()); + + DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue()); + validateDataType(type, data_type_validation_settings); + + if constexpr (cast_type == CastType::accurateOrNull) + return makeNullable(type); + + if constexpr (internal) + return type; + + if (keep_nullable && arguments.front().type->isNullable() && type->canBeInsideNullable()) + return makeNullable(type); + + return type; + } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForNothing() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + +private: + ContextPtr context; + std::optional diagnostic; + bool keep_nullable; + DataTypeValidationSettings data_type_validation_settings; +}; + + +struct CastOverloadName +{ + static constexpr auto cast_name = "CAST"; + static constexpr auto accurate_cast_name = "accurateCast"; + static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; +}; + +struct CastInternalOverloadName +{ + static constexpr auto cast_name = "_CAST"; + static constexpr auto accurate_cast_name = "accurate_Cast"; + static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; +}; + +template +using CastOverloadResolver = CastOverloadResolverImpl; + +template +using CastInternalOverloadResolver = CastOverloadResolverImpl; + + +FunctionOverloadResolverPtr createInternalCastOverloadResolver(CastType type, std::optional diagnostic) +{ + switch (type) + { + case CastType::nonAccurate: + return CastInternalOverloadResolver::createImpl(diagnostic); + case CastType::accurate: + return CastInternalOverloadResolver::createImpl(diagnostic); + case CastType::accurateOrNull: + return CastInternalOverloadResolver::createImpl(diagnostic); + } +} + + REGISTER_FUNCTION(CastOverloadResolvers) { factory.registerFunction>({}, FunctionFactory::CaseInsensitive); diff --git a/src/Functions/CastOverloadResolver.h b/src/Functions/CastOverloadResolver.h index 670cd364a29..4346478e5b6 100644 --- a/src/Functions/CastOverloadResolver.h +++ b/src/Functions/CastOverloadResolver.h @@ -1,138 +1,29 @@ #pragma once -#include -#include + +#include +#include +#include + namespace DB { -namespace ErrorCodes +class IFunctionOverloadResolver; +using FunctionOverloadResolverPtr = std::shared_ptr; + +enum class CastType { - extern const int ILLEGAL_TYPE_OF_ARGUMENT; -} - -/** CastInternal does not preserve nullability of the data type, - * i.e. CastInternal(toNullable(toInt8(1)) as Int32) will be Int32(1). - * - * Cast preserves nullability according to setting `cast_keep_nullable`, - * i.e. Cast(toNullable(toInt8(1)) as Int32) will be Nullable(Int32(1)) if `cast_keep_nullable` == 1. - */ -template -class CastOverloadResolverImpl : public IFunctionOverloadResolver -{ -public: - using MonotonicityForRange = FunctionCastBase::MonotonicityForRange; - using Diagnostic = FunctionCastBase::Diagnostic; - - static constexpr auto name = cast_type == CastType::accurate - ? CastName::accurate_cast_name - : (cast_type == CastType::accurateOrNull ? CastName::accurate_cast_or_null_name : CastName::cast_name); - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 2; } - - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - explicit CastOverloadResolverImpl(ContextPtr context_, std::optional diagnostic_, bool keep_nullable_, const DataTypeValidationSettings & data_type_validation_settings_) - : context(context_) - , diagnostic(std::move(diagnostic_)) - , keep_nullable(keep_nullable_) - , data_type_validation_settings(data_type_validation_settings_) - { - } - - static FunctionOverloadResolverPtr create(ContextPtr context) - { - const auto & settings_ref = context->getSettingsRef(); - - if constexpr (internal) - return createImpl(context, {}, false /*keep_nullable*/); - - return createImpl(context, {}, settings_ref.cast_keep_nullable, DataTypeValidationSettings(settings_ref)); - } - - static FunctionOverloadResolverPtr createImpl(ContextPtr context, std::optional diagnostic = {}, bool keep_nullable = false, const DataTypeValidationSettings & data_type_validation_settings = {}) - { - assert(!internal || !keep_nullable); - return std::make_unique(context, std::move(diagnostic), keep_nullable, data_type_validation_settings); - } - - static FunctionOverloadResolverPtr createImpl(std::optional diagnostic = {}, bool keep_nullable = false, const DataTypeValidationSettings & data_type_validation_settings = {}) - { - assert(!internal || !keep_nullable); - return std::make_unique(ContextPtr(), std::move(diagnostic), keep_nullable, data_type_validation_settings); - } - -protected: - - FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override - { - DataTypes data_types(arguments.size()); - - for (size_t i = 0; i < arguments.size(); ++i) - data_types[i] = arguments[i].type; - - auto monotonicity = MonotonicityHelper::getMonotonicityInformation(arguments.front().type, return_type.get()); - return std::make_unique>(context, name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); - } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - const auto & column = arguments.back().column; - if (!column) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument to {} must be a constant string describing type. " - "Instead there is non-constant column of type {}", getName(), arguments.back().type->getName()); - - const auto * type_col = checkAndGetColumnConst(column.get()); - if (!type_col) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument to {} must be a constant string describing type. " - "Instead there is a column with the following structure: {}", getName(), column->dumpStructure()); - - DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue()); - validateDataType(type, data_type_validation_settings); - - if constexpr (cast_type == CastType::accurateOrNull) - return makeNullable(type); - - if constexpr (internal) - return type; - - if (keep_nullable && arguments.front().type->isNullable() && type->canBeInsideNullable()) - return makeNullable(type); - - return type; - } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForNothing() const override { return false; } - bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } - -private: - ContextPtr context; - std::optional diagnostic; - bool keep_nullable; - DataTypeValidationSettings data_type_validation_settings; + nonAccurate, + accurate, + accurateOrNull }; - -struct CastOverloadName +struct CastDiagnostic { - static constexpr auto cast_name = "CAST"; - static constexpr auto accurate_cast_name = "accurateCast"; - static constexpr auto accurate_cast_or_null_name = "accurateCastOrNull"; + std::string column_from; + std::string column_to; }; -struct CastInternalOverloadName -{ - static constexpr auto cast_name = "_CAST"; - static constexpr auto accurate_cast_name = "accurate_Cast"; - static constexpr auto accurate_cast_or_null_name = "accurate_CastOrNull"; -}; - -template -using CastOverloadResolver = CastOverloadResolverImpl; - -template -using CastInternalOverloadResolver = CastOverloadResolverImpl; +FunctionOverloadResolverPtr createInternalCastOverloadResolver(CastType type, std::optional diagnostic); } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index d5f1f175a37..b0a262ff36b 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -53,6 +53,7 @@ #include #include #include +#include #include #include #include @@ -3127,14 +3128,8 @@ class ExecutableFunctionCast : public IExecutableFunction public: using WrapperType = std::function; - struct Diagnostic - { - std::string column_from; - std::string column_to; - }; - explicit ExecutableFunctionCast( - WrapperType && wrapper_function_, const char * name_, std::optional diagnostic_) + WrapperType && wrapper_function_, const char * name_, std::optional diagnostic_) : wrapper_function(std::move(wrapper_function_)), name(name_), diagnostic(std::move(diagnostic_)) {} String getName() const override { return name; } @@ -3170,24 +3165,16 @@ protected: private: WrapperType wrapper_function; const char * name; - std::optional diagnostic; + std::optional diagnostic; }; struct CastName { static constexpr auto name = "CAST"; }; struct CastInternalName { static constexpr auto name = "_CAST"; }; -enum class CastType -{ - nonAccurate, - accurate, - accurateOrNull -}; - class FunctionCastBase : public IFunctionBase { public: using MonotonicityForRange = std::function; - using Diagnostic = ExecutableFunctionCast::Diagnostic; }; template @@ -3201,7 +3188,7 @@ public: , MonotonicityForRange && monotonicity_for_range_ , const DataTypes & argument_types_ , const DataTypePtr & return_type_ - , std::optional diagnostic_ + , std::optional diagnostic_ , CastType cast_type_) : cast_name(cast_name_), monotonicity_for_range(std::move(monotonicity_for_range_)) , argument_types(argument_types_), return_type(return_type_), diagnostic(std::move(diagnostic_)) @@ -3251,7 +3238,7 @@ private: DataTypes argument_types; DataTypePtr return_type; - std::optional diagnostic; + std::optional diagnostic; CastType cast_type; ContextPtr context; diff --git a/src/Functions/FunctionsStringArray.h b/src/Functions/FunctionsStringArray.h index d7d7e3b5100..22b19daf149 100644 --- a/src/Functions/FunctionsStringArray.h +++ b/src/Functions/FunctionsStringArray.h @@ -9,15 +9,16 @@ #include #include #include -#include #include #include +#include #include -#include +#include #include #include #include + namespace DB { @@ -910,9 +911,7 @@ private: } else { - ColumnsWithTypeAndName cols; - cols.emplace_back(col_arr.getDataPtr(), nested_type, "tmp"); - return ConvertImplGenericToString::execute(cols, std::make_shared(), col_arr.size()); + return castColumn({col_arr.getDataPtr(), nested_type, "tmp"}, std::make_shared()); } } diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index 7b93f5e063a..1fb47832418 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -9,12 +9,14 @@ #include #include #include -#include #include #include #include #include +#include +#include + #include #include @@ -803,18 +805,7 @@ public: { if (arguments.size() == 1) { - if (!castType(arguments[0].type.get(), [&](const auto & type) - { - using FromDataType = std::decay_t; - res = ConvertImpl::execute(arguments, result_type, input_rows_count); - return true; - })) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of function {}, must be Integer, Date, Date32, DateTime " - "or DateTime64 when arguments size is 1.", - arguments[0].column->getName(), getName()); - } + return castColumn(arguments[0], result_type); } else { diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index fdab85c4640..c6721b29c1c 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -7,15 +7,16 @@ #include #include -#include #include #include #include +#include + #include -#include #include + namespace DB { namespace ErrorCodes diff --git a/src/Functions/toBool.cpp b/src/Functions/toBool.cpp index 765da0c3206..6f2c436c1ea 100644 --- a/src/Functions/toBool.cpp +++ b/src/Functions/toBool.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include @@ -12,6 +11,14 @@ namespace { class FunctionToBool : public IFunction { + private: + ContextPtr context; + + static String getReturnTypeName(const DataTypePtr & argument) + { + return argument->isNullable() ? "Nullable(Bool)" : "Bool"; + } + public: static constexpr auto name = "toBool"; @@ -32,8 +39,7 @@ namespace DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - auto bool_type = DataTypeFactory::instance().get("Bool"); - return arguments[0]->isNullable() ? makeNullable(bool_type) : bool_type; + return DataTypeFactory::instance().get(getReturnTypeName(arguments[0])); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t) const override @@ -42,18 +48,17 @@ namespace { arguments[0], { - DataTypeString().createColumnConst(arguments[0].column->size(), arguments[0].type->isNullable() ? "Nullable(Bool)" : "Bool"), + DataTypeString().createColumnConst(arguments[0].column->size(), getReturnTypeName(arguments[0].type)), std::make_shared(), "" } }; - FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); + FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {}); auto func_cast = func_builder_cast->build(cast_args); return func_cast->execute(cast_args, result_type, arguments[0].column->size()); } }; - } REGISTER_FUNCTION(ToBool) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 61dd84d6c1d..3aad4cf9247 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -3,9 +3,9 @@ #include #include #include +#include #include #include -#include #include #include #include @@ -21,6 +21,7 @@ #include #include + namespace DB { @@ -248,7 +249,7 @@ const ActionsDAG::Node & ActionsDAG::addCast(const Node & node_to_cast, const Da const auto * cast_type_constant_node = &addColumn(std::move(column)); ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node}; - FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); + FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {}); return addFunction(func_builder_cast, std::move(children), result_name); } @@ -1381,9 +1382,9 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( const auto * right_arg = &actions_dag->addColumn(std::move(column)); const auto * left_arg = dst_node; - FunctionCastBase::Diagnostic diagnostic = {dst_node->result_name, res_elem.name}; + CastDiagnostic diagnostic = {dst_node->result_name, res_elem.name}; FunctionOverloadResolverPtr func_builder_cast - = CastInternalOverloadResolver::createImpl(std::move(diagnostic)); + = createInternalCastOverloadResolver(CastType::nonAccurate, std::move(diagnostic)); NodeRawConstPtrs children = { left_arg, right_arg }; dst_node = &actions_dag->addFunction(func_builder_cast, std::move(children), {}); diff --git a/src/Interpreters/castColumn.cpp b/src/Interpreters/castColumn.cpp index 44e669a21ab..906dfb84b14 100644 --- a/src/Interpreters/castColumn.cpp +++ b/src/Interpreters/castColumn.cpp @@ -1,13 +1,16 @@ #include - -#include #include +#include +#include +#include +#include +#include + namespace DB { -template -static ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache = nullptr) +static ColumnPtr castColumn(CastType cast_type, const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache = nullptr) { if (arg.type->equals(*type) && cast_type != CastType::accurateOrNull) return arg.column; @@ -23,37 +26,34 @@ static ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr "" } }; - auto get_cast_func = [&arguments] + auto get_cast_func = [cast_type, &arguments] { - FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); + + FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(cast_type, {}); return func_builder_cast->build(arguments); }; FunctionBasePtr func_cast = cache ? cache->getOrSet(cast_type, from_name, to_name, std::move(get_cast_func)) : get_cast_func(); - if constexpr (cast_type == CastType::accurateOrNull) - { + if (cast_type == CastType::accurateOrNull) return func_cast->execute(arguments, makeNullable(type), arg.column->size()); - } else - { return func_cast->execute(arguments, type, arg.column->size()); - } } ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache) { - return castColumn(arg, type, cache); + return castColumn(CastType::nonAccurate, arg, type, cache); } ColumnPtr castColumnAccurate(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache) { - return castColumn(arg, type, cache); + return castColumn(CastType::accurate, arg, type, cache); } ColumnPtr castColumnAccurateOrNull(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache) { - return castColumn(arg, type, cache); + return castColumn(CastType::accurateOrNull, arg, type, cache); } } diff --git a/src/Interpreters/castColumn.h b/src/Interpreters/castColumn.h index 8d2c05025bb..b9ed3403d0f 100644 --- a/src/Interpreters/castColumn.h +++ b/src/Interpreters/castColumn.h @@ -2,11 +2,15 @@ #include #include -#include +#include + namespace DB { +class IFunctionBase; +using FunctionBasePtr = std::shared_ptr; + struct InternalCastFunctionCache { private: @@ -15,7 +19,7 @@ private: std::map, FunctionBasePtr> impl; mutable std::mutex mutex; public: - template + template FunctionBasePtr getOrSet(CastType cast_type, const String & from, const String & to, Getter && getter) { std::lock_guard lock{mutex}; diff --git a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h index cf8c5848db5..24ffdc10581 100644 --- a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h +++ b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h @@ -1,22 +1,33 @@ #pragma once + #include "config.h" + #if USE_PARQUET || USE_ORC + #include #include #include +#include +#include +#include #include #include #include #include #include + + namespace arrow { -class Schema; -class DataType; -class Field; + class Schema; + class DataType; + class Field; } + + namespace DB { + namespace ErrorCodes { extern const int THERE_IS_NO_COLUMN; @@ -211,5 +222,7 @@ private: } } }; + } + #endif diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 2e0e09c4b18..d827353ad8e 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -35,7 +35,6 @@ #include #include -#include #include #include diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 8238886db4e..1884671a41d 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include @@ -12,6 +13,9 @@ #include #include +#include + + namespace DB { diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 4ac1662c741..2932bce4262 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1,6 +1,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -10,7 +13,6 @@ #include #include #include -#include #include #include #include @@ -18,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -1842,7 +1845,7 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme ColumnsWithTypeAndName arguments{ {nullptr, key_expr_type, ""}, {DataTypeString().createColumnConst(1, common_type_maybe_nullable->getName()), common_type_maybe_nullable, ""}}; - FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); + FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {}); auto func_cast = func_builder_cast->build(arguments); /// If we know the given range only contains one value, then we treat all functions as positive monotonic. diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index f858cb95846..43e3b0c505a 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -1,9 +1,12 @@ #include #include +#include #include #include +#include #include + namespace DB { @@ -160,7 +163,7 @@ const ActionsDAG::Node & addCast( const auto * cast_type_constant_node = &dag->addColumn(std::move(column)); ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node}; - FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); + FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {}); return addFunction(dag, func_builder_cast, std::move(children), node_remap); } diff --git a/src/Storages/PartitionedSink.cpp b/src/Storages/PartitionedSink.cpp index 18442a8691f..71c1dd7ab69 100644 --- a/src/Storages/PartitionedSink.cpp +++ b/src/Storages/PartitionedSink.cpp @@ -2,8 +2,6 @@ #include -#include - #include #include #include @@ -34,7 +32,7 @@ PartitionedSink::PartitionedSink( , sample_block(sample_block_) { ASTs arguments(1, partition_by); - ASTPtr partition_by_string = makeASTFunction(FunctionToString::name, std::move(arguments)); + ASTPtr partition_by_string = makeASTFunction("toString", std::move(arguments)); auto syntax_result = TreeRewriter(context).analyze(partition_by_string, sample_block.getNamesAndTypesList()); partition_by_expr = ExpressionAnalyzer(partition_by_string, syntax_result, context).getActions(false); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index a33e5884bf5..63ed84680c9 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -6,8 +6,6 @@ #include -#include - #include #include #include diff --git a/src/Storages/System/StorageSystemBackups.cpp b/src/Storages/System/StorageSystemBackups.cpp index 23578054492..46ab70ff04a 100644 --- a/src/Storages/System/StorageSystemBackups.cpp +++ b/src/Storages/System/StorageSystemBackups.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include From 3c15a5e7148114e51c76d899e80e4705ba6c389e Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 10 Nov 2023 17:24:50 +0100 Subject: [PATCH 05/28] hide VERSION_INLINE_DATA under storage_metadata_write_full_object_key feature --- src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp | 10 ++++++++-- src/Disks/ObjectStorages/DiskObjectStorageMetadata.h | 2 +- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp index e923e49a7f1..3271a190193 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp @@ -117,6 +117,9 @@ void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const if (storage_metadata_write_full_object_key) write_version = VERSION_FULL_OBJECT_KEY; + if (!inline_data.empty() && write_version < VERSION_INLINE_DATA) + write_version = VERSION_INLINE_DATA; + chassert(write_version >= VERSION_ABSOLUTE_PATHS && write_version <= VERSION_FULL_OBJECT_KEY); writeIntText(write_version, buf); writeChar('\n', buf); @@ -153,8 +156,11 @@ void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const writeBoolText(read_only, buf); writeChar('\n', buf); - writeEscapedString(inline_data, buf); - writeChar('\n', buf); + if (write_version >= VERSION_INLINE_DATA) + { + writeEscapedString(inline_data, buf); + writeChar('\n', buf); + } buf.finalize(); if (sync) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h index 658914b7611..729d93af10d 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h @@ -19,7 +19,7 @@ private: static constexpr UInt32 VERSION_INLINE_DATA = 4; static constexpr UInt32 VERSION_FULL_OBJECT_KEY = 5; /// only for reading data - UInt32 version = VERSION_INLINE_DATA; + UInt32 version = VERSION_READ_ONLY_FLAG; /// Absolute paths of blobs ObjectKeysWithMetadata keys_with_meta; From 4de5ee8acc0121ba95a22033c0ef60fa2e409a90 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 01:01:43 +0100 Subject: [PATCH 06/28] Something --- src/AggregateFunctions/AggregateFunctionUniq.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index 1752d5751d5..f20fb8cb933 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -466,7 +466,7 @@ public: std::vector data_vec; data_vec.resize(places.size()); - for (unsigned long i = 0; i < data_vec.size(); i++) + for (size_t i = 0; i < data_vec.size(); ++i) data_vec[i] = &this->data(places[i]).set; DataSet::parallelizeMergePrepare(data_vec, thread_pool); From a90f3b14e8bbe08f6cf5c781f33947e67fad8c3a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 02:21:24 +0100 Subject: [PATCH 07/28] Split FunctionsStringArray --- src/Functions/FunctionTokens.h | 208 ++++ src/Functions/FunctionsStringArray.cpp | 73 -- src/Functions/FunctionsStringArray.h | 989 ------------------ src/Functions/URL/URLHierarchy.cpp | 12 +- src/Functions/URL/URLPathHierarchy.cpp | 11 +- .../URL/extractURLParameterNames.cpp | 11 +- src/Functions/URL/extractURLParameters.cpp | 12 +- src/Functions/alphaTokens.cpp | 111 ++ src/Functions/arrayStringConcat.cpp | 221 ++++ src/Functions/extractAll.cpp | 123 +++ src/Functions/splitByChar.cpp | 122 +++ src/Functions/splitByNonAlpha.cpp | 120 +++ src/Functions/splitByRegexp.cpp | 157 +++ src/Functions/splitByString.cpp | 149 +++ src/Functions/splitByWhitespace.cpp | 108 ++ 15 files changed, 1353 insertions(+), 1074 deletions(-) create mode 100644 src/Functions/FunctionTokens.h delete mode 100644 src/Functions/FunctionsStringArray.cpp delete mode 100644 src/Functions/FunctionsStringArray.h create mode 100644 src/Functions/alphaTokens.cpp create mode 100644 src/Functions/arrayStringConcat.cpp create mode 100644 src/Functions/extractAll.cpp create mode 100644 src/Functions/splitByChar.cpp create mode 100644 src/Functions/splitByNonAlpha.cpp create mode 100644 src/Functions/splitByRegexp.cpp create mode 100644 src/Functions/splitByString.cpp create mode 100644 src/Functions/splitByWhitespace.cpp diff --git a/src/Functions/FunctionTokens.h b/src/Functions/FunctionTokens.h new file mode 100644 index 00000000000..a79d37a7a87 --- /dev/null +++ b/src/Functions/FunctionTokens.h @@ -0,0 +1,208 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; +} + + +/** Functions that split strings into an array of strings or vice versa. + * + * splitByChar(sep, s[, max_substrings]) + * splitByString(sep, s[, max_substrings]) + * splitByRegexp(regexp, s[, max_substrings]) + * + * splitByWhitespace(s[, max_substrings]) - split the string by whitespace characters + * splitByNonAlpha(s[, max_substrings]) - split the string by whitespace and punctuation characters + * + * extractAll(s, regexp) - select from the string the subsequences corresponding to the regexp. + * - first subpattern, if regexp has subpattern; + * - zero subpattern (the match part, otherwise); + * - otherwise, an empty array + * + * alphaTokens(s[, max_substrings]) - select from the string subsequence `[a-zA-Z]+`. + * + * URL functions are located separately. + */ + + +/// A function that takes a string, and returns an array of substrings created by some generator. +template +class FunctionTokens : public IFunction +{ +private: + using Pos = const char *; + bool max_substrings_includes_remaining_string; + +public: + static constexpr auto name = Generator::name; + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + + explicit FunctionTokens(ContextPtr context) + { + const Settings & settings = context->getSettingsRef(); + max_substrings_includes_remaining_string = settings.splitby_max_substrings_includes_remaining_string; + } + + String getName() const override { return name; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + bool isVariadic() const override { return Generator::isVariadic(); } + + size_t getNumberOfArguments() const override { return Generator::getNumberOfArguments(); } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + Generator::checkArguments(*this, arguments); + + return std::make_shared(std::make_shared()); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override + { + Generator generator; + generator.init(arguments, max_substrings_includes_remaining_string); + + const auto & array_argument = arguments[generator.strings_argument_position]; + + const ColumnString * col_str = checkAndGetColumn(array_argument.column.get()); + const ColumnConst * col_str_const = checkAndGetColumnConstStringOrFixedString(array_argument.column.get()); + + auto col_res = ColumnArray::create(ColumnString::create()); + + ColumnString & res_strings = typeid_cast(col_res->getData()); + ColumnString::Chars & res_strings_chars = res_strings.getChars(); + ColumnString::Offsets & res_strings_offsets = res_strings.getOffsets(); + + ColumnArray::Offsets & res_offsets = col_res->getOffsets(); + + if (col_str) + { + const ColumnString::Chars & src_chars = col_str->getChars(); + const ColumnString::Offsets & src_offsets = col_str->getOffsets(); + + res_offsets.reserve(src_offsets.size()); + res_strings_offsets.reserve(src_offsets.size() * 5); /// Constant 5 - at random. + res_strings_chars.reserve(src_chars.size()); + + Pos token_begin = nullptr; + Pos token_end = nullptr; + + size_t size = src_offsets.size(); + ColumnString::Offset current_src_offset = 0; + ColumnArray::Offset current_dst_offset = 0; + ColumnString::Offset current_dst_strings_offset = 0; + for (size_t i = 0; i < size; ++i) + { + Pos pos = reinterpret_cast(&src_chars[current_src_offset]); + current_src_offset = src_offsets[i]; + Pos end = reinterpret_cast(&src_chars[current_src_offset]) - 1; + + generator.set(pos, end); + size_t j = 0; + while (generator.get(token_begin, token_end)) + { + size_t token_size = token_end - token_begin; + + res_strings_chars.resize(res_strings_chars.size() + token_size + 1); + memcpySmallAllowReadWriteOverflow15(&res_strings_chars[current_dst_strings_offset], token_begin, token_size); + res_strings_chars[current_dst_strings_offset + token_size] = 0; + + current_dst_strings_offset += token_size + 1; + res_strings_offsets.push_back(current_dst_strings_offset); + ++j; + } + + current_dst_offset += j; + res_offsets.push_back(current_dst_offset); + } + + return col_res; + } + else if (col_str_const) + { + String src = col_str_const->getValue(); + Array dst; + + generator.set(src.data(), src.data() + src.size()); + Pos token_begin = nullptr; + Pos token_end = nullptr; + + while (generator.get(token_begin, token_end)) + dst.push_back(String(token_begin, token_end - token_begin)); + + return result_type->createColumnConst(col_str_const->size(), dst); + } + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns {}, {} of arguments of function {}", + array_argument.column->getName(), array_argument.column->getName(), getName()); + } +}; + + +/// Helper functions for implementations +static inline std::optional extractMaxSplits( + const ColumnsWithTypeAndName & arguments, size_t max_substrings_argument_position) +{ + if (max_substrings_argument_position >= arguments.size()) + return std::nullopt; + + if (const ColumnConst * column = checkAndGetColumn(arguments[max_substrings_argument_position].column.get())) + return column->getUInt(0); + + return std::nullopt; +} + +static inline void checkArgumentsWithSeparatorAndOptionalMaxSubstrings( + const IFunction & func, const ColumnsWithTypeAndName & arguments) +{ + FunctionArgumentDescriptors mandatory_args{ + {"separator", &isString, isColumnConst, "const String"}, + {"s", &isString, nullptr, "String"} + }; + + FunctionArgumentDescriptors optional_args{ + {"max_substrings", &isNativeInteger, isColumnConst, "const Number"}, + }; + + validateFunctionArgumentTypes(func, arguments, mandatory_args, optional_args); +} + +static inline void checkArgumentsWithOptionalMaxSubstrings(const IFunction & func, const ColumnsWithTypeAndName & arguments) +{ + FunctionArgumentDescriptors mandatory_args{ + {"s", &isString, nullptr, "String"}, + }; + + FunctionArgumentDescriptors optional_args{ + {"max_substrings", &isNativeInteger, isColumnConst, "const Number"}, + }; + + validateFunctionArgumentTypes(func, arguments, mandatory_args, optional_args); +} + +} diff --git a/src/Functions/FunctionsStringArray.cpp b/src/Functions/FunctionsStringArray.cpp deleted file mode 100644 index 4afee55704f..00000000000 --- a/src/Functions/FunctionsStringArray.cpp +++ /dev/null @@ -1,73 +0,0 @@ -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; -} - -template -std::optional extractMaxSplitsImpl(const ColumnWithTypeAndName & argument) -{ - const auto * col = checkAndGetColumnConst>(argument.column.get()); - if (!col) - return std::nullopt; - - auto value = col->template getValue(); - return static_cast(value); -} - -std::optional extractMaxSplits(const ColumnsWithTypeAndName & arguments, size_t max_substrings_argument_position) -{ - if (max_substrings_argument_position >= arguments.size()) - return std::nullopt; - - std::optional max_splits; - if (!((max_splits = extractMaxSplitsImpl(arguments[max_substrings_argument_position])) || (max_splits = extractMaxSplitsImpl(arguments[max_substrings_argument_position])) - || (max_splits = extractMaxSplitsImpl(arguments[max_substrings_argument_position])) || (max_splits = extractMaxSplitsImpl(arguments[max_substrings_argument_position])) - || (max_splits = extractMaxSplitsImpl(arguments[max_substrings_argument_position])) || (max_splits = extractMaxSplitsImpl(arguments[max_substrings_argument_position])) - || (max_splits = extractMaxSplitsImpl(arguments[max_substrings_argument_position])) || (max_splits = extractMaxSplitsImpl(arguments[max_substrings_argument_position])))) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {}, which is {}-th argument", - arguments[max_substrings_argument_position].column->getName(), - max_substrings_argument_position + 1); - - if (*max_splits <= 0) - return std::nullopt; - - return max_splits; -} - -DataTypePtr FunctionArrayStringConcat::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const -{ - FunctionArgumentDescriptors mandatory_args{ - {"arr", &isArray, nullptr, "Array"}, - }; - - FunctionArgumentDescriptors optional_args{ - {"separator", &isString, isColumnConst, "const String"}, - }; - - validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); - - return std::make_shared(); -} - -REGISTER_FUNCTION(StringArray) -{ - factory.registerFunction(); - - factory.registerFunction(); - factory.registerAlias("splitByAlpha", FunctionSplitByAlpha::name); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); -} - -} diff --git a/src/Functions/FunctionsStringArray.h b/src/Functions/FunctionsStringArray.h deleted file mode 100644 index 22b19daf149..00000000000 --- a/src/Functions/FunctionsStringArray.h +++ /dev/null @@ -1,989 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_COLUMN; -} - - -/** Functions that split strings into an array of strings or vice versa. - * - * splitByChar(sep, s[, max_substrings]) - * splitByString(sep, s[, max_substrings]) - * splitByRegexp(regexp, s[, max_substrings]) - * - * splitByWhitespace(s[, max_substrings]) - split the string by whitespace characters - * splitByNonAlpha(s[, max_substrings]) - split the string by whitespace and punctuation characters - * - * extractAll(s, regexp) - select from the string the subsequences corresponding to the regexp. - * - first subpattern, if regexp has subpattern; - * - zero subpattern (the match part, otherwise); - * - otherwise, an empty array - * - * arrayStringConcat(arr) - * arrayStringConcat(arr, delimiter) - * - join an array of strings into one string via a separator. - * - * alphaTokens(s[, max_substrings]) - select from the string subsequence `[a-zA-Z]+`. - * - * URL functions are located separately. - */ - - -using Pos = const char *; - -std::optional extractMaxSplits(const ColumnsWithTypeAndName & arguments, size_t max_substrings_argument_position); - -/// Substring generators. All of them have a common interface. - -class SplitByAlphaImpl -{ -private: - Pos pos; - Pos end; - std::optional max_splits; - size_t splits; - bool max_substrings_includes_remaining_string; - -public: - static constexpr auto name = "alphaTokens"; - static String getName() { return name; } - - static bool isVariadic() { return true; } - - static size_t getNumberOfArguments() { return 0; } - - static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) - { - FunctionArgumentDescriptors mandatory_args{ - {"s", &isString, nullptr, "String"}, - }; - - FunctionArgumentDescriptors optional_args{ - {"max_substrings", &isNativeInteger, isColumnConst, "const Number"}, - }; - - validateFunctionArgumentTypes(func, arguments, mandatory_args, optional_args); - } - - static constexpr auto strings_argument_position = 0uz; - - void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_) - { - max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_; - max_splits = extractMaxSplits(arguments, 1); - } - - /// Called for each next string. - void set(Pos pos_, Pos end_) - { - pos = pos_; - end = end_; - splits = 0; - } - - /// Get the next token, if any, or return false. - bool get(Pos & token_begin, Pos & token_end) - { - /// Skip garbage - while (pos < end && !isAlphaASCII(*pos)) - ++pos; - - if (pos == end) - return false; - - token_begin = pos; - - if (max_splits) - { - if (max_substrings_includes_remaining_string) - { - if (splits == *max_splits - 1) - { - token_end = end; - pos = end; - return true; - } - } - else - if (splits == *max_splits) - return false; - } - - while (pos < end && isAlphaASCII(*pos)) - ++pos; - - token_end = pos; - ++splits; - - return true; - } -}; - -class SplitByNonAlphaImpl -{ -private: - Pos pos; - Pos end; - std::optional max_splits; - size_t splits; - bool max_substrings_includes_remaining_string; - -public: - /// Get the name of the function. - static constexpr auto name = "splitByNonAlpha"; - static String getName() { return name; } - - static bool isVariadic() { return true; } - static size_t getNumberOfArguments() { return 0; } - - static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) - { - SplitByAlphaImpl::checkArguments(func, arguments); - } - - static constexpr auto strings_argument_position = 0uz; - - void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_) - { - max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_; - max_splits = extractMaxSplits(arguments, 1); - } - - /// Called for each next string. - void set(Pos pos_, Pos end_) - { - pos = pos_; - end = end_; - splits = 0; - } - - /// Get the next token, if any, or return false. - bool get(Pos & token_begin, Pos & token_end) - { - /// Skip garbage - while (pos < end && (isWhitespaceASCII(*pos) || isPunctuationASCII(*pos))) - ++pos; - - if (pos == end) - return false; - - token_begin = pos; - - if (max_splits) - { - if (max_substrings_includes_remaining_string) - { - if (splits == *max_splits - 1) - { - token_end = end; - pos = end; - return true; - } - } - else - if (splits == *max_splits) - return false; - } - - while (pos < end && !(isWhitespaceASCII(*pos) || isPunctuationASCII(*pos))) - ++pos; - - token_end = pos; - splits++; - - return true; - } -}; - -class SplitByWhitespaceImpl -{ -private: - Pos pos; - Pos end; - std::optional max_splits; - size_t splits; - bool max_substrings_includes_remaining_string; - -public: - static constexpr auto name = "splitByWhitespace"; - static String getName() { return name; } - - static bool isVariadic() { return true; } - static size_t getNumberOfArguments() { return 0; } - - static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) - { - return SplitByNonAlphaImpl::checkArguments(func, arguments); - } - - static constexpr auto strings_argument_position = 0uz; - - void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_) - { - max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_; - max_splits = extractMaxSplits(arguments, 1); - } - - /// Called for each next string. - void set(Pos pos_, Pos end_) - { - pos = pos_; - end = end_; - splits = 0; - } - - /// Get the next token, if any, or return false. - bool get(Pos & token_begin, Pos & token_end) - { - /// Skip garbage - while (pos < end && isWhitespaceASCII(*pos)) - ++pos; - - if (pos == end) - return false; - - token_begin = pos; - - if (max_splits) - { - if (max_substrings_includes_remaining_string) - { - if (splits == *max_splits - 1) - { - token_end = end; - pos = end; - return true; - } - } - else - if (splits == *max_splits) - return false; - } - - while (pos < end && !isWhitespaceASCII(*pos)) - ++pos; - - token_end = pos; - splits++; - - return true; - } -}; - -class SplitByCharImpl -{ -private: - Pos pos; - Pos end; - char separator; - std::optional max_splits; - size_t splits; - bool max_substrings_includes_remaining_string; - -public: - static constexpr auto name = "splitByChar"; - static String getName() { return name; } - static bool isVariadic() { return true; } - static size_t getNumberOfArguments() { return 0; } - - static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) - { - FunctionArgumentDescriptors mandatory_args{ - {"separator", &isString, isColumnConst, "const String"}, - {"s", &isString, nullptr, "String"} - }; - - FunctionArgumentDescriptors optional_args{ - {"max_substrings", &isNativeInteger, isColumnConst, "const Number"}, - }; - - validateFunctionArgumentTypes(func, arguments, mandatory_args, optional_args); - } - - static constexpr auto strings_argument_position = 1uz; - - void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_) - { - const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get()); - - if (!col) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. " - "Must be constant string.", arguments[0].column->getName(), getName()); - - String sep_str = col->getValue(); - - if (sep_str.size() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal separator for function {}. Must be exactly one byte.", getName()); - - separator = sep_str[0]; - - max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_; - max_splits = extractMaxSplits(arguments, 2); - } - - void set(Pos pos_, Pos end_) - { - pos = pos_; - end = end_; - splits = 0; - } - - bool get(Pos & token_begin, Pos & token_end) - { - if (!pos) - return false; - - token_begin = pos; - - if (max_splits) - { - if (max_substrings_includes_remaining_string) - { - if (splits == *max_splits - 1) - { - token_end = end; - pos = nullptr; - return true; - } - } - else - if (splits == *max_splits) - return false; - } - - pos = reinterpret_cast(memchr(pos, separator, end - pos)); - if (pos) - { - token_end = pos; - ++pos; - ++splits; - } - else - token_end = end; - - return true; - } -}; - - -class SplitByStringImpl -{ -private: - Pos pos; - Pos end; - String separator; - std::optional max_splits; - size_t splits; - bool max_substrings_includes_remaining_string; - -public: - static constexpr auto name = "splitByString"; - static String getName() { return name; } - static bool isVariadic() { return true; } - static size_t getNumberOfArguments() { return 0; } - - static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) - { - SplitByCharImpl::checkArguments(func, arguments); - } - - static constexpr auto strings_argument_position = 1uz; - - void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_) - { - const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get()); - - if (!col) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. " - "Must be constant string.", arguments[0].column->getName(), getName()); - - separator = col->getValue(); - - max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_; - max_splits = extractMaxSplits(arguments, 2); - } - - /// Called for each next string. - void set(Pos pos_, Pos end_) - { - pos = pos_; - end = end_; - splits = 0; - } - - /// Get the next token, if any, or return false. - bool get(Pos & token_begin, Pos & token_end) - { - if (separator.empty()) - { - if (pos == end) - return false; - - token_begin = pos; - - if (max_splits) - { - if (max_substrings_includes_remaining_string) - { - if (splits == *max_splits - 1) - { - token_end = end; - pos = end; - return true; - } - } - else - if (splits == *max_splits) - return false; - } - - pos += 1; - token_end = pos; - ++splits; - } - else - { - if (!pos) - return false; - - token_begin = pos; - - if (max_splits) - { - if (max_substrings_includes_remaining_string) - { - if (splits == *max_splits - 1) - { - token_end = end; - pos = nullptr; - return true; - } - } - else - if (splits == *max_splits) - return false; - } - - pos = reinterpret_cast(memmem(pos, end - pos, separator.data(), separator.size())); - if (pos) - { - token_end = pos; - pos += separator.size(); - ++splits; - } - else - token_end = end; - } - - return true; - } -}; - -class SplitByRegexpImpl -{ -private: - Regexps::RegexpPtr re; - OptimizedRegularExpression::MatchVec matches; - - Pos pos; - Pos end; - - std::optional max_splits; - size_t splits; - bool max_substrings_includes_remaining_string; - -public: - static constexpr auto name = "splitByRegexp"; - static String getName() { return name; } - - static bool isVariadic() { return true; } - static size_t getNumberOfArguments() { return 0; } - - static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) - { - SplitByStringImpl::checkArguments(func, arguments); - } - - static constexpr auto strings_argument_position = 1uz; - - void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_) - { - const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get()); - - if (!col) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. " - "Must be constant string.", arguments[0].column->getName(), getName()); - - if (!col->getValue().empty()) - re = std::make_shared(Regexps::createRegexp(col->getValue())); - - max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_; - max_splits = extractMaxSplits(arguments, 2); - } - - /// Called for each next string. - void set(Pos pos_, Pos end_) - { - pos = pos_; - end = end_; - splits = 0; - } - - /// Get the next token, if any, or return false. - bool get(Pos & token_begin, Pos & token_end) - { - if (!re) - { - if (pos == end) - return false; - - token_begin = pos; - - if (max_splits) - { - if (max_substrings_includes_remaining_string) - { - if (splits == *max_splits - 1) - { - token_end = end; - pos = end; - return true; - } - } - else - if (splits == *max_splits) - return false; - } - - pos += 1; - token_end = pos; - ++splits; - } - else - { - if (!pos || pos > end) - return false; - - token_begin = pos; - - if (max_splits) - { - if (max_substrings_includes_remaining_string) - { - if (splits == *max_splits - 1) - { - token_end = end; - pos = nullptr; - return true; - } - } - else - if (splits == *max_splits) - return false; - } - - if (!re->match(pos, end - pos, matches) || !matches[0].length) - { - token_end = end; - pos = end + 1; - } - else - { - token_end = pos + matches[0].offset; - pos = token_end + matches[0].length; - ++splits; - } - } - - return true; - } -}; - -class ExtractAllImpl -{ -private: - Regexps::RegexpPtr re; - OptimizedRegularExpression::MatchVec matches; - size_t capture; - - Pos pos; - Pos end; -public: - static constexpr auto name = "extractAll"; - static String getName() { return name; } - static bool isVariadic() { return false; } - static size_t getNumberOfArguments() { return 2; } - - static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) - { - FunctionArgumentDescriptors mandatory_args{ - {"haystack", &isString, nullptr, "String"}, - {"pattern", &isString, isColumnConst, "const String"} - }; - - validateFunctionArgumentTypes(func, arguments, mandatory_args); - } - - static constexpr auto strings_argument_position = 0uz; - - void init(const ColumnsWithTypeAndName & arguments, bool /*max_substrings_includes_remaining_string*/) - { - const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get()); - - if (!col) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. " - "Must be constant string.", arguments[1].column->getName(), getName()); - - re = std::make_shared(Regexps::createRegexp(col->getValue())); - capture = re->getNumberOfSubpatterns() > 0 ? 1 : 0; - - matches.resize(capture + 1); - } - - /// Called for each next string. - void set(Pos pos_, Pos end_) - { - pos = pos_; - end = end_; - } - - /// Get the next token, if any, or return false. - bool get(Pos & token_begin, Pos & token_end) - { - if (!pos || pos > end) - return false; - - if (!re->match(pos, end - pos, matches) || !matches[0].length) - return false; - - if (matches[capture].offset == std::string::npos) - { - /// Empty match. - token_begin = pos; - token_end = pos; - } - else - { - token_begin = pos + matches[capture].offset; - token_end = token_begin + matches[capture].length; - } - - pos += matches[0].offset + matches[0].length; - - return true; - } -}; - -/// A function that takes a string, and returns an array of substrings created by some generator. -template -class FunctionTokens : public IFunction -{ -private: - bool max_substrings_includes_remaining_string; - -public: - static constexpr auto name = Generator::name; - static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - - explicit FunctionTokens(ContextPtr context) - { - const Settings & settings = context->getSettingsRef(); - max_substrings_includes_remaining_string = settings.splitby_max_substrings_includes_remaining_string; - } - - String getName() const override { return name; } - - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - - bool isVariadic() const override { return Generator::isVariadic(); } - - size_t getNumberOfArguments() const override { return Generator::getNumberOfArguments(); } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - Generator::checkArguments(*this, arguments); - - return std::make_shared(std::make_shared()); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override - { - Generator generator; - generator.init(arguments, max_substrings_includes_remaining_string); - - const auto & array_argument = arguments[generator.strings_argument_position]; - - const ColumnString * col_str = checkAndGetColumn(array_argument.column.get()); - const ColumnConst * col_str_const = checkAndGetColumnConstStringOrFixedString(array_argument.column.get()); - - auto col_res = ColumnArray::create(ColumnString::create()); - - ColumnString & res_strings = typeid_cast(col_res->getData()); - ColumnString::Chars & res_strings_chars = res_strings.getChars(); - ColumnString::Offsets & res_strings_offsets = res_strings.getOffsets(); - - ColumnArray::Offsets & res_offsets = col_res->getOffsets(); - - if (col_str) - { - const ColumnString::Chars & src_chars = col_str->getChars(); - const ColumnString::Offsets & src_offsets = col_str->getOffsets(); - - res_offsets.reserve(src_offsets.size()); - res_strings_offsets.reserve(src_offsets.size() * 5); /// Constant 5 - at random. - res_strings_chars.reserve(src_chars.size()); - - Pos token_begin = nullptr; - Pos token_end = nullptr; - - size_t size = src_offsets.size(); - ColumnString::Offset current_src_offset = 0; - ColumnArray::Offset current_dst_offset = 0; - ColumnString::Offset current_dst_strings_offset = 0; - for (size_t i = 0; i < size; ++i) - { - Pos pos = reinterpret_cast(&src_chars[current_src_offset]); - current_src_offset = src_offsets[i]; - Pos end = reinterpret_cast(&src_chars[current_src_offset]) - 1; - - generator.set(pos, end); - size_t j = 0; - while (generator.get(token_begin, token_end)) - { - size_t token_size = token_end - token_begin; - - res_strings_chars.resize(res_strings_chars.size() + token_size + 1); - memcpySmallAllowReadWriteOverflow15(&res_strings_chars[current_dst_strings_offset], token_begin, token_size); - res_strings_chars[current_dst_strings_offset + token_size] = 0; - - current_dst_strings_offset += token_size + 1; - res_strings_offsets.push_back(current_dst_strings_offset); - ++j; - } - - current_dst_offset += j; - res_offsets.push_back(current_dst_offset); - } - - return col_res; - } - else if (col_str_const) - { - String src = col_str_const->getValue(); - Array dst; - - generator.set(src.data(), src.data() + src.size()); - Pos token_begin = nullptr; - Pos token_end = nullptr; - - while (generator.get(token_begin, token_end)) - dst.push_back(String(token_begin, token_end - token_begin)); - - return result_type->createColumnConst(col_str_const->size(), dst); - } - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns {}, {} of arguments of function {}", - array_argument.column->getName(), array_argument.column->getName(), getName()); - } -}; - - -/// Joins an array of type serializable to string into one string via a separator. -class FunctionArrayStringConcat : public IFunction -{ -private: - static void executeInternal( - const ColumnString::Chars & src_chars, - const ColumnString::Offsets & src_string_offsets, - const ColumnArray::Offsets & src_array_offsets, - const char * delimiter, - const size_t delimiter_size, - ColumnString::Chars & dst_chars, - ColumnString::Offsets & dst_string_offsets, - const char8_t * null_map) - { - size_t size = src_array_offsets.size(); - - if (!size) - return; - - /// With a small margin - as if the separator goes after the last string of the array. - dst_chars.resize( - src_chars.size() - + delimiter_size * src_string_offsets.size() /// Separators after each string... - + src_array_offsets.size() /// Zero byte after each joined string - - src_string_offsets.size()); /// The former zero byte after each string of the array - - /// There will be as many strings as there were arrays. - dst_string_offsets.resize(src_array_offsets.size()); - - ColumnArray::Offset current_src_array_offset = 0; - - ColumnString::Offset current_dst_string_offset = 0; - - /// Loop through the array of strings. - for (size_t i = 0; i < size; ++i) - { - bool first_non_null = true; - /// Loop through the rows within the array. /// NOTE You can do everything in one copy, if the separator has a size of 1. - for (auto next_src_array_offset = src_array_offsets[i]; current_src_array_offset < next_src_array_offset; ++current_src_array_offset) - { - if (null_map && null_map[current_src_array_offset]) [[unlikely]] - continue; - - if (!first_non_null) - { - memcpy(&dst_chars[current_dst_string_offset], delimiter, delimiter_size); - current_dst_string_offset += delimiter_size; - } - first_non_null = false; - - const auto current_src_string_offset = current_src_array_offset ? src_string_offsets[current_src_array_offset - 1] : 0; - size_t bytes_to_copy = src_string_offsets[current_src_array_offset] - current_src_string_offset - 1; - - memcpySmallAllowReadWriteOverflow15( - &dst_chars[current_dst_string_offset], &src_chars[current_src_string_offset], bytes_to_copy); - - current_dst_string_offset += bytes_to_copy; - } - - dst_chars[current_dst_string_offset] = 0; - ++current_dst_string_offset; - - dst_string_offsets[i] = current_dst_string_offset; - } - - dst_chars.resize(dst_string_offsets.back()); - } - - static void executeInternal( - const ColumnString & col_string, - const ColumnArray & col_arr, - const String & delimiter, - ColumnString & col_res, - const char8_t * null_map = nullptr) - { - executeInternal( - col_string.getChars(), - col_string.getOffsets(), - col_arr.getOffsets(), - delimiter.data(), - delimiter.size(), - col_res.getChars(), - col_res.getOffsets(), - null_map); - } - - static ColumnPtr serializeNestedColumn(const ColumnArray & col_arr, const DataTypePtr & nested_type) - { - if (isString(nested_type)) - { - return col_arr.getDataPtr(); - } - else if (const ColumnNullable * col_nullable = checkAndGetColumn(col_arr.getData()); - col_nullable && isString(col_nullable->getNestedColumn().getDataType())) - { - return col_nullable->getNestedColumnPtr(); - } - else - { - return castColumn({col_arr.getDataPtr(), nested_type, "tmp"}, std::make_shared()); - } - } - -public: - static constexpr auto name = "arrayStringConcat"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - - String getName() const override - { - return name; - } - - bool isVariadic() const override { return true; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override - { - String delimiter; - if (arguments.size() == 2) - { - const ColumnConst * col_delim = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get()); - if (!col_delim) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument for function {} must be constant string.", getName()); - - delimiter = col_delim->getValue(); - } - - const auto & nested_type = assert_cast(*arguments[0].type).getNestedType(); - if (const ColumnConst * col_const_arr = checkAndGetColumnConst(arguments[0].column.get()); - col_const_arr && isString(nested_type)) - { - Array src_arr = col_const_arr->getValue(); - String dst_str; - bool first_non_null = true; - for (size_t i = 0, size = src_arr.size(); i < size; ++i) - { - if (src_arr[i].isNull()) - continue; - if (!first_non_null) - dst_str += delimiter; - first_non_null = false; - dst_str += src_arr[i].get(); - } - - return result_type->createColumnConst(col_const_arr->size(), dst_str); - } - - ColumnPtr src_column = arguments[0].column->convertToFullColumnIfConst(); - const ColumnArray & col_arr = assert_cast(*src_column.get()); - - ColumnPtr str_subcolumn = serializeNestedColumn(col_arr, nested_type); - const ColumnString & col_string = assert_cast(*str_subcolumn.get()); - - auto col_res = ColumnString::create(); - if (const ColumnNullable * col_nullable = checkAndGetColumn(col_arr.getData())) - executeInternal(col_string, col_arr, delimiter, *col_res, col_nullable->getNullMapData().data()); - else - executeInternal(col_string, col_arr, delimiter, *col_res); - return col_res; - } -}; - - -using FunctionSplitByAlpha = FunctionTokens; -using FunctionSplitByNonAlpha = FunctionTokens; -using FunctionSplitByWhitespace = FunctionTokens; -using FunctionSplitByChar = FunctionTokens; -using FunctionSplitByString = FunctionTokens; -using FunctionSplitByRegexp = FunctionTokens; -using FunctionExtractAll = FunctionTokens; - -} diff --git a/src/Functions/URL/URLHierarchy.cpp b/src/Functions/URL/URLHierarchy.cpp index 96b64d3182b..25c6c9ef40b 100644 --- a/src/Functions/URL/URLHierarchy.cpp +++ b/src/Functions/URL/URLHierarchy.cpp @@ -1,9 +1,15 @@ #include -#include +#include + namespace DB { +namespace +{ + +using Pos = const char *; + class URLPathHierarchyImpl { private: @@ -14,7 +20,6 @@ private: public: static constexpr auto name = "URLPathHierarchy"; - static String getName() { return name; } static bool isVariadic() { return false; } static size_t getNumberOfArguments() { return 1; } @@ -95,9 +100,10 @@ public: }; -struct NameURLPathHierarchy { static constexpr auto name = "URLPathHierarchy"; }; using FunctionURLPathHierarchy = FunctionTokens; +} + REGISTER_FUNCTION(URLPathHierarchy) { factory.registerFunction(); diff --git a/src/Functions/URL/URLPathHierarchy.cpp b/src/Functions/URL/URLPathHierarchy.cpp index 7fd6601d780..9a60d4cf989 100644 --- a/src/Functions/URL/URLPathHierarchy.cpp +++ b/src/Functions/URL/URLPathHierarchy.cpp @@ -1,9 +1,14 @@ #include -#include +#include namespace DB { +namespace +{ + +using Pos = const char *; + class URLHierarchyImpl { private: @@ -13,7 +18,6 @@ private: public: static constexpr auto name = "URLHierarchy"; - static String getName() { return name; } static bool isVariadic() { return false; } static size_t getNumberOfArguments() { return 1; } @@ -97,9 +101,10 @@ public: }; -struct NameURLHierarchy { static constexpr auto name = "URLHierarchy"; }; using FunctionURLHierarchy = FunctionTokens; +} + REGISTER_FUNCTION(URLHierarchy) { factory.registerFunction(); diff --git a/src/Functions/URL/extractURLParameterNames.cpp b/src/Functions/URL/extractURLParameterNames.cpp index b792d9140d6..08da148b43e 100644 --- a/src/Functions/URL/extractURLParameterNames.cpp +++ b/src/Functions/URL/extractURLParameterNames.cpp @@ -1,9 +1,14 @@ #include -#include +#include namespace DB { +namespace +{ + +using Pos = const char *; + class ExtractURLParameterNamesImpl { private: @@ -13,7 +18,6 @@ private: public: static constexpr auto name = "extractURLParameterNames"; - static String getName() { return name; } static bool isVariadic() { return false; } static size_t getNumberOfArguments() { return 1; } @@ -80,9 +84,10 @@ public: } }; -struct NameExtractURLParameterNames { static constexpr auto name = "extractURLParameterNames"; }; using FunctionExtractURLParameterNames = FunctionTokens; +} + REGISTER_FUNCTION(ExtractURLParameterNames) { factory.registerFunction(); diff --git a/src/Functions/URL/extractURLParameters.cpp b/src/Functions/URL/extractURLParameters.cpp index e1243d8fbcd..939622dd9d1 100644 --- a/src/Functions/URL/extractURLParameters.cpp +++ b/src/Functions/URL/extractURLParameters.cpp @@ -1,9 +1,15 @@ #include -#include +#include + namespace DB { +namespace +{ + +using Pos = const char *; + class ExtractURLParametersImpl { private: @@ -13,7 +19,6 @@ private: public: static constexpr auto name = "extractURLParameters"; - static String getName() { return name; } static bool isVariadic() { return false; } static size_t getNumberOfArguments() { return 1; } @@ -88,9 +93,10 @@ public: } }; -struct NameExtractURLParameters { static constexpr auto name = "extractURLParameters"; }; using FunctionExtractURLParameters = FunctionTokens; +} + REGISTER_FUNCTION(ExtractURLParameters) { factory.registerFunction(); diff --git a/src/Functions/alphaTokens.cpp b/src/Functions/alphaTokens.cpp new file mode 100644 index 00000000000..9b20509eee0 --- /dev/null +++ b/src/Functions/alphaTokens.cpp @@ -0,0 +1,111 @@ + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; +} + + +/** Functions that split strings into an array of strings or vice versa. + * + * alphaTokens(s[, max_substrings]) - select from the string subsequence `[a-zA-Z]+`. + */ +namespace +{ + +using Pos = const char *; + +class SplitByAlphaImpl +{ +private: + Pos pos; + Pos end; + std::optional max_splits; + size_t splits; + bool max_substrings_includes_remaining_string; + +public: + static constexpr auto name = "alphaTokens"; + + static bool isVariadic() { return true; } + + static size_t getNumberOfArguments() { return 0; } + + static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) + { + checkArgumentsWithOptionalMaxSubstrings(func, arguments); + } + + static constexpr auto strings_argument_position = 0uz; + + void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_) + { + max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_; + max_splits = extractMaxSplits(arguments, 1); + } + + /// Called for each next string. + void set(Pos pos_, Pos end_) + { + pos = pos_; + end = end_; + splits = 0; + } + + /// Get the next token, if any, or return false. + bool get(Pos & token_begin, Pos & token_end) + { + /// Skip garbage + while (pos < end && !isAlphaASCII(*pos)) + ++pos; + + if (pos == end) + return false; + + token_begin = pos; + + if (max_splits) + { + if (max_substrings_includes_remaining_string) + { + if (splits == *max_splits - 1) + { + token_end = end; + pos = end; + return true; + } + } + else + if (splits == *max_splits) + return false; + } + + while (pos < end && isAlphaASCII(*pos)) + ++pos; + + token_end = pos; + ++splits; + + return true; + } +}; + +using FunctionSplitByAlpha = FunctionTokens; + +} + +REGISTER_FUNCTION(SplitByAlpha) +{ + factory.registerFunction(); + factory.registerAlias("splitByAlpha", FunctionSplitByAlpha::name); +} + +} diff --git a/src/Functions/arrayStringConcat.cpp b/src/Functions/arrayStringConcat.cpp new file mode 100644 index 00000000000..f5eb3fd49cc --- /dev/null +++ b/src/Functions/arrayStringConcat.cpp @@ -0,0 +1,221 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +/** arrayStringConcat(arr) + * arrayStringConcat(arr, delimiter) + * - join an array of strings into one string via a separator. + */ +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +namespace +{ + +/// Joins an array of type serializable to string into one string via a separator. +class FunctionArrayStringConcat : public IFunction +{ +private: + static void executeInternal( + const ColumnString::Chars & src_chars, + const ColumnString::Offsets & src_string_offsets, + const ColumnArray::Offsets & src_array_offsets, + const char * delimiter, + const size_t delimiter_size, + ColumnString::Chars & dst_chars, + ColumnString::Offsets & dst_string_offsets, + const char8_t * null_map) + { + size_t size = src_array_offsets.size(); + + if (!size) + return; + + /// With a small margin - as if the separator goes after the last string of the array. + dst_chars.resize( + src_chars.size() + + delimiter_size * src_string_offsets.size() /// Separators after each string... + + src_array_offsets.size() /// Zero byte after each joined string + - src_string_offsets.size()); /// The former zero byte after each string of the array + + /// There will be as many strings as there were arrays. + dst_string_offsets.resize(src_array_offsets.size()); + + ColumnArray::Offset current_src_array_offset = 0; + + ColumnString::Offset current_dst_string_offset = 0; + + /// Loop through the array of strings. + for (size_t i = 0; i < size; ++i) + { + bool first_non_null = true; + /// Loop through the rows within the array. /// NOTE You can do everything in one copy, if the separator has a size of 1. + for (auto next_src_array_offset = src_array_offsets[i]; current_src_array_offset < next_src_array_offset; ++current_src_array_offset) + { + if (null_map && null_map[current_src_array_offset]) [[unlikely]] + continue; + + if (!first_non_null) + { + memcpy(&dst_chars[current_dst_string_offset], delimiter, delimiter_size); + current_dst_string_offset += delimiter_size; + } + first_non_null = false; + + const auto current_src_string_offset = current_src_array_offset ? src_string_offsets[current_src_array_offset - 1] : 0; + size_t bytes_to_copy = src_string_offsets[current_src_array_offset] - current_src_string_offset - 1; + + memcpySmallAllowReadWriteOverflow15( + &dst_chars[current_dst_string_offset], &src_chars[current_src_string_offset], bytes_to_copy); + + current_dst_string_offset += bytes_to_copy; + } + + dst_chars[current_dst_string_offset] = 0; + ++current_dst_string_offset; + + dst_string_offsets[i] = current_dst_string_offset; + } + + dst_chars.resize(dst_string_offsets.back()); + } + + static void executeInternal( + const ColumnString & col_string, + const ColumnArray & col_arr, + const String & delimiter, + ColumnString & col_res, + const char8_t * null_map = nullptr) + { + executeInternal( + col_string.getChars(), + col_string.getOffsets(), + col_arr.getOffsets(), + delimiter.data(), + delimiter.size(), + col_res.getChars(), + col_res.getOffsets(), + null_map); + } + + static ColumnPtr serializeNestedColumn(const ColumnArray & col_arr, const DataTypePtr & nested_type) + { + if (isString(nested_type)) + { + return col_arr.getDataPtr(); + } + else if (const ColumnNullable * col_nullable = checkAndGetColumn(col_arr.getData()); + col_nullable && isString(col_nullable->getNestedColumn().getDataType())) + { + return col_nullable->getNestedColumnPtr(); + } + else + { + return castColumn({col_arr.getDataPtr(), nested_type, "tmp"}, std::make_shared()); + } + } + +public: + static constexpr auto name = "arrayStringConcat"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + bool isVariadic() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors mandatory_args + { + {"arr", &isArray, nullptr, "Array"}, + }; + + FunctionArgumentDescriptors optional_args + { + {"separator", &isString, isColumnConst, "const String"}, + }; + + validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override + { + String delimiter; + if (arguments.size() == 2) + { + const ColumnConst * col_delim = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get()); + if (!col_delim) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument for function {} must be constant string.", getName()); + + delimiter = col_delim->getValue(); + } + + const auto & nested_type = assert_cast(*arguments[0].type).getNestedType(); + if (const ColumnConst * col_const_arr = checkAndGetColumnConst(arguments[0].column.get()); + col_const_arr && isString(nested_type)) + { + Array src_arr = col_const_arr->getValue(); + String dst_str; + bool first_non_null = true; + for (size_t i = 0, size = src_arr.size(); i < size; ++i) + { + if (src_arr[i].isNull()) + continue; + if (!first_non_null) + dst_str += delimiter; + first_non_null = false; + dst_str += src_arr[i].get(); + } + + return result_type->createColumnConst(col_const_arr->size(), dst_str); + } + + ColumnPtr src_column = arguments[0].column->convertToFullColumnIfConst(); + const ColumnArray & col_arr = assert_cast(*src_column.get()); + + ColumnPtr str_subcolumn = serializeNestedColumn(col_arr, nested_type); + const ColumnString & col_string = assert_cast(*str_subcolumn.get()); + + auto col_res = ColumnString::create(); + if (const ColumnNullable * col_nullable = checkAndGetColumn(col_arr.getData())) + executeInternal(col_string, col_arr, delimiter, *col_res, col_nullable->getNullMapData().data()); + else + executeInternal(col_string, col_arr, delimiter, *col_res); + return col_res; + } +}; + +} + +REGISTER_FUNCTION(ArrayStringConcat) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/extractAll.cpp b/src/Functions/extractAll.cpp new file mode 100644 index 00000000000..d73dbd7d4a4 --- /dev/null +++ b/src/Functions/extractAll.cpp @@ -0,0 +1,123 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; +} + + +/** Functions that split strings into an array of strings or vice versa. + * + * extractAll(s, regexp) - select from the string the subsequences corresponding to the regexp. + * - first subpattern, if regexp has subpattern; + * - zero subpattern (the match part, otherwise); + * - otherwise, an empty array + */ +namespace +{ + +using Pos = const char *; + +class ExtractAllImpl +{ +private: + Regexps::RegexpPtr re; + OptimizedRegularExpression::MatchVec matches; + size_t capture; + + Pos pos; + Pos end; +public: + static constexpr auto name = "extractAll"; + static String getName() { return name; } + static bool isVariadic() { return false; } + static size_t getNumberOfArguments() { return 2; } + + static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) + { + FunctionArgumentDescriptors mandatory_args{ + {"haystack", &isString, nullptr, "String"}, + {"pattern", &isString, isColumnConst, "const String"} + }; + + validateFunctionArgumentTypes(func, arguments, mandatory_args); + } + + static constexpr auto strings_argument_position = 0uz; + + void init(const ColumnsWithTypeAndName & arguments, bool /*max_substrings_includes_remaining_string*/) + { + const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get()); + + if (!col) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. " + "Must be constant string.", arguments[1].column->getName(), getName()); + + re = std::make_shared(Regexps::createRegexp(col->getValue())); + capture = re->getNumberOfSubpatterns() > 0 ? 1 : 0; + + matches.resize(capture + 1); + } + + /// Called for each next string. + void set(Pos pos_, Pos end_) + { + pos = pos_; + end = end_; + } + + /// Get the next token, if any, or return false. + bool get(Pos & token_begin, Pos & token_end) + { + if (!pos || pos > end) + return false; + + if (!re->match(pos, end - pos, matches) || !matches[0].length) + return false; + + if (matches[capture].offset == std::string::npos) + { + /// Empty match. + token_begin = pos; + token_end = pos; + } + else + { + token_begin = pos + matches[capture].offset; + token_end = token_begin + matches[capture].length; + } + + pos += matches[0].offset + matches[0].length; + + return true; + } +}; + +using FunctionExtractAll = FunctionTokens; + +} + +REGISTER_FUNCTION(ExtractAll) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/splitByChar.cpp b/src/Functions/splitByChar.cpp new file mode 100644 index 00000000000..d537039dc23 --- /dev/null +++ b/src/Functions/splitByChar.cpp @@ -0,0 +1,122 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; +} + + +/** Functions that split strings into an array of strings or vice versa. + * + * splitByChar(sep, s[, max_substrings]) + */ +namespace +{ + +using Pos = const char *; + +class SplitByCharImpl +{ +private: + Pos pos; + Pos end; + char separator; + std::optional max_splits; + size_t splits; + bool max_substrings_includes_remaining_string; + +public: + static constexpr auto name = "splitByChar"; + static bool isVariadic() { return true; } + static size_t getNumberOfArguments() { return 0; } + + static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) + { + checkArgumentsWithSeparatorAndOptionalMaxSubstrings(func, arguments); + } + + static constexpr auto strings_argument_position = 1uz; + + void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_) + { + const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get()); + + if (!col) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. " + "Must be constant string.", arguments[0].column->getName(), name); + + String sep_str = col->getValue(); + + if (sep_str.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal separator for function {}. Must be exactly one byte.", name); + + separator = sep_str[0]; + + max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_; + max_splits = extractMaxSplits(arguments, 2); + } + + void set(Pos pos_, Pos end_) + { + pos = pos_; + end = end_; + splits = 0; + } + + bool get(Pos & token_begin, Pos & token_end) + { + if (!pos) + return false; + + token_begin = pos; + + if (max_splits) + { + if (max_substrings_includes_remaining_string) + { + if (splits == *max_splits - 1) + { + token_end = end; + pos = nullptr; + return true; + } + } + else + if (splits == *max_splits) + return false; + } + + pos = reinterpret_cast(memchr(pos, separator, end - pos)); + if (pos) + { + token_end = pos; + ++pos; + ++splits; + } + else + token_end = end; + + return true; + } +}; + +using FunctionSplitByChar = FunctionTokens; + +} + +REGISTER_FUNCTION(SplitByChar) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/splitByNonAlpha.cpp b/src/Functions/splitByNonAlpha.cpp new file mode 100644 index 00000000000..67d3d100420 --- /dev/null +++ b/src/Functions/splitByNonAlpha.cpp @@ -0,0 +1,120 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; +} + + +/** Functions that split strings into an array of strings or vice versa. + * + * splitByNonAlpha(s[, max_substrings]) - split the string by whitespace and punctuation characters + */ +namespace +{ + +using Pos = const char *; + +class SplitByNonAlphaImpl +{ +private: + Pos pos; + Pos end; + std::optional max_splits; + size_t splits; + bool max_substrings_includes_remaining_string; + +public: + /// Get the name of the function. + static constexpr auto name = "splitByNonAlpha"; + + static bool isVariadic() { return true; } + static size_t getNumberOfArguments() { return 0; } + + static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) + { + checkArgumentsWithOptionalMaxSubstrings(func, arguments); + } + + static constexpr auto strings_argument_position = 0uz; + + void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_) + { + max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_; + max_splits = extractMaxSplits(arguments, 1); + } + + /// Called for each next string. + void set(Pos pos_, Pos end_) + { + pos = pos_; + end = end_; + splits = 0; + } + + /// Get the next token, if any, or return false. + bool get(Pos & token_begin, Pos & token_end) + { + /// Skip garbage + while (pos < end && (isWhitespaceASCII(*pos) || isPunctuationASCII(*pos))) + ++pos; + + if (pos == end) + return false; + + token_begin = pos; + + if (max_splits) + { + if (max_substrings_includes_remaining_string) + { + if (splits == *max_splits - 1) + { + token_end = end; + pos = end; + return true; + } + } + else + if (splits == *max_splits) + return false; + } + + while (pos < end && !(isWhitespaceASCII(*pos) || isPunctuationASCII(*pos))) + ++pos; + + token_end = pos; + splits++; + + return true; + } +}; + +using FunctionSplitByNonAlpha = FunctionTokens; + +} + +REGISTER_FUNCTION(SplitByNonAlpha) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/splitByRegexp.cpp b/src/Functions/splitByRegexp.cpp new file mode 100644 index 00000000000..cfffbfbb856 --- /dev/null +++ b/src/Functions/splitByRegexp.cpp @@ -0,0 +1,157 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; +} + + +/** Functions that split strings into an array of strings or vice versa. + * + * splitByRegexp(regexp, s[, max_substrings]) + */ +namespace +{ + +using Pos = const char *; + +class SplitByRegexpImpl +{ +private: + Regexps::RegexpPtr re; + OptimizedRegularExpression::MatchVec matches; + + Pos pos; + Pos end; + + std::optional max_splits; + size_t splits; + bool max_substrings_includes_remaining_string; + +public: + static constexpr auto name = "splitByRegexp"; + + static bool isVariadic() { return true; } + static size_t getNumberOfArguments() { return 0; } + + static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) + { + checkArgumentsWithSeparatorAndOptionalMaxSubstrings(func, arguments); + } + + static constexpr auto strings_argument_position = 1uz; + + void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_) + { + const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get()); + + if (!col) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. " + "Must be constant string.", arguments[0].column->getName(), name); + + if (!col->getValue().empty()) + re = std::make_shared(Regexps::createRegexp(col->getValue())); + + max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_; + max_splits = extractMaxSplits(arguments, 2); + } + + /// Called for each next string. + void set(Pos pos_, Pos end_) + { + pos = pos_; + end = end_; + splits = 0; + } + + /// Get the next token, if any, or return false. + bool get(Pos & token_begin, Pos & token_end) + { + if (!re) + { + if (pos == end) + return false; + + token_begin = pos; + + if (max_splits) + { + if (max_substrings_includes_remaining_string) + { + if (splits == *max_splits - 1) + { + token_end = end; + pos = end; + return true; + } + } + else + if (splits == *max_splits) + return false; + } + + pos += 1; + token_end = pos; + ++splits; + } + else + { + if (!pos || pos > end) + return false; + + token_begin = pos; + + if (max_splits) + { + if (max_substrings_includes_remaining_string) + { + if (splits == *max_splits - 1) + { + token_end = end; + pos = nullptr; + return true; + } + } + else + if (splits == *max_splits) + return false; + } + + if (!re->match(pos, end - pos, matches) || !matches[0].length) + { + token_end = end; + pos = end + 1; + } + else + { + token_end = pos + matches[0].offset; + pos = token_end + matches[0].length; + ++splits; + } + } + + return true; + } +}; + +using FunctionSplitByRegexp = FunctionTokens; + +} + +REGISTER_FUNCTION(SplitByRegexp) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/splitByString.cpp b/src/Functions/splitByString.cpp new file mode 100644 index 00000000000..6c9ba3fd090 --- /dev/null +++ b/src/Functions/splitByString.cpp @@ -0,0 +1,149 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; +} + + +/** Functions that split strings into an array of strings or vice versa. + * + * splitByString(sep, s[, max_substrings]) + */ +namespace +{ + +using Pos = const char *; + +class SplitByStringImpl +{ +private: + Pos pos; + Pos end; + String separator; + std::optional max_splits; + size_t splits; + bool max_substrings_includes_remaining_string; + +public: + static constexpr auto name = "splitByString"; + static bool isVariadic() { return true; } + static size_t getNumberOfArguments() { return 0; } + + static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) + { + checkArgumentsWithSeparatorAndOptionalMaxSubstrings(func, arguments); + } + + static constexpr auto strings_argument_position = 1uz; + + void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_) + { + const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get()); + + if (!col) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. " + "Must be constant string.", arguments[0].column->getName(), name); + + separator = col->getValue(); + + max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_; + max_splits = extractMaxSplits(arguments, 2); + } + + /// Called for each next string. + void set(Pos pos_, Pos end_) + { + pos = pos_; + end = end_; + splits = 0; + } + + /// Get the next token, if any, or return false. + bool get(Pos & token_begin, Pos & token_end) + { + if (separator.empty()) + { + if (pos == end) + return false; + + token_begin = pos; + + if (max_splits) + { + if (max_substrings_includes_remaining_string) + { + if (splits == *max_splits - 1) + { + token_end = end; + pos = end; + return true; + } + } + else + if (splits == *max_splits) + return false; + } + + pos += 1; + token_end = pos; + ++splits; + } + else + { + if (!pos) + return false; + + token_begin = pos; + + if (max_splits) + { + if (max_substrings_includes_remaining_string) + { + if (splits == *max_splits - 1) + { + token_end = end; + pos = nullptr; + return true; + } + } + else + if (splits == *max_splits) + return false; + } + + pos = reinterpret_cast(memmem(pos, end - pos, separator.data(), separator.size())); + if (pos) + { + token_end = pos; + pos += separator.size(); + ++splits; + } + else + token_end = end; + } + + return true; + } +}; + +using FunctionSplitByString = FunctionTokens; + +} + +REGISTER_FUNCTION(SplitByString) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/splitByWhitespace.cpp b/src/Functions/splitByWhitespace.cpp new file mode 100644 index 00000000000..5feafb3c11d --- /dev/null +++ b/src/Functions/splitByWhitespace.cpp @@ -0,0 +1,108 @@ +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; +} + + +/** Functions that split strings into an array of strings or vice versa. + * + * splitByWhitespace(s[, max_substrings]) - split the string by whitespace characters + */ +namespace +{ + +using Pos = const char *; + +class SplitByWhitespaceImpl +{ +private: + Pos pos; + Pos end; + std::optional max_splits; + size_t splits; + bool max_substrings_includes_remaining_string; + +public: + static constexpr auto name = "splitByWhitespace"; + + static bool isVariadic() { return true; } + static size_t getNumberOfArguments() { return 0; } + + static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) + { + checkArgumentsWithOptionalMaxSubstrings(func, arguments); + } + + static constexpr auto strings_argument_position = 0uz; + + void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_) + { + max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_; + max_splits = extractMaxSplits(arguments, 1); + } + + /// Called for each next string. + void set(Pos pos_, Pos end_) + { + pos = pos_; + end = end_; + splits = 0; + } + + /// Get the next token, if any, or return false. + bool get(Pos & token_begin, Pos & token_end) + { + /// Skip garbage + while (pos < end && isWhitespaceASCII(*pos)) + ++pos; + + if (pos == end) + return false; + + token_begin = pos; + + if (max_splits) + { + if (max_substrings_includes_remaining_string) + { + if (splits == *max_splits - 1) + { + token_end = end; + pos = end; + return true; + } + } + else + if (splits == *max_splits) + return false; + } + + while (pos < end && !isWhitespaceASCII(*pos)) + ++pos; + + token_end = pos; + splits++; + + return true; + } +}; + +using FunctionSplitByWhitespace = FunctionTokens; + +} + +REGISTER_FUNCTION(SplitByWhitespace) +{ + factory.registerFunction(); +} + +} From 2efe26eee9e8df0a976140ea89d8e1994e6dd97e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 03:38:46 +0100 Subject: [PATCH 08/28] Fix error and remove garbage --- src/Functions/appendTrailingCharIfAbsent.cpp | 5 +-- src/Functions/arrayStringConcat.cpp | 47 ++++++-------------- 2 files changed, 16 insertions(+), 36 deletions(-) diff --git a/src/Functions/appendTrailingCharIfAbsent.cpp b/src/Functions/appendTrailingCharIfAbsent.cpp index 62c0bbd4598..7ff35e599be 100644 --- a/src/Functions/appendTrailingCharIfAbsent.cpp +++ b/src/Functions/appendTrailingCharIfAbsent.cpp @@ -4,7 +4,6 @@ #include #include #include -#include namespace DB @@ -46,10 +45,10 @@ private: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (!isString(arguments[0])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", arguments[0]->getName(), getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of the first argument of function {}", arguments[0]->getName(), getName()); if (!isString(arguments[1])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", arguments[1]->getName(), getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of the second argument of function {}", arguments[1]->getName(), getName()); return std::make_shared(); } diff --git a/src/Functions/arrayStringConcat.cpp b/src/Functions/arrayStringConcat.cpp index f5eb3fd49cc..0194cc4871a 100644 --- a/src/Functions/arrayStringConcat.cpp +++ b/src/Functions/arrayStringConcat.cpp @@ -119,19 +119,16 @@ private: static ColumnPtr serializeNestedColumn(const ColumnArray & col_arr, const DataTypePtr & nested_type) { - if (isString(nested_type)) + DataTypePtr type = nested_type; + ColumnPtr column = col_arr.getDataPtr(); + + if (type->isNullable()) { - return col_arr.getDataPtr(); - } - else if (const ColumnNullable * col_nullable = checkAndGetColumn(col_arr.getData()); - col_nullable && isString(col_nullable->getNestedColumn().getDataType())) - { - return col_nullable->getNestedColumnPtr(); - } - else - { - return castColumn({col_arr.getDataPtr(), nested_type, "tmp"}, std::make_shared()); + type = removeNullable(type); + column = assert_cast(*column).getNestedColumnPtr(); } + + return castColumn({column, type, "tmp"}, std::make_shared()); } public: @@ -147,6 +144,9 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_args @@ -164,7 +164,7 @@ public: return std::make_shared(); } - 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 { String delimiter; if (arguments.size() == 2) @@ -177,27 +177,7 @@ public: } const auto & nested_type = assert_cast(*arguments[0].type).getNestedType(); - if (const ColumnConst * col_const_arr = checkAndGetColumnConst(arguments[0].column.get()); - col_const_arr && isString(nested_type)) - { - Array src_arr = col_const_arr->getValue(); - String dst_str; - bool first_non_null = true; - for (size_t i = 0, size = src_arr.size(); i < size; ++i) - { - if (src_arr[i].isNull()) - continue; - if (!first_non_null) - dst_str += delimiter; - first_non_null = false; - dst_str += src_arr[i].get(); - } - - return result_type->createColumnConst(col_const_arr->size(), dst_str); - } - - ColumnPtr src_column = arguments[0].column->convertToFullColumnIfConst(); - const ColumnArray & col_arr = assert_cast(*src_column.get()); + const ColumnArray & col_arr = assert_cast(*arguments[0].column); ColumnPtr str_subcolumn = serializeNestedColumn(col_arr, nested_type); const ColumnString & col_string = assert_cast(*str_subcolumn.get()); @@ -207,6 +187,7 @@ public: executeInternal(col_string, col_arr, delimiter, *col_res, col_nullable->getNullMapData().data()); else executeInternal(col_string, col_arr, delimiter, *col_res); + return col_res; } }; From 28bf9779603fa906ce9b028832d9e4400d0c6a3c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 03:59:16 +0100 Subject: [PATCH 09/28] Fix test --- src/Functions/FunctionTokens.h | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionTokens.h b/src/Functions/FunctionTokens.h index a79d37a7a87..0a669e0279c 100644 --- a/src/Functions/FunctionTokens.h +++ b/src/Functions/FunctionTokens.h @@ -172,7 +172,11 @@ static inline std::optional extractMaxSplits( return std::nullopt; if (const ColumnConst * column = checkAndGetColumn(arguments[max_substrings_argument_position].column.get())) - return column->getUInt(0); + { + size_t res = column->getUInt(0); + if (res) + return res; + } return std::nullopt; } From 66d297ee478a2d670371bbe82c5897c95810307e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 04:25:49 +0100 Subject: [PATCH 10/28] Fix style --- src/Functions/FunctionTokens.h | 1 - src/Functions/alphaTokens.cpp | 7 ------- src/Functions/extractAll.cpp | 1 - src/Functions/splitByNonAlpha.cpp | 7 ------- src/Functions/splitByRegexp.cpp | 1 - src/Functions/splitByString.cpp | 1 - src/Functions/splitByWhitespace.cpp | 7 ------- 7 files changed, 25 deletions(-) diff --git a/src/Functions/FunctionTokens.h b/src/Functions/FunctionTokens.h index 0a669e0279c..5c4e582c637 100644 --- a/src/Functions/FunctionTokens.h +++ b/src/Functions/FunctionTokens.h @@ -24,7 +24,6 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; } diff --git a/src/Functions/alphaTokens.cpp b/src/Functions/alphaTokens.cpp index 9b20509eee0..35cacdbdbb8 100644 --- a/src/Functions/alphaTokens.cpp +++ b/src/Functions/alphaTokens.cpp @@ -7,13 +7,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_COLUMN; -} - - /** Functions that split strings into an array of strings or vice versa. * * alphaTokens(s[, max_substrings]) - select from the string subsequence `[a-zA-Z]+`. diff --git a/src/Functions/extractAll.cpp b/src/Functions/extractAll.cpp index d73dbd7d4a4..ad49f32f769 100644 --- a/src/Functions/extractAll.cpp +++ b/src/Functions/extractAll.cpp @@ -19,7 +19,6 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; } diff --git a/src/Functions/splitByNonAlpha.cpp b/src/Functions/splitByNonAlpha.cpp index 67d3d100420..467e7b0b5c3 100644 --- a/src/Functions/splitByNonAlpha.cpp +++ b/src/Functions/splitByNonAlpha.cpp @@ -17,13 +17,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_COLUMN; -} - - /** Functions that split strings into an array of strings or vice versa. * * splitByNonAlpha(s[, max_substrings]) - split the string by whitespace and punctuation characters diff --git a/src/Functions/splitByRegexp.cpp b/src/Functions/splitByRegexp.cpp index cfffbfbb856..77328205c01 100644 --- a/src/Functions/splitByRegexp.cpp +++ b/src/Functions/splitByRegexp.cpp @@ -12,7 +12,6 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; } diff --git a/src/Functions/splitByString.cpp b/src/Functions/splitByString.cpp index 6c9ba3fd090..7d6803b2f27 100644 --- a/src/Functions/splitByString.cpp +++ b/src/Functions/splitByString.cpp @@ -11,7 +11,6 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; } diff --git a/src/Functions/splitByWhitespace.cpp b/src/Functions/splitByWhitespace.cpp index 5feafb3c11d..168e429c6f5 100644 --- a/src/Functions/splitByWhitespace.cpp +++ b/src/Functions/splitByWhitespace.cpp @@ -6,13 +6,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_COLUMN; -} - - /** Functions that split strings into an array of strings or vice versa. * * splitByWhitespace(s[, max_substrings]) - split the string by whitespace characters From fb0335efb0d94054e7c856e658037f118e035d8d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 06:20:34 +0100 Subject: [PATCH 11/28] What if I add some indices --- docker/test/base/setup_export_logs.sh | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 0ff79e24bf8..801d1439c60 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -15,7 +15,17 @@ CLICKHOUSE_CI_LOGS_USER=${CLICKHOUSE_CI_LOGS_USER:-ci} # Pre-configured destination cluster, where to export the data CLICKHOUSE_CI_LOGS_CLUSTER=${CLICKHOUSE_CI_LOGS_CLUSTER:-system_logs_export} -EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime('UTC'), check_name String, instance_type String, instance_id String, "} +EXTRA_COLUMNS=${EXTRA_COLUMNS:-" + pull_request_number UInt32, + commit_sha String, + check_start_time DateTime('UTC'), + check_name LowCardinality(String), + instance_type LowCardinality(String), + instance_id String, + INDEX ix_pr (pull_request_number) TYPE set(100), + INDEX ix_commit (commit_sha) TYPE set(100), + INDEX ix_check_time (check_start_time) TYPE minmax, +"} EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type, '' AS instance_id"} EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "} From 629c5c0e458af51c7c610f7da172ca3abfe09c95 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 06:21:31 +0100 Subject: [PATCH 12/28] Change tables version --- docker/test/base/setup_export_logs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 801d1439c60..01a2a561168 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -137,7 +137,7 @@ function setup_logs_replication do # Calculate hash of its structure. Note: 4 is the version of extra columns - increment it if extra columns are changed: hash=$(clickhouse-client --query " - SELECT sipHash64(4, groupArray((name, type))) + SELECT sipHash64(6, groupArray((name, type))) FROM (SELECT name, type FROM system.columns WHERE database = 'system' AND table = '$table' ORDER BY position) From c52a91caf6e1134860932324df7d8b35c4a74a49 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 06:59:20 +0100 Subject: [PATCH 13/28] Remove dependencies --- src/Interpreters/CrossToInnerJoinVisitor.cpp | 11 ++--- .../GatherFunctionQuantileVisitor.cpp | 42 +++++++++---------- src/Interpreters/addMissingDefaults.cpp | 3 +- 3 files changed, 25 insertions(+), 31 deletions(-) diff --git a/src/Interpreters/CrossToInnerJoinVisitor.cpp b/src/Interpreters/CrossToInnerJoinVisitor.cpp index 8755daa1dc8..005450c2a2c 100644 --- a/src/Interpreters/CrossToInnerJoinVisitor.cpp +++ b/src/Interpreters/CrossToInnerJoinVisitor.cpp @@ -1,13 +1,8 @@ #include #include -#include -#include -#include #include #include #include -#include -#include #include #include #include @@ -15,11 +10,11 @@ #include #include #include -#include #include #include + namespace DB { @@ -117,7 +112,7 @@ std::map> moveExpressionToJoinOn( std::map> asts_to_join_on; for (const auto & node : splitConjunctionsAst(ast)) { - if (const auto * func = node->as(); func && func->name == NameEquals::name) + if (const auto * func = node->as(); func && func->name == "equals") { if (!func->arguments || func->arguments->children.size() != 2) return {}; @@ -154,7 +149,7 @@ ASTPtr makeOnExpression(const std::vector & expressions) for (const auto & ast : expressions) arguments.emplace_back(ast->clone()); - return makeASTFunction(NameAnd::name, std::move(arguments)); + return makeASTFunction("and", std::move(arguments)); } std::vector getTables(const ASTSelectQuery & select) diff --git a/src/Interpreters/GatherFunctionQuantileVisitor.cpp b/src/Interpreters/GatherFunctionQuantileVisitor.cpp index 805fcfec181..664bb9e9383 100644 --- a/src/Interpreters/GatherFunctionQuantileVisitor.cpp +++ b/src/Interpreters/GatherFunctionQuantileVisitor.cpp @@ -1,10 +1,9 @@ #include -#include #include -#include #include + namespace DB { @@ -14,22 +13,23 @@ namespace ErrorCodes } /// Mapping from quantile functions for single value to plural -static const std::unordered_map quantile_fuse_name_mapping = { - {NameQuantile::name, NameQuantiles::name}, - {NameQuantileBFloat16::name, NameQuantilesBFloat16::name}, - {NameQuantileBFloat16Weighted::name, NameQuantilesBFloat16Weighted::name}, - {NameQuantileDeterministic::name, NameQuantilesDeterministic::name}, - {NameQuantileExact::name, NameQuantilesExact::name}, - {NameQuantileExactExclusive::name, NameQuantilesExactExclusive::name}, - {NameQuantileExactHigh::name, NameQuantilesExactHigh::name}, - {NameQuantileExactInclusive::name, NameQuantilesExactInclusive::name}, - {NameQuantileExactLow::name, NameQuantilesExactLow::name}, - {NameQuantileExactWeighted::name, NameQuantilesExactWeighted::name}, - {NameQuantileInterpolatedWeighted::name, NameQuantilesInterpolatedWeighted::name}, - {NameQuantileTDigest::name, NameQuantilesTDigest::name}, - {NameQuantileTDigestWeighted::name, NameQuantilesTDigestWeighted::name}, - {NameQuantileTiming::name, NameQuantilesTiming::name}, - {NameQuantileTimingWeighted::name, NameQuantilesTimingWeighted::name}, +static const std::unordered_map quantile_fuse_name_mapping = +{ + {"quantile", "quantiles"}, + {"quantileBFloat16", "quantilesBFloat16"}, + {"quantileBFloat16Weighted", "quantilesBFloat16Weighted"}, + {"quantileDeterministic", "quantilesDeterministic"}, + {"quantileExact", "quantilesExact"}, + {"quantileExactExclusive", "quantilesExactExclusive"}, + {"quantileExactHigh", "quantilesExactHigh"}, + {"quantileExactInclusive", "quantilesExactInclusive"}, + {"quantileExactLow", "quantilesExactLow"}, + {"quantileExactWeighted", "quantilesExactWeighted"}, + {"quantileInterpolatedWeighted", "quantilesInterpolatedWeighted"}, + {"quantileTDigest", "quantilesTDigest"}, + {"quantileTDigestWeighted", "quantilesTDigestWeighted"}, + {"quantileTiming", "quantilesTiming"}, + {"quantileTimingWeighted", "quantilesTimingWeighted"}, }; String GatherFunctionQuantileData::toFusedNameOrSelf(const String & func_name) @@ -63,9 +63,9 @@ void GatherFunctionQuantileData::FuseQuantileAggregatesData::addFuncNode(ASTPtr const auto & arguments = func->arguments->children; - bool need_two_args = func->name == NameQuantileDeterministic::name || func->name == NameQuantileExactWeighted::name - || func->name == NameQuantileInterpolatedWeighted::name || func->name == NameQuantileTimingWeighted::name - || func->name == NameQuantileTDigestWeighted::name || func->name == NameQuantileBFloat16Weighted::name; + bool need_two_args = func->name == "quantileDeterministic" || func->name == "quantileExactWeighted" + || func->name == "quantileInterpolatedWeighted" || func->name == "quantileTimingWeighted" + || func->name == "quantileTDigestWeighted" || func->name == "quantileBFloat16Weighted"; if (arguments.size() != (need_two_args ? 2 : 1)) return; diff --git a/src/Interpreters/addMissingDefaults.cpp b/src/Interpreters/addMissingDefaults.cpp index 3febcfc74a8..fbf17d7efb7 100644 --- a/src/Interpreters/addMissingDefaults.cpp +++ b/src/Interpreters/addMissingDefaults.cpp @@ -8,8 +8,7 @@ #include #include #include -#include -#include +#include namespace DB From df24ef42b1979bda684f3e272eef2d0b8ea11022 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 07:27:10 +0100 Subject: [PATCH 14/28] Publish stripped binary --- cmake/split_debug_symbols.cmake | 2 ++ programs/CMakeLists.txt | 7 +++++++ programs/self-extracting/CMakeLists.txt | 6 +++--- tests/ci/build_check.py | 15 +++++++++++---- 4 files changed, 23 insertions(+), 7 deletions(-) diff --git a/cmake/split_debug_symbols.cmake b/cmake/split_debug_symbols.cmake index d6821eb6c48..67c2c386f20 100644 --- a/cmake/split_debug_symbols.cmake +++ b/cmake/split_debug_symbols.cmake @@ -1,3 +1,5 @@ +# Generates a separate file with debug symbols while stripping it from the main binary. +# This is needed for Debian packages. macro(clickhouse_split_debug_symbols) set(oneValueArgs TARGET DESTINATION_DIR BINARY_PATH) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index eb4a898d472..f20e1058b82 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -439,6 +439,13 @@ else() install (TARGETS clickhouse RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) endif() +# A target to get stripped binary. +# Note: this is different to the above (extract debug symbols to a separate place) +add_custom_target(clickhouse-stripped + COMMAND "${STRIP_PATH}" -o "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-stripped" --strip-debug --remove-section=.comment --remove-section=.note "${CMAKE_CURRENT_BINARY_DIR}/clickhouse" + DEPENDS clickhouse + COMMENT "Stripping clickhouse binary" VERBATIM) + if (ENABLE_TESTS) set (CLICKHOUSE_UNIT_TESTS_TARGETS unit_tests_dbms) add_custom_target (clickhouse-tests ALL DEPENDS ${CLICKHOUSE_UNIT_TESTS_TARGETS}) diff --git a/programs/self-extracting/CMakeLists.txt b/programs/self-extracting/CMakeLists.txt index f3ff0bbcd78..4b6dd07f618 100644 --- a/programs/self-extracting/CMakeLists.txt +++ b/programs/self-extracting/CMakeLists.txt @@ -11,8 +11,8 @@ else () endif () add_custom_target (self-extracting ALL - ${CMAKE_COMMAND} -E remove clickhouse + ${CMAKE_COMMAND} -E remove clickhouse clickhouse-stripped COMMAND ${COMPRESSOR} ${DECOMPRESSOR} clickhouse ../clickhouse - DEPENDS clickhouse compressor + COMMAND ${COMPRESSOR} ${DECOMPRESSOR} clickhouse-stripped ../clickhouse-stripped + DEPENDS clickhouse clickhouse-stripped compressor ) - diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 584ece0a736..c18abcf1191 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -212,10 +212,17 @@ def upload_master_static_binaries( elif pr_info.base_ref != "master": return - s3_path = "/".join((pr_info.base_ref, static_binary_name, "clickhouse")) - binary = build_output_path / "clickhouse" - url = s3_helper.upload_build_file_to_s3(binary, s3_path) - print(f"::notice ::Binary static URL: {url}") + # Full binary with debug info: + s3_path_full = "/".join((pr_info.base_ref, static_binary_name, "clickhouse-full")) + binary_full = build_output_path / "clickhouse" + url_full = s3_helper.upload_build_file_to_s3(binary_full, s3_path_full) + print(f"::notice ::Binary static URL (with debug info): {url_full}") + + # Stripped binary without debug info: + s3_path_compact = "/".join((pr_info.base_ref, static_binary_name, "clickhouse")) + binary_compact = build_output_path / "clickhouse-stripped" + url_compact = s3_helper.upload_build_file_to_s3(binary_compact, s3_path_compact) + print(f"::notice ::Binary static URL (compact): {url_compact}") def main(): From 4b1fa685bb286a2b7c533c602868f643751f1088 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 07:28:26 +0100 Subject: [PATCH 15/28] Publish stripped binary --- programs/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index f20e1058b82..eb117e74f6b 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -441,7 +441,7 @@ endif() # A target to get stripped binary. # Note: this is different to the above (extract debug symbols to a separate place) -add_custom_target(clickhouse-stripped +add_custom_target(clickhouse-stripped ALL COMMAND "${STRIP_PATH}" -o "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-stripped" --strip-debug --remove-section=.comment --remove-section=.note "${CMAKE_CURRENT_BINARY_DIR}/clickhouse" DEPENDS clickhouse COMMENT "Stripping clickhouse binary" VERBATIM) From 4460abafbef08cc632562fa8b21bf3de08865977 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 07:37:39 +0100 Subject: [PATCH 16/28] No idea why it did not work --- docker/test/base/setup_export_logs.sh | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 01a2a561168..dc6ac5642f0 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -15,17 +15,7 @@ CLICKHOUSE_CI_LOGS_USER=${CLICKHOUSE_CI_LOGS_USER:-ci} # Pre-configured destination cluster, where to export the data CLICKHOUSE_CI_LOGS_CLUSTER=${CLICKHOUSE_CI_LOGS_CLUSTER:-system_logs_export} -EXTRA_COLUMNS=${EXTRA_COLUMNS:-" - pull_request_number UInt32, - commit_sha String, - check_start_time DateTime('UTC'), - check_name LowCardinality(String), - instance_type LowCardinality(String), - instance_id String, - INDEX ix_pr (pull_request_number) TYPE set(100), - INDEX ix_commit (commit_sha) TYPE set(100), - INDEX ix_check_time (check_start_time) TYPE minmax, -"} +EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime('UTC'), check_name LowCardinality(String), instance_type LowCardinality(String), instance_id String, INDEX ix_pr (pull_request_number) TYPE set(100), INDEX ix_commit (commit_sha) TYPE set(100), INDEX ix_check_time (check_start_time) TYPE minmax, "} EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type, '' AS instance_id"} EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "} From 9824631044c136abd996412be14378c04050cf9f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 07:46:35 +0100 Subject: [PATCH 17/28] Apply LowCardinality in more places --- docker/test/base/setup_export_logs.sh | 2 +- tests/ci/clickhouse_helper.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index dc6ac5642f0..119c9398848 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -16,7 +16,7 @@ CLICKHOUSE_CI_LOGS_USER=${CLICKHOUSE_CI_LOGS_USER:-ci} CLICKHOUSE_CI_LOGS_CLUSTER=${CLICKHOUSE_CI_LOGS_CLUSTER:-system_logs_export} EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime('UTC'), check_name LowCardinality(String), instance_type LowCardinality(String), instance_id String, INDEX ix_pr (pull_request_number) TYPE set(100), INDEX ix_commit (commit_sha) TYPE set(100), INDEX ix_check_time (check_start_time) TYPE minmax, "} -EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type, '' AS instance_id"} +EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, toLowCardinality('') AS check_name, toLowCardinality('') AS instance_type, '' AS instance_id"} EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "} function __set_connection_args diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 3ae20bd871d..f338a1e14c3 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -297,8 +297,8 @@ class CiLogsCredentials: return "" extra_columns = ( f"CAST({pr_info.number} AS UInt32) AS pull_request_number, '{pr_info.sha}' AS commit_sha, " - f"toDateTime('{check_start_time}', 'UTC') AS check_start_time, '{check_name}' AS check_name, " - f"'{get_instance_type()}' AS instance_type, '{get_instance_id()}' AS instance_id" + f"toDateTime('{check_start_time}', 'UTC') AS check_start_time, toLowCardinality('{check_name}') AS check_name, " + f"toLowCardinality('{get_instance_type()}') AS instance_type, '{get_instance_id()}' AS instance_id" ) return ( f'-e EXTRA_COLUMNS_EXPRESSION="{extra_columns}" ' From 0064872bc7953a665fe0040982d62a13781bc5d6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 07:46:56 +0100 Subject: [PATCH 18/28] Apply LowCardinality in more places --- docker/test/base/setup_export_logs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 119c9398848..f09b8225da4 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -127,7 +127,7 @@ function setup_logs_replication do # Calculate hash of its structure. Note: 4 is the version of extra columns - increment it if extra columns are changed: hash=$(clickhouse-client --query " - SELECT sipHash64(6, groupArray((name, type))) + SELECT sipHash64(7, groupArray((name, type))) FROM (SELECT name, type FROM system.columns WHERE database = 'system' AND table = '$table' ORDER BY position) From f2a67ae8111b0d9b4b557c29d0efcd7d5c93d68f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 09:24:26 +0100 Subject: [PATCH 19/28] Addition --- docker/packager/binary/build.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index cc2613cbaf5..150ce1ab385 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -126,6 +126,7 @@ fi mv ./programs/clickhouse* /output || mv ./programs/*_fuzzer /output [ -x ./programs/self-extracting/clickhouse ] && mv ./programs/self-extracting/clickhouse /output +[ -x ./programs/self-extracting/clickhouse-stripped ] && mv ./programs/self-extracting/clickhouse-stripped /output mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds mv ./programs/*.dict ./programs/*.options ./programs/*_seed_corpus.zip /output ||: # libFuzzer oss-fuzz compatible infrastructure From f1ccee34ccd87f55abf3a6a84396f9a46e6fd7f8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 09:28:30 +0100 Subject: [PATCH 20/28] Fix tests and docs --- docs/en/interfaces/http.md | 10 +++++----- docs/ru/interfaces/http.md | 10 +++++----- docs/zh/interfaces/http.md | 10 +++++----- .../0_stateless/00408_http_keep_alive.reference | 6 +++--- tests/queries/0_stateless/00501_http_head.reference | 5 ++--- 5 files changed, 20 insertions(+), 21 deletions(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 0e2c0c00e4c..63f75fb7830 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -438,7 +438,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Query-Id: 96fe0052-01e6-43ce-b12a-6b7370de6e8a < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} < # HELP "Query" "Number of executing queries" @@ -603,7 +603,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Connection: Keep-Alive < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} < * Connection #0 to host localhost left intact @@ -643,7 +643,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Connection: Keep-Alive < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} < * Connection #0 to host localhost left intact @@ -695,7 +695,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Connection: Keep-Alive < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} < Absolute Path File @@ -714,7 +714,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Connection: Keep-Alive < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} < Relative Path File diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index 16927408bc4..be8cfbdda6c 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -366,7 +366,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Query-Id: 96fe0052-01e6-43ce-b12a-6b7370de6e8a < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} < # HELP "Query" "Number of executing queries" @@ -529,7 +529,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Connection: Keep-Alive < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} < * Connection #0 to host localhost left intact @@ -569,7 +569,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Connection: Keep-Alive < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} < * Connection #0 to host localhost left intact @@ -621,7 +621,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Connection: Keep-Alive < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} < Absolute Path File @@ -640,7 +640,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Connection: Keep-Alive < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} < Relative Path File diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index dfdcf53bd3f..84ca5ed0c47 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -362,7 +362,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Query-Id: 96fe0052-01e6-43ce-b12a-6b7370de6e8a < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} < # HELP "Query" "Number of executing queries" @@ -520,7 +520,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Connection: Keep-Alive < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} < * Connection #0 to host localhost left intact @@ -560,7 +560,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Connection: Keep-Alive < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} < * Connection #0 to host localhost left intact @@ -612,7 +612,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Connection: Keep-Alive < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} < Absolute Path File @@ -631,7 +631,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Connection: Keep-Alive < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} < Relative Path File diff --git a/tests/queries/0_stateless/00408_http_keep_alive.reference b/tests/queries/0_stateless/00408_http_keep_alive.reference index 5f9cc1079a7..17a7fd690a8 100644 --- a/tests/queries/0_stateless/00408_http_keep_alive.reference +++ b/tests/queries/0_stateless/00408_http_keep_alive.reference @@ -1,6 +1,6 @@ < Connection: Keep-Alive -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < Connection: Keep-Alive -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 < Connection: Keep-Alive -< Keep-Alive: timeout=3 +< Keep-Alive: timeout=10 diff --git a/tests/queries/0_stateless/00501_http_head.reference b/tests/queries/0_stateless/00501_http_head.reference index 9727c63b2d8..3d2d563acf4 100644 --- a/tests/queries/0_stateless/00501_http_head.reference +++ b/tests/queries/0_stateless/00501_http_head.reference @@ -2,11 +2,10 @@ HTTP/1.1 200 OK Connection: Keep-Alive Content-Type: text/tab-separated-values; charset=UTF-8 Transfer-Encoding: chunked -Keep-Alive: timeout=3 +Keep-Alive: timeout=10 HTTP/1.1 200 OK Connection: Keep-Alive Content-Type: text/tab-separated-values; charset=UTF-8 Transfer-Encoding: chunked -Keep-Alive: timeout=3 - +Keep-Alive: timeout=10 From 90f16d67ac63c7e2780027f43866a959ba8fe652 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 09:31:01 +0100 Subject: [PATCH 21/28] Fix test --- tests/queries/0_stateless/00501_http_head.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/00501_http_head.reference b/tests/queries/0_stateless/00501_http_head.reference index 3d2d563acf4..8351327b356 100644 --- a/tests/queries/0_stateless/00501_http_head.reference +++ b/tests/queries/0_stateless/00501_http_head.reference @@ -9,3 +9,4 @@ Connection: Keep-Alive Content-Type: text/tab-separated-values; charset=UTF-8 Transfer-Encoding: chunked Keep-Alive: timeout=10 + From 504cc33a2a84885873d7b915244fea62860fd9d6 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 9 Nov 2023 15:51:17 +0300 Subject: [PATCH 22/28] Analyzer special functions projection names fix --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 21 ++++++++++++++++--- tests/analyzer_tech_debt.txt | 1 - .../02338_analyzer_constants_basic.reference | 4 ++-- .../02378_analyzer_projection_names.reference | 17 ++++++++++++++- .../02378_analyzer_projection_names.sql | 3 +++ ..._injective_functions_elimination.reference | 2 +- 6 files changed, 40 insertions(+), 8 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index c6fbd728b8f..7855c4f34a8 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1467,9 +1467,15 @@ ProjectionName QueryAnalyzer::calculateFunctionProjectionName(const QueryTreeNod const ProjectionNames & arguments_projection_names) { const auto & function_node_typed = function_node->as(); + const auto & function_node_name = function_node_typed.getFunctionName(); + + bool is_array_function = function_node_name == "array"; + bool is_tuple_function = function_node_name == "tuple"; WriteBufferFromOwnString buffer; - buffer << function_node_typed.getFunctionName(); + + if (!is_array_function && !is_tuple_function) + buffer << function_node_name; if (!parameters_projection_names.empty()) { @@ -1487,7 +1493,16 @@ ProjectionName QueryAnalyzer::calculateFunctionProjectionName(const QueryTreeNod buffer << ')'; } - buffer << '('; + char open_bracket = '('; + char close_bracket = ')'; + + if (is_array_function) + { + open_bracket = '['; + close_bracket = ']'; + } + + buffer << open_bracket; size_t function_arguments_projection_names_size = arguments_projection_names.size(); for (size_t i = 0; i < function_arguments_projection_names_size; ++i) @@ -1498,7 +1513,7 @@ ProjectionName QueryAnalyzer::calculateFunctionProjectionName(const QueryTreeNod buffer << ", "; } - buffer << ')'; + buffer << close_bracket; return buffer.str(); } diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 018482c2301..bbb9abda079 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -35,7 +35,6 @@ 02352_grouby_shadows_arg 02354_annoy 02402_merge_engine_with_view -02426_orc_bug 02428_parameterized_view 02458_use_structure_from_insertion_table 02479_race_condition_between_insert_and_droppin_mv diff --git a/tests/queries/0_stateless/02338_analyzer_constants_basic.reference b/tests/queries/0_stateless/02338_analyzer_constants_basic.reference index f3a69e4d835..32f8a5eb124 100644 --- a/tests/queries/0_stateless/02338_analyzer_constants_basic.reference +++ b/tests/queries/0_stateless/02338_analyzer_constants_basic.reference @@ -25,11 +25,11 @@ NULL Nullable(Nothing) (1, 1) Tuple(UInt8, UInt8) (1,1) -- -array((1, 1)) Array(Tuple(UInt8, UInt8)) +[(1, 1)] Array(Tuple(UInt8, UInt8)) [(1,1)] NULL Nullable(Nothing) 1 UInt8 \'test\' String [1, 2, 3] Array(UInt8) -array((1, 1), (1, 1)) Array(Tuple(UInt8, UInt8)) +[(1, 1), (1, 1)] Array(Tuple(UInt8, UInt8)) \N 1 test [1,2,3] [(1,1),(1,1)] diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.reference b/tests/queries/0_stateless/02378_analyzer_projection_names.reference index a82d4d4c5d2..f8b18e6df15 100644 --- a/tests/queries/0_stateless/02378_analyzer_projection_names.reference +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.reference @@ -13,7 +13,7 @@ concat(\'Value_1\', \'Value_2\') String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)')); -CAST(tuple(1, \'Value\'), \'Tuple (id UInt64, value String)\') Tuple(id UInt64, value String) +CAST((1, \'Value\'), \'Tuple (id UInt64, value String)\') Tuple(id UInt64, value String) SELECT 'Columns'; Columns DESCRIBE (SELECT test_table.id, test_table.id, id FROM test_table); @@ -737,3 +737,18 @@ t2.id UInt64 t2.value String t3.id UInt64 t3.value String +SELECT 'Special functions array, tuple'; +Special functions array, tuple +DESCRIBE (SELECT [], array(), [1], array(1), [1, 2], array(1, 2), tuple(1), (1, 2), [[], []], [([], [])], ([], []), ([([], []), ([], [])])); +[] Array(Nothing) +[] Array(Nothing) +[1] Array(UInt8) +[1] Array(UInt8) +[1, 2] Array(UInt8) +[1, 2] Array(UInt8) +(1) Tuple(UInt8) +(1, 2) Tuple(UInt8, UInt8) +[[], []] Array(Array(Nothing)) +[([], [])] Array(Tuple(Array(Nothing), Array(Nothing))) +([], []) Tuple(Array(Nothing), Array(Nothing)) +[([], []), ([], [])] Array(Tuple(Array(Nothing), Array(Nothing))) diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.sql b/tests/queries/0_stateless/02378_analyzer_projection_names.sql index c69a1c1ad26..f5ac5f7476f 100644 --- a/tests/queries/0_stateless/02378_analyzer_projection_names.sql +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.sql @@ -533,6 +533,9 @@ SELECT '--'; DESCRIBE (SELECT id, value, t1.id, t1.value, t2.id, t2.value, t3.id, t3.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id, value) INNER JOIN test_table_join_3 AS t3 USING (id, value)); +SELECT 'Special functions array, tuple'; +DESCRIBE (SELECT [], array(), [1], array(1), [1, 2], array(1, 2), tuple(1), (1, 2), [[], []], [([], [])], ([], []), ([([], []), ([], [])])); + -- { echoOff } DROP TABLE test_table_join_1; diff --git a/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.reference b/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.reference index c42b9ce0cc4..5b808310f0e 100644 --- a/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.reference +++ b/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.reference @@ -1,6 +1,6 @@ QUERY id: 0 PROJECTION COLUMNS - uniqCombined(tuple(\'\')) UInt64 + uniqCombined((\'\')) UInt64 PROJECTION LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: uniqCombined, function_type: aggregate, result_type: UInt64 From 91d820991e845254c17148ffc1da890ad5fa78c7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 10 Nov 2023 13:36:52 +0300 Subject: [PATCH 23/28] Updated tests --- .../queries/0_stateless/01232_untuple.reference | 2 +- .../02483_cuturlparameter_with_arrays.reference | 8 ++++---- .../02890_untuple_column_names.reference | 16 ++++++++-------- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/01232_untuple.reference b/tests/queries/0_stateless/01232_untuple.reference index 7bb80bf618f..0358cde1354 100644 --- a/tests/queries/0_stateless/01232_untuple.reference +++ b/tests/queries/0_stateless/01232_untuple.reference @@ -2,7 +2,7 @@ hello 1 3 world 9 9 (0,1) -key tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'1\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'2\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'3\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'4\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'5\') +key tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'1\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'2\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'3\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'4\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'5\') 1 20 20 10 20 30 2 11 20 10 20 30 3 70 20 10 20 30 diff --git a/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.reference b/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.reference index 348408a15cc..89c4ab58b21 100644 --- a/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.reference +++ b/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.reference @@ -31,7 +31,7 @@ SELECT FORMAT Vertical; Row 1: ────── -cutURLParameter('http://bigmir.net/?a=b&c=d', array()): http://bigmir.net/?a=b&c=d +cutURLParameter('http://bigmir.net/?a=b&c=d', []): http://bigmir.net/?a=b&c=d cutURLParameter('http://bigmir.net/?a=b&c=d', ['a']): http://bigmir.net/?c=d cutURLParameter('http://bigmir.net/?a=b&c=d', ['a', 'c']): http://bigmir.net/? cutURLParameter('http://bigmir.net/?a=b&c=d', ['c']): http://bigmir.net/?a=b @@ -44,7 +44,7 @@ cutURLParameter('http://bigmir.net/?a=b&c=d#e&g=h', ['c', 'g']): http: cutURLParameter('http://bigmir.net/?a=b&c=d#e&g=h', ['e', 'g']): http://bigmir.net/?a=b&c=d#e cutURLParameter('http://bigmir.net/?a=b&c=d#test?e=f&g=h', ['test', 'e']): http://bigmir.net/?a=b&c=d#test?g=h cutURLParameter('http://bigmir.net/?a=b&c=d#test?e=f&g=h', ['test', 'g']): http://bigmir.net/?a=b&c=d#test?e=f -cutURLParameter('//bigmir.net/?a=b&c=d', array()): //bigmir.net/?a=b&c=d +cutURLParameter('//bigmir.net/?a=b&c=d', []): //bigmir.net/?a=b&c=d cutURLParameter('//bigmir.net/?a=b&c=d', ['a']): //bigmir.net/?c=d cutURLParameter('//bigmir.net/?a=b&c=d', ['a', 'c']): //bigmir.net/? cutURLParameter('//bigmir.net/?a=b&c=d#e=f', ['a', 'e']): //bigmir.net/?c=d# @@ -89,7 +89,7 @@ SELECT FORMAT Vertical; Row 1: ────── -cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), array()): http://bigmir.net/?a=b&c=d +cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), []): http://bigmir.net/?a=b&c=d cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), ['a']): http://bigmir.net/?c=d cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), ['a', 'c']): http://bigmir.net/? cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), ['c']): http://bigmir.net/?a=b @@ -102,7 +102,7 @@ cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e&g=h'), ['c', 'g']): cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e&g=h'), ['e', 'g']): http://bigmir.net/?a=b&c=d#e cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h'), ['test', 'e']): http://bigmir.net/?a=b&c=d#test?g=h cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h'), ['test', 'g']): http://bigmir.net/?a=b&c=d#test?e=f -cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), array()): //bigmir.net/?a=b&c=d +cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), []): //bigmir.net/?a=b&c=d cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), ['a']): //bigmir.net/?c=d cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), ['a', 'c']): //bigmir.net/? cutURLParameter(materialize('//bigmir.net/?a=b&c=d#e=f'), ['a', 'e']): //bigmir.net/?c=d# diff --git a/tests/queries/0_stateless/02890_untuple_column_names.reference b/tests/queries/0_stateless/02890_untuple_column_names.reference index 831d5dac39a..388f974c45f 100644 --- a/tests/queries/0_stateless/02890_untuple_column_names.reference +++ b/tests/queries/0_stateless/02890_untuple_column_names.reference @@ -5,12 +5,12 @@ tupleElement(CAST(tuple(1), 'Tuple(a Int)'), 1): 1 tupleElement(CAST(tuple('s'), 'Tuple(a String)'), 1): s Row 1: ────── -tupleElement(CAST(tuple(1), 'Tuple(a Int)'), 'a'): 1 -tupleElement(CAST(tuple('s'), 'Tuple(a String)'), 'a'): s +tupleElement(CAST((1), 'Tuple(a Int)'), 'a'): 1 +tupleElement(CAST(('s'), 'Tuple(a String)'), 'a'): s Row 1: ────── -tupleElement(CAST(tuple(1), 'Tuple(a Int)'), 'a'): 1 -tupleElement(CAST(tuple(1), 'Tuple(a Int)'), 'a'): 1 +tupleElement(CAST((1), 'Tuple(a Int)'), 'a'): 1 +tupleElement(CAST((1), 'Tuple(a Int)'), 'a'): 1 -- tuple element alias + untuple() alias Row 1: ────── @@ -44,12 +44,12 @@ tupleElement(CAST(tuple(1), 'Tuple(Int)'), 1): 1 tupleElement(CAST(tuple('s'), 'Tuple(String)'), 1): s Row 1: ────── -tupleElement(CAST(tuple(1), 'Tuple(Int)'), '1'): 1 -tupleElement(CAST(tuple('s'), 'Tuple(String)'), '1'): s +tupleElement(CAST((1), 'Tuple(Int)'), '1'): 1 +tupleElement(CAST(('s'), 'Tuple(String)'), '1'): s Row 1: ────── -tupleElement(CAST(tuple(1), 'Tuple(Int)'), '1'): 1 -tupleElement(CAST(tuple(1), 'Tuple(Int)'), '1'): 1 +tupleElement(CAST((1), 'Tuple(Int)'), '1'): 1 +tupleElement(CAST((1), 'Tuple(Int)'), '1'): 1 -- tuple() loses the column names (would be good to fix, see #36773) Row 1: ────── From 7b16e90b2a694fd61461bf8cd9328009fb792cee Mon Sep 17 00:00:00 2001 From: ruslandoga Date: Sat, 11 Nov 2023 20:54:03 +0900 Subject: [PATCH 24/28] add plausible/ecto_ch to client-libraries.md --- docs/en/interfaces/third-party/client-libraries.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/interfaces/third-party/client-libraries.md b/docs/en/interfaces/third-party/client-libraries.md index 7b5c4f27a2a..5aa634785aa 100644 --- a/docs/en/interfaces/third-party/client-libraries.md +++ b/docs/en/interfaces/third-party/client-libraries.md @@ -74,6 +74,7 @@ ClickHouse Inc does **not** maintain the libraries listed below and hasn’t don ### Elixir - [clickhousex](https://github.com/appodeal/clickhousex/) - [pillar](https://github.com/sofakingworld/pillar) + - [ecto_ch](https://github.com/plausible/ecto_ch) ### Nim - [nim-clickhouse](https://github.com/leonardoce/nim-clickhouse) ### Haskell From bc68fcf7599b5b6bd52f42d693f3efb34915873d Mon Sep 17 00:00:00 2001 From: Shri Bodas Date: Sat, 11 Nov 2023 10:11:47 -0800 Subject: [PATCH 25/28] canonicalRand is replaced by randCanonical --- docs/en/sql-reference/data-types/float.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/float.md b/docs/en/sql-reference/data-types/float.md index 3b55271f707..f1b99153b41 100644 --- a/docs/en/sql-reference/data-types/float.md +++ b/docs/en/sql-reference/data-types/float.md @@ -16,7 +16,7 @@ CREATE TABLE IF NOT EXISTS float_vs_decimal my_decimal Decimal64(3) )Engine=MergeTree ORDER BY tuple() -INSERT INTO float_vs_decimal SELECT round(canonicalRand(), 3) AS res, res FROM system.numbers LIMIT 1000000; # Generate 1 000 000 random number with 2 decimal places and store them as a float and as a decimal +INSERT INTO float_vs_decimal SELECT round(randCanonical(), 3) AS res, res FROM system.numbers LIMIT 1000000; # Generate 1 000 000 random number with 2 decimal places and store them as a float and as a decimal SELECT sum(my_float), sum(my_decimal) FROM float_vs_decimal; > 500279.56300000014 500279.563 From 0bfcd3958eb525fedd9a9ad056331125d1152052 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Sat, 11 Nov 2023 11:37:26 +0100 Subject: [PATCH 26/28] Add test describing MV deduplication issues --- ...02912_ingestion_mv_deduplication.reference | 24 +++ .../02912_ingestion_mv_deduplication.sql | 146 ++++++++++++++++++ 2 files changed, 170 insertions(+) create mode 100644 tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference create mode 100644 tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql diff --git a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference new file mode 100644 index 00000000000..a22ecdd1d78 --- /dev/null +++ b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference @@ -0,0 +1,24 @@ +-- Original issue with max_insert_delayed_streams_for_parallel_write = 1 +-- Landing +2022-09-01 12:23:34 42 +2023-09-01 12:23:34 42 +-- MV +2022-09-01 12:00:00 84 +2023-09-01 12:00:00 42 +-- Original issue with max_insert_delayed_streams_for_parallel_write > 1 +-- Landing +2022-09-01 12:23:34 42 +2023-09-01 12:23:34 42 +-- MV +2022-09-01 12:00:00 42 +-- Regression introduced in https://github.com/ClickHouse/ClickHouse/pull/54184 +-- Landing (Agg/Replacing)MergeTree +org-1 prod checkout user 1 +org-1 prod login account 1 +org-1 prod login user 1 +org-1 stage login user 1 +--- MV +org-1 prod checkout user 1 +org-1 prod login account 3 +org-1 prod login user 3 +org-1 stage login user 1 diff --git a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql new file mode 100644 index 00000000000..e445ac9822b --- /dev/null +++ b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql @@ -0,0 +1,146 @@ +-- Tags: replica +SET session_timezone = 'UTC'; +SET deduplicate_blocks_in_dependent_materialized_views = 0; + +SELECT '-- Original issue with max_insert_delayed_streams_for_parallel_write = 1'; +/* + + This is the expected behavior when mv deduplication is set to false. TODO: Add more details about what happened. + +*/ +SET max_insert_delayed_streams_for_parallel_write = 1; + +CREATE TABLE landing +( + time DateTime, + number Int64 +) +Engine=ReplicatedReplacingMergeTree('/clickhouse/' || currentDatabase() || '/landing/{shard}/', '{replica}') +PARTITION BY toYYYYMMDD(time) +ORDER BY time; + +CREATE MATERIALIZED VIEW mv +ENGINE = ReplicatedSummingMergeTree('/clickhouse/' || currentDatabase() || '/mv/{shard}/', '{replica}') +PARTITION BY toYYYYMMDD(hour) ORDER BY hour +AS SELECT + toStartOfHour(time) AS hour, + sum(number) AS sum_amount +FROM landing +GROUP BY hour; + +INSERT INTO landing VALUES ('2022-09-01 12:23:34', 42); +INSERT INTO landing VALUES ('2022-09-01 12:23:34', 42),('2023-09-01 12:23:34', 42); + +SELECT '-- Landing'; +SELECT * FROM landing FINAL ORDER BY time; +SELECT '-- MV'; +SELECT * FROM mv FINAL ORDER BY hour; + +DROP TABLE IF EXISTS landing SYNC; +DROP TABLE IF EXISTS mv SYNC; + +SELECT '-- Original issue with max_insert_delayed_streams_for_parallel_write > 1'; +/* + + This is the unexpected behavior. TODO: Add more details about what happened. + +*/ +SET max_insert_delayed_streams_for_parallel_write = 10; + +CREATE TABLE landing +( + time DateTime, + number Int64 +) +Engine=ReplicatedReplacingMergeTree('/clickhouse/' || currentDatabase() || '/landing/{shard}/', '{replica}') +PARTITION BY toYYYYMMDD(time) +ORDER BY time; + +CREATE MATERIALIZED VIEW mv +ENGINE = ReplicatedSummingMergeTree('/clickhouse/' || currentDatabase() || '/mv/{shard}/', '{replica}') +PARTITION BY toYYYYMMDD(hour) ORDER BY hour +AS SELECT + toStartOfHour(time) AS hour, + sum(number) AS sum_amount +FROM landing +GROUP BY hour; + +INSERT INTO landing VALUES ('2022-09-01 12:23:34', 42); +INSERT INTO landing VALUES ('2022-09-01 12:23:34', 42),('2023-09-01 12:23:34', 42); + +SELECT '-- Landing'; +SELECT * FROM landing FINAL ORDER BY time; +SELECT '-- MV'; +SELECT * FROM mv FINAL ORDER BY hour; + +SET max_insert_delayed_streams_for_parallel_write = 1; +DROP TABLE IF EXISTS landing SYNC; +DROP TABLE IF EXISTS mv SYNC; + +SELECT '-- Regression introduced in https://github.com/ClickHouse/ClickHouse/pull/54184'; +/* + + This is a test to prevent regression introduced in https://github.com/ClickHouse/ClickHouse/pull/54184 from happening again + +*/ + +CREATE TABLE landing +( + `time` DateTime, + `pk1` LowCardinality(String), + `pk2` LowCardinality(String), + `pk3` LowCardinality(String), + `pk4` String +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{layer}-{shard}/landing', '{replica}') +ORDER BY (pk1, pk2, pk3, pk4); + +CREATE TABLE ds +( + `pk1` LowCardinality(String), + `pk2` LowCardinality(String), + `pk3` LowCardinality(String), + `pk4` LowCardinality(String), + `occurences` AggregateFunction(count) +) +ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/{layer}-{shard}/ds', '{replica}') +ORDER BY (pk1, pk2, pk3, pk4); + +CREATE MATERIALIZED VIEW mv TO ds AS +SELECT + pk1, + pk2, + pk4, + pk3, + countState() AS occurences +FROM landing +GROUP BY pk1, pk2, pk4, pk3; + +INSERT INTO landing (time, pk1, pk2, pk4, pk3) +VALUES ('2023-01-01 00:00:00','org-1','prod','login','user'),('2023-01-01 00:00:00','org-1','prod','login','user'),('2023-01-01 00:00:00','org-1','prod','login','user'),('2023-02-01 00:00:00','org-1','stage','login','user'),('2023-02-01 00:00:00','org-1','prod','login','account'),('2023-02-01 00:00:00','org-1','prod','checkout','user'),('2023-03-01 00:00:00','org-1','prod','login','account'),('2023-03-01 00:00:00','org-1','prod','login','account'); + +SELECT '-- Landing (Agg/Replacing)MergeTree'; +SELECT + pk1, + pk2, + pk4, + pk3, + count() as occurences +FROM landing +GROUP BY pk1, pk2, pk4, pk3 +ORDER BY pk1, pk2, pk4, pk3; + +SELECT '--- MV'; +SELECT + pk1, + pk2, + pk4, + pk3, + countMerge(occurences) AS occurences +FROM ds +GROUP BY pk1, pk2, pk4, pk3 +ORDER BY pk1, pk2, pk4, pk3; + +DROP TABLE IF EXISTS landing SYNC; +DROP TABLE IF EXISTS ds SYNC; +DROP TABLE IF EXISTS mv SYNC; From 684f3aa2407e024547bcbdf8f9e38c79acb3ca04 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Sat, 11 Nov 2023 11:52:23 +0100 Subject: [PATCH 27/28] Improve test --- ...02912_ingestion_mv_deduplication.reference | 9 ++- .../02912_ingestion_mv_deduplication.sql | 77 ++++++++++++++++--- 2 files changed, 76 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference index a22ecdd1d78..946897a4fe3 100644 --- a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference +++ b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference @@ -5,12 +5,19 @@ -- MV 2022-09-01 12:00:00 84 2023-09-01 12:00:00 42 --- Original issue with max_insert_delayed_streams_for_parallel_write > 1 +-- Original issue with deduplicate_blocks_in_dependent_materialized_views = 0 AND max_insert_delayed_streams_for_parallel_write > 1 -- Landing 2022-09-01 12:23:34 42 2023-09-01 12:23:34 42 -- MV 2022-09-01 12:00:00 42 +-- Original issue with deduplicate_blocks_in_dependent_materialized_views = 1 AND max_insert_delayed_streams_for_parallel_write > 1 +-- Landing +2022-09-01 12:23:34 42 +2023-09-01 12:23:34 42 +-- MV +2022-09-01 12:00:00 42 +2023-09-01 12:00:00 42 -- Regression introduced in https://github.com/ClickHouse/ClickHouse/pull/54184 -- Landing (Agg/Replacing)MergeTree org-1 prod checkout user 1 diff --git a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql index e445ac9822b..68901b67c91 100644 --- a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql +++ b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql @@ -1,14 +1,17 @@ -- Tags: replica SET session_timezone = 'UTC'; -SET deduplicate_blocks_in_dependent_materialized_views = 0; SELECT '-- Original issue with max_insert_delayed_streams_for_parallel_write = 1'; /* - This is the expected behavior when mv deduplication is set to false. TODO: Add more details about what happened. + This is the expected behavior when mv deduplication is set to false. + + - 1st insert works for landing and mv tables + - 2nd insert gets first block 20220901 deduplicated and second one inserted in landing table + - 2nd insert gets both blocks inserted in mv table */ -SET max_insert_delayed_streams_for_parallel_write = 1; +SET deduplicate_blocks_in_dependent_materialized_views = 0, max_insert_delayed_streams_for_parallel_write = 1; CREATE TABLE landing ( @@ -39,13 +42,66 @@ SELECT * FROM mv FINAL ORDER BY hour; DROP TABLE IF EXISTS landing SYNC; DROP TABLE IF EXISTS mv SYNC; -SELECT '-- Original issue with max_insert_delayed_streams_for_parallel_write > 1'; +SELECT '-- Original issue with deduplicate_blocks_in_dependent_materialized_views = 0 AND max_insert_delayed_streams_for_parallel_write > 1'; /* - This is the unexpected behavior. TODO: Add more details about what happened. + This is the unexpected behavior due to setting max_insert_delayed_streams_for_parallel_write > 1. + + This unexpected behavior was present since version 21.9 or earlier but due to this PR https://github.com/ClickHouse/ClickHouse/pull/34780 + when max_insert_delayed_streams_for_parallel_write setting it to 1 by default the issue was mitigated. + + This is what happens: + + - 1st insert works for landing and mv tables + - 2nd insert gets first block 20220901 deduplicated and second one inserted in landing table + - 2nd insert is not inserting anything in mv table due to a bug computing blocks to be discarded */ -SET max_insert_delayed_streams_for_parallel_write = 10; +SET deduplicate_blocks_in_dependent_materialized_views = 0, max_insert_delayed_streams_for_parallel_write = 10; + +CREATE TABLE landing +( + time DateTime, + number Int64 +) +Engine=ReplicatedReplacingMergeTree('/clickhouse/' || currentDatabase() || '/landing/{shard}/', '{replica}') +PARTITION BY toYYYYMMDD(time) +ORDER BY time; + +CREATE MATERIALIZED VIEW mv +ENGINE = ReplicatedSummingMergeTree('/clickhouse/' || currentDatabase() || '/mv/{shard}/', '{replica}') +PARTITION BY toYYYYMMDD(hour) ORDER BY hour +AS SELECT + toStartOfHour(time) AS hour, + sum(number) AS sum_amount +FROM landing +GROUP BY hour; + +INSERT INTO landing VALUES ('2022-09-01 12:23:34', 42); +INSERT INTO landing VALUES ('2022-09-01 12:23:34', 42),('2023-09-01 12:23:34', 42); + +SELECT '-- Landing'; +SELECT * FROM landing FINAL ORDER BY time; +SELECT '-- MV'; +SELECT * FROM mv FINAL ORDER BY hour; + +SET max_insert_delayed_streams_for_parallel_write = 1; +DROP TABLE IF EXISTS landing SYNC; +DROP TABLE IF EXISTS mv SYNC; + +SELECT '-- Original issue with deduplicate_blocks_in_dependent_materialized_views = 1 AND max_insert_delayed_streams_for_parallel_write > 1'; +/* + + By setting deduplicate_blocks_in_dependent_materialized_views = 1 we can make the code go through a different path getting an expected + behavior again, even with max_insert_delayed_streams_for_parallel_write > 1. + + This is what happens now: + + - 1st insert works for landing and mv tables + - 2nd insert gets first block 20220901 deduplicated and second one inserted for landing and mv tables + +*/ +SET deduplicate_blocks_in_dependent_materialized_views = 1, max_insert_delayed_streams_for_parallel_write = 10; CREATE TABLE landing ( @@ -80,7 +136,10 @@ DROP TABLE IF EXISTS mv SYNC; SELECT '-- Regression introduced in https://github.com/ClickHouse/ClickHouse/pull/54184'; /* - This is a test to prevent regression introduced in https://github.com/ClickHouse/ClickHouse/pull/54184 from happening again + This is a test to prevent regression introduced in https://github.com/ClickHouse/ClickHouse/pull/54184 from happening again. + + The PR was trying to fix the unexpected behavior when deduplicate_blocks_in_dependent_materialized_views = 0 AND + max_insert_delayed_streams_for_parallel_write > 1 but it ended up adding a new regression. */ @@ -92,7 +151,7 @@ CREATE TABLE landing `pk3` LowCardinality(String), `pk4` String ) -ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{layer}-{shard}/landing', '{replica}') +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/' || currentDatabase() || '/landing/{shard}/', '{replica}') ORDER BY (pk1, pk2, pk3, pk4); CREATE TABLE ds @@ -103,7 +162,7 @@ CREATE TABLE ds `pk4` LowCardinality(String), `occurences` AggregateFunction(count) ) -ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/{layer}-{shard}/ds', '{replica}') +ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/' || currentDatabase() || '/ds/{shard}/', '{replica}') ORDER BY (pk1, pk2, pk3, pk4); CREATE MATERIALIZED VIEW mv TO ds AS From 82b41f232ad397da6d92a13bca583ee1a3c8a847 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 12 Nov 2023 00:51:48 +0100 Subject: [PATCH 28/28] Fix bad test CC @AVMusorin --- .../0_stateless/01555_system_distribution_queue_mask.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql index 472e042a18b..3a90765226a 100644 --- a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql +++ b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql @@ -18,7 +18,7 @@ system stop distributed sends dist_01555; insert into dist_01555 values (1)(2); -- since test_cluster_with_incorrect_pw contains incorrect password ignore error system flush distributed dist_01555; -- { serverError 516 } -select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 5 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; +select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 3600 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; drop table dist_01555; @@ -31,7 +31,7 @@ create table dist_01555 (key Int) Engine=Distributed(test_cluster_with_incorrect insert into dist_01555 values (1)(2); -- since test_cluster_with_incorrect_pw contains incorrect password ignore error system flush distributed dist_01555; -- { serverError 516 } -select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 5 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; +select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 3600 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; drop table dist_01555;