From f6a220916991d379d6bce7a320304e982184ded5 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Mon, 30 Mar 2020 18:36:02 +0200 Subject: [PATCH 001/211] Add target specific macros --- dbms/src/Functions/DynamicTarget/Target.h | 106 ++++++++++++++++++++++ src/Functions/IFunctionImpl.h | 2 +- 2 files changed, 107 insertions(+), 1 deletion(-) create mode 100644 dbms/src/Functions/DynamicTarget/Target.h diff --git a/dbms/src/Functions/DynamicTarget/Target.h b/dbms/src/Functions/DynamicTarget/Target.h new file mode 100644 index 00000000000..5e3032ded3e --- /dev/null +++ b/dbms/src/Functions/DynamicTarget/Target.h @@ -0,0 +1,106 @@ +#pragma once + +namespace DB::DynamicTarget +{ + +enum class TargetArch : int { + Scalar, + SSE4, + AVX, + AVX2, + AVX512, +}; + +#if defined(__GNUC__) +// TODO: There are lots of different AVX512 :( +# define BEGIN_AVX512_SPECIFIC_CODE \ + _Pragma("GCC push_options") \ + _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2,tune=native\")") +# define BEGIN_AVX2_SPECIFIC_CODE \ + _Pragma("GCC push_options") \ + _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2,tune=native\")") +# define BEGIN_AVX_SPECIFIC_CODE \ + _Pragma("GCC push_options") \ + _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,tune=native\")") +# define BEGIN_SSE4_SPECIFIC_CODE \ + _Pragma("GCC push_options") \ + _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,tune=native\")") +# define END_TARGET_SPECIFIC_CODE \ + _Pragma("GCC pop_options") +#elif defined(__clang__) +// TODO: There are lots of different AVX512 :( +# define BEGIN_AVX512_SPECIFIC_CODE \ + _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2\"))))") +# define BEGIN_AVX2_SPECIFIC_CODE \ + _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2\"))))") +# define BEGIN_AVX_SPECIFIC_CODE \ + _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx\"))))") +# define BEGIN_SSE4_SPECIFIC_CODE \ + _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx\"))))") +# define END_TARGET_SPECIFIC_CODE \ + _Pragma("clang attribute pop") +#else +# error "Only CLANG and GCC compilers are supported" +#endif + +#define DECLARE_DEFAULT_CODE (...) \ +namespace TargetSpecific::Default { \ + __VA_ARGS__ \ +} + +#define DECLARE_SSE4_SPECIFIC_CODE (...) \ +BEGIN_SSE4_SPECIFIC_CODE \ +namespace TargetSpecific::SSE4 { \ + __VA_ARGS__ \ +} \ +END_TARGET_SPECIFIC_CODE + +#define DECLARE_AVX_SPECIFIC_CODE (...) \ +BEGIN_AVX_SPECIFIC_CODE \ +namespace TargetSpecific::AVX { \ + __VA_ARGS__ \ +} \ +END_TARGET_SPECIFIC_CODE + +#define DECLARE_AVX2_SPECIFIC_CODE (...) \ +BEGIN_AVX2_SPECIFIC_CODE \ +namespace TargetSpecific::AVX2 { \ + __VA_ARGS__ \ +} \ +END_TARGET_SPECIFIC_CODE + +#define DECLARE_AVX512_SPECIFIC_CODE (...) \ +BEGIN_AVX512_SPECIFIC_CODE \ +namespace TargetSpecific::AVX512 { \ + __VA_ARGS__ \ +} \ +END_TARGET_SPECIFIC_CODE + +#define DYNAMIC_CODE (...) \ +DECLARE_DEFAULT_CODE (__VA_ARGS__) \ +DECLARE_SSE4_SPECIFIC_CODE (__VA_ARGS__) \ +DECLARE_AVX_SPECIFIC_CODE (__VA_ARGS__) \ +DECLARE_AVX2_SPECIFIC_CODE (__VA_ARGS__) \ +DECLARE_AVX512_SPECIFIC_CODE(__VA_ARGS__) + +DECLARE_DEFAULT_CODE( + constexpr auto BuildArch = TargetArch::Scalar; +) // DECLARE_DEFAULT_CODE + +DECLARE_SSE4_SPECIFIC_CODE( + constexpr auto BuildArch = TargetArch::SSE4; +) // DECLARE_SSE4_SPECIFIC_CODE + +DECLARE_AVX_SPECIFIC_CODE( + constexpr auto BuildArch = TargetArch::AVX; +) // DECLARE_AVX_SPECIFIC_CODE + +DECLARE_AVX2_SPECIFIC_CODE( + constexpr auto BuildArch = TargetArch::AVX2; +) // DECLARE_AVX2_SPECIFIC_CODE + +DECLARE_AVX512_SPECIFIC_CODE( + constexpr auto BuildArch = TargetArch::AVX512; +) // DECLARE_AVX512_SPECIFIC_CODE + +} // namespace DB::DynamicTarget \ No newline at end of file diff --git a/src/Functions/IFunctionImpl.h b/src/Functions/IFunctionImpl.h index 116363705de..4da3e9ca056 100644 --- a/src/Functions/IFunctionImpl.h +++ b/src/Functions/IFunctionImpl.h @@ -5,7 +5,7 @@ /// In order to implement a new function you can choose one of two options: /// * Implement interface for IFunction (old function interface, which is planned to be removed sometimes) /// * Implement three interfaces for IExecutableFunctionImpl, IFunctionBaseImpl and IFunctionOverloadResolverImpl -/// Generally saying, IFunction represents a union of tree new interfaces. However, it can't be used for all cases. +/// Generally saying, IFunction represents a union of three new interfaces. However, it can't be used for all cases. /// Examples: /// * Function properties may depend on arguments type (e.g. toUInt32(UInt8) is globally monotonic, toUInt32(UInt64) - only on intervals) /// * In implementation of lambda functions DataTypeFunction needs an functional object with known arguments and return type From 37d13d4bce5f4260979e38617457b9136ca1a15f Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 2 Apr 2020 15:48:14 +0200 Subject: [PATCH 002/211] Compilable --- .../Functions/DynamicTarget/CMakeLists.txt | 11 + .../DynamicTarget/DynamicFunctionAdaptors.h | 263 ++++++++++++++++++ dbms/src/Functions/DynamicTarget/Selector.h | 39 +++ dbms/src/Functions/DynamicTarget/Target.cpp | 12 + dbms/src/Functions/DynamicTarget/Target.h | 19 +- src/Compression/CompressionFactory.cpp | 2 +- src/Functions/CMakeLists.txt | 3 + src/Functions/FunctionStartsEndsWith.h | 103 ++++--- src/Functions/IFunctionImpl.h | 2 +- 9 files changed, 408 insertions(+), 46 deletions(-) create mode 100644 dbms/src/Functions/DynamicTarget/CMakeLists.txt create mode 100644 dbms/src/Functions/DynamicTarget/DynamicFunctionAdaptors.h create mode 100644 dbms/src/Functions/DynamicTarget/Selector.h create mode 100644 dbms/src/Functions/DynamicTarget/Target.cpp diff --git a/dbms/src/Functions/DynamicTarget/CMakeLists.txt b/dbms/src/Functions/DynamicTarget/CMakeLists.txt new file mode 100644 index 00000000000..154e47ab5f6 --- /dev/null +++ b/dbms/src/Functions/DynamicTarget/CMakeLists.txt @@ -0,0 +1,11 @@ +include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) +add_headers_and_sources(clickhouse_functions_dynamic_target .) +add_library(clickhouse_functions_dynamic_target + ${clickhouse_functions_dynamic_target_sources} + ${clickhouse_functions_dynamic_target_headers}) +target_link_libraries(clickhouse_functions_dynamic_target PRIVATE dbms) + +if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL") + # Won't generate debug info for files with heavy template instantiation to achieve faster linking and lower size. + target_compile_options(clickhouse_functions_dynamic_target PRIVATE "-g0") +endif () diff --git a/dbms/src/Functions/DynamicTarget/DynamicFunctionAdaptors.h b/dbms/src/Functions/DynamicTarget/DynamicFunctionAdaptors.h new file mode 100644 index 00000000000..123faa859e9 --- /dev/null +++ b/dbms/src/Functions/DynamicTarget/DynamicFunctionAdaptors.h @@ -0,0 +1,263 @@ +#pragma once +#include + +namespace DB +{ + +/// Adaptors are implement user interfaces from IFunction.h via developer interfaces from IFunctionImpl.h +/// Typically, you don't need to change this classes. + +class ExecutableFunctionAdaptor final : public IExecutableFunction +{ +public: + explicit ExecutableFunctionAdaptor(ExecutableFunctionImplPtr impl_) : impl(std::move(impl_)) {} + + String getName() const final { return impl->getName(); } + + void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run) final; + + void createLowCardinalityResultCache(size_t cache_size) override; + +private: + ExecutableFunctionImplPtr impl; + + /// Cache is created by function createLowCardinalityResultCache() + ExecutableFunctionLowCardinalityResultCachePtr low_cardinality_result_cache; + + bool defaultImplementationForConstantArguments( + Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run); + + bool defaultImplementationForNulls( + Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run); + + void executeWithoutLowCardinalityColumns( + Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run); +}; + +class FunctionBaseAdaptor final : public IFunctionBase +{ +public: + explicit FunctionBaseAdaptor(FunctionBaseImplPtr impl_) : impl(std::move(impl_)) {} + + String getName() const final { return impl->getName(); } + + const DataTypes & getArgumentTypes() const final { return impl->getArgumentTypes(); } + const DataTypePtr & getReturnType() const final { return impl->getReturnType(); } + + ExecutableFunctionPtr prepare(const Block & sample_block, const ColumnNumbers & arguments, size_t result) const final + { + return std::make_shared(impl->prepare(sample_block, arguments, result)); + } + +#if USE_EMBEDDED_COMPILER + + bool isCompilable() const final { return impl->isCompilable(); } + + llvm::Value * compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const override + { + return impl->compile(builder, std::move(values)); + } + +#endif + + bool isStateful() const final { return impl->isStateful(); } + bool isSuitableForConstantFolding() const final { return impl->isSuitableForConstantFolding(); } + + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments) const final + { + return impl->getResultIfAlwaysReturnsConstantAndHasArguments(block, arguments); + } + + bool isInjective(const Block & sample_block) final { return impl->isInjective(sample_block); } + bool isDeterministic() const final { return impl->isDeterministic(); } + bool isDeterministicInScopeOfQuery() const final { return impl->isDeterministicInScopeOfQuery(); } + bool hasInformationAboutMonotonicity() const final { return impl->hasInformationAboutMonotonicity(); } + + Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const final + { + return impl->getMonotonicityForRange(type, left, right); + } + + const IFunctionBaseImpl * getImpl() const { return impl.get(); } + +private: + FunctionBaseImplPtr impl; +}; + + +class FunctionOverloadResolverAdaptor final : public IFunctionOverloadResolver +{ +public: + explicit FunctionOverloadResolverAdaptor(FunctionOverloadResolverImplPtr impl_) : impl(std::move(impl_)) {} + + String getName() const final { return impl->getName(); } + + bool isDeterministic() const final { return impl->isDeterministic(); } + + bool isDeterministicInScopeOfQuery() const final { return impl->isDeterministicInScopeOfQuery(); } + + bool isStateful() const final { return impl->isStateful(); } + + bool isVariadic() const final { return impl->isVariadic(); } + + size_t getNumberOfArguments() const final { return impl->getNumberOfArguments(); } + + void checkNumberOfArguments(size_t number_of_arguments) const final; + + FunctionBaseImplPtr buildImpl(const ColumnsWithTypeAndName & arguments) const + { + return impl->build(arguments, getReturnType(arguments)); + } + + FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const final + { + return std::make_shared(buildImpl(arguments)); + } + + void getLambdaArgumentTypes(DataTypes & arguments) const final + { + checkNumberOfArguments(arguments.size()); + impl->getLambdaArgumentTypes(arguments); + } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return impl->getArgumentsThatAreAlwaysConstant(); } + + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t number_of_arguments) const final + { + return impl->getArgumentsThatDontImplyNullableReturnType(number_of_arguments); + } + +private: + FunctionOverloadResolverImplPtr impl; + + DataTypePtr getReturnTypeWithoutLowCardinality(const ColumnsWithTypeAndName & arguments) const; + DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const; +}; + + +/// Following classes are implement IExecutableFunctionImpl, IFunctionBaseImpl and IFunctionOverloadResolverImpl via IFunction. + +class DefaultExecutable final : public IExecutableFunctionImpl +{ +public: + explicit DefaultExecutable(std::shared_ptr function_) : function(std::move(function_)) {} + + String getName() const override { return function->getName(); } + +protected: + void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final + { + return function->executeImpl(block, arguments, result, input_rows_count); + } + void executeDryRun(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final + { + return function->executeImplDryRun(block, arguments, result, input_rows_count); + } + bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); } + bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); } + bool useDefaultImplementationForLowCardinalityColumns() const final { return function->useDefaultImplementationForLowCardinalityColumns(); } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return function->getArgumentsThatAreAlwaysConstant(); } + bool canBeExecutedOnDefaultArguments() const override { return function->canBeExecutedOnDefaultArguments(); } + +private: + std::shared_ptr function; +}; + +class DefaultFunction final : public IFunctionBaseImpl +{ +public: + DefaultFunction(std::shared_ptr function_, DataTypes arguments_, DataTypePtr return_type_) + : function(std::move(function_)), arguments(std::move(arguments_)), return_type(std::move(return_type_)) {} + + String getName() const override { return function->getName(); } + + const DataTypes & getArgumentTypes() const override { return arguments; } + const DataTypePtr & getReturnType() const override { return return_type; } + +#if USE_EMBEDDED_COMPILER + + bool isCompilable() const override { return function->isCompilable(arguments); } + + llvm::Value * compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const override { return function->compile(builder, arguments, std::move(values)); } + +#endif + + ExecutableFunctionImplPtr prepare(const Block & /*sample_block*/, const ColumnNumbers & /*arguments*/, size_t /*result*/) const override + { + return std::make_unique(function); + } + + bool isSuitableForConstantFolding() const override { return function->isSuitableForConstantFolding(); } + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments_) const override + { + return function->getResultIfAlwaysReturnsConstantAndHasArguments(block, arguments_); + } + + bool isStateful() const override { return function->isStateful(); } + + bool isInjective(const Block & sample_block) override { return function->isInjective(sample_block); } + + bool isDeterministic() const override { return function->isDeterministic(); } + + bool isDeterministicInScopeOfQuery() const override { return function->isDeterministicInScopeOfQuery(); } + + bool hasInformationAboutMonotonicity() const override { return function->hasInformationAboutMonotonicity(); } + + using Monotonicity = IFunctionBase::Monotonicity; + Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override + { + return function->getMonotonicityForRange(type, left, right); + } +private: + std::shared_ptr function; + DataTypes arguments; + DataTypePtr return_type; +}; + +class DefaultOverloadResolver : public IFunctionOverloadResolverImpl +{ +public: + explicit DefaultOverloadResolver(std::shared_ptr function_) : function(std::move(function_)) {} + + void checkNumberOfArgumentsIfVariadic(size_t number_of_arguments) const override + { + return function->checkNumberOfArgumentsIfVariadic(number_of_arguments); + } + + bool isDeterministic() const override { return function->isDeterministic(); } + bool isDeterministicInScopeOfQuery() const override { return function->isDeterministicInScopeOfQuery(); } + + String getName() const override { return function->getName(); } + bool isStateful() const override { return function->isStateful(); } + bool isVariadic() const override { return function->isVariadic(); } + size_t getNumberOfArguments() const override { return function->getNumberOfArguments(); } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return function->getArgumentsThatAreAlwaysConstant(); } + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t number_of_arguments) const override + { + return function->getArgumentsThatDontImplyNullableReturnType(number_of_arguments); + } + + DataTypePtr getReturnType(const DataTypes & arguments) const override { return function->getReturnTypeImpl(arguments); } + DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const override { return function->getReturnTypeImpl(arguments); } + + bool useDefaultImplementationForNulls() const override { return function->useDefaultImplementationForNulls(); } + bool useDefaultImplementationForLowCardinalityColumns() const override { return function->useDefaultImplementationForLowCardinalityColumns(); } + bool canBeExecutedOnLowCardinalityDictionary() const override { return function->canBeExecutedOnLowCardinalityDictionary(); } + + FunctionBaseImplPtr build(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; + return std::make_unique(function, data_types, return_type); + } + + void getLambdaArgumentTypes(DataTypes & arguments) const override { function->getLambdaArgumentTypes(arguments); } + +private: + std::shared_ptr function; +}; + + +} diff --git a/dbms/src/Functions/DynamicTarget/Selector.h b/dbms/src/Functions/DynamicTarget/Selector.h new file mode 100644 index 00000000000..257172a7223 --- /dev/null +++ b/dbms/src/Functions/DynamicTarget/Selector.h @@ -0,0 +1,39 @@ +#pragma once + +#include "Target.h" + +namespace DB::DynamicTarget +{ + +class PerformanceStatistic +{}; + +template +class SelectorExecutor +{ +public: + using Executor = std::function; + // Should register all executors before + void registerExecutor(std::optional arch, Executor executor) + { + if (!arch || IsArchSupported(*arch)) { + executors_.emplace_back(std::move(executor)); + } + } + + void execute(Params... params) + { + if (executors_.empty()) { + throw "There are no realizations for this arch Arch"; + } + int impl = 0; + // TODO: choose implementation. + executors_[impl](params...); + } + +private: + std::vector executors_; + PerformanceStatistic statistic_; +}; + +} // namespace DB::DynamicTarget \ No newline at end of file diff --git a/dbms/src/Functions/DynamicTarget/Target.cpp b/dbms/src/Functions/DynamicTarget/Target.cpp new file mode 100644 index 00000000000..e0bdb509164 --- /dev/null +++ b/dbms/src/Functions/DynamicTarget/Target.cpp @@ -0,0 +1,12 @@ +#include "Target.h" + +namespace DB::DynamicTarget +{ + +bool IsArchSupported(TargetArch arch) +{ + // TODO(dakovalkov): use cpuid + return arch != TargetArch::AVX512; +} + +} // namespace DB::DynamicTarget \ No newline at end of file diff --git a/dbms/src/Functions/DynamicTarget/Target.h b/dbms/src/Functions/DynamicTarget/Target.h index 5e3032ded3e..2873871d16c 100644 --- a/dbms/src/Functions/DynamicTarget/Target.h +++ b/dbms/src/Functions/DynamicTarget/Target.h @@ -43,40 +43,45 @@ enum class TargetArch : int { # error "Only CLANG and GCC compilers are supported" #endif -#define DECLARE_DEFAULT_CODE (...) \ +#define DECLARE_DEFAULT_CODE(...) \ namespace TargetSpecific::Default { \ + using namespace DB::DynamicTarget::TargetSpecific::Default; \ __VA_ARGS__ \ } -#define DECLARE_SSE4_SPECIFIC_CODE (...) \ +#define DECLARE_SSE4_SPECIFIC_CODE(...) \ BEGIN_SSE4_SPECIFIC_CODE \ namespace TargetSpecific::SSE4 { \ + using namespace DB::DynamicTarget::TargetSpecific::SSE4; \ __VA_ARGS__ \ } \ END_TARGET_SPECIFIC_CODE -#define DECLARE_AVX_SPECIFIC_CODE (...) \ +#define DECLARE_AVX_SPECIFIC_CODE(...) \ BEGIN_AVX_SPECIFIC_CODE \ namespace TargetSpecific::AVX { \ + using namespace DB::DynamicTarget::TargetSpecific::AVX; \ __VA_ARGS__ \ } \ END_TARGET_SPECIFIC_CODE -#define DECLARE_AVX2_SPECIFIC_CODE (...) \ +#define DECLARE_AVX2_SPECIFIC_CODE(...) \ BEGIN_AVX2_SPECIFIC_CODE \ namespace TargetSpecific::AVX2 { \ + using namespace DB::DynamicTarget::TargetSpecific::AVX2; \ __VA_ARGS__ \ } \ END_TARGET_SPECIFIC_CODE -#define DECLARE_AVX512_SPECIFIC_CODE (...) \ +#define DECLARE_AVX512_SPECIFIC_CODE(...) \ BEGIN_AVX512_SPECIFIC_CODE \ namespace TargetSpecific::AVX512 { \ + using namespace DB::DynamicTarget::TargetSpecific::AVX512; \ __VA_ARGS__ \ } \ END_TARGET_SPECIFIC_CODE -#define DYNAMIC_CODE (...) \ +#define DECLARE_MULTITARGET_CODE(...) \ DECLARE_DEFAULT_CODE (__VA_ARGS__) \ DECLARE_SSE4_SPECIFIC_CODE (__VA_ARGS__) \ DECLARE_AVX_SPECIFIC_CODE (__VA_ARGS__) \ @@ -103,4 +108,6 @@ DECLARE_AVX512_SPECIFIC_CODE( constexpr auto BuildArch = TargetArch::AVX512; ) // DECLARE_AVX512_SPECIFIC_CODE +bool IsArchSupported(TargetArch arch); + } // namespace DB::DynamicTarget \ No newline at end of file diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 5d5c5c14de6..2598fc07b08 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -120,7 +120,7 @@ void CompressionCodecFactory::registerCompressionCodecWithType( if (byte_code) if (!family_code_with_codec.emplace(*byte_code, creator).second) - throw Exception("CompressionCodecFactory: the codec family name '" + family_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); + throw Exception("CompressionCodecFactory: the codec family code '" + std::to_string(*byte_code) + "' is not unique", ErrorCodes::LOGICAL_ERROR); } void CompressionCodecFactory::registerCompressionCodec(const String & family_name, std::optional byte_code, Creator creator) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 069a63aa9e1..e9a33283d5b 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -91,3 +91,6 @@ target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_url) add_subdirectory(array) target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_array) + +add_subdirectory(DynamicTarget) +target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_dynamic_target) \ No newline at end of file diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 4f56a827f4c..0e63b616558 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -6,6 +6,8 @@ #include #include +#include +#include namespace DB { @@ -27,43 +29,14 @@ struct NameEndsWith static constexpr auto name = "endsWith"; }; +using DynamicTarget::TargetArch; + +DECLARE_MULTITARGET_CODE( + template -class FunctionStartsEndsWith : public IFunction -{ +class FunctionStartsEndsWithImpl { public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 2; - } - - bool useDefaultImplementationForConstants() const override - { - return true; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isStringOrFixedString(arguments[0])) - throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!isStringOrFixedString(arguments[1])) - throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + static void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) { const IColumn * haystack_column = block.getByPosition(arguments[0]).column.get(); const IColumn * needle_column = block.getByPosition(arguments[1]).column.get(); @@ -82,14 +55,14 @@ public: else if (const ColumnConst * haystack_const_fixed = checkAndGetColumnConst(haystack_column)) dispatch>(ConstSource(*haystack_const_fixed), needle_column, vec_res); else - throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Illegal combination of columns as arguments of function " "getName()", ErrorCodes::ILLEGAL_COLUMN); block.getByPosition(result).column = std::move(col_res); } private: template - void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray & res_data) const + static void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray & res_data) { if (const ColumnString * needle = checkAndGetColumn(needle_column)) execute(haystack_source, StringSource(*needle), res_data); @@ -100,7 +73,7 @@ private: else if (const ColumnConst * needle_const_fixed = checkAndGetColumnConst(needle_column)) execute>(haystack_source, ConstSource(*needle_const_fixed), res_data); else - throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Illegal combination of columns as arguments of function " "getName()", ErrorCodes::ILLEGAL_COLUMN); } template @@ -136,4 +109,58 @@ private: } }; +) // DECLARE_MULTITARGET_CODE + +template +class FunctionStartsEndsWith : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + FunctionStartsEndsWith() { + executor_.registerExecutor(std::nullopt, TargetSpecific::Default::FunctionStartsEndsWithImpl::executeImpl); + executor_.registerExecutor(TargetArch::SSE4, TargetSpecific::SSE4::FunctionStartsEndsWithImpl::executeImpl); + executor_.registerExecutor(TargetArch::AVX, TargetSpecific::AVX::FunctionStartsEndsWithImpl::executeImpl); + executor_.registerExecutor(TargetArch::AVX2, TargetSpecific::AVX2::FunctionStartsEndsWithImpl::executeImpl); + executor_.registerExecutor(TargetArch::AVX512, TargetSpecific::AVX512::FunctionStartsEndsWithImpl::executeImpl); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 2; + } + + bool useDefaultImplementationForConstants() const override + { + return true; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isStringOrFixedString(arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isStringOrFixedString(arguments[1])) + throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + executor_.execute(block, arguments, result, input_rows_count); + } +private: + DynamicTarget::SelectorExecutor executor_; +}; + } diff --git a/src/Functions/IFunctionImpl.h b/src/Functions/IFunctionImpl.h index 4da3e9ca056..27e7aec6bd4 100644 --- a/src/Functions/IFunctionImpl.h +++ b/src/Functions/IFunctionImpl.h @@ -194,7 +194,7 @@ using FunctionOverloadResolverImplPtr = std::unique_ptr +class IFunction { public: virtual ~IFunction() = default; From 43657809d8f36c0e0595e0928d76b942381bad95 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Sun, 5 Apr 2020 14:01:33 +0200 Subject: [PATCH 003/211] awful adaptor --- dbms/src/Functions/DynamicTarget/Selector.h | 324 ++++++++++++++++++-- dbms/src/Functions/DynamicTarget/Target.cpp | 2 +- dbms/src/Functions/DynamicTarget/Target.h | 6 +- src/Functions/CMakeLists.txt | 2 +- src/Functions/FunctionStartsEndsWith.h | 98 +++--- 5 files changed, 358 insertions(+), 74 deletions(-) diff --git a/dbms/src/Functions/DynamicTarget/Selector.h b/dbms/src/Functions/DynamicTarget/Selector.h index 257172a7223..70c14dd2d11 100644 --- a/dbms/src/Functions/DynamicTarget/Selector.h +++ b/dbms/src/Functions/DynamicTarget/Selector.h @@ -2,38 +2,324 @@ #include "Target.h" +#include + +#include + namespace DB::DynamicTarget { -class PerformanceStatistic -{}; +// TODO(dakovalkov): This is copied and pasted struct from LZ4_decompress_faster.h -template -class SelectorExecutor +/** When decompressing uniform sequence of blocks (for example, blocks from one file), + * you can pass single PerformanceStatistics object to subsequent invocations of 'decompress' method. + * It will accumulate statistics and use it as a feedback to choose best specialization of algorithm at runtime. + * One PerformanceStatistics object cannot be used concurrently from different threads. + */ +struct PerformanceStatistics { -public: - using Executor = std::function; - // Should register all executors before - void registerExecutor(std::optional arch, Executor executor) + struct Element { - if (!arch || IsArchSupported(*arch)) { - executors_.emplace_back(std::move(executor)); + double count = 0; + double sum = 0; + + double adjustedCount() const + { + return count - NUM_INVOCATIONS_TO_THROW_OFF; } + + double mean() const + { + return sum / adjustedCount(); + } + + /// For better convergence, we don't use proper estimate of stddev. + /// We want to eventually separate between two algorithms even in case + /// when there is no statistical significant difference between them. + double sigma() const + { + return mean() / sqrt(adjustedCount()); + } + + void update(double seconds, double bytes) + { + ++count; + + if (count > NUM_INVOCATIONS_TO_THROW_OFF) + sum += seconds / bytes; + } + + double sample(pcg64 & stat_rng) const + { + /// If there is a variant with not enough statistics, always choose it. + /// And in that case prefer variant with less number of invocations. + + if (adjustedCount() < 2) + return adjustedCount() - 1; + else + return std::normal_distribution<>(mean(), sigma())(stat_rng); + } + }; + + /// Cold invocations may be affected by additional memory latencies. Don't take first invocations into account. + static constexpr double NUM_INVOCATIONS_TO_THROW_OFF = 2; + + /// How to select method to run. + /// -1 - automatically, based on statistics (default); + /// -2 - choose methods in round robin fashion (for performance testing). + /// >= 0 - always choose specified method (for performance testing); + ssize_t choose_method = -1; + + std::vector data; + + /// It's Ok that generator is not seeded. + pcg64 rng; + + /// To select from different algorithms we use a kind of "bandits" algorithm. + /// Sample random values from estimated normal distributions and choose the minimal. + size_t select() + { + if (choose_method < 0) + { + std::vector samples(data.size()); + for (size_t i = 0; i < data.size(); ++i) + samples[i] = choose_method == -1 + ? data[i].sample(rng) + : data[i].adjustedCount(); + + return std::min_element(samples.begin(), samples.end()) - samples.begin(); + } + else + return choose_method; } - void execute(Params... params) + size_t size() { + return data.size(); + } + + void emplace_back() { + data.emplace_back(); + } + + PerformanceStatistics() {} + PerformanceStatistics(ssize_t choose_method_) : choose_method(choose_method_) {} +}; + +// template +// class PerformanceExecutor +// { +// public: +// using Executor = std::function; +// // Should register all executors before execute +// void registerExecutor(Executor executor) +// { +// executors.emplace_back(std::move(executor)); +// } + +// // The performance of the execution is time / weight. +// // Weight is usualy the +// void execute(int weight, Params... params) +// { +// if (executors_.empty()) { +// throw "There are no realizations for current Arch"; +// } +// int impl = 0; +// // TODO: choose implementation. +// executors_[impl](params...); +// } + +// private: +// std::vector executors; +// PerformanceStatistics statistics; +// }; + +class FunctionDynamicAdaptor : public IFunction +{ +public: + template + FunctionDynamicAdaptor(const Context & context_) : context(context_) { - if (executors_.empty()) { - throw "There are no realizations for this arch Arch"; + registerImplementation(); + } + + virtual String getName() const override { + return impls.front()->getName(); + } + + virtual void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + int id = statistics.select(); + // TODO(dakovalkov): measure time and change statistics. + impls[id]->executeImpl(block, arguments, result, input_rows_count); + } + virtual void executeImplDryRun(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + impls.front()->executeImplDryRun(block, arguments, result, input_rows_count); + } + + virtual bool useDefaultImplementationForNulls() const override + { + return impls.front()->useDefaultImplementationForNulls(); + } + + virtual bool useDefaultImplementationForConstants() const override + { + return impls.front()->useDefaultImplementationForConstants(); + } + + virtual bool useDefaultImplementationForLowCardinalityColumns() const override + { + return impls.front()->useDefaultImplementationForLowCardinalityColumns(); + } + + virtual bool canBeExecutedOnLowCardinalityDictionary() const override + { + return impls.front()->canBeExecutedOnLowCardinalityDictionary(); + } + + virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const override + { + return impls.front()->getArgumentsThatAreAlwaysConstant(); + } + + virtual bool canBeExecutedOnDefaultArguments() const override + { + return impls.front()->canBeExecutedOnDefaultArguments(); + } + +#if USE_EMBEDDED_COMPILER + + virtual bool isCompilable() const override + { + return impls.front()->isCompilable(); + } + + virtual llvm::Value * compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const override + { + return impls.front()->compile(builder, std::move(values)); + } + +#endif + + /// Properties from IFunctionBase (see IFunction.h) + virtual bool isSuitableForConstantFolding() const override + { + return impls.front()->isSuitableForConstantFolding(); + } + virtual ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments) const override + { + return impls.front()->getResultIfAlwaysReturnsConstantAndHasArguments(block, arguments); + } + virtual bool isInjective(const Block & sample_block) override + { + return impls.front()->isInjective(sample_block); + } + virtual bool isDeterministic() const override + { + return impls.front()->isDeterministic(); + } + virtual bool isDeterministicInScopeOfQuery() const override + { + return impls.front()->isDeterministicInScopeOfQuery(); + } + virtual bool isStateful() const override + { + return impls.front()->isStateful(); + } + virtual bool hasInformationAboutMonotonicity() const override + { + return impls.front()->hasInformationAboutMonotonicity(); + } + + using Monotonicity = IFunctionBase::Monotonicity; + virtual Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override + { + return impls.front()->getMonotonicityForRange(type, left, right); + } + + virtual size_t getNumberOfArguments() const override { + return impls.front()->getNumberOfArguments(); + } + + virtual DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return impls.front()->getReturnTypeImpl(arguments); + } + + virtual DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + return impls.front()->getReturnTypeImpl(arguments); + } + + virtual bool isVariadic() const override + { + return impls.front()->isVariadic(); + } + + virtual void checkNumberOfArgumentsIfVariadic(size_t number_of_arguments) const override + { + impls.front()->checkNumberOfArgumentsIfVariadic(number_of_arguments); + } + + virtual void getLambdaArgumentTypes(DataTypes & arguments) const override + { + impls.front()->getLambdaArgumentTypes(arguments); + } + + virtual ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t number_of_arguments) const override + { + return impls.front()->getArgumentsThatDontImplyNullableReturnType(number_of_arguments); + } + +protected: + +#if USE_EMBEDDED_COMPILER + + virtual bool isCompilableImpl(const DataTypes & /* types */) const override + { + return false; + // return impls.front()->isCompilableImpl(types); + } + + virtual llvm::Value * compileImpl(llvm::IRBuilderBase & /* builder */, const DataTypes & /* types */, ValuePlaceholders /* ph */) const override + { + throw "safasf Error"; + // return impls.front()->compileImpl(builder, types, ph); + } + +#endif + /* + * Register implementation of the function. + */ + template + void registerImplementation(TargetArch arch = TargetArch::Default) { + if (arch == TargetArch::Default || IsArchSupported(arch)) { + impls.emplace_back(Function::create(context)); + statistics.emplace_back(); } - int impl = 0; - // TODO: choose implementation. - executors_[impl](params...); } private: - std::vector executors_; - PerformanceStatistic statistic_; + const Context & context; + std::vector impls; + PerformanceStatistics statistics; }; -} // namespace DB::DynamicTarget \ No newline at end of file +#define DECLARE_STANDART_TARGET_ADAPTOR(Function) \ +class Function : public FunctionDynamicAdaptor \ +{ \ +public: \ + Function(const Context & context) : FunctionDynamicAdaptor(context) \ + { \ + registerImplementation(TargetArch::SSE4); \ + registerImplementation(TargetArch::AVX); \ + registerImplementation(TargetArch::AVX2); \ + registerImplementation(TargetArch::AVX512); \ + } \ + static FunctionPtr create(const Context & context) \ + { \ + return std::make_shared(context); \ + } \ +} + +} // namespace DB::DynamicTarget diff --git a/dbms/src/Functions/DynamicTarget/Target.cpp b/dbms/src/Functions/DynamicTarget/Target.cpp index e0bdb509164..54c41a1786c 100644 --- a/dbms/src/Functions/DynamicTarget/Target.cpp +++ b/dbms/src/Functions/DynamicTarget/Target.cpp @@ -9,4 +9,4 @@ bool IsArchSupported(TargetArch arch) return arch != TargetArch::AVX512; } -} // namespace DB::DynamicTarget \ No newline at end of file +} // namespace DB::DynamicTarget diff --git a/dbms/src/Functions/DynamicTarget/Target.h b/dbms/src/Functions/DynamicTarget/Target.h index 2873871d16c..e1772a11857 100644 --- a/dbms/src/Functions/DynamicTarget/Target.h +++ b/dbms/src/Functions/DynamicTarget/Target.h @@ -4,7 +4,7 @@ namespace DB::DynamicTarget { enum class TargetArch : int { - Scalar, + Default, // Without any additional compiler options. SSE4, AVX, AVX2, @@ -89,7 +89,7 @@ DECLARE_AVX2_SPECIFIC_CODE (__VA_ARGS__) \ DECLARE_AVX512_SPECIFIC_CODE(__VA_ARGS__) DECLARE_DEFAULT_CODE( - constexpr auto BuildArch = TargetArch::Scalar; + constexpr auto BuildArch = TargetArch::Default; ) // DECLARE_DEFAULT_CODE DECLARE_SSE4_SPECIFIC_CODE( @@ -110,4 +110,4 @@ DECLARE_AVX512_SPECIFIC_CODE( bool IsArchSupported(TargetArch arch); -} // namespace DB::DynamicTarget \ No newline at end of file +} // namespace DB::DynamicTarget diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index e9a33283d5b..85b1b717d47 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -93,4 +93,4 @@ add_subdirectory(array) target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_array) add_subdirectory(DynamicTarget) -target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_dynamic_target) \ No newline at end of file +target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_dynamic_target) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 0e63b616558..497abc92508 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -29,14 +29,45 @@ struct NameEndsWith static constexpr auto name = "endsWith"; }; -using DynamicTarget::TargetArch; - DECLARE_MULTITARGET_CODE( template -class FunctionStartsEndsWithImpl { +class FunctionStartsEndsWith : public IFunction +{ public: - static void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) + static constexpr auto name = Name::name; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 2; + } + + bool useDefaultImplementationForConstants() const override + { + return true; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isStringOrFixedString(arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isStringOrFixedString(arguments[1])) + throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { const IColumn * haystack_column = block.getByPosition(arguments[0]).column.get(); const IColumn * needle_column = block.getByPosition(arguments[1]).column.get(); @@ -55,14 +86,14 @@ public: else if (const ColumnConst * haystack_const_fixed = checkAndGetColumnConst(haystack_column)) dispatch>(ConstSource(*haystack_const_fixed), needle_column, vec_res); else - throw Exception("Illegal combination of columns as arguments of function " "getName()", ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); block.getByPosition(result).column = std::move(col_res); } private: template - static void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray & res_data) + void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray & res_data) const { if (const ColumnString * needle = checkAndGetColumn(needle_column)) execute(haystack_source, StringSource(*needle), res_data); @@ -73,7 +104,7 @@ private: else if (const ColumnConst * needle_const_fixed = checkAndGetColumnConst(needle_column)) execute>(haystack_source, ConstSource(*needle_const_fixed), res_data); else - throw Exception("Illegal combination of columns as arguments of function " "getName()", ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } template @@ -112,55 +143,22 @@ private: ) // DECLARE_MULTITARGET_CODE template -class FunctionStartsEndsWith : public IFunction +class FunctionStartsEndsWith : public DynamicTarget::FunctionDynamicAdaptor { public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) + FunctionStartsEndsWith(const Context & context_) + : FunctionDynamicAdaptor>(context_) { - return std::make_shared(); + registerImplementation>(TargetArch::SSE4); + registerImplementation>(TargetArch::AVX); + registerImplementation>(TargetArch::AVX2); + registerImplementation>(TargetArch::AVX512); } - - FunctionStartsEndsWith() { - executor_.registerExecutor(std::nullopt, TargetSpecific::Default::FunctionStartsEndsWithImpl::executeImpl); - executor_.registerExecutor(TargetArch::SSE4, TargetSpecific::SSE4::FunctionStartsEndsWithImpl::executeImpl); - executor_.registerExecutor(TargetArch::AVX, TargetSpecific::AVX::FunctionStartsEndsWithImpl::executeImpl); - executor_.registerExecutor(TargetArch::AVX2, TargetSpecific::AVX2::FunctionStartsEndsWithImpl::executeImpl); - executor_.registerExecutor(TargetArch::AVX512, TargetSpecific::AVX512::FunctionStartsEndsWithImpl::executeImpl); - } - - String getName() const override + static FunctionPtr create(const Context & context) { - return name; + return std::make_shared>(context); \ } - - size_t getNumberOfArguments() const override - { - return 2; - } - - bool useDefaultImplementationForConstants() const override - { - return true; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isStringOrFixedString(arguments[0])) - throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!isStringOrFixedString(arguments[1])) - throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - executor_.execute(block, arguments, result, input_rows_count); - } -private: - DynamicTarget::SelectorExecutor executor_; }; -} +} \ No newline at end of file From e0a497d575c8c776510cce47e066156e848dac80 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Sun, 5 Apr 2020 15:14:59 +0200 Subject: [PATCH 004/211] Better interface --- dbms/src/Functions/DynamicTarget/Selector.h | 162 ++------------------ src/Functions/FunctionStartsEndsWith.h | 19 ++- 2 files changed, 27 insertions(+), 154 deletions(-) diff --git a/dbms/src/Functions/DynamicTarget/Selector.h b/dbms/src/Functions/DynamicTarget/Selector.h index 70c14dd2d11..a59022a6c28 100644 --- a/dbms/src/Functions/DynamicTarget/Selector.h +++ b/dbms/src/Functions/DynamicTarget/Selector.h @@ -133,161 +133,30 @@ struct PerformanceStatistics // PerformanceStatistics statistics; // }; -class FunctionDynamicAdaptor : public IFunction +template +class FunctionDynamicAdaptor : public DefaultFunction { public: - template - FunctionDynamicAdaptor(const Context & context_) : context(context_) + template + FunctionDynamicAdaptor(const Context & context_, Params ...params) + : DefaultFunction(params...) + , context(context_) { - registerImplementation(); - } - - virtual String getName() const override { - return impls.front()->getName(); + statistics.emplace_back(); } virtual void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { int id = statistics.select(); // TODO(dakovalkov): measure time and change statistics. - impls[id]->executeImpl(block, arguments, result, input_rows_count); - } - virtual void executeImplDryRun(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - impls.front()->executeImplDryRun(block, arguments, result, input_rows_count); - } - - virtual bool useDefaultImplementationForNulls() const override - { - return impls.front()->useDefaultImplementationForNulls(); - } - - virtual bool useDefaultImplementationForConstants() const override - { - return impls.front()->useDefaultImplementationForConstants(); - } - - virtual bool useDefaultImplementationForLowCardinalityColumns() const override - { - return impls.front()->useDefaultImplementationForLowCardinalityColumns(); - } - - virtual bool canBeExecutedOnLowCardinalityDictionary() const override - { - return impls.front()->canBeExecutedOnLowCardinalityDictionary(); - } - - virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const override - { - return impls.front()->getArgumentsThatAreAlwaysConstant(); - } - - virtual bool canBeExecutedOnDefaultArguments() const override - { - return impls.front()->canBeExecutedOnDefaultArguments(); - } - -#if USE_EMBEDDED_COMPILER - - virtual bool isCompilable() const override - { - return impls.front()->isCompilable(); - } - - virtual llvm::Value * compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const override - { - return impls.front()->compile(builder, std::move(values)); - } - -#endif - - /// Properties from IFunctionBase (see IFunction.h) - virtual bool isSuitableForConstantFolding() const override - { - return impls.front()->isSuitableForConstantFolding(); - } - virtual ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments) const override - { - return impls.front()->getResultIfAlwaysReturnsConstantAndHasArguments(block, arguments); - } - virtual bool isInjective(const Block & sample_block) override - { - return impls.front()->isInjective(sample_block); - } - virtual bool isDeterministic() const override - { - return impls.front()->isDeterministic(); - } - virtual bool isDeterministicInScopeOfQuery() const override - { - return impls.front()->isDeterministicInScopeOfQuery(); - } - virtual bool isStateful() const override - { - return impls.front()->isStateful(); - } - virtual bool hasInformationAboutMonotonicity() const override - { - return impls.front()->hasInformationAboutMonotonicity(); - } - - using Monotonicity = IFunctionBase::Monotonicity; - virtual Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override - { - return impls.front()->getMonotonicityForRange(type, left, right); - } - - virtual size_t getNumberOfArguments() const override { - return impls.front()->getNumberOfArguments(); - } - - virtual DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - return impls.front()->getReturnTypeImpl(arguments); - } - - virtual DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - return impls.front()->getReturnTypeImpl(arguments); - } - - virtual bool isVariadic() const override - { - return impls.front()->isVariadic(); - } - - virtual void checkNumberOfArgumentsIfVariadic(size_t number_of_arguments) const override - { - impls.front()->checkNumberOfArgumentsIfVariadic(number_of_arguments); - } - - virtual void getLambdaArgumentTypes(DataTypes & arguments) const override - { - impls.front()->getLambdaArgumentTypes(arguments); - } - - virtual ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t number_of_arguments) const override - { - return impls.front()->getArgumentsThatDontImplyNullableReturnType(number_of_arguments); + if (id == 0) { + DefaultFunction::executeImpl(block, arguments, result, input_rows_count); + } else { + impls[id - 1]->executeImpl(block, arguments, result, input_rows_count); + } } protected: - -#if USE_EMBEDDED_COMPILER - - virtual bool isCompilableImpl(const DataTypes & /* types */) const override - { - return false; - // return impls.front()->isCompilableImpl(types); - } - - virtual llvm::Value * compileImpl(llvm::IRBuilderBase & /* builder */, const DataTypes & /* types */, ValuePlaceholders /* ph */) const override - { - throw "safasf Error"; - // return impls.front()->compileImpl(builder, types, ph); - } - -#endif /* * Register implementation of the function. */ @@ -301,15 +170,16 @@ protected: private: const Context & context; - std::vector impls; + std::vector impls; // Alternative implementations. PerformanceStatistics statistics; }; +// TODO(dakovalkov): May be it's better to delete this macros and write every function explicitly for better readability. #define DECLARE_STANDART_TARGET_ADAPTOR(Function) \ -class Function : public FunctionDynamicAdaptor \ +class Function : public FunctionDynamicAdaptor \ { \ public: \ - Function(const Context & context) : FunctionDynamicAdaptor(context) \ + Function(const Context & context) : FunctionDynamicAdaptor(context) \ { \ registerImplementation(TargetArch::SSE4); \ registerImplementation(TargetArch::AVX); \ diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 497abc92508..e883dc3267a 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -143,22 +143,25 @@ private: ) // DECLARE_MULTITARGET_CODE template -class FunctionStartsEndsWith : public DynamicTarget::FunctionDynamicAdaptor +class FunctionStartsEndsWith + : public DynamicTarget::FunctionDynamicAdaptor> { public: - static constexpr auto name = Name::name; FunctionStartsEndsWith(const Context & context_) - : FunctionDynamicAdaptor>(context_) + : DynamicTarget::FunctionDynamicAdaptor>(context_) { - registerImplementation>(TargetArch::SSE4); - registerImplementation>(TargetArch::AVX); - registerImplementation>(TargetArch::AVX2); - registerImplementation>(TargetArch::AVX512); + registerImplementation>(DynamicTarget::TargetArch::SSE4); + registerImplementation>(DynamicTarget::TargetArch::AVX); + registerImplementation>(DynamicTarget::TargetArch::AVX2); + registerImplementation>(DynamicTarget::TargetArch::AVX512); } static FunctionPtr create(const Context & context) { - return std::make_shared>(context); \ + return std::make_shared>(context); } }; +// template +// using FunctionStartsEndsWith = TargetSpecific::Default::FunctionStartsEndsWith; + } \ No newline at end of file From 09bb9041ec09a4ac3d4ebd12820cbe06f9b3d64e Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Sun, 5 Apr 2020 21:39:12 +0200 Subject: [PATCH 005/211] Add descriptions, move to Functions/, rename, measure time, and so on... --- .../Functions/DynamicTarget/CMakeLists.txt | 11 - .../DynamicTarget/DynamicFunctionAdaptors.h | 263 ------------------ .../Selector.h => PerformanceAdaptors.h} | 133 +++++---- .../Target.cpp => TargetSpecific.cpp} | 6 +- .../Target.h => TargetSpecific.h} | 77 ++++- src/Functions/CMakeLists.txt | 3 - src/Functions/FunctionStartsEndsWith.h | 25 +- 7 files changed, 157 insertions(+), 361 deletions(-) delete mode 100644 dbms/src/Functions/DynamicTarget/CMakeLists.txt delete mode 100644 dbms/src/Functions/DynamicTarget/DynamicFunctionAdaptors.h rename dbms/src/Functions/{DynamicTarget/Selector.h => PerformanceAdaptors.h} (56%) rename dbms/src/Functions/{DynamicTarget/Target.cpp => TargetSpecific.cpp} (59%) rename dbms/src/Functions/{DynamicTarget/Target.h => TargetSpecific.h} (56%) diff --git a/dbms/src/Functions/DynamicTarget/CMakeLists.txt b/dbms/src/Functions/DynamicTarget/CMakeLists.txt deleted file mode 100644 index 154e47ab5f6..00000000000 --- a/dbms/src/Functions/DynamicTarget/CMakeLists.txt +++ /dev/null @@ -1,11 +0,0 @@ -include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) -add_headers_and_sources(clickhouse_functions_dynamic_target .) -add_library(clickhouse_functions_dynamic_target - ${clickhouse_functions_dynamic_target_sources} - ${clickhouse_functions_dynamic_target_headers}) -target_link_libraries(clickhouse_functions_dynamic_target PRIVATE dbms) - -if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL") - # Won't generate debug info for files with heavy template instantiation to achieve faster linking and lower size. - target_compile_options(clickhouse_functions_dynamic_target PRIVATE "-g0") -endif () diff --git a/dbms/src/Functions/DynamicTarget/DynamicFunctionAdaptors.h b/dbms/src/Functions/DynamicTarget/DynamicFunctionAdaptors.h deleted file mode 100644 index 123faa859e9..00000000000 --- a/dbms/src/Functions/DynamicTarget/DynamicFunctionAdaptors.h +++ /dev/null @@ -1,263 +0,0 @@ -#pragma once -#include - -namespace DB -{ - -/// Adaptors are implement user interfaces from IFunction.h via developer interfaces from IFunctionImpl.h -/// Typically, you don't need to change this classes. - -class ExecutableFunctionAdaptor final : public IExecutableFunction -{ -public: - explicit ExecutableFunctionAdaptor(ExecutableFunctionImplPtr impl_) : impl(std::move(impl_)) {} - - String getName() const final { return impl->getName(); } - - void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run) final; - - void createLowCardinalityResultCache(size_t cache_size) override; - -private: - ExecutableFunctionImplPtr impl; - - /// Cache is created by function createLowCardinalityResultCache() - ExecutableFunctionLowCardinalityResultCachePtr low_cardinality_result_cache; - - bool defaultImplementationForConstantArguments( - Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run); - - bool defaultImplementationForNulls( - Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run); - - void executeWithoutLowCardinalityColumns( - Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run); -}; - -class FunctionBaseAdaptor final : public IFunctionBase -{ -public: - explicit FunctionBaseAdaptor(FunctionBaseImplPtr impl_) : impl(std::move(impl_)) {} - - String getName() const final { return impl->getName(); } - - const DataTypes & getArgumentTypes() const final { return impl->getArgumentTypes(); } - const DataTypePtr & getReturnType() const final { return impl->getReturnType(); } - - ExecutableFunctionPtr prepare(const Block & sample_block, const ColumnNumbers & arguments, size_t result) const final - { - return std::make_shared(impl->prepare(sample_block, arguments, result)); - } - -#if USE_EMBEDDED_COMPILER - - bool isCompilable() const final { return impl->isCompilable(); } - - llvm::Value * compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const override - { - return impl->compile(builder, std::move(values)); - } - -#endif - - bool isStateful() const final { return impl->isStateful(); } - bool isSuitableForConstantFolding() const final { return impl->isSuitableForConstantFolding(); } - - ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments) const final - { - return impl->getResultIfAlwaysReturnsConstantAndHasArguments(block, arguments); - } - - bool isInjective(const Block & sample_block) final { return impl->isInjective(sample_block); } - bool isDeterministic() const final { return impl->isDeterministic(); } - bool isDeterministicInScopeOfQuery() const final { return impl->isDeterministicInScopeOfQuery(); } - bool hasInformationAboutMonotonicity() const final { return impl->hasInformationAboutMonotonicity(); } - - Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const final - { - return impl->getMonotonicityForRange(type, left, right); - } - - const IFunctionBaseImpl * getImpl() const { return impl.get(); } - -private: - FunctionBaseImplPtr impl; -}; - - -class FunctionOverloadResolverAdaptor final : public IFunctionOverloadResolver -{ -public: - explicit FunctionOverloadResolverAdaptor(FunctionOverloadResolverImplPtr impl_) : impl(std::move(impl_)) {} - - String getName() const final { return impl->getName(); } - - bool isDeterministic() const final { return impl->isDeterministic(); } - - bool isDeterministicInScopeOfQuery() const final { return impl->isDeterministicInScopeOfQuery(); } - - bool isStateful() const final { return impl->isStateful(); } - - bool isVariadic() const final { return impl->isVariadic(); } - - size_t getNumberOfArguments() const final { return impl->getNumberOfArguments(); } - - void checkNumberOfArguments(size_t number_of_arguments) const final; - - FunctionBaseImplPtr buildImpl(const ColumnsWithTypeAndName & arguments) const - { - return impl->build(arguments, getReturnType(arguments)); - } - - FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const final - { - return std::make_shared(buildImpl(arguments)); - } - - void getLambdaArgumentTypes(DataTypes & arguments) const final - { - checkNumberOfArguments(arguments.size()); - impl->getLambdaArgumentTypes(arguments); - } - - ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return impl->getArgumentsThatAreAlwaysConstant(); } - - ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t number_of_arguments) const final - { - return impl->getArgumentsThatDontImplyNullableReturnType(number_of_arguments); - } - -private: - FunctionOverloadResolverImplPtr impl; - - DataTypePtr getReturnTypeWithoutLowCardinality(const ColumnsWithTypeAndName & arguments) const; - DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const; -}; - - -/// Following classes are implement IExecutableFunctionImpl, IFunctionBaseImpl and IFunctionOverloadResolverImpl via IFunction. - -class DefaultExecutable final : public IExecutableFunctionImpl -{ -public: - explicit DefaultExecutable(std::shared_ptr function_) : function(std::move(function_)) {} - - String getName() const override { return function->getName(); } - -protected: - void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final - { - return function->executeImpl(block, arguments, result, input_rows_count); - } - void executeDryRun(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final - { - return function->executeImplDryRun(block, arguments, result, input_rows_count); - } - bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); } - bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); } - bool useDefaultImplementationForLowCardinalityColumns() const final { return function->useDefaultImplementationForLowCardinalityColumns(); } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return function->getArgumentsThatAreAlwaysConstant(); } - bool canBeExecutedOnDefaultArguments() const override { return function->canBeExecutedOnDefaultArguments(); } - -private: - std::shared_ptr function; -}; - -class DefaultFunction final : public IFunctionBaseImpl -{ -public: - DefaultFunction(std::shared_ptr function_, DataTypes arguments_, DataTypePtr return_type_) - : function(std::move(function_)), arguments(std::move(arguments_)), return_type(std::move(return_type_)) {} - - String getName() const override { return function->getName(); } - - const DataTypes & getArgumentTypes() const override { return arguments; } - const DataTypePtr & getReturnType() const override { return return_type; } - -#if USE_EMBEDDED_COMPILER - - bool isCompilable() const override { return function->isCompilable(arguments); } - - llvm::Value * compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const override { return function->compile(builder, arguments, std::move(values)); } - -#endif - - ExecutableFunctionImplPtr prepare(const Block & /*sample_block*/, const ColumnNumbers & /*arguments*/, size_t /*result*/) const override - { - return std::make_unique(function); - } - - bool isSuitableForConstantFolding() const override { return function->isSuitableForConstantFolding(); } - ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments_) const override - { - return function->getResultIfAlwaysReturnsConstantAndHasArguments(block, arguments_); - } - - bool isStateful() const override { return function->isStateful(); } - - bool isInjective(const Block & sample_block) override { return function->isInjective(sample_block); } - - bool isDeterministic() const override { return function->isDeterministic(); } - - bool isDeterministicInScopeOfQuery() const override { return function->isDeterministicInScopeOfQuery(); } - - bool hasInformationAboutMonotonicity() const override { return function->hasInformationAboutMonotonicity(); } - - using Monotonicity = IFunctionBase::Monotonicity; - Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override - { - return function->getMonotonicityForRange(type, left, right); - } -private: - std::shared_ptr function; - DataTypes arguments; - DataTypePtr return_type; -}; - -class DefaultOverloadResolver : public IFunctionOverloadResolverImpl -{ -public: - explicit DefaultOverloadResolver(std::shared_ptr function_) : function(std::move(function_)) {} - - void checkNumberOfArgumentsIfVariadic(size_t number_of_arguments) const override - { - return function->checkNumberOfArgumentsIfVariadic(number_of_arguments); - } - - bool isDeterministic() const override { return function->isDeterministic(); } - bool isDeterministicInScopeOfQuery() const override { return function->isDeterministicInScopeOfQuery(); } - - String getName() const override { return function->getName(); } - bool isStateful() const override { return function->isStateful(); } - bool isVariadic() const override { return function->isVariadic(); } - size_t getNumberOfArguments() const override { return function->getNumberOfArguments(); } - - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return function->getArgumentsThatAreAlwaysConstant(); } - ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t number_of_arguments) const override - { - return function->getArgumentsThatDontImplyNullableReturnType(number_of_arguments); - } - - DataTypePtr getReturnType(const DataTypes & arguments) const override { return function->getReturnTypeImpl(arguments); } - DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const override { return function->getReturnTypeImpl(arguments); } - - bool useDefaultImplementationForNulls() const override { return function->useDefaultImplementationForNulls(); } - bool useDefaultImplementationForLowCardinalityColumns() const override { return function->useDefaultImplementationForLowCardinalityColumns(); } - bool canBeExecutedOnLowCardinalityDictionary() const override { return function->canBeExecutedOnLowCardinalityDictionary(); } - - FunctionBaseImplPtr build(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; - return std::make_unique(function, data_types, return_type); - } - - void getLambdaArgumentTypes(DataTypes & arguments) const override { function->getLambdaArgumentTypes(arguments); } - -private: - std::shared_ptr function; -}; - - -} diff --git a/dbms/src/Functions/DynamicTarget/Selector.h b/dbms/src/Functions/PerformanceAdaptors.h similarity index 56% rename from dbms/src/Functions/DynamicTarget/Selector.h rename to dbms/src/Functions/PerformanceAdaptors.h index a59022a6c28..492a4791170 100644 --- a/dbms/src/Functions/DynamicTarget/Selector.h +++ b/dbms/src/Functions/PerformanceAdaptors.h @@ -1,21 +1,20 @@ #pragma once -#include "Target.h" - +#include #include +#include + #include -namespace DB::DynamicTarget +/// This file contains Adaptors which help to combine several implementations of the function. +/// Adaptors check that implementation can be executed on the current platform and choose +/// that one which works faster according to previous runs. + +namespace DB { -// TODO(dakovalkov): This is copied and pasted struct from LZ4_decompress_faster.h - -/** When decompressing uniform sequence of blocks (for example, blocks from one file), - * you can pass single PerformanceStatistics object to subsequent invocations of 'decompress' method. - * It will accumulate statistics and use it as a feedback to choose best specialization of algorithm at runtime. - * One PerformanceStatistics object cannot be used concurrently from different threads. - */ +// TODO(dakovalkov): This is copied and pasted struct from LZ4_decompress_faster.h with little changes. struct PerformanceStatistics { struct Element @@ -105,71 +104,97 @@ struct PerformanceStatistics PerformanceStatistics(ssize_t choose_method_) : choose_method(choose_method_) {} }; -// template -// class PerformanceExecutor -// { -// public: -// using Executor = std::function; -// // Should register all executors before execute -// void registerExecutor(Executor executor) -// { -// executors.emplace_back(std::move(executor)); -// } - -// // The performance of the execution is time / weight. -// // Weight is usualy the -// void execute(int weight, Params... params) -// { -// if (executors_.empty()) { -// throw "There are no realizations for current Arch"; -// } -// int impl = 0; -// // TODO: choose implementation. -// executors_[impl](params...); -// } - -// private: -// std::vector executors; -// PerformanceStatistics statistics; -// }; - +/// Combine several IExecutableFunctionImpl into one. +/// All the implementations should be equivalent. +/// Implementation to execute will be selected based on performance on previous runs. +/// DefaultFunction should be executable on every supported platform, while alternative implementations +/// could use extended set of instructions (AVX, NEON, etc). +/// It's convenient to inherit your func from this and register all alternative implementations in the constructor. template -class FunctionDynamicAdaptor : public DefaultFunction +class ExecutableFunctionPerformanceAdaptor : public DefaultFunction { public: template - FunctionDynamicAdaptor(const Context & context_, Params ...params) - : DefaultFunction(params...) - , context(context_) + ExecutableFunctionPerformanceAdaptor(Params ...params) : DefaultFunction(params...) + { + statistics.emplace_back(); + } + + virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + auto id = statistics.select(); + Stopwatch watch; + if (id == 0) { + DefaultFunction::execute(block, arguments, result, input_rows_count); + } else { + impls[id - 1]->execute(block, arguments, result, input_rows_count); + } + watch.stop(); + // TODO(dakovalkov): Calculate something more informative. + size_t rows_summary = 0; + for (auto i : arguments) { + rows_summary += block.getByPosition(i).column->size(); + } + if (rows_summary >= 1000) { + statistics.data[id].update(watch.elapsedSeconds(), rows_summary); + } + } + + // Register alternative implementation. + template + void registerImplementation(TargetArch arch, Params... params) { + if (arch == TargetArch::Default || IsArchSupported(arch)) { + impls.emplace_back(std::make_shared(params...)); + statistics.emplace_back(); + } + } + +private: + std::vector impls; // Alternative implementations. + PerformanceStatistics statistics; +}; + +// The same as ExecutableFunctionPerformanceAdaptor, but combine via IFunction interface. +template +class FunctionPerformanceAdaptor : public DefaultFunction +{ +public: + template + FunctionPerformanceAdaptor(Params ...params) : DefaultFunction(params...) { statistics.emplace_back(); } virtual void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { - int id = statistics.select(); - // TODO(dakovalkov): measure time and change statistics. + auto id = statistics.select(); + Stopwatch watch; if (id == 0) { DefaultFunction::executeImpl(block, arguments, result, input_rows_count); } else { impls[id - 1]->executeImpl(block, arguments, result, input_rows_count); } + watch.stop(); + // TODO(dakovalkov): Calculate something more informative. + size_t rows_summary = 0; + for (auto i : arguments) { + rows_summary += block.getByPosition(i).column->size(); + } + if (rows_summary >= 1000) { + statistics.data[id].update(watch.elapsedSeconds(), rows_summary); + } } -protected: - /* - * Register implementation of the function. - */ - template - void registerImplementation(TargetArch arch = TargetArch::Default) { + // Register alternative implementation. + template + void registerImplementation(TargetArch arch, Params... params) { if (arch == TargetArch::Default || IsArchSupported(arch)) { - impls.emplace_back(Function::create(context)); + impls.emplace_back(std::make_shared(params...)); statistics.emplace_back(); } } private: - const Context & context; std::vector impls; // Alternative implementations. PerformanceStatistics statistics; }; @@ -179,7 +204,7 @@ private: class Function : public FunctionDynamicAdaptor \ { \ public: \ - Function(const Context & context) : FunctionDynamicAdaptor(context) \ + Function(const Context &) : FunctionDynamicAdaptor() \ { \ registerImplementation(TargetArch::SSE4); \ registerImplementation(TargetArch::AVX); \ @@ -192,4 +217,4 @@ public: \ } \ } -} // namespace DB::DynamicTarget +} // namespace DB diff --git a/dbms/src/Functions/DynamicTarget/Target.cpp b/dbms/src/Functions/TargetSpecific.cpp similarity index 59% rename from dbms/src/Functions/DynamicTarget/Target.cpp rename to dbms/src/Functions/TargetSpecific.cpp index 54c41a1786c..837a6796bf2 100644 --- a/dbms/src/Functions/DynamicTarget/Target.cpp +++ b/dbms/src/Functions/TargetSpecific.cpp @@ -1,6 +1,6 @@ -#include "Target.h" +#include -namespace DB::DynamicTarget +namespace DB { bool IsArchSupported(TargetArch arch) @@ -9,4 +9,4 @@ bool IsArchSupported(TargetArch arch) return arch != TargetArch::AVX512; } -} // namespace DB::DynamicTarget +} // namespace DB diff --git a/dbms/src/Functions/DynamicTarget/Target.h b/dbms/src/Functions/TargetSpecific.h similarity index 56% rename from dbms/src/Functions/DynamicTarget/Target.h rename to dbms/src/Functions/TargetSpecific.h index e1772a11857..ddb9fbb74f8 100644 --- a/dbms/src/Functions/DynamicTarget/Target.h +++ b/dbms/src/Functions/TargetSpecific.h @@ -1,6 +1,60 @@ #pragma once -namespace DB::DynamicTarget +/// This file contains macros and helpers for writing platform-dependent code. +/// +/// Macroses DECLARE__SPECIFIC_CODE will wrap code inside them into the namespace TargetSpecific:: and enable +/// Arch-specific compile options. +/// Thus, it's allowed to call functions inside only after checking platform in runtime (see IsArchSupported() below) +/// For similarities there is a macros DECLARE_DEFAULT_CODE, which wraps code into the namespace TargetSpecific::Default +/// but dosn't specify any additional copile options. +/// +/// Example: +/// +/// DECLARE_DEFAULT_CODE ( +/// int func() { +/// return 1; +/// } +/// ) // DECLARE_DEFAULT_CODE +/// +/// DECLARE_AVX2_SPECIFIC_CODE ( +/// int func() { +/// return 2; +/// } +/// ) // DECLARE_DEFAULT_CODE +/// +/// int func() { +/// if (IsArchSupported(TargetArch::AVX2)) +/// return TargetSpecifc::AVX2::func(); +/// return TargetSpecifc::Default::func(); +/// } +/// +/// Sometimes code may benefit from compiling with different options. +/// For these purposes use DECLARE_MULTITARGET_CODE macros. It will create several copies of the code and +/// compile it with different options. These copies are available via TargetSpecifc namespaces described above. +/// +/// Inside every TargetSpecific namespace there is a constexpr variable BuildArch, which TODO +/// +/// Example: +/// +/// DECLARE_MULTITARGET_CODE( +/// int func(int size, ...) { +/// int iteration_size = 1; +/// if constexpr (BuildArch == TargetArch::SSE4) +/// iteration_size = 2 +/// else if constexpr (BuildArch == TargetArch::AVX || BuildArch == TargetArch::AVX2) +/// iteration_size = 4; +/// else if constexpr (BuildArch == TargetArch::AVX512) +/// iteration_size = 8; +/// for (int i = 0; i < size; i += iteration_size) +/// ... +/// } +/// ) // DECLARE_MULTITARGET_CODE +/// +/// // All 5 versions of func are available here. Use runtime detection to choose one. +/// +/// If you want to write IFunction or IExecutableFuncionImpl with runtime dispatching, see PerformanceAdaptors.h. + +namespace DB { enum class TargetArch : int { @@ -11,11 +65,14 @@ enum class TargetArch : int { AVX512, }; +// Runtime detection. +bool IsArchSupported(TargetArch arch); + #if defined(__GNUC__) // TODO: There are lots of different AVX512 :( # define BEGIN_AVX512_SPECIFIC_CODE \ _Pragma("GCC push_options") \ - _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2,tune=native\")") + _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2,avx512f,tune=native\")") # define BEGIN_AVX2_SPECIFIC_CODE \ _Pragma("GCC push_options") \ _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2,tune=native\")") @@ -40,19 +97,19 @@ enum class TargetArch : int { # define END_TARGET_SPECIFIC_CODE \ _Pragma("clang attribute pop") #else -# error "Only CLANG and GCC compilers are supported" +# error "Only CLANG and GCC compilers are supported for vectorized code generation" #endif #define DECLARE_DEFAULT_CODE(...) \ namespace TargetSpecific::Default { \ - using namespace DB::DynamicTarget::TargetSpecific::Default; \ + using namespace DB::TargetSpecific::Default; \ __VA_ARGS__ \ } #define DECLARE_SSE4_SPECIFIC_CODE(...) \ BEGIN_SSE4_SPECIFIC_CODE \ namespace TargetSpecific::SSE4 { \ - using namespace DB::DynamicTarget::TargetSpecific::SSE4; \ + using namespace DB::TargetSpecific::SSE4; \ __VA_ARGS__ \ } \ END_TARGET_SPECIFIC_CODE @@ -60,7 +117,7 @@ END_TARGET_SPECIFIC_CODE #define DECLARE_AVX_SPECIFIC_CODE(...) \ BEGIN_AVX_SPECIFIC_CODE \ namespace TargetSpecific::AVX { \ - using namespace DB::DynamicTarget::TargetSpecific::AVX; \ + using namespace DB::TargetSpecific::AVX; \ __VA_ARGS__ \ } \ END_TARGET_SPECIFIC_CODE @@ -68,7 +125,7 @@ END_TARGET_SPECIFIC_CODE #define DECLARE_AVX2_SPECIFIC_CODE(...) \ BEGIN_AVX2_SPECIFIC_CODE \ namespace TargetSpecific::AVX2 { \ - using namespace DB::DynamicTarget::TargetSpecific::AVX2; \ + using namespace DB::TargetSpecific::AVX2; \ __VA_ARGS__ \ } \ END_TARGET_SPECIFIC_CODE @@ -76,7 +133,7 @@ END_TARGET_SPECIFIC_CODE #define DECLARE_AVX512_SPECIFIC_CODE(...) \ BEGIN_AVX512_SPECIFIC_CODE \ namespace TargetSpecific::AVX512 { \ - using namespace DB::DynamicTarget::TargetSpecific::AVX512; \ + using namespace DB::TargetSpecific::AVX512; \ __VA_ARGS__ \ } \ END_TARGET_SPECIFIC_CODE @@ -108,6 +165,4 @@ DECLARE_AVX512_SPECIFIC_CODE( constexpr auto BuildArch = TargetArch::AVX512; ) // DECLARE_AVX512_SPECIFIC_CODE -bool IsArchSupported(TargetArch arch); - -} // namespace DB::DynamicTarget +} // namespace DB diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 85b1b717d47..069a63aa9e1 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -91,6 +91,3 @@ target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_url) add_subdirectory(array) target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_array) - -add_subdirectory(DynamicTarget) -target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_dynamic_target) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index e883dc3267a..1306f3572f7 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -6,8 +6,8 @@ #include #include -#include -#include +#include +#include namespace DB { @@ -36,10 +36,6 @@ class FunctionStartsEndsWith : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } String getName() const override { @@ -144,16 +140,16 @@ private: template class FunctionStartsEndsWith - : public DynamicTarget::FunctionDynamicAdaptor> + : public FunctionPerformanceAdaptor> { public: - FunctionStartsEndsWith(const Context & context_) - : DynamicTarget::FunctionDynamicAdaptor>(context_) + FunctionStartsEndsWith(const Context &) + : FunctionPerformanceAdaptor>() { - registerImplementation>(DynamicTarget::TargetArch::SSE4); - registerImplementation>(DynamicTarget::TargetArch::AVX); - registerImplementation>(DynamicTarget::TargetArch::AVX2); - registerImplementation>(DynamicTarget::TargetArch::AVX512); + registerImplementation>(TargetArch::SSE4); + registerImplementation>(TargetArch::AVX); + registerImplementation>(TargetArch::AVX2); + registerImplementation>(TargetArch::AVX512); } static FunctionPtr create(const Context & context) { @@ -161,7 +157,4 @@ public: } }; -// template -// using FunctionStartsEndsWith = TargetSpecific::Default::FunctionStartsEndsWith; - } \ No newline at end of file From 9d875d8adb796fc6b42ae69ea05541dce7360d75 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Mon, 6 Apr 2020 09:31:26 +0200 Subject: [PATCH 006/211] Arch detection --- dbms/src/Functions/TargetSpecific.cpp | 29 +++++++++++++++++++++++++-- dbms/src/Functions/TargetSpecific.h | 10 ++++----- 2 files changed, 32 insertions(+), 7 deletions(-) diff --git a/dbms/src/Functions/TargetSpecific.cpp b/dbms/src/Functions/TargetSpecific.cpp index 837a6796bf2..18040111cca 100644 --- a/dbms/src/Functions/TargetSpecific.cpp +++ b/dbms/src/Functions/TargetSpecific.cpp @@ -1,12 +1,37 @@ #include +#if defined(__GNUC__) || defined(__clang__) +# include +#else +# error "Only CLANG and GCC compilers are supported for dynamic dispatch" +#endif + namespace DB { +int GetSupportedArches() { + unsigned int eax, ebx, ecx, edx; + if (!__get_cpuid(1, &eax, &ebx, &ecx, &edx)) { + return 0; + } + int res = 0; + if (ecx & bit_SSE4_2) + res |= static_cast(TargetArch::SSE4); + if ((ecx & bit_OSXSAVE) && (ecx & bit_AVX)) { + // TODO(dakovalkov): check XGETBV. + res |= static_cast(TargetArch::AVX); + if (__get_cpuid(7, &eax, &ebx, &ecx, &edx) && (ebx & bit_AVX2)) { + res |= static_cast(TargetArch::AVX2); + } + // TODO(dakovalkov): check AVX512 support. + } + return res; +} + bool IsArchSupported(TargetArch arch) { - // TODO(dakovalkov): use cpuid - return arch != TargetArch::AVX512; + static int arches = GetSupportedArches(); + return arch == TargetArch::Default || (arches & static_cast(arch)); } } // namespace DB diff --git a/dbms/src/Functions/TargetSpecific.h b/dbms/src/Functions/TargetSpecific.h index ddb9fbb74f8..c5cd78fe03c 100644 --- a/dbms/src/Functions/TargetSpecific.h +++ b/dbms/src/Functions/TargetSpecific.h @@ -58,11 +58,11 @@ namespace DB { enum class TargetArch : int { - Default, // Without any additional compiler options. - SSE4, - AVX, - AVX2, - AVX512, + Default = 0, // Without any additional compiler options. + SSE4 = (1 << 0), + AVX = (1 << 1), + AVX2 = (1 << 2), + AVX512 = (1 << 3), }; // Runtime detection. From 9026187c2c17668c3a932271cbcd782e07769d83 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Mon, 6 Apr 2020 09:44:54 +0200 Subject: [PATCH 007/211] Cosmetics --- src/Functions/FunctionStartsEndsWith.h | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 1306f3572f7..730f0b9efbb 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -1,14 +1,13 @@ -#include #include #include #include +#include +#include +#include #include #include #include -#include -#include - namespace DB { @@ -157,4 +156,4 @@ public: } }; -} \ No newline at end of file +} From f07f9188ddd9406aa5a1d95596f95c3bb8318fe8 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Mon, 13 Apr 2020 11:25:53 +0200 Subject: [PATCH 008/211] Fix unsuccessful rebase --- {dbms/src => src}/Functions/PerformanceAdaptors.h | 2 +- {dbms/src => src}/Functions/TargetSpecific.cpp | 0 {dbms/src => src}/Functions/TargetSpecific.h | 0 3 files changed, 1 insertion(+), 1 deletion(-) rename {dbms/src => src}/Functions/PerformanceAdaptors.h (99%) rename {dbms/src => src}/Functions/TargetSpecific.cpp (100%) rename {dbms/src => src}/Functions/TargetSpecific.h (100%) diff --git a/dbms/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h similarity index 99% rename from dbms/src/Functions/PerformanceAdaptors.h rename to src/Functions/PerformanceAdaptors.h index 492a4791170..12f4b84dab9 100644 --- a/dbms/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -215,6 +215,6 @@ public: \ { \ return std::make_shared(context); \ } \ -} +}; } // namespace DB diff --git a/dbms/src/Functions/TargetSpecific.cpp b/src/Functions/TargetSpecific.cpp similarity index 100% rename from dbms/src/Functions/TargetSpecific.cpp rename to src/Functions/TargetSpecific.cpp diff --git a/dbms/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h similarity index 100% rename from dbms/src/Functions/TargetSpecific.h rename to src/Functions/TargetSpecific.h From acbd3b3a7088d1665063024f16ab73f5c44ffa9b Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Mon, 13 Apr 2020 11:26:56 +0200 Subject: [PATCH 009/211] Compile rand function with AVX --- src/Functions/FunctionsRandom.cpp | 13 ++++--------- src/Functions/FunctionsRandom.h | 26 +++++++++++++++++++++++--- src/Functions/generateUUIDv4.cpp | 3 ++- src/Functions/randConstant.cpp | 3 ++- tests/performance/rand.xml | 23 +++++++++++++++++++++++ 5 files changed, 54 insertions(+), 14 deletions(-) create mode 100644 tests/performance/rand.xml diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 19b2f08cdba..7506b118d5f 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -8,21 +8,15 @@ namespace DB { +DECLARE_MULTITARGET_CODE( + namespace { - /// NOTE Probably - /// http://www.pcg-random.org/ - /// or http://www.math.sci.hiroshima-u.ac.jp/~m-mat/MT/SFMT/ - /// or http://docs.yeppp.info/c/group__yep_random___w_e_l_l1024a.html - /// could go better. - struct LinearCongruentialGenerator { - /// Constants from `man lrand48_r`. static constexpr UInt64 a = 0x5DEECE66D; static constexpr UInt64 c = 0xB; - /// And this is from `head -c8 /dev/urandom | xxd -p` UInt64 current = 0x09826f4a081cee35ULL; void seed(UInt64 value) @@ -63,7 +57,8 @@ void RandImpl::execute(char * output, size_t size) unalignedStore(output + 12, generator3.next()); } - /// It is guaranteed (by PaddedPODArray) that we can overwrite up to 15 bytes after end. } +) //DECLARE_MULTITARGET_CODE + } diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 1ac6d24a356..f2fab585a47 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -5,7 +5,10 @@ #include #include - +#include +#include +// #include "TargetSpecific.h" +// #include "PerformanceAdaptors.h" namespace DB { @@ -34,9 +37,10 @@ namespace ErrorCodes * This means that the timer must be of sufficient resolution to give different values to each block. */ +DECLARE_MULTITARGET_CODE( + struct RandImpl { - /// Fill memory with random data. The memory region must be 15-bytes padded. static void execute(char * output, size_t size); }; @@ -46,7 +50,6 @@ class FunctionRandom : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } String getName() const override { @@ -83,4 +86,21 @@ public: } }; +) // DECLARE_MULTITARGET_CODE + +template +class FunctionRandom : public FunctionPerformanceAdaptor> +{ +public: + FunctionRandom() { + registerImplementation>(TargetArch::SSE4); + registerImplementation>(TargetArch::AVX); + registerImplementation>(TargetArch::AVX2); + registerImplementation>(TargetArch::AVX512); + } + static FunctionPtr create(const Context &) { + return std::make_shared>(); + } +}; + } diff --git a/src/Functions/generateUUIDv4.cpp b/src/Functions/generateUUIDv4.cpp index 39013519d2f..d543226ba5c 100644 --- a/src/Functions/generateUUIDv4.cpp +++ b/src/Functions/generateUUIDv4.cpp @@ -32,7 +32,8 @@ public: size_t size = input_rows_count; vec_to.resize(size); - RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(UInt128)); + // TODO(dakovalkov): rewrite this workaround + TargetSpecific::Default::RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(UInt128)); for (UInt128 & uuid: vec_to) { diff --git a/src/Functions/randConstant.cpp b/src/Functions/randConstant.cpp index bad4b199ee2..3eba5abf10d 100644 --- a/src/Functions/randConstant.cpp +++ b/src/Functions/randConstant.cpp @@ -99,7 +99,8 @@ public: argument_types.emplace_back(arguments.back().type); typename ColumnVector::Container vec_to(1); - RandImpl::execute(reinterpret_cast(vec_to.data()), sizeof(ToType)); + // TODO(dakovalkov): Rewrite this workaround + TargetSpecific::Default::RandImpl::execute(reinterpret_cast(vec_to.data()), sizeof(ToType)); ToType value = vec_to[0]; return std::make_unique>(value, argument_types, return_type); diff --git a/tests/performance/rand.xml b/tests/performance/rand.xml new file mode 100644 index 00000000000..6f73c2b1f73 --- /dev/null +++ b/tests/performance/rand.xml @@ -0,0 +1,23 @@ + + + + 20000 + + + 40000 + + + + + + + table + + numbers(100000000) + + + + + SELECT rand() FROM {table} + SELECT rand64() FROM {table} + From 9674482a6c8f433b120a43c0ce52c5f49dfd8768 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Tue, 14 Apr 2020 17:46:53 +0200 Subject: [PATCH 010/211] clang fixes --- .gitmodules | 3 +++ contrib/SIMDxorshift | 1 + src/Functions/TargetSpecific.cpp | 2 +- src/Functions/TargetSpecific.h | 26 +++++++++++++------------- 4 files changed, 18 insertions(+), 14 deletions(-) create mode 160000 contrib/SIMDxorshift diff --git a/.gitmodules b/.gitmodules index 7f5d1307a6e..c14fef40457 100644 --- a/.gitmodules +++ b/.gitmodules @@ -160,3 +160,6 @@ [submodule "contrib/fmtlib"] path = contrib/fmtlib url = https://github.com/fmtlib/fmt.git +[submodule "contrib/SIMDxorshift"] + path = contrib/SIMDxorshift + url = https://github.com/lemire/SIMDxorshift diff --git a/contrib/SIMDxorshift b/contrib/SIMDxorshift new file mode 160000 index 00000000000..270eb8936c9 --- /dev/null +++ b/contrib/SIMDxorshift @@ -0,0 +1 @@ +Subproject commit 270eb8936c9b4bd038c39f1783a8eba6b8f15b09 diff --git a/src/Functions/TargetSpecific.cpp b/src/Functions/TargetSpecific.cpp index 18040111cca..f22a586c333 100644 --- a/src/Functions/TargetSpecific.cpp +++ b/src/Functions/TargetSpecific.cpp @@ -1,6 +1,6 @@ #include -#if defined(__GNUC__) || defined(__clang__) +#if defined(__GNUC__) # include #else # error "Only CLANG and GCC compilers are supported for dynamic dispatch" diff --git a/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h index c5cd78fe03c..e5818632843 100644 --- a/src/Functions/TargetSpecific.h +++ b/src/Functions/TargetSpecific.h @@ -68,7 +68,19 @@ enum class TargetArch : int { // Runtime detection. bool IsArchSupported(TargetArch arch); -#if defined(__GNUC__) +#if defined(__clang__) +// TODO: There are lots of different AVX512 :( +# define BEGIN_AVX512_SPECIFIC_CODE \ + _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2\"))))") +# define BEGIN_AVX2_SPECIFIC_CODE \ + _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2\"))))") +# define BEGIN_AVX_SPECIFIC_CODE \ + _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx\"))))") +# define BEGIN_SSE4_SPECIFIC_CODE \ + _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx\"))))") +# define END_TARGET_SPECIFIC_CODE \ + _Pragma("clang attribute pop") +#elif defined(__GNUC__) // TODO: There are lots of different AVX512 :( # define BEGIN_AVX512_SPECIFIC_CODE \ _Pragma("GCC push_options") \ @@ -84,18 +96,6 @@ bool IsArchSupported(TargetArch arch); _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,tune=native\")") # define END_TARGET_SPECIFIC_CODE \ _Pragma("GCC pop_options") -#elif defined(__clang__) -// TODO: There are lots of different AVX512 :( -# define BEGIN_AVX512_SPECIFIC_CODE \ - _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2\"))))") -# define BEGIN_AVX2_SPECIFIC_CODE \ - _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2\"))))") -# define BEGIN_AVX_SPECIFIC_CODE \ - _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx\"))))") -# define BEGIN_SSE4_SPECIFIC_CODE \ - _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx\"))))") -# define END_TARGET_SPECIFIC_CODE \ - _Pragma("clang attribute pop") #else # error "Only CLANG and GCC compilers are supported for vectorized code generation" #endif From 808bb14c5cf1b22234352d5dc9992df16c49d30c Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Tue, 14 Apr 2020 17:48:33 +0200 Subject: [PATCH 011/211] Add xorshift-rand from lemire for comparing performance --- contrib/CMakeLists.txt | 1 + contrib/SIMDxorshift-cmake/CMakeLists.txt | 12 ++++ src/Functions/CMakeLists.txt | 4 ++ src/Functions/FunctionsRandom.cpp | 1 - src/Functions/FunctionsRandom.h | 1 - src/Functions/SIMDxorshift.cpp | 40 +++++++++++ src/Functions/SIMDxorshift.h | 84 +++++++++++++++++++++++ src/Functions/registerFunctionsRandom.cpp | 2 + tests/performance/rand.xml | 1 + 9 files changed, 144 insertions(+), 2 deletions(-) create mode 100644 contrib/SIMDxorshift-cmake/CMakeLists.txt create mode 100644 src/Functions/SIMDxorshift.cpp create mode 100644 src/Functions/SIMDxorshift.h diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index d122188ad0b..344a06f29b7 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -27,6 +27,7 @@ add_subdirectory (murmurhash) add_subdirectory (replxx-cmake) add_subdirectory (ryu-cmake) add_subdirectory (unixodbc-cmake) +add_subdirectory (SIMDxorshift-cmake) add_subdirectory (poco-cmake) diff --git a/contrib/SIMDxorshift-cmake/CMakeLists.txt b/contrib/SIMDxorshift-cmake/CMakeLists.txt new file mode 100644 index 00000000000..573173ff1b4 --- /dev/null +++ b/contrib/SIMDxorshift-cmake/CMakeLists.txt @@ -0,0 +1,12 @@ +set(SIMDXORSHIFT_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/SIMDxorshift/include") +set(SIMDXORSHIFT_SRC_DIR "${SIMDXORSHIFT_INCLUDE_DIR}/../src") +set(SIMDXORSHIFT_SRC + ${SIMDXORSHIFT_SRC_DIR}/xorshift128plus.c + ${SIMDXORSHIFT_SRC_DIR}/simdxorshift128plus.c +) + +set(SIMDXORSHIFT_LIBRARY "simdxorshift") + +add_library(${SIMDXORSHIFT_LIBRARY} ${SIMDXORSHIFT_SRC}) +target_include_directories(${SIMDXORSHIFT_LIBRARY} PUBLIC "${SIMDXORSHIFT_INCLUDE_DIR}") +target_compile_options(${SIMDXORSHIFT_LIBRARY} PRIVATE -mavx2) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 069a63aa9e1..451dfe97a03 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -83,6 +83,10 @@ if(USE_RAPIDJSON) target_include_directories(clickhouse_functions SYSTEM PRIVATE ${RAPIDJSON_INCLUDE_DIR}) endif() + +target_link_libraries(clickhouse_functions PUBLIC "simdxorshift") +message(STATUS "Using SIMDXORSHIFT ${SIMDXORSHIFT_LIBRARY}") + add_subdirectory(GatherUtils) target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_gatherutils) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 7506b118d5f..1f91c54c598 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -4,7 +4,6 @@ #include #include - namespace DB { diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index f2fab585a47..c6bcd9cb1ae 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -44,7 +44,6 @@ struct RandImpl static void execute(char * output, size_t size); }; - template class FunctionRandom : public IFunction { diff --git a/src/Functions/SIMDxorshift.cpp b/src/Functions/SIMDxorshift.cpp new file mode 100644 index 00000000000..6cad047f6da --- /dev/null +++ b/src/Functions/SIMDxorshift.cpp @@ -0,0 +1,40 @@ +#include +#include +#include +#include +#include + +#include + +#include + +extern "C" { +#include +} + +namespace DB +{ + +BEGIN_AVX_SPECIFIC_CODE + +void RandImplXorshift::execute(char * output, size_t size) +{ + avx_xorshift128plus_key_t mykey; + avx_xorshift128plus_init(324, 4444, &mykey); + // TODO(set last 16 bytes) + for (auto * end = output + size - 16; output < end; output += 32) { + unalignedStore<__m256i>(output, avx_xorshift128plus(&mykey)); + } +} + +struct NameRandXorshift { static constexpr auto name = "randxorshift"; }; +using FunctionRandXorshift = FunctionRandomXorshift; + +void registerFunctionRandXorshift(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +END_TARGET_SPECIFIC_CODE + +} diff --git a/src/Functions/SIMDxorshift.h b/src/Functions/SIMDxorshift.h new file mode 100644 index 00000000000..e46943f695a --- /dev/null +++ b/src/Functions/SIMDxorshift.h @@ -0,0 +1,84 @@ +#pragma once + +#include +#include +#include +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +struct RandImplXorshift +{ + static void execute(char * output, size_t size); +}; + +template +class FunctionRandomXorshift : public IFunction +{ +public: + static constexpr auto name = Name::name; + + static FunctionPtr create(const Context &) { + return std::make_shared>(); + } + + String getName() const override + { + return name; + } + + bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const override { return false; } + bool useDefaultImplementationForNulls() const override { return false; } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() > 1) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 0 or 1.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return std::make_shared>(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + auto col_to = ColumnVector::create(); + typename ColumnVector::Container & vec_to = col_to->getData(); + + size_t size = input_rows_count; + vec_to.resize(size); + RandImplXorshift::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(ToType)); + + block.getByPosition(result).column = std::move(col_to); + } +}; + +// template +// class FunctionRandom : public FunctionPerformanceAdaptor> +// { +// public: +// FunctionRandom() { +// registerImplementation>(TargetArch::SSE4); +// registerImplementation>(TargetArch::AVX); +// registerImplementation>(TargetArch::AVX2); +// registerImplementation>(TargetArch::AVX512); +// } +// static FunctionPtr create(const Context &) { +// return std::make_shared>(); +// } +// }; + +} diff --git a/src/Functions/registerFunctionsRandom.cpp b/src/Functions/registerFunctionsRandom.cpp index 3638474c4fe..422ec91f025 100644 --- a/src/Functions/registerFunctionsRandom.cpp +++ b/src/Functions/registerFunctionsRandom.cpp @@ -10,6 +10,7 @@ void registerFunctionRandomPrintableASCII(FunctionFactory & factory); void registerFunctionRandomString(FunctionFactory & factory); void registerFunctionRandomFixedString(FunctionFactory & factory); void registerFunctionRandomStringUTF8(FunctionFactory & factory); +void registerFunctionRandXorshift(FunctionFactory & factory); void registerFunctionsRandom(FunctionFactory & factory) { @@ -21,6 +22,7 @@ void registerFunctionsRandom(FunctionFactory & factory) registerFunctionRandomString(factory); registerFunctionRandomFixedString(factory); registerFunctionRandomStringUTF8(factory); + registerFunctionRandXorshift(factory); } } diff --git a/tests/performance/rand.xml b/tests/performance/rand.xml index 6f73c2b1f73..32ec38ddb4e 100644 --- a/tests/performance/rand.xml +++ b/tests/performance/rand.xml @@ -20,4 +20,5 @@ SELECT rand() FROM {table} SELECT rand64() FROM {table} + SELECT randxorshift() FROM {table} From 0afa67208218eff5c4004b2d9084eb3b4266bcca Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Fri, 15 May 2020 10:40:27 +0200 Subject: [PATCH 012/211] make randxorshift great again --- src/Functions/FunctionsRandom.h | 21 ++++----- src/Functions/SIMDxorshift.cpp | 76 ++++++++++++++++++++++++++++----- src/Functions/SIMDxorshift.h | 64 +++++---------------------- 3 files changed, 88 insertions(+), 73 deletions(-) diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index c6bcd9cb1ae..990c3a5f466 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -44,8 +44,10 @@ struct RandImpl static void execute(char * output, size_t size); }; -template -class FunctionRandom : public IFunction +) // DECLARE_MULTITARGET_CODE + +template +class FunctionRandomImpl : public IFunction { public: static constexpr auto name = Name::name; @@ -85,21 +87,20 @@ public: } }; -) // DECLARE_MULTITARGET_CODE - template -class FunctionRandom : public FunctionPerformanceAdaptor> +class FunctionRandom : public FunctionPerformanceAdaptor> { public: FunctionRandom() { - registerImplementation>(TargetArch::SSE4); - registerImplementation>(TargetArch::AVX); - registerImplementation>(TargetArch::AVX2); - registerImplementation>(TargetArch::AVX512); + registerImplementation>(TargetArch::SSE4); + registerImplementation>(TargetArch::AVX); + registerImplementation>(TargetArch::AVX2); + registerImplementation>(TargetArch::AVX512); } + static FunctionPtr create(const Context &) { return std::make_shared>(); } }; -} +} // namespace DB diff --git a/src/Functions/SIMDxorshift.cpp b/src/Functions/SIMDxorshift.cpp index 6cad047f6da..739077b5480 100644 --- a/src/Functions/SIMDxorshift.cpp +++ b/src/Functions/SIMDxorshift.cpp @@ -8,33 +8,89 @@ #include -extern "C" { +extern "C" +{ +#include #include } namespace DB { -BEGIN_AVX_SPECIFIC_CODE +DECLARE_DEFAULT_CODE( -void RandImplXorshift::execute(char * output, size_t size) +void RandXorshiftImpl::execute(char * output, size_t size) { - avx_xorshift128plus_key_t mykey; - avx_xorshift128plus_init(324, 4444, &mykey); - // TODO(set last 16 bytes) - for (auto * end = output + size - 16; output < end; output += 32) { - unalignedStore<__m256i>(output, avx_xorshift128plus(&mykey)); + char * end = output + size; + + xorshift128plus_key_s mykey; + + xorshift128plus_init(0xe9ef384566799595ULL ^ reinterpret_cast(output), + 0xa321e1523f4f88c7ULL ^ reinterpret_cast(output), + &mykey); + + const int bytes_per_write = 8; + const intptr_t mask = bytes_per_write - 1; + + // Process head to make output aligned. + unalignedStore(output, xorshift128plus(&mykey)); + output = reinterpret_cast((reinterpret_cast(output) | mask) + 1); + + while (end - output > 0) { + *reinterpret_cast(output) = xorshift128plus(&mykey); + output += bytes_per_write; } } +) // DECLARE_DEFAULT_CODE + +DECLARE_AVX2_SPECIFIC_CODE( + +void RandXorshiftImpl::execute(char * output, size_t size) +{ + char * end = output + size; + + avx_xorshift128plus_key_t mykey; + avx_xorshift128plus_init(0xe9ef384566799595ULL ^ reinterpret_cast(output), + 0xa321e1523f4f88c7ULL ^ reinterpret_cast(output), + &mykey); + + const int safe_overwrite = 16; // How many bytes we can write behind the end. + const int bytes_per_write = 32; + const intptr_t mask = bytes_per_write - 1; + + if (size + safe_overwrite <= bytes_per_write) { + _mm_storeu_si128(reinterpret_cast<__m128i*>(output), + _mm256_extracti128_si256(avx_xorshift128plus(&mykey), 0)); + return; + } + + // Process head to make output aligned. + _mm256_storeu_si256(reinterpret_cast<__m256i*>(output), avx_xorshift128plus(&mykey)); + output = reinterpret_cast((reinterpret_cast(output) | mask) + 1); + + while ((end - output) + safe_overwrite >= bytes_per_write) { + _mm256_store_si256(reinterpret_cast<__m256i*>(output), avx_xorshift128plus(&mykey)); + output += bytes_per_write; + } + + // Process tail. + if ((end - output) > 0) { + _mm_store_si128(reinterpret_cast<__m128i*>(output), + _mm256_extracti128_si256(avx_xorshift128plus(&mykey), 0)); + } +} + +) // DECLARE_AVX2_SPECIFIC_CODE + struct NameRandXorshift { static constexpr auto name = "randxorshift"; }; using FunctionRandXorshift = FunctionRandomXorshift; +struct NameRandXorshift64 { static constexpr auto name = "randxorshift64"; }; +using FunctionRandXorshift64 = FunctionRandomXorshift; void registerFunctionRandXorshift(FunctionFactory & factory) { factory.registerFunction(); } -END_TARGET_SPECIFIC_CODE - } diff --git a/src/Functions/SIMDxorshift.h b/src/Functions/SIMDxorshift.h index e46943f695a..46732c4d876 100644 --- a/src/Functions/SIMDxorshift.h +++ b/src/Functions/SIMDxorshift.h @@ -5,6 +5,7 @@ #include #include +#include #include #include @@ -16,69 +17,26 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -struct RandImplXorshift +DECLARE_MULTITARGET_CODE( + +struct RandXorshiftImpl { static void execute(char * output, size_t size); }; +) // DECLARE_MULTITARGET_CODE + template -class FunctionRandomXorshift : public IFunction +class FunctionRandomXorshift : public FunctionPerformanceAdaptor> { public: - static constexpr auto name = Name::name; + FunctionRandomXorshift() { + registerImplementation>(TargetArch::AVX2); + } static FunctionPtr create(const Context &) { return std::make_shared>(); } - - String getName() const override - { - return name; - } - - bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return false; } - bool useDefaultImplementationForNulls() const override { return false; } - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (arguments.size() > 1) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be 0 or 1.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - return std::make_shared>(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - auto col_to = ColumnVector::create(); - typename ColumnVector::Container & vec_to = col_to->getData(); - - size_t size = input_rows_count; - vec_to.resize(size); - RandImplXorshift::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(ToType)); - - block.getByPosition(result).column = std::move(col_to); - } }; -// template -// class FunctionRandom : public FunctionPerformanceAdaptor> -// { -// public: -// FunctionRandom() { -// registerImplementation>(TargetArch::SSE4); -// registerImplementation>(TargetArch::AVX); -// registerImplementation>(TargetArch::AVX2); -// registerImplementation>(TargetArch::AVX512); -// } -// static FunctionPtr create(const Context &) { -// return std::make_shared>(); -// } -// }; - -} +} // namespace DB From 089b3ca0085b24716c422ff16ed8342ae340d935 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Fri, 15 May 2020 12:10:34 +0200 Subject: [PATCH 013/211] Check target properly --- src/Functions/FunctionStartsEndsWith.h | 9 +++++---- src/Functions/FunctionsRandom.h | 2 +- src/Functions/PerformanceAdaptors.h | 25 +++++++---------------- src/Functions/TargetSpecific.cpp | 27 ++++++++++++++++++++++--- src/Functions/TargetSpecific.h | 28 ++++++++++++++------------ 5 files changed, 52 insertions(+), 39 deletions(-) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 730f0b9efbb..29c81796d0e 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -145,11 +145,12 @@ public: FunctionStartsEndsWith(const Context &) : FunctionPerformanceAdaptor>() { - registerImplementation>(TargetArch::SSE4); - registerImplementation>(TargetArch::AVX); - registerImplementation>(TargetArch::AVX2); - registerImplementation>(TargetArch::AVX512); + registerImplementation> (TargetArch::SSE4); + registerImplementation> (TargetArch::AVX); + registerImplementation> (TargetArch::AVX2); + registerImplementation>(TargetArch::AVX512f); } + static FunctionPtr create(const Context & context) { return std::make_shared>(context); diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 990c3a5f466..98d04d61ad1 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -95,7 +95,7 @@ public: registerImplementation>(TargetArch::SSE4); registerImplementation>(TargetArch::AVX); registerImplementation>(TargetArch::AVX2); - registerImplementation>(TargetArch::AVX512); + registerImplementation>(TargetArch::AVX512f); } static FunctionPtr create(const Context &) { diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index 12f4b84dab9..a97fdbce0b0 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -104,6 +104,11 @@ struct PerformanceStatistics PerformanceStatistics(ssize_t choose_method_) : choose_method(choose_method_) {} }; +struct PerformanceAdaptorOptions +{ + +}; + /// Combine several IExecutableFunctionImpl into one. /// All the implementations should be equivalent. /// Implementation to execute will be selected based on performance on previous runs. @@ -152,6 +157,7 @@ public: private: std::vector impls; // Alternative implementations. PerformanceStatistics statistics; + PerformanceAdaptorOptions options; }; // The same as ExecutableFunctionPerformanceAdaptor, but combine via IFunction interface. @@ -197,24 +203,7 @@ public: private: std::vector impls; // Alternative implementations. PerformanceStatistics statistics; -}; - -// TODO(dakovalkov): May be it's better to delete this macros and write every function explicitly for better readability. -#define DECLARE_STANDART_TARGET_ADAPTOR(Function) \ -class Function : public FunctionDynamicAdaptor \ -{ \ -public: \ - Function(const Context &) : FunctionDynamicAdaptor() \ - { \ - registerImplementation(TargetArch::SSE4); \ - registerImplementation(TargetArch::AVX); \ - registerImplementation(TargetArch::AVX2); \ - registerImplementation(TargetArch::AVX512); \ - } \ - static FunctionPtr create(const Context & context) \ - { \ - return std::make_shared(context); \ - } \ + PerformanceAdaptorOptions options; }; } // namespace DB diff --git a/src/Functions/TargetSpecific.cpp b/src/Functions/TargetSpecific.cpp index f22a586c333..aa017823e54 100644 --- a/src/Functions/TargetSpecific.cpp +++ b/src/Functions/TargetSpecific.cpp @@ -2,6 +2,7 @@ #if defined(__GNUC__) # include +# include #else # error "Only CLANG and GCC compilers are supported for dynamic dispatch" #endif @@ -9,6 +10,11 @@ namespace DB { +__attribute__ ((target("xsave"))) +uint64_t xgetbv(uint32_t ecx) { + return _xgetbv(ecx); +} + int GetSupportedArches() { unsigned int eax, ebx, ecx, edx; if (!__get_cpuid(1, &eax, &ebx, &ecx, &edx)) { @@ -17,13 +23,15 @@ int GetSupportedArches() { int res = 0; if (ecx & bit_SSE4_2) res |= static_cast(TargetArch::SSE4); - if ((ecx & bit_OSXSAVE) && (ecx & bit_AVX)) { - // TODO(dakovalkov): check XGETBV. + // (xgetbv(0) & 0x6) == 0x6 checks that XMM state and YMM state are enabled. + if ((ecx & bit_OSXSAVE) && (ecx & bit_AVX) && (xgetbv(0) & 0x6) == 0x6) { res |= static_cast(TargetArch::AVX); if (__get_cpuid(7, &eax, &ebx, &ecx, &edx) && (ebx & bit_AVX2)) { res |= static_cast(TargetArch::AVX2); + if (ebx & bit_AVX512F) { + res |= static_cast(TargetArch::AVX512f); + } } - // TODO(dakovalkov): check AVX512 support. } return res; } @@ -34,4 +42,17 @@ bool IsArchSupported(TargetArch arch) return arch == TargetArch::Default || (arches & static_cast(arch)); } +String ToString(TargetArch arch) +{ + switch (arch) { + case TargetArch::Default: return "default"; + case TargetArch::SSE4: return "sse4"; + case TargetArch::AVX: return "avx"; + case TargetArch::AVX2: return "avx2"; + case TargetArch::AVX512f: return "avx512f"; + } + + __builtin_unreachable(); +} + } // namespace DB diff --git a/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h index e5818632843..accb1dd7fab 100644 --- a/src/Functions/TargetSpecific.h +++ b/src/Functions/TargetSpecific.h @@ -1,5 +1,7 @@ #pragma once +#include + /// This file contains macros and helpers for writing platform-dependent code. /// /// Macroses DECLARE__SPECIFIC_CODE will wrap code inside them into the namespace TargetSpecific:: and enable @@ -62,16 +64,17 @@ enum class TargetArch : int { SSE4 = (1 << 0), AVX = (1 << 1), AVX2 = (1 << 2), - AVX512 = (1 << 3), + AVX512f = (1 << 3), }; // Runtime detection. bool IsArchSupported(TargetArch arch); +String ToString(TargetArch arch); + #if defined(__clang__) -// TODO: There are lots of different AVX512 :( -# define BEGIN_AVX512_SPECIFIC_CODE \ - _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2\"))))") +# define BEGIN_AVX512f_SPECIFIC_CODE \ + _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2,avx512f\"))))") # define BEGIN_AVX2_SPECIFIC_CODE \ _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2\"))))") # define BEGIN_AVX_SPECIFIC_CODE \ @@ -81,8 +84,7 @@ bool IsArchSupported(TargetArch arch); # define END_TARGET_SPECIFIC_CODE \ _Pragma("clang attribute pop") #elif defined(__GNUC__) -// TODO: There are lots of different AVX512 :( -# define BEGIN_AVX512_SPECIFIC_CODE \ +# define BEGIN_AVX512f_SPECIFIC_CODE \ _Pragma("GCC push_options") \ _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2,avx512f,tune=native\")") # define BEGIN_AVX2_SPECIFIC_CODE \ @@ -130,10 +132,10 @@ namespace TargetSpecific::AVX2 { \ } \ END_TARGET_SPECIFIC_CODE -#define DECLARE_AVX512_SPECIFIC_CODE(...) \ -BEGIN_AVX512_SPECIFIC_CODE \ -namespace TargetSpecific::AVX512 { \ - using namespace DB::TargetSpecific::AVX512; \ +#define DECLARE_AVX512f_SPECIFIC_CODE(...) \ +BEGIN_AVX512f_SPECIFIC_CODE \ +namespace TargetSpecific::AVX512f { \ + using namespace DB::TargetSpecific::AVX512f; \ __VA_ARGS__ \ } \ END_TARGET_SPECIFIC_CODE @@ -143,7 +145,7 @@ DECLARE_DEFAULT_CODE (__VA_ARGS__) \ DECLARE_SSE4_SPECIFIC_CODE (__VA_ARGS__) \ DECLARE_AVX_SPECIFIC_CODE (__VA_ARGS__) \ DECLARE_AVX2_SPECIFIC_CODE (__VA_ARGS__) \ -DECLARE_AVX512_SPECIFIC_CODE(__VA_ARGS__) +DECLARE_AVX512f_SPECIFIC_CODE(__VA_ARGS__) DECLARE_DEFAULT_CODE( constexpr auto BuildArch = TargetArch::Default; @@ -161,8 +163,8 @@ DECLARE_AVX2_SPECIFIC_CODE( constexpr auto BuildArch = TargetArch::AVX2; ) // DECLARE_AVX2_SPECIFIC_CODE -DECLARE_AVX512_SPECIFIC_CODE( - constexpr auto BuildArch = TargetArch::AVX512; +DECLARE_AVX512f_SPECIFIC_CODE( + constexpr auto BuildArch = TargetArch::AVX512f; ) // DECLARE_AVX512_SPECIFIC_CODE } // namespace DB From 43f5ca868a14f49d0382d4954a247f47c0f919e8 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Fri, 15 May 2020 14:00:20 +0200 Subject: [PATCH 014/211] Cosmetics --- src/Functions/FunctionStartsEndsWith.h | 2 +- src/Functions/FunctionsRandom.cpp | 10 +++- src/Functions/FunctionsRandom.h | 8 +-- src/Functions/PerformanceAdaptors.h | 2 +- src/Functions/SIMDxorshift.cpp | 13 ++--- src/Functions/TargetSpecific.cpp | 4 +- src/Functions/TargetSpecific.h | 69 ++++++++++++++------------ 7 files changed, 61 insertions(+), 47 deletions(-) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 29c81796d0e..77692b5c414 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -148,7 +148,7 @@ public: registerImplementation> (TargetArch::SSE4); registerImplementation> (TargetArch::AVX); registerImplementation> (TargetArch::AVX2); - registerImplementation>(TargetArch::AVX512f); + registerImplementation>(TargetArch::AVX512F); } static FunctionPtr create(const Context & context) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 1f91c54c598..11861d2d12c 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -11,11 +11,19 @@ DECLARE_MULTITARGET_CODE( namespace { + /// NOTE Probably + /// http://www.pcg-random.org/ + /// or http://www.math.sci.hiroshima-u.ac.jp/~m-mat/MT/SFMT/ + /// or http://docs.yeppp.info/c/group__yep_random___w_e_l_l1024a.html + /// could go better. + struct LinearCongruentialGenerator { + /// Constants from `man lrand48_r`. static constexpr UInt64 a = 0x5DEECE66D; static constexpr UInt64 c = 0xB; + /// And this is from `head -c8 /dev/urandom | xxd -p` UInt64 current = 0x09826f4a081cee35ULL; void seed(UInt64 value) @@ -55,7 +63,7 @@ void RandImpl::execute(char * output, size_t size) unalignedStore(output + 8, generator2.next()); unalignedStore(output + 12, generator3.next()); } - + /// It is guaranteed (by PaddedPODArray) that we can overwrite up to 15 bytes after end. } ) //DECLARE_MULTITARGET_CODE diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 98d04d61ad1..ccc218574b0 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -92,10 +92,10 @@ class FunctionRandom : public FunctionPerformanceAdaptor>(TargetArch::SSE4); - registerImplementation>(TargetArch::AVX); - registerImplementation>(TargetArch::AVX2); - registerImplementation>(TargetArch::AVX512f); + registerImplementation>(TargetArch::SSE4); + registerImplementation>(TargetArch::AVX); + registerImplementation>(TargetArch::AVX2); + registerImplementation>(TargetArch::AVX512F); } static FunctionPtr create(const Context &) { diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index a97fdbce0b0..2ecd60e67fe 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -160,7 +160,7 @@ private: PerformanceAdaptorOptions options; }; -// The same as ExecutableFunctionPerformanceAdaptor, but combine via IFunction interface. +/// The same as ExecutableFunctionPerformanceAdaptor, but combine via IFunction interface. template class FunctionPerformanceAdaptor : public DefaultFunction { diff --git a/src/Functions/SIMDxorshift.cpp b/src/Functions/SIMDxorshift.cpp index 739077b5480..9a45257d28d 100644 --- a/src/Functions/SIMDxorshift.cpp +++ b/src/Functions/SIMDxorshift.cpp @@ -29,8 +29,8 @@ void RandXorshiftImpl::execute(char * output, size_t size) 0xa321e1523f4f88c7ULL ^ reinterpret_cast(output), &mykey); - const int bytes_per_write = 8; - const intptr_t mask = bytes_per_write - 1; + constexpr int bytes_per_write = 8; + constexpr intptr_t mask = bytes_per_write - 1; // Process head to make output aligned. unalignedStore(output, xorshift128plus(&mykey)); @@ -55,9 +55,9 @@ void RandXorshiftImpl::execute(char * output, size_t size) 0xa321e1523f4f88c7ULL ^ reinterpret_cast(output), &mykey); - const int safe_overwrite = 16; // How many bytes we can write behind the end. - const int bytes_per_write = 32; - const intptr_t mask = bytes_per_write - 1; + constexpr int safe_overwrite = 16; // How many bytes we can write behind the end. + constexpr int bytes_per_write = 32; + constexpr intptr_t mask = bytes_per_write - 1; if (size + safe_overwrite <= bytes_per_write) { _mm_storeu_si128(reinterpret_cast<__m128i*>(output), @@ -91,6 +91,7 @@ using FunctionRandXorshift64 = FunctionRandomXorshift(); + factory.registerFunction(); } -} +} // namespace DB diff --git a/src/Functions/TargetSpecific.cpp b/src/Functions/TargetSpecific.cpp index aa017823e54..19604a83ab7 100644 --- a/src/Functions/TargetSpecific.cpp +++ b/src/Functions/TargetSpecific.cpp @@ -29,7 +29,7 @@ int GetSupportedArches() { if (__get_cpuid(7, &eax, &ebx, &ecx, &edx) && (ebx & bit_AVX2)) { res |= static_cast(TargetArch::AVX2); if (ebx & bit_AVX512F) { - res |= static_cast(TargetArch::AVX512f); + res |= static_cast(TargetArch::AVX512F); } } } @@ -49,7 +49,7 @@ String ToString(TargetArch arch) case TargetArch::SSE4: return "sse4"; case TargetArch::AVX: return "avx"; case TargetArch::AVX2: return "avx2"; - case TargetArch::AVX512f: return "avx512f"; + case TargetArch::AVX512F: return "avx512f"; } __builtin_unreachable(); diff --git a/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h index accb1dd7fab..888d88d1d77 100644 --- a/src/Functions/TargetSpecific.h +++ b/src/Functions/TargetSpecific.h @@ -4,42 +4,47 @@ /// This file contains macros and helpers for writing platform-dependent code. /// -/// Macroses DECLARE__SPECIFIC_CODE will wrap code inside them into the namespace TargetSpecific:: and enable -/// Arch-specific compile options. -/// Thus, it's allowed to call functions inside only after checking platform in runtime (see IsArchSupported() below) -/// For similarities there is a macros DECLARE_DEFAULT_CODE, which wraps code into the namespace TargetSpecific::Default -/// but dosn't specify any additional copile options. +/// Macroses DECLARE__SPECIFIC_CODE will wrap code inside them into the +/// namespace TargetSpecific:: and enable Arch-specific compile options. +/// Thus, it's allowed to call functions inside these namespaces only after +/// checking platform in runtime (see IsArchSupported() below). +/// +/// For similarities there is a macros DECLARE_DEFAULT_CODE, which wraps code +/// into the namespace TargetSpecific::Default but dosn't specify any additional +/// copile options. /// /// Example: /// /// DECLARE_DEFAULT_CODE ( -/// int func() { +/// int funcImpl() { /// return 1; /// } /// ) // DECLARE_DEFAULT_CODE /// /// DECLARE_AVX2_SPECIFIC_CODE ( -/// int func() { +/// int funcImpl() { /// return 2; /// } /// ) // DECLARE_DEFAULT_CODE /// /// int func() { /// if (IsArchSupported(TargetArch::AVX2)) -/// return TargetSpecifc::AVX2::func(); -/// return TargetSpecifc::Default::func(); +/// return TargetSpecifc::AVX2::funcImpl(); +/// return TargetSpecifc::Default::funcImpl(); /// } /// /// Sometimes code may benefit from compiling with different options. -/// For these purposes use DECLARE_MULTITARGET_CODE macros. It will create several copies of the code and -/// compile it with different options. These copies are available via TargetSpecifc namespaces described above. +/// For these purposes use DECLARE_MULTITARGET_CODE macros. It will create several +/// copies of the code and compile it with different options. These copies are +/// available via TargetSpecifc namespaces described above. /// -/// Inside every TargetSpecific namespace there is a constexpr variable BuildArch, which TODO +/// Inside every TargetSpecific namespace there is a constexpr variable BuildArch, +/// which indicates the target platform for current code. /// /// Example: /// /// DECLARE_MULTITARGET_CODE( -/// int func(int size, ...) { +/// int funcImpl(int size, ...) { /// int iteration_size = 1; /// if constexpr (BuildArch == TargetArch::SSE4) /// iteration_size = 2 @@ -60,11 +65,11 @@ namespace DB { enum class TargetArch : int { - Default = 0, // Without any additional compiler options. - SSE4 = (1 << 0), - AVX = (1 << 1), - AVX2 = (1 << 2), - AVX512f = (1 << 3), + Default = 0, // Without any additional compiler options. + SSE4 = (1 << 0), + AVX = (1 << 1), + AVX2 = (1 << 2), + AVX512F = (1 << 3), }; // Runtime detection. @@ -73,7 +78,7 @@ bool IsArchSupported(TargetArch arch); String ToString(TargetArch arch); #if defined(__clang__) -# define BEGIN_AVX512f_SPECIFIC_CODE \ +# define BEGIN_AVX512F_SPECIFIC_CODE \ _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2,avx512f\"))))") # define BEGIN_AVX2_SPECIFIC_CODE \ _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2\"))))") @@ -84,7 +89,7 @@ String ToString(TargetArch arch); # define END_TARGET_SPECIFIC_CODE \ _Pragma("clang attribute pop") #elif defined(__GNUC__) -# define BEGIN_AVX512f_SPECIFIC_CODE \ +# define BEGIN_AVX512F_SPECIFIC_CODE \ _Pragma("GCC push_options") \ _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2,avx512f,tune=native\")") # define BEGIN_AVX2_SPECIFIC_CODE \ @@ -132,20 +137,20 @@ namespace TargetSpecific::AVX2 { \ } \ END_TARGET_SPECIFIC_CODE -#define DECLARE_AVX512f_SPECIFIC_CODE(...) \ -BEGIN_AVX512f_SPECIFIC_CODE \ -namespace TargetSpecific::AVX512f { \ - using namespace DB::TargetSpecific::AVX512f; \ +#define DECLARE_AVX512F_SPECIFIC_CODE(...) \ +BEGIN_AVX512F_SPECIFIC_CODE \ +namespace TargetSpecific::AVX512F { \ + using namespace DB::TargetSpecific::AVX512F; \ __VA_ARGS__ \ } \ END_TARGET_SPECIFIC_CODE #define DECLARE_MULTITARGET_CODE(...) \ -DECLARE_DEFAULT_CODE (__VA_ARGS__) \ -DECLARE_SSE4_SPECIFIC_CODE (__VA_ARGS__) \ -DECLARE_AVX_SPECIFIC_CODE (__VA_ARGS__) \ -DECLARE_AVX2_SPECIFIC_CODE (__VA_ARGS__) \ -DECLARE_AVX512f_SPECIFIC_CODE(__VA_ARGS__) +DECLARE_DEFAULT_CODE (__VA_ARGS__) \ +DECLARE_SSE4_SPECIFIC_CODE (__VA_ARGS__) \ +DECLARE_AVX_SPECIFIC_CODE (__VA_ARGS__) \ +DECLARE_AVX2_SPECIFIC_CODE (__VA_ARGS__) \ +DECLARE_AVX512F_SPECIFIC_CODE(__VA_ARGS__) DECLARE_DEFAULT_CODE( constexpr auto BuildArch = TargetArch::Default; @@ -163,8 +168,8 @@ DECLARE_AVX2_SPECIFIC_CODE( constexpr auto BuildArch = TargetArch::AVX2; ) // DECLARE_AVX2_SPECIFIC_CODE -DECLARE_AVX512f_SPECIFIC_CODE( - constexpr auto BuildArch = TargetArch::AVX512f; -) // DECLARE_AVX512_SPECIFIC_CODE +DECLARE_AVX512F_SPECIFIC_CODE( + constexpr auto BuildArch = TargetArch::AVX512F; +) // DECLARE_AVX512F_SPECIFIC_CODE } // namespace DB From d6d67b0da42b84302bfd13b624b3dad422941cd7 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Fri, 15 May 2020 14:06:12 +0200 Subject: [PATCH 015/211] Fix bug --- src/Functions/SIMDxorshift.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Functions/SIMDxorshift.cpp b/src/Functions/SIMDxorshift.cpp index 9a45257d28d..f7c5f953e09 100644 --- a/src/Functions/SIMDxorshift.cpp +++ b/src/Functions/SIMDxorshift.cpp @@ -55,11 +55,12 @@ void RandXorshiftImpl::execute(char * output, size_t size) 0xa321e1523f4f88c7ULL ^ reinterpret_cast(output), &mykey); - constexpr int safe_overwrite = 16; // How many bytes we can write behind the end. + constexpr int safe_overwrite = 15; // How many bytes we can write behind the end. constexpr int bytes_per_write = 32; constexpr intptr_t mask = bytes_per_write - 1; - if (size + safe_overwrite <= bytes_per_write) { + if (size + safe_overwrite < bytes_per_write) { + // size <= 16. _mm_storeu_si128(reinterpret_cast<__m128i*>(output), _mm256_extracti128_si256(avx_xorshift128plus(&mykey), 0)); return; @@ -74,7 +75,7 @@ void RandXorshiftImpl::execute(char * output, size_t size) output += bytes_per_write; } - // Process tail. + // Process tail. (end - output) <= 16. if ((end - output) > 0) { _mm_store_si128(reinterpret_cast<__m128i*>(output), _mm256_extracti128_si256(avx_xorshift128plus(&mykey), 0)); From 80ab14e3f96a4013a687d352bc2457c31ed1d099 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Sat, 16 May 2020 08:15:39 +0200 Subject: [PATCH 016/211] Get rid of copy-paste in PerformanceAdaptor --- src/Functions/FunctionStartsEndsWith.h | 9 +- src/Functions/FunctionsRandom.h | 14 ++- src/Functions/PerformanceAdaptors.h | 154 +++++++++++++++---------- src/Functions/SIMDxorshift.cpp | 2 - src/Functions/SIMDxorshift.h | 12 +- 5 files changed, 119 insertions(+), 72 deletions(-) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 77692b5c414..44850257f99 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -41,6 +41,11 @@ public: return name; } + static String getImplementationTag() + { + return ToString(BuildArch); + } + size_t getNumberOfArguments() const override { return 2; @@ -143,7 +148,9 @@ class FunctionStartsEndsWith { public: FunctionStartsEndsWith(const Context &) - : FunctionPerformanceAdaptor>() + : FunctionPerformanceAdaptor>( + PerformanceAdaptorOptions() + ) { registerImplementation> (TargetArch::SSE4); registerImplementation> (TargetArch::AVX); diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index ccc218574b0..ae54243164f 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -42,6 +42,7 @@ DECLARE_MULTITARGET_CODE( struct RandImpl { static void execute(char * output, size_t size); + static String getImplementationTag() { return ToString(BuildArch); } }; ) // DECLARE_MULTITARGET_CODE @@ -57,6 +58,11 @@ public: return name; } + static String getImplementationTag() + { + return RandImpl::getImplementationTag(); + } + bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } bool useDefaultImplementationForNulls() const override { return false; } @@ -91,14 +97,18 @@ template class FunctionRandom : public FunctionPerformanceAdaptor> { public: - FunctionRandom() { + FunctionRandom() + : FunctionPerformanceAdaptor>( + PerformanceAdaptorOptions()) + { registerImplementation>(TargetArch::SSE4); registerImplementation>(TargetArch::AVX); registerImplementation>(TargetArch::AVX2); registerImplementation>(TargetArch::AVX512F); } - static FunctionPtr create(const Context &) { + static FunctionPtr create(const Context &) + { return std::make_shared>(); } }; diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index 2ecd60e67fe..f7bb8cfd6ee 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -92,10 +92,14 @@ struct PerformanceStatistics return choose_method; } - size_t size() { + size_t size() const { return data.size(); } + bool empty() const { + return size() == 0; + } + void emplace_back() { data.emplace_back(); } @@ -106,7 +110,47 @@ struct PerformanceStatistics struct PerformanceAdaptorOptions { + std::optional> implementations; +}; +// Redirects IExecutableFunctionImpl::execute() and IFunction:executeImpl() to executeFunctionImpl(); +template +class FunctionExecutor; + +template +class FunctionExecutor>> + : public DefaultFunction +{ +public: + using BaseFunctionPtr = ExecutableFunctionImplPtr; + + template + FunctionExecutor(Args ...args) : DefaultFunction(args...) {} + + virtual void executeFunctionImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0; + + virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + executeFunctionImpl(block, arguments, result, input_rows_count); + } +}; + +template +class FunctionExecutor>> + : public DefaultFunction +{ +public: + using BaseFunctionPtr = FunctionPtr; + + template + FunctionExecutor(Args ...args) : DefaultFunction(args...) {} + + virtual void executeFunctionImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0; + + virtual void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + executeFunctionImpl(block, arguments, result, input_rows_count); + } }; /// Combine several IExecutableFunctionImpl into one. @@ -116,23 +160,60 @@ struct PerformanceAdaptorOptions /// could use extended set of instructions (AVX, NEON, etc). /// It's convenient to inherit your func from this and register all alternative implementations in the constructor. template -class ExecutableFunctionPerformanceAdaptor : public DefaultFunction +class FunctionPerformanceAdaptor : public FunctionExecutor { public: + using BaseFunctionPtr = FunctionExecutor::BaseFunctionPtr; + template - ExecutableFunctionPerformanceAdaptor(Params ...params) : DefaultFunction(params...) + FunctionPerformanceAdaptor(PerformanceAdaptorOptions options_, Params ...params) + : FunctionExecutor(params...) + , options(std::move(options_)) { - statistics.emplace_back(); + if (isImplementationEnabled(DefaultFunction::getImplementationTag())) { + statistics.emplace_back(); + } } - virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + // Register alternative implementation. + template + void registerImplementation(TargetArch arch, Params... params) { + if (IsArchSupported(arch) && isImplementationEnabled(Function::getImplementationTag())) { + impls.emplace_back(std::make_shared(params...)); + statistics.emplace_back(); + } + } + + bool isImplementationEnabled(const String & impl_tag) { + if (!options.implementations) { + return true; + } + for (const auto & tag : *options.implementations) { + if (tag == impl_tag) { + return true; + } + } + return false; + } + +protected: + virtual void executeFunctionImpl(Block & block, const ColumnNumbers & arguments, + size_t result, size_t input_rows_count) override { + if (statistics.empty()) + throw "No implementations"; auto id = statistics.select(); Stopwatch watch; - if (id == 0) { - DefaultFunction::execute(block, arguments, result, input_rows_count); + if (id == impls.size()) { + if constexpr (std::is_base_of_v) + DefaultFunction::executeImpl(block, arguments, result, input_rows_count); + else + DefaultFunction::execute(block, arguments, result, input_rows_count); } else { - impls[id - 1]->execute(block, arguments, result, input_rows_count); + if constexpr (std::is_base_of_v) + impls[id]->executeImpl(block, arguments, result, input_rows_count); + else + impls[id]->execute(block, arguments, result, input_rows_count); } watch.stop(); // TODO(dakovalkov): Calculate something more informative. @@ -145,63 +226,8 @@ public: } } - // Register alternative implementation. - template - void registerImplementation(TargetArch arch, Params... params) { - if (arch == TargetArch::Default || IsArchSupported(arch)) { - impls.emplace_back(std::make_shared(params...)); - statistics.emplace_back(); - } - } - private: - std::vector impls; // Alternative implementations. - PerformanceStatistics statistics; - PerformanceAdaptorOptions options; -}; - -/// The same as ExecutableFunctionPerformanceAdaptor, but combine via IFunction interface. -template -class FunctionPerformanceAdaptor : public DefaultFunction -{ -public: - template - FunctionPerformanceAdaptor(Params ...params) : DefaultFunction(params...) - { - statistics.emplace_back(); - } - - virtual void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - auto id = statistics.select(); - Stopwatch watch; - if (id == 0) { - DefaultFunction::executeImpl(block, arguments, result, input_rows_count); - } else { - impls[id - 1]->executeImpl(block, arguments, result, input_rows_count); - } - watch.stop(); - // TODO(dakovalkov): Calculate something more informative. - size_t rows_summary = 0; - for (auto i : arguments) { - rows_summary += block.getByPosition(i).column->size(); - } - if (rows_summary >= 1000) { - statistics.data[id].update(watch.elapsedSeconds(), rows_summary); - } - } - - // Register alternative implementation. - template - void registerImplementation(TargetArch arch, Params... params) { - if (arch == TargetArch::Default || IsArchSupported(arch)) { - impls.emplace_back(std::make_shared(params...)); - statistics.emplace_back(); - } - } - -private: - std::vector impls; // Alternative implementations. + std::vector impls; // Alternative implementations. PerformanceStatistics statistics; PerformanceAdaptorOptions options; }; diff --git a/src/Functions/SIMDxorshift.cpp b/src/Functions/SIMDxorshift.cpp index f7c5f953e09..b5c8b0995ac 100644 --- a/src/Functions/SIMDxorshift.cpp +++ b/src/Functions/SIMDxorshift.cpp @@ -4,8 +4,6 @@ #include #include -#include - #include extern "C" diff --git a/src/Functions/SIMDxorshift.h b/src/Functions/SIMDxorshift.h index 46732c4d876..c8b741c06b1 100644 --- a/src/Functions/SIMDxorshift.h +++ b/src/Functions/SIMDxorshift.h @@ -22,19 +22,25 @@ DECLARE_MULTITARGET_CODE( struct RandXorshiftImpl { static void execute(char * output, size_t size); + static String getImplementationTag() { return ToString(BuildArch); } }; ) // DECLARE_MULTITARGET_CODE template -class FunctionRandomXorshift : public FunctionPerformanceAdaptor> +class FunctionRandomXorshift + : public FunctionPerformanceAdaptor> { public: - FunctionRandomXorshift() { + FunctionRandomXorshift() + : FunctionPerformanceAdaptor>( + PerformanceAdaptorOptions()) + { registerImplementation>(TargetArch::AVX2); } - static FunctionPtr create(const Context &) { + static FunctionPtr create(const Context &) + { return std::make_shared>(); } }; From f59b13a58d54ed5c447028dc4275cfed6ac38b88 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Sat, 16 May 2020 08:59:08 +0200 Subject: [PATCH 017/211] Fix style issues --- src/Common/ErrorCodes.cpp | 1 + src/Functions/FunctionsRandom.h | 2 +- src/Functions/PerformanceAdaptors.h | 56 +++++++++++++++++++---------- src/Functions/SIMDxorshift.cpp | 35 ++++++++++-------- src/Functions/SIMDxorshift.h | 2 +- src/Functions/TargetSpecific.cpp | 42 ++++++++++++---------- src/Functions/TargetSpecific.h | 5 +-- 7 files changed, 88 insertions(+), 55 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index cb4c591041c..2681bd0773c 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -498,6 +498,7 @@ namespace ErrorCodes extern const int ALTER_OF_COLUMN_IS_FORBIDDEN = 524; extern const int INCORRECT_DISK_INDEX = 525; extern const int UNKNOWN_VOLUME_TYPE = 526; + extern const int NO_SUITABLE_FUNCTION_IMPLEMENTATION = 527; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index ae54243164f..6130ee1c2a5 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -113,4 +113,4 @@ public: } }; -} // namespace DB +} diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index f7bb8cfd6ee..f7b9c12c7cb 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -14,6 +14,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NO_SUITABLE_FUNCTION_IMPLEMENTATION; +} + // TODO(dakovalkov): This is copied and pasted struct from LZ4_decompress_faster.h with little changes. struct PerformanceStatistics { @@ -92,15 +97,18 @@ struct PerformanceStatistics return choose_method; } - size_t size() const { + size_t size() const + { return data.size(); } - bool empty() const { + bool empty() const + { return size() == 0; } - void emplace_back() { + void emplace_back() + { data.emplace_back(); } @@ -113,7 +121,7 @@ struct PerformanceAdaptorOptions std::optional> implementations; }; -// Redirects IExecutableFunctionImpl::execute() and IFunction:executeImpl() to executeFunctionImpl(); +/// Redirects IExecutableFunctionImpl::execute() and IFunction:executeImpl() to executeFunctionImpl(); template class FunctionExecutor; @@ -170,28 +178,28 @@ public: : FunctionExecutor(params...) , options(std::move(options_)) { - if (isImplementationEnabled(DefaultFunction::getImplementationTag())) { + if (isImplementationEnabled(DefaultFunction::getImplementationTag())) statistics.emplace_back(); - } } - // Register alternative implementation. + /// Register alternative implementation. template void registerImplementation(TargetArch arch, Params... params) { - if (IsArchSupported(arch) && isImplementationEnabled(Function::getImplementationTag())) { + if (IsArchSupported(arch) && isImplementationEnabled(Function::getImplementationTag())) + { impls.emplace_back(std::make_shared(params...)); statistics.emplace_back(); } } bool isImplementationEnabled(const String & impl_tag) { - if (!options.implementations) { + if (!options.implementations) return true; - } - for (const auto & tag : *options.implementations) { - if (tag == impl_tag) { + + for (const auto & tag : *options.implementations) + { + if (tag == impl_tag) return true; - } } return false; } @@ -201,27 +209,37 @@ protected: size_t result, size_t input_rows_count) override { if (statistics.empty()) - throw "No implementations"; + throw Exception("All available implementations are disabled by user config", + ErrorCodes::NO_SUITABLE_FUNCTION_IMPLEMENTATION); + auto id = statistics.select(); Stopwatch watch; - if (id == impls.size()) { + + if (id == impls.size()) + { if constexpr (std::is_base_of_v) DefaultFunction::executeImpl(block, arguments, result, input_rows_count); else DefaultFunction::execute(block, arguments, result, input_rows_count); - } else { + } + else + { if constexpr (std::is_base_of_v) impls[id]->executeImpl(block, arguments, result, input_rows_count); else impls[id]->execute(block, arguments, result, input_rows_count); } watch.stop(); + // TODO(dakovalkov): Calculate something more informative. size_t rows_summary = 0; - for (auto i : arguments) { + for (auto i : arguments) + { rows_summary += block.getByPosition(i).column->size(); } - if (rows_summary >= 1000) { + + if (rows_summary >= 1000) + { statistics.data[id].update(watch.elapsedSeconds(), rows_summary); } } @@ -232,4 +250,4 @@ private: PerformanceAdaptorOptions options; }; -} // namespace DB +} diff --git a/src/Functions/SIMDxorshift.cpp b/src/Functions/SIMDxorshift.cpp index b5c8b0995ac..a8410ed957a 100644 --- a/src/Functions/SIMDxorshift.cpp +++ b/src/Functions/SIMDxorshift.cpp @@ -28,15 +28,16 @@ void RandXorshiftImpl::execute(char * output, size_t size) &mykey); constexpr int bytes_per_write = 8; - constexpr intptr_t mask = bytes_per_write - 1; - + constexpr intptr_t mask = bytes_per_write - 1; + // Process head to make output aligned. unalignedStore(output, xorshift128plus(&mykey)); output = reinterpret_cast((reinterpret_cast(output) | mask) + 1); - while (end - output > 0) { + while (end - output > 0) + { *reinterpret_cast(output) = xorshift128plus(&mykey); - output += bytes_per_write; + output += bytes_per_write; } } @@ -46,6 +47,9 @@ DECLARE_AVX2_SPECIFIC_CODE( void RandXorshiftImpl::execute(char * output, size_t size) { + if (size == 0) + return; + char * end = output + size; avx_xorshift128plus_key_t mykey; @@ -53,31 +57,34 @@ void RandXorshiftImpl::execute(char * output, size_t size) 0xa321e1523f4f88c7ULL ^ reinterpret_cast(output), &mykey); - constexpr int safe_overwrite = 15; // How many bytes we can write behind the end. + constexpr int safe_overwrite = 15; /// How many bytes we can write behind the end. constexpr int bytes_per_write = 32; - constexpr intptr_t mask = bytes_per_write - 1; + constexpr intptr_t mask = bytes_per_write - 1; - if (size + safe_overwrite < bytes_per_write) { - // size <= 16. + if (size + safe_overwrite < bytes_per_write) + { + /// size <= 16. _mm_storeu_si128(reinterpret_cast<__m128i*>(output), _mm256_extracti128_si256(avx_xorshift128plus(&mykey), 0)); return; } - // Process head to make output aligned. + /// Process head to make output aligned. _mm256_storeu_si256(reinterpret_cast<__m256i*>(output), avx_xorshift128plus(&mykey)); output = reinterpret_cast((reinterpret_cast(output) | mask) + 1); - while ((end - output) + safe_overwrite >= bytes_per_write) { + while ((end - output) + safe_overwrite >= bytes_per_write) + { _mm256_store_si256(reinterpret_cast<__m256i*>(output), avx_xorshift128plus(&mykey)); output += bytes_per_write; } - // Process tail. (end - output) <= 16. - if ((end - output) > 0) { + /// Process tail. (end - output) <= 16. + if ((end - output) > 0) + { _mm_store_si128(reinterpret_cast<__m128i*>(output), _mm256_extracti128_si256(avx_xorshift128plus(&mykey), 0)); - } + } } ) // DECLARE_AVX2_SPECIFIC_CODE @@ -93,4 +100,4 @@ void registerFunctionRandXorshift(FunctionFactory & factory) factory.registerFunction(); } -} // namespace DB +} diff --git a/src/Functions/SIMDxorshift.h b/src/Functions/SIMDxorshift.h index c8b741c06b1..c9e46cf7192 100644 --- a/src/Functions/SIMDxorshift.h +++ b/src/Functions/SIMDxorshift.h @@ -45,4 +45,4 @@ public: } }; -} // namespace DB +} diff --git a/src/Functions/TargetSpecific.cpp b/src/Functions/TargetSpecific.cpp index 19604a83ab7..891d63d8258 100644 --- a/src/Functions/TargetSpecific.cpp +++ b/src/Functions/TargetSpecific.cpp @@ -11,25 +11,30 @@ namespace DB { __attribute__ ((target("xsave"))) -uint64_t xgetbv(uint32_t ecx) { +UInt64 xgetbv(UInt32 ecx) +{ return _xgetbv(ecx); } -int GetSupportedArches() { - unsigned int eax, ebx, ecx, edx; - if (!__get_cpuid(1, &eax, &ebx, &ecx, &edx)) { +UInt32 GetSupportedArches() +{ + UInt32 eax, ebx, ecx, edx; + if (!__get_cpuid(1, &eax, &ebx, &ecx, &edx)) return 0; - } - int res = 0; + + UInt32 res = 0; if (ecx & bit_SSE4_2) - res |= static_cast(TargetArch::SSE4); - // (xgetbv(0) & 0x6) == 0x6 checks that XMM state and YMM state are enabled. - if ((ecx & bit_OSXSAVE) && (ecx & bit_AVX) && (xgetbv(0) & 0x6) == 0x6) { - res |= static_cast(TargetArch::AVX); - if (__get_cpuid(7, &eax, &ebx, &ecx, &edx) && (ebx & bit_AVX2)) { - res |= static_cast(TargetArch::AVX2); - if (ebx & bit_AVX512F) { - res |= static_cast(TargetArch::AVX512F); + res |= static_cast(TargetArch::SSE4); + /// (xgetbv(0) & 0x6) == 0x6 checks that XMM state and YMM state are enabled. + if ((ecx & bit_OSXSAVE) && (ecx & bit_AVX) && (xgetbv(0) & 0x6) == 0x6) + { + res |= static_cast(TargetArch::AVX); + if (__get_cpuid(7, &eax, &ebx, &ecx, &edx) && (ebx & bit_AVX2)) + { + res |= static_cast(TargetArch::AVX2); + if (ebx & bit_AVX512F) + { + res |= static_cast(TargetArch::AVX512F); } } } @@ -38,13 +43,14 @@ int GetSupportedArches() { bool IsArchSupported(TargetArch arch) { - static int arches = GetSupportedArches(); - return arch == TargetArch::Default || (arches & static_cast(arch)); + static UInt32 arches = GetSupportedArches(); + return arch == TargetArch::Default || (arches & static_cast(arch)); } String ToString(TargetArch arch) { - switch (arch) { + switch (arch) + { case TargetArch::Default: return "default"; case TargetArch::SSE4: return "sse4"; case TargetArch::AVX: return "avx"; @@ -55,4 +61,4 @@ String ToString(TargetArch arch) __builtin_unreachable(); } -} // namespace DB +} diff --git a/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h index 888d88d1d77..7af792ae3c7 100644 --- a/src/Functions/TargetSpecific.h +++ b/src/Functions/TargetSpecific.h @@ -64,7 +64,8 @@ namespace DB { -enum class TargetArch : int { +enum class TargetArch : UInt32 +{ Default = 0, // Without any additional compiler options. SSE4 = (1 << 0), AVX = (1 << 1), @@ -172,4 +173,4 @@ DECLARE_AVX512F_SPECIFIC_CODE( constexpr auto BuildArch = TargetArch::AVX512F; ) // DECLARE_AVX512F_SPECIFIC_CODE -} // namespace DB +} From 2609b1c370989238a0e619d7db246474e19d266f Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Sat, 16 May 2020 09:01:46 +0200 Subject: [PATCH 018/211] Save test --- tests/performance/rand.xml | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/tests/performance/rand.xml b/tests/performance/rand.xml index 32ec38ddb4e..a007eb50179 100644 --- a/tests/performance/rand.xml +++ b/tests/performance/rand.xml @@ -1,10 +1,10 @@ - 20000 + 10000 - 40000 + 20000 @@ -13,12 +13,13 @@ table - numbers(100000000) + numbers(10000000) - SELECT rand() FROM {table} - SELECT rand64() FROM {table} - SELECT randxorshift() FROM {table} + SELECT count() FROM (SELECT rand() FROM {table}) + SELECT count() FROM (SELECT randxorshift() FROM {table}) + SELECT count() FROM (SELECT rand64() FROM {table}) + SELECT count() FROM (SELECT randxorshift64() FROM {table}) From e1dc2330891a5fb1fa5946dae3841c7ad8714ec1 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Sat, 16 May 2020 17:20:07 +0200 Subject: [PATCH 019/211] Fix clang build (probably) --- src/Functions/TargetSpecific.h | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h index 7af792ae3c7..a9ed8bfe71e 100644 --- a/src/Functions/TargetSpecific.h +++ b/src/Functions/TargetSpecific.h @@ -79,16 +79,15 @@ bool IsArchSupported(TargetArch arch); String ToString(TargetArch arch); #if defined(__clang__) -# define BEGIN_AVX512F_SPECIFIC_CODE \ - _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2,avx512f\"))))") -# define BEGIN_AVX2_SPECIFIC_CODE \ - _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2\"))))") -# define BEGIN_AVX_SPECIFIC_CODE \ - _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx\"))))") -# define BEGIN_SSE4_SPECIFIC_CODE \ - _Pragma("clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx\"))))") -# define END_TARGET_SPECIFIC_CODE \ - _Pragma("clang attribute pop") +# define BEGIN_AVX512F_SPECIFIC_CODE _Pragma(\ + "clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx,avx2,avx512f\"))),apply_to=function)") +# define BEGIN_AVX2_SPECIFIC_CODE _Pragma(\ + "clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx,avx2\"))),apply_to=function)") +# define BEGIN_AVX_SPECIFIC_CODE _Pragma(\ + "clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx\"))),apply_to=function)") +# define BEGIN_SSE4_SPECIFIC_CODE _Pragma(\ + "clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx\"))),apply_to=function)") +# define END_TARGET_SPECIFIC_CODE _Pragma("clang attribute pop") #elif defined(__GNUC__) # define BEGIN_AVX512F_SPECIFIC_CODE \ _Pragma("GCC push_options") \ From a4ff8bb9331d289a98dac69a92f5f3f0fec73217 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Sat, 16 May 2020 17:43:05 +0200 Subject: [PATCH 020/211] Get rid of handwritten cpu feature detection --- src/Functions/FunctionStartsEndsWith.h | 2 +- src/Functions/FunctionsRandom.h | 2 +- src/Functions/TargetSpecific.cpp | 46 +++++++------------------- src/Functions/TargetSpecific.h | 26 +++++++-------- 4 files changed, 27 insertions(+), 49 deletions(-) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 44850257f99..9435adbe858 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -152,7 +152,7 @@ public: PerformanceAdaptorOptions() ) { - registerImplementation> (TargetArch::SSE4); + registerImplementation> (TargetArch::SSE42); registerImplementation> (TargetArch::AVX); registerImplementation> (TargetArch::AVX2); registerImplementation>(TargetArch::AVX512F); diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 6130ee1c2a5..8fb1758a60c 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -101,7 +101,7 @@ public: : FunctionPerformanceAdaptor>( PerformanceAdaptorOptions()) { - registerImplementation>(TargetArch::SSE4); + registerImplementation>(TargetArch::SSE42); registerImplementation>(TargetArch::AVX); registerImplementation>(TargetArch::AVX2); registerImplementation>(TargetArch::AVX512F); diff --git a/src/Functions/TargetSpecific.cpp b/src/Functions/TargetSpecific.cpp index 891d63d8258..4168fb60a59 100644 --- a/src/Functions/TargetSpecific.cpp +++ b/src/Functions/TargetSpecific.cpp @@ -1,44 +1,22 @@ #include -#if defined(__GNUC__) -# include -# include -#else -# error "Only CLANG and GCC compilers are supported for dynamic dispatch" -#endif +#include namespace DB { -__attribute__ ((target("xsave"))) -UInt64 xgetbv(UInt32 ecx) -{ - return _xgetbv(ecx); -} - UInt32 GetSupportedArches() { - UInt32 eax, ebx, ecx, edx; - if (!__get_cpuid(1, &eax, &ebx, &ecx, &edx)) - return 0; - - UInt32 res = 0; - if (ecx & bit_SSE4_2) - res |= static_cast(TargetArch::SSE4); - /// (xgetbv(0) & 0x6) == 0x6 checks that XMM state and YMM state are enabled. - if ((ecx & bit_OSXSAVE) && (ecx & bit_AVX) && (xgetbv(0) & 0x6) == 0x6) - { - res |= static_cast(TargetArch::AVX); - if (__get_cpuid(7, &eax, &ebx, &ecx, &edx) && (ebx & bit_AVX2)) - { - res |= static_cast(TargetArch::AVX2); - if (ebx & bit_AVX512F) - { - res |= static_cast(TargetArch::AVX512F); - } - } - } - return res; + UInt32 result = 0; + if (Cpu::haveSSE42()) + result |= static_cast(TargetArch::SSE42); + if (Cpu::haveAVX()) + result |= static_cast(TargetArch::AVX); + if (Cpu::haveAVX2()) + result |= static_cast(TargetArch::AVX2); + if (Cpu::haveAVX512F()) + result |= static_cast(TargetArch::AVX512F); + return result; } bool IsArchSupported(TargetArch arch) @@ -52,7 +30,7 @@ String ToString(TargetArch arch) switch (arch) { case TargetArch::Default: return "default"; - case TargetArch::SSE4: return "sse4"; + case TargetArch::SSE42: return "sse42"; case TargetArch::AVX: return "avx"; case TargetArch::AVX2: return "avx2"; case TargetArch::AVX512F: return "avx512f"; diff --git a/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h index a9ed8bfe71e..7a946effb53 100644 --- a/src/Functions/TargetSpecific.h +++ b/src/Functions/TargetSpecific.h @@ -46,7 +46,7 @@ /// DECLARE_MULTITARGET_CODE( /// int funcImpl(int size, ...) { /// int iteration_size = 1; -/// if constexpr (BuildArch == TargetArch::SSE4) +/// if constexpr (BuildArch == TargetArch::SSE42) /// iteration_size = 2 /// else if constexpr (BuildArch == TargetArch::AVX || BuildArch == TargetArch::AVX2) /// iteration_size = 4; @@ -66,8 +66,8 @@ namespace DB enum class TargetArch : UInt32 { - Default = 0, // Without any additional compiler options. - SSE4 = (1 << 0), + Default = 0, /// Without any additional compiler options. + SSE42 = (1 << 0), /// SSE4.2 AVX = (1 << 1), AVX2 = (1 << 2), AVX512F = (1 << 3), @@ -85,7 +85,7 @@ String ToString(TargetArch arch); "clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx,avx2\"))),apply_to=function)") # define BEGIN_AVX_SPECIFIC_CODE _Pragma(\ "clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx\"))),apply_to=function)") -# define BEGIN_SSE4_SPECIFIC_CODE _Pragma(\ +# define BEGIN_SSE42_SPECIFIC_CODE _Pragma(\ "clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx\"))),apply_to=function)") # define END_TARGET_SPECIFIC_CODE _Pragma("clang attribute pop") #elif defined(__GNUC__) @@ -98,7 +98,7 @@ String ToString(TargetArch arch); # define BEGIN_AVX_SPECIFIC_CODE \ _Pragma("GCC push_options") \ _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,tune=native\")") -# define BEGIN_SSE4_SPECIFIC_CODE \ +# define BEGIN_SSE42_SPECIFIC_CODE \ _Pragma("GCC push_options") \ _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,tune=native\")") # define END_TARGET_SPECIFIC_CODE \ @@ -113,10 +113,10 @@ namespace TargetSpecific::Default { \ __VA_ARGS__ \ } -#define DECLARE_SSE4_SPECIFIC_CODE(...) \ -BEGIN_SSE4_SPECIFIC_CODE \ -namespace TargetSpecific::SSE4 { \ - using namespace DB::TargetSpecific::SSE4; \ +#define DECLARE_SSE42_SPECIFIC_CODE(...) \ +BEGIN_SSE42_SPECIFIC_CODE \ +namespace TargetSpecific::SSE42 { \ + using namespace DB::TargetSpecific::SSE42; \ __VA_ARGS__ \ } \ END_TARGET_SPECIFIC_CODE @@ -147,7 +147,7 @@ END_TARGET_SPECIFIC_CODE #define DECLARE_MULTITARGET_CODE(...) \ DECLARE_DEFAULT_CODE (__VA_ARGS__) \ -DECLARE_SSE4_SPECIFIC_CODE (__VA_ARGS__) \ +DECLARE_SSE42_SPECIFIC_CODE (__VA_ARGS__) \ DECLARE_AVX_SPECIFIC_CODE (__VA_ARGS__) \ DECLARE_AVX2_SPECIFIC_CODE (__VA_ARGS__) \ DECLARE_AVX512F_SPECIFIC_CODE(__VA_ARGS__) @@ -156,9 +156,9 @@ DECLARE_DEFAULT_CODE( constexpr auto BuildArch = TargetArch::Default; ) // DECLARE_DEFAULT_CODE -DECLARE_SSE4_SPECIFIC_CODE( - constexpr auto BuildArch = TargetArch::SSE4; -) // DECLARE_SSE4_SPECIFIC_CODE +DECLARE_SSE42_SPECIFIC_CODE( + constexpr auto BuildArch = TargetArch::SSE42; +) // DECLARE_SSE42_SPECIFIC_CODE DECLARE_AVX_SPECIFIC_CODE( constexpr auto BuildArch = TargetArch::AVX; From ad0ddc936a75ec1e9bd26f98fff0258827d2bedf Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Sat, 16 May 2020 19:21:23 +0200 Subject: [PATCH 021/211] Cosmetics, fix style issues --- src/Functions/PerformanceAdaptors.h | 16 +++++++++------- .../{SIMDxorshift.cpp => RandXorshift.cpp} | 2 +- src/Functions/{SIMDxorshift.h => RandXorshift.h} | 0 src/Functions/TargetSpecific.h | 2 +- 4 files changed, 11 insertions(+), 9 deletions(-) rename src/Functions/{SIMDxorshift.cpp => RandXorshift.cpp} (98%) rename src/Functions/{SIMDxorshift.h => RandXorshift.h} (100%) diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index f7b9c12c7cb..ea3f2ae0b47 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -133,7 +133,7 @@ public: using BaseFunctionPtr = ExecutableFunctionImplPtr; template - FunctionExecutor(Args ...args) : DefaultFunction(args...) {} + FunctionExecutor(Args&&... args) : DefaultFunction(std::forward(args)...) {} virtual void executeFunctionImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0; @@ -151,7 +151,7 @@ public: using BaseFunctionPtr = FunctionPtr; template - FunctionExecutor(Args ...args) : DefaultFunction(args...) {} + FunctionExecutor(Args&&... args) : DefaultFunction(std::forward(args)...) {} virtual void executeFunctionImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0; @@ -174,8 +174,8 @@ public: using BaseFunctionPtr = FunctionExecutor::BaseFunctionPtr; template - FunctionPerformanceAdaptor(PerformanceAdaptorOptions options_, Params ...params) - : FunctionExecutor(params...) + FunctionPerformanceAdaptor(PerformanceAdaptorOptions options_, Params&&... params) + : FunctionExecutor(std::forward(params)...) , options(std::move(options_)) { if (isImplementationEnabled(DefaultFunction::getImplementationTag())) @@ -184,15 +184,17 @@ public: /// Register alternative implementation. template - void registerImplementation(TargetArch arch, Params... params) { + void registerImplementation(TargetArch arch, Params&&... params) + { if (IsArchSupported(arch) && isImplementationEnabled(Function::getImplementationTag())) { - impls.emplace_back(std::make_shared(params...)); + impls.emplace_back(std::make_shared(std::forward(params)...)); statistics.emplace_back(); } } - bool isImplementationEnabled(const String & impl_tag) { + bool isImplementationEnabled(const String & impl_tag) + { if (!options.implementations) return true; diff --git a/src/Functions/SIMDxorshift.cpp b/src/Functions/RandXorshift.cpp similarity index 98% rename from src/Functions/SIMDxorshift.cpp rename to src/Functions/RandXorshift.cpp index a8410ed957a..652bb90f559 100644 --- a/src/Functions/SIMDxorshift.cpp +++ b/src/Functions/RandXorshift.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include extern "C" { diff --git a/src/Functions/SIMDxorshift.h b/src/Functions/RandXorshift.h similarity index 100% rename from src/Functions/SIMDxorshift.h rename to src/Functions/RandXorshift.h diff --git a/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h index 7a946effb53..f5bd0267c52 100644 --- a/src/Functions/TargetSpecific.h +++ b/src/Functions/TargetSpecific.h @@ -4,7 +4,7 @@ /// This file contains macros and helpers for writing platform-dependent code. /// -/// Macroses DECLARE__SPECIFIC_CODE will wrap code inside them into the +/// Macros DECLARE__SPECIFIC_CODE will wrap code inside them into the /// namespace TargetSpecific:: and enable Arch-specific compile options. /// Thus, it's allowed to call functions inside these namespaces only after /// checking platform in runtime (see IsArchSupported() below). From 234a828dd0315800d96ac87568dea7353464c9ba Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Sat, 16 May 2020 21:30:00 +0200 Subject: [PATCH 022/211] Add option to disable multitarget build --- src/Functions/CMakeLists.txt | 6 +++++ src/Functions/FunctionStartsEndsWith.h | 11 +++++---- src/Functions/FunctionsRandom.h | 11 +++++---- src/Functions/RandXorshift.h | 5 ++++- src/Functions/TargetSpecific.h | 31 ++++++++++++++++++-------- 5 files changed, 46 insertions(+), 18 deletions(-) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 451dfe97a03..8c9cf159e30 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -83,6 +83,12 @@ if(USE_RAPIDJSON) target_include_directories(clickhouse_functions SYSTEM PRIVATE ${RAPIDJSON_INCLUDE_DIR}) endif() +option(ENABLE_MULTITARGET_CODE "" ON) +if (ENABLE_MULTITARGET_CODE) + add_definitions(-DUSE_MULTITARGET_CODE=1) +else() + add_definitions(-DUSE_MULTITARGET_CODE=0) +endif() target_link_libraries(clickhouse_functions PUBLIC "simdxorshift") message(STATUS "Using SIMDXORSHIFT ${SIMDXORSHIFT_LIBRARY}") diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 9435adbe858..f433f9c46c2 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -152,10 +152,13 @@ public: PerformanceAdaptorOptions() ) { - registerImplementation> (TargetArch::SSE42); - registerImplementation> (TargetArch::AVX); - registerImplementation> (TargetArch::AVX2); - registerImplementation>(TargetArch::AVX512F); + if constexpr (UseMultitargetCode) + { + registerImplementation> (TargetArch::SSE42); + registerImplementation> (TargetArch::AVX); + registerImplementation> (TargetArch::AVX2); + registerImplementation>(TargetArch::AVX512F); + } } static FunctionPtr create(const Context & context) diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 8fb1758a60c..a957c0b5e97 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -101,10 +101,13 @@ public: : FunctionPerformanceAdaptor>( PerformanceAdaptorOptions()) { - registerImplementation>(TargetArch::SSE42); - registerImplementation>(TargetArch::AVX); - registerImplementation>(TargetArch::AVX2); - registerImplementation>(TargetArch::AVX512F); + if constexpr (UseMultitargetCode) + { + registerImplementation>(TargetArch::SSE42); + registerImplementation>(TargetArch::AVX); + registerImplementation>(TargetArch::AVX2); + registerImplementation>(TargetArch::AVX512F); + } } static FunctionPtr create(const Context &) diff --git a/src/Functions/RandXorshift.h b/src/Functions/RandXorshift.h index c9e46cf7192..5f3f3c9c04f 100644 --- a/src/Functions/RandXorshift.h +++ b/src/Functions/RandXorshift.h @@ -36,7 +36,10 @@ public: : FunctionPerformanceAdaptor>( PerformanceAdaptorOptions()) { - registerImplementation>(TargetArch::AVX2); + if constexpr (UseMultitargetCode) + { + registerImplementation>(TargetArch::AVX2); + } } static FunctionPtr create(const Context &) diff --git a/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h index f5bd0267c52..0c9eb7357d1 100644 --- a/src/Functions/TargetSpecific.h +++ b/src/Functions/TargetSpecific.h @@ -78,6 +78,10 @@ bool IsArchSupported(TargetArch arch); String ToString(TargetArch arch); +#if USE_MULTITARGET_CODE && defined(__GNUC__) && defined(__x86_64__) + +constexpr bool UseMultitargetCode = true; + #if defined(__clang__) # define BEGIN_AVX512F_SPECIFIC_CODE _Pragma(\ "clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx,avx2,avx512f\"))),apply_to=function)") @@ -88,7 +92,7 @@ String ToString(TargetArch arch); # define BEGIN_SSE42_SPECIFIC_CODE _Pragma(\ "clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx\"))),apply_to=function)") # define END_TARGET_SPECIFIC_CODE _Pragma("clang attribute pop") -#elif defined(__GNUC__) +#else # define BEGIN_AVX512F_SPECIFIC_CODE \ _Pragma("GCC push_options") \ _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2,avx512f,tune=native\")") @@ -103,16 +107,8 @@ String ToString(TargetArch arch); _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,tune=native\")") # define END_TARGET_SPECIFIC_CODE \ _Pragma("GCC pop_options") -#else -# error "Only CLANG and GCC compilers are supported for vectorized code generation" #endif -#define DECLARE_DEFAULT_CODE(...) \ -namespace TargetSpecific::Default { \ - using namespace DB::TargetSpecific::Default; \ - __VA_ARGS__ \ -} - #define DECLARE_SSE42_SPECIFIC_CODE(...) \ BEGIN_SSE42_SPECIFIC_CODE \ namespace TargetSpecific::SSE42 { \ @@ -145,6 +141,23 @@ namespace TargetSpecific::AVX512F { \ } \ END_TARGET_SPECIFIC_CODE +#else + +constexpr bool UseMultitargetCode = false; + +#define DECLARE_SSE42_SPECIFIC_CODE(...) +#define DECLARE_AVX_SPECIFIC_CODE(...) +#define DECLARE_AVX2_SPECIFIC_CODE(...) +#define DECLARE_AVX512F_SPECIFIC_CODE(...) + +#endif + +#define DECLARE_DEFAULT_CODE(...) \ +namespace TargetSpecific::Default { \ + using namespace DB::TargetSpecific::Default; \ + __VA_ARGS__ \ +} + #define DECLARE_MULTITARGET_CODE(...) \ DECLARE_DEFAULT_CODE (__VA_ARGS__) \ DECLARE_SSE42_SPECIFIC_CODE (__VA_ARGS__) \ From 9387981abeb28cdb89faeb55d9a8b1007e637810 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Sun, 17 May 2020 17:13:01 +0200 Subject: [PATCH 023/211] Add afwul option for choosing implementation --- src/Core/Settings.h | 1 + src/Core/SettingsCollection.h | 2 +- src/Functions/FunctionStartsEndsWith.h | 6 +-- src/Functions/FunctionsRandom.h | 9 ++-- src/Functions/PerformanceAdaptors.h | 25 ++++++----- src/Functions/RandXorshift.cpp | 61 ++++++++++++++++++++++++++ src/Functions/RandXorshift.h | 16 ++++--- src/Functions/TargetSpecific.cpp | 8 ++-- 8 files changed, 98 insertions(+), 30 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 142e0872d72..68bebd0b6b0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -424,6 +424,7 @@ struct Settings : public SettingsCollection M(SettingBool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \ M(SettingSeconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \ M(SettingBool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \ + M(SettingString, function_implementation, "", "Choose implementation. If empty enable all of them.", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ diff --git a/src/Core/SettingsCollection.h b/src/Core/SettingsCollection.h index 1fe5762de4c..c34bd1a2990 100644 --- a/src/Core/SettingsCollection.h +++ b/src/Core/SettingsCollection.h @@ -327,7 +327,7 @@ using SettingLogQueriesType = SettingEnum; enum class SettingsBinaryFormat { - OLD, /// Part of the settings are serialized as strings, and other part as varints. This is the old behaviour. + OLD, /// Part of the settings are serialized as strings, and other part as variants. This is the old behaviour. STRINGS, /// All settings are serialized as strings. Before each value the flag `is_ignorable` is serialized. DEFAULT = STRINGS, }; diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index f433f9c46c2..71b02e3b264 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -147,10 +147,8 @@ class FunctionStartsEndsWith : public FunctionPerformanceAdaptor> { public: - FunctionStartsEndsWith(const Context &) - : FunctionPerformanceAdaptor>( - PerformanceAdaptorOptions() - ) + FunctionStartsEndsWith(const Context & context_) + : FunctionPerformanceAdaptor>(context_) { if constexpr (UseMultitargetCode) { diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index a957c0b5e97..a716826d4e1 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -97,9 +97,8 @@ template class FunctionRandom : public FunctionPerformanceAdaptor> { public: - FunctionRandom() - : FunctionPerformanceAdaptor>( - PerformanceAdaptorOptions()) + FunctionRandom(const Context & context_) + : FunctionPerformanceAdaptor>(context_) { if constexpr (UseMultitargetCode) { @@ -110,9 +109,9 @@ public: } } - static FunctionPtr create(const Context &) + static FunctionPtr create(const Context & context) { - return std::make_shared>(); + return std::make_shared>(context); } }; diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index ea3f2ae0b47..eaaa594a4bf 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -4,6 +4,7 @@ #include #include +#include #include @@ -174,9 +175,9 @@ public: using BaseFunctionPtr = FunctionExecutor::BaseFunctionPtr; template - FunctionPerformanceAdaptor(PerformanceAdaptorOptions options_, Params&&... params) + FunctionPerformanceAdaptor(const Context & context_, Params&&... params) : FunctionExecutor(std::forward(params)...) - , options(std::move(options_)) + , context(context_) { if (isImplementationEnabled(DefaultFunction::getImplementationTag())) statistics.emplace_back(); @@ -195,15 +196,17 @@ public: bool isImplementationEnabled(const String & impl_tag) { - if (!options.implementations) - return true; + const String & tag = context.getSettingsRef().function_implementation.value; + return tag.empty() || tag == impl_tag; + // if (!options.implementations) + // return true; - for (const auto & tag : *options.implementations) - { - if (tag == impl_tag) - return true; - } - return false; + // for (const auto & tag : *options.implementations) + // { + // if (tag == impl_tag) + // return true; + // } + // return false; } protected: @@ -249,7 +252,7 @@ protected: private: std::vector impls; // Alternative implementations. PerformanceStatistics statistics; - PerformanceAdaptorOptions options; + const Context & context; }; } diff --git a/src/Functions/RandXorshift.cpp b/src/Functions/RandXorshift.cpp index 652bb90f559..9f1dded700c 100644 --- a/src/Functions/RandXorshift.cpp +++ b/src/Functions/RandXorshift.cpp @@ -19,6 +19,9 @@ DECLARE_DEFAULT_CODE( void RandXorshiftImpl::execute(char * output, size_t size) { + if (size == 0) + return; + char * end = output + size; xorshift128plus_key_s mykey; @@ -89,6 +92,64 @@ void RandXorshiftImpl::execute(char * output, size_t size) ) // DECLARE_AVX2_SPECIFIC_CODE +DECLARE_AVX2_SPECIFIC_CODE( + +void RandXorshiftImpl2::execute(char * output, size_t size) +{ + if (size == 0) + return; + + char * end = output + size; + + avx_xorshift128plus_key_t mykey; + avx_xorshift128plus_init(0xe9ef384566799595ULL ^ reinterpret_cast(output), + 0xa321e1523f4f88c7ULL ^ reinterpret_cast(output), + &mykey); + + avx_xorshift128plus_key_t mykey2; + avx_xorshift128plus_init(0xdfe532a6b5a5eb2cULL ^ reinterpret_cast(output), + 0x21cdf6cd1e22bf9cULL ^ reinterpret_cast(output), + &mykey2); + + constexpr int safe_overwrite = 15; /// How many bytes we can write behind the end. + constexpr int bytes_per_write = 32; + constexpr intptr_t mask = bytes_per_write - 1; + + if (size + safe_overwrite < bytes_per_write) + { + /// size <= 16. + _mm_storeu_si128(reinterpret_cast<__m128i*>(output), + _mm256_extracti128_si256(avx_xorshift128plus(&mykey), 0)); + return; + } + + /// Process head to make output aligned. + _mm256_storeu_si256(reinterpret_cast<__m256i*>(output), avx_xorshift128plus(&mykey)); + output = reinterpret_cast((reinterpret_cast(output) | mask) + 1); + + while ((end - output) + safe_overwrite >= bytes_per_write * 2) + { + _mm256_store_si256(reinterpret_cast<__m256i*>(output), avx_xorshift128plus(&mykey)); + _mm256_store_si256(reinterpret_cast<__m256i*>(output + bytes_per_write), avx_xorshift128plus(&mykey2)); + output += bytes_per_write * 2; + } + + if ((end - output) + safe_overwrite >= bytes_per_write) + { + _mm256_store_si256(reinterpret_cast<__m256i*>(output), avx_xorshift128plus(&mykey)); + output += bytes_per_write; + } + + /// Process tail. (end - output) <= 16. + if ((end - output) > 0) + { + _mm_store_si128(reinterpret_cast<__m128i*>(output), + _mm256_extracti128_si256(avx_xorshift128plus(&mykey), 0)); + } +} + +) // DECLARE_AVX2_SPECIFIC_CODE + struct NameRandXorshift { static constexpr auto name = "randxorshift"; }; using FunctionRandXorshift = FunctionRandomXorshift; struct NameRandXorshift64 { static constexpr auto name = "randxorshift64"; }; diff --git a/src/Functions/RandXorshift.h b/src/Functions/RandXorshift.h index 5f3f3c9c04f..2dd7723ff0a 100644 --- a/src/Functions/RandXorshift.h +++ b/src/Functions/RandXorshift.h @@ -25,6 +25,12 @@ struct RandXorshiftImpl static String getImplementationTag() { return ToString(BuildArch); } }; +struct RandXorshiftImpl2 +{ + static void execute(char * output, size_t size); + static String getImplementationTag() { return ToString(BuildArch) + "_v2"; } +}; + ) // DECLARE_MULTITARGET_CODE template @@ -32,19 +38,19 @@ class FunctionRandomXorshift : public FunctionPerformanceAdaptor> { public: - FunctionRandomXorshift() - : FunctionPerformanceAdaptor>( - PerformanceAdaptorOptions()) + FunctionRandomXorshift(const Context & context_) + : FunctionPerformanceAdaptor>(context_) { if constexpr (UseMultitargetCode) { registerImplementation>(TargetArch::AVX2); + registerImplementation>(TargetArch::AVX2); } } - static FunctionPtr create(const Context &) + static FunctionPtr create(const Context & context) { - return std::make_shared>(); + return std::make_shared>(context); } }; diff --git a/src/Functions/TargetSpecific.cpp b/src/Functions/TargetSpecific.cpp index 4168fb60a59..65f8641ee8e 100644 --- a/src/Functions/TargetSpecific.cpp +++ b/src/Functions/TargetSpecific.cpp @@ -8,13 +8,13 @@ namespace DB UInt32 GetSupportedArches() { UInt32 result = 0; - if (Cpu::haveSSE42()) + if (Cpu::CpuFlagsCache::have_SSE42) result |= static_cast(TargetArch::SSE42); - if (Cpu::haveAVX()) + if (Cpu::CpuFlagsCache::have_AVX) result |= static_cast(TargetArch::AVX); - if (Cpu::haveAVX2()) + if (Cpu::CpuFlagsCache::have_AVX2) result |= static_cast(TargetArch::AVX2); - if (Cpu::haveAVX512F()) + if (Cpu::CpuFlagsCache::have_AVX512F) result |= static_cast(TargetArch::AVX512F); return result; } From 35e4f43ac749a479dee960452e866cb575b588a0 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Mon, 18 May 2020 09:24:22 +0200 Subject: [PATCH 024/211] Add alternative implenetation for rand --- src/Functions/FunctionsRandom.cpp | 46 +++++++++++++++++++++++++++++++ src/Functions/FunctionsRandom.h | 12 ++++++-- src/Functions/RandXorshift.h | 5 ---- 3 files changed, 56 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 11861d2d12c..9c6d90e9e73 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -66,6 +66,52 @@ void RandImpl::execute(char * output, size_t size) /// It is guaranteed (by PaddedPODArray) that we can overwrite up to 15 bytes after end. } +void RandImpl2::execute(char * output, size_t size) +{ + if (size == 0) + return; + + LinearCongruentialGenerator generator0; + LinearCongruentialGenerator generator1; + LinearCongruentialGenerator generator2; + LinearCongruentialGenerator generator3; + LinearCongruentialGenerator generator4; + LinearCongruentialGenerator generator5; + LinearCongruentialGenerator generator6; + LinearCongruentialGenerator generator7; + + seed(generator0, 0xfb4121280b2ab902ULL + reinterpret_cast(output)); + seed(generator1, 0x0121cf76df39c673ULL + reinterpret_cast(output)); + seed(generator2, 0x17ae86e3a19a602fULL + reinterpret_cast(output)); + seed(generator3, 0x8b6e16da7e06d622ULL + reinterpret_cast(output)); + seed(generator4, 0xfb4122280b2ab102ULL + reinterpret_cast(output)); + seed(generator5, 0x0121c276df39c173ULL + reinterpret_cast(output)); + seed(generator6, 0x17ae82e3a19a612fULL + reinterpret_cast(output)); + seed(generator7, 0x8b6e12da7e06d122ULL + reinterpret_cast(output)); + + const char * end = output + size; + + for (; (end - output + 15) <= 32; output += 32) + { + unalignedStore(output, generator0.next()); + unalignedStore(output + 4, generator1.next()); + unalignedStore(output + 8, generator2.next()); + unalignedStore(output + 12, generator3.next()); + unalignedStore(output + 16, generator4.next()); + unalignedStore(output + 20, generator5.next()); + unalignedStore(output + 24, generator6.next()); + unalignedStore(output + 28, generator7.next()); + } + + while (end - output > 0) { + unalignedStore(output, generator0.next()); + unalignedStore(output + 4, generator1.next()); + unalignedStore(output + 8, generator2.next()); + unalignedStore(output + 12, generator3.next()); + output += 16; + } +} + ) //DECLARE_MULTITARGET_CODE } diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index a716826d4e1..443f44a4e44 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -7,8 +7,7 @@ #include #include -// #include "TargetSpecific.h" -// #include "PerformanceAdaptors.h" + namespace DB { @@ -45,6 +44,12 @@ struct RandImpl static String getImplementationTag() { return ToString(BuildArch); } }; +struct RandImpl2 +{ + static void execute(char * output, size_t size); + static String getImplementationTag() { return ToString(BuildArch) + "_v2"; } +}; + ) // DECLARE_MULTITARGET_CODE template @@ -106,6 +111,9 @@ public: registerImplementation>(TargetArch::AVX); registerImplementation>(TargetArch::AVX2); registerImplementation>(TargetArch::AVX512F); + + registerImplementation>(TargetArch::Default); + registerImplementation>(TargetArch::AVX2); } } diff --git a/src/Functions/RandXorshift.h b/src/Functions/RandXorshift.h index 2dd7723ff0a..b74fdeecbef 100644 --- a/src/Functions/RandXorshift.h +++ b/src/Functions/RandXorshift.h @@ -12,11 +12,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - DECLARE_MULTITARGET_CODE( struct RandXorshiftImpl From b2b3ba59420dce136943e4f0275c6f125540676d Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Mon, 18 May 2020 10:48:35 +0200 Subject: [PATCH 025/211] Finaly fix clang build --- src/Functions/FunctionStartsEndsWith.h | 8 ++++---- src/Functions/FunctionsRandom.h | 12 ++++++------ src/Functions/PerformanceAdaptors.h | 2 +- src/Functions/RandXorshift.h | 4 ++-- src/Functions/TargetSpecific.h | 4 ++++ 5 files changed, 17 insertions(+), 13 deletions(-) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 71b02e3b264..b0465ecefa6 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -152,10 +152,10 @@ public: { if constexpr (UseMultitargetCode) { - registerImplementation> (TargetArch::SSE42); - registerImplementation> (TargetArch::AVX); - registerImplementation> (TargetArch::AVX2); - registerImplementation>(TargetArch::AVX512F); + this->template registerImplementation> (TargetArch::SSE42); + this->template registerImplementation> (TargetArch::AVX); + this->template registerImplementation> (TargetArch::AVX2); + this->template registerImplementation>(TargetArch::AVX512F); } } diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 443f44a4e44..eeba5546fc9 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -107,13 +107,13 @@ public: { if constexpr (UseMultitargetCode) { - registerImplementation>(TargetArch::SSE42); - registerImplementation>(TargetArch::AVX); - registerImplementation>(TargetArch::AVX2); - registerImplementation>(TargetArch::AVX512F); + this->template registerImplementation>(TargetArch::SSE42); + this->template registerImplementation>(TargetArch::AVX); + this->template registerImplementation>(TargetArch::AVX2); + this->template registerImplementation>(TargetArch::AVX512F); - registerImplementation>(TargetArch::Default); - registerImplementation>(TargetArch::AVX2); + this->template registerImplementation>(TargetArch::Default); + this->template registerImplementation>(TargetArch::AVX2); } } diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index eaaa594a4bf..0b5e3e10104 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -172,7 +172,7 @@ template class FunctionPerformanceAdaptor : public FunctionExecutor { public: - using BaseFunctionPtr = FunctionExecutor::BaseFunctionPtr; + using BaseFunctionPtr = typename FunctionExecutor::BaseFunctionPtr; template FunctionPerformanceAdaptor(const Context & context_, Params&&... params) diff --git a/src/Functions/RandXorshift.h b/src/Functions/RandXorshift.h index b74fdeecbef..8713d85fdbd 100644 --- a/src/Functions/RandXorshift.h +++ b/src/Functions/RandXorshift.h @@ -38,8 +38,8 @@ public: { if constexpr (UseMultitargetCode) { - registerImplementation>(TargetArch::AVX2); - registerImplementation>(TargetArch::AVX2); + this->template registerImplementation>(TargetArch::AVX2); + this->template registerImplementation>(TargetArch::AVX2); } } diff --git a/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h index 0c9eb7357d1..e69bd22f271 100644 --- a/src/Functions/TargetSpecific.h +++ b/src/Functions/TargetSpecific.h @@ -112,6 +112,7 @@ constexpr bool UseMultitargetCode = true; #define DECLARE_SSE42_SPECIFIC_CODE(...) \ BEGIN_SSE42_SPECIFIC_CODE \ namespace TargetSpecific::SSE42 { \ + void __dummy_function_clang(); \ using namespace DB::TargetSpecific::SSE42; \ __VA_ARGS__ \ } \ @@ -120,6 +121,7 @@ END_TARGET_SPECIFIC_CODE #define DECLARE_AVX_SPECIFIC_CODE(...) \ BEGIN_AVX_SPECIFIC_CODE \ namespace TargetSpecific::AVX { \ + void __dummy_function_clang(); \ using namespace DB::TargetSpecific::AVX; \ __VA_ARGS__ \ } \ @@ -128,6 +130,7 @@ END_TARGET_SPECIFIC_CODE #define DECLARE_AVX2_SPECIFIC_CODE(...) \ BEGIN_AVX2_SPECIFIC_CODE \ namespace TargetSpecific::AVX2 { \ + void __dummy_function_clang(); \ using namespace DB::TargetSpecific::AVX2; \ __VA_ARGS__ \ } \ @@ -136,6 +139,7 @@ END_TARGET_SPECIFIC_CODE #define DECLARE_AVX512F_SPECIFIC_CODE(...) \ BEGIN_AVX512F_SPECIFIC_CODE \ namespace TargetSpecific::AVX512F { \ + void __dummy_function_clang(); \ using namespace DB::TargetSpecific::AVX512F; \ __VA_ARGS__ \ } \ From bd847514baa5194e65c406a93179c2d6d05eae84 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Mon, 18 May 2020 13:44:56 +0200 Subject: [PATCH 026/211] better random --- src/Functions/FunctionsRandom.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 9c6d90e9e73..1de5fb50b8a 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -103,7 +103,7 @@ void RandImpl2::execute(char * output, size_t size) unalignedStore(output + 28, generator7.next()); } - while (end - output > 0) { + if (end - output > 0) { unalignedStore(output, generator0.next()); unalignedStore(output + 4, generator1.next()); unalignedStore(output + 8, generator2.next()); From ea1285328be4579b738c53d97ad8e34b6cf5f3e6 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Mon, 18 May 2020 19:07:36 +0200 Subject: [PATCH 027/211] Fix gcc build, fix PVS error --- src/Functions/FunctionsRandom.cpp | 3 +- src/Functions/TargetSpecific.h | 157 ++++++++++++++++-------------- 2 files changed, 86 insertions(+), 74 deletions(-) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 1de5fb50b8a..fd493d5605b 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -103,7 +103,8 @@ void RandImpl2::execute(char * output, size_t size) unalignedStore(output + 28, generator7.next()); } - if (end - output > 0) { + if (end - output > 0) + { unalignedStore(output, generator0.next()); unalignedStore(output + 4, generator1.next()); unalignedStore(output + 8, generator2.next()); diff --git a/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h index e69bd22f271..11dae939bbd 100644 --- a/src/Functions/TargetSpecific.h +++ b/src/Functions/TargetSpecific.h @@ -2,72 +2,73 @@ #include -/// This file contains macros and helpers for writing platform-dependent code. -/// -/// Macros DECLARE__SPECIFIC_CODE will wrap code inside them into the -/// namespace TargetSpecific:: and enable Arch-specific compile options. -/// Thus, it's allowed to call functions inside these namespaces only after -/// checking platform in runtime (see IsArchSupported() below). -/// -/// For similarities there is a macros DECLARE_DEFAULT_CODE, which wraps code -/// into the namespace TargetSpecific::Default but dosn't specify any additional -/// copile options. -/// -/// Example: -/// -/// DECLARE_DEFAULT_CODE ( -/// int funcImpl() { -/// return 1; -/// } -/// ) // DECLARE_DEFAULT_CODE -/// -/// DECLARE_AVX2_SPECIFIC_CODE ( -/// int funcImpl() { -/// return 2; -/// } -/// ) // DECLARE_DEFAULT_CODE -/// -/// int func() { -/// if (IsArchSupported(TargetArch::AVX2)) -/// return TargetSpecifc::AVX2::funcImpl(); -/// return TargetSpecifc::Default::funcImpl(); -/// } -/// -/// Sometimes code may benefit from compiling with different options. -/// For these purposes use DECLARE_MULTITARGET_CODE macros. It will create several -/// copies of the code and compile it with different options. These copies are -/// available via TargetSpecifc namespaces described above. -/// -/// Inside every TargetSpecific namespace there is a constexpr variable BuildArch, -/// which indicates the target platform for current code. -/// -/// Example: -/// -/// DECLARE_MULTITARGET_CODE( -/// int funcImpl(int size, ...) { -/// int iteration_size = 1; -/// if constexpr (BuildArch == TargetArch::SSE42) -/// iteration_size = 2 -/// else if constexpr (BuildArch == TargetArch::AVX || BuildArch == TargetArch::AVX2) -/// iteration_size = 4; -/// else if constexpr (BuildArch == TargetArch::AVX512) -/// iteration_size = 8; -/// for (int i = 0; i < size; i += iteration_size) -/// ... -/// } -/// ) // DECLARE_MULTITARGET_CODE -/// -/// // All 5 versions of func are available here. Use runtime detection to choose one. -/// -/// If you want to write IFunction or IExecutableFuncionImpl with runtime dispatching, see PerformanceAdaptors.h. +/* This file contains macros and helpers for writing platform-dependent code. + * + * Macros DECLARE__SPECIFIC_CODE will wrap code inside them into the + * namespace TargetSpecific:: and enable Arch-specific compile options. + * Thus, it's allowed to call functions inside these namespaces only after + * checking platform in runtime (see IsArchSupported() below). + * + * For similarities there is a macros DECLARE_DEFAULT_CODE, which wraps code + * into the namespace TargetSpecific::Default but dosn't specify any additional + * copile options. + * + * Example: + * + * DECLARE_DEFAULT_CODE ( + * int funcImpl() { + * return 1; + * } + * ) // DECLARE_DEFAULT_CODE + * + * DECLARE_AVX2_SPECIFIC_CODE ( + * int funcImpl() { + * return 2; + * } + * ) // DECLARE_DEFAULT_CODE + * + * int func() { + * if (IsArchSupported(TargetArch::AVX2)) + * return TargetSpecifc::AVX2::funcImpl(); + * return TargetSpecifc::Default::funcImpl(); + * } + * + * Sometimes code may benefit from compiling with different options. + * For these purposes use DECLARE_MULTITARGET_CODE macros. It will create several + * copies of the code and compile it with different options. These copies are + * available via TargetSpecifc namespaces described above. + * + * Inside every TargetSpecific namespace there is a constexpr variable BuildArch, + * which indicates the target platform for current code. + * + * Example: + * + * DECLARE_MULTITARGET_CODE( + * int funcImpl(int size, ...) { + * int iteration_size = 1; + * if constexpr (BuildArch == TargetArch::SSE42) + * iteration_size = 2 + * else if constexpr (BuildArch == TargetArch::AVX || BuildArch == TargetArch::AVX2) + * iteration_size = 4; + * else if constexpr (BuildArch == TargetArch::AVX512) + * iteration_size = 8; + * for (int i = 0; i < size; i += iteration_size) + * ... + * } + * ) // DECLARE_MULTITARGET_CODE + * + * // All 5 versions of func are available here. Use runtime detection to choose one. + * + * If you want to write IFunction or IExecutableFuncionImpl with runtime dispatching, see PerformanceAdaptors.h. + */ namespace DB { enum class TargetArch : UInt32 { - Default = 0, /// Without any additional compiler options. - SSE42 = (1 << 0), /// SSE4.2 + Default = 0, /// Without any additional compiler options. + SSE42 = (1 << 0), /// SSE4.2 AVX = (1 << 1), AVX2 = (1 << 2), AVX512F = (1 << 3), @@ -83,15 +84,21 @@ String ToString(TargetArch arch); constexpr bool UseMultitargetCode = true; #if defined(__clang__) -# define BEGIN_AVX512F_SPECIFIC_CODE _Pragma(\ - "clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx,avx2,avx512f\"))),apply_to=function)") -# define BEGIN_AVX2_SPECIFIC_CODE _Pragma(\ - "clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx,avx2\"))),apply_to=function)") -# define BEGIN_AVX_SPECIFIC_CODE _Pragma(\ - "clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx\"))),apply_to=function)") -# define BEGIN_SSE42_SPECIFIC_CODE _Pragma(\ - "clang attribute push (__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx\"))),apply_to=function)") -# define END_TARGET_SPECIFIC_CODE _Pragma("clang attribute pop") +# define BEGIN_AVX512F_SPECIFIC_CODE \ + _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx,avx2,avx512f\"))),apply_to=function)") +# define BEGIN_AVX2_SPECIFIC_CODE \ + _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx,avx2\"))),apply_to=function)") +# define BEGIN_AVX_SPECIFIC_CODE \ + _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx\"))),apply_to=function)") +# define BEGIN_SSE42_SPECIFIC_CODE \ + _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx\"))),apply_to=function)") +# define END_TARGET_SPECIFIC_CODE \ + _Pragma("clang attribute pop") + +/* Clang shows warning when there aren't any objects to apply pragma. + * To prevent this warning we define this function inside every macros with pragmas. + */ +# define DUMMY_FUNCTION_DEFINITION void __dummy_function_definition(); #else # define BEGIN_AVX512F_SPECIFIC_CODE \ _Pragma("GCC push_options") \ @@ -107,12 +114,16 @@ constexpr bool UseMultitargetCode = true; _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,tune=native\")") # define END_TARGET_SPECIFIC_CODE \ _Pragma("GCC pop_options") + +/* GCC doesn't show such warning, we don't need to define anything. + */ +# define DUMMY_FUNCTION_DEFINITION #endif #define DECLARE_SSE42_SPECIFIC_CODE(...) \ BEGIN_SSE42_SPECIFIC_CODE \ namespace TargetSpecific::SSE42 { \ - void __dummy_function_clang(); \ + DUMMY_FUNCTION_DEFINITION \ using namespace DB::TargetSpecific::SSE42; \ __VA_ARGS__ \ } \ @@ -121,7 +132,7 @@ END_TARGET_SPECIFIC_CODE #define DECLARE_AVX_SPECIFIC_CODE(...) \ BEGIN_AVX_SPECIFIC_CODE \ namespace TargetSpecific::AVX { \ - void __dummy_function_clang(); \ + DUMMY_FUNCTION_DEFINITION \ using namespace DB::TargetSpecific::AVX; \ __VA_ARGS__ \ } \ @@ -130,7 +141,7 @@ END_TARGET_SPECIFIC_CODE #define DECLARE_AVX2_SPECIFIC_CODE(...) \ BEGIN_AVX2_SPECIFIC_CODE \ namespace TargetSpecific::AVX2 { \ - void __dummy_function_clang(); \ + DUMMY_FUNCTION_DEFINITION \ using namespace DB::TargetSpecific::AVX2; \ __VA_ARGS__ \ } \ @@ -139,7 +150,7 @@ END_TARGET_SPECIFIC_CODE #define DECLARE_AVX512F_SPECIFIC_CODE(...) \ BEGIN_AVX512F_SPECIFIC_CODE \ namespace TargetSpecific::AVX512F { \ - void __dummy_function_clang(); \ + DUMMY_FUNCTION_DEFINITION \ using namespace DB::TargetSpecific::AVX512F; \ __VA_ARGS__ \ } \ From c524642d245487315d8a9b37d7d3783c8b66139f Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Mon, 18 May 2020 22:07:24 +0200 Subject: [PATCH 028/211] Delete awful template PerformanceAdaptro and add simple ImplementationSelector instead --- src/Functions/FunctionStartsEndsWith.h | 29 ++-- src/Functions/FunctionsRandom.h | 34 +++-- src/Functions/PerformanceAdaptors.h | 179 ++++++++++--------------- src/Functions/RandXorshift.h | 23 +++- 4 files changed, 134 insertions(+), 131 deletions(-) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index b0465ecefa6..76aa4530c99 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -143,26 +143,39 @@ private: ) // DECLARE_MULTITARGET_CODE template -class FunctionStartsEndsWith - : public FunctionPerformanceAdaptor> +class FunctionStartsEndsWith : public TargetSpecific::Default::FunctionStartsEndsWith { public: - FunctionStartsEndsWith(const Context & context_) - : FunctionPerformanceAdaptor>(context_) + FunctionStartsEndsWith(const Context & context) : selector(context) { + selector.registerImplementation>(); + if constexpr (UseMultitargetCode) { - this->template registerImplementation> (TargetArch::SSE42); - this->template registerImplementation> (TargetArch::AVX); - this->template registerImplementation> (TargetArch::AVX2); - this->template registerImplementation>(TargetArch::AVX512F); + selector.registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); } } + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + selector.selectAndExecute(block, arguments, result, input_rows_count); + } + static FunctionPtr create(const Context & context) { return std::make_shared>(context); } + +private: + ImplementationSelector selector; }; } diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index eeba5546fc9..995f8ffeb9c 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -99,28 +99,44 @@ public: }; template -class FunctionRandom : public FunctionPerformanceAdaptor> +class FunctionRandom : public FunctionRandomImpl { public: - FunctionRandom(const Context & context_) - : FunctionPerformanceAdaptor>(context_) + FunctionRandom(const Context & context) : selector(context) { + selector.registerImplementation>(); + selector.registerImplementation>(); + if constexpr (UseMultitargetCode) { - this->template registerImplementation>(TargetArch::SSE42); - this->template registerImplementation>(TargetArch::AVX); - this->template registerImplementation>(TargetArch::AVX2); - this->template registerImplementation>(TargetArch::AVX512F); + selector.registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); - this->template registerImplementation>(TargetArch::Default); - this->template registerImplementation>(TargetArch::AVX2); + selector.registerImplementation>(); } } + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + selector.selectAndExecute(block, arguments, result, input_rows_count); + } + static FunctionPtr create(const Context & context) { return std::make_shared>(context); } + +private: + ImplementationSelector selector; }; } diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index 0b5e3e10104..b86730952fb 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -117,123 +117,67 @@ struct PerformanceStatistics PerformanceStatistics(ssize_t choose_method_) : choose_method(choose_method_) {} }; -struct PerformanceAdaptorOptions -{ - std::optional> implementations; -}; - -/// Redirects IExecutableFunctionImpl::execute() and IFunction:executeImpl() to executeFunctionImpl(); -template -class FunctionExecutor; - -template -class FunctionExecutor>> - : public DefaultFunction +/* Class which is used to store implementations for the function and selecting the best one to run + * based on processor architecture and statistics from previous runs. + * + * FunctionInterface is typically IFunction or IExecutableFunctionImpl, but practically it can be + * any interface that contains "execute" method (IFunction is an exception and is supported as well). + * + * Example of usage: + * + * class MyDefaulImpl : public IFunction {...}; + * class MySecondImpl : public IFunction {...}; + * class MyAVX2Impl : public IFunction {...}; + * + * /// All methods but execute/executeImpl are usually not bottleneck, so just use them from + * /// default implementation. + * class MyFunction : public MyDefaultImpl + * { + * MyFunction(const Context & context) : selector(context) { + * /// Register all implementations in constructor. + * /// There could be as many implementation for every target as you want. + * selector.registerImplementation(); + * selector.registerImplementation(); + * selector.registreImplementation(); + * } + * + * void executeImpl(...) override { + * selector.selectAndExecute(...); + * } + * + * static FunctionPtr create(const Context & context) { + * return std::make_shared(context); + * } + * private: + * ImplementationSelector selector; + * }; + */ +template +class ImplementationSelector { public: - using BaseFunctionPtr = ExecutableFunctionImplPtr; + using ImplementationPtr = std::shared_ptr; - template - FunctionExecutor(Args&&... args) : DefaultFunction(std::forward(args)...) {} + ImplementationSelector(const Context & context_) : context(context_) {} - virtual void executeFunctionImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0; - - virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + /* Select the best implementation based on previous runs. + * If FunctionInterface is IFunction, then "executeImpl" method of the implementation will be called + * and "execute" otherwise. + */ + void selectAndExecute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) { - executeFunctionImpl(block, arguments, result, input_rows_count); - } -}; - -template -class FunctionExecutor>> - : public DefaultFunction -{ -public: - using BaseFunctionPtr = FunctionPtr; - - template - FunctionExecutor(Args&&... args) : DefaultFunction(std::forward(args)...) {} - - virtual void executeFunctionImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0; - - virtual void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - executeFunctionImpl(block, arguments, result, input_rows_count); - } -}; - -/// Combine several IExecutableFunctionImpl into one. -/// All the implementations should be equivalent. -/// Implementation to execute will be selected based on performance on previous runs. -/// DefaultFunction should be executable on every supported platform, while alternative implementations -/// could use extended set of instructions (AVX, NEON, etc). -/// It's convenient to inherit your func from this and register all alternative implementations in the constructor. -template -class FunctionPerformanceAdaptor : public FunctionExecutor -{ -public: - using BaseFunctionPtr = typename FunctionExecutor::BaseFunctionPtr; - - template - FunctionPerformanceAdaptor(const Context & context_, Params&&... params) - : FunctionExecutor(std::forward(params)...) - , context(context_) - { - if (isImplementationEnabled(DefaultFunction::getImplementationTag())) - statistics.emplace_back(); - } - - /// Register alternative implementation. - template - void registerImplementation(TargetArch arch, Params&&... params) - { - if (IsArchSupported(arch) && isImplementationEnabled(Function::getImplementationTag())) - { - impls.emplace_back(std::make_shared(std::forward(params)...)); - statistics.emplace_back(); - } - } - - bool isImplementationEnabled(const String & impl_tag) - { - const String & tag = context.getSettingsRef().function_implementation.value; - return tag.empty() || tag == impl_tag; - // if (!options.implementations) - // return true; - - // for (const auto & tag : *options.implementations) - // { - // if (tag == impl_tag) - // return true; - // } - // return false; - } - -protected: - virtual void executeFunctionImpl(Block & block, const ColumnNumbers & arguments, - size_t result, size_t input_rows_count) override - { - if (statistics.empty()) - throw Exception("All available implementations are disabled by user config", + if (implementations.empty()) + throw Exception("There are no available implementations for function " "TODO(dakovalkov): add name", ErrorCodes::NO_SUITABLE_FUNCTION_IMPLEMENTATION); auto id = statistics.select(); Stopwatch watch; - if (id == impls.size()) - { - if constexpr (std::is_base_of_v) - DefaultFunction::executeImpl(block, arguments, result, input_rows_count); - else - DefaultFunction::execute(block, arguments, result, input_rows_count); - } + if constexpr (std::is_same_v) + implementations[id]->executeImpl(block, arguments, result, input_rows_count); else - { - if constexpr (std::is_base_of_v) - impls[id]->executeImpl(block, arguments, result, input_rows_count); - else - impls[id]->execute(block, arguments, result, input_rows_count); - } + implementations[id]->execute(block, arguments, result, input_rows_count); + watch.stop(); // TODO(dakovalkov): Calculate something more informative. @@ -249,10 +193,29 @@ protected: } } + /* Register new implementation for function. + * + * Arch - required instruction set for running the implementation. It's guarantied that no one method would + * be called (even the constructor and static methods) if the processor doesn't support this instruction set. + * + * FunctionImpl - implementation, should be inherited from template argument FunctionInterface. + * + * All function arguments will be forwarded to the implementation constructor. + */ + template + void registerImplementation(Args&&... args) + { + if (IsArchSupported(Arch)) + { + implementations.emplace_back(std::make_shared(std::forward(args)...)); + statistics.emplace_back(); + } + } + private: - std::vector impls; // Alternative implementations. - PerformanceStatistics statistics; const Context & context; + std::vector implementations; + PerformanceStatistics statistics; }; } diff --git a/src/Functions/RandXorshift.h b/src/Functions/RandXorshift.h index 8713d85fdbd..49655d637f2 100644 --- a/src/Functions/RandXorshift.h +++ b/src/Functions/RandXorshift.h @@ -29,24 +29,35 @@ struct RandXorshiftImpl2 ) // DECLARE_MULTITARGET_CODE template -class FunctionRandomXorshift - : public FunctionPerformanceAdaptor> +class FunctionRandomXorshift : public FunctionRandomImpl { public: - FunctionRandomXorshift(const Context & context_) - : FunctionPerformanceAdaptor>(context_) + FunctionRandomXorshift(const Context & context) : selector(context) { + selector.registerImplementation>(); + if constexpr (UseMultitargetCode) { - this->template registerImplementation>(TargetArch::AVX2); - this->template registerImplementation>(TargetArch::AVX2); + selector.registerImplementation>(); + selector.registerImplementation>(); } } + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + selector.selectAndExecute(block, arguments, result, input_rows_count); + } + static FunctionPtr create(const Context & context) { return std::make_shared>(context); } + +private: + ImplementationSelector selector; }; } From 66d530e90190b38cf2ce766403eb811342fb3f3a Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Tue, 19 May 2020 12:54:41 +0200 Subject: [PATCH 029/211] fast rand for performance tests --- src/Functions/FunctionsRandom.cpp | 165 +++++++++++++++++++++++++--- src/Functions/FunctionsRandom.h | 79 +++++++++---- src/Functions/PerformanceAdaptors.h | 9 +- src/Functions/generateUUIDv4.cpp | 2 +- src/Functions/randConstant.cpp | 2 +- 5 files changed, 220 insertions(+), 37 deletions(-) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index fd493d5605b..d0d25e56c60 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -7,8 +7,15 @@ namespace DB { +/* + +// TODO(dakovalkov): remove this workaround. +#pragma GCC diagnostic ignored "-Wvector-operation-performance" + DECLARE_MULTITARGET_CODE( +*/ + namespace { /// NOTE Probably @@ -80,22 +87,22 @@ void RandImpl2::execute(char * output, size_t size) LinearCongruentialGenerator generator6; LinearCongruentialGenerator generator7; - seed(generator0, 0xfb4121280b2ab902ULL + reinterpret_cast(output)); - seed(generator1, 0x0121cf76df39c673ULL + reinterpret_cast(output)); - seed(generator2, 0x17ae86e3a19a602fULL + reinterpret_cast(output)); - seed(generator3, 0x8b6e16da7e06d622ULL + reinterpret_cast(output)); - seed(generator4, 0xfb4122280b2ab102ULL + reinterpret_cast(output)); - seed(generator5, 0x0121c276df39c173ULL + reinterpret_cast(output)); - seed(generator6, 0x17ae82e3a19a612fULL + reinterpret_cast(output)); - seed(generator7, 0x8b6e12da7e06d122ULL + reinterpret_cast(output)); + seed(generator0, 0xfaaae481acb5874aULL + reinterpret_cast(output)); + seed(generator1, 0x3181a34f32887db6ULL + reinterpret_cast(output)); + seed(generator2, 0xb6970e4a91b66afdULL + reinterpret_cast(output)); + seed(generator3, 0xc16062649e83dc13ULL + reinterpret_cast(output)); + seed(generator4, 0xbb093972da5c8d92ULL + reinterpret_cast(output)); + seed(generator5, 0xc37dcc410dcfed31ULL + reinterpret_cast(output)); + seed(generator6, 0x45e1526b7a4367d5ULL + reinterpret_cast(output)); + seed(generator7, 0x99c2759203868a7fULL + reinterpret_cast(output)); const char * end = output + size; for (; (end - output + 15) <= 32; output += 32) { - unalignedStore(output, generator0.next()); - unalignedStore(output + 4, generator1.next()); - unalignedStore(output + 8, generator2.next()); + unalignedStore(output, generator0.next()); + unalignedStore(output + 4, generator1.next()); + unalignedStore(output + 8, generator2.next()); unalignedStore(output + 12, generator3.next()); unalignedStore(output + 16, generator4.next()); unalignedStore(output + 20, generator5.next()); @@ -105,14 +112,144 @@ void RandImpl2::execute(char * output, size_t size) if (end - output > 0) { - unalignedStore(output, generator0.next()); - unalignedStore(output + 4, generator1.next()); - unalignedStore(output + 8, generator2.next()); + unalignedStore(output, generator0.next()); + unalignedStore(output + 4, generator1.next()); + unalignedStore(output + 8, generator2.next()); unalignedStore(output + 12, generator3.next()); output += 16; } } +/* + +typedef UInt64 UInt64x16 __attribute__ ((vector_size (128))); +typedef UInt64 UInt64x8 __attribute__ ((vector_size (64))); +typedef UInt64 UInt64x4 __attribute__ ((vector_size (32))); + +typedef UInt32 UInt32x16 __attribute__ ((vector_size (64))); +typedef UInt32 UInt32x8 __attribute__ ((vector_size (32))); +typedef UInt32 UInt32x4 __attribute__ ((vector_size (16))); + +void RandImpl3::execute(char * output, size_t size) +{ + if (size == 0) + return; + + char * end = output + size; + + UInt64x4 generators = { + 0xfb4121280b2ab902ULL + reinterpret_cast(output), + 0x0121cf76df39c673ULL + reinterpret_cast(output), + 0x17ae86e3a19a602fULL + reinterpret_cast(output), + 0x8b6e16da7e06d622ULL + reinterpret_cast(output), + }; + + constexpr int bytes_per_write = sizeof(UInt32x4); + constexpr int safe_overwrite = 15; + + while ((end - output) + safe_overwrite >= bytes_per_write) + { + generators *= LinearCongruentialGenerator::a; + generators += LinearCongruentialGenerator::c; + unalignedStore(output, __builtin_convertvector(generators, UInt32x4)); + output += bytes_per_write; + } +} + +void RandImpl4::execute(char * output, size_t size) +{ + if (size == 0) + return; + + char * end = output + size; + + UInt64x8 generators = { + 0x5f186ce5faee450bULL + reinterpret_cast(output), + 0x9adb2ca3c72ac2eeULL + reinterpret_cast(output), + 0x07acf8bfa2537705ULL + reinterpret_cast(output), + 0x692b1b533834db92ULL + reinterpret_cast(output), + 0x5148b84cdda30081ULL + reinterpret_cast(output), + 0xe17b8a75a301ad47ULL + reinterpret_cast(output), + 0x6d4a5d69ed2a5f56ULL + reinterpret_cast(output), + 0x114e23266201b333ULL + reinterpret_cast(output), + }; + + constexpr int bytes_per_write = sizeof(UInt32x8); + constexpr int safe_overwrite = 15; + + while ((end - output) + safe_overwrite >= bytes_per_write) + { + generators *= LinearCongruentialGenerator::a; + generators += LinearCongruentialGenerator::c; + unalignedStore(output, __builtin_convertvector(generators, UInt32x8)); + output += bytes_per_write; + } + + if ((end - output) > 0) + { + generators *= LinearCongruentialGenerator::a; + generators += LinearCongruentialGenerator::c; + UInt32x8 values = __builtin_convertvector(generators, UInt32x8); + for (int i = 0; (end - output) > 0; ++i) + { + unalignedStore(output, values[i]); + output += sizeof(UInt32); + } + } +} + +void RandImpl5::execute(char * output, size_t size) +{ + if (size == 0) + return; + + char * end = output + size; + + UInt64x16 generators = { + 0xfb4121280b2ab902ULL + reinterpret_cast(output), + 0x0121cf76df39c673ULL + reinterpret_cast(output), + 0x17ae86e3a19a602fULL + reinterpret_cast(output), + 0x8b6e16da7e06d622ULL + reinterpret_cast(output), + 0xfb4121f80b2ab902ULL + reinterpret_cast(output), + 0x0122cf767f39c633ULL + reinterpret_cast(output), + 0x14ae86e3a79a502fULL + reinterpret_cast(output), + 0x876316da7e06d622ULL + reinterpret_cast(output), + 0xfb4821280b2ab912ULL + reinterpret_cast(output), + 0x0126cf76df39c633ULL + reinterpret_cast(output), + 0x17a486e3a19a602fULL + reinterpret_cast(output), + 0x8b6216da7e08d622ULL + reinterpret_cast(output), + 0xfb4101f80b5ab902ULL + reinterpret_cast(output), + 0x01226f767f34c633ULL + reinterpret_cast(output), + 0x14ae86e3a75a502fULL + reinterpret_cast(output), + 0x876e36da7e36d622ULL + reinterpret_cast(output), + }; + + constexpr int bytes_per_write = sizeof(UInt32x16); + constexpr int safe_overwrite = 15; + + while ((end - output) + safe_overwrite >= bytes_per_write) + { + generators *= LinearCongruentialGenerator::a; + generators += LinearCongruentialGenerator::c; + unalignedStore(output, __builtin_convertvector(generators, UInt32x16)); + output += bytes_per_write; + } + + if ((end - output) > 0) + { + generators *= LinearCongruentialGenerator::a; + generators += LinearCongruentialGenerator::c; + UInt32x16 values = __builtin_convertvector(generators, UInt32x16); + for (int i = 0; (end - output) > 0; ++i) + { + unalignedStore(output, values[i]); + output += sizeof(UInt32); + } + } +} + ) //DECLARE_MULTITARGET_CODE +*/ + } diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 995f8ffeb9c..9a06d8df7a3 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -36,22 +36,48 @@ namespace ErrorCodes * This means that the timer must be of sufficient resolution to give different values to each block. */ +/* + DECLARE_MULTITARGET_CODE( +*/ + struct RandImpl { static void execute(char * output, size_t size); - static String getImplementationTag() { return ToString(BuildArch); } + static String getImplementationTag() { return ToString(TargetArch::Default); } }; struct RandImpl2 { static void execute(char * output, size_t size); - static String getImplementationTag() { return ToString(BuildArch) + "_v2"; } + static String getImplementationTag() { return ToString(TargetArch::Default) + "_v2"; } +}; + +/* + +struct RandImpl3 +{ + static void execute(char * output, size_t size); + static String getImplementationTag() { return ToString(BuildArch) + "_v3"; } +}; + +struct RandImpl4 +{ + static void execute(char * output, size_t size); + static String getImplementationTag() { return ToString(BuildArch) + "_v4"; } +}; + +struct RandImpl5 +{ + static void execute(char * output, size_t size); + static String getImplementationTag() { return ToString(BuildArch) + "_v5"; } }; ) // DECLARE_MULTITARGET_CODE +*/ + template class FunctionRandomImpl : public IFunction { @@ -99,30 +125,45 @@ public: }; template -class FunctionRandom : public FunctionRandomImpl +class FunctionRandom : public FunctionRandomImpl { public: FunctionRandom(const Context & context) : selector(context) { + // selector.registerImplementation>(); selector.registerImplementation>(); - selector.registerImplementation>(); + FunctionRandomImpl>(); - if constexpr (UseMultitargetCode) - { - selector.registerImplementation>(); - selector.registerImplementation>(); - selector.registerImplementation>(); - selector.registerImplementation>(); + // if constexpr (UseMultitargetCode) + // { + // selector.registerImplementation>(); + // selector.registerImplementation>(); + // selector.registerImplementation>(); + // selector.registerImplementation>(); - selector.registerImplementation>(); - } + // selector.registerImplementation>(); + + // selector.registerImplementation>(); + // selector.registerImplementation>(); + + // selector.registerImplementation>(); + // selector.registerImplementation>(); + + // selector.registerImplementation>(); + // selector.registerImplementation>(); + // } } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index b86730952fb..717ad196e61 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -207,8 +207,13 @@ public: { if (IsArchSupported(Arch)) { - implementations.emplace_back(std::make_shared(std::forward(args)...)); - statistics.emplace_back(); + // TODO(dakovalkov): make this option better. + const auto & choose_impl = context.getSettingsRef().function_implementation.value; + if (choose_impl.empty() || choose_impl == FunctionImpl::getImplementationTag()) + { + implementations.emplace_back(std::make_shared(std::forward(args)...)); + statistics.emplace_back(); + } } } diff --git a/src/Functions/generateUUIDv4.cpp b/src/Functions/generateUUIDv4.cpp index d543226ba5c..4db3bd4c73d 100644 --- a/src/Functions/generateUUIDv4.cpp +++ b/src/Functions/generateUUIDv4.cpp @@ -33,7 +33,7 @@ public: size_t size = input_rows_count; vec_to.resize(size); // TODO(dakovalkov): rewrite this workaround - TargetSpecific::Default::RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(UInt128)); + RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(UInt128)); for (UInt128 & uuid: vec_to) { diff --git a/src/Functions/randConstant.cpp b/src/Functions/randConstant.cpp index 3eba5abf10d..163f943d206 100644 --- a/src/Functions/randConstant.cpp +++ b/src/Functions/randConstant.cpp @@ -100,7 +100,7 @@ public: typename ColumnVector::Container vec_to(1); // TODO(dakovalkov): Rewrite this workaround - TargetSpecific::Default::RandImpl::execute(reinterpret_cast(vec_to.data()), sizeof(ToType)); + RandImpl::execute(reinterpret_cast(vec_to.data()), sizeof(ToType)); ToType value = vec_to[0]; return std::make_unique>(value, argument_types, return_type); From 90bc3e6136a751aed685d0fd2b7a6187652e6853 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Wed, 20 May 2020 14:42:21 +0200 Subject: [PATCH 030/211] More rand implementations --- src/Functions/FunctionsRandom.cpp | 317 +++++++++++++++++------------- src/Functions/FunctionsRandom.h | 125 ++++++++---- src/Functions/generateUUIDv4.cpp | 2 +- src/Functions/randConstant.cpp | 2 +- 4 files changed, 272 insertions(+), 174 deletions(-) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index d0d25e56c60..496e0edcc5a 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -3,19 +3,18 @@ #include #include #include +#include namespace DB { -/* - // TODO(dakovalkov): remove this workaround. -#pragma GCC diagnostic ignored "-Wvector-operation-performance" +#if !defined(__clang__) +# pragma GCC diagnostic ignored "-Wvector-operation-performance" +#endif DECLARE_MULTITARGET_CODE( -*/ - namespace { /// NOTE Probably @@ -45,10 +44,16 @@ namespace } }; - void seed(LinearCongruentialGenerator & generator, intptr_t additional_seed) + UInt64 calcSeed(UInt64 rand_seed, UInt64 additional_seed) { - generator.seed(intHash64(randomSeed() ^ intHash64(additional_seed))); + return intHash64(rand_seed ^ intHash64(additional_seed)); } + + void seed(LinearCongruentialGenerator & generator, UInt64 rand_seed, intptr_t additional_seed) + { + generator.seed(calcSeed(rand_seed, additional_seed)); + } + } void RandImpl::execute(char * output, size_t size) @@ -58,10 +63,12 @@ void RandImpl::execute(char * output, size_t size) LinearCongruentialGenerator generator2; LinearCongruentialGenerator generator3; - seed(generator0, 0xfb4121280b2ab902ULL + reinterpret_cast(output)); - seed(generator1, 0x0121cf76df39c673ULL + reinterpret_cast(output)); - seed(generator2, 0x17ae86e3a19a602fULL + reinterpret_cast(output)); - seed(generator3, 0x8b6e16da7e06d622ULL + reinterpret_cast(output)); + UInt64 rand_seed = randomSeed(); + + seed(generator0, rand_seed, 0xfb4121280b2ab902ULL + reinterpret_cast(output)); + seed(generator1, rand_seed, 0x0121cf76df39c673ULL + reinterpret_cast(output)); + seed(generator2, rand_seed, 0x17ae86e3a19a602fULL + reinterpret_cast(output)); + seed(generator3, rand_seed, 0x8b6e16da7e06d622ULL + reinterpret_cast(output)); for (const char * end = output + size; output < end; output += 16) { @@ -73,55 +80,6 @@ void RandImpl::execute(char * output, size_t size) /// It is guaranteed (by PaddedPODArray) that we can overwrite up to 15 bytes after end. } -void RandImpl2::execute(char * output, size_t size) -{ - if (size == 0) - return; - - LinearCongruentialGenerator generator0; - LinearCongruentialGenerator generator1; - LinearCongruentialGenerator generator2; - LinearCongruentialGenerator generator3; - LinearCongruentialGenerator generator4; - LinearCongruentialGenerator generator5; - LinearCongruentialGenerator generator6; - LinearCongruentialGenerator generator7; - - seed(generator0, 0xfaaae481acb5874aULL + reinterpret_cast(output)); - seed(generator1, 0x3181a34f32887db6ULL + reinterpret_cast(output)); - seed(generator2, 0xb6970e4a91b66afdULL + reinterpret_cast(output)); - seed(generator3, 0xc16062649e83dc13ULL + reinterpret_cast(output)); - seed(generator4, 0xbb093972da5c8d92ULL + reinterpret_cast(output)); - seed(generator5, 0xc37dcc410dcfed31ULL + reinterpret_cast(output)); - seed(generator6, 0x45e1526b7a4367d5ULL + reinterpret_cast(output)); - seed(generator7, 0x99c2759203868a7fULL + reinterpret_cast(output)); - - const char * end = output + size; - - for (; (end - output + 15) <= 32; output += 32) - { - unalignedStore(output, generator0.next()); - unalignedStore(output + 4, generator1.next()); - unalignedStore(output + 8, generator2.next()); - unalignedStore(output + 12, generator3.next()); - unalignedStore(output + 16, generator4.next()); - unalignedStore(output + 20, generator5.next()); - unalignedStore(output + 24, generator6.next()); - unalignedStore(output + 28, generator7.next()); - } - - if (end - output > 0) - { - unalignedStore(output, generator0.next()); - unalignedStore(output + 4, generator1.next()); - unalignedStore(output + 8, generator2.next()); - unalignedStore(output + 12, generator3.next()); - output += 16; - } -} - -/* - typedef UInt64 UInt64x16 __attribute__ ((vector_size (128))); typedef UInt64 UInt64x8 __attribute__ ((vector_size (64))); typedef UInt64 UInt64x4 __attribute__ ((vector_size (32))); @@ -130,58 +88,85 @@ typedef UInt32 UInt32x16 __attribute__ ((vector_size (64))); typedef UInt32 UInt32x8 __attribute__ ((vector_size (32))); typedef UInt32 UInt32x4 __attribute__ ((vector_size (16))); -void RandImpl3::execute(char * output, size_t size) +template +struct DummyStruct; + +template <> +struct DummyStruct<4> { + using UInt64Type = UInt64x4; + using UInt32Type = UInt32x4; +}; +template <> +struct DummyStruct<8> +{ + using UInt64Type = UInt64x8; + using UInt32Type = UInt32x8; +}; +template <> +struct DummyStruct<16> +{ + using UInt64Type = UInt64x16; + using UInt32Type = UInt32x16; +}; + +template +using VecUInt64 = typename DummyStruct::UInt64Type; +template +using VecUInt32 = typename DummyStruct::UInt32Type; + +namespace { + +constexpr std::array random_numbers = { + 0x0c8ff307dabc0c4cULL, + 0xf4bce78bf3821c1bULL, + 0x4eb628a1e189c21aULL, + 0x85ae000d253e0dbcULL, + + 0xc98073e6480f8a10ULL, + 0xb17e9b70a084d570ULL, + 0x1361c752b768da8cULL, + 0x3d915f60c06d144dULL, + + 0xd5bc9b7aced79587ULL, + 0x66c28000ba8a66cfULL, + 0x0fb58da7a48820f5ULL, + 0x540ee1b57aa861a1ULL, + + 0x212f11936ef2db04ULL, + 0xa3939cd900edcc58ULL, + 0xc676c84420170102ULL, + 0xcbdc824e8b4bf3edULL, +}; + +}; + +template +void RandVecImpl::execute(char * output, size_t size) +{ + static_assert(VectorSize >= 4); + static_assert(VectorSize <= random_numbers.size()); + if (size == 0) return; char * end = output + size; - UInt64x4 generators = { - 0xfb4121280b2ab902ULL + reinterpret_cast(output), - 0x0121cf76df39c673ULL + reinterpret_cast(output), - 0x17ae86e3a19a602fULL + reinterpret_cast(output), - 0x8b6e16da7e06d622ULL + reinterpret_cast(output), - }; - - constexpr int bytes_per_write = sizeof(UInt32x4); constexpr int safe_overwrite = 15; + constexpr int bytes_per_write = sizeof(VecUInt32); + + UInt64 rand_seed = randomSeed(); + + VecUInt64 generators{}; + for (int i = 0; i < VectorSize; ++i) + generators[i] = calcSeed(rand_seed, random_numbers[VectorSize] + reinterpret_cast(output)); while ((end - output) + safe_overwrite >= bytes_per_write) { generators *= LinearCongruentialGenerator::a; generators += LinearCongruentialGenerator::c; - unalignedStore(output, __builtin_convertvector(generators, UInt32x4)); - output += bytes_per_write; - } -} - -void RandImpl4::execute(char * output, size_t size) -{ - if (size == 0) - return; - - char * end = output + size; - - UInt64x8 generators = { - 0x5f186ce5faee450bULL + reinterpret_cast(output), - 0x9adb2ca3c72ac2eeULL + reinterpret_cast(output), - 0x07acf8bfa2537705ULL + reinterpret_cast(output), - 0x692b1b533834db92ULL + reinterpret_cast(output), - 0x5148b84cdda30081ULL + reinterpret_cast(output), - 0xe17b8a75a301ad47ULL + reinterpret_cast(output), - 0x6d4a5d69ed2a5f56ULL + reinterpret_cast(output), - 0x114e23266201b333ULL + reinterpret_cast(output), - }; - - constexpr int bytes_per_write = sizeof(UInt32x8); - constexpr int safe_overwrite = 15; - - while ((end - output) + safe_overwrite >= bytes_per_write) - { - generators *= LinearCongruentialGenerator::a; - generators += LinearCongruentialGenerator::c; - unalignedStore(output, __builtin_convertvector(generators, UInt32x8)); + VecUInt32 values = __builtin_convertvector(generators >> 16, VecUInt32); + unalignedStore>(output, values); output += bytes_per_write; } @@ -189,7 +174,7 @@ void RandImpl4::execute(char * output, size_t size) { generators *= LinearCongruentialGenerator::a; generators += LinearCongruentialGenerator::c; - UInt32x8 values = __builtin_convertvector(generators, UInt32x8); + VecUInt32 values = __builtin_convertvector(generators >> 16, VecUInt32); for (int i = 0; (end - output) > 0; ++i) { unalignedStore(output, values[i]); @@ -198,49 +183,50 @@ void RandImpl4::execute(char * output, size_t size) } } -void RandImpl5::execute(char * output, size_t size) +template struct RandVecImpl<4>; +template struct RandVecImpl<8>; +template struct RandVecImpl<16>; + +template +void RandVecImpl2::execute(char * output, size_t size) { + static_assert(VectorSize >= 4); + if (size == 0) return; char * end = output + size; - UInt64x16 generators = { - 0xfb4121280b2ab902ULL + reinterpret_cast(output), - 0x0121cf76df39c673ULL + reinterpret_cast(output), - 0x17ae86e3a19a602fULL + reinterpret_cast(output), - 0x8b6e16da7e06d622ULL + reinterpret_cast(output), - 0xfb4121f80b2ab902ULL + reinterpret_cast(output), - 0x0122cf767f39c633ULL + reinterpret_cast(output), - 0x14ae86e3a79a502fULL + reinterpret_cast(output), - 0x876316da7e06d622ULL + reinterpret_cast(output), - 0xfb4821280b2ab912ULL + reinterpret_cast(output), - 0x0126cf76df39c633ULL + reinterpret_cast(output), - 0x17a486e3a19a602fULL + reinterpret_cast(output), - 0x8b6216da7e08d622ULL + reinterpret_cast(output), - 0xfb4101f80b5ab902ULL + reinterpret_cast(output), - 0x01226f767f34c633ULL + reinterpret_cast(output), - 0x14ae86e3a75a502fULL + reinterpret_cast(output), - 0x876e36da7e36d622ULL + reinterpret_cast(output), - }; - - constexpr int bytes_per_write = sizeof(UInt32x16); constexpr int safe_overwrite = 15; + constexpr int bytes_per_write = 2 * sizeof(VecUInt32); + + UInt64 rand_seed = randomSeed(); + VecUInt64 gens1{}, gens2{}; + for (int i = 0; i < VectorSize; ++i) + { + gens1[i] = calcSeed(rand_seed, i * 1123465ull * reinterpret_cast(output)); + gens2[i] = calcSeed(rand_seed, i * 6432453ull * reinterpret_cast(output)); + } while ((end - output) + safe_overwrite >= bytes_per_write) { - generators *= LinearCongruentialGenerator::a; - generators += LinearCongruentialGenerator::c; - unalignedStore(output, __builtin_convertvector(generators, UInt32x16)); + gens1 *= LinearCongruentialGenerator::a; + gens1 += LinearCongruentialGenerator::c; + VecUInt32 values1 = __builtin_convertvector(gens1 >> 16, VecUInt32); + unalignedStore>(output, values1); + gens2 *= LinearCongruentialGenerator::a; + gens2 += LinearCongruentialGenerator::c; + VecUInt32 values2 = __builtin_convertvector(gens2 >> 16, VecUInt32); + unalignedStore>(output, values2); output += bytes_per_write; } - - if ((end - output) > 0) + + while ((end - output) > 0) { - generators *= LinearCongruentialGenerator::a; - generators += LinearCongruentialGenerator::c; - UInt32x16 values = __builtin_convertvector(generators, UInt32x16); - for (int i = 0; (end - output) > 0; ++i) + gens1 *= LinearCongruentialGenerator::a; + gens1 += LinearCongruentialGenerator::c; + VecUInt32 values = __builtin_convertvector(gens1 >> 16, VecUInt32); + for (int i = 0; (end - output) > 0 && i < VectorSize; ++i) { unalignedStore(output, values[i]); output += sizeof(UInt32); @@ -248,8 +234,73 @@ void RandImpl5::execute(char * output, size_t size) } } +template struct RandVecImpl2<4>; +template struct RandVecImpl2<8>; +template struct RandVecImpl2<16>; + +// template +// void RandVecImpl4::execute(char * output, size_t size) +// { +// static_assert(VectorSize >= 4); + +// if (size == 0) +// return; + +// char * end = output + size; + +// constexpr int safe_overwrite = 15; +// constexpr int bytes_per_write = 4 * sizeof(VecUInt32); + +// VecUInt64 gens1{}, gens2{}, gens3{}, gens4{}; +// for (int i = 0; i < VectorSize; ++i) +// { +// gens1[i] = calcSeed(i * 1123465ull * reinterpret_cast(output)); +// gens2[i] = calcSeed(i * 6432453ull * reinterpret_cast(output)); +// gens3[i] = calcSeed(i * 1346434ull * reinterpret_cast(output)); +// gens4[i] = calcSeed(i * 5344753ull * reinterpret_cast(output)); +// } + +// while ((end - output) + safe_overwrite >= bytes_per_write) +// { +// gens1 *= LinearCongruentialGenerator::a; +// gens1 += LinearCongruentialGenerator::c; +// VecUInt32 values1 = __builtin_convertvector(gens1 >> 16, VecUInt32); +// unalignedStore>(output, values1); +// gens2 *= LinearCongruentialGenerator::a; +// gens2 += LinearCongruentialGenerator::c; +// VecUInt32 values2 = __builtin_convertvector(gens2 >> 16, VecUInt32); +// unalignedStore>(output, values2); +// gens3 *= LinearCongruentialGenerator::a; +// gens3 += LinearCongruentialGenerator::c; +// VecUInt32 values3 = __builtin_convertvector(gens3 >> 16, VecUInt32); +// unalignedStore>(output, values3); +// gens4 *= LinearCongruentialGenerator::a; +// gens4 += LinearCongruentialGenerator::c; +// VecUInt32 values4 = __builtin_convertvector(gens4 >> 16, VecUInt32); +// unalignedStore>(output, values4); +// output += bytes_per_write; +// } + +// while ((end - output) > 0) +// { +// gens1 *= LinearCongruentialGenerator::a; +// gens1 += LinearCongruentialGenerator::c; +// VecUInt32 values = __builtin_convertvector(gens1 >> 16, VecUInt32); +// for (int i = 0; (end - output) > 0 && i < VectorSize; i += 4) +// { +// unalignedStore(output, values[i]); +// unalignedStore(output + 4, values[i + 1]); +// unalignedStore(output + 8, values[i + 2]); +// unalignedStore(output + 12, values[i + 3]); +// output += 16; +// } +// } +// } + +// template struct RandVecImpl2<4>; +// template struct RandVecImpl2<8>; +// template struct RandVecImpl2<16>; + ) //DECLARE_MULTITARGET_CODE -*/ - } diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 9a06d8df7a3..557e1fbe868 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -36,26 +36,20 @@ namespace ErrorCodes * This means that the timer must be of sufficient resolution to give different values to each block. */ -/* - DECLARE_MULTITARGET_CODE( -*/ - struct RandImpl { static void execute(char * output, size_t size); - static String getImplementationTag() { return ToString(TargetArch::Default); } + static String getImplementationTag() { return ToString(BuildArch); } }; struct RandImpl2 { static void execute(char * output, size_t size); - static String getImplementationTag() { return ToString(TargetArch::Default) + "_v2"; } + static String getImplementationTag() { return ToString(BuildArch) + "_v2"; } }; -/* - struct RandImpl3 { static void execute(char * output, size_t size); @@ -74,9 +68,27 @@ struct RandImpl5 static String getImplementationTag() { return ToString(BuildArch) + "_v5"; } }; -) // DECLARE_MULTITARGET_CODE +template +struct RandVecImpl +{ + static void execute(char * outpu, size_t size); + static String getImplementationTag() { return ToString(BuildArch) + "_vec_" + toString(VectorSize); } +}; -*/ +template +struct RandVecImpl2 +{ + static void execute(char * outpu, size_t size); + static String getImplementationTag() { return ToString(BuildArch) + "_vec2_" + toString(VectorSize); } +}; + +struct RandImpl6 +{ + static void execute(char * outpu, size_t size); + static String getImplementationTag() { return ToString(BuildArch) + "_v6"; } +}; + +) // DECLARE_MULTITARGET_CODE template class FunctionRandomImpl : public IFunction @@ -125,45 +137,80 @@ public: }; template -class FunctionRandom : public FunctionRandomImpl +class FunctionRandom : public FunctionRandomImpl { public: FunctionRandom(const Context & context) : selector(context) { - // selector.registerImplementation>(); selector.registerImplementation>(); + FunctionRandomImpl>(); + selector.registerImplementation>(); - // if constexpr (UseMultitargetCode) - // { - // selector.registerImplementation>(); - // selector.registerImplementation>(); - // selector.registerImplementation>(); - // selector.registerImplementation>(); + if constexpr (UseMultitargetCode) + { + selector.registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); - // selector.registerImplementation>(); + selector.registerImplementation>(); - // selector.registerImplementation>(); - // selector.registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); - // selector.registerImplementation>(); - // selector.registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); - // selector.registerImplementation>(); - // selector.registerImplementation>(); - // } + selector.registerImplementation>(); + selector.registerImplementation>(); + + // vec impl + selector.registerImplementation, ToType, Name>>(); + selector.registerImplementation, ToType, Name>>(); + + selector.registerImplementation, ToType, Name>>(); + selector.registerImplementation, ToType, Name>>(); + + selector.registerImplementation, ToType, Name>>(); + selector.registerImplementation, ToType, Name>>(); + + // vec impl 2 + selector.registerImplementation, ToType, Name>>(); + selector.registerImplementation, ToType, Name>>(); + + selector.registerImplementation, ToType, Name>>(); + selector.registerImplementation, ToType, Name>>(); + + selector.registerImplementation, ToType, Name>>(); + selector.registerImplementation, ToType, Name>>(); + + selector.registerImplementation>(); + } } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override diff --git a/src/Functions/generateUUIDv4.cpp b/src/Functions/generateUUIDv4.cpp index 4db3bd4c73d..d543226ba5c 100644 --- a/src/Functions/generateUUIDv4.cpp +++ b/src/Functions/generateUUIDv4.cpp @@ -33,7 +33,7 @@ public: size_t size = input_rows_count; vec_to.resize(size); // TODO(dakovalkov): rewrite this workaround - RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(UInt128)); + TargetSpecific::Default::RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(UInt128)); for (UInt128 & uuid: vec_to) { diff --git a/src/Functions/randConstant.cpp b/src/Functions/randConstant.cpp index 163f943d206..3eba5abf10d 100644 --- a/src/Functions/randConstant.cpp +++ b/src/Functions/randConstant.cpp @@ -100,7 +100,7 @@ public: typename ColumnVector::Container vec_to(1); // TODO(dakovalkov): Rewrite this workaround - RandImpl::execute(reinterpret_cast(vec_to.data()), sizeof(ToType)); + TargetSpecific::Default::RandImpl::execute(reinterpret_cast(vec_to.data()), sizeof(ToType)); ToType value = vec_to[0]; return std::make_unique>(value, argument_types, return_type); From e6031585710b3b3d3a1921b88e8d50b086b65dc9 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Wed, 20 May 2020 14:43:33 +0200 Subject: [PATCH 031/211] Rand implementations --- src/Functions/FunctionsRandom.cpp | 249 ++++++++++++++++++++++++++++++ 1 file changed, 249 insertions(+) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 496e0edcc5a..283013bdb9b 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -80,6 +80,62 @@ void RandImpl::execute(char * output, size_t size) /// It is guaranteed (by PaddedPODArray) that we can overwrite up to 15 bytes after end. } +void RandImpl2::execute(char * output, size_t size) +{ + if (size == 0) + return; + + LinearCongruentialGenerator generator0; + LinearCongruentialGenerator generator1; + LinearCongruentialGenerator generator2; + LinearCongruentialGenerator generator3; + LinearCongruentialGenerator generator4; + LinearCongruentialGenerator generator5; + LinearCongruentialGenerator generator6; + LinearCongruentialGenerator generator7; + + UInt64 rand_seed = randomSeed(); + + seed(generator0, rand_seed, 0xfaaae481acb5874aULL + reinterpret_cast(output)); + seed(generator1, rand_seed, 0x3181a34f32887db6ULL + reinterpret_cast(output)); + seed(generator2, rand_seed, 0xb6970e4a91b66afdULL + reinterpret_cast(output)); + seed(generator3, rand_seed, 0xc16062649e83dc13ULL + reinterpret_cast(output)); + seed(generator4, rand_seed, 0xbb093972da5c8d92ULL + reinterpret_cast(output)); + seed(generator5, rand_seed, 0xc37dcc410dcfed31ULL + reinterpret_cast(output)); + seed(generator6, rand_seed, 0x45e1526b7a4367d5ULL + reinterpret_cast(output)); + seed(generator7, rand_seed, 0x99c2759203868a7fULL + reinterpret_cast(output)); + + const char * end = output + size; + + constexpr int bytes_per_write = 32; + constexpr int safe_overwrite = 15; + + for (; (end - output) + safe_overwrite >= bytes_per_write; output += safe_overwrite) + { + unalignedStore(output, generator0.next()); + unalignedStore(output + 4, generator1.next()); + unalignedStore(output + 8, generator2.next()); + unalignedStore(output + 12, generator3.next()); + unalignedStore(output + 16, generator4.next()); + unalignedStore(output + 20, generator5.next()); + unalignedStore(output + 24, generator6.next()); + unalignedStore(output + 28, generator7.next()); + } + + seed(generator0, rand_seed, 0xfaaae481acb5874aULL + reinterpret_cast(output)); + seed(generator1, rand_seed, 0x3181a34f32887db6ULL + reinterpret_cast(output)); + seed(generator2, rand_seed, 0xb6970e4a91b66afdULL + reinterpret_cast(output)); + seed(generator3, rand_seed, 0xc16062649e83dc13ULL + reinterpret_cast(output)); + + if (end - output > 0) + { + unalignedStore(output, generator0.next()); + unalignedStore(output + 4, generator1.next()); + unalignedStore(output + 8, generator2.next()); + unalignedStore(output + 12, generator3.next()); + } +} + typedef UInt64 UInt64x16 __attribute__ ((vector_size (128))); typedef UInt64 UInt64x8 __attribute__ ((vector_size (64))); typedef UInt64 UInt64x4 __attribute__ ((vector_size (32))); @@ -115,6 +171,128 @@ using VecUInt64 = typename DummyStruct::UInt64Type; template using VecUInt32 = typename DummyStruct::UInt32Type; +void RandImpl3::execute(char * output, size_t size) +{ + if (size == 0) + return; + + char * end = output + size; + + UInt64x4 generators = { + 0xfb4121280b2ab902ULL + reinterpret_cast(output), + 0x0121cf76df39c673ULL + reinterpret_cast(output), + 0x17ae86e3a19a602fULL + reinterpret_cast(output), + 0x8b6e16da7e06d622ULL + reinterpret_cast(output), + }; + + constexpr int bytes_per_write = sizeof(UInt32x4); + constexpr int safe_overwrite = 15; + + while ((end - output) + safe_overwrite >= bytes_per_write) + { + generators *= LinearCongruentialGenerator::a; + generators += LinearCongruentialGenerator::c; + unalignedStore(output, __builtin_convertvector(generators >> 16, UInt32x4)); + output += bytes_per_write; + } +} + +void RandImpl4::execute(char * output, size_t size) +{ + if (size == 0) + return; + + char * end = output + size; + + UInt64 rand_seed = randomSeed(); + + UInt64x8 generators = { + calcSeed(rand_seed, 0xfb4121280b2ab902ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x0121cf76df39c673ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x17ae86e3a19a602fULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x8b6e16da7e06d622ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0xfb4121f80b2ab902ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x0122cf767f39c633ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x14ae86e3a79a502fULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x876316da7e06d622ULL + reinterpret_cast(output)), + }; + + constexpr int bytes_per_write = sizeof(UInt32x8); + constexpr int safe_overwrite = 15; + + while ((end - output) + safe_overwrite >= bytes_per_write) + { + generators *= LinearCongruentialGenerator::a; + generators += LinearCongruentialGenerator::c; + unalignedStore(output, __builtin_convertvector(generators >> 16, UInt32x8)); + output += bytes_per_write; + } + + if ((end - output) > 0) + { + generators *= LinearCongruentialGenerator::a; + generators += LinearCongruentialGenerator::c; + UInt32x8 values = __builtin_convertvector(generators >> 16, UInt32x8); + for (int i = 0; (end - output) > 0; ++i) + { + unalignedStore(output, values[i]); + output += sizeof(UInt32); + } + } +} + +void RandImpl5::execute(char * output, size_t size) +{ + if (size == 0) + return; + + char * end = output + size; + + UInt64 rand_seed = randomSeed(); + + UInt64x16 generators = { + calcSeed(rand_seed, 0xfb4121280b2ab902ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x0121cf76df39c673ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x17ae86e3a19a602fULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x8b6e16da7e06d622ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0xfb4121f80b2ab902ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x0122cf767f39c633ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x14ae86e3a79a502fULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x876316da7e06d622ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0xfb4821280b2ab912ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x0126cf76df39c633ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x17a486e3a19a602fULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x8b6216da7e08d622ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0xfb4101f80b5ab902ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x01226f767f34c633ULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x14ae86e3a75a502fULL + reinterpret_cast(output)), + calcSeed(rand_seed, 0x876e36da7e36d622ULL + reinterpret_cast(output)), + }; + + constexpr int bytes_per_write = sizeof(UInt32x16); + constexpr int safe_overwrite = 15; + + while ((end - output) + safe_overwrite >= bytes_per_write) + { + generators *= LinearCongruentialGenerator::a; + generators += LinearCongruentialGenerator::c; + unalignedStore(output, __builtin_convertvector(generators >> 16, UInt32x16)); + output += bytes_per_write; + } + + if ((end - output) > 0) + { + generators *= LinearCongruentialGenerator::a; + generators += LinearCongruentialGenerator::c; + UInt32x16 values = __builtin_convertvector(generators >> 16, UInt32x16); + for (int i = 0; (end - output) > 0; ++i) + { + unalignedStore(output, values[i]); + output += sizeof(UInt32); + } + } +} + namespace { constexpr std::array random_numbers = { @@ -303,4 +481,75 @@ template struct RandVecImpl2<16>; ) //DECLARE_MULTITARGET_CODE +DECLARE_AVX2_SPECIFIC_CODE( + +void RandImpl6::execute(char * output, size_t size) +{ + if (size == 0) + return; + + char * end = output + size; + + UInt64x8 generators = { + 0x5f186ce5faee450bULL + reinterpret_cast(output), + 0x9adb2ca3c72ac2eeULL + reinterpret_cast(output), + 0x07acf8bfa2537705ULL + reinterpret_cast(output), + 0x692b1b533834db92ULL + reinterpret_cast(output), + 0x5148b84cdda30081ULL + reinterpret_cast(output), + 0xe17b8a75a301ad47ULL + reinterpret_cast(output), + 0x6d4a5d69ed2a5f56ULL + reinterpret_cast(output), + 0x114e23266201b333ULL + reinterpret_cast(output), + }; + + union { + UInt64x8 vec; + __m256i mm[2]; + } gens {generators}; + + constexpr int bytes_per_write = sizeof(UInt32x8); + constexpr int safe_overwrite = 15; + + const auto low_a = _mm256_set1_epi64x(0xDEECE66D); + // const auto high_a = _mm256_set1_epi64x(5); + const auto c = _mm256_set1_epi64x(11); + + while ((end - output) + safe_overwrite >= bytes_per_write) + { + { + auto gens_high = _mm256_srli_epi64(gens.mm[0], 32); + auto low_low_res = _mm256_mul_epu32(gens.mm[0], low_a); + auto high_low_res = _mm256_slli_epi64(_mm256_mul_epu32(gens_high, low_a), 32); + auto low_high_res = _mm256_slli_epi64(gens.mm[0], 32) + _mm256_slli_epi64(gens.mm[0], 34); + gens.mm[0] = _mm256_add_epi64(_mm256_add_epi64(low_low_res, high_low_res), + _mm256_add_epi64(low_high_res, c)); + } + { + auto gens_high = _mm256_srli_epi64(gens.mm[1], 32); + auto low_low_res = _mm256_mul_epu32(gens.mm[1], low_a); + auto high_low_res = _mm256_slli_epi64(_mm256_mul_epu32(gens_high, low_a), 32); + auto low_high_res = _mm256_slli_epi64(gens.mm[1], 32) + _mm256_slli_epi64(gens.mm[1], 34); + gens.mm[1] = _mm256_add_epi64(_mm256_add_epi64(low_low_res, high_low_res), + _mm256_add_epi64(low_high_res, c)); + } + // generators *= LinearCongruentialGenerator::a; + // generators += LinearCongruentialGenerator::c; + unalignedStore(output, __builtin_convertvector(gens.vec >> 16, UInt32x8)); + output += bytes_per_write; + } + + if ((end - output) > 0) + { + generators *= LinearCongruentialGenerator::a; + generators += LinearCongruentialGenerator::c; + UInt32x8 values = __builtin_convertvector(generators >> 16, UInt32x8); + for (int i = 0; (end - output) > 0; ++i) + { + unalignedStore(output, values[i]); + output += sizeof(UInt32); + } + } +} + +) // DECLARE_AVX2_SPECIFIC_CODE + } From feaed1e020934ddac683fab616fd2927e6d256a8 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Wed, 20 May 2020 17:43:01 +0200 Subject: [PATCH 032/211] rand isn't avx2-vectorizable, I give it up --- src/Functions/FunctionsRandom.cpp | 581 +++++++++--------------------- src/Functions/FunctionsRandom.h | 111 +----- src/Functions/VectorExtension.h | 101 ++++++ 3 files changed, 278 insertions(+), 515 deletions(-) create mode 100644 src/Functions/VectorExtension.h diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 283013bdb9b..5ab51e9e3b8 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -8,11 +9,6 @@ namespace DB { -// TODO(dakovalkov): remove this workaround. -#if !defined(__clang__) -# pragma GCC diagnostic ignored "-Wvector-operation-performance" -#endif - DECLARE_MULTITARGET_CODE( namespace @@ -80,250 +76,34 @@ void RandImpl::execute(char * output, size_t size) /// It is guaranteed (by PaddedPODArray) that we can overwrite up to 15 bytes after end. } -void RandImpl2::execute(char * output, size_t size) -{ - if (size == 0) - return; - - LinearCongruentialGenerator generator0; - LinearCongruentialGenerator generator1; - LinearCongruentialGenerator generator2; - LinearCongruentialGenerator generator3; - LinearCongruentialGenerator generator4; - LinearCongruentialGenerator generator5; - LinearCongruentialGenerator generator6; - LinearCongruentialGenerator generator7; - - UInt64 rand_seed = randomSeed(); - - seed(generator0, rand_seed, 0xfaaae481acb5874aULL + reinterpret_cast(output)); - seed(generator1, rand_seed, 0x3181a34f32887db6ULL + reinterpret_cast(output)); - seed(generator2, rand_seed, 0xb6970e4a91b66afdULL + reinterpret_cast(output)); - seed(generator3, rand_seed, 0xc16062649e83dc13ULL + reinterpret_cast(output)); - seed(generator4, rand_seed, 0xbb093972da5c8d92ULL + reinterpret_cast(output)); - seed(generator5, rand_seed, 0xc37dcc410dcfed31ULL + reinterpret_cast(output)); - seed(generator6, rand_seed, 0x45e1526b7a4367d5ULL + reinterpret_cast(output)); - seed(generator7, rand_seed, 0x99c2759203868a7fULL + reinterpret_cast(output)); - - const char * end = output + size; - - constexpr int bytes_per_write = 32; - constexpr int safe_overwrite = 15; - - for (; (end - output) + safe_overwrite >= bytes_per_write; output += safe_overwrite) - { - unalignedStore(output, generator0.next()); - unalignedStore(output + 4, generator1.next()); - unalignedStore(output + 8, generator2.next()); - unalignedStore(output + 12, generator3.next()); - unalignedStore(output + 16, generator4.next()); - unalignedStore(output + 20, generator5.next()); - unalignedStore(output + 24, generator6.next()); - unalignedStore(output + 28, generator7.next()); - } - - seed(generator0, rand_seed, 0xfaaae481acb5874aULL + reinterpret_cast(output)); - seed(generator1, rand_seed, 0x3181a34f32887db6ULL + reinterpret_cast(output)); - seed(generator2, rand_seed, 0xb6970e4a91b66afdULL + reinterpret_cast(output)); - seed(generator3, rand_seed, 0xc16062649e83dc13ULL + reinterpret_cast(output)); - - if (end - output > 0) - { - unalignedStore(output, generator0.next()); - unalignedStore(output + 4, generator1.next()); - unalignedStore(output + 8, generator2.next()); - unalignedStore(output + 12, generator3.next()); - } -} - -typedef UInt64 UInt64x16 __attribute__ ((vector_size (128))); -typedef UInt64 UInt64x8 __attribute__ ((vector_size (64))); -typedef UInt64 UInt64x4 __attribute__ ((vector_size (32))); - -typedef UInt32 UInt32x16 __attribute__ ((vector_size (64))); -typedef UInt32 UInt32x8 __attribute__ ((vector_size (32))); -typedef UInt32 UInt32x4 __attribute__ ((vector_size (16))); - -template -struct DummyStruct; - -template <> -struct DummyStruct<4> -{ - using UInt64Type = UInt64x4; - using UInt32Type = UInt32x4; -}; -template <> -struct DummyStruct<8> -{ - using UInt64Type = UInt64x8; - using UInt32Type = UInt32x8; -}; -template <> -struct DummyStruct<16> -{ - using UInt64Type = UInt64x16; - using UInt32Type = UInt32x16; -}; - -template -using VecUInt64 = typename DummyStruct::UInt64Type; -template -using VecUInt32 = typename DummyStruct::UInt32Type; - -void RandImpl3::execute(char * output, size_t size) -{ - if (size == 0) - return; - - char * end = output + size; - - UInt64x4 generators = { - 0xfb4121280b2ab902ULL + reinterpret_cast(output), - 0x0121cf76df39c673ULL + reinterpret_cast(output), - 0x17ae86e3a19a602fULL + reinterpret_cast(output), - 0x8b6e16da7e06d622ULL + reinterpret_cast(output), - }; - - constexpr int bytes_per_write = sizeof(UInt32x4); - constexpr int safe_overwrite = 15; - - while ((end - output) + safe_overwrite >= bytes_per_write) - { - generators *= LinearCongruentialGenerator::a; - generators += LinearCongruentialGenerator::c; - unalignedStore(output, __builtin_convertvector(generators >> 16, UInt32x4)); - output += bytes_per_write; - } -} - -void RandImpl4::execute(char * output, size_t size) -{ - if (size == 0) - return; - - char * end = output + size; - - UInt64 rand_seed = randomSeed(); - - UInt64x8 generators = { - calcSeed(rand_seed, 0xfb4121280b2ab902ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x0121cf76df39c673ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x17ae86e3a19a602fULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x8b6e16da7e06d622ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0xfb4121f80b2ab902ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x0122cf767f39c633ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x14ae86e3a79a502fULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x876316da7e06d622ULL + reinterpret_cast(output)), - }; - - constexpr int bytes_per_write = sizeof(UInt32x8); - constexpr int safe_overwrite = 15; - - while ((end - output) + safe_overwrite >= bytes_per_write) - { - generators *= LinearCongruentialGenerator::a; - generators += LinearCongruentialGenerator::c; - unalignedStore(output, __builtin_convertvector(generators >> 16, UInt32x8)); - output += bytes_per_write; - } - - if ((end - output) > 0) - { - generators *= LinearCongruentialGenerator::a; - generators += LinearCongruentialGenerator::c; - UInt32x8 values = __builtin_convertvector(generators >> 16, UInt32x8); - for (int i = 0; (end - output) > 0; ++i) - { - unalignedStore(output, values[i]); - output += sizeof(UInt32); - } - } -} - -void RandImpl5::execute(char * output, size_t size) -{ - if (size == 0) - return; - - char * end = output + size; - - UInt64 rand_seed = randomSeed(); - - UInt64x16 generators = { - calcSeed(rand_seed, 0xfb4121280b2ab902ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x0121cf76df39c673ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x17ae86e3a19a602fULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x8b6e16da7e06d622ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0xfb4121f80b2ab902ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x0122cf767f39c633ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x14ae86e3a79a502fULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x876316da7e06d622ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0xfb4821280b2ab912ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x0126cf76df39c633ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x17a486e3a19a602fULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x8b6216da7e08d622ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0xfb4101f80b5ab902ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x01226f767f34c633ULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x14ae86e3a75a502fULL + reinterpret_cast(output)), - calcSeed(rand_seed, 0x876e36da7e36d622ULL + reinterpret_cast(output)), - }; - - constexpr int bytes_per_write = sizeof(UInt32x16); - constexpr int safe_overwrite = 15; - - while ((end - output) + safe_overwrite >= bytes_per_write) - { - generators *= LinearCongruentialGenerator::a; - generators += LinearCongruentialGenerator::c; - unalignedStore(output, __builtin_convertvector(generators >> 16, UInt32x16)); - output += bytes_per_write; - } - - if ((end - output) > 0) - { - generators *= LinearCongruentialGenerator::a; - generators += LinearCongruentialGenerator::c; - UInt32x16 values = __builtin_convertvector(generators >> 16, UInt32x16); - for (int i = 0; (end - output) > 0; ++i) - { - unalignedStore(output, values[i]); - output += sizeof(UInt32); - } - } -} - namespace { -constexpr std::array random_numbers = { - 0x0c8ff307dabc0c4cULL, - 0xf4bce78bf3821c1bULL, - 0x4eb628a1e189c21aULL, - 0x85ae000d253e0dbcULL, +// The array of random numbers from 'head -c8 /dev/urandom | xxd -p'. +// Can be used for creating seeds for random generators. +constexpr std::array random_numbers = { + 0x0c8ff307dabc0c4cULL, 0xf4bce78bf3821c1bULL, 0x4eb628a1e189c21aULL, 0x85ae000d253e0dbcULL, + 0xc98073e6480f8a10ULL, 0xb17e9b70a084d570ULL, 0x1361c752b768da8cULL, 0x3d915f60c06d144dULL, + 0xd5bc9b7aced79587ULL, 0x66c28000ba8a66cfULL, 0x0fb58da7a48820f5ULL, 0x540ee1b57aa861a1ULL, + 0x212f11936ef2db04ULL, 0xa3939cd900edcc58ULL, 0xc676c84420170102ULL, 0xcbdc824e8b4bf3edULL, - 0xc98073e6480f8a10ULL, - 0xb17e9b70a084d570ULL, - 0x1361c752b768da8cULL, - 0x3d915f60c06d144dULL, - - 0xd5bc9b7aced79587ULL, - 0x66c28000ba8a66cfULL, - 0x0fb58da7a48820f5ULL, - 0x540ee1b57aa861a1ULL, - - 0x212f11936ef2db04ULL, - 0xa3939cd900edcc58ULL, - 0xc676c84420170102ULL, - 0xcbdc824e8b4bf3edULL, + 0x8296f9d93cc94e3bULL, 0x78a7e826d62085b2ULL, 0xaa30620211fc6c69ULL, 0xbd38de52f0a93677ULL, + 0x19983de8d79dcc4eULL, 0x8afe883ef2199e6fULL, 0xb7160f7ed022b60aULL, 0x2ce173d373ddafd4ULL, + 0x15762761bb55b9acULL, 0x3e448fc94fdd28e7ULL, 0xa5121232adfbe70aULL, 0xb1e0f6d286112804ULL, + 0x6062e96de9554806ULL, 0xcc679b329c28882aULL, 0x5c6d29f45cbc060eULL, 0x1af1325a86ffb162ULL, }; }; -template -void RandVecImpl::execute(char * output, size_t size) +using namespace VectorExtension; + +template +void RandVecImpl::execute(char * output, size_t size) { - static_assert(VectorSize >= 4); - static_assert(VectorSize <= random_numbers.size()); + static_assert(VecSize >= 4); + static_assert(VecSize <= random_numbers.size()); + + using VecUInt64 = UInt64x; + using VecUInt32 = UInt32x; if (size == 0) return; @@ -331,80 +111,38 @@ void RandVecImpl::execute(char * output, size_t size) char * end = output + size; constexpr int safe_overwrite = 15; - constexpr int bytes_per_write = sizeof(VecUInt32); + constexpr int bytes_per_write = sizeof(VecUInt32); UInt64 rand_seed = randomSeed(); - VecUInt64 generators{}; - for (int i = 0; i < VectorSize; ++i) - generators[i] = calcSeed(rand_seed, random_numbers[VectorSize] + reinterpret_cast(output)); + UInt64 a = LinearCongruentialGenerator::a; + // TODO(dakovalkov): try to remove this. + /// Note: GCC likes to expand multiplication by a constant into shifts + additions. + /// In this case a few multiplications become tens of shifts and additions. That leads to a huge slow down. + /// To avoid it we pretend that 'a' is not a constant. Actually we hope that rand_seed is never 0. + if (rand_seed == 0) + a = LinearCongruentialGenerator::a + 2; + + constexpr UInt64 c = LinearCongruentialGenerator::c; + + VecUInt64 generators{}; + for (int i = 0; i < VecSize; ++i) + generators[i] = calcSeed(rand_seed, random_numbers[i] + reinterpret_cast(output)); while ((end - output) + safe_overwrite >= bytes_per_write) { - generators *= LinearCongruentialGenerator::a; - generators += LinearCongruentialGenerator::c; - VecUInt32 values = __builtin_convertvector(generators >> 16, VecUInt32); - unalignedStore>(output, values); + generators = generators * a + c;; + VecUInt32 values = __builtin_convertvector(generators >> 16, VecUInt32); + unalignedStore(output, values); output += bytes_per_write; } - if ((end - output) > 0) - { - generators *= LinearCongruentialGenerator::a; - generators += LinearCongruentialGenerator::c; - VecUInt32 values = __builtin_convertvector(generators >> 16, VecUInt32); - for (int i = 0; (end - output) > 0; ++i) - { - unalignedStore(output, values[i]); - output += sizeof(UInt32); - } - } -} - -template struct RandVecImpl<4>; -template struct RandVecImpl<8>; -template struct RandVecImpl<16>; - -template -void RandVecImpl2::execute(char * output, size_t size) -{ - static_assert(VectorSize >= 4); - - if (size == 0) - return; - - char * end = output + size; - - constexpr int safe_overwrite = 15; - constexpr int bytes_per_write = 2 * sizeof(VecUInt32); - - UInt64 rand_seed = randomSeed(); - VecUInt64 gens1{}, gens2{}; - for (int i = 0; i < VectorSize; ++i) - { - gens1[i] = calcSeed(rand_seed, i * 1123465ull * reinterpret_cast(output)); - gens2[i] = calcSeed(rand_seed, i * 6432453ull * reinterpret_cast(output)); - } - - while ((end - output) + safe_overwrite >= bytes_per_write) - { - gens1 *= LinearCongruentialGenerator::a; - gens1 += LinearCongruentialGenerator::c; - VecUInt32 values1 = __builtin_convertvector(gens1 >> 16, VecUInt32); - unalignedStore>(output, values1); - gens2 *= LinearCongruentialGenerator::a; - gens2 += LinearCongruentialGenerator::c; - VecUInt32 values2 = __builtin_convertvector(gens2 >> 16, VecUInt32); - unalignedStore>(output, values2); - output += bytes_per_write; - } - + // Process tail while ((end - output) > 0) { - gens1 *= LinearCongruentialGenerator::a; - gens1 += LinearCongruentialGenerator::c; - VecUInt32 values = __builtin_convertvector(gens1 >> 16, VecUInt32); - for (int i = 0; (end - output) > 0 && i < VectorSize; ++i) + generators = generators * a + c;; + VecUInt32 values = __builtin_convertvector(generators >> 16, VecUInt32); + for (int i = 0; i < VecSize && (end - output) > 0; ++i) { unalignedStore(output, values[i]); output += sizeof(UInt32); @@ -412,137 +150,60 @@ void RandVecImpl2::execute(char * output, size_t size) } } -template struct RandVecImpl2<4>; -template struct RandVecImpl2<8>; -template struct RandVecImpl2<16>; - -// template -// void RandVecImpl4::execute(char * output, size_t size) -// { -// static_assert(VectorSize >= 4); - -// if (size == 0) -// return; - -// char * end = output + size; - -// constexpr int safe_overwrite = 15; -// constexpr int bytes_per_write = 4 * sizeof(VecUInt32); - -// VecUInt64 gens1{}, gens2{}, gens3{}, gens4{}; -// for (int i = 0; i < VectorSize; ++i) -// { -// gens1[i] = calcSeed(i * 1123465ull * reinterpret_cast(output)); -// gens2[i] = calcSeed(i * 6432453ull * reinterpret_cast(output)); -// gens3[i] = calcSeed(i * 1346434ull * reinterpret_cast(output)); -// gens4[i] = calcSeed(i * 5344753ull * reinterpret_cast(output)); -// } - -// while ((end - output) + safe_overwrite >= bytes_per_write) -// { -// gens1 *= LinearCongruentialGenerator::a; -// gens1 += LinearCongruentialGenerator::c; -// VecUInt32 values1 = __builtin_convertvector(gens1 >> 16, VecUInt32); -// unalignedStore>(output, values1); -// gens2 *= LinearCongruentialGenerator::a; -// gens2 += LinearCongruentialGenerator::c; -// VecUInt32 values2 = __builtin_convertvector(gens2 >> 16, VecUInt32); -// unalignedStore>(output, values2); -// gens3 *= LinearCongruentialGenerator::a; -// gens3 += LinearCongruentialGenerator::c; -// VecUInt32 values3 = __builtin_convertvector(gens3 >> 16, VecUInt32); -// unalignedStore>(output, values3); -// gens4 *= LinearCongruentialGenerator::a; -// gens4 += LinearCongruentialGenerator::c; -// VecUInt32 values4 = __builtin_convertvector(gens4 >> 16, VecUInt32); -// unalignedStore>(output, values4); -// output += bytes_per_write; -// } - -// while ((end - output) > 0) -// { -// gens1 *= LinearCongruentialGenerator::a; -// gens1 += LinearCongruentialGenerator::c; -// VecUInt32 values = __builtin_convertvector(gens1 >> 16, VecUInt32); -// for (int i = 0; (end - output) > 0 && i < VectorSize; i += 4) -// { -// unalignedStore(output, values[i]); -// unalignedStore(output + 4, values[i + 1]); -// unalignedStore(output + 8, values[i + 2]); -// unalignedStore(output + 12, values[i + 3]); -// output += 16; -// } -// } -// } - -// template struct RandVecImpl2<4>; -// template struct RandVecImpl2<8>; -// template struct RandVecImpl2<16>; - -) //DECLARE_MULTITARGET_CODE - -DECLARE_AVX2_SPECIFIC_CODE( - -void RandImpl6::execute(char * output, size_t size) +template +void RandVecImpl2::execute(char * output, size_t size) { + static_assert(VecSize >= 4); + static_assert(2 * VecSize <= random_numbers.size()); + + using VecUInt64 = UInt64x; + using VecUInt32 = UInt32x; + if (size == 0) return; char * end = output + size; - UInt64x8 generators = { - 0x5f186ce5faee450bULL + reinterpret_cast(output), - 0x9adb2ca3c72ac2eeULL + reinterpret_cast(output), - 0x07acf8bfa2537705ULL + reinterpret_cast(output), - 0x692b1b533834db92ULL + reinterpret_cast(output), - 0x5148b84cdda30081ULL + reinterpret_cast(output), - 0xe17b8a75a301ad47ULL + reinterpret_cast(output), - 0x6d4a5d69ed2a5f56ULL + reinterpret_cast(output), - 0x114e23266201b333ULL + reinterpret_cast(output), - }; - - union { - UInt64x8 vec; - __m256i mm[2]; - } gens {generators}; - - constexpr int bytes_per_write = sizeof(UInt32x8); constexpr int safe_overwrite = 15; + constexpr int bytes_per_write = 2 * sizeof(VecUInt32); - const auto low_a = _mm256_set1_epi64x(0xDEECE66D); - // const auto high_a = _mm256_set1_epi64x(5); - const auto c = _mm256_set1_epi64x(11); + UInt64 rand_seed = randomSeed(); + + UInt64 a = LinearCongruentialGenerator::a; + // TODO(dakovalkov): try to remove this. + /// Note: GCC likes to expand multiplication by a constant into shifts + additions. + /// In this case a few multiplications become tens of shifts and additions. That leads to a huge slow down. + /// To avoid it we pretend that 'a' is not a constant. Actually we hope that rand_seed is never 0. + if (rand_seed == 0) + a = LinearCongruentialGenerator::a + 2; + + constexpr UInt64 c = LinearCongruentialGenerator::c; + + VecUInt64 gens1{}; + VecUInt64 gens2{}; + for (int i = 0; i < VecSize; ++i) + { + gens1[i] = calcSeed(rand_seed, random_numbers[i] + reinterpret_cast(output)); + gens2[i] = calcSeed(rand_seed, random_numbers[i + VecSize] + reinterpret_cast(output)); + } while ((end - output) + safe_overwrite >= bytes_per_write) { - { - auto gens_high = _mm256_srli_epi64(gens.mm[0], 32); - auto low_low_res = _mm256_mul_epu32(gens.mm[0], low_a); - auto high_low_res = _mm256_slli_epi64(_mm256_mul_epu32(gens_high, low_a), 32); - auto low_high_res = _mm256_slli_epi64(gens.mm[0], 32) + _mm256_slli_epi64(gens.mm[0], 34); - gens.mm[0] = _mm256_add_epi64(_mm256_add_epi64(low_low_res, high_low_res), - _mm256_add_epi64(low_high_res, c)); - } - { - auto gens_high = _mm256_srli_epi64(gens.mm[1], 32); - auto low_low_res = _mm256_mul_epu32(gens.mm[1], low_a); - auto high_low_res = _mm256_slli_epi64(_mm256_mul_epu32(gens_high, low_a), 32); - auto low_high_res = _mm256_slli_epi64(gens.mm[1], 32) + _mm256_slli_epi64(gens.mm[1], 34); - gens.mm[1] = _mm256_add_epi64(_mm256_add_epi64(low_low_res, high_low_res), - _mm256_add_epi64(low_high_res, c)); - } - // generators *= LinearCongruentialGenerator::a; - // generators += LinearCongruentialGenerator::c; - unalignedStore(output, __builtin_convertvector(gens.vec >> 16, UInt32x8)); + gens1 = gens1 * a + c;; + VecUInt32 values1 = __builtin_convertvector(gens1 >> 16, VecUInt32); + unalignedStore(output, values1); + gens2 = gens2 * a + c;; + VecUInt32 values2 = __builtin_convertvector(gens2 >> 16, VecUInt32); + unalignedStore(output + sizeof(VecUInt32), values2); output += bytes_per_write; } - if ((end - output) > 0) + // Process tail + while ((end - output) > 0) { - generators *= LinearCongruentialGenerator::a; - generators += LinearCongruentialGenerator::c; - UInt32x8 values = __builtin_convertvector(generators >> 16, UInt32x8); - for (int i = 0; (end - output) > 0; ++i) + gens1 = gens1 * a + c;; + VecUInt32 values = __builtin_convertvector(gens1 >> 16, VecUInt32); + for (int i = 0; i < VecSize && (end - output) > 0; ++i) { unalignedStore(output, values[i]); output += sizeof(UInt32); @@ -550,6 +211,86 @@ void RandImpl6::execute(char * output, size_t size) } } +template +void RandVecImpl4::execute(char * output, size_t size) +{ + static_assert(VecSize >= 4); + static_assert(4 * VecSize <= random_numbers.size()); + + using VecUInt64 = UInt64x; + using VecUInt32 = UInt32x; + + if (size == 0) + return; + + char * end = output + size; + + constexpr int safe_overwrite = 15; + constexpr int bytes_per_write = 4 * sizeof(VecUInt32); + + UInt64 rand_seed = randomSeed(); + + UInt64 a = LinearCongruentialGenerator::a; + // TODO(dakovalkov): try to remove this. + /// Note: GCC likes to expand multiplication by a constant into shifts + additions. + /// In this case a few multiplications become tens of shifts and additions. That leads to a huge slow down. + /// To avoid it we pretend that 'a' is not a constant. Actually we hope that rand_seed is never 0. + if (rand_seed == 0) + a = LinearCongruentialGenerator::a + 2; + + constexpr UInt64 c = LinearCongruentialGenerator::c; + + VecUInt64 gens1{}; + VecUInt64 gens2{}; + VecUInt64 gens3{}; + VecUInt64 gens4{}; + for (int i = 0; i < VecSize; ++i) + { + gens1[i] = calcSeed(rand_seed, random_numbers[i] + reinterpret_cast(output)); + gens2[i] = calcSeed(rand_seed, random_numbers[i + VecSize] + reinterpret_cast(output)); + gens3[i] = calcSeed(rand_seed, random_numbers[i + 2 * VecSize] + reinterpret_cast(output)); + gens4[i] = calcSeed(rand_seed, random_numbers[i + 3 * VecSize] + reinterpret_cast(output)); + } + + while ((end - output) + safe_overwrite >= bytes_per_write) + { + gens1 = gens1 * a + c; + VecUInt32 values1 = __builtin_convertvector(gens1 >> 16, VecUInt32); + unalignedStore(output, values1); + gens2 = gens2 * a + c; + VecUInt32 values2 = __builtin_convertvector(gens2 >> 16, VecUInt32); + unalignedStore(output + sizeof(VecUInt32), values2); + gens3 = gens3 * a + c; + VecUInt32 values3 = __builtin_convertvector(gens3 >> 16, VecUInt32); + unalignedStore(output + 2 * sizeof(VecUInt32), values3); + gens4 = gens4 * a + c; + VecUInt32 values4 = __builtin_convertvector(gens4 >> 16, VecUInt32); + unalignedStore(output + 3 * sizeof(VecUInt32), values4); + output += bytes_per_write; + } + + // Process tail + while ((end - output) > 0) + { + gens1 = gens1 * a + c;; + VecUInt32 values = __builtin_convertvector(gens1 >> 16, VecUInt32); + for (int i = 0; i < VecSize && (end - output) > 0; ++i) + { + unalignedStore(output, values[i]); + output += sizeof(UInt32); + } + } +} + +) // DECLARE_MULTITARGET_CODE + +DECLARE_AVX2_SPECIFIC_CODE( + template struct RandVecImpl4<4>; ) // DECLARE_AVX2_SPECIFIC_CODE +DECLARE_AVX512F_SPECIFIC_CODE( + template struct RandVecImpl4<8>; +) // DECLARE_AVX512F_SPECIFIC_CODE + + } diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 557e1fbe868..a82f199356e 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -43,49 +43,26 @@ struct RandImpl static void execute(char * output, size_t size); static String getImplementationTag() { return ToString(BuildArch); } }; - -struct RandImpl2 -{ - static void execute(char * output, size_t size); - static String getImplementationTag() { return ToString(BuildArch) + "_v2"; } -}; - -struct RandImpl3 -{ - static void execute(char * output, size_t size); - static String getImplementationTag() { return ToString(BuildArch) + "_v3"; } -}; - -struct RandImpl4 -{ - static void execute(char * output, size_t size); - static String getImplementationTag() { return ToString(BuildArch) + "_v4"; } -}; - -struct RandImpl5 -{ - static void execute(char * output, size_t size); - static String getImplementationTag() { return ToString(BuildArch) + "_v5"; } -}; - -template +// Isn't used now. +template struct RandVecImpl { - static void execute(char * outpu, size_t size); - static String getImplementationTag() { return ToString(BuildArch) + "_vec_" + toString(VectorSize); } + static void execute(char * output, size_t size); + static String getImplementationTag() { return ToString(BuildArch) + "_vec_" + toString(VecSize); } }; - -template +// Isn't used now. +template struct RandVecImpl2 { - static void execute(char * outpu, size_t size); - static String getImplementationTag() { return ToString(BuildArch) + "_vec2_" + toString(VectorSize); } + static void execute(char * output, size_t size); + static String getImplementationTag() { return ToString(BuildArch) + "_vec2_" + toString(VecSize); } }; -struct RandImpl6 +template +struct RandVecImpl4 { - static void execute(char * outpu, size_t size); - static String getImplementationTag() { return ToString(BuildArch) + "_v6"; } + static void execute(char * output, size_t size); + static String getImplementationTag() { return ToString(BuildArch) + "_vec4_" + toString(VecSize); } }; ) // DECLARE_MULTITARGET_CODE @@ -144,72 +121,16 @@ public: { selector.registerImplementation>(); - selector.registerImplementation>(); if constexpr (UseMultitargetCode) { - selector.registerImplementation>(); - selector.registerImplementation>(); + // vec impl 4 selector.registerImplementation>(); + FunctionRandomImpl, ToType, Name>>(); + selector.registerImplementation>(); + FunctionRandomImpl, ToType, Name>>(); - selector.registerImplementation>(); - - selector.registerImplementation>(); - selector.registerImplementation>(); - - selector.registerImplementation>(); - selector.registerImplementation>(); - - selector.registerImplementation>(); - selector.registerImplementation>(); - - // vec impl - selector.registerImplementation, ToType, Name>>(); - selector.registerImplementation, ToType, Name>>(); - - selector.registerImplementation, ToType, Name>>(); - selector.registerImplementation, ToType, Name>>(); - - selector.registerImplementation, ToType, Name>>(); - selector.registerImplementation, ToType, Name>>(); - - // vec impl 2 - selector.registerImplementation, ToType, Name>>(); - selector.registerImplementation, ToType, Name>>(); - - selector.registerImplementation, ToType, Name>>(); - selector.registerImplementation, ToType, Name>>(); - - selector.registerImplementation, ToType, Name>>(); - selector.registerImplementation, ToType, Name>>(); - - selector.registerImplementation>(); } } diff --git a/src/Functions/VectorExtension.h b/src/Functions/VectorExtension.h new file mode 100644 index 00000000000..49a029bb0d9 --- /dev/null +++ b/src/Functions/VectorExtension.h @@ -0,0 +1,101 @@ +#pragma once + +#include +// Contains types declarations and wrappers for GCC vector extension. + +// TODO(dakovalkov): remove this workaround. +#if !defined(__clang__) +# pragma GCC diagnostic ignored "-Wvector-operation-performance" +#endif + +namespace DB::VectorExtension +{ + +typedef UInt64 UInt64x2 __attribute__ ((vector_size (sizeof(UInt64) * 2))); +typedef UInt64 UInt64x4 __attribute__ ((vector_size (sizeof(UInt64) * 4))); +typedef UInt64 UInt64x8 __attribute__ ((vector_size (sizeof(UInt64) * 8))); +typedef UInt64 UInt64x16 __attribute__ ((vector_size (sizeof(UInt64) * 16))); +typedef UInt64 UInt64x32 __attribute__ ((vector_size (sizeof(UInt64) * 32))); + +typedef UInt32 UInt32x2 __attribute__ ((vector_size (sizeof(UInt32) * 2))); +typedef UInt32 UInt32x4 __attribute__ ((vector_size (sizeof(UInt32) * 4))); +typedef UInt32 UInt32x8 __attribute__ ((vector_size (sizeof(UInt32) * 8))); +typedef UInt32 UInt32x16 __attribute__ ((vector_size (sizeof(UInt32) * 16))); +typedef UInt32 UInt32x32 __attribute__ ((vector_size (sizeof(UInt32) * 32))); +typedef UInt32 UInt32x64 __attribute__ ((vector_size (sizeof(UInt32) * 64))); + +typedef UInt16 UInt16x2 __attribute__ ((vector_size (sizeof(UInt16) * 2))); +typedef UInt16 UInt16x4 __attribute__ ((vector_size (sizeof(UInt16) * 4))); +typedef UInt16 UInt16x8 __attribute__ ((vector_size (sizeof(UInt16) * 8))); +typedef UInt16 UInt16x16 __attribute__ ((vector_size (sizeof(UInt16) * 16))); +typedef UInt16 UInt16x32 __attribute__ ((vector_size (sizeof(UInt16) * 32))); +typedef UInt16 UInt16x64 __attribute__ ((vector_size (sizeof(UInt16) * 64))); + +typedef UInt8 UInt8x2 __attribute__ ((vector_size (sizeof(UInt8) * 2))); +typedef UInt8 UInt8x4 __attribute__ ((vector_size (sizeof(UInt8) * 4))); +typedef UInt8 UInt8x8 __attribute__ ((vector_size (sizeof(UInt8) * 8))); +typedef UInt8 UInt8x16 __attribute__ ((vector_size (sizeof(UInt8) * 16))); +typedef UInt8 UInt8x32 __attribute__ ((vector_size (sizeof(UInt8) * 32))); +typedef UInt8 UInt8x64 __attribute__ ((vector_size (sizeof(UInt8) * 64))); + +namespace detail +{ + template + struct DummyStruct; + + template <> + struct DummyStruct<4> + { + using UInt8Type = UInt8x4; + using UInt16Type = UInt16x4; + using UInt32Type = UInt32x4; + using UInt64Type = UInt64x4; + }; + template <> + struct DummyStruct<8> + { + using UInt8Type = UInt8x8; + using UInt16Type = UInt16x8; + using UInt32Type = UInt32x8; + using UInt64Type = UInt64x8; + }; + template <> + struct DummyStruct<16> + { + using UInt8Type = UInt8x16; + using UInt16Type = UInt16x16; + using UInt32Type = UInt32x16; + using UInt64Type = UInt64x16; + }; + template <> + struct DummyStruct<32> + { + using UInt8Type = UInt8x32; + using UInt16Type = UInt16x32; + using UInt32Type = UInt32x32; + using UInt64Type = UInt64x32; + }; + +} + +// Same as above via template, e.g. UInt64x<8> +template +using UInt8x = typename detail::DummyStruct::UInt8Type; +template +using UInt16x = typename detail::DummyStruct::UInt16Type; +template +using UInt32x = typename detail::DummyStruct::UInt32Type; +template +using UInt64x = typename detail::DummyStruct::UInt64Type; + +/* Casts vectors of the same size. + * UInt32x4 x{}; + * UInt64x4 y = ConvertVector(x); + */ +// template +// inline To ConvertVector(From a) +// { +// return __builtin_convertvector(a, To); +// } + +} From cfc87767c5803f253f80f0ddb4725e4ae4ff1b00 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Wed, 20 May 2020 18:16:11 +0200 Subject: [PATCH 033/211] Fix bug in collecting statistics --- src/Functions/PerformanceAdaptors.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index 717ad196e61..efe4243be79 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -186,6 +186,7 @@ public: { rows_summary += block.getByPosition(i).column->size(); } + rows_summary += block.getByPosition(result).column->size(); if (rows_summary >= 1000) { From d4a8d91789c949e9bfddc606152426e80bdc0c6c Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Wed, 20 May 2020 18:44:01 +0200 Subject: [PATCH 034/211] Fix style --- src/Functions/FunctionsRandom.cpp | 15 ++++++++------- src/Functions/FunctionsRandom.h | 4 +--- src/Functions/PerformanceAdaptors.h | 2 +- 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 5ab51e9e3b8..0f7359f835f 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -76,7 +76,8 @@ void RandImpl::execute(char * output, size_t size) /// It is guaranteed (by PaddedPODArray) that we can overwrite up to 15 bytes after end. } -namespace { +namespace +{ // The array of random numbers from 'head -c8 /dev/urandom | xxd -p'. // Can be used for creating seeds for random generators. @@ -103,11 +104,11 @@ void RandVecImpl::execute(char * output, size_t size) static_assert(VecSize <= random_numbers.size()); using VecUInt64 = UInt64x; - using VecUInt32 = UInt32x; + using VecUInt32 = UInt32x; if (size == 0) return; - + char * end = output + size; constexpr int safe_overwrite = 15; @@ -157,11 +158,11 @@ void RandVecImpl2::execute(char * output, size_t size) static_assert(2 * VecSize <= random_numbers.size()); using VecUInt64 = UInt64x; - using VecUInt32 = UInt32x; + using VecUInt32 = UInt32x; if (size == 0) return; - + char * end = output + size; constexpr int safe_overwrite = 15; @@ -218,11 +219,11 @@ void RandVecImpl4::execute(char * output, size_t size) static_assert(4 * VecSize <= random_numbers.size()); using VecUInt64 = UInt64x; - using VecUInt32 = UInt32x; + using VecUInt32 = UInt32x; if (size == 0) return; - + char * end = output + size; constexpr int safe_overwrite = 15; diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index a82f199356e..5f1e549d961 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -124,13 +124,11 @@ public: if constexpr (UseMultitargetCode) { - // vec impl 4 selector.registerImplementation, ToType, Name>>(); - + selector.registerImplementation, ToType, Name>>(); - } } diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index efe4243be79..daa65300570 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -177,7 +177,7 @@ public: implementations[id]->executeImpl(block, arguments, result, input_rows_count); else implementations[id]->execute(block, arguments, result, input_rows_count); - + watch.stop(); // TODO(dakovalkov): Calculate something more informative. From 310ca8562c19fd063de47906f59f66bd3ccb6834 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 21 May 2020 09:32:42 +0200 Subject: [PATCH 035/211] Add auto-vectorization support for binary operations --- src/Functions/FunctionBinaryArithmetic.h | 71 ++++++++++++++++++++++-- src/Functions/FunctionsRandom.cpp | 1 - 2 files changed, 65 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 30b6da8b696..aba64a52519 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -28,6 +28,8 @@ #include #include +#include + #if !defined(ARCADIA_BUILD) # include #endif @@ -52,12 +54,7 @@ namespace ErrorCodes extern const int CANNOT_ADD_DIFFERENT_AGGREGATE_STATES; } - -/** Arithmetic operations: +, -, *, /, %, - * intDiv (integer division) - * Bitwise operations: |, &, ^, ~. - * Etc. - */ +DECLARE_MULTITARGET_CODE( template struct BinaryOperationImplBase @@ -89,6 +86,68 @@ struct BinaryOperationImplBase } }; +) // DECLARE_MULTITARGET_CODE + + +/** Arithmetic operations: +, -, *, /, %, + * intDiv (integer division) + * Bitwise operations: |, &, ^, ~. + * Etc. + */ +template +struct BinaryOperationImplBase +{ + using ResultType = ResultType_; + static const constexpr bool allow_fixed_string = false; + + static void vectorVector(const A * __restrict a, const B * __restrict b, ResultType * __restrict c, size_t size) + { + if (IsArchSupported(TargetArch::AVX512F)) + TargetSpecific::AVX512F::BinaryOperationImplBase::vectorVector(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX2)) + TargetSpecific::AVX2::BinaryOperationImplBase::vectorVector(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX)) + TargetSpecific::AVX::BinaryOperationImplBase::vectorVector(a, b, c, size); + else if (IsArchSupported(TargetArch::SSE42)) + TargetSpecific::SSE42::BinaryOperationImplBase::vectorVector(a, b, c, size); + else + TargetSpecific::Default::BinaryOperationImplBase::vectorVector(a, b, c, size); + } + + static void vectorConstant(const A * __restrict a, B b, ResultType * __restrict c, size_t size) + { + if (IsArchSupported(TargetArch::AVX512F)) + TargetSpecific::AVX512F::BinaryOperationImplBase::vectorConstant(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX2)) + TargetSpecific::AVX2::BinaryOperationImplBase::vectorConstant(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX)) + TargetSpecific::AVX::BinaryOperationImplBase::vectorConstant(a, b, c, size); + else if (IsArchSupported(TargetArch::SSE42)) + TargetSpecific::SSE42::BinaryOperationImplBase::vectorConstant(a, b, c, size); + else + TargetSpecific::Default::BinaryOperationImplBase::vectorConstant(a, b, c, size); + } + + static void constantVector(A a, const B * __restrict b, ResultType * __restrict c, size_t size) + { + if (IsArchSupported(TargetArch::AVX512F)) + TargetSpecific::AVX512F::BinaryOperationImplBase::constantVector(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX2)) + TargetSpecific::AVX2::BinaryOperationImplBase::constantVector(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX)) + TargetSpecific::AVX::BinaryOperationImplBase::constantVector(a, b, c, size); + else if (IsArchSupported(TargetArch::SSE42)) + TargetSpecific::SSE42::BinaryOperationImplBase::constantVector(a, b, c, size); + else + TargetSpecific::Default::BinaryOperationImplBase::constantVector(a, b, c, size); + } + + static ResultType constantConstant(A a, B b) + { + return Op::template apply(a, b); + } +}; + template struct FixedStringOperationImpl { diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 0f7359f835f..f673b6c5f81 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -293,5 +293,4 @@ DECLARE_AVX512F_SPECIFIC_CODE( template struct RandVecImpl4<8>; ) // DECLARE_AVX512F_SPECIFIC_CODE - } From 8483dfa272fee7063f5cfc24bb7066bf48cf78cc Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 21 May 2020 09:40:27 +0200 Subject: [PATCH 036/211] Delete needless rand implementations --- src/Functions/FunctionsRandom.cpp | 119 +----------------------------- src/Functions/FunctionsRandom.h | 22 ++---- tests/performance/rand.xml | 4 +- 3 files changed, 8 insertions(+), 137 deletions(-) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index f673b6c5f81..cfb3b92c818 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -99,121 +99,6 @@ using namespace VectorExtension; template void RandVecImpl::execute(char * output, size_t size) -{ - static_assert(VecSize >= 4); - static_assert(VecSize <= random_numbers.size()); - - using VecUInt64 = UInt64x; - using VecUInt32 = UInt32x; - - if (size == 0) - return; - - char * end = output + size; - - constexpr int safe_overwrite = 15; - constexpr int bytes_per_write = sizeof(VecUInt32); - - UInt64 rand_seed = randomSeed(); - - UInt64 a = LinearCongruentialGenerator::a; - // TODO(dakovalkov): try to remove this. - /// Note: GCC likes to expand multiplication by a constant into shifts + additions. - /// In this case a few multiplications become tens of shifts and additions. That leads to a huge slow down. - /// To avoid it we pretend that 'a' is not a constant. Actually we hope that rand_seed is never 0. - if (rand_seed == 0) - a = LinearCongruentialGenerator::a + 2; - - constexpr UInt64 c = LinearCongruentialGenerator::c; - - VecUInt64 generators{}; - for (int i = 0; i < VecSize; ++i) - generators[i] = calcSeed(rand_seed, random_numbers[i] + reinterpret_cast(output)); - - while ((end - output) + safe_overwrite >= bytes_per_write) - { - generators = generators * a + c;; - VecUInt32 values = __builtin_convertvector(generators >> 16, VecUInt32); - unalignedStore(output, values); - output += bytes_per_write; - } - - // Process tail - while ((end - output) > 0) - { - generators = generators * a + c;; - VecUInt32 values = __builtin_convertvector(generators >> 16, VecUInt32); - for (int i = 0; i < VecSize && (end - output) > 0; ++i) - { - unalignedStore(output, values[i]); - output += sizeof(UInt32); - } - } -} - -template -void RandVecImpl2::execute(char * output, size_t size) -{ - static_assert(VecSize >= 4); - static_assert(2 * VecSize <= random_numbers.size()); - - using VecUInt64 = UInt64x; - using VecUInt32 = UInt32x; - - if (size == 0) - return; - - char * end = output + size; - - constexpr int safe_overwrite = 15; - constexpr int bytes_per_write = 2 * sizeof(VecUInt32); - - UInt64 rand_seed = randomSeed(); - - UInt64 a = LinearCongruentialGenerator::a; - // TODO(dakovalkov): try to remove this. - /// Note: GCC likes to expand multiplication by a constant into shifts + additions. - /// In this case a few multiplications become tens of shifts and additions. That leads to a huge slow down. - /// To avoid it we pretend that 'a' is not a constant. Actually we hope that rand_seed is never 0. - if (rand_seed == 0) - a = LinearCongruentialGenerator::a + 2; - - constexpr UInt64 c = LinearCongruentialGenerator::c; - - VecUInt64 gens1{}; - VecUInt64 gens2{}; - for (int i = 0; i < VecSize; ++i) - { - gens1[i] = calcSeed(rand_seed, random_numbers[i] + reinterpret_cast(output)); - gens2[i] = calcSeed(rand_seed, random_numbers[i + VecSize] + reinterpret_cast(output)); - } - - while ((end - output) + safe_overwrite >= bytes_per_write) - { - gens1 = gens1 * a + c;; - VecUInt32 values1 = __builtin_convertvector(gens1 >> 16, VecUInt32); - unalignedStore(output, values1); - gens2 = gens2 * a + c;; - VecUInt32 values2 = __builtin_convertvector(gens2 >> 16, VecUInt32); - unalignedStore(output + sizeof(VecUInt32), values2); - output += bytes_per_write; - } - - // Process tail - while ((end - output) > 0) - { - gens1 = gens1 * a + c;; - VecUInt32 values = __builtin_convertvector(gens1 >> 16, VecUInt32); - for (int i = 0; i < VecSize && (end - output) > 0; ++i) - { - unalignedStore(output, values[i]); - output += sizeof(UInt32); - } - } -} - -template -void RandVecImpl4::execute(char * output, size_t size) { static_assert(VecSize >= 4); static_assert(4 * VecSize <= random_numbers.size()); @@ -286,11 +171,11 @@ void RandVecImpl4::execute(char * output, size_t size) ) // DECLARE_MULTITARGET_CODE DECLARE_AVX2_SPECIFIC_CODE( - template struct RandVecImpl4<4>; + template struct RandVecImpl<4>; ) // DECLARE_AVX2_SPECIFIC_CODE DECLARE_AVX512F_SPECIFIC_CODE( - template struct RandVecImpl4<8>; + template struct RandVecImpl<8>; ) // DECLARE_AVX512F_SPECIFIC_CODE } diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 5f1e549d961..5251f8fd622 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -43,27 +43,15 @@ struct RandImpl static void execute(char * output, size_t size); static String getImplementationTag() { return ToString(BuildArch); } }; -// Isn't used now. + +/// Implementation is in .cpp file. +/// Every specialization should be explicitly written in .cpp file. template struct RandVecImpl { static void execute(char * output, size_t size); static String getImplementationTag() { return ToString(BuildArch) + "_vec_" + toString(VecSize); } }; -// Isn't used now. -template -struct RandVecImpl2 -{ - static void execute(char * output, size_t size); - static String getImplementationTag() { return ToString(BuildArch) + "_vec2_" + toString(VecSize); } -}; - -template -struct RandVecImpl4 -{ - static void execute(char * output, size_t size); - static String getImplementationTag() { return ToString(BuildArch) + "_vec4_" + toString(VecSize); } -}; ) // DECLARE_MULTITARGET_CODE @@ -125,10 +113,10 @@ public: if constexpr (UseMultitargetCode) { selector.registerImplementation, ToType, Name>>(); + FunctionRandomImpl, ToType, Name>>(); selector.registerImplementation, ToType, Name>>(); + FunctionRandomImpl, ToType, Name>>(); } } diff --git a/tests/performance/rand.xml b/tests/performance/rand.xml index a007eb50179..bd34a7a83d8 100644 --- a/tests/performance/rand.xml +++ b/tests/performance/rand.xml @@ -13,13 +13,11 @@ table - numbers(10000000) + zeros(100000000) SELECT count() FROM (SELECT rand() FROM {table}) - SELECT count() FROM (SELECT randxorshift() FROM {table}) SELECT count() FROM (SELECT rand64() FROM {table}) - SELECT count() FROM (SELECT randxorshift64() FROM {table}) From e317dfb6e4fc5e6869ab7730ddc22b58ad9da0ca Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 21 May 2020 10:11:48 +0200 Subject: [PATCH 037/211] Fix FBA --- src/Functions/FunctionBinaryArithmetic.h | 70 ++++++++++++++++-------- 1 file changed, 46 insertions(+), 24 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index aba64a52519..40a387d09b2 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -102,44 +102,66 @@ struct BinaryOperationImplBase static void vectorVector(const A * __restrict a, const B * __restrict b, ResultType * __restrict c, size_t size) { - if (IsArchSupported(TargetArch::AVX512F)) - TargetSpecific::AVX512F::BinaryOperationImplBase::vectorVector(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX2)) - TargetSpecific::AVX2::BinaryOperationImplBase::vectorVector(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX)) - TargetSpecific::AVX::BinaryOperationImplBase::vectorVector(a, b, c, size); - else if (IsArchSupported(TargetArch::SSE42)) - TargetSpecific::SSE42::BinaryOperationImplBase::vectorVector(a, b, c, size); + if constexpr (UseMultitargetCode) + { + if (IsArchSupported(TargetArch::AVX512F)) + TargetSpecific::AVX512F::BinaryOperationImplBase::vectorVector(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX2)) + TargetSpecific::AVX2::BinaryOperationImplBase::vectorVector(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX)) + TargetSpecific::AVX::BinaryOperationImplBase::vectorVector(a, b, c, size); + else if (IsArchSupported(TargetArch::SSE42)) + TargetSpecific::SSE42::BinaryOperationImplBase::vectorVector(a, b, c, size); + else + TargetSpecific::Default::BinaryOperationImplBase::vectorVector(a, b, c, size); + } else + { TargetSpecific::Default::BinaryOperationImplBase::vectorVector(a, b, c, size); + } + } static void vectorConstant(const A * __restrict a, B b, ResultType * __restrict c, size_t size) { - if (IsArchSupported(TargetArch::AVX512F)) - TargetSpecific::AVX512F::BinaryOperationImplBase::vectorConstant(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX2)) - TargetSpecific::AVX2::BinaryOperationImplBase::vectorConstant(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX)) - TargetSpecific::AVX::BinaryOperationImplBase::vectorConstant(a, b, c, size); - else if (IsArchSupported(TargetArch::SSE42)) - TargetSpecific::SSE42::BinaryOperationImplBase::vectorConstant(a, b, c, size); + if constexpr (UseMultitargetCode) + { + if (IsArchSupported(TargetArch::AVX512F)) + TargetSpecific::AVX512F::BinaryOperationImplBase::vectorConstant(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX2)) + TargetSpecific::AVX2::BinaryOperationImplBase::vectorConstant(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX)) + TargetSpecific::AVX::BinaryOperationImplBase::vectorConstant(a, b, c, size); + else if (IsArchSupported(TargetArch::SSE42)) + TargetSpecific::SSE42::BinaryOperationImplBase::vectorConstant(a, b, c, size); + else + TargetSpecific::Default::BinaryOperationImplBase::vectorConstant(a, b, c, size); + } else + { TargetSpecific::Default::BinaryOperationImplBase::vectorConstant(a, b, c, size); + } } static void constantVector(A a, const B * __restrict b, ResultType * __restrict c, size_t size) { - if (IsArchSupported(TargetArch::AVX512F)) - TargetSpecific::AVX512F::BinaryOperationImplBase::constantVector(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX2)) - TargetSpecific::AVX2::BinaryOperationImplBase::constantVector(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX)) - TargetSpecific::AVX::BinaryOperationImplBase::constantVector(a, b, c, size); - else if (IsArchSupported(TargetArch::SSE42)) - TargetSpecific::SSE42::BinaryOperationImplBase::constantVector(a, b, c, size); + if constexpr (UseMultitargetCode) + { + if (IsArchSupported(TargetArch::AVX512F)) + TargetSpecific::AVX512F::BinaryOperationImplBase::constantVector(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX2)) + TargetSpecific::AVX2::BinaryOperationImplBase::constantVector(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX)) + TargetSpecific::AVX::BinaryOperationImplBase::constantVector(a, b, c, size); + else if (IsArchSupported(TargetArch::SSE42)) + TargetSpecific::SSE42::BinaryOperationImplBase::constantVector(a, b, c, size); + else + TargetSpecific::Default::BinaryOperationImplBase::constantVector(a, b, c, size); + } else + { TargetSpecific::Default::BinaryOperationImplBase::constantVector(a, b, c, size); + } } static ResultType constantConstant(A a, B b) From 0d1577c5da6f54e76d447c457803800ac01b4baf Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Sun, 24 May 2020 14:25:07 +0200 Subject: [PATCH 038/211] Better avx2 implementation for rand(). Expected to be ~10% faster --- src/Functions/FunctionBinaryArithmetic.h | 1 - src/Functions/FunctionsRandom.cpp | 123 ++++++++++++----------- src/Functions/FunctionsRandom.h | 14 +-- 3 files changed, 64 insertions(+), 74 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 40a387d09b2..c311b8d5d0a 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -119,7 +119,6 @@ struct BinaryOperationImplBase { TargetSpecific::Default::BinaryOperationImplBase::vectorVector(a, b, c, size); } - } static void vectorConstant(const A * __restrict a, B b, ResultType * __restrict c, size_t size) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index cfb3b92c818..9eaa44b0eb5 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -9,8 +9,6 @@ namespace DB { -DECLARE_MULTITARGET_CODE( - namespace { /// NOTE Probably @@ -50,8 +48,23 @@ namespace generator.seed(calcSeed(rand_seed, additional_seed)); } + /// The array of random numbers from 'head -c8 /dev/urandom | xxd -p'. + /// Can be used for creating seeds for random generators. + constexpr std::array random_numbers = { + 0x0c8ff307dabc0c4cULL, 0xf4bce78bf3821c1bULL, 0x4eb628a1e189c21aULL, 0x85ae000d253e0dbcULL, + 0xc98073e6480f8a10ULL, 0xb17e9b70a084d570ULL, 0x1361c752b768da8cULL, 0x3d915f60c06d144dULL, + 0xd5bc9b7aced79587ULL, 0x66c28000ba8a66cfULL, 0x0fb58da7a48820f5ULL, 0x540ee1b57aa861a1ULL, + 0x212f11936ef2db04ULL, 0xa3939cd900edcc58ULL, 0xc676c84420170102ULL, 0xcbdc824e8b4bf3edULL, + + 0x8296f9d93cc94e3bULL, 0x78a7e826d62085b2ULL, 0xaa30620211fc6c69ULL, 0xbd38de52f0a93677ULL, + 0x19983de8d79dcc4eULL, 0x8afe883ef2199e6fULL, 0xb7160f7ed022b60aULL, 0x2ce173d373ddafd4ULL, + 0x15762761bb55b9acULL, 0x3e448fc94fdd28e7ULL, 0xa5121232adfbe70aULL, 0xb1e0f6d286112804ULL, + 0x6062e96de9554806ULL, 0xcc679b329c28882aULL, 0x5c6d29f45cbc060eULL, 0x1af1325a86ffb162ULL, + }; } +DECLARE_DEFAULT_CODE( + void RandImpl::execute(char * output, size_t size) { LinearCongruentialGenerator generator0; @@ -61,10 +74,10 @@ void RandImpl::execute(char * output, size_t size) UInt64 rand_seed = randomSeed(); - seed(generator0, rand_seed, 0xfb4121280b2ab902ULL + reinterpret_cast(output)); - seed(generator1, rand_seed, 0x0121cf76df39c673ULL + reinterpret_cast(output)); - seed(generator2, rand_seed, 0x17ae86e3a19a602fULL + reinterpret_cast(output)); - seed(generator3, rand_seed, 0x8b6e16da7e06d622ULL + reinterpret_cast(output)); + seed(generator0, rand_seed, random_numbers[0] + reinterpret_cast(output)); + seed(generator1, rand_seed, random_numbers[1] + reinterpret_cast(output)); + seed(generator2, rand_seed, random_numbers[2] + reinterpret_cast(output)); + seed(generator3, rand_seed, random_numbers[3] + reinterpret_cast(output)); for (const char * end = output + size; output < end; output += 16) { @@ -76,43 +89,40 @@ void RandImpl::execute(char * output, size_t size) /// It is guaranteed (by PaddedPODArray) that we can overwrite up to 15 bytes after end. } -namespace -{ +) // DECLARE_DEFAULT_CODE -// The array of random numbers from 'head -c8 /dev/urandom | xxd -p'. -// Can be used for creating seeds for random generators. -constexpr std::array random_numbers = { - 0x0c8ff307dabc0c4cULL, 0xf4bce78bf3821c1bULL, 0x4eb628a1e189c21aULL, 0x85ae000d253e0dbcULL, - 0xc98073e6480f8a10ULL, 0xb17e9b70a084d570ULL, 0x1361c752b768da8cULL, 0x3d915f60c06d144dULL, - 0xd5bc9b7aced79587ULL, 0x66c28000ba8a66cfULL, 0x0fb58da7a48820f5ULL, 0x540ee1b57aa861a1ULL, - 0x212f11936ef2db04ULL, 0xa3939cd900edcc58ULL, 0xc676c84420170102ULL, 0xcbdc824e8b4bf3edULL, - - 0x8296f9d93cc94e3bULL, 0x78a7e826d62085b2ULL, 0xaa30620211fc6c69ULL, 0xbd38de52f0a93677ULL, - 0x19983de8d79dcc4eULL, 0x8afe883ef2199e6fULL, 0xb7160f7ed022b60aULL, 0x2ce173d373ddafd4ULL, - 0x15762761bb55b9acULL, 0x3e448fc94fdd28e7ULL, 0xa5121232adfbe70aULL, 0xb1e0f6d286112804ULL, - 0x6062e96de9554806ULL, 0xcc679b329c28882aULL, 0x5c6d29f45cbc060eULL, 0x1af1325a86ffb162ULL, -}; - -}; +DECLARE_AVX2_SPECIFIC_CODE( using namespace VectorExtension; -template -void RandVecImpl::execute(char * output, size_t size) +/* Takes 2 vectors with LinearCongruentialGenerator states and combines them into vector with random values. + * From every rand-state we use only bits 15...47 to generate random vector. + */ +inline UInt64x4 CombineValues(UInt64x4 a, UInt64x4 b) { - static_assert(VecSize >= 4); - static_assert(4 * VecSize <= random_numbers.size()); - - using VecUInt64 = UInt64x; - using VecUInt32 = UInt32x; + auto xa = reinterpret_cast<__m256i>(a); + auto xb = reinterpret_cast<__m256i>(b); + /// Every state is 8-byte value and we need to use only 4 from the middle. + /// Swap the low half and the high half of every state to move these bytes from the middle to sides. + /// xa = xa[1, 0, 3, 2, 5, 4, 7, 6] + xa = _mm256_shuffle_epi32(xa, 0xb1); + /// Now every 8-byte value in xa is xx....xx and every value in xb is ..xxxx.. where x is random byte we want to use. + /// Just blend them to get the result vector. + /// res = xa[0],xb[1,2],xa[3,4],xb[5,6],xa[7,8],xb[9,10],xa[11,12],xb[13,14],xa[15] + auto res = _mm256_blend_epi16(xa, xb, 0x66); + return reinterpret_cast(res); +} +void RandImpl::execute(char * output, size_t size) +{ if (size == 0) return; char * end = output + size; + constexpr int vec_size = 4; constexpr int safe_overwrite = 15; - constexpr int bytes_per_write = 4 * sizeof(VecUInt32); + constexpr int bytes_per_write = 4 * sizeof(UInt64x4); UInt64 rand_seed = randomSeed(); @@ -126,56 +136,49 @@ void RandVecImpl::execute(char * output, size_t size) constexpr UInt64 c = LinearCongruentialGenerator::c; - VecUInt64 gens1{}; - VecUInt64 gens2{}; - VecUInt64 gens3{}; - VecUInt64 gens4{}; - for (int i = 0; i < VecSize; ++i) + UInt64x4 gens1{}; + UInt64x4 gens2{}; + UInt64x4 gens3{}; + UInt64x4 gens4{}; + for (int i = 0; i < vec_size; ++i) { gens1[i] = calcSeed(rand_seed, random_numbers[i] + reinterpret_cast(output)); - gens2[i] = calcSeed(rand_seed, random_numbers[i + VecSize] + reinterpret_cast(output)); - gens3[i] = calcSeed(rand_seed, random_numbers[i + 2 * VecSize] + reinterpret_cast(output)); - gens4[i] = calcSeed(rand_seed, random_numbers[i + 3 * VecSize] + reinterpret_cast(output)); + gens2[i] = calcSeed(rand_seed, random_numbers[i + vec_size] + reinterpret_cast(output)); + gens3[i] = calcSeed(rand_seed, random_numbers[i + 2 * vec_size] + reinterpret_cast(output)); + gens4[i] = calcSeed(rand_seed, random_numbers[i + 3 * vec_size] + reinterpret_cast(output)); } while ((end - output) + safe_overwrite >= bytes_per_write) { gens1 = gens1 * a + c; - VecUInt32 values1 = __builtin_convertvector(gens1 >> 16, VecUInt32); - unalignedStore(output, values1); gens2 = gens2 * a + c; - VecUInt32 values2 = __builtin_convertvector(gens2 >> 16, VecUInt32); - unalignedStore(output + sizeof(VecUInt32), values2); + unalignedStore(output, CombineValues(gens1, gens2)); gens3 = gens3 * a + c; - VecUInt32 values3 = __builtin_convertvector(gens3 >> 16, VecUInt32); - unalignedStore(output + 2 * sizeof(VecUInt32), values3); gens4 = gens4 * a + c; - VecUInt32 values4 = __builtin_convertvector(gens4 >> 16, VecUInt32); - unalignedStore(output + 3 * sizeof(VecUInt32), values4); + unalignedStore(output + sizeof(UInt64x4), CombineValues(gens3, gens4)); + gens1 = gens1 * a + c; + gens2 = gens2 * a + c; + unalignedStore(output + 2 * sizeof(UInt64x4), CombineValues(gens1, gens2)); + gens3 = gens3 * a + c; + gens4 = gens4 * a + c; + unalignedStore(output + 3 * sizeof(UInt64x4), CombineValues(gens3, gens4)); output += bytes_per_write; } // Process tail while ((end - output) > 0) { - gens1 = gens1 * a + c;; - VecUInt32 values = __builtin_convertvector(gens1 >> 16, VecUInt32); - for (int i = 0; i < VecSize && (end - output) > 0; ++i) + gens1 = gens1 * a + c; + gens2 = gens2 * a + c; + UInt64x4 values = CombineValues(gens1, gens2); + for (int i = 0; i < vec_size && (end - output) > 0; ++i) { - unalignedStore(output, values[i]); - output += sizeof(UInt32); + unalignedStore(output, values[i]); + output += sizeof(UInt64); } } } -) // DECLARE_MULTITARGET_CODE - -DECLARE_AVX2_SPECIFIC_CODE( - template struct RandVecImpl<4>; ) // DECLARE_AVX2_SPECIFIC_CODE -DECLARE_AVX512F_SPECIFIC_CODE( - template struct RandVecImpl<8>; -) // DECLARE_AVX512F_SPECIFIC_CODE - } diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 5251f8fd622..e10b249df8e 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -44,15 +44,6 @@ struct RandImpl static String getImplementationTag() { return ToString(BuildArch); } }; -/// Implementation is in .cpp file. -/// Every specialization should be explicitly written in .cpp file. -template -struct RandVecImpl -{ - static void execute(char * output, size_t size); - static String getImplementationTag() { return ToString(BuildArch) + "_vec_" + toString(VecSize); } -}; - ) // DECLARE_MULTITARGET_CODE template @@ -113,10 +104,7 @@ public: if constexpr (UseMultitargetCode) { selector.registerImplementation, ToType, Name>>(); - - selector.registerImplementation, ToType, Name>>(); + FunctionRandomImpl>(); } } From 4a9891c601e459c6d9b9e638cd591ce7f3edbf13 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Mon, 25 May 2020 17:16:19 +0200 Subject: [PATCH 039/211] use vectorized rand in generateUUIDv4, add multitarget build in intHash32/64 --- src/Functions/FunctionsHashing.h | 37 +++++++++++++++++++++++++- src/Functions/FunctionsRandom.cpp | 7 ++--- src/Functions/VectorExtension.h | 23 ----------------- src/Functions/generateUUIDv4.cpp | 43 ++++++++++++++++++++++++++++--- tests/performance/rand.xml | 1 + 5 files changed, 81 insertions(+), 30 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index f647390e1c8..6f00981a22a 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -40,6 +40,8 @@ #include #include #include +#include +#include #include #include @@ -573,12 +575,13 @@ public: }; +DECLARE_MULTITARGET_CODE( + template class FunctionIntHash : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } private: using ToType = typename Impl::ReturnType; @@ -612,6 +615,8 @@ public: return name; } + static String getImplementationTag() { return ToString(BuildArch); } + size_t getNumberOfArguments() const override { return 1; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override @@ -646,6 +651,36 @@ public: } }; +) // DECLARE_MULTITARGET_CODE + +template +class FunctionIntHash : public TargetSpecific::Default::FunctionIntHash +{ +public: + FunctionIntHash(const Context & context) : selector(context) + { + selector.registerImplementation>(); + + selector.registerImplementation>(); + selector.registerImplementation>(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + selector.selectAndExecute(block, arguments, result, input_rows_count); + } + + static FunctionPtr create(const Context & context) + { + return std::make_shared(context); + } + +private: + ImplementationSelector selector; +}; template class FunctionAnyHash : public IFunction diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 9eaa44b0eb5..6b2e79e90ca 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -108,9 +108,9 @@ inline UInt64x4 CombineValues(UInt64x4 a, UInt64x4 b) xa = _mm256_shuffle_epi32(xa, 0xb1); /// Now every 8-byte value in xa is xx....xx and every value in xb is ..xxxx.. where x is random byte we want to use. /// Just blend them to get the result vector. - /// res = xa[0],xb[1,2],xa[3,4],xb[5,6],xa[7,8],xb[9,10],xa[11,12],xb[13,14],xa[15] - auto res = _mm256_blend_epi16(xa, xb, 0x66); - return reinterpret_cast(res); + /// result = xa[0],xb[1,2],xa[3,4],xb[5,6],xa[7,8],xb[9,10],xa[11,12],xb[13,14],xa[15] + __m256i result = _mm256_blend_epi16(xa, xb, 0x66); + return reinterpret_cast(result); } void RandImpl::execute(char * output, size_t size) @@ -140,6 +140,7 @@ void RandImpl::execute(char * output, size_t size) UInt64x4 gens2{}; UInt64x4 gens3{}; UInt64x4 gens4{}; + for (int i = 0; i < vec_size; ++i) { gens1[i] = calcSeed(rand_seed, random_numbers[i] + reinterpret_cast(output)); diff --git a/src/Functions/VectorExtension.h b/src/Functions/VectorExtension.h index 49a029bb0d9..24c2ae9a18f 100644 --- a/src/Functions/VectorExtension.h +++ b/src/Functions/VectorExtension.h @@ -3,33 +3,23 @@ #include // Contains types declarations and wrappers for GCC vector extension. -// TODO(dakovalkov): remove this workaround. -#if !defined(__clang__) -# pragma GCC diagnostic ignored "-Wvector-operation-performance" -#endif - namespace DB::VectorExtension { typedef UInt64 UInt64x2 __attribute__ ((vector_size (sizeof(UInt64) * 2))); typedef UInt64 UInt64x4 __attribute__ ((vector_size (sizeof(UInt64) * 4))); typedef UInt64 UInt64x8 __attribute__ ((vector_size (sizeof(UInt64) * 8))); -typedef UInt64 UInt64x16 __attribute__ ((vector_size (sizeof(UInt64) * 16))); -typedef UInt64 UInt64x32 __attribute__ ((vector_size (sizeof(UInt64) * 32))); typedef UInt32 UInt32x2 __attribute__ ((vector_size (sizeof(UInt32) * 2))); typedef UInt32 UInt32x4 __attribute__ ((vector_size (sizeof(UInt32) * 4))); typedef UInt32 UInt32x8 __attribute__ ((vector_size (sizeof(UInt32) * 8))); typedef UInt32 UInt32x16 __attribute__ ((vector_size (sizeof(UInt32) * 16))); -typedef UInt32 UInt32x32 __attribute__ ((vector_size (sizeof(UInt32) * 32))); -typedef UInt32 UInt32x64 __attribute__ ((vector_size (sizeof(UInt32) * 64))); typedef UInt16 UInt16x2 __attribute__ ((vector_size (sizeof(UInt16) * 2))); typedef UInt16 UInt16x4 __attribute__ ((vector_size (sizeof(UInt16) * 4))); typedef UInt16 UInt16x8 __attribute__ ((vector_size (sizeof(UInt16) * 8))); typedef UInt16 UInt16x16 __attribute__ ((vector_size (sizeof(UInt16) * 16))); typedef UInt16 UInt16x32 __attribute__ ((vector_size (sizeof(UInt16) * 32))); -typedef UInt16 UInt16x64 __attribute__ ((vector_size (sizeof(UInt16) * 64))); typedef UInt8 UInt8x2 __attribute__ ((vector_size (sizeof(UInt8) * 2))); typedef UInt8 UInt8x4 __attribute__ ((vector_size (sizeof(UInt8) * 4))); @@ -65,15 +55,12 @@ namespace detail using UInt8Type = UInt8x16; using UInt16Type = UInt16x16; using UInt32Type = UInt32x16; - using UInt64Type = UInt64x16; }; template <> struct DummyStruct<32> { using UInt8Type = UInt8x32; using UInt16Type = UInt16x32; - using UInt32Type = UInt32x32; - using UInt64Type = UInt64x32; }; } @@ -88,14 +75,4 @@ using UInt32x = typename detail::DummyStruct::UInt32Type; template using UInt64x = typename detail::DummyStruct::UInt64Type; -/* Casts vectors of the same size. - * UInt32x4 x{}; - * UInt64x4 y = ConvertVector(x); - */ -// template -// inline To ConvertVector(From a) -// { -// return __builtin_convertvector(a, To); -// } - } diff --git a/src/Functions/generateUUIDv4.cpp b/src/Functions/generateUUIDv4.cpp index d543226ba5c..04dd5877560 100644 --- a/src/Functions/generateUUIDv4.cpp +++ b/src/Functions/generateUUIDv4.cpp @@ -5,17 +5,24 @@ namespace DB { +#define DECLARE_SEVERAL_IMPLEMENTATIONS(...) \ +DECLARE_DEFAULT_CODE (__VA_ARGS__) \ +DECLARE_AVX2_SPECIFIC_CODE(__VA_ARGS__) + +DECLARE_SEVERAL_IMPLEMENTATIONS( + class FunctionGenerateUUIDv4 : public IFunction { public: static constexpr auto name = "generateUUIDv4"; - static FunctionPtr create(const Context &) { return std::make_shared(); } String getName() const override { return name; } + static String getImplementationTag() { return ToString(BuildArch); } + size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const DataTypes &) const override @@ -32,8 +39,9 @@ public: size_t size = input_rows_count; vec_to.resize(size); - // TODO(dakovalkov): rewrite this workaround - TargetSpecific::Default::RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(UInt128)); + + /// RandImpl is target-dependent and is not the same in different TargetSpecific namespaces. + RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(UInt128)); for (UInt128 & uuid: vec_to) { @@ -47,6 +55,35 @@ public: } }; +) // DECLARE_SEVERAL_IMPLEMENTATIONS +#undef DECLARE_SEVERAL_IMPLEMENTATIONS + +class FunctionGenerateUUIDv4 : public TargetSpecific::Default::FunctionGenerateUUIDv4 +{ +public: + FunctionGenerateUUIDv4(const Context & context) : selector(context) + { + selector.registerImplementation(); + + selector.registerImplementation(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + selector.selectAndExecute(block, arguments, result, input_rows_count); + } + + static FunctionPtr create(const Context & context) + { + return std::make_shared(context); + } + +private: + ImplementationSelector selector; +}; + void registerFunctionGenerateUUIDv4(FunctionFactory & factory) { factory.registerFunction(); diff --git a/tests/performance/rand.xml b/tests/performance/rand.xml index bd34a7a83d8..ed629e5a2a7 100644 --- a/tests/performance/rand.xml +++ b/tests/performance/rand.xml @@ -20,4 +20,5 @@ SELECT count() FROM (SELECT rand() FROM {table}) SELECT count() FROM (SELECT rand64() FROM {table}) + SELECT count() FROM (SELECT generateUUIDv4() FROM {table}) From 991cbf397aabc03782da6713d35d2e44ee6ae835 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Tue, 26 May 2020 13:15:44 +0200 Subject: [PATCH 040/211] Thread safe performance statistics --- src/Functions/FunctionsRandom.cpp | 2 +- src/Functions/FunctionsRandom.h | 1 + src/Functions/PerformanceAdaptors.h | 203 ++++++++++++++++------------ 3 files changed, 119 insertions(+), 87 deletions(-) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 6b2e79e90ca..2c7b2e5f1f5 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -109,7 +109,7 @@ inline UInt64x4 CombineValues(UInt64x4 a, UInt64x4 b) /// Now every 8-byte value in xa is xx....xx and every value in xb is ..xxxx.. where x is random byte we want to use. /// Just blend them to get the result vector. /// result = xa[0],xb[1,2],xa[3,4],xb[5,6],xa[7,8],xb[9,10],xa[11,12],xb[13,14],xa[15] - __m256i result = _mm256_blend_epi16(xa, xb, 0x66); + __m256i result = _mm256_blend_epi16(xa, xb, 0x66); return reinterpret_cast(result); } diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index e10b249df8e..8cbe4286285 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -40,6 +40,7 @@ DECLARE_MULTITARGET_CODE( struct RandImpl { + /// Fill memory with random data. The memory region must be 15-bytes padded. static void execute(char * output, size_t size); static String getImplementationTag() { return ToString(BuildArch); } }; diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index daa65300570..9dbc2a68f86 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -6,6 +6,7 @@ #include #include +#include #include /// This file contains Adaptors which help to combine several implementations of the function. @@ -20,104 +21,138 @@ namespace ErrorCodes extern const int NO_SUITABLE_FUNCTION_IMPLEMENTATION; } -// TODO(dakovalkov): This is copied and pasted struct from LZ4_decompress_faster.h with little changes. -struct PerformanceStatistics +namespace detail { - struct Element + class PerformanceStatistics { - double count = 0; - double sum = 0; - - double adjustedCount() const + public: + size_t select(bool considarable) { - return count - NUM_INVOCATIONS_TO_THROW_OFF; + /// We don't need to choose/measure anything if there's only one variant. + if (size() == 1) + return 0; + + std::lock_guard guard(lock); + + size_t best = 0; + double best_sample = data[0].sample(rng); + + for (size_t i = 1; i < data.size(); ++i) + { + double sample = data[i].sample(rng); + if (sample < best_sample) + { + best_sample = sample; + best = i; + } + } + + if (considarable) + data[best].run(); + + return best; } - double mean() const + void complete(size_t id, double seconds, double bytes) { - return sum / adjustedCount(); + if (size() == 1) + return; + + std::lock_guard guard(lock); + data[id].complete(seconds, bytes); } - /// For better convergence, we don't use proper estimate of stddev. - /// We want to eventually separate between two algorithms even in case - /// when there is no statistical significant difference between them. - double sigma() const + size_t size() const { - return mean() / sqrt(adjustedCount()); + return data.size(); } - void update(double seconds, double bytes) + bool empty() const { - ++count; - - if (count > NUM_INVOCATIONS_TO_THROW_OFF) - sum += seconds / bytes; + return size() == 0; } - double sample(pcg64 & stat_rng) const + void emplace_back() { - /// If there is a variant with not enough statistics, always choose it. - /// And in that case prefer variant with less number of invocations. + data.emplace_back(); + } + + private: + struct Element + { + int completed_count = 0; + int running_count = 0; + double sum = 0; + + int adjustedCount() const + { + return completed_count - NUM_INVOCATIONS_TO_THROW_OFF; + } + + double mean() const + { + return sum / adjustedCount(); + } + + /// For better convergence, we don't use proper estimate of stddev. + /// We want to eventually separate between two algorithms even in case + /// when there is no statistical significant difference between them. + double sigma() const + { + return mean() / sqrt(adjustedCount()); + } + + void run() + { + ++running_count; + } + + void complete(double seconds, double bytes) + { + --running_count; + ++completed_count; + + if (adjustedCount() > 0) + sum += seconds / bytes; + } + + double sample(pcg64 & stat_rng) const + { + /// If there is a variant with not enough statistics, always choose it. + /// And in that case prefer variant with less number of invocations. + if (adjustedCount() < 2) + return adjustedCount() - 1 + running_count * 2; - if (adjustedCount() < 2) - return adjustedCount() - 1; - else return std::normal_distribution<>(mean(), sigma())(stat_rng); - } + } + }; + + std::vector data; + std::mutex lock; + /// It's Ok that generator is not seeded. + pcg64 rng; + /// Cold invocations may be affected by additional memory latencies. Don't take first invocations into account. + static constexpr int NUM_INVOCATIONS_TO_THROW_OFF = 2; }; - /// Cold invocations may be affected by additional memory latencies. Don't take first invocations into account. - static constexpr double NUM_INVOCATIONS_TO_THROW_OFF = 2; + template + std::true_type hasImplementationTagTest(const T&); + std::false_type hasImplementationTagTest(...); - /// How to select method to run. - /// -1 - automatically, based on statistics (default); - /// -2 - choose methods in round robin fashion (for performance testing). - /// >= 0 - always choose specified method (for performance testing); - ssize_t choose_method = -1; + template + constexpr bool has_implementation_tag = decltype(hasImplementationTagTest(std::declval()))::value; - std::vector data; - - /// It's Ok that generator is not seeded. - pcg64 rng; - - /// To select from different algorithms we use a kind of "bandits" algorithm. - /// Sample random values from estimated normal distributions and choose the minimal. - size_t select() + template + String getImplementationTag(TargetArch arch) { - if (choose_method < 0) - { - std::vector samples(data.size()); - for (size_t i = 0; i < data.size(); ++i) - samples[i] = choose_method == -1 - ? data[i].sample(rng) - : data[i].adjustedCount(); - - return std::min_element(samples.begin(), samples.end()) - samples.begin(); - } + if constexpr (has_implementation_tag) + return ToString(arch) + "_" + T::getImplementationTag(); else - return choose_method; + return ToString(arch); } +} - size_t size() const - { - return data.size(); - } - - bool empty() const - { - return size() == 0; - } - - void emplace_back() - { - data.emplace_back(); - } - - PerformanceStatistics() {} - PerformanceStatistics(ssize_t choose_method_) : choose_method(choose_method_) {} -}; - -/* Class which is used to store implementations for the function and selecting the best one to run +/* Class which is used to store implementations for the function and to select the best one to run * based on processor architecture and statistics from previous runs. * * FunctionInterface is typically IFunction or IExecutableFunctionImpl, but practically it can be @@ -170,7 +205,10 @@ public: throw Exception("There are no available implementations for function " "TODO(dakovalkov): add name", ErrorCodes::NO_SUITABLE_FUNCTION_IMPLEMENTATION); - auto id = statistics.select(); + /// Statistics shouldn't rely on small blocks. + bool considerable = (input_rows_count > 1000); + + size_t id = statistics.select(considerable); Stopwatch watch; if constexpr (std::is_same_v) @@ -180,17 +218,10 @@ public: watch.stop(); - // TODO(dakovalkov): Calculate something more informative. - size_t rows_summary = 0; - for (auto i : arguments) + if (considerable) { - rows_summary += block.getByPosition(i).column->size(); - } - rows_summary += block.getByPosition(result).column->size(); - - if (rows_summary >= 1000) - { - statistics.data[id].update(watch.elapsedSeconds(), rows_summary); + // TODO(dakovalkov): Calculate something more informative than rows count. + statistics.complete(id, watch.elapsedSeconds(), input_rows_count); } } @@ -210,7 +241,7 @@ public: { // TODO(dakovalkov): make this option better. const auto & choose_impl = context.getSettingsRef().function_implementation.value; - if (choose_impl.empty() || choose_impl == FunctionImpl::getImplementationTag()) + if (choose_impl.empty() || choose_impl == detail::getImplementationTag(Arch)) { implementations.emplace_back(std::make_shared(std::forward(args)...)); statistics.emplace_back(); @@ -221,7 +252,7 @@ public: private: const Context & context; std::vector implementations; - PerformanceStatistics statistics; + detail::PerformanceStatistics statistics; }; } From ef030349ff29c7550b9c3b2931cf4db57d9a6e11 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Tue, 26 May 2020 17:56:46 +0200 Subject: [PATCH 041/211] Add hashes to multitarget code, a lot of cosmetics --- src/Functions/CMakeLists.txt | 4 +- src/Functions/FunctionBinaryArithmetic.h | 81 +++++++++---------- src/Functions/FunctionStartsEndsWith.h | 21 +++-- src/Functions/FunctionsHashing.h | 42 +++++++++- src/Functions/FunctionsRandom.h | 15 +--- src/Functions/PerformanceAdaptors.h | 28 +++++-- src/Functions/RandXorshift.h | 15 ++-- src/Functions/TargetSpecific.h | 54 ++++++++----- src/Functions/generateUUIDv4.cpp | 2 + src/Functions/randConstant.cpp | 2 +- tests/performance/arithmetic.xml | 3 +- .../synthetic_hardware_benchmark.xml | 3 +- 12 files changed, 156 insertions(+), 114 deletions(-) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 8c9cf159e30..2cc3208f6c4 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -85,9 +85,9 @@ endif() option(ENABLE_MULTITARGET_CODE "" ON) if (ENABLE_MULTITARGET_CODE) - add_definitions(-DUSE_MULTITARGET_CODE=1) + add_definitions(-DENABLE_MULTITARGET_CODE=1) else() - add_definitions(-DUSE_MULTITARGET_CODE=0) + add_definitions(-DENABLE_MULTITARGET_CODE=0) endif() target_link_libraries(clickhouse_functions PUBLIC "simdxorshift") diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index c311b8d5d0a..9a5d610d2af 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -102,65 +102,56 @@ struct BinaryOperationImplBase static void vectorVector(const A * __restrict a, const B * __restrict b, ResultType * __restrict c, size_t size) { - if constexpr (UseMultitargetCode) - { - if (IsArchSupported(TargetArch::AVX512F)) - TargetSpecific::AVX512F::BinaryOperationImplBase::vectorVector(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX2)) - TargetSpecific::AVX2::BinaryOperationImplBase::vectorVector(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX)) - TargetSpecific::AVX::BinaryOperationImplBase::vectorVector(a, b, c, size); - else if (IsArchSupported(TargetArch::SSE42)) - TargetSpecific::SSE42::BinaryOperationImplBase::vectorVector(a, b, c, size); - else - TargetSpecific::Default::BinaryOperationImplBase::vectorVector(a, b, c, size); - } + #if USE_MULTITARGET_CODE + if (IsArchSupported(TargetArch::AVX512F)) + TargetSpecific::AVX512F::BinaryOperationImplBase::vectorVector(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX2)) + TargetSpecific::AVX2::BinaryOperationImplBase::vectorVector(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX)) + TargetSpecific::AVX::BinaryOperationImplBase::vectorVector(a, b, c, size); + else if (IsArchSupported(TargetArch::SSE42)) + TargetSpecific::SSE42::BinaryOperationImplBase::vectorVector(a, b, c, size); else - { TargetSpecific::Default::BinaryOperationImplBase::vectorVector(a, b, c, size); - } + #else + TargetSpecific::Default::BinaryOperationImplBase::vectorVector(a, b, c, size); + #endif } static void vectorConstant(const A * __restrict a, B b, ResultType * __restrict c, size_t size) { - if constexpr (UseMultitargetCode) - { - if (IsArchSupported(TargetArch::AVX512F)) - TargetSpecific::AVX512F::BinaryOperationImplBase::vectorConstant(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX2)) - TargetSpecific::AVX2::BinaryOperationImplBase::vectorConstant(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX)) - TargetSpecific::AVX::BinaryOperationImplBase::vectorConstant(a, b, c, size); - else if (IsArchSupported(TargetArch::SSE42)) - TargetSpecific::SSE42::BinaryOperationImplBase::vectorConstant(a, b, c, size); - else - TargetSpecific::Default::BinaryOperationImplBase::vectorConstant(a, b, c, size); - } + #if USE_MULTITARGET_CODE + if (IsArchSupported(TargetArch::AVX512F)) + TargetSpecific::AVX512F::BinaryOperationImplBase::vectorConstant(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX2)) + TargetSpecific::AVX2::BinaryOperationImplBase::vectorConstant(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX)) + TargetSpecific::AVX::BinaryOperationImplBase::vectorConstant(a, b, c, size); + else if (IsArchSupported(TargetArch::SSE42)) + TargetSpecific::SSE42::BinaryOperationImplBase::vectorConstant(a, b, c, size); else - { TargetSpecific::Default::BinaryOperationImplBase::vectorConstant(a, b, c, size); - } + #else + TargetSpecific::Default::BinaryOperationImplBase::vectorConstant(a, b, c, size); + #endif } static void constantVector(A a, const B * __restrict b, ResultType * __restrict c, size_t size) { - if constexpr (UseMultitargetCode) - { - if (IsArchSupported(TargetArch::AVX512F)) - TargetSpecific::AVX512F::BinaryOperationImplBase::constantVector(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX2)) - TargetSpecific::AVX2::BinaryOperationImplBase::constantVector(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX)) - TargetSpecific::AVX::BinaryOperationImplBase::constantVector(a, b, c, size); - else if (IsArchSupported(TargetArch::SSE42)) - TargetSpecific::SSE42::BinaryOperationImplBase::constantVector(a, b, c, size); - else - TargetSpecific::Default::BinaryOperationImplBase::constantVector(a, b, c, size); - } + #if USE_MULTITARGET_CODE + if (IsArchSupported(TargetArch::AVX512F)) + TargetSpecific::AVX512F::BinaryOperationImplBase::constantVector(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX2)) + TargetSpecific::AVX2::BinaryOperationImplBase::constantVector(a, b, c, size); + else if (IsArchSupported(TargetArch::AVX)) + TargetSpecific::AVX::BinaryOperationImplBase::constantVector(a, b, c, size); + else if (IsArchSupported(TargetArch::SSE42)) + TargetSpecific::SSE42::BinaryOperationImplBase::constantVector(a, b, c, size); else - { TargetSpecific::Default::BinaryOperationImplBase::constantVector(a, b, c, size); - } + #else + TargetSpecific::Default::BinaryOperationImplBase::constantVector(a, b, c, size); + #endif } static ResultType constantConstant(A a, B b) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 76aa4530c99..b148653e1b3 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -151,17 +151,16 @@ public: selector.registerImplementation>(); - if constexpr (UseMultitargetCode) - { - selector.registerImplementation>(); - selector.registerImplementation>(); - selector.registerImplementation>(); - selector.registerImplementation>(); - } + #if USE_MULTITARGET_CODE + selector.registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); + #endif } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 6f00981a22a..4562b9001a9 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -615,8 +615,6 @@ public: return name; } - static String getImplementationTag() { return ToString(BuildArch); } - size_t getNumberOfArguments() const override { return 1; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override @@ -661,11 +659,13 @@ public: { selector.registerImplementation>(); - + + #if USE_MULTITARGET_CODE selector.registerImplementation>(); selector.registerImplementation>(); + #endif } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override @@ -682,12 +682,13 @@ private: ImplementationSelector selector; }; +DECLARE_MULTITARGET_CODE( + template class FunctionAnyHash : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } private: using ToType = typename Impl::ReturnType; @@ -974,6 +975,39 @@ public: } }; +) // DECLARE_MULTITARGET_CODE + +template +class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash +{ +public: + FunctionAnyHash(const Context & context) : selector(context) + { + selector.registerImplementation>(); + + #if USE_MULTITARGET_CODE + selector.registerImplementation>(); + selector.registerImplementation>(); + #endif + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + selector.selectAndExecute(block, arguments, result, input_rows_count); + } + + static FunctionPtr create(const Context & context) + { + return std::make_shared(context); + } + +private: + ImplementationSelector selector; +}; + struct URLHashImpl { diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 8cbe4286285..346c94e1d9f 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -42,7 +42,6 @@ struct RandImpl { /// Fill memory with random data. The memory region must be 15-bytes padded. static void execute(char * output, size_t size); - static String getImplementationTag() { return ToString(BuildArch); } }; ) // DECLARE_MULTITARGET_CODE @@ -58,11 +57,6 @@ public: return name; } - static String getImplementationTag() - { - return RandImpl::getImplementationTag(); - } - bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } bool useDefaultImplementationForNulls() const override { return false; } @@ -102,11 +96,10 @@ public: selector.registerImplementation>(); - if constexpr (UseMultitargetCode) - { - selector.registerImplementation>(); - } + #if USE_MULTITARGET_CODE + selector.registerImplementation>(); + #endif } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index 9dbc2a68f86..1d4b6be6102 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -9,9 +9,9 @@ #include #include -/// This file contains Adaptors which help to combine several implementations of the function. -/// Adaptors check that implementation can be executed on the current platform and choose -/// that one which works faster according to previous runs. +/* This file contains helper class ImplementationSelector. It makes easier to combine + * several implementations of IFunction/IExecutableFunctionImpl. + */ namespace DB { @@ -120,9 +120,16 @@ namespace detail { /// If there is a variant with not enough statistics, always choose it. /// And in that case prefer variant with less number of invocations. - if (adjustedCount() < 2) - return adjustedCount() - 1 + running_count * 2; + if (adjustedCount() < 2) + { + // TODO(dakovalkov): rewrite it. + int all_count = adjustedCount() + running_count; + if (all_count < 3) + return all_count - 2; + else + return adjustedCount() + running_count * 100; + } return std::normal_distribution<>(mean(), sigma())(stat_rng); } }; @@ -142,6 +149,9 @@ namespace detail template constexpr bool has_implementation_tag = decltype(hasImplementationTagTest(std::declval()))::value; + /* Implementation tag is used to run specific implementation (for debug/testing purposes). + * It can be specified via static method ::getImplementationTag() in Function (optional). + */ template String getImplementationTag(TargetArch arch) { @@ -161,8 +171,9 @@ namespace detail * Example of usage: * * class MyDefaulImpl : public IFunction {...}; - * class MySecondImpl : public IFunction {...}; + * DECLARE_AVX2_SPECIFIC_CODE( * class MyAVX2Impl : public IFunction {...}; + * ) * * /// All methods but execute/executeImpl are usually not bottleneck, so just use them from * /// default implementation. @@ -172,8 +183,9 @@ namespace detail * /// Register all implementations in constructor. * /// There could be as many implementation for every target as you want. * selector.registerImplementation(); - * selector.registerImplementation(); - * selector.registreImplementation(); + * #if USE_MULTITARGET_CODE + * selector.registreImplementation(); + * #endif * } * * void executeImpl(...) override { diff --git a/src/Functions/RandXorshift.h b/src/Functions/RandXorshift.h index 49655d637f2..1d109adc087 100644 --- a/src/Functions/RandXorshift.h +++ b/src/Functions/RandXorshift.h @@ -23,7 +23,7 @@ struct RandXorshiftImpl struct RandXorshiftImpl2 { static void execute(char * output, size_t size); - static String getImplementationTag() { return ToString(BuildArch) + "_v2"; } + static String getImplementationTag() { return "v2"; } }; ) // DECLARE_MULTITARGET_CODE @@ -37,13 +37,12 @@ public: selector.registerImplementation>(); - if constexpr (UseMultitargetCode) - { - selector.registerImplementation>(); - selector.registerImplementation>(); - } + #if USE_MULTITARGET_CODE + selector.registerImplementation>(); + selector.registerImplementation>(); + #endif } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override diff --git a/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h index 11dae939bbd..ed9c0d3c244 100644 --- a/src/Functions/TargetSpecific.h +++ b/src/Functions/TargetSpecific.h @@ -4,16 +4,22 @@ /* This file contains macros and helpers for writing platform-dependent code. * - * Macros DECLARE__SPECIFIC_CODE will wrap code inside them into the + * Macros DECLARE__SPECIFIC_CODE will wrap code inside it into the * namespace TargetSpecific:: and enable Arch-specific compile options. * Thus, it's allowed to call functions inside these namespaces only after * checking platform in runtime (see IsArchSupported() below). * + * If compiler is not gcc/clang or target isn't x86_64 or ENABLE_MULTITARGET_CODE + * was set to OFF in cmake, all code inside these macroses will be removed and + * USE_MUTLITARGE_CODE will be set to 0. Use #if USE_MUTLITARGE_CODE whenever you + * use anything from this namespaces. + * * For similarities there is a macros DECLARE_DEFAULT_CODE, which wraps code * into the namespace TargetSpecific::Default but dosn't specify any additional - * copile options. + * copile options. Functions and classes inside this macros are available regardless + * of USE_MUTLITARGE_CODE. * - * Example: + * Example of usage: * * DECLARE_DEFAULT_CODE ( * int funcImpl() { @@ -28,15 +34,17 @@ * ) // DECLARE_DEFAULT_CODE * * int func() { + * #if USE_MULTITARGET_CODE * if (IsArchSupported(TargetArch::AVX2)) * return TargetSpecifc::AVX2::funcImpl(); + * #endif * return TargetSpecifc::Default::funcImpl(); * } * * Sometimes code may benefit from compiling with different options. - * For these purposes use DECLARE_MULTITARGET_CODE macros. It will create several - * copies of the code and compile it with different options. These copies are - * available via TargetSpecifc namespaces described above. + * For these purposes use DECLARE_MULTITARGET_CODE macros. It will create a copy + * of the code for every supported target and compile it with different options. + * These copies are available via TargetSpecifc namespaces described above. * * Inside every TargetSpecific namespace there is a constexpr variable BuildArch, * which indicates the target platform for current code. @@ -50,16 +58,16 @@ * iteration_size = 2 * else if constexpr (BuildArch == TargetArch::AVX || BuildArch == TargetArch::AVX2) * iteration_size = 4; - * else if constexpr (BuildArch == TargetArch::AVX512) - * iteration_size = 8; * for (int i = 0; i < size; i += iteration_size) * ... * } * ) // DECLARE_MULTITARGET_CODE * - * // All 5 versions of func are available here. Use runtime detection to choose one. + * // All target-specific and default implementations are available here via + * TargetSpecific::::funcImpl. Use runtime detection to choose one. * - * If you want to write IFunction or IExecutableFuncionImpl with runtime dispatching, see PerformanceAdaptors.h. + * If you want to write IFunction or IExecutableFuncionImpl with several implementations + * see PerformanceAdaptors.h. */ namespace DB @@ -74,24 +82,24 @@ enum class TargetArch : UInt32 AVX512F = (1 << 3), }; -// Runtime detection. +/// Runtime detection. bool IsArchSupported(TargetArch arch); String ToString(TargetArch arch); -#if USE_MULTITARGET_CODE && defined(__GNUC__) && defined(__x86_64__) +#if ENABLE_MULTITARGET_CODE && defined(__GNUC__) && defined(__x86_64__) -constexpr bool UseMultitargetCode = true; +#define USE_MULTITARGET_CODE 1 #if defined(__clang__) # define BEGIN_AVX512F_SPECIFIC_CODE \ - _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx,avx2,avx512f\"))),apply_to=function)") + _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f\"))),apply_to=function)") # define BEGIN_AVX2_SPECIFIC_CODE \ - _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx,avx2\"))),apply_to=function)") + _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2\"))),apply_to=function)") # define BEGIN_AVX_SPECIFIC_CODE \ - _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx,avx\"))),apply_to=function)") + _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx\"))),apply_to=function)") # define BEGIN_SSE42_SPECIFIC_CODE \ - _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,mmx\"))),apply_to=function)") + _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt\"))),apply_to=function)") # define END_TARGET_SPECIFIC_CODE \ _Pragma("clang attribute pop") @@ -102,16 +110,16 @@ constexpr bool UseMultitargetCode = true; #else # define BEGIN_AVX512F_SPECIFIC_CODE \ _Pragma("GCC push_options") \ - _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2,avx512f,tune=native\")") + _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,tune=native\")") # define BEGIN_AVX2_SPECIFIC_CODE \ _Pragma("GCC push_options") \ - _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,avx2,tune=native\")") + _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,tune=native\")") # define BEGIN_AVX_SPECIFIC_CODE \ _Pragma("GCC push_options") \ - _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,avx,tune=native\")") + _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,tune=native\")") # define BEGIN_SSE42_SPECIFIC_CODE \ _Pragma("GCC push_options") \ - _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,abm,mmx,tune=native\")") + _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,tune=native\")") # define END_TARGET_SPECIFIC_CODE \ _Pragma("GCC pop_options") @@ -158,8 +166,10 @@ END_TARGET_SPECIFIC_CODE #else -constexpr bool UseMultitargetCode = false; +#define USE_MULTITARGET_CODE 0 +/* Multitarget code is disabled, just delete target-specific code. + */ #define DECLARE_SSE42_SPECIFIC_CODE(...) #define DECLARE_AVX_SPECIFIC_CODE(...) #define DECLARE_AVX2_SPECIFIC_CODE(...) diff --git a/src/Functions/generateUUIDv4.cpp b/src/Functions/generateUUIDv4.cpp index 04dd5877560..a205f853d2a 100644 --- a/src/Functions/generateUUIDv4.cpp +++ b/src/Functions/generateUUIDv4.cpp @@ -66,8 +66,10 @@ public: selector.registerImplementation(); + #if USE_MULTITARGET_CODE selector.registerImplementation(); + #endif } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override diff --git a/src/Functions/randConstant.cpp b/src/Functions/randConstant.cpp index 3eba5abf10d..ebf2f752b66 100644 --- a/src/Functions/randConstant.cpp +++ b/src/Functions/randConstant.cpp @@ -99,7 +99,7 @@ public: argument_types.emplace_back(arguments.back().type); typename ColumnVector::Container vec_to(1); - // TODO(dakovalkov): Rewrite this workaround + TargetSpecific::Default::RandImpl::execute(reinterpret_cast(vec_to.data()), sizeof(ToType)); ToType value = vec_to[0]; diff --git a/tests/performance/arithmetic.xml b/tests/performance/arithmetic.xml index 45f0d62f227..e56d35d43b9 100644 --- a/tests/performance/arithmetic.xml +++ b/tests/performance/arithmetic.xml @@ -1,4 +1,5 @@ - + 30000000000 diff --git a/tests/performance/synthetic_hardware_benchmark.xml b/tests/performance/synthetic_hardware_benchmark.xml index 2233bfeca8d..2688c5a1aec 100644 --- a/tests/performance/synthetic_hardware_benchmark.xml +++ b/tests/performance/synthetic_hardware_benchmark.xml @@ -1,4 +1,5 @@ - + 30000000000 From 71fabcedc4939b6ef917072d3e224de4bc65f6b8 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Tue, 26 May 2020 18:29:37 +0200 Subject: [PATCH 042/211] Fix test --- tests/performance/arithmetic.xml | 1 + tests/performance/synthetic_hardware_benchmark.xml | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/performance/arithmetic.xml b/tests/performance/arithmetic.xml index e56d35d43b9..69b9bf30e45 100644 --- a/tests/performance/arithmetic.xml +++ b/tests/performance/arithmetic.xml @@ -1,5 +1,6 @@ + 30000000000 diff --git a/tests/performance/synthetic_hardware_benchmark.xml b/tests/performance/synthetic_hardware_benchmark.xml index 2688c5a1aec..deae39ab80f 100644 --- a/tests/performance/synthetic_hardware_benchmark.xml +++ b/tests/performance/synthetic_hardware_benchmark.xml @@ -1,5 +1,6 @@ + 30000000000 From cdb353856dee0ef0cb39022cc4b6623469a92d15 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 28 May 2020 13:39:48 +0200 Subject: [PATCH 043/211] remove vectorization from binary arithmetic --- src/Functions/FunctionBinaryArithmetic.h | 80 ++---------------------- tests/performance/arithmetic.xml | 4 +- 2 files changed, 6 insertions(+), 78 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 9a5d610d2af..292f7da0475 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -54,8 +54,11 @@ namespace ErrorCodes extern const int CANNOT_ADD_DIFFERENT_AGGREGATE_STATES; } -DECLARE_MULTITARGET_CODE( - +/** Arithmetic operations: +, -, *, /, %, + * intDiv (integer division) + * Bitwise operations: |, &, ^, ~. + * Etc. + */ template struct BinaryOperationImplBase { @@ -86,79 +89,6 @@ struct BinaryOperationImplBase } }; -) // DECLARE_MULTITARGET_CODE - - -/** Arithmetic operations: +, -, *, /, %, - * intDiv (integer division) - * Bitwise operations: |, &, ^, ~. - * Etc. - */ -template -struct BinaryOperationImplBase -{ - using ResultType = ResultType_; - static const constexpr bool allow_fixed_string = false; - - static void vectorVector(const A * __restrict a, const B * __restrict b, ResultType * __restrict c, size_t size) - { - #if USE_MULTITARGET_CODE - if (IsArchSupported(TargetArch::AVX512F)) - TargetSpecific::AVX512F::BinaryOperationImplBase::vectorVector(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX2)) - TargetSpecific::AVX2::BinaryOperationImplBase::vectorVector(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX)) - TargetSpecific::AVX::BinaryOperationImplBase::vectorVector(a, b, c, size); - else if (IsArchSupported(TargetArch::SSE42)) - TargetSpecific::SSE42::BinaryOperationImplBase::vectorVector(a, b, c, size); - else - TargetSpecific::Default::BinaryOperationImplBase::vectorVector(a, b, c, size); - #else - TargetSpecific::Default::BinaryOperationImplBase::vectorVector(a, b, c, size); - #endif - } - - static void vectorConstant(const A * __restrict a, B b, ResultType * __restrict c, size_t size) - { - #if USE_MULTITARGET_CODE - if (IsArchSupported(TargetArch::AVX512F)) - TargetSpecific::AVX512F::BinaryOperationImplBase::vectorConstant(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX2)) - TargetSpecific::AVX2::BinaryOperationImplBase::vectorConstant(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX)) - TargetSpecific::AVX::BinaryOperationImplBase::vectorConstant(a, b, c, size); - else if (IsArchSupported(TargetArch::SSE42)) - TargetSpecific::SSE42::BinaryOperationImplBase::vectorConstant(a, b, c, size); - else - TargetSpecific::Default::BinaryOperationImplBase::vectorConstant(a, b, c, size); - #else - TargetSpecific::Default::BinaryOperationImplBase::vectorConstant(a, b, c, size); - #endif - } - - static void constantVector(A a, const B * __restrict b, ResultType * __restrict c, size_t size) - { - #if USE_MULTITARGET_CODE - if (IsArchSupported(TargetArch::AVX512F)) - TargetSpecific::AVX512F::BinaryOperationImplBase::constantVector(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX2)) - TargetSpecific::AVX2::BinaryOperationImplBase::constantVector(a, b, c, size); - else if (IsArchSupported(TargetArch::AVX)) - TargetSpecific::AVX::BinaryOperationImplBase::constantVector(a, b, c, size); - else if (IsArchSupported(TargetArch::SSE42)) - TargetSpecific::SSE42::BinaryOperationImplBase::constantVector(a, b, c, size); - else - TargetSpecific::Default::BinaryOperationImplBase::constantVector(a, b, c, size); - #else - TargetSpecific::Default::BinaryOperationImplBase::constantVector(a, b, c, size); - #endif - } - - static ResultType constantConstant(A a, B b) - { - return Op::template apply(a, b); - } -}; template struct FixedStringOperationImpl diff --git a/tests/performance/arithmetic.xml b/tests/performance/arithmetic.xml index 69b9bf30e45..45f0d62f227 100644 --- a/tests/performance/arithmetic.xml +++ b/tests/performance/arithmetic.xml @@ -1,6 +1,4 @@ - - + 30000000000 From 13e1102f03e35dce5f29d9434ff84ee9dff3c605 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 28 May 2020 13:46:47 +0200 Subject: [PATCH 044/211] Disable xorshift --- contrib/CMakeLists.txt | 1 - src/Functions/CMakeLists.txt | 3 --- src/Functions/RandXorshift.cpp | 3 +++ src/Functions/RandXorshift.h | 5 +++++ src/Functions/registerFunctionsRandom.cpp | 2 -- 5 files changed, 8 insertions(+), 6 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 344a06f29b7..d122188ad0b 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -27,7 +27,6 @@ add_subdirectory (murmurhash) add_subdirectory (replxx-cmake) add_subdirectory (ryu-cmake) add_subdirectory (unixodbc-cmake) -add_subdirectory (SIMDxorshift-cmake) add_subdirectory (poco-cmake) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 2cc3208f6c4..e999955086e 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -90,9 +90,6 @@ else() add_definitions(-DENABLE_MULTITARGET_CODE=0) endif() -target_link_libraries(clickhouse_functions PUBLIC "simdxorshift") -message(STATUS "Using SIMDXORSHIFT ${SIMDXORSHIFT_LIBRARY}") - add_subdirectory(GatherUtils) target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_gatherutils) diff --git a/src/Functions/RandXorshift.cpp b/src/Functions/RandXorshift.cpp index 9f1dded700c..4fa2280861f 100644 --- a/src/Functions/RandXorshift.cpp +++ b/src/Functions/RandXorshift.cpp @@ -1,3 +1,5 @@ +/// Disable xorshift +#if 0 #include #include #include @@ -162,3 +164,4 @@ void registerFunctionRandXorshift(FunctionFactory & factory) } } +#endif diff --git a/src/Functions/RandXorshift.h b/src/Functions/RandXorshift.h index 1d109adc087..8e068cf5dff 100644 --- a/src/Functions/RandXorshift.h +++ b/src/Functions/RandXorshift.h @@ -1,5 +1,8 @@ #pragma once +/// disable xorshift +#if 0 + #include #include #include @@ -60,3 +63,5 @@ private: }; } + +#endif diff --git a/src/Functions/registerFunctionsRandom.cpp b/src/Functions/registerFunctionsRandom.cpp index 422ec91f025..3638474c4fe 100644 --- a/src/Functions/registerFunctionsRandom.cpp +++ b/src/Functions/registerFunctionsRandom.cpp @@ -10,7 +10,6 @@ void registerFunctionRandomPrintableASCII(FunctionFactory & factory); void registerFunctionRandomString(FunctionFactory & factory); void registerFunctionRandomFixedString(FunctionFactory & factory); void registerFunctionRandomStringUTF8(FunctionFactory & factory); -void registerFunctionRandXorshift(FunctionFactory & factory); void registerFunctionsRandom(FunctionFactory & factory) { @@ -22,7 +21,6 @@ void registerFunctionsRandom(FunctionFactory & factory) registerFunctionRandomString(factory); registerFunctionRandomFixedString(factory); registerFunctionRandomStringUTF8(factory); - registerFunctionRandXorshift(factory); } } From 07bdd9eaf6189703715b2dfe861ecf5913953752 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 28 May 2020 13:48:56 +0200 Subject: [PATCH 045/211] Fix style issues --- src/Functions/FunctionsHashing.h | 6 +++--- src/Functions/PerformanceAdaptors.h | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 4562b9001a9..7edb8937275 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -659,7 +659,7 @@ public: { selector.registerImplementation>(); - + #if USE_MULTITARGET_CODE selector.registerImplementation>(); @@ -670,9 +670,9 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { - selector.selectAndExecute(block, arguments, result, input_rows_count); + selector.selectAndExecute(block, arguments, result, input_rows_count); } - + static FunctionPtr create(const Context & context) { return std::make_shared(context); diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index 1d4b6be6102..de321ee5605 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -32,7 +32,7 @@ namespace detail if (size() == 1) return 0; - std::lock_guard guard(lock); + std::lock_guard guard(lock); size_t best = 0; double best_sample = data[0].sample(rng); @@ -57,8 +57,8 @@ namespace detail { if (size() == 1) return; - - std::lock_guard guard(lock); + + std::lock_guard guard(lock); data[id].complete(seconds, bytes); } From 43b836adc13e7a5556fce396ee7e6a47f10413d3 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 28 May 2020 13:50:59 +0200 Subject: [PATCH 046/211] cosmetics --- src/Functions/FunctionStartsEndsWith.h | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index b148653e1b3..30d6a150620 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -41,11 +41,6 @@ public: return name; } - static String getImplementationTag() - { - return ToString(BuildArch); - } - size_t getNumberOfArguments() const override { return 2; From 4c16f7a70ffe244f1e82f3ff06482288f212290a Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 28 May 2020 15:01:00 +0200 Subject: [PATCH 047/211] cosmetics --- src/Functions/FunctionsRandom.cpp | 2 +- src/Functions/PerformanceAdaptors.h | 9 +-------- tests/performance/general_purpose_hashes.xml | 4 +++- 3 files changed, 5 insertions(+), 10 deletions(-) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 2c7b2e5f1f5..ced87d08cfa 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -1,5 +1,5 @@ -#include #include +#include #include #include #include diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index de321ee5605..bbe50d2e994 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -122,14 +122,7 @@ namespace detail /// And in that case prefer variant with less number of invocations. if (adjustedCount() < 2) - { - // TODO(dakovalkov): rewrite it. - int all_count = adjustedCount() + running_count; - if (all_count < 3) - return all_count - 2; - else - return adjustedCount() + running_count * 100; - } + return adjustedCount() - 1 + running_count; return std::normal_distribution<>(mean(), sigma())(stat_rng); } }; diff --git a/tests/performance/general_purpose_hashes.xml b/tests/performance/general_purpose_hashes.xml index 31a1bd65835..ada1df439fe 100644 --- a/tests/performance/general_purpose_hashes.xml +++ b/tests/performance/general_purpose_hashes.xml @@ -1,4 +1,6 @@ - + + gp_hash_func From 0f730b2ace141c5b17a1a76568bfd2aefd72e976 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 28 May 2020 17:35:05 +0200 Subject: [PATCH 048/211] multitarget great circle --- src/Functions/greatCircleDistance.cpp | 45 +++++++++++++++++++++++-- tests/performance/great_circle_dist.xml | 4 ++- 2 files changed, 45 insertions(+), 4 deletions(-) diff --git a/src/Functions/greatCircleDistance.cpp b/src/Functions/greatCircleDistance.cpp index 238499f8def..89337f83ddf 100644 --- a/src/Functions/greatCircleDistance.cpp +++ b/src/Functions/greatCircleDistance.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include #include @@ -153,6 +155,12 @@ enum class Method WGS84_METERS, }; +} + +DECLARE_MULTITARGET_CODE( + +namespace +{ template float distance(float lon1deg, float lat1deg, float lon2deg, float lat2deg) @@ -220,7 +228,6 @@ float distance(float lon1deg, float lat1deg, float lon2deg, float lat2deg) } - template class FunctionGeoDistance : public IFunction { @@ -230,8 +237,6 @@ public: : ((method == Method::SPHERE_METERS) ? "greatCircleDistance" : "geoDistance"); - static FunctionPtr create(const Context &) { return std::make_shared>(); } - private: String getName() const override { return name; } size_t getNumberOfArguments() const override { return 4; } @@ -272,6 +277,40 @@ private: } }; +) // DECLARE_MULTITARGET_CODE + +template +class FunctionGeoDistance : public TargetSpecific::Default::FunctionGeoDistance +{ +public: + explicit FunctionGeoDistance(const Context & context) : selector(context) + { + selector.registerImplementation>(); + + #if USE_MULTITARGET_CODE + selector.registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); + #endif + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + selector.selectAndExecute(block, arguments, result, input_rows_count); + } + + static FunctionPtr create(const Context & context) + { + return std::make_shared>(context); + } + +private: + ImplementationSelector selector; +}; void registerFunctionGeoDistance(FunctionFactory & factory) { diff --git a/tests/performance/great_circle_dist.xml b/tests/performance/great_circle_dist.xml index 13f9e6fde56..a57097bcbe7 100644 --- a/tests/performance/great_circle_dist.xml +++ b/tests/performance/great_circle_dist.xml @@ -1,4 +1,6 @@ - + + SELECT count() FROM numbers(1000000) WHERE NOT ignore(greatCircleDistance((rand(1) % 360) * 1. - 180, (number % 150) * 1.2 - 90, (number % 360) + toFloat64(rand(2)) / 4294967296 - 180, (rand(3) % 180) * 1. - 90)) From b0537bf31e1de4e5f52a399ff877a2313c11d31b Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 28 May 2020 17:38:07 +0200 Subject: [PATCH 049/211] Fix clang builds --- src/Functions/FunctionStartsEndsWith.h | 2 +- src/Functions/FunctionsHashing.h | 4 ++-- src/Functions/FunctionsRandom.cpp | 4 +++- src/Functions/FunctionsRandom.h | 2 +- src/Functions/RandXorshift.h | 2 +- src/Functions/generateUUIDv4.cpp | 2 +- 6 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 30d6a150620..69627eb2ead 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -141,7 +141,7 @@ template class FunctionStartsEndsWith : public TargetSpecific::Default::FunctionStartsEndsWith { public: - FunctionStartsEndsWith(const Context & context) : selector(context) + explicit FunctionStartsEndsWith(const Context & context) : selector(context) { selector.registerImplementation>(); diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 7edb8937275..b4c87dd761a 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -655,7 +655,7 @@ template class FunctionIntHash : public TargetSpecific::Default::FunctionIntHash { public: - FunctionIntHash(const Context & context) : selector(context) + explicit FunctionIntHash(const Context & context) : selector(context) { selector.registerImplementation>(); @@ -981,7 +981,7 @@ template class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash { public: - FunctionAnyHash(const Context & context) : selector(context) + explicit FunctionAnyHash(const Context & context) : selector(context) { selector.registerImplementation>(); diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index ced87d08cfa..fba44d458bb 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -4,7 +4,9 @@ #include #include #include -#include +#if USE_MULTITARGET_CODE +# include +#endif namespace DB { diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 346c94e1d9f..bc11f671c1b 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -91,7 +91,7 @@ template class FunctionRandom : public FunctionRandomImpl { public: - FunctionRandom(const Context & context) : selector(context) + explicit FunctionRandom(const Context & context) : selector(context) { selector.registerImplementation>(); diff --git a/src/Functions/RandXorshift.h b/src/Functions/RandXorshift.h index 8e068cf5dff..c005d7377dd 100644 --- a/src/Functions/RandXorshift.h +++ b/src/Functions/RandXorshift.h @@ -35,7 +35,7 @@ template class FunctionRandomXorshift : public FunctionRandomImpl { public: - FunctionRandomXorshift(const Context & context) : selector(context) + explicit FunctionRandomXorshift(const Context & context) : selector(context) { selector.registerImplementation>(); diff --git a/src/Functions/generateUUIDv4.cpp b/src/Functions/generateUUIDv4.cpp index a205f853d2a..53113a77273 100644 --- a/src/Functions/generateUUIDv4.cpp +++ b/src/Functions/generateUUIDv4.cpp @@ -61,7 +61,7 @@ public: class FunctionGenerateUUIDv4 : public TargetSpecific::Default::FunctionGenerateUUIDv4 { public: - FunctionGenerateUUIDv4(const Context & context) : selector(context) + explicit FunctionGenerateUUIDv4(const Context & context) : selector(context) { selector.registerImplementation(); From 278592106cd29732df3474a1da4a01a4d3d16dd0 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 28 May 2020 18:21:23 +0200 Subject: [PATCH 050/211] cosmetics --- src/Functions/FunctionBinaryArithmetic.h | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 292f7da0475..30b6da8b696 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -28,8 +28,6 @@ #include #include -#include - #if !defined(ARCADIA_BUILD) # include #endif @@ -54,11 +52,13 @@ namespace ErrorCodes extern const int CANNOT_ADD_DIFFERENT_AGGREGATE_STATES; } + /** Arithmetic operations: +, -, *, /, %, * intDiv (integer division) * Bitwise operations: |, &, ^, ~. * Etc. */ + template struct BinaryOperationImplBase { @@ -89,7 +89,6 @@ struct BinaryOperationImplBase } }; - template struct FixedStringOperationImpl { From 478ee2c185870bf9bd719abfb6951879b9a4e730 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Fri, 29 May 2020 06:54:18 +0200 Subject: [PATCH 051/211] delete SIMDxorshift --- .gitmodules | 3 - contrib/SIMDxorshift | 1 - contrib/SIMDxorshift-cmake/CMakeLists.txt | 12 -- src/Functions/RandXorshift.cpp | 167 ---------------------- src/Functions/RandXorshift.h | 67 --------- src/Functions/generateUUIDv4.cpp | 2 - 6 files changed, 252 deletions(-) delete mode 160000 contrib/SIMDxorshift delete mode 100644 contrib/SIMDxorshift-cmake/CMakeLists.txt delete mode 100644 src/Functions/RandXorshift.cpp delete mode 100644 src/Functions/RandXorshift.h diff --git a/.gitmodules b/.gitmodules index c14fef40457..7f5d1307a6e 100644 --- a/.gitmodules +++ b/.gitmodules @@ -160,6 +160,3 @@ [submodule "contrib/fmtlib"] path = contrib/fmtlib url = https://github.com/fmtlib/fmt.git -[submodule "contrib/SIMDxorshift"] - path = contrib/SIMDxorshift - url = https://github.com/lemire/SIMDxorshift diff --git a/contrib/SIMDxorshift b/contrib/SIMDxorshift deleted file mode 160000 index 270eb8936c9..00000000000 --- a/contrib/SIMDxorshift +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 270eb8936c9b4bd038c39f1783a8eba6b8f15b09 diff --git a/contrib/SIMDxorshift-cmake/CMakeLists.txt b/contrib/SIMDxorshift-cmake/CMakeLists.txt deleted file mode 100644 index 573173ff1b4..00000000000 --- a/contrib/SIMDxorshift-cmake/CMakeLists.txt +++ /dev/null @@ -1,12 +0,0 @@ -set(SIMDXORSHIFT_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/SIMDxorshift/include") -set(SIMDXORSHIFT_SRC_DIR "${SIMDXORSHIFT_INCLUDE_DIR}/../src") -set(SIMDXORSHIFT_SRC - ${SIMDXORSHIFT_SRC_DIR}/xorshift128plus.c - ${SIMDXORSHIFT_SRC_DIR}/simdxorshift128plus.c -) - -set(SIMDXORSHIFT_LIBRARY "simdxorshift") - -add_library(${SIMDXORSHIFT_LIBRARY} ${SIMDXORSHIFT_SRC}) -target_include_directories(${SIMDXORSHIFT_LIBRARY} PUBLIC "${SIMDXORSHIFT_INCLUDE_DIR}") -target_compile_options(${SIMDXORSHIFT_LIBRARY} PRIVATE -mavx2) diff --git a/src/Functions/RandXorshift.cpp b/src/Functions/RandXorshift.cpp deleted file mode 100644 index 4fa2280861f..00000000000 --- a/src/Functions/RandXorshift.cpp +++ /dev/null @@ -1,167 +0,0 @@ -/// Disable xorshift -#if 0 -#include -#include -#include -#include -#include - -#include - -extern "C" -{ -#include -#include -} - -namespace DB -{ - -DECLARE_DEFAULT_CODE( - -void RandXorshiftImpl::execute(char * output, size_t size) -{ - if (size == 0) - return; - - char * end = output + size; - - xorshift128plus_key_s mykey; - - xorshift128plus_init(0xe9ef384566799595ULL ^ reinterpret_cast(output), - 0xa321e1523f4f88c7ULL ^ reinterpret_cast(output), - &mykey); - - constexpr int bytes_per_write = 8; - constexpr intptr_t mask = bytes_per_write - 1; - - // Process head to make output aligned. - unalignedStore(output, xorshift128plus(&mykey)); - output = reinterpret_cast((reinterpret_cast(output) | mask) + 1); - - while (end - output > 0) - { - *reinterpret_cast(output) = xorshift128plus(&mykey); - output += bytes_per_write; - } -} - -) // DECLARE_DEFAULT_CODE - -DECLARE_AVX2_SPECIFIC_CODE( - -void RandXorshiftImpl::execute(char * output, size_t size) -{ - if (size == 0) - return; - - char * end = output + size; - - avx_xorshift128plus_key_t mykey; - avx_xorshift128plus_init(0xe9ef384566799595ULL ^ reinterpret_cast(output), - 0xa321e1523f4f88c7ULL ^ reinterpret_cast(output), - &mykey); - - constexpr int safe_overwrite = 15; /// How many bytes we can write behind the end. - constexpr int bytes_per_write = 32; - constexpr intptr_t mask = bytes_per_write - 1; - - if (size + safe_overwrite < bytes_per_write) - { - /// size <= 16. - _mm_storeu_si128(reinterpret_cast<__m128i*>(output), - _mm256_extracti128_si256(avx_xorshift128plus(&mykey), 0)); - return; - } - - /// Process head to make output aligned. - _mm256_storeu_si256(reinterpret_cast<__m256i*>(output), avx_xorshift128plus(&mykey)); - output = reinterpret_cast((reinterpret_cast(output) | mask) + 1); - - while ((end - output) + safe_overwrite >= bytes_per_write) - { - _mm256_store_si256(reinterpret_cast<__m256i*>(output), avx_xorshift128plus(&mykey)); - output += bytes_per_write; - } - - /// Process tail. (end - output) <= 16. - if ((end - output) > 0) - { - _mm_store_si128(reinterpret_cast<__m128i*>(output), - _mm256_extracti128_si256(avx_xorshift128plus(&mykey), 0)); - } -} - -) // DECLARE_AVX2_SPECIFIC_CODE - -DECLARE_AVX2_SPECIFIC_CODE( - -void RandXorshiftImpl2::execute(char * output, size_t size) -{ - if (size == 0) - return; - - char * end = output + size; - - avx_xorshift128plus_key_t mykey; - avx_xorshift128plus_init(0xe9ef384566799595ULL ^ reinterpret_cast(output), - 0xa321e1523f4f88c7ULL ^ reinterpret_cast(output), - &mykey); - - avx_xorshift128plus_key_t mykey2; - avx_xorshift128plus_init(0xdfe532a6b5a5eb2cULL ^ reinterpret_cast(output), - 0x21cdf6cd1e22bf9cULL ^ reinterpret_cast(output), - &mykey2); - - constexpr int safe_overwrite = 15; /// How many bytes we can write behind the end. - constexpr int bytes_per_write = 32; - constexpr intptr_t mask = bytes_per_write - 1; - - if (size + safe_overwrite < bytes_per_write) - { - /// size <= 16. - _mm_storeu_si128(reinterpret_cast<__m128i*>(output), - _mm256_extracti128_si256(avx_xorshift128plus(&mykey), 0)); - return; - } - - /// Process head to make output aligned. - _mm256_storeu_si256(reinterpret_cast<__m256i*>(output), avx_xorshift128plus(&mykey)); - output = reinterpret_cast((reinterpret_cast(output) | mask) + 1); - - while ((end - output) + safe_overwrite >= bytes_per_write * 2) - { - _mm256_store_si256(reinterpret_cast<__m256i*>(output), avx_xorshift128plus(&mykey)); - _mm256_store_si256(reinterpret_cast<__m256i*>(output + bytes_per_write), avx_xorshift128plus(&mykey2)); - output += bytes_per_write * 2; - } - - if ((end - output) + safe_overwrite >= bytes_per_write) - { - _mm256_store_si256(reinterpret_cast<__m256i*>(output), avx_xorshift128plus(&mykey)); - output += bytes_per_write; - } - - /// Process tail. (end - output) <= 16. - if ((end - output) > 0) - { - _mm_store_si128(reinterpret_cast<__m128i*>(output), - _mm256_extracti128_si256(avx_xorshift128plus(&mykey), 0)); - } -} - -) // DECLARE_AVX2_SPECIFIC_CODE - -struct NameRandXorshift { static constexpr auto name = "randxorshift"; }; -using FunctionRandXorshift = FunctionRandomXorshift; -struct NameRandXorshift64 { static constexpr auto name = "randxorshift64"; }; -using FunctionRandXorshift64 = FunctionRandomXorshift; - -void registerFunctionRandXorshift(FunctionFactory & factory) -{ - factory.registerFunction(); - factory.registerFunction(); -} - -} -#endif diff --git a/src/Functions/RandXorshift.h b/src/Functions/RandXorshift.h deleted file mode 100644 index c005d7377dd..00000000000 --- a/src/Functions/RandXorshift.h +++ /dev/null @@ -1,67 +0,0 @@ -#pragma once - -/// disable xorshift -#if 0 - -#include -#include -#include -#include - -#include -#include -#include - -namespace DB -{ - -DECLARE_MULTITARGET_CODE( - -struct RandXorshiftImpl -{ - static void execute(char * output, size_t size); - static String getImplementationTag() { return ToString(BuildArch); } -}; - -struct RandXorshiftImpl2 -{ - static void execute(char * output, size_t size); - static String getImplementationTag() { return "v2"; } -}; - -) // DECLARE_MULTITARGET_CODE - -template -class FunctionRandomXorshift : public FunctionRandomImpl -{ -public: - explicit FunctionRandomXorshift(const Context & context) : selector(context) - { - selector.registerImplementation>(); - - #if USE_MULTITARGET_CODE - selector.registerImplementation>(); - selector.registerImplementation>(); - #endif - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - selector.selectAndExecute(block, arguments, result, input_rows_count); - } - - static FunctionPtr create(const Context & context) - { - return std::make_shared>(context); - } - -private: - ImplementationSelector selector; -}; - -} - -#endif diff --git a/src/Functions/generateUUIDv4.cpp b/src/Functions/generateUUIDv4.cpp index 53113a77273..7dbb73c0cf3 100644 --- a/src/Functions/generateUUIDv4.cpp +++ b/src/Functions/generateUUIDv4.cpp @@ -21,8 +21,6 @@ public: return name; } - static String getImplementationTag() { return ToString(BuildArch); } - size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const DataTypes &) const override From 1bf8940c18a19ff3fbc0b95cc56669bc9f4300da Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Fri, 29 May 2020 11:12:08 +0200 Subject: [PATCH 052/211] better randomFixedString and randomString --- src/Functions/randomFixedString.cpp | 45 +++++++++++++++++++-------- src/Functions/randomString.cpp | 47 +++++++++++++++++++++-------- 2 files changed, 67 insertions(+), 25 deletions(-) diff --git a/src/Functions/randomFixedString.cpp b/src/Functions/randomFixedString.cpp index 9fb7550346b..16b6726b5d1 100644 --- a/src/Functions/randomFixedString.cpp +++ b/src/Functions/randomFixedString.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -21,13 +23,12 @@ namespace ErrorCodes /* Generate random fixed string with fully random bytes (including zero). */ -class FunctionRandomFixedString : public IFunction +template +class FunctionRandomFixedStringImpl : public IFunction { public: static constexpr auto name = "randomFixedString"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - String getName() const override { return name; } bool isVariadic() const override { return false; } @@ -68,20 +69,40 @@ public: /// Fill random bytes. data_to.resize(total_size); - pcg64_fast rng(randomSeed()); /// TODO It is inefficient. We should use SIMD PRNG instead. - - auto * pos = data_to.data(); - auto * end = pos + data_to.size(); - while (pos < end) - { - unalignedStore(pos, rng()); - pos += sizeof(UInt64); // We have padding in column buffers that we can overwrite. - } + RandImpl::execute(reinterpret_cast(data_to.data()), total_size); block.getByPosition(result).column = std::move(col_to); } }; +class FunctionRandomFixedString : public FunctionRandomFixedStringImpl +{ +public: + explicit FunctionRandomFixedString(const Context & context) : selector(context) + { + selector.registerImplementation>(); + + #if USE_MULTITARGET_CODE + selector.registerImplementation>(); + #endif + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + selector.selectAndExecute(block, arguments, result, input_rows_count); + } + + static FunctionPtr create(const Context & context) + { + return std::make_shared(context); + } + +private: + ImplementationSelector selector; +}; + void registerFunctionRandomFixedString(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/randomString.cpp b/src/Functions/randomString.cpp index 4ea470f0a65..5ed8e459549 100644 --- a/src/Functions/randomString.cpp +++ b/src/Functions/randomString.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -19,13 +21,12 @@ namespace ErrorCodes /* Generate random string of specified length with fully random bytes (including zero). */ -class FunctionRandomString : public IFunction +template +class FunctionRandomStringImpl : public IFunction { public: static constexpr auto name = "randomString"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - String getName() const override { return name; } bool isVariadic() const override { return true; } @@ -83,18 +84,10 @@ public: /// Fill random bytes. data_to.resize(offsets_to.back()); - pcg64_fast rng(randomSeed()); /// TODO It is inefficient. We should use SIMD PRNG instead. - - auto * pos = data_to.data(); - auto * end = pos + data_to.size(); - while (pos < end) - { - unalignedStore(pos, rng()); - pos += sizeof(UInt64); // We have padding in column buffers that we can overwrite. - } + RandImpl::execute(reinterpret_cast(data_to.data()), data_to.size()); /// Put zero bytes in between. - pos = data_to.data(); + auto * pos = data_to.data(); for (size_t row_num = 0; row_num < input_rows_count; ++row_num) pos[offsets_to[row_num] - 1] = 0; @@ -102,6 +95,34 @@ public: } }; +class FunctionRandomString : public FunctionRandomStringImpl +{ +public: + explicit FunctionRandomString(const Context & context) : selector(context) + { + selector.registerImplementation>(); + + #if USE_MULTITARGET_CODE + selector.registerImplementation>(); + #endif + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + selector.selectAndExecute(block, arguments, result, input_rows_count); + } + + static FunctionPtr create(const Context & context) + { + return std::make_shared(context); + } + +private: + ImplementationSelector selector; +}; + void registerFunctionRandomString(FunctionFactory & factory) { factory.registerFunction(); From 8390fcaa23ab7d78922c6b9817387fd0eea28178 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Fri, 29 May 2020 11:18:39 +0200 Subject: [PATCH 053/211] Cosmetics --- src/Functions/FunctionsRandom.cpp | 1 + src/Functions/FunctionsRandom.h | 4 ++-- src/Functions/randomFixedString.cpp | 2 +- src/Functions/randomString.cpp | 2 +- 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index fba44d458bb..e77bab9c0a5 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -8,6 +8,7 @@ # include #endif + namespace DB { diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index bc11f671c1b..b80ddb6f59e 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -3,10 +3,10 @@ #include #include #include -#include - #include #include +#include + namespace DB { diff --git a/src/Functions/randomFixedString.cpp b/src/Functions/randomFixedString.cpp index 16b6726b5d1..669dc084999 100644 --- a/src/Functions/randomFixedString.cpp +++ b/src/Functions/randomFixedString.cpp @@ -82,7 +82,7 @@ public: { selector.registerImplementation>(); - + #if USE_MULTITARGET_CODE selector.registerImplementation>(); diff --git a/src/Functions/randomString.cpp b/src/Functions/randomString.cpp index 5ed8e459549..df3278c3800 100644 --- a/src/Functions/randomString.cpp +++ b/src/Functions/randomString.cpp @@ -102,7 +102,7 @@ public: { selector.registerImplementation>(); - + #if USE_MULTITARGET_CODE selector.registerImplementation>(); From e185f3d157295d33698d58975061eec382df47bf Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Fri, 29 May 2020 11:23:22 +0200 Subject: [PATCH 054/211] Fix style issue --- src/Functions/greatCircleDistance.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/greatCircleDistance.cpp b/src/Functions/greatCircleDistance.cpp index 89337f83ddf..bff92d7738d 100644 --- a/src/Functions/greatCircleDistance.cpp +++ b/src/Functions/greatCircleDistance.cpp @@ -287,7 +287,7 @@ public: { selector.registerImplementation>(); - + #if USE_MULTITARGET_CODE selector.registerImplementation>(); From 47a902a6ce593d9ee55a29fdb0b35bc6f44152a7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 11 Jun 2020 18:55:44 +0300 Subject: [PATCH 055/211] Simple github hook --- utils/github-hook/hook.py | 195 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 195 insertions(+) create mode 100644 utils/github-hook/hook.py diff --git a/utils/github-hook/hook.py b/utils/github-hook/hook.py new file mode 100644 index 00000000000..13d62b311f7 --- /dev/null +++ b/utils/github-hook/hook.py @@ -0,0 +1,195 @@ +# -*- coding: utf-8 -*- +import json +import requests +import time +import os + +DB = 'gh-data' +RETRIES = 5 + + +def process_issue_event(response): + issue = response['issue'] + return dict( + action=response['action'], + sender=response['sender']['login'], + updated_at=issue['updated_at'], + url=issue['url'], + number=issue['number'], + author=issue['user']['login'], + labels=[label['name'] for label in issue['labels']], + state=issue['state'], + assignees=[assignee['login'] for assignee in issue['assignees']], + created_at=issue['created_at'], + body=issue['body'], + title=issue['title'], + comments=issue['comments'], + raw_json=json.dumps(response),) + + +def process_issue_comment_event(response): + issue = response['issue'] + comment = response['comment'] + + return dict( + action='comment_' + response['action'], + sender=response['sender']['login'], + updated_at=issue['updated_at'], + url=issue['url'], + number=issue['number'], + author=issue['user']['login'], + labels=[label['name'] for label in issue['labels']], + state=issue['state'], + assignees=[assignee['login'] for assignee in issue['assignees']], + created_at=issue['created_at'], + body=issue['body'], + title=issue['title'], + comments=issue['comments'], + comment_body=comment['body'], + comment_author=comment['user']['login'], + comment_url=comment['url'], + comment_created_at=comment['created_at'], + comment_updated_at=comment['updated_at'], + raw_json=json.dumps(response),) + + +def process_pull_request_event(response): + pull_request = response['pull_request'] + result = dict( + updated_at=pull_request['updated_at'], + number=pull_request['number'], + action=response['action'], + sender=response['sender']['login'], + url=pull_request['url'], + author=pull_request['user']['login'], + labels=[label['name'] for label in pull_request['labels']], + state=pull_request['state'], + body=pull_request['body'], + title=pull_request['title'], + created_at=pull_request['created_at'], + assignees=[assignee['login'] for assignee in pull_request['assignees']], + requested_reviewers=[reviewer['login'] for reviewer in pull_request['requested_reviewers']], + head_repo=pull_request['head']['repo']['full_name'], + head_ref=pull_request['head']['ref'], + head_clone_url=pull_request['head']['repo']['clone_url'], + head_ssh_url=pull_request['head']['repo']['ssh_url'], + base_repo=pull_request['base']['repo']['full_name'], + base_ref=pull_request['base']['ref'], + base_clone_url=pull_request['base']['repo']['clone_url'], + base_ssh_url=pull_request['base']['repo']['ssh_url'], + raw_json=json.dumps(response), + ) + + if 'mergeable' in pull_request and pull_request['mergeable'] is not None: + result['mergeable'] = 1 if pull_request['mergeable'] else 0 + + if 'merged_by' in pull_request and pull_request['merged_by'] is not None: + result['merged_by'] = pull_request['merged_by']['login'] + + if 'merged_at' in pull_request and pull_request['merged_at'] is not None: + result['merged_at'] = pull_request['merged_at'] + + if 'closed_at' in pull_request and pull_request['closed_at'] is not None: + result['closed_at'] = pull_request['closed_at'] + + if 'merge_commit_sha' in pull_request and pull_request['merge_commit_sha'] is not None: + result['merge_commit_sha'] = pull_request['merge_commit_sha'] + + if 'draft' in pull_request: + result['draft'] = 1 if pull_request['draft'] else 0 + + for field in ['comments', 'review_comments', 'commits', 'additions', 'deletions', 'changed_files']: + if field in pull_request: + result[field] = pull_request[field] + + return result + + +def process_pull_request_review(response): + result = process_pull_request_event(response) + review = response['review'] + result['action'] = 'review_' + result['action'] + result['review_body'] = review['body'] if review['body'] is not None else '' + result['review_id'] = review['id'] + result['review_author'] = review['user']['login'] + result['review_commit_sha'] = review['commit_id'] + result['review_submitted_at'] = review['submitted_at'] + result['review_state'] = review['state'] + return result + + +def process_pull_request_review_comment(response): + result = process_pull_request_event(response) + comment = response['comment'] + result['action'] = 'review_comment_' + result['action'] + result['review_id'] = comment['pull_request_review_id'] + result['review_comment_path'] = comment['path'] + result['review_commit_sha'] = comment['commit_id'] + result['review_comment_body'] = comment['body'] + result['review_comment_author'] = comment['user']['login'] + result['review_comment_created_at'] = comment['created_at'] + result['review_comment_updated_at'] = comment['updated_at'] + return result + + +def event_processor_dispatcher(headers, body, inserter): + if 'X-Github-Event' in headers: + if headers['X-Github-Event'] == 'issues': + result = process_issue_event(body) + inserter.insert_event_into(DB, 'issues', result) + elif headers['X-Github-Event'] == 'issue_comment': + result = process_issue_comment_event(body) + inserter.insert_event_into(DB, 'issues', result) + elif headers['X-Github-Event'] == 'pull_request': + result = process_pull_request_event(body) + inserter.insert_event_into(DB, 'pull_requests', result) + elif headers['X-Github-Event'] == 'pull_request_review': + result = process_pull_request_review(body) + inserter.insert_event_into(DB, 'pull_requests', result) + elif headers['X-Github-Event'] == 'pull_request_review_comment': + result = process_pull_request_review_comment(body) + inserter.insert_event_into(DB, 'pull_requests', result) + + +class ClickHouseInserter(object): + def __init__(self, url, user, password): + self.url = url + self.auth = { + 'X-ClickHouse-User': user, + 'X-ClickHouse-Key': password + } + + def insert_event_into(self, db, table, event): + params = { + 'database': db, + 'query': 'INSERT INTO {table} FORMAT JSONEachRow'.format(table=table), + 'date_time_input_format': 'best_effort' + } + event_str = json.dumps(event) + for i in range(RETRIES): + try: + response = requests.post(self.url, params=params, data=event_str, headers=self.auth, verify=False) + response.raise_for_status() + break + except Exception as ex: + print("Exception inseting into ClickHouse:", ex) + time.sleep(0.1) + + +def test(event, context): + inserter = ClickHouseInserter( + os.getenv('CLICKHOUSE_URL'), + os.getenv('CLICKHOUSE_USER'), + os.getenv('CLICKHOUSE_PASSWORD')) + + body = json.loads(event['body'], strict=False) + headers = event['headers'] + event_processor_dispatcher(headers, body, inserter) + + return { + 'statusCode': 200, + 'headers': { + 'Content-Type': 'text/plain' + }, + 'isBase64Encoded': False, + } From af2fe2ba553e7112ef474d73473fbff047c5ae60 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 15 Jun 2020 19:55:33 +0300 Subject: [PATCH 056/211] Compilable setColumns, setConstraints, setIndices --- src/Storages/IStorage.cpp | 105 ++++++++---------- src/Storages/IStorage.h | 21 ++-- src/Storages/Kafka/StorageKafka.cpp | 4 +- src/Storages/LiveView/StorageBlocks.h | 4 +- src/Storages/LiveView/StorageLiveView.cpp | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 6 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 7 +- src/Storages/StorageBuffer.cpp | 8 +- src/Storages/StorageDictionary.cpp | 4 +- src/Storages/StorageDistributed.cpp | 8 +- src/Storages/StorageFile.cpp | 11 +- src/Storages/StorageGenerateRandom.cpp | 4 +- src/Storages/StorageHDFS.cpp | 7 +- src/Storages/StorageInMemoryMetadata.cpp | 15 +++ src/Storages/StorageInMemoryMetadata.h | 13 +++ src/Storages/StorageInput.cpp | 4 +- src/Storages/StorageLog.cpp | 6 +- src/Storages/StorageMaterializedView.cpp | 6 +- src/Storages/StorageMemory.cpp | 6 +- src/Storages/StorageMerge.cpp | 6 +- src/Storages/StorageMySQL.cpp | 6 +- src/Storages/StorageNull.cpp | 2 +- src/Storages/StorageNull.h | 6 +- src/Storages/StorageS3.cpp | 6 +- src/Storages/StorageSet.cpp | 7 +- src/Storages/StorageStripeLog.cpp | 6 +- src/Storages/StorageTinyLog.cpp | 6 +- src/Storages/StorageURL.cpp | 7 +- src/Storages/StorageValues.cpp | 4 +- src/Storages/StorageView.cpp | 4 +- src/Storages/System/IStorageSystemOneBlock.h | 4 +- src/Storages/System/StorageSystemColumns.cpp | 4 +- .../System/StorageSystemDetachedParts.cpp | 4 +- src/Storages/System/StorageSystemDisks.cpp | 4 +- src/Storages/System/StorageSystemNumbers.cpp | 4 +- src/Storages/System/StorageSystemOne.cpp | 4 +- .../System/StorageSystemPartsBase.cpp | 4 +- src/Storages/System/StorageSystemReplicas.cpp | 4 +- .../System/StorageSystemStoragePolicies.cpp | 4 +- src/Storages/System/StorageSystemTables.cpp | 4 +- src/Storages/System/StorageSystemZeros.cpp | 5 +- 41 files changed, 211 insertions(+), 137 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index a244f836f5c..6c045a6f365 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -34,22 +34,22 @@ namespace ErrorCodes const ColumnsDescription & IStorage::getColumns() const { - return metadata.columns; + return metadata->columns; } const IndicesDescription & IStorage::getSecondaryIndices() const { - return metadata.secondary_indices; + return metadata->secondary_indices; } bool IStorage::hasSecondaryIndices() const { - return !metadata.secondary_indices.empty(); + return !metadata->secondary_indices.empty(); } const ConstraintsDescription & IStorage::getConstraints() const { - return metadata.constraints; + return metadata->constraints; } Block IStorage::getSampleBlock() const @@ -287,23 +287,6 @@ void IStorage::check(const Block & block, bool need_all) const } } -void IStorage::setColumns(ColumnsDescription columns_) -{ - if (columns_.getOrdinary().empty()) - throw Exception("Empty list of columns passed", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); - metadata.columns = std::move(columns_); -} - -void IStorage::setSecondaryIndices(IndicesDescription secondary_indices_) -{ - metadata.secondary_indices = std::move(secondary_indices_); -} - -void IStorage::setConstraints(ConstraintsDescription constraints_) -{ - metadata.constraints = std::move(constraints_); -} - bool IStorage::isVirtualColumn(const String & column_name) const { /// Virtual column maybe overriden by real column @@ -382,7 +365,7 @@ void IStorage::alter( StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); params.apply(new_metadata, context); DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata); - setColumns(std::move(new_metadata.columns)); + setInMemoryMetadata(new_metadata); } @@ -417,137 +400,137 @@ NamesAndTypesList IStorage::getVirtuals() const const KeyDescription & IStorage::getPartitionKey() const { - return metadata.partition_key; + return metadata->partition_key; } void IStorage::setPartitionKey(const KeyDescription & partition_key_) { - metadata.partition_key = partition_key_; + metadata->partition_key = partition_key_; } bool IStorage::isPartitionKeyDefined() const { - return metadata.partition_key.definition_ast != nullptr; + return metadata->partition_key.definition_ast != nullptr; } bool IStorage::hasPartitionKey() const { - return !metadata.partition_key.column_names.empty(); + return !metadata->partition_key.column_names.empty(); } Names IStorage::getColumnsRequiredForPartitionKey() const { if (hasPartitionKey()) - return metadata.partition_key.expression->getRequiredColumns(); + return metadata->partition_key.expression->getRequiredColumns(); return {}; } const KeyDescription & IStorage::getSortingKey() const { - return metadata.sorting_key; + return metadata->sorting_key; } void IStorage::setSortingKey(const KeyDescription & sorting_key_) { - metadata.sorting_key = sorting_key_; + metadata->sorting_key = sorting_key_; } bool IStorage::isSortingKeyDefined() const { - return metadata.sorting_key.definition_ast != nullptr; + return metadata->sorting_key.definition_ast != nullptr; } bool IStorage::hasSortingKey() const { - return !metadata.sorting_key.column_names.empty(); + return !metadata->sorting_key.column_names.empty(); } Names IStorage::getColumnsRequiredForSortingKey() const { if (hasSortingKey()) - return metadata.sorting_key.expression->getRequiredColumns(); + return metadata->sorting_key.expression->getRequiredColumns(); return {}; } Names IStorage::getSortingKeyColumns() const { if (hasSortingKey()) - return metadata.sorting_key.column_names; + return metadata->sorting_key.column_names; return {}; } const KeyDescription & IStorage::getPrimaryKey() const { - return metadata.primary_key; + return metadata->primary_key; } void IStorage::setPrimaryKey(const KeyDescription & primary_key_) { - metadata.primary_key = primary_key_; + metadata->primary_key = primary_key_; } bool IStorage::isPrimaryKeyDefined() const { - return metadata.primary_key.definition_ast != nullptr; + return metadata->primary_key.definition_ast != nullptr; } bool IStorage::hasPrimaryKey() const { - return !metadata.primary_key.column_names.empty(); + return !metadata->primary_key.column_names.empty(); } Names IStorage::getColumnsRequiredForPrimaryKey() const { if (hasPrimaryKey()) - return metadata.primary_key.expression->getRequiredColumns(); + return metadata->primary_key.expression->getRequiredColumns(); return {}; } Names IStorage::getPrimaryKeyColumns() const { - if (!metadata.primary_key.column_names.empty()) - return metadata.primary_key.column_names; + if (!metadata->primary_key.column_names.empty()) + return metadata->primary_key.column_names; return {}; } const KeyDescription & IStorage::getSamplingKey() const { - return metadata.sampling_key; + return metadata->sampling_key; } void IStorage::setSamplingKey(const KeyDescription & sampling_key_) { - metadata.sampling_key = sampling_key_; + metadata->sampling_key = sampling_key_; } bool IStorage::isSamplingKeyDefined() const { - return metadata.sampling_key.definition_ast != nullptr; + return metadata->sampling_key.definition_ast != nullptr; } bool IStorage::hasSamplingKey() const { - return !metadata.sampling_key.column_names.empty(); + return !metadata->sampling_key.column_names.empty(); } Names IStorage::getColumnsRequiredForSampling() const { if (hasSamplingKey()) - return metadata.sampling_key.expression->getRequiredColumns(); + return metadata->sampling_key.expression->getRequiredColumns(); return {}; } TTLTableDescription IStorage::getTableTTLs() const { std::lock_guard lock(ttl_mutex); - return metadata.table_ttl; + return metadata->table_ttl; } void IStorage::setTableTTLs(const TTLTableDescription & table_ttl_) { std::lock_guard lock(ttl_mutex); - metadata.table_ttl = table_ttl_; + metadata->table_ttl = table_ttl_; } bool IStorage::hasAnyTableTTL() const @@ -558,43 +541,43 @@ bool IStorage::hasAnyTableTTL() const TTLColumnsDescription IStorage::getColumnTTLs() const { std::lock_guard lock(ttl_mutex); - return metadata.column_ttls_by_name; + return metadata->column_ttls_by_name; } void IStorage::setColumnTTLs(const TTLColumnsDescription & column_ttls_by_name_) { std::lock_guard lock(ttl_mutex); - metadata.column_ttls_by_name = column_ttls_by_name_; + metadata->column_ttls_by_name = column_ttls_by_name_; } bool IStorage::hasAnyColumnTTL() const { std::lock_guard lock(ttl_mutex); - return !metadata.column_ttls_by_name.empty(); + return !metadata->column_ttls_by_name.empty(); } TTLDescription IStorage::getRowsTTL() const { std::lock_guard lock(ttl_mutex); - return metadata.table_ttl.rows_ttl; + return metadata->table_ttl.rows_ttl; } bool IStorage::hasRowsTTL() const { std::lock_guard lock(ttl_mutex); - return metadata.table_ttl.rows_ttl.expression != nullptr; + return metadata->table_ttl.rows_ttl.expression != nullptr; } TTLDescriptions IStorage::getMoveTTLs() const { std::lock_guard lock(ttl_mutex); - return metadata.table_ttl.move_ttl; + return metadata->table_ttl.move_ttl; } bool IStorage::hasAnyMoveTTL() const { std::lock_guard lock(ttl_mutex); - return !metadata.table_ttl.move_ttl.empty(); + return !metadata->table_ttl.move_ttl.empty(); } @@ -660,32 +643,32 @@ ColumnDependencies IStorage::getColumnDependencies(const NameSet & updated_colum ASTPtr IStorage::getSettingsChanges() const { - if (metadata.settings_changes) - return metadata.settings_changes->clone(); + if (metadata->settings_changes) + return metadata->settings_changes->clone(); return nullptr; } void IStorage::setSettingsChanges(const ASTPtr & settings_changes_) { if (settings_changes_) - metadata.settings_changes = settings_changes_->clone(); + metadata->settings_changes = settings_changes_->clone(); else - metadata.settings_changes = nullptr; + metadata->settings_changes = nullptr; } const SelectQueryDescription & IStorage::getSelectQuery() const { - return metadata.select; + return metadata->select; } void IStorage::setSelectQuery(const SelectQueryDescription & select_) { - metadata.select = select_; + metadata->select = select_; } bool IStorage::hasSelectQuery() const { - return metadata.select.select_query != nullptr; + return metadata->select.select_query != nullptr; } } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index c7c8e382a87..4d01bb5370d 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -140,27 +140,24 @@ public: public: /// thread-unsafe part. lockStructure must be acquired const ColumnsDescription & getColumns() const; /// returns combined set of columns - void setColumns(ColumnsDescription columns_); /// sets only real columns, possibly overwrites virtual ones. - - void setSecondaryIndices(IndicesDescription secondary_indices_); const IndicesDescription & getSecondaryIndices() const; /// Has at least one non primary index bool hasSecondaryIndices() const; const ConstraintsDescription & getConstraints() const; - void setConstraints(ConstraintsDescription constraints_); /// Storage settings ASTPtr getSettingsChanges() const; void setSettingsChanges(const ASTPtr & settings_changes_); - bool hasSettingsChanges() const { return metadata.settings_changes != nullptr; } + bool hasSettingsChanges() const { return metadata->settings_changes != nullptr; } /// Select query for *View storages. const SelectQueryDescription & getSelectQuery() const; void setSelectQuery(const SelectQueryDescription & select_); bool hasSelectQuery() const; - StorageInMemoryMetadata getInMemoryMetadata() const { return metadata; } + StorageInMemoryMetadata getInMemoryMetadata() const { return *metadata; } + void setInMemoryMetadata(const StorageInMemoryMetadata & metadata_) { metadata = std::make_shared(metadata_); } Block getSampleBlock() const; /// ordinary + materialized. Block getSampleBlockWithVirtuals() const; /// ordinary + materialized + virtuals. @@ -207,7 +204,7 @@ private: /// TODO (alesap) just use multiversion for atomic metadata mutable std::mutex ttl_mutex; - StorageInMemoryMetadata metadata; + StorageMetadataPtr metadata; private: RWLockImpl::LockHolder tryLockTimed( const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const SettingSeconds & acquire_timeout) const; @@ -354,7 +351,7 @@ public: /** ALTER tables in the form of column changes that do not affect the change to Storage or its parameters. * This method must fully execute the ALTER query, taking care of the locks itself. - * To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata. + * To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata-> */ virtual void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder); @@ -445,7 +442,7 @@ public: /// struct). void setPartitionKey(const KeyDescription & partition_key_); /// Returns ASTExpressionList of partition key expression for storage or nullptr if there is none. - ASTPtr getPartitionKeyAST() const { return metadata.partition_key.definition_ast; } + ASTPtr getPartitionKeyAST() const { return metadata->partition_key.definition_ast; } /// Storage has user-defined (in CREATE query) partition key. bool isPartitionKeyDefined() const; /// Storage has partition key. @@ -460,7 +457,7 @@ public: /// struct). void setSortingKey(const KeyDescription & sorting_key_); /// Returns ASTExpressionList of sorting key expression for storage or nullptr if there is none. - ASTPtr getSortingKeyAST() const { return metadata.sorting_key.definition_ast; } + ASTPtr getSortingKeyAST() const { return metadata->sorting_key.definition_ast; } /// Storage has user-defined (in CREATE query) sorting key. bool isSortingKeyDefined() const; /// Storage has sorting key. It means, that it contains at least one column. @@ -477,7 +474,7 @@ public: /// struct). void setPrimaryKey(const KeyDescription & primary_key_); /// Returns ASTExpressionList of primary key expression for storage or nullptr if there is none. - ASTPtr getPrimaryKeyAST() const { return metadata.primary_key.definition_ast; } + ASTPtr getPrimaryKeyAST() const { return metadata->primary_key.definition_ast; } /// Storage has user-defined (in CREATE query) sorting key. bool isPrimaryKeyDefined() const; /// Storage has primary key (maybe part of some other key). It means, that @@ -495,7 +492,7 @@ public: /// struct). void setSamplingKey(const KeyDescription & sampling_key_); /// Returns sampling expression AST for storage or nullptr if there is none. - ASTPtr getSamplingKeyAST() const { return metadata.sampling_key.definition_ast; } + ASTPtr getSamplingKeyAST() const { return metadata->sampling_key.definition_ast; } /// Storage has user-defined (in CREATE query) sampling key. bool isSamplingKeyDefined() const; /// Storage has sampling key. diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index bb721417c5b..2109afed932 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -137,7 +137,9 @@ StorageKafka::StorageKafka( , intermediate_commit(kafka_settings->kafka_commit_every_batch.value) , settings_adjustments(createSettingsAdjustments()) { - setColumns(columns_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + setInMemoryMetadata(metadata_); task = global_context.getSchedulePool().createTask(log->name(), [this]{ threadFunc(); }); task->deactivate(); diff --git a/src/Storages/LiveView/StorageBlocks.h b/src/Storages/LiveView/StorageBlocks.h index 2a9d7766fd7..78d60163d5e 100644 --- a/src/Storages/LiveView/StorageBlocks.h +++ b/src/Storages/LiveView/StorageBlocks.h @@ -18,7 +18,9 @@ public: QueryProcessingStage::Enum to_stage_) : IStorage(table_id_), pipes(std::move(pipes_)), to_stage(to_stage_) { - setColumns(columns_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + setInMemoryMetadata(metadata_); } static StoragePtr createStorage(const StorageID & table_id, const ColumnsDescription & columns, Pipes pipes, QueryProcessingStage::Enum to_stage) diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 8a04a9e49e4..ade2d1c967d 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -251,7 +251,9 @@ StorageLiveView::StorageLiveView( live_view_context = std::make_unique(global_context); live_view_context->makeQueryContext(); - setColumns(columns_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + setInMemoryMetadata(metadata_); if (!query.select) throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 84470088ebe..14be526d7f6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -387,11 +387,7 @@ void MergeTreeData::setProperties(const StorageInMemoryMetadata & new_metadata, checkProperties(new_metadata, attach); /// Other parts of metadata initialized is separate methods - setColumns(std::move(new_metadata.columns)); - setSecondaryIndices(std::move(new_metadata.secondary_indices)); - setConstraints(std::move(new_metadata.constraints)); - setSortingKey(std::move(new_metadata.sorting_key)); - setPrimaryKey(std::move(new_metadata.primary_key)); + setInMemoryMetadata(new_metadata); } namespace diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 3031402715a..342a89c38ea 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -49,12 +49,7 @@ protected: : IStorage(getIDFromPart(part_)) , part(part_) { - setColumns(part_->storage.getColumns()); - setSecondaryIndices(part_->storage.getSecondaryIndices()); - setPrimaryKey(part_->storage.getPrimaryKey()); - setSortingKey(part_->storage.getSortingKey()); - setColumnTTLs(part->storage.getColumnTTLs()); - setTableTTLs(part->storage.getTableTTLs()); + setInMemoryMetadata(part_->storage.getInMemoryMetadata()); } private: diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 08dc81b4945..007625790f4 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -77,8 +77,10 @@ StorageBuffer::StorageBuffer( , log(&Poco::Logger::get("StorageBuffer (" + table_id_.getFullTableName() + ")")) , bg_pool(global_context.getBufferFlushSchedulePool()) { - setColumns(columns_); - setConstraints(constraints_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + metadata_.setConstraints(constraints_); + setInMemoryMetadata(metadata_); } @@ -778,7 +780,7 @@ void StorageBuffer::alter(const AlterCommands & params, const Context & context, StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); params.apply(new_metadata, context); DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata); - setColumns(std::move(new_metadata.columns)); + setInMemoryMetadata(new_metadata); } diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 9b2c5784d85..4348973ec60 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -100,7 +100,9 @@ StorageDictionary::StorageDictionary( : IStorage(table_id_) , dictionary_name(dictionary_name_) { - setColumns(ColumnsDescription{getNamesAndTypes(dictionary_structure_)}); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(ColumnsDescription{getNamesAndTypes(dictionary_structure_)}); + setInMemoryMetadata(metadata_); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 3383c609520..bf5f729ed19 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -283,8 +283,10 @@ StorageDistributed::StorageDistributed( , storage_policy(storage_policy_) , relative_data_path(relative_data_path_) { - setColumns(columns_); - setConstraints(constraints_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + metadata_.setConstraints(constraints_); + setInMemoryMetadata(metadata_); if (sharding_key_) { @@ -562,7 +564,7 @@ void StorageDistributed::alter(const AlterCommands & params, const Context & con StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); params.apply(new_metadata, context); DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata); - setColumns(std::move(new_metadata.columns)); + setInMemoryMetadata(new_metadata); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 0bcb624bec4..f94a7b71e56 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -166,7 +166,10 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us auto & first_path = paths[0]; Block header = StorageDistributedDirectoryMonitor::createStreamFromFile(first_path)->getHeader(); - setColumns(ColumnsDescription(header.getNamesAndTypesList())); + + StorageInMemoryMetadata metadata_; + metadata_.setColumns(ColumnsDescription(header.getNamesAndTypesList())); + setInMemoryMetadata(metadata_); } } } @@ -188,10 +191,12 @@ StorageFile::StorageFile(CommonArguments args) , compression_method(args.compression_method) , base_path(args.context.getPath()) { + StorageInMemoryMetadata metadata_; if (args.format_name != "Distributed") - setColumns(args.columns); + metadata_.setColumns(args.columns); - setConstraints(args.constraints); + metadata_.setConstraints(args.constraints); + setInMemoryMetadata(metadata_); } class StorageFileSource : public SourceWithProgress diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 8c186f38943..f69478a4bdd 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -388,7 +388,9 @@ StorageGenerateRandom::StorageGenerateRandom(const StorageID & table_id_, const : IStorage(table_id_), max_array_length(max_array_length_), max_string_length(max_string_length_) { random_seed = random_seed_ ? sipHash64(*random_seed_) : randomSeed(); - setColumns(columns_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + setInMemoryMetadata(metadata_); } diff --git a/src/Storages/StorageHDFS.cpp b/src/Storages/StorageHDFS.cpp index 352e0a43f39..082e40f6d6d 100644 --- a/src/Storages/StorageHDFS.cpp +++ b/src/Storages/StorageHDFS.cpp @@ -49,8 +49,11 @@ StorageHDFS::StorageHDFS(const String & uri_, , compression_method(compression_method_) { context.getRemoteHostFilter().checkURL(Poco::URI(uri)); - setColumns(columns_); - setConstraints(constraints_); + + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + metadata_.setConstraints(constraints_); + setInMemoryMetadata(metadata_); } namespace diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 8d23bd7bccf..ac2c0417c45 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -51,4 +51,19 @@ StorageInMemoryMetadata & StorageInMemoryMetadata::operator=(const StorageInMemo } +void StorageInMemoryMetadata::setColumns(ColumnsDescription columns_) +{ + columns = std::move(columns_); +} + +void StorageInMemoryMetadata::setSecondaryIndices(IndicesDescription secondary_indices_) +{ + secondary_indices = std::move(secondary_indices_); +} + +void StorageInMemoryMetadata::setConstraints(ConstraintsDescription constraints_) +{ + constraints = std::move(constraints_); +} + } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 889f8e49f69..f4d6e9b38b3 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -8,6 +8,7 @@ #include #include +#include namespace DB { @@ -47,6 +48,18 @@ struct StorageInMemoryMetadata StorageInMemoryMetadata(const StorageInMemoryMetadata & other); StorageInMemoryMetadata & operator=(const StorageInMemoryMetadata & other); + + + //////////////////////////////////////////////////////////////////////// + void setColumns(ColumnsDescription columns_); /// sets only real columns, possibly overwrites virtual ones. + + void setSecondaryIndices(IndicesDescription secondary_indices_); + + void setConstraints(ConstraintsDescription constraints_); + }; +using StorageMetadataPtr = std::shared_ptr; +using MultiVersionStorageMetadataPtr = MultiVersion; + } diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index e30ae55e715..92287051bf3 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -21,7 +21,9 @@ namespace ErrorCodes StorageInput::StorageInput(const StorageID & table_id, const ColumnsDescription & columns_) : IStorage(table_id) { - setColumns(columns_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + setInMemoryMetadata(metadata_); } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index bc2bbb2ce67..09be868bcfa 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -431,8 +431,10 @@ StorageLog::StorageLog( , max_compress_block_size(max_compress_block_size_) , file_checker(disk, table_path + "sizes.json") { - setColumns(columns_); - setConstraints(constraints_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + metadata_.setConstraints(constraints_); + setInMemoryMetadata(metadata_); if (relative_path_.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index a0c2fa87eb2..34d5e1d4374 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -50,7 +50,9 @@ StorageMaterializedView::StorageMaterializedView( bool attach_) : IStorage(table_id_), global_context(local_context.getGlobalContext()) { - setColumns(columns_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + setInMemoryMetadata(metadata_); if (!query.select) throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); @@ -209,7 +211,7 @@ void StorageMaterializedView::alter( /// end modify query DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata); - setColumns(std::move(new_metadata.columns)); + setInMemoryMetadata(new_metadata); } diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index b55a6227127..bb89bdb5c48 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -78,8 +78,10 @@ private: StorageMemory::StorageMemory(const StorageID & table_id_, ColumnsDescription columns_description_, ConstraintsDescription constraints_) : IStorage(table_id_) { - setColumns(std::move(columns_description_)); - setConstraints(std::move(constraints_)); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(std::move(columns_description_)); + metadata_.setConstraints(std::move(constraints_)); + setInMemoryMetadata(metadata_); } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 8264eaa4cb6..3685a777bf0 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -51,7 +51,9 @@ StorageMerge::StorageMerge( , table_name_regexp(table_name_regexp_) , global_context(context_) { - setColumns(columns_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + setInMemoryMetadata(metadata_); } template @@ -393,7 +395,7 @@ void StorageMerge::alter( StorageInMemoryMetadata storage_metadata = getInMemoryMetadata(); params.apply(storage_metadata, context); DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, storage_metadata); - setColumns(storage_metadata.columns); + setInMemoryMetadata(storage_metadata); } Block StorageMerge::getQueryHeader( diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index ee44ca7948e..f9aad8a58a7 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -56,8 +56,10 @@ StorageMySQL::StorageMySQL( , pool(std::move(pool_)) , global_context(context_) { - setColumns(columns_); - setConstraints(constraints_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + metadata_.setConstraints(constraints_); + setInMemoryMetadata(metadata_); } diff --git a/src/Storages/StorageNull.cpp b/src/Storages/StorageNull.cpp index 182ce09ef96..7589c4b44dc 100644 --- a/src/Storages/StorageNull.cpp +++ b/src/Storages/StorageNull.cpp @@ -54,7 +54,7 @@ void StorageNull::alter( StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); params.apply(new_metadata, context); DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata); - setColumns(std::move(new_metadata.columns)); + setInMemoryMetadata(new_metadata); } } diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index 5fb4a16a24b..fe8bd05d53a 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -59,8 +59,10 @@ protected: StorageNull(const StorageID & table_id_, ColumnsDescription columns_description_, ConstraintsDescription constraints_) : IStorage(table_id_) { - setColumns(std::move(columns_description_)); - setConstraints(std::move(constraints_)); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_description_); + metadata_.setConstraints(constraints_); + setInMemoryMetadata(metadata_); } }; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 397d064ba15..acaa2bcc7d6 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -206,8 +206,10 @@ StorageS3::StorageS3( , compression_method(compression_method_) { context_global.getRemoteHostFilter().checkURL(uri_.uri); - setColumns(columns_); - setConstraints(constraints_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + metadata_.setConstraints(constraints_); + setInMemoryMetadata(metadata_); auto settings = context_.getStorageS3Settings().getSettings(uri.endpoint); Aws::Auth::AWSCredentials credentials(access_key_id_, secret_access_key_); diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 86bfed5ac84..38b4d30c25b 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -96,8 +96,11 @@ StorageSetOrJoinBase::StorageSetOrJoinBase( const Context & context_) : IStorage(table_id_) { - setColumns(columns_); - setConstraints(constraints_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + metadata_.setConstraints(constraints_); + setInMemoryMetadata(metadata_); + if (relative_path_.empty()) throw Exception("Join and Set storages require data path", ErrorCodes::INCORRECT_FILE_NAME); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index b61d52657dd..b68505fa147 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -223,8 +223,10 @@ StorageStripeLog::StorageStripeLog( , file_checker(disk, table_path + "sizes.json") , log(&Poco::Logger::get("StorageStripeLog")) { - setColumns(columns_); - setConstraints(constraints_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + metadata_.setConstraints(constraints_); + setInMemoryMetadata(metadata_); if (relative_path_.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 2a62068516e..5bca6072da0 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -336,8 +336,10 @@ StorageTinyLog::StorageTinyLog( , file_checker(disk, table_path + "sizes.json") , log(&Poco::Logger::get("StorageTinyLog")) { - setColumns(columns_); - setConstraints(constraints_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + metadata_.setConstraints(constraints_); + setInMemoryMetadata(metadata_); if (relative_path_.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index a69e140fe5a..0301412e029 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -43,8 +43,11 @@ IStorageURLBase::IStorageURLBase( , format_name(format_name_) { context_global.getRemoteHostFilter().checkURL(uri); - setColumns(columns_); - setConstraints(constraints_); + + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + metadata_.setConstraints(constraints_); + setInMemoryMetadata(metadata_); } namespace diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index cf0b39df8f1..5ba36a936e2 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -16,7 +16,9 @@ StorageValues::StorageValues( const NamesAndTypesList & virtuals_) : IStorage(table_id_), res_block(res_block_), virtuals(virtuals_) { - setColumns(columns_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + setInMemoryMetadata(metadata_); } Pipes StorageValues::read( diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 055faed5899..60ae681e002 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -38,7 +38,9 @@ StorageView::StorageView( const ColumnsDescription & columns_) : IStorage(table_id_) { - setColumns(columns_); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(columns_); + setInMemoryMetadata(metadata_); if (!query.select) throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index b3951bc3f75..1ceff26ba83 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -23,7 +23,9 @@ protected: public: IStorageSystemOneBlock(const String & name_) : IStorage({"system", name_}) { - setColumns(ColumnsDescription(Self::getNamesAndTypes())); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(ColumnsDescription(Self::getNamesAndTypes())); + setInMemoryMetadata(metadata_); } Pipes read(const Names & column_names, diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 90e52ad373e..6359e367106 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -26,7 +26,8 @@ namespace ErrorCodes StorageSystemColumns::StorageSystemColumns(const std::string & name_) : IStorage({"system", name_}) { - setColumns(ColumnsDescription( + StorageInMemoryMetadata metadata_; + metadata_.setColumns(ColumnsDescription( { { "database", std::make_shared() }, { "table", std::make_shared() }, @@ -45,6 +46,7 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_) { "is_in_sampling_key", std::make_shared() }, { "compression_codec", std::make_shared() }, })); + setInMemoryMetadata(metadata_); } diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index f3fd51330d9..ef88c3ca058 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -30,7 +30,8 @@ protected: explicit StorageSystemDetachedParts() : IStorage({"system", "detached_parts"}) { - setColumns(ColumnsDescription{{ + StorageInMemoryMetadata metadata_; + metadata_.setColumns(ColumnsDescription{{ {"database", std::make_shared()}, {"table", std::make_shared()}, {"partition_id", std::make_shared(std::make_shared())}, @@ -41,6 +42,7 @@ protected: {"max_block_number", std::make_shared(std::make_shared())}, {"level", std::make_shared(std::make_shared())} }}); + setInMemoryMetadata(metadata_); } Pipes read( diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index b5a5026b2e7..5905080539e 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -14,7 +14,8 @@ namespace ErrorCodes StorageSystemDisks::StorageSystemDisks(const std::string & name_) : IStorage({"system", name_}) { - setColumns(ColumnsDescription( + StorageInMemoryMetadata metadata_; + metadata_.setColumns(ColumnsDescription( { {"name", std::make_shared()}, {"path", std::make_shared()}, @@ -22,6 +23,7 @@ StorageSystemDisks::StorageSystemDisks(const std::string & name_) {"total_space", std::make_shared()}, {"keep_free_space", std::make_shared()}, })); + setInMemoryMetadata(metadata_); } Pipes StorageSystemDisks::read( diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index 20dcc58f652..0fa7b71555e 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -118,7 +118,9 @@ private: StorageSystemNumbers::StorageSystemNumbers(const StorageID & table_id, bool multithreaded_, std::optional limit_, UInt64 offset_, bool even_distribution_) : IStorage(table_id), multithreaded(multithreaded_), even_distribution(even_distribution_), limit(limit_), offset(offset_) { - setColumns(ColumnsDescription({{"number", std::make_shared()}})); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(ColumnsDescription({{"number", std::make_shared()}})); + setInMemoryMetadata(metadata_); } Pipes StorageSystemNumbers::read( diff --git a/src/Storages/System/StorageSystemOne.cpp b/src/Storages/System/StorageSystemOne.cpp index 6cbb634d2b7..e7c8c446847 100644 --- a/src/Storages/System/StorageSystemOne.cpp +++ b/src/Storages/System/StorageSystemOne.cpp @@ -14,7 +14,9 @@ namespace DB StorageSystemOne::StorageSystemOne(const std::string & name_) : IStorage({"system", name_}) { - setColumns(ColumnsDescription({{"dummy", std::make_shared()}})); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(ColumnsDescription({{"dummy", std::make_shared()}})); + setInMemoryMetadata(metadata_); } diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 925a5df889e..42a432489f4 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -277,7 +277,9 @@ StorageSystemPartsBase::StorageSystemPartsBase(std::string name_, NamesAndTypesL add_alias("bytes", "bytes_on_disk"); add_alias("marks_size", "marks_bytes"); - setColumns(tmp_columns); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(tmp_columns); + setInMemoryMetadata(metadata_); } NamesAndTypesList StorageSystemPartsBase::getVirtuals() const diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index c2cd3a1e4b1..ca71e7e5f74 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -19,7 +19,8 @@ namespace DB StorageSystemReplicas::StorageSystemReplicas(const std::string & name_) : IStorage({"system", name_}) { - setColumns(ColumnsDescription({ + StorageInMemoryMetadata metadata_; + metadata_.setColumns(ColumnsDescription({ { "database", std::make_shared() }, { "table", std::make_shared() }, { "engine", std::make_shared() }, @@ -52,6 +53,7 @@ StorageSystemReplicas::StorageSystemReplicas(const std::string & name_) { "active_replicas", std::make_shared() }, { "zookeeper_exception", std::make_shared() }, })); + setInMemoryMetadata(metadata_); } diff --git a/src/Storages/System/StorageSystemStoragePolicies.cpp b/src/Storages/System/StorageSystemStoragePolicies.cpp index acbc9d72a20..dbb47dc771a 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.cpp +++ b/src/Storages/System/StorageSystemStoragePolicies.cpp @@ -17,7 +17,8 @@ namespace ErrorCodes StorageSystemStoragePolicies::StorageSystemStoragePolicies(const std::string & name_) : IStorage({"system", name_}) { - setColumns( + StorageInMemoryMetadata metadata_; + metadata_.setColumns( ColumnsDescription({ {"policy_name", std::make_shared()}, {"volume_name", std::make_shared()}, @@ -26,6 +27,7 @@ StorageSystemStoragePolicies::StorageSystemStoragePolicies(const std::string & n {"max_data_part_size", std::make_shared()}, {"move_factor", std::make_shared()} })); + setInMemoryMetadata(metadata_); } Pipes StorageSystemStoragePolicies::read( diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 2bf6595bf53..84d441a8c6e 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -33,7 +33,8 @@ namespace ErrorCodes StorageSystemTables::StorageSystemTables(const std::string & name_) : IStorage({"system", name_}) { - setColumns(ColumnsDescription( + StorageInMemoryMetadata metadata_; + metadata_.setColumns(ColumnsDescription( { {"database", std::make_shared()}, {"name", std::make_shared()}, @@ -55,6 +56,7 @@ StorageSystemTables::StorageSystemTables(const std::string & name_) {"total_rows", std::make_shared(std::make_shared())}, {"total_bytes", std::make_shared(std::make_shared())}, })); + setInMemoryMetadata(metadata_); } diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index cd2fa0a6059..438d31e7e02 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -84,7 +84,10 @@ private: StorageSystemZeros::StorageSystemZeros(const StorageID & table_id_, bool multithreaded_, std::optional limit_) : IStorage(table_id_), multithreaded(multithreaded_), limit(limit_) { - setColumns(ColumnsDescription({{"zero", std::make_shared()}})); + StorageInMemoryMetadata metadata_; + metadata_.setColumns(ColumnsDescription({{"zero", std::make_shared()}})); + setInMemoryMetadata(metadata_); + } Pipes StorageSystemZeros::read( From aa30649ce5eb3edc14641b595ccca6c3cba38dfa Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 15 Jun 2020 20:10:14 +0300 Subject: [PATCH 057/211] Working setColumns, setConstraints, setIndices --- src/Storages/IStorage.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 5 +---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 4d01bb5370d..403f5293588 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -82,7 +82,7 @@ public: IStorage() = delete; /// Storage fields should be initialized in separate methods like setColumns /// or setTableTTLs. - explicit IStorage(StorageID storage_id_) : storage_id(std::move(storage_id_)) {} //-V730 + explicit IStorage(StorageID storage_id_) : storage_id(std::move(storage_id_)), metadata(std::make_shared()) {} //-V730 virtual ~IStorage() = default; IStorage(const IStorage &) = delete; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 14be526d7f6..3414143c46b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -142,9 +142,8 @@ MergeTreeData::MergeTreeData( if (relative_data_path.empty()) throw Exception("MergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME); - setSettingsChanges(metadata_.settings_changes); - const auto settings = getSettings(); setProperties(metadata_, attach); + const auto settings = getSettings(); /// NOTE: using the same columns list as is read when performing actual merges. merging_params.check(getColumns().getAllPhysical()); @@ -385,8 +384,6 @@ void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata void MergeTreeData::setProperties(const StorageInMemoryMetadata & new_metadata, bool attach) { checkProperties(new_metadata, attach); - - /// Other parts of metadata initialized is separate methods setInMemoryMetadata(new_metadata); } From 5fc41c7eccc9d98d524ee997ae689d5e5333820a Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 15 Jun 2020 20:17:06 +0300 Subject: [PATCH 058/211] Move set*Key methods to StorageInMemoryMetadata --- src/Storages/IStorage.cpp | 21 --------------------- src/Storages/IStorage.h | 12 ------------ src/Storages/MergeTree/MergeTreeData.cpp | 3 --- src/Storages/StorageInMemoryMetadata.h | 12 ++++++++++++ 4 files changed, 12 insertions(+), 36 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 6c045a6f365..2bbbabbff08 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -403,11 +403,6 @@ const KeyDescription & IStorage::getPartitionKey() const return metadata->partition_key; } -void IStorage::setPartitionKey(const KeyDescription & partition_key_) -{ - metadata->partition_key = partition_key_; -} - bool IStorage::isPartitionKeyDefined() const { return metadata->partition_key.definition_ast != nullptr; @@ -430,11 +425,6 @@ const KeyDescription & IStorage::getSortingKey() const return metadata->sorting_key; } -void IStorage::setSortingKey(const KeyDescription & sorting_key_) -{ - metadata->sorting_key = sorting_key_; -} - bool IStorage::isSortingKeyDefined() const { return metadata->sorting_key.definition_ast != nullptr; @@ -464,11 +454,6 @@ const KeyDescription & IStorage::getPrimaryKey() const return metadata->primary_key; } -void IStorage::setPrimaryKey(const KeyDescription & primary_key_) -{ - metadata->primary_key = primary_key_; -} - bool IStorage::isPrimaryKeyDefined() const { return metadata->primary_key.definition_ast != nullptr; @@ -498,12 +483,6 @@ const KeyDescription & IStorage::getSamplingKey() const return metadata->sampling_key; } -void IStorage::setSamplingKey(const KeyDescription & sampling_key_) -{ - metadata->sampling_key = sampling_key_; -} - - bool IStorage::isSamplingKeyDefined() const { return metadata->sampling_key.definition_ast != nullptr; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 403f5293588..ec7e8fc1795 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -438,9 +438,6 @@ public: /// Returns structure with partition key. const KeyDescription & getPartitionKey() const; - /// Set partition key for storage (methods bellow, are just wrappers for this - /// struct). - void setPartitionKey(const KeyDescription & partition_key_); /// Returns ASTExpressionList of partition key expression for storage or nullptr if there is none. ASTPtr getPartitionKeyAST() const { return metadata->partition_key.definition_ast; } /// Storage has user-defined (in CREATE query) partition key. @@ -453,9 +450,6 @@ public: /// Returns structure with sorting key. const KeyDescription & getSortingKey() const; - /// Set sorting key for storage (methods bellow, are just wrappers for this - /// struct). - void setSortingKey(const KeyDescription & sorting_key_); /// Returns ASTExpressionList of sorting key expression for storage or nullptr if there is none. ASTPtr getSortingKeyAST() const { return metadata->sorting_key.definition_ast; } /// Storage has user-defined (in CREATE query) sorting key. @@ -470,9 +464,6 @@ public: /// Returns structure with primary key. const KeyDescription & getPrimaryKey() const; - /// Set primary key for storage (methods bellow, are just wrappers for this - /// struct). - void setPrimaryKey(const KeyDescription & primary_key_); /// Returns ASTExpressionList of primary key expression for storage or nullptr if there is none. ASTPtr getPrimaryKeyAST() const { return metadata->primary_key.definition_ast; } /// Storage has user-defined (in CREATE query) sorting key. @@ -488,9 +479,6 @@ public: /// Returns structure with sampling key. const KeyDescription & getSamplingKey() const; - /// Set sampling key for storage (methods bellow, are just wrappers for this - /// struct). - void setSamplingKey(const KeyDescription & sampling_key_); /// Returns sampling expression AST for storage or nullptr if there is none. ASTPtr getSamplingKeyAST() const { return metadata->sampling_key.definition_ast; } /// Storage has user-defined (in CREATE query) sampling key. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3414143c46b..ab0544c641b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -154,8 +154,6 @@ MergeTreeData::MergeTreeData( if (!pk_sample_block.has(metadata_.sampling_key.column_names[0]) && !attach && !settings->compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility. throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS); - - setSamplingKey(metadata_.sampling_key); } MergeTreeDataFormatVersion min_format_version(0); @@ -472,7 +470,6 @@ void MergeTreeData::initPartitionKey(const KeyDescription & new_partition_key) } } } - setPartitionKey(new_partition_key); } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index f4d6e9b38b3..3b3c9d07c89 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -57,6 +57,18 @@ struct StorageInMemoryMetadata void setConstraints(ConstraintsDescription constraints_); + /// Set partition key for storage (methods bellow, are just wrappers for this + /// struct). + void setPartitionKey(const KeyDescription & partition_key_); + /// Set sorting key for storage (methods bellow, are just wrappers for this + /// struct). + void setSortingKey(const KeyDescription & sorting_key_); + /// Set primary key for storage (methods bellow, are just wrappers for this + /// struct). + void setPrimaryKey(const KeyDescription & primary_key_); + /// Set sampling key for storage (methods bellow, are just wrappers for this + /// struct). + void setSamplingKey(const KeyDescription & sampling_key_); }; using StorageMetadataPtr = std::shared_ptr; From e667eb57b2aeaea856c24ebbb6e869ee508a368d Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 15 Jun 2020 20:24:49 +0300 Subject: [PATCH 059/211] Working set*Keys methods --- src/Storages/MergeTree/MergeTreeData.cpp | 36 +++++++++++++----------- src/Storages/MergeTree/MergeTreeData.h | 2 +- 2 files changed, 20 insertions(+), 18 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ab0544c641b..8af38bdf500 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -118,7 +118,7 @@ const char * DELETE_ON_DESTROY_MARKER_PATH = "delete-on-destroy.txt"; MergeTreeData::MergeTreeData( const StorageID & table_id_, const String & relative_data_path_, - const StorageInMemoryMetadata & metadata_, + StorageInMemoryMetadata metadata_, Context & context_, const String & date_column_name, const MergingParams & merging_params_, @@ -142,28 +142,15 @@ MergeTreeData::MergeTreeData( if (relative_data_path.empty()) throw Exception("MergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME); - setProperties(metadata_, attach); - const auto settings = getSettings(); - - /// NOTE: using the same columns list as is read when performing actual merges. - merging_params.check(getColumns().getAllPhysical()); - - if (metadata_.sampling_key.definition_ast != nullptr) - { - const auto & pk_sample_block = getPrimaryKey().sample_block; - if (!pk_sample_block.has(metadata_.sampling_key.column_names[0]) && !attach - && !settings->compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility. - throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS); - } - MergeTreeDataFormatVersion min_format_version(0); + /// TODO(alesap) Move to register methods if (!date_column_name.empty()) { try { auto partition_by_ast = makeASTFunction("toYYYYMM", std::make_shared(date_column_name)); - auto partition_key = KeyDescription::getKeyFromAST(partition_by_ast, getColumns(), global_context); - initPartitionKey(partition_key); + metadata_.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, metadata_.columns, global_context); + initPartitionKey(metadata_.partition_key); if (minmax_idx_date_column_pos == -1) throw Exception("Could not find Date column", ErrorCodes::BAD_TYPE_OF_FIELD); @@ -182,6 +169,21 @@ MergeTreeData::MergeTreeData( min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING; } + setProperties(metadata_, attach); + const auto settings = getSettings(); + + /// NOTE: using the same columns list as is read when performing actual merges. + merging_params.check(getColumns().getAllPhysical()); + + if (metadata_.sampling_key.definition_ast != nullptr) + { + const auto & pk_sample_block = getPrimaryKey().sample_block; + if (!pk_sample_block.has(metadata_.sampling_key.column_names[0]) && !attach + && !settings->compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility. + throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS); + } + + setTTLExpressions(metadata_); /// format_file always contained on any data path diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 007c6898e60..12350b7bd10 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -323,7 +323,7 @@ public: /// attach - whether the existing table is attached or the new table is created. MergeTreeData(const StorageID & table_id_, const String & relative_data_path_, - const StorageInMemoryMetadata & metadata_, + StorageInMemoryMetadata metadata_, Context & context_, const String & date_column_name, const MergingParams & merging_params_, From 33a74a3ea05ee7ff405e7255c7faeeae08de144c Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 15 Jun 2020 20:50:53 +0300 Subject: [PATCH 060/211] TTL methods in StorageInMemoryMetadata --- src/Storages/IStorage.cpp | 11 ----------- src/Storages/IStorage.h | 2 -- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- src/Storages/StorageInMemoryMetadata.cpp | 10 ++++++++++ src/Storages/StorageInMemoryMetadata.h | 4 ++++ src/Storages/StorageMergeTree.cpp | 1 - 6 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 2bbbabbff08..afe61008553 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -506,11 +506,6 @@ TTLTableDescription IStorage::getTableTTLs() const return metadata->table_ttl; } -void IStorage::setTableTTLs(const TTLTableDescription & table_ttl_) -{ - std::lock_guard lock(ttl_mutex); - metadata->table_ttl = table_ttl_; -} bool IStorage::hasAnyTableTTL() const { @@ -523,12 +518,6 @@ TTLColumnsDescription IStorage::getColumnTTLs() const return metadata->column_ttls_by_name; } -void IStorage::setColumnTTLs(const TTLColumnsDescription & column_ttls_by_name_) -{ - std::lock_guard lock(ttl_mutex); - metadata->column_ttls_by_name = column_ttls_by_name_; -} - bool IStorage::hasAnyColumnTTL() const { std::lock_guard lock(ttl_mutex); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index ec7e8fc1795..f3081386c76 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -500,12 +500,10 @@ public: /// Common tables TTLs (for rows and moves). TTLTableDescription getTableTTLs() const; - void setTableTTLs(const TTLTableDescription & table_ttl_); bool hasAnyTableTTL() const; /// Separate TTLs for columns. TTLColumnsDescription getColumnTTLs() const; - void setColumnTTLs(const TTLColumnsDescription & column_ttls_by_name_); bool hasAnyColumnTTL() const; /// Just wrapper for table TTLs, return rows part of table TTLs. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8af38bdf500..24c787e7c63 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -520,8 +520,8 @@ void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_meta void MergeTreeData::setTTLExpressions(const StorageInMemoryMetadata & new_metadata) { checkTTLExpressions(new_metadata); - setColumnTTLs(new_metadata.column_ttls_by_name); - setTableTTLs(new_metadata.table_ttl); + //setColumnTTLs(new_metadata.column_ttls_by_name); + //setTableTTLs(new_metadata.table_ttl); } diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index ac2c0417c45..2d29ac433e9 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -66,4 +66,14 @@ void StorageInMemoryMetadata::setConstraints(ConstraintsDescription constraints_ constraints = std::move(constraints_); } +void StorageInMemoryMetadata::setTableTTLs(const TTLTableDescription & table_ttl_) +{ + table_ttl = table_ttl_; +} + +void StorageInMemoryMetadata::setColumnTTLs(const TTLColumnsDescription & column_ttls_by_name_) +{ + column_ttls_by_name = column_ttls_by_name_; +} + } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 3b3c9d07c89..b5c1a1997b6 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -69,6 +69,10 @@ struct StorageInMemoryMetadata /// Set sampling key for storage (methods bellow, are just wrappers for this /// struct). void setSamplingKey(const KeyDescription & sampling_key_); + + void setTableTTLs(const TTLTableDescription & table_ttl_); + + void setColumnTTLs(const TTLColumnsDescription & column_ttls_by_name_); }; using StorageMetadataPtr = std::shared_ptr; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2a7efa164d4..7007a544eac 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -281,7 +281,6 @@ void StorageMergeTree::alter( changeSettings(new_metadata.settings_changes, table_lock_holder); /// Reinitialize primary key because primary key column types might have changed. setProperties(new_metadata); - setTTLExpressions(new_metadata); DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata); From b47a7327fdbd5f2753b84aed98595c1a7d4df5e3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 15 Jun 2020 21:08:05 +0300 Subject: [PATCH 061/211] All set methods in metadata --- src/Storages/IStorage.cpp | 13 ------------- src/Storages/IStorage.h | 2 -- src/Storages/MergeTree/MergeTreeData.cpp | 4 +++- src/Storages/StorageInMemoryMetadata.cpp | 13 +++++++++++++ src/Storages/StorageInMemoryMetadata.h | 4 ++++ src/Storages/StorageMaterializedView.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 1 - src/Storages/StorageView.cpp | 4 ++-- 8 files changed, 25 insertions(+), 22 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index afe61008553..e5ab14e046e 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -616,24 +616,11 @@ ASTPtr IStorage::getSettingsChanges() const return nullptr; } -void IStorage::setSettingsChanges(const ASTPtr & settings_changes_) -{ - if (settings_changes_) - metadata->settings_changes = settings_changes_->clone(); - else - metadata->settings_changes = nullptr; -} - const SelectQueryDescription & IStorage::getSelectQuery() const { return metadata->select; } -void IStorage::setSelectQuery(const SelectQueryDescription & select_) -{ - metadata->select = select_; -} - bool IStorage::hasSelectQuery() const { return metadata->select.select_query != nullptr; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index f3081386c76..0f48f3bf63c 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -148,12 +148,10 @@ public: /// thread-unsafe part. lockStructure must be acquired /// Storage settings ASTPtr getSettingsChanges() const; - void setSettingsChanges(const ASTPtr & settings_changes_); bool hasSettingsChanges() const { return metadata->settings_changes != nullptr; } /// Select query for *View storages. const SelectQueryDescription & getSelectQuery() const; - void setSelectQuery(const SelectQueryDescription & select_); bool hasSelectQuery() const; StorageInMemoryMetadata getInMemoryMetadata() const { return *metadata; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 24c787e7c63..8971b50a0fd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1474,7 +1474,9 @@ void MergeTreeData::changeSettings( MergeTreeSettings copy = *getSettings(); copy.applyChanges(new_changes); storage_settings.set(std::make_unique(copy)); - setSettingsChanges(new_settings); + StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); + new_metadata.setSettingsChanges(new_settings); + setInMemoryMetadata(new_metadata); } } diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 2d29ac433e9..b6dd2f38c4e 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -76,4 +76,17 @@ void StorageInMemoryMetadata::setColumnTTLs(const TTLColumnsDescription & column column_ttls_by_name = column_ttls_by_name_; } +void StorageInMemoryMetadata::setSettingsChanges(const ASTPtr & settings_changes_) +{ + if (settings_changes_) + settings_changes = settings_changes_; + else + settings_changes = nullptr; +} + +void StorageInMemoryMetadata::setSelectQuery(const SelectQueryDescription & select_) +{ + select = select_; +} + } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index b5c1a1997b6..b129cdc7756 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -73,6 +73,10 @@ struct StorageInMemoryMetadata void setTableTTLs(const TTLTableDescription & table_ttl_); void setColumnTTLs(const TTLColumnsDescription & column_ttls_by_name_); + + void setSettingsChanges(const ASTPtr & settings_changes_); + + void setSelectQuery(const SelectQueryDescription & select_); }; using StorageMetadataPtr = std::shared_ptr; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 34d5e1d4374..638a13612f2 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -52,7 +52,6 @@ StorageMaterializedView::StorageMaterializedView( { StorageInMemoryMetadata metadata_; metadata_.setColumns(columns_); - setInMemoryMetadata(metadata_); if (!query.select) throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); @@ -68,7 +67,8 @@ StorageMaterializedView::StorageMaterializedView( throw Exception("UNION is not supported for MATERIALIZED VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW); auto select = SelectQueryDescription::getSelectQueryFromASTForMatView(query.select->clone(), local_context); - setSelectQuery(select); + metadata_.setSelectQuery(select); + setInMemoryMetadata(metadata_); if (!has_inner_table) target_table_id = query.to_table_id; @@ -206,7 +206,7 @@ void StorageMaterializedView::alter( DatabaseCatalog::instance().updateDependency(old_select.select_table_id, table_id, new_select.select_table_id, table_id); - setSelectQuery(new_select); + new_metadata.setSelectQuery(new_select); } /// end modify query diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d8e45b97438..cb5e5aaf701 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3648,7 +3648,6 @@ void StorageReplicatedMergeTree::alter( StorageInMemoryMetadata future_metadata = getInMemoryMetadata(); params.apply(future_metadata, query_context); - changeSettings(future_metadata.settings_changes, table_lock_holder); DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id, future_metadata); diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 60ae681e002..d8392b2edd8 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -40,7 +40,6 @@ StorageView::StorageView( { StorageInMemoryMetadata metadata_; metadata_.setColumns(columns_); - setInMemoryMetadata(metadata_); if (!query.select) throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); @@ -48,7 +47,8 @@ StorageView::StorageView( SelectQueryDescription description; description.inner_query = query.select->ptr(); - setSelectQuery(description); + metadata_.setSelectQuery(description); + setInMemoryMetadata(metadata_); } From 36ba0192df07424d5c5b7c1ca8a197648238c38a Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 15 Jun 2020 22:08:58 +0300 Subject: [PATCH 062/211] Metadata in read and write methods of IStorage --- src/Core/ExternalTable.cpp | 2 +- .../CreatingSetsBlockInputStream.cpp | 2 +- .../PushingToViewsBlockOutputStream.cpp | 2 +- src/DataStreams/RemoteQueryExecutor.cpp | 7 +++++-- src/Interpreters/InterpreterInsertQuery.cpp | 3 ++- src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- src/Interpreters/InterpreterSelectQuery.h | 1 + .../Transforms/CreatingSetsTransform.cpp | 2 +- src/Server/TCPHandler.cpp | 3 ++- src/Storages/IStorage.h | 3 +++ src/Storages/Kafka/StorageKafka.cpp | 3 ++- src/Storages/Kafka/StorageKafka.h | 2 ++ src/Storages/LiveView/StorageBlocks.h | 1 + src/Storages/LiveView/StorageLiveView.cpp | 1 + src/Storages/LiveView/StorageLiveView.h | 1 + .../MergeTree/StorageFromMergeTreeDataPart.h | 1 + src/Storages/StorageBuffer.cpp | 13 ++++++++++--- src/Storages/StorageBuffer.h | 3 ++- src/Storages/StorageDictionary.cpp | 1 + src/Storages/StorageDictionary.h | 4 +++- src/Storages/StorageDistributed.cpp | 3 ++- src/Storages/StorageDistributed.h | 3 ++- src/Storages/StorageFile.cpp | 2 ++ src/Storages/StorageFile.h | 2 ++ src/Storages/StorageGenerateRandom.cpp | 1 + src/Storages/StorageGenerateRandom.h | 1 + src/Storages/StorageHDFS.cpp | 5 +++-- src/Storages/StorageHDFS.h | 6 ++++-- src/Storages/StorageInput.cpp | 4 +++- src/Storages/StorageInput.h | 1 + src/Storages/StorageJoin.cpp | 1 + src/Storages/StorageJoin.h | 1 + src/Storages/StorageLog.cpp | 4 ++-- src/Storages/StorageLog.h | 3 ++- src/Storages/StorageMaterializedView.cpp | 11 ++++++++--- src/Storages/StorageMaterializedView.h | 3 ++- src/Storages/StorageMemory.cpp | 4 ++-- src/Storages/StorageMemory.h | 3 ++- src/Storages/StorageMerge.cpp | 4 +++- src/Storages/StorageMerge.h | 1 + src/Storages/StorageMergeTree.cpp | 3 ++- src/Storages/StorageMergeTree.h | 3 ++- src/Storages/StorageMySQL.cpp | 4 ++-- src/Storages/StorageMySQL.h | 3 ++- src/Storages/StorageNull.h | 3 ++- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- src/Storages/StorageReplicatedMergeTree.h | 3 ++- src/Storages/StorageS3.cpp | 3 ++- src/Storages/StorageS3.h | 3 ++- src/Storages/StorageSet.cpp | 2 +- src/Storages/StorageSet.h | 2 +- src/Storages/StorageStripeLog.cpp | 4 ++-- src/Storages/StorageStripeLog.h | 3 ++- src/Storages/StorageTinyLog.cpp | 4 ++-- src/Storages/StorageTinyLog.h | 3 ++- src/Storages/StorageURL.cpp | 6 ++++-- src/Storages/StorageURL.h | 3 ++- src/Storages/StorageValues.cpp | 1 + src/Storages/StorageValues.h | 1 + src/Storages/StorageView.cpp | 1 + src/Storages/StorageView.h | 1 + src/Storages/StorageXDBC.cpp | 8 +++++--- src/Storages/StorageXDBC.h | 17 +++++++++-------- src/Storages/System/IStorageSystemOneBlock.h | 4 +++- src/Storages/System/StorageSystemColumns.cpp | 1 + src/Storages/System/StorageSystemColumns.h | 1 + .../System/StorageSystemDetachedParts.cpp | 13 +++++++------ src/Storages/System/StorageSystemDisks.cpp | 1 + src/Storages/System/StorageSystemDisks.h | 1 + src/Storages/System/StorageSystemNumbers.cpp | 1 + src/Storages/System/StorageSystemNumbers.h | 1 + src/Storages/System/StorageSystemOne.cpp | 1 + src/Storages/System/StorageSystemOne.h | 1 + src/Storages/System/StorageSystemPartsBase.cpp | 13 +++++++------ src/Storages/System/StorageSystemPartsBase.h | 13 +++++++------ src/Storages/System/StorageSystemReplicas.cpp | 1 + src/Storages/System/StorageSystemReplicas.h | 1 + .../System/StorageSystemStoragePolicies.cpp | 13 +++++++------ .../System/StorageSystemStoragePolicies.h | 13 +++++++------ src/Storages/System/StorageSystemTables.cpp | 1 + src/Storages/System/StorageSystemTables.h | 1 + src/Storages/System/StorageSystemZeros.cpp | 13 +++++++------ src/Storages/System/StorageSystemZeros.h | 13 +++++++------ src/Storages/tests/gtest_storage_log.cpp | 6 ++++-- 84 files changed, 208 insertions(+), 111 deletions(-) diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 5ec6980dbfa..3639a109b42 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -167,7 +167,7 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, auto temporary_table = TemporaryTableHolder(context, ColumnsDescription{columns}, {}); auto storage = temporary_table.getTable(); context.addExternalTable(data->table_name, std::move(temporary_table)); - BlockOutputStreamPtr output = storage->write(ASTPtr(), context); + BlockOutputStreamPtr output = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), context); /// Write data auto sink = std::make_shared(std::move(output)); diff --git a/src/DataStreams/CreatingSetsBlockInputStream.cpp b/src/DataStreams/CreatingSetsBlockInputStream.cpp index e40b5979b05..2a2275a4e89 100644 --- a/src/DataStreams/CreatingSetsBlockInputStream.cpp +++ b/src/DataStreams/CreatingSetsBlockInputStream.cpp @@ -101,7 +101,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) BlockOutputStreamPtr table_out; if (subquery.table) - table_out = subquery.table->write({}, context); + table_out = subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), context); bool done_with_set = !subquery.set; bool done_with_join = !subquery.join; diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 2c2e6972158..fa213b054df 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -107,7 +107,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( /// Do not push to destination table if the flag is set if (!no_destination) { - output = storage->write(query_ptr, context); + output = storage->write(query_ptr, storage->getInMemoryMetadataPtr(), context); replicated_output = dynamic_cast(output.get()); } } diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index cf3b2c4abcd..071cb6e9aba 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -319,12 +319,15 @@ void RemoteQueryExecutor::sendExternalTables() for (const auto & table : external_tables) { StoragePtr cur = table.second; + auto metadata_snapshot = cur->getInMemoryMetadataPtr(); QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage(context); Pipes pipes; - pipes = cur->read(cur->getColumns().getNamesOfPhysical(), {}, context, - read_from_table_stage, DEFAULT_BLOCK_SIZE, 1); + pipes = cur->read( + cur->getColumns().getNamesOfPhysical(), + metadata_snapshot, {}, context, + read_from_table_stage, DEFAULT_BLOCK_SIZE, 1); auto data = std::make_unique(); data->table_name = table.first; diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 7deed262eda..1841c82b710 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -117,6 +117,7 @@ BlockIO InterpreterInsertQuery::execute() StoragePtr table = getTable(query); auto table_lock = table->lockStructureForShare( true, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto metadata_snapshot = table->getInMemoryMetadataPtr(); auto query_sample_block = getSampleBlock(query, table); if (!query.table_function) @@ -226,7 +227,7 @@ BlockIO InterpreterInsertQuery::execute() /// NOTE: we explicitly ignore bound materialized views when inserting into Kafka Storage. /// Otherwise we'll get duplicates when MV reads same rows again from Kafka. if (table->noPushingToViews() && !no_destination) - out = table->write(query_ptr, context); + out = table->write(query_ptr, metadata_snapshot, context); else out = std::make_shared(table, context, query_ptr, no_destination); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ac17a3042d8..c22296cfb26 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -255,6 +255,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( table_lock = storage->lockStructureForShare( false, context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); table_id = storage->getStorageID(); + metadata_snapshot = storage->getInMemoryMetadataPtr(); } if (has_input || !joined_tables.resolveTables()) @@ -1293,7 +1294,6 @@ void InterpreterSelectQuery::executeFetchColumns( else if (storage) { /// Table. - if (max_streams == 0) throw Exception("Logical error: zero number of streams requested", ErrorCodes::LOGICAL_ERROR); @@ -1324,7 +1324,7 @@ void InterpreterSelectQuery::executeFetchColumns( query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage); } - Pipes pipes = storage->read(required_columns, query_info, *context, processing_stage, max_block_size, max_streams); + Pipes pipes = storage->read(required_columns, metadata_snapshot, query_info, *context, processing_stage, max_block_size, max_streams); if (pipes.empty()) { diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 8ed775f60ae..8f7237ffd7e 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -184,6 +184,7 @@ private: StoragePtr storage; StorageID table_id = StorageID::createEmpty(); /// Will be initialized if storage is not nullptr TableStructureReadLockHolder table_lock; + StorageMetadataPtr metadata_snapshot; /// Used when we read from prepared input, not table or subquery. BlockInputStreamPtr input; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index af8fa4097df..c0e34d9fbd4 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -66,7 +66,7 @@ void CreatingSetsTransform::startSubquery(SubqueryForSet & subquery) elapsed_nanoseconds = 0; if (subquery.table) - table_out = subquery.table->write({}, context); + table_out = subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), context); done_with_set = !subquery.set; done_with_join = !subquery.join; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 7e17604c4c7..a01cc4fa0aa 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -952,8 +952,9 @@ bool TCPHandler::receiveData(bool scalar) storage = temporary_table.getTable(); query_context->addExternalTable(temporary_id.table_name, std::move(temporary_table)); } + auto metadata_snapshot = storage->getInMemoryMetadataPtr(); /// The data will be written directly to the table. - state.io.out = storage->write(ASTPtr(), *query_context); + state.io.out = storage->write(ASTPtr(), metadata_snapshot, *query_context); } if (state.need_receive_data_for_input) state.block_for_input = block; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 0f48f3bf63c..28ad7b0ea8b 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -155,6 +155,7 @@ public: /// thread-unsafe part. lockStructure must be acquired bool hasSelectQuery() const; StorageInMemoryMetadata getInMemoryMetadata() const { return *metadata; } + StorageMetadataPtr getInMemoryMetadataPtr() const { return metadata; } void setInMemoryMetadata(const StorageInMemoryMetadata & metadata_) { metadata = std::make_shared(metadata_); } Block getSampleBlock() const; /// ordinary + materialized. @@ -292,6 +293,7 @@ public: */ virtual Pipes read( const Names & /*column_names*/, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, @@ -309,6 +311,7 @@ public: */ virtual BlockOutputStreamPtr write( const ASTPtr & /*query*/, + const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) { throw Exception("Method write is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 2109afed932..190397bc675 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -201,6 +201,7 @@ String StorageKafka::getDefaultClientId(const StorageID & table_id_) Pipes StorageKafka::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /* query_info */, const Context & context, QueryProcessingStage::Enum /* processed_stage */, @@ -231,7 +232,7 @@ Pipes StorageKafka::read( } -BlockOutputStreamPtr StorageKafka::write(const ASTPtr &, const Context & context) +BlockOutputStreamPtr StorageKafka::write(const ASTPtr &, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) { auto modified_context = std::make_shared(context); modified_context->applySettingsChanges(settings_adjustments); diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index be3f89687fe..6f479ba2089 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -39,6 +39,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -47,6 +48,7 @@ public: BlockOutputStreamPtr write( const ASTPtr & query, + const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; void pushReadBuffer(ConsumerBufferPtr buf); diff --git a/src/Storages/LiveView/StorageBlocks.h b/src/Storages/LiveView/StorageBlocks.h index 78d60163d5e..56fd0c620c2 100644 --- a/src/Storages/LiveView/StorageBlocks.h +++ b/src/Storages/LiveView/StorageBlocks.h @@ -37,6 +37,7 @@ public: Pipes read( const Names & /*column_names*/, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index ade2d1c967d..cb4964f3c55 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -520,6 +520,7 @@ void StorageLiveView::refresh(const Context & context) Pipes StorageLiveView::read( const Names & /*column_names*/, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 458e74eb506..13386c7a4e6 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -126,6 +126,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 342a89c38ea..826af505b12 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -21,6 +21,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 007625790f4..3e419921115 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -145,6 +145,7 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context Pipes StorageBuffer::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -157,6 +158,7 @@ Pipes StorageBuffer::read( { auto destination = DatabaseCatalog::instance().getTable(destination_id, context); + auto destination_metadata_snapshot = destination->getInMemoryMetadataPtr(); if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); @@ -177,7 +179,9 @@ Pipes StorageBuffer::read( query_info.input_order_info = query_info.order_optimizer->getInputOrder(destination); /// The destination table has the same structure of the requested columns and we can simply read blocks from there. - pipes_from_dst = destination->read(column_names, query_info, context, processed_stage, max_block_size, num_streams); + pipes_from_dst = destination->read( + column_names, destination_metadata_snapshot, query_info, + context, processed_stage, max_block_size, num_streams); } else { @@ -210,7 +214,10 @@ Pipes StorageBuffer::read( } else { - pipes_from_dst = destination->read(columns_intersection, query_info, context, processed_stage, max_block_size, num_streams); + pipes_from_dst = destination->read( + columns_intersection, destination_metadata_snapshot, query_info, + context, processed_stage, max_block_size, num_streams); + for (auto & pipe : pipes_from_dst) { pipe.addSimpleTransform(std::make_shared( @@ -425,7 +432,7 @@ private: }; -BlockOutputStreamPtr StorageBuffer::write(const ASTPtr & /*query*/, const Context & /*context*/) +BlockOutputStreamPtr StorageBuffer::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) { return std::make_shared(*this); } diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 02fd35136bf..7cd73dc556c 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -57,13 +57,14 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; void startup() override; /// Flush all buffers into the subordinate table and stop background thread. diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 4348973ec60..25126ad951d 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -113,6 +113,7 @@ void StorageDictionary::checkTableCanBeDropped() const Pipes StorageDictionary::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index 7bb6fc22480..6175902381b 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -16,7 +16,9 @@ public: void checkTableCanBeDropped() const override; - Pipes read(const Names & column_names, + Pipes read( + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index bf5f729ed19..719811bbc6b 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -464,6 +464,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Con Pipes StorageDistributed::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -510,7 +511,7 @@ Pipes StorageDistributed::read( } -BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const Context & context) +BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) { auto cluster = getCluster(); const auto & settings = context.getSettingsRef(); diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 63021e0a169..3f148cfff01 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -70,13 +70,14 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; /// Removes temporary data in local filesystem. void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index f94a7b71e56..07df2b4ec8a 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -377,6 +377,7 @@ private: Pipes StorageFile::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -479,6 +480,7 @@ private: BlockOutputStreamPtr StorageFile::write( const ASTPtr & /*query*/, + const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) { if (format_name == "Distributed") diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index fa5034d946c..65589d245b9 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -26,6 +26,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -34,6 +35,7 @@ public: BlockOutputStreamPtr write( const ASTPtr & query, + const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; void truncate(const ASTPtr & /*query*/, const Context & /* context */, TableStructureWriteLockHolder &) override; diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index f69478a4bdd..f1d97a4e5c4 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -429,6 +429,7 @@ void registerStorageGenerateRandom(StorageFactory & factory) Pipes StorageGenerateRandom::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/StorageGenerateRandom.h b/src/Storages/StorageGenerateRandom.h index 955b8bd671d..0d068eb951e 100644 --- a/src/Storages/StorageGenerateRandom.h +++ b/src/Storages/StorageGenerateRandom.h @@ -17,6 +17,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/StorageHDFS.cpp b/src/Storages/StorageHDFS.cpp index 082e40f6d6d..77afc4c47c8 100644 --- a/src/Storages/StorageHDFS.cpp +++ b/src/Storages/StorageHDFS.cpp @@ -264,9 +264,10 @@ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, c Pipes StorageHDFS::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & context_, - QueryProcessingStage::Enum /*processed_stage*/, + QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned num_streams) { @@ -300,7 +301,7 @@ Pipes StorageHDFS::read( return pipes; } -BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const Context & /*context*/) +BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) { return std::make_shared(uri, format_name, diff --git a/src/Storages/StorageHDFS.h b/src/Storages/StorageHDFS.h index 5b250247b84..62425cc518f 100644 --- a/src/Storages/StorageHDFS.h +++ b/src/Storages/StorageHDFS.h @@ -19,14 +19,16 @@ class StorageHDFS final : public ext::shared_ptr_helper, public ISt public: String getName() const override { return "HDFS"; } - Pipes read(const Names & column_names, + Pipes read( + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; NamesAndTypesList getVirtuals() const override; diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index 92287051bf3..4117a6b3a37 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -58,7 +58,9 @@ void StorageInput::setInputStream(BlockInputStreamPtr input_stream_) } -Pipes StorageInput::read(const Names & /*column_names*/, +Pipes StorageInput::read( + const Names & /*column_names*/, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/StorageInput.h b/src/Storages/StorageInput.h index 60bda222c2e..f4425ee8cd5 100644 --- a/src/Storages/StorageInput.h +++ b/src/Storages/StorageInput.h @@ -19,6 +19,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 5cceefe907b..7ed4c1c110b 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -435,6 +435,7 @@ private: // TODO: multiple stream read and index read Pipes StorageJoin::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index f956abb4d3b..40dbf1b44dd 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -38,6 +38,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 09be868bcfa..a09a99b30e1 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -577,6 +577,7 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount() const Pipes StorageLog::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -620,8 +621,7 @@ Pipes StorageLog::read( return pipes; } -BlockOutputStreamPtr StorageLog::write( - const ASTPtr & /*query*/, const Context & /*context*/) +BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) { loadMarks(); return std::make_shared(*this); diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 2c2abdb0275..60f885ce45c 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -26,13 +26,14 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 638a13612f2..a387eadabe0 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -108,6 +108,7 @@ QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(cons Pipes StorageMaterializedView::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -117,11 +118,12 @@ Pipes StorageMaterializedView::read( auto storage = getTargetTable(); auto lock = storage->lockStructureForShare( false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto metadata_snapshot = storage->getInMemoryMetadataPtr(); if (query_info.order_optimizer) query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage); - Pipes pipes = storage->read(column_names, query_info, context, processed_stage, max_block_size, num_streams); + Pipes pipes = storage->read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); for (auto & pipe : pipes) pipe.addTableLock(lock); @@ -129,12 +131,15 @@ Pipes StorageMaterializedView::read( return pipes; } -BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const Context & context) +BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) { auto storage = getTargetTable(); auto lock = storage->lockStructureForShare( true, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - auto stream = storage->write(query, context); + + auto metadata_snapshot = storage->getInMemoryMetadataPtr(); + auto stream = storage->write(query, metadata_snapshot, context); + stream->addTableLock(lock); return stream; } diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 480c75aa114..42fe186a068 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -31,7 +31,7 @@ public: return getTargetTable()->mayBenefitFromIndexForIn(left_in_operand, query_context); } - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; void drop() override; @@ -63,6 +63,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index bb89bdb5c48..f9c39d78a05 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -87,6 +87,7 @@ StorageMemory::StorageMemory(const StorageID & table_id_, ColumnsDescription col Pipes StorageMemory::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, @@ -119,8 +120,7 @@ Pipes StorageMemory::read( } -BlockOutputStreamPtr StorageMemory::write( - const ASTPtr & /*query*/, const Context & /*context*/) +BlockOutputStreamPtr StorageMemory::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) { return std::make_shared(*this); } diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 5a79358d76d..3c583533462 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -30,13 +30,14 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; void drop() override; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 3685a777bf0..6656e91189c 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -129,6 +129,7 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & Pipes StorageMerge::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -243,6 +244,7 @@ Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const Quer return pipes; } + auto metadata_snapshot = storage->getInMemoryMetadataPtr(); auto storage_stage = storage->getQueryProcessingStage(*modified_context, QueryProcessingStage::Complete, query_info.query); if (processed_stage <= storage_stage) { @@ -250,7 +252,7 @@ Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const Quer if (real_column_names.empty()) real_column_names.push_back(ExpressionActions::getSmallestColumn(storage->getColumns().getAllPhysical())); - pipes = storage->read(real_column_names, modified_query_info, *modified_context, processed_stage, max_block_size, UInt32(streams_num)); + pipes = storage->read(real_column_names, metadata_snapshot, modified_query_info, *modified_context, processed_stage, max_block_size, UInt32(streams_num)); } else if (processed_stage > storage_stage) { diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index adf4a40e675..a5d3b8d2667 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -31,6 +31,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7007a544eac..e3f48a05d6e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -177,6 +177,7 @@ StorageMergeTree::~StorageMergeTree() Pipes StorageMergeTree::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -196,7 +197,7 @@ std::optional StorageMergeTree::totalBytes() const return getTotalActiveSizeInBytes(); } -BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const Context & context) +BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) { return std::make_shared(*this, context.getSettingsRef().max_partitions_per_insert_block); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index c6c8f99a62a..679726826d4 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -39,6 +39,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -48,7 +49,7 @@ public: std::optional totalRows() const override; std::optional totalBytes() const override; - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; /** Perform the next step in combining the parts. */ diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index f9aad8a58a7..dce9e0f38ec 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -65,6 +65,7 @@ StorageMySQL::StorageMySQL( Pipes StorageMySQL::read( const Names & column_names_, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info_, const Context & context_, QueryProcessingStage::Enum /*processed_stage*/, @@ -198,8 +199,7 @@ private: }; -BlockOutputStreamPtr StorageMySQL::write( - const ASTPtr & /*query*/, const Context & context) +BlockOutputStreamPtr StorageMySQL::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) { return std::make_shared(*this, remote_database_name, remote_table_name, pool.get(), context.getSettingsRef().mysql_max_rows_to_insert); } diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index 8b98536e4d7..287c65db6f3 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -39,13 +39,14 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; private: friend class StorageMySQLBlockOutputStream; diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index fe8bd05d53a..72934d185c7 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -24,6 +24,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo &, const Context & /*context*/, QueryProcessingStage::Enum /*processing_stage*/, @@ -35,7 +36,7 @@ public: return pipes; } - BlockOutputStreamPtr write(const ASTPtr &, const Context &) override + BlockOutputStreamPtr write(const ASTPtr &, const StorageMetadataPtr & /*metadata_snapshot*/, const Context &) override { return std::make_shared(getSampleBlock()); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index cb5e5aaf701..650578d7560 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3381,6 +3381,7 @@ ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock StorageReplicatedMerg Pipes StorageReplicatedMergeTree::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -3442,7 +3443,7 @@ void StorageReplicatedMergeTree::assertNotReadonly() const } -BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const Context & context) +BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) { const auto storage_settings_ptr = getSettings(); assertNotReadonly(); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index aae0b9c81b8..5fcfd98e71d 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -89,6 +89,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -98,7 +99,7 @@ public: std::optional totalRows() const override; std::optional totalBytes() const override; - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & query_context) override; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index acaa2bcc7d6..093f4450ecb 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -285,6 +285,7 @@ Strings listFilesWithRegexpMatching(Aws::S3::S3Client & client, const S3::URI & Pipes StorageS3::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -320,7 +321,7 @@ Pipes StorageS3::read( return narrowPipes(std::move(pipes), num_streams); } -BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const Context & /*context*/) +BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) { return std::make_shared( format_name, min_upload_part_size, getSampleBlock(), context_global, diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index fc19fe06da0..665c00b8033 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -48,13 +48,14 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; NamesAndTypesList getVirtuals() const override; diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 38b4d30c25b..cddd4657cd1 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -81,7 +81,7 @@ void SetOrJoinBlockOutputStream::writeSuffix() } -BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const Context & /*context*/) +BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) { UInt64 id = ++increment; return std::make_shared(*this, path, path + "tmp/", toString(id) + ".bin"); diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index cf85dfd5d5b..b7785aadc6a 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -21,7 +21,7 @@ class StorageSetOrJoinBase : public IStorage public: void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; Strings getDataPaths() const override { return {path}; } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index b68505fa147..c320d0afb42 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -253,6 +253,7 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora Pipes StorageStripeLog::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -299,8 +300,7 @@ Pipes StorageStripeLog::read( } -BlockOutputStreamPtr StorageStripeLog::write( - const ASTPtr & /*query*/, const Context & /*context*/) +BlockOutputStreamPtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) { return std::make_shared(*this); } diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index ed8e5da081e..d06758a60e8 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -27,13 +27,14 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 5bca6072da0..42b70f716f4 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -394,6 +394,7 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const Storage Pipes StorageTinyLog::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -413,8 +414,7 @@ Pipes StorageTinyLog::read( } -BlockOutputStreamPtr StorageTinyLog::write( - const ASTPtr & /*query*/, const Context & /*context*/) +BlockOutputStreamPtr StorageTinyLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) { return std::make_shared(*this); } diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index 102ec76fda3..a55bf6d0dcf 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -26,13 +26,14 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 0301412e029..0361718c616 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -155,7 +155,9 @@ std::function IStorageURLBase::getReadPOSTDataCallback(con } -Pipes IStorageURLBase::read(const Names & column_names, +Pipes IStorageURLBase::read( + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -183,7 +185,7 @@ Pipes IStorageURLBase::read(const Names & column_names, return pipes; } -BlockOutputStreamPtr IStorageURLBase::write(const ASTPtr & /*query*/, const Context & /*context*/) +BlockOutputStreamPtr IStorageURLBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) { return std::make_shared( uri, format_name, getSampleBlock(), context_global, diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 5a6584f0301..ecd57024a44 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -21,13 +21,14 @@ class IStorageURLBase : public IStorage public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; protected: IStorageURLBase( diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index 5ba36a936e2..bb29b4a0932 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -23,6 +23,7 @@ StorageValues::StorageValues( Pipes StorageValues::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/StorageValues.h b/src/Storages/StorageValues.h index 254f3bfa8aa..88fb023fb2b 100644 --- a/src/Storages/StorageValues.h +++ b/src/Storages/StorageValues.h @@ -17,6 +17,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index d8392b2edd8..52b7e8764d9 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -54,6 +54,7 @@ StorageView::StorageView( Pipes StorageView::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/StorageView.h b/src/Storages/StorageView.h index c6b48d4d357..143ed3c06c4 100644 --- a/src/Storages/StorageView.h +++ b/src/Storages/StorageView.h @@ -23,6 +23,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index 08538798389..c7fa8a88251 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -82,7 +82,9 @@ std::function StorageXDBC::getReadPOSTDataCallback(const N return [query](std::ostream & os) { os << "query=" << query; }; } -Pipes StorageXDBC::read(const Names & column_names, +Pipes StorageXDBC::read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -92,10 +94,10 @@ Pipes StorageXDBC::read(const Names & column_names, check(column_names); bridge_helper->startBridgeSync(); - return IStorageURLBase::read(column_names, query_info, context, processed_stage, max_block_size, num_streams); + return IStorageURLBase::read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } -BlockOutputStreamPtr StorageXDBC::write(const ASTPtr & /*query*/, const Context & context) +BlockOutputStreamPtr StorageXDBC::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) { bridge_helper->startBridgeSync(); diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index afc61dac5cd..4488122656d 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -15,13 +15,14 @@ namespace DB class StorageXDBC : public IStorageURLBase { public: - - Pipes read(const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams) override; + Pipes read( + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; StorageXDBC(const StorageID & table_id_, const std::string & remote_database_name, @@ -29,7 +30,7 @@ public: const ColumnsDescription & columns_, const Context & context_, BridgeHelperPtr bridge_helper_); - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; private: diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 1ceff26ba83..7644f62b96d 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -28,7 +28,9 @@ public: setInMemoryMetadata(metadata_); } - Pipes read(const Names & column_names, + Pipes read( + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 6359e367106..646a5434b64 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -242,6 +242,7 @@ private: Pipes StorageSystemColumns::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/System/StorageSystemColumns.h b/src/Storages/System/StorageSystemColumns.h index 66b423efb96..7336b406183 100644 --- a/src/Storages/System/StorageSystemColumns.h +++ b/src/Storages/System/StorageSystemColumns.h @@ -19,6 +19,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index ef88c3ca058..3d24d90bbef 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -46,12 +46,13 @@ protected: } Pipes read( - const Names & /* column_names */, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum /*processed_stage*/, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) override + const Names & /* column_names */, + const StorageMetadataPtr & /*metadata_snapshot*/, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum /*processed_stage*/, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) override { StoragesInfoStream stream(query_info, context); diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index 5905080539e..36fde616bd4 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -28,6 +28,7 @@ StorageSystemDisks::StorageSystemDisks(const std::string & name_) Pipes StorageSystemDisks::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/System/StorageSystemDisks.h b/src/Storages/System/StorageSystemDisks.h index b136a217508..714a0a5428c 100644 --- a/src/Storages/System/StorageSystemDisks.h +++ b/src/Storages/System/StorageSystemDisks.h @@ -22,6 +22,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index 0fa7b71555e..fd7e04cfb1f 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -125,6 +125,7 @@ StorageSystemNumbers::StorageSystemNumbers(const StorageID & table_id, bool mult Pipes StorageSystemNumbers::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo &, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/System/StorageSystemNumbers.h b/src/Storages/System/StorageSystemNumbers.h index 4d205728496..88d3651c7f9 100644 --- a/src/Storages/System/StorageSystemNumbers.h +++ b/src/Storages/System/StorageSystemNumbers.h @@ -31,6 +31,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/System/StorageSystemOne.cpp b/src/Storages/System/StorageSystemOne.cpp index e7c8c446847..af736c215b5 100644 --- a/src/Storages/System/StorageSystemOne.cpp +++ b/src/Storages/System/StorageSystemOne.cpp @@ -22,6 +22,7 @@ StorageSystemOne::StorageSystemOne(const std::string & name_) Pipes StorageSystemOne::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo &, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/System/StorageSystemOne.h b/src/Storages/System/StorageSystemOne.h index 9fb3a670949..3b3e531dc86 100644 --- a/src/Storages/System/StorageSystemOne.h +++ b/src/Storages/System/StorageSystemOne.h @@ -23,6 +23,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 42a432489f4..e599bbb19e3 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -224,12 +224,13 @@ StoragesInfo StoragesInfoStream::next() } Pipes StorageSystemPartsBase::read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum /*processed_stage*/, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const Names & column_names, + const StorageMetadataPtr & /*metadata_*/, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum /*processed_stage*/, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) { bool has_state_column = hasStateColumn(column_names); diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 7b9ce7cbae2..a46cecec9dd 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -56,12 +56,13 @@ class StorageSystemPartsBase : public IStorage { public: Pipes read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams) override; + const Names & column_names, + const StorageMetadataPtr & metadata_, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; NamesAndTypesList getVirtuals() const override; diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index ca71e7e5f74..24861fcbd6a 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -59,6 +59,7 @@ StorageSystemReplicas::StorageSystemReplicas(const std::string & name_) Pipes StorageSystemReplicas::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/System/StorageSystemReplicas.h b/src/Storages/System/StorageSystemReplicas.h index 94b0d6c9d06..b068ebc8b0a 100644 --- a/src/Storages/System/StorageSystemReplicas.h +++ b/src/Storages/System/StorageSystemReplicas.h @@ -20,6 +20,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/System/StorageSystemStoragePolicies.cpp b/src/Storages/System/StorageSystemStoragePolicies.cpp index dbb47dc771a..a80747c1fa1 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.cpp +++ b/src/Storages/System/StorageSystemStoragePolicies.cpp @@ -31,12 +31,13 @@ StorageSystemStoragePolicies::StorageSystemStoragePolicies(const std::string & n } Pipes StorageSystemStoragePolicies::read( - const Names & column_names, - const SelectQueryInfo & /*query_info*/, - const Context & context, - QueryProcessingStage::Enum /*processed_stage*/, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, + const SelectQueryInfo & /*query_info*/, + const Context & context, + QueryProcessingStage::Enum /*processed_stage*/, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) { check(column_names); diff --git a/src/Storages/System/StorageSystemStoragePolicies.h b/src/Storages/System/StorageSystemStoragePolicies.h index 79e89863bf0..a1427da8559 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.h +++ b/src/Storages/System/StorageSystemStoragePolicies.h @@ -21,12 +21,13 @@ public: std::string getName() const override { return "SystemStoragePolicies"; } Pipes read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams) override; + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; protected: StorageSystemStoragePolicies(const std::string & name_); diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 84d441a8c6e..f04b3ea20c9 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -448,6 +448,7 @@ private: Pipes StorageSystemTables::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, diff --git a/src/Storages/System/StorageSystemTables.h b/src/Storages/System/StorageSystemTables.h index bab3aef6e15..54551205684 100644 --- a/src/Storages/System/StorageSystemTables.h +++ b/src/Storages/System/StorageSystemTables.h @@ -20,6 +20,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & /*metadata_*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index 438d31e7e02..2bc53b5093e 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -91,12 +91,13 @@ StorageSystemZeros::StorageSystemZeros(const StorageID & table_id_, bool multith } Pipes StorageSystemZeros::read( - const Names & column_names, - const SelectQueryInfo &, - const Context & /*context*/, - QueryProcessingStage::Enum /*processed_stage*/, - size_t max_block_size, - unsigned num_streams) + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, + const SelectQueryInfo &, + const Context & /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + unsigned num_streams) { check(column_names); diff --git a/src/Storages/System/StorageSystemZeros.h b/src/Storages/System/StorageSystemZeros.h index 3768885d03d..f169861122a 100644 --- a/src/Storages/System/StorageSystemZeros.h +++ b/src/Storages/System/StorageSystemZeros.h @@ -21,12 +21,13 @@ public: std::string getName() const override { return "SystemZeros"; } Pipes read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams) override; + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; bool hasEvenlyDistributedRead() const override { return true; } diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index fff352210e7..618d524987b 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -71,6 +71,7 @@ TYPED_TEST_SUITE(StorageLogTest, DiskImplementations); std::string writeData(int rows, DB::StoragePtr & table, const DB::Context & context) { using namespace DB; + auto metadata_snapshot = table->getInMemoryMetadataPtr(); std::string data; @@ -97,7 +98,7 @@ std::string writeData(int rows, DB::StoragePtr & table, const DB::Context & cont block.insert(column); } - BlockOutputStreamPtr out = table->write({}, context); + BlockOutputStreamPtr out = table->write({}, metadata_snapshot, context); out->write(block); return data; @@ -107,13 +108,14 @@ std::string writeData(int rows, DB::StoragePtr & table, const DB::Context & cont std::string readData(DB::StoragePtr & table, const DB::Context & context) { using namespace DB; + auto metadata_snapshot = table->getInMemoryMetadataPtr(); Names column_names; column_names.push_back("a"); QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context); - BlockInputStreamPtr in = std::make_shared(std::move(table->read(column_names, {}, context, stage, 8192, 1)[0])); + BlockInputStreamPtr in = std::make_shared(std::move(table->read(column_names, metadata_snapshot, {}, context, stage, 8192, 1)[0])); Block sample; { From 0f286ac133fa360b997d3edbc2891b016c6134c5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Jun 2020 15:03:27 +0300 Subject: [PATCH 063/211] Copy some methods to metdata --- src/Storages/StorageInMemoryMetadata.cpp | 121 +++++++++++++++++++++++ src/Storages/StorageInMemoryMetadata.h | 31 +++++- 2 files changed, 151 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index b6dd2f38c4e..bf747fb9b5a 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -89,4 +89,125 @@ void StorageInMemoryMetadata::setSelectQuery(const SelectQueryDescription & sele select = select_; } +const ColumnsDescription & StorageInMemoryMetadata::getColumns() const +{ + return columns; +} + +const IndicesDescription & StorageInMemoryMetadata::getSecondaryIndices() const +{ + return secondary_indices; +} + +bool StorageInMemoryMetadata::hasSecondaryIndices() const +{ + return !secondary_indices.empty(); +} + +const ConstraintsDescription & StorageInMemoryMetadata::getConstraints() const +{ + return constraints; +} + +TTLTableDescription StorageInMemoryMetadata::getTableTTLs() const +{ + return table_ttl; +} + +bool StorageInMemoryMetadata::hasAnyTableTTL() const +{ + return hasAnyMoveTTL() || hasRowsTTL(); +} + +TTLColumnsDescription StorageInMemoryMetadata::getColumnTTLs() const +{ + return column_ttls_by_name; +} + +bool StorageInMemoryMetadata::hasAnyColumnTTL() const +{ + return !column_ttls_by_name.empty(); +} + +TTLDescription StorageInMemoryMetadata::getRowsTTL() const +{ + return table_ttl.rows_ttl; +} + +bool StorageInMemoryMetadata::hasRowsTTL() const +{ + return table_ttl.rows_ttl.expression != nullptr; +} + +TTLDescriptions StorageInMemoryMetadata::getMoveTTLs() const +{ + return table_ttl.move_ttl; +} + +bool StorageInMemoryMetadata::hasAnyMoveTTL() const +{ + return !table_ttl.move_ttl.empty(); +} + +ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet & updated_columns) const +{ + if (updated_columns.empty()) + return {}; + + ColumnDependencies res; + + NameSet indices_columns; + NameSet required_ttl_columns; + NameSet updated_ttl_columns; + + auto add_dependent_columns = [&updated_columns](const auto & expression, auto & to_set) + { + auto requiered_columns = expression->getRequiredColumns(); + for (const auto & dependency : requiered_columns) + { + if (updated_columns.count(dependency)) + { + to_set.insert(requiered_columns.begin(), requiered_columns.end()); + return true; + } + } + + return false; + }; + + for (const auto & index : getSecondaryIndices()) + add_dependent_columns(index.expression, indices_columns); + + if (hasRowsTTL()) + { + auto rows_expression = getRowsTTL().expression; + if (add_dependent_columns(rows_expression, required_ttl_columns)) + { + /// Filter all columns, if rows TTL expression have to be recalculated. + for (const auto & column : getColumns().getAllPhysical()) + updated_ttl_columns.insert(column.name); + } + } + + for (const auto & [name, entry] : getColumnTTLs()) + { + if (add_dependent_columns(entry.expression, required_ttl_columns)) + updated_ttl_columns.insert(name); + } + + for (const auto & entry : getMoveTTLs()) + add_dependent_columns(entry.expression, required_ttl_columns); + + for (const auto & column : indices_columns) + res.emplace(column, ColumnDependency::SKIP_INDEX); + for (const auto & column : required_ttl_columns) + res.emplace(column, ColumnDependency::TTL_EXPRESSION); + for (const auto & column : updated_ttl_columns) + res.emplace(column, ColumnDependency::TTL_TARGET); + + return res; + +} + + } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index b129cdc7756..fb7bcbaa349 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -1,12 +1,13 @@ #pragma once #include +#include #include #include #include #include -#include #include +#include #include @@ -77,6 +78,34 @@ struct StorageInMemoryMetadata void setSettingsChanges(const ASTPtr & settings_changes_); void setSelectQuery(const SelectQueryDescription & select_); + + const ColumnsDescription & getColumns() const; /// returns combined set of columns + const IndicesDescription & getSecondaryIndices() const; + /// Has at least one non primary index + bool hasSecondaryIndices() const; + + const ConstraintsDescription & getConstraints() const; + + /// Common tables TTLs (for rows and moves). + TTLTableDescription getTableTTLs() const; + bool hasAnyTableTTL() const; + + /// Separate TTLs for columns. + TTLColumnsDescription getColumnTTLs() const; + bool hasAnyColumnTTL() const; + + /// Just wrapper for table TTLs, return rows part of table TTLs. + TTLDescription getRowsTTL() const; + bool hasRowsTTL() const; + + /// Just wrapper for table TTLs, return moves (to disks or volumes) parts of + /// table TTL. + TTLDescriptions getMoveTTLs() const; + bool hasAnyMoveTTL() const; + + /// Returns columns, which will be needed to calculate dependencies (skip + /// indices, TTL expressions) if we update @updated_columns set of columns. + ColumnDependencies getColumnDependencies(const NameSet & updated_columns) const; }; using StorageMetadataPtr = std::shared_ptr; From 0bcd22008a2e42c79c2ff8724b065a854e335335 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Jun 2020 15:19:21 +0300 Subject: [PATCH 064/211] Get column dependencies in StorageInMemoryMetadata --- src/Interpreters/MutationsInterpreter.cpp | 11 ++-- src/Interpreters/MutationsInterpreter.h | 1 + src/Storages/IStorage.cpp | 61 ------------------- src/Storages/IStorage.h | 4 -- .../MergeTree/MergeTreeDataMergerMutator.cpp | 17 ++++-- .../MergeTree/MergeTreeDataMergerMutator.h | 12 +++- src/Storages/StorageMergeTree.cpp | 7 ++- src/Storages/StorageReplicatedMergeTree.cpp | 10 ++- 8 files changed, 41 insertions(+), 82 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 123de1b6e84..ce47ce6e476 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -137,13 +137,13 @@ ASTPtr prepareQueryAffectedAST(const std::vector & commands) return select; } -ColumnDependencies getAllColumnDependencies(const StoragePtr & storage, const NameSet & updated_columns) +ColumnDependencies getAllColumnDependencies(const StorageMetadataPtr & metadata_snapshot, const NameSet & updated_columns) { NameSet new_updated_columns = updated_columns; ColumnDependencies dependencies; while (!new_updated_columns.empty()) { - auto new_dependencies = storage->getColumnDependencies(new_updated_columns); + auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns); new_updated_columns.clear(); for (const auto & dependency : new_dependencies) { @@ -204,6 +204,7 @@ MutationsInterpreter::MutationsInterpreter( const Context & context_, bool can_execute_) : storage(std::move(storage_)) + , metadata_snapshot(storage->getInMemoryMetadataPtr()) , commands(std::move(commands_)) , context(context_) , can_execute(can_execute_) @@ -329,7 +330,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } /// Columns, that we need to read for calculation of skip indices or TTL expressions. - auto dependencies = getAllColumnDependencies(storage, updated_columns); + auto dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns); /// First, break a sequence of commands into stages. for (const auto & command : commands) @@ -423,7 +424,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } auto all_columns_vec = all_columns.getNames(); - auto all_dependencies = getAllColumnDependencies(storage, NameSet(all_columns_vec.begin(), all_columns_vec.end())); + auto all_dependencies = getAllColumnDependencies(metadata_snapshot, NameSet(all_columns_vec.begin(), all_columns_vec.end())); for (const auto & dependency : all_dependencies) { @@ -432,7 +433,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } /// Recalc only skip indices of columns, that could be updated by TTL. - auto new_dependencies = storage->getColumnDependencies(new_updated_columns); + auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns); for (const auto & dependency : new_dependencies) { if (dependency.kind == ColumnDependency::SKIP_INDEX) diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 35c4f8ece0a..158ed8d55af 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -47,6 +47,7 @@ private: std::optional getStorageSortDescriptionIfPossible(const Block & header) const; StoragePtr storage; + StorageMetadataPtr metadata_snapshot; MutationCommands commands; Context context; bool can_execute; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index e5ab14e046e..6dae96a3322 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -548,67 +548,6 @@ bool IStorage::hasAnyMoveTTL() const return !metadata->table_ttl.move_ttl.empty(); } - -ColumnDependencies IStorage::getColumnDependencies(const NameSet & updated_columns) const -{ - if (updated_columns.empty()) - return {}; - - ColumnDependencies res; - - NameSet indices_columns; - NameSet required_ttl_columns; - NameSet updated_ttl_columns; - - auto add_dependent_columns = [&updated_columns](const auto & expression, auto & to_set) - { - auto requiered_columns = expression->getRequiredColumns(); - for (const auto & dependency : requiered_columns) - { - if (updated_columns.count(dependency)) - { - to_set.insert(requiered_columns.begin(), requiered_columns.end()); - return true; - } - } - - return false; - }; - - for (const auto & index : getSecondaryIndices()) - add_dependent_columns(index.expression, indices_columns); - - if (hasRowsTTL()) - { - auto rows_expression = getRowsTTL().expression; - if (add_dependent_columns(rows_expression, required_ttl_columns)) - { - /// Filter all columns, if rows TTL expression have to be recalculated. - for (const auto & column : getColumns().getAllPhysical()) - updated_ttl_columns.insert(column.name); - } - } - - for (const auto & [name, entry] : getColumnTTLs()) - { - if (add_dependent_columns(entry.expression, required_ttl_columns)) - updated_ttl_columns.insert(name); - } - - for (const auto & entry : getMoveTTLs()) - add_dependent_columns(entry.expression, required_ttl_columns); - - for (const auto & column : indices_columns) - res.emplace(column, ColumnDependency::SKIP_INDEX); - for (const auto & column : required_ttl_columns) - res.emplace(column, ColumnDependency::TTL_EXPRESSION); - for (const auto & column : updated_ttl_columns) - res.emplace(column, ColumnDependency::TTL_TARGET); - - return res; - -} - ASTPtr IStorage::getSettingsChanges() const { if (metadata->settings_changes) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 28ad7b0ea8b..d3e65b6a845 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -492,10 +492,6 @@ public: /// Returns column names that need to be read for FINAL to work. Names getColumnsRequiredForFinal() const { return getColumnsRequiredForSortingKey(); } - /// Returns columns, which will be needed to calculate dependencies (skip - /// indices, TTL expressions) if we update @updated_columns set of columns. - ColumnDependencies getColumnDependencies(const NameSet & updated_columns) const; - /// Returns storage policy if storage supports it. virtual StoragePolicyPtr getStoragePolicy() const { return {}; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index d861173d8a0..595370e7ecc 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -578,8 +578,14 @@ public: /// parts should be sorted. MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( - const FutureMergedMutatedPart & future_part, MergeList::Entry & merge_entry, TableStructureReadLockHolder &, - time_t time_of_merge, const ReservationPtr & space_reservation, bool deduplicate, bool force_ttl) + const FutureMergedMutatedPart & future_part, + const StorageMetadataPtr & /*metadata_snapshot*/, + MergeList::Entry & merge_entry, + TableStructureReadLockHolder &, + time_t time_of_merge, + const ReservationPtr & space_reservation, + bool deduplicate, + bool force_ttl) { static const String TMP_PREFIX = "tmp_merge_"; @@ -975,6 +981,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTemporaryPart( const FutureMergedMutatedPart & future_part, + const StorageMetadataPtr & metadata_snapshot, const MutationCommands & commands, MergeListEntry & merge_entry, time_t time_of_mutation, @@ -1069,7 +1076,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor bool need_remove_expired_values = false; - if (in && shouldExecuteTTL(in->getHeader().getNamesAndTypesList().getNames(), commands_for_part)) + if (in && shouldExecuteTTL(metadata_snapshot, in->getHeader().getNamesAndTypesList().getNames(), commands_for_part)) need_remove_expired_values = true; /// All columns from part are changed and may be some more that were missing before in part @@ -1556,7 +1563,7 @@ std::set MergeTreeDataMergerMutator::getIndicesToRecalculate( return indices_to_recalc; } -bool MergeTreeDataMergerMutator::shouldExecuteTTL(const Names & columns, const MutationCommands & commands) const +bool MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const Names & columns, const MutationCommands & commands) const { if (!data.hasAnyTTL()) return false; @@ -1565,7 +1572,7 @@ bool MergeTreeDataMergerMutator::shouldExecuteTTL(const Names & columns, const M if (command.type == MutationCommand::MATERIALIZE_TTL) return true; - auto dependencies = data.getColumnDependencies(NameSet(columns.begin(), columns.end())); + auto dependencies = metadata_snapshot->getColumnDependencies(NameSet(columns.begin(), columns.end())); for (const auto & dependency : dependencies) if (dependency.kind == ColumnDependency::TTL_EXPRESSION || dependency.kind == ColumnDependency::TTL_TARGET) return true; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 7c2ee53fc1d..185961972a8 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -105,12 +105,18 @@ public: */ MergeTreeData::MutableDataPartPtr mergePartsToTemporaryPart( const FutureMergedMutatedPart & future_part, - MergeListEntry & merge_entry, TableStructureReadLockHolder & table_lock_holder, time_t time_of_merge, - const ReservationPtr & space_reservation, bool deduplicate, bool force_ttl); + const StorageMetadataPtr & metadata_snapshot, + MergeListEntry & merge_entry, + TableStructureReadLockHolder & table_lock_holder, + time_t time_of_merge, + const ReservationPtr & space_reservation, + bool deduplicate, + bool force_ttl); /// Mutate a single data part with the specified commands. Will create and return a temporary part. MergeTreeData::MutableDataPartPtr mutatePartToTemporaryPart( const FutureMergedMutatedPart & future_part, + const StorageMetadataPtr & metadata_snapshot, const MutationCommands & commands, MergeListEntry & merge_entry, time_t time_of_mutation, @@ -164,7 +170,7 @@ private: const IndicesDescription & all_indices, const MutationCommands & commands_for_removes); - bool shouldExecuteTTL(const Names & columns, const MutationCommands & commands) const; + bool shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const Names & columns, const MutationCommands & commands) const; /// Return set of indices which should be recalculated during mutation also /// wraps input stream into additional expression stream diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index e3f48a05d6e..1b00487c816 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -591,6 +591,7 @@ bool StorageMergeTree::merge( { auto table_lock_holder = lockStructureForShare( true, RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto metadata_snapshot = getInMemoryMetadataPtr(); FutureMergedMutatedPart future_part; @@ -693,7 +694,7 @@ bool StorageMergeTree::merge( bool force_ttl = (final && hasAnyTTL()); new_part = merger_mutator.mergePartsToTemporaryPart( - future_part, *merge_entry, table_lock_holder, time(nullptr), + future_part, metadata_snapshot, *merge_entry, table_lock_holder, time(nullptr), merging_tagger->reserved_space, deduplicate, force_ttl); merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); @@ -739,6 +740,7 @@ bool StorageMergeTree::tryMutatePart() { auto table_lock_holder = lockStructureForShare( true, RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + StorageMetadataPtr metadata_snapshot = getInMemoryMetadataPtr(); size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements; FutureMergedMutatedPart future_part; @@ -832,7 +834,8 @@ bool StorageMergeTree::tryMutatePart() try { - new_part = merger_mutator.mutatePartToTemporaryPart(future_part, commands, *merge_entry, + new_part = merger_mutator.mutatePartToTemporaryPart( + future_part, metadata_snapshot, commands, *merge_entry, time(nullptr), global_context, tagger->reserved_space, table_lock_holder); renameTempPartAndReplace(new_part); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7abf90d3eac..810a4fa5c97 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1304,6 +1304,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) auto table_lock = lockStructureForShare( false, RWLockImpl::NO_QUERY, storage_settings_ptr->lock_acquire_timeout_for_background_operations); + StorageMetadataPtr metadata_snapshot = getInMemoryMetadataPtr(); FutureMergedMutatedPart future_merged_part(parts, entry.new_part_type); if (future_merged_part.name != entry.new_part_name) @@ -1331,7 +1332,9 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) try { part = merger_mutator.mergePartsToTemporaryPart( - future_merged_part, *merge_entry, table_lock, entry.create_time, reserved_space, entry.deduplicate, entry.force_ttl); + future_merged_part, metadata_snapshot, *merge_entry, + table_lock, entry.create_time, reserved_space, entry.deduplicate, + entry.force_ttl); merger_mutator.renameMergedTemporaryPart(part, parts, &transaction); @@ -1428,6 +1431,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM auto table_lock = lockStructureForShare( false, RWLockImpl::NO_QUERY, storage_settings_ptr->lock_acquire_timeout_for_background_operations); + StorageMetadataPtr metadata_snapshot = getInMemoryMetadataPtr(); MutableDataPartPtr new_part; Transaction transaction(*this); @@ -1454,7 +1458,9 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM try { - new_part = merger_mutator.mutatePartToTemporaryPart(future_mutated_part, commands, *merge_entry, entry.create_time, global_context, reserved_space, table_lock); + new_part = merger_mutator.mutatePartToTemporaryPart( + future_mutated_part, metadata_snapshot, commands, *merge_entry, + entry.create_time, global_context, reserved_space, table_lock); renameTempPartAndReplace(new_part, nullptr, &transaction); try From 53cb5210debb5baa10d521d90bd6afb7988245e2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Jun 2020 15:48:10 +0300 Subject: [PATCH 065/211] Move getSampleBlockNonMaterialized to StorageInMemoryMetadata --- src/Interpreters/InterpreterInsertQuery.cpp | 9 ++++++--- src/Interpreters/InterpreterInsertQuery.h | 3 ++- src/Interpreters/SystemLog.h | 3 ++- src/Storages/IStorage.cpp | 9 --------- src/Storages/IStorage.h | 1 - src/Storages/Kafka/KafkaBlockInputStream.cpp | 13 ++++++++++--- src/Storages/Kafka/KafkaBlockInputStream.h | 8 +++++++- src/Storages/Kafka/KafkaBlockOutputStream.cpp | 10 ++++++++-- src/Storages/Kafka/KafkaBlockOutputStream.h | 6 +++++- src/Storages/Kafka/StorageKafka.cpp | 11 ++++++----- src/Storages/StorageBuffer.cpp | 4 +++- src/Storages/StorageDistributed.cpp | 4 ++-- src/Storages/StorageInMemoryMetadata.cpp | 8 ++++++++ src/Storages/StorageInMemoryMetadata.h | 2 ++ 14 files changed, 61 insertions(+), 30 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 1841c82b710..d281dc5ccca 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -73,9 +73,12 @@ StoragePtr InterpreterInsertQuery::getTable(ASTInsertQuery & query) return DatabaseCatalog::instance().getTable(query.table_id, context); } -Block InterpreterInsertQuery::getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table) const +Block InterpreterInsertQuery::getSampleBlock( + const ASTInsertQuery & query, + const StoragePtr & table, + const StorageMetadataPtr & metadata_snapshot) const { - Block table_sample_non_materialized = table->getSampleBlockNonMaterialized(); + Block table_sample_non_materialized = metadata_snapshot->getSampleBlockNonMaterialized(); /// If the query does not include information about columns if (!query.columns) { @@ -119,7 +122,7 @@ BlockIO InterpreterInsertQuery::execute() true, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = table->getInMemoryMetadataPtr(); - auto query_sample_block = getSampleBlock(query, table); + auto query_sample_block = getSampleBlock(query, table, metadata_snapshot); if (!query.table_function) context.checkAccess(AccessType::INSERT, query.table_id, query_sample_block.getNames()); diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index fef962d24a3..3386b471d26 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -34,7 +35,7 @@ public: private: StoragePtr getTable(ASTInsertQuery & query); - Block getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table) const; + Block getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const; ASTPtr query_ptr; const Context & context; diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index e49ce574478..cf163226b93 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -438,8 +438,9 @@ void SystemLog::prepareTable() if (table) { + auto metadata_snapshot = table->getInMemoryMetadataPtr(); const Block expected = LogElement::createBlock(); - const Block actual = table->getSampleBlockNonMaterialized(); + const Block actual = metadata_snapshot->getSampleBlockNonMaterialized(); if (!blocksHaveEqualStructure(actual, expected)) { diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 6dae96a3322..e675d51b4b7 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -74,15 +74,6 @@ Block IStorage::getSampleBlockWithVirtuals() const return res; } -Block IStorage::getSampleBlockNonMaterialized() const -{ - Block res; - - for (const auto & column : getColumns().getOrdinary()) - res.insert({column.type->createColumn(), column.type, column.name}); - - return res; -} Block IStorage::getSampleBlockForColumns(const Names & column_names) const { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index d3e65b6a845..42581ebb63b 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -160,7 +160,6 @@ public: /// thread-unsafe part. lockStructure must be acquired Block getSampleBlock() const; /// ordinary + materialized. Block getSampleBlockWithVirtuals() const; /// ordinary + materialized + virtuals. - Block getSampleBlockNonMaterialized() const; /// ordinary. Block getSampleBlockForColumns(const Names & column_names) const; /// ordinary + materialized + aliases + virtuals. /// Verify that all the requested names are in the table and are set correctly: diff --git a/src/Storages/Kafka/KafkaBlockInputStream.cpp b/src/Storages/Kafka/KafkaBlockInputStream.cpp index 3edfcc7b9d2..dd2bb68c11a 100644 --- a/src/Storages/Kafka/KafkaBlockInputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockInputStream.cpp @@ -13,14 +13,21 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } KafkaBlockInputStream::KafkaBlockInputStream( - StorageKafka & storage_, const std::shared_ptr & context_, const Names & columns, size_t max_block_size_, bool commit_in_suffix_) + StorageKafka & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const std::shared_ptr & context_, + const Names & columns, + size_t max_block_size_, + bool commit_in_suffix_) : storage(storage_) + , metadata_snapshot(metadata_snapshot_) , context(context_) , column_names(columns) , max_block_size(max_block_size_) , commit_in_suffix(commit_in_suffix_) - , non_virtual_header(storage.getSampleBlockNonMaterialized()) - , virtual_header(storage.getSampleBlockForColumns({"_topic", "_key", "_offset", "_partition", "_timestamp","_timestamp_ms","_headers.name","_headers.value"})) + , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) + , virtual_header(storage.getSampleBlockForColumns( + {"_topic", "_key", "_offset", "_partition", "_timestamp", "_timestamp_ms", "_headers.name", "_headers.value"})) { } diff --git a/src/Storages/Kafka/KafkaBlockInputStream.h b/src/Storages/Kafka/KafkaBlockInputStream.h index 387f5088721..4851050a56e 100644 --- a/src/Storages/Kafka/KafkaBlockInputStream.h +++ b/src/Storages/Kafka/KafkaBlockInputStream.h @@ -14,7 +14,12 @@ class KafkaBlockInputStream : public IBlockInputStream { public: KafkaBlockInputStream( - StorageKafka & storage_, const std::shared_ptr & context_, const Names & columns, size_t max_block_size_, bool commit_in_suffix = true); + StorageKafka & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const std::shared_ptr & context_, + const Names & columns, + size_t max_block_size_, + bool commit_in_suffix = true); ~KafkaBlockInputStream() override; String getName() const override { return storage.getName(); } @@ -29,6 +34,7 @@ public: private: StorageKafka & storage; + StorageMetadataPtr metadata_snapshot; const std::shared_ptr context; Names column_names; UInt64 max_block_size; diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.cpp b/src/Storages/Kafka/KafkaBlockOutputStream.cpp index 17ef5aa104c..60ac714bd52 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockOutputStream.cpp @@ -11,13 +11,19 @@ namespace ErrorCodes extern const int CANNOT_CREATE_IO_BUFFER; } -KafkaBlockOutputStream::KafkaBlockOutputStream(StorageKafka & storage_, const std::shared_ptr & context_) : storage(storage_), context(context_) +KafkaBlockOutputStream::KafkaBlockOutputStream( + StorageKafka & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const std::shared_ptr & context_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , context(context_) { } Block KafkaBlockOutputStream::getHeader() const { - return storage.getSampleBlockNonMaterialized(); + return metadata_snapshot->getSampleBlockNonMaterialized(); } void KafkaBlockOutputStream::writePrefix() diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.h b/src/Storages/Kafka/KafkaBlockOutputStream.h index 7a973724f1b..1121d2a119e 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.h +++ b/src/Storages/Kafka/KafkaBlockOutputStream.h @@ -10,7 +10,10 @@ namespace DB class KafkaBlockOutputStream : public IBlockOutputStream { public: - explicit KafkaBlockOutputStream(StorageKafka & storage_, const std::shared_ptr & context_); + explicit KafkaBlockOutputStream( + StorageKafka & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const std::shared_ptr & context_); Block getHeader() const override; @@ -22,6 +25,7 @@ public: private: StorageKafka & storage; + StorageMetadataPtr metadata_snapshot; const std::shared_ptr context; ProducerBufferPtr buffer; BlockOutputStreamPtr child; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 190397bc675..b46cf0579ec 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -201,7 +201,7 @@ String StorageKafka::getDefaultClientId(const StorageID & table_id_) Pipes StorageKafka::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /* query_info */, const Context & context, QueryProcessingStage::Enum /* processed_stage */, @@ -224,7 +224,7 @@ Pipes StorageKafka::read( /// TODO: probably that leads to awful performance. /// FIXME: seems that doesn't help with extra reading and committing unprocessed messages. /// TODO: rewrite KafkaBlockInputStream to KafkaSource. Now it is used in other place. - pipes.emplace_back(std::make_shared(std::make_shared(*this, modified_context, column_names, 1))); + pipes.emplace_back(std::make_shared(std::make_shared(*this, metadata_snapshot, modified_context, column_names, 1))); } LOG_DEBUG(log, "Starting reading {} streams", pipes.size()); @@ -232,14 +232,14 @@ Pipes StorageKafka::read( } -BlockOutputStreamPtr StorageKafka::write(const ASTPtr &, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) +BlockOutputStreamPtr StorageKafka::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context & context) { auto modified_context = std::make_shared(context); modified_context->applySettingsChanges(settings_adjustments); if (topics.size() > 1) throw Exception("Can't write to Kafka table with multiple topics!", ErrorCodes::NOT_IMPLEMENTED); - return std::make_shared(*this, modified_context); + return std::make_shared(*this, metadata_snapshot, modified_context); } @@ -519,6 +519,7 @@ bool StorageKafka::streamToViews() auto table = DatabaseCatalog::instance().getTable(table_id, global_context); if (!table) throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); + auto metadata_snapshot = getInMemoryMetadataPtr(); // Create an INSERT query for streaming data auto insert = std::make_shared(); @@ -538,7 +539,7 @@ bool StorageKafka::streamToViews() for (size_t i = 0; i < num_created_consumers; ++i) { auto stream - = std::make_shared(*this, kafka_context, block_io.out->getHeader().getNames(), block_size, false); + = std::make_shared(*this, metadata_snapshot, kafka_context, block_io.out->getHeader().getNames(), block_size, false); streams.emplace_back(stream); // Limit read batch to maximum block size to allow DDL diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 3e419921115..4754732159c 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -642,6 +642,7 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl LOG_ERROR(log, "Destination table {} doesn't exist. Block of data is discarded.", destination_id.getNameForLogs()); return; } + auto destination_metadata_snapshot = table->getInMemoryMetadataPtr(); auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock(); @@ -651,7 +652,8 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl /** We will insert columns that are the intersection set of columns of the buffer table and the subordinate table. * This will support some of the cases (but not all) when the table structure does not match. */ - Block structure_of_destination_table = allow_materialized ? table->getSampleBlock() : table->getSampleBlockNonMaterialized(); + Block structure_of_destination_table + = allow_materialized ? table->getSampleBlock() : destination_metadata_snapshot->getSampleBlockNonMaterialized(); Block block_to_write; for (size_t i : ext::range(0, structure_of_destination_table.columns())) { diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 719811bbc6b..66066ec3c18 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -511,7 +511,7 @@ Pipes StorageDistributed::read( } -BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) +BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context & context) { auto cluster = getCluster(); const auto & settings = context.getSettingsRef(); @@ -536,7 +536,7 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const StorageMeta /// DistributedBlockOutputStream will not own cluster, but will own ConnectionPools of the cluster return std::make_shared( - context, *this, createInsertToRemoteTableQuery(remote_database, remote_table, getSampleBlockNonMaterialized()), cluster, + context, *this, createInsertToRemoteTableQuery(remote_database, remote_table, metadata_snapshot->getSampleBlockNonMaterialized()), cluster, insert_sync, timeout); } diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index bf747fb9b5a..2c5b6279e10 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -209,5 +209,13 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet } +Block StorageInMemoryMetadata::getSampleBlockNonMaterialized() const +{ + Block res; + for (const auto & column : getColumns().getOrdinary()) + res.insert({column.type->createColumn(), column.type, column.name}); + + return res; +} } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index fb7bcbaa349..d6c00bb35c8 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -106,6 +106,8 @@ struct StorageInMemoryMetadata /// Returns columns, which will be needed to calculate dependencies (skip /// indices, TTL expressions) if we update @updated_columns set of columns. ColumnDependencies getColumnDependencies(const NameSet & updated_columns) const; + + Block getSampleBlockNonMaterialized() const; /// ordinary. }; using StorageMetadataPtr = std::shared_ptr; From 08b9aa6b2ed0b39e542e0077efea231374a1ba32 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Jun 2020 15:58:05 +0300 Subject: [PATCH 066/211] getSampleBlockWithVirtuals in StorageInMemoryMetadata --- .../PushingToViewsBlockOutputStream.cpp | 21 +++++++++++++----- .../PushingToViewsBlockOutputStream.h | 9 ++++++-- src/Interpreters/InterpreterInsertQuery.cpp | 4 ++-- src/Storages/IStorage.cpp | 13 ----------- src/Storages/IStorage.h | 1 - src/Storages/StorageInMemoryMetadata.cpp | 22 +++++++++++++++++++ src/Storages/StorageInMemoryMetadata.h | 2 ++ 7 files changed, 48 insertions(+), 24 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index fa213b054df..2e02c26d38c 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -19,8 +19,14 @@ namespace DB PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( const StoragePtr & storage_, - const Context & context_, const ASTPtr & query_ptr_, bool no_destination) - : storage(storage_), context(context_), query_ptr(query_ptr_) + const StorageMetadataPtr & metadata_snapshot_, + const Context & context_, + const ASTPtr & query_ptr_, + bool no_destination) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , context(context_) + , query_ptr(query_ptr_) { /** TODO This is a very important line. At any insertion into the table one of streams should own lock. * Although now any insertion into the table is done via PushingToViewsBlockOutputStream, @@ -60,6 +66,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( for (const auto & database_table : dependencies) { auto dependent_table = DatabaseCatalog::instance().getTable(database_table, context); + auto dependent_metadata_snapshot = dependent_table->getInMemoryMetadataPtr(); ASTPtr query; BlockOutputStreamPtr out; @@ -97,9 +104,11 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( out = io.out; } else if (dynamic_cast(dependent_table.get())) - out = std::make_shared(dependent_table, *insert_context, ASTPtr(), true); + out = std::make_shared( + dependent_table, dependent_metadata_snapshot, *insert_context, ASTPtr(), true); else - out = std::make_shared(dependent_table, *insert_context, ASTPtr()); + out = std::make_shared( + dependent_table, dependent_metadata_snapshot, *insert_context, ASTPtr()); views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out), nullptr}); } @@ -118,9 +127,9 @@ Block PushingToViewsBlockOutputStream::getHeader() const /// If we don't write directly to the destination /// then expect that we're inserting with precalculated virtual columns if (output) - return storage->getSampleBlock(); + return metadata_snapshot->getSampleBlock(); else - return storage->getSampleBlockWithVirtuals(); + return metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals()); } diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index c5fef413a23..ca09126a561 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -17,8 +17,12 @@ class ReplicatedMergeTreeBlockOutputStream; class PushingToViewsBlockOutputStream : public IBlockOutputStream { public: - PushingToViewsBlockOutputStream(const StoragePtr & storage_, - const Context & context_, const ASTPtr & query_ptr_, bool no_destination = false); + PushingToViewsBlockOutputStream( + const StoragePtr & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const Context & context_, + const ASTPtr & query_ptr_, + bool no_destination = false); Block getHeader() const override; void write(const Block & block) override; @@ -29,6 +33,7 @@ public: private: StoragePtr storage; + StorageMetadataPtr metadata_snapshot; BlockOutputStreamPtr output; ReplicatedMergeTreeBlockOutputStream * replicated_output = nullptr; diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index d281dc5ccca..f61ef0e7381 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -83,7 +83,7 @@ Block InterpreterInsertQuery::getSampleBlock( if (!query.columns) { if (no_destination) - return table->getSampleBlockWithVirtuals(); + return metadata_snapshot->getSampleBlockWithVirtuals(table->getVirtuals()); else return table_sample_non_materialized; } @@ -232,7 +232,7 @@ BlockIO InterpreterInsertQuery::execute() if (table->noPushingToViews() && !no_destination) out = table->write(query_ptr, metadata_snapshot, context); else - out = std::make_shared(table, context, query_ptr, no_destination); + out = std::make_shared(table, metadata_snapshot, context, query_ptr, no_destination); /// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order. diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index e675d51b4b7..fd012c3cd75 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -62,19 +62,6 @@ Block IStorage::getSampleBlock() const return res; } -Block IStorage::getSampleBlockWithVirtuals() const -{ - auto res = getSampleBlock(); - - /// Virtual columns must be appended after ordinary, because user can - /// override them. - for (const auto & column : getVirtuals()) - res.insert({column.type->createColumn(), column.type, column.name}); - - return res; -} - - Block IStorage::getSampleBlockForColumns(const Names & column_names) const { Block res; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 42581ebb63b..e7a7786c2d6 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -159,7 +159,6 @@ public: /// thread-unsafe part. lockStructure must be acquired void setInMemoryMetadata(const StorageInMemoryMetadata & metadata_) { metadata = std::make_shared(metadata_); } Block getSampleBlock() const; /// ordinary + materialized. - Block getSampleBlockWithVirtuals() const; /// ordinary + materialized + virtuals. Block getSampleBlockForColumns(const Names & column_names) const; /// ordinary + materialized + aliases + virtuals. /// Verify that all the requested names are in the table and are set correctly: diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 2c5b6279e10..f3719562af7 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -218,4 +218,26 @@ Block StorageInMemoryMetadata::getSampleBlockNonMaterialized() const return res; } + +Block StorageInMemoryMetadata::getSampleBlockWithVirtuals(const NamesAndTypesList & virtuals) const +{ + auto res = getSampleBlock(); + + /// Virtual columns must be appended after ordinary, because user can + /// override them. + for (const auto & column : virtuals) + res.insert({column.type->createColumn(), column.type, column.name}); + + return res; +} + +Block StorageInMemoryMetadata::getSampleBlock() const +{ + Block res; + + for (const auto & column : getColumns().getAllPhysical()) + res.insert({column.type->createColumn(), column.type, column.name}); + + return res; +} } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index d6c00bb35c8..2da766caacd 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -107,7 +107,9 @@ struct StorageInMemoryMetadata /// indices, TTL expressions) if we update @updated_columns set of columns. ColumnDependencies getColumnDependencies(const NameSet & updated_columns) const; + Block getSampleBlock() const; /// ordinary + materialized. Block getSampleBlockNonMaterialized() const; /// ordinary. + Block getSampleBlockWithVirtuals(const NamesAndTypesList & virtuals) const; /// ordinary + materialized + virtuals. }; using StorageMetadataPtr = std::shared_ptr; From 71f99a274dae57e78738159792e18ee3707a865c Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Jun 2020 17:25:08 +0300 Subject: [PATCH 067/211] Compileable getSampleBlockWithColumns in StorageInMemoryMetadata --- src/Interpreters/InterpreterSelectQuery.cpp | 6 +- src/Storages/IStorage.cpp | 31 -------- src/Storages/IStorage.h | 1 - src/Storages/Kafka/KafkaBlockInputStream.cpp | 6 +- .../MergeTreeBaseSelectProcessor.cpp | 21 +++--- .../MergeTree/MergeTreeBaseSelectProcessor.h | 2 + .../MergeTree/MergeTreeDataMergerMutator.cpp | 6 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 74 ++++++++++++++----- .../MergeTree/MergeTreeDataSelectExecutor.h | 5 ++ src/Storages/MergeTree/MergeTreeReadPool.cpp | 31 +++++--- src/Storages/MergeTree/MergeTreeReadPool.h | 3 +- .../MergeTreeReverseSelectProcessor.cpp | 5 +- .../MergeTreeReverseSelectProcessor.h | 1 + .../MergeTree/MergeTreeSelectProcessor.cpp | 5 +- .../MergeTree/MergeTreeSelectProcessor.h | 1 + .../MergeTree/MergeTreeSequentialSource.cpp | 6 +- .../MergeTree/MergeTreeSequentialSource.h | 5 +- ...rgeTreeThreadSelectBlockInputProcessor.cpp | 5 +- ...MergeTreeThreadSelectBlockInputProcessor.h | 1 + .../MergeTree/StorageFromMergeTreeDataPart.h | 6 +- src/Storages/StorageBuffer.cpp | 12 +-- src/Storages/StorageInMemoryMetadata.cpp | 50 +++++++++++++ src/Storages/StorageInMemoryMetadata.h | 2 + src/Storages/StorageJoin.cpp | 4 +- src/Storages/StorageMemory.cpp | 34 ++++++--- src/Storages/StorageMerge.cpp | 14 ++-- src/Storages/StorageMerge.h | 8 +- src/Storages/StorageMergeTree.cpp | 5 +- src/Storages/StorageNull.h | 9 ++- src/Storages/StorageReplicatedMergeTree.cpp | 6 +- src/Storages/StorageS3.cpp | 12 +-- src/Storages/StorageS3.h | 5 -- src/Storages/StorageStripeLog.cpp | 27 +++++-- src/Storages/StorageURL.cpp | 9 ++- src/Storages/StorageURL.h | 6 +- src/Storages/StorageView.cpp | 7 +- src/Storages/StorageXDBC.cpp | 19 +++-- src/Storages/StorageXDBC.h | 35 +++++---- 38 files changed, 309 insertions(+), 176 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 40d7ed9ecc9..f73245179ce 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -377,14 +377,14 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (storage) { - source_header = storage->getSampleBlockForColumns(required_columns); + source_header = metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals()); /// Fix source_header for filter actions. if (row_policy_filter) { filter_info = std::make_shared(); filter_info->column_name = generateFilterActions(filter_info->actions, row_policy_filter, required_columns); - source_header = storage->getSampleBlockForColumns(filter_info->actions->getRequiredColumns()); + source_header = metadata_snapshot->getSampleBlockForColumns(filter_info->actions->getRequiredColumns(), storage->getVirtuals()); } } @@ -1336,7 +1336,7 @@ void InterpreterSelectQuery::executeFetchColumns( if (pipes.empty()) { - Pipe pipe(std::make_shared(storage->getSampleBlockForColumns(required_columns))); + Pipe pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals()))); if (query_info.prewhere_info) { diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index fd012c3cd75..d090dc9e51d 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -62,37 +62,6 @@ Block IStorage::getSampleBlock() const return res; } -Block IStorage::getSampleBlockForColumns(const Names & column_names) const -{ - Block res; - - std::unordered_map columns_map; - - NamesAndTypesList all_columns = getColumns().getAll(); - for (const auto & elem : all_columns) - columns_map.emplace(elem.name, elem.type); - - /// Virtual columns must be appended after ordinary, because user can - /// override them. - for (const auto & column : getVirtuals()) - columns_map.emplace(column.name, column.type); - - for (const auto & name : column_names) - { - auto it = columns_map.find(name); - if (it != columns_map.end()) - { - res.insert({it->second->createColumn(), it->second, it->first}); - } - else - { - throw Exception( - "Column " + backQuote(name) + " not found in table " + getStorageID().getNameForLogs(), ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); - } - } - - return res; -} namespace { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index e7a7786c2d6..a4173c1c9fa 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -159,7 +159,6 @@ public: /// thread-unsafe part. lockStructure must be acquired void setInMemoryMetadata(const StorageInMemoryMetadata & metadata_) { metadata = std::make_shared(metadata_); } Block getSampleBlock() const; /// ordinary + materialized. - Block getSampleBlockForColumns(const Names & column_names) const; /// ordinary + materialized + aliases + virtuals. /// Verify that all the requested names are in the table and are set correctly: /// list of names is not empty and the names do not repeat. diff --git a/src/Storages/Kafka/KafkaBlockInputStream.cpp b/src/Storages/Kafka/KafkaBlockInputStream.cpp index dd2bb68c11a..847b0d915cd 100644 --- a/src/Storages/Kafka/KafkaBlockInputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockInputStream.cpp @@ -26,8 +26,8 @@ KafkaBlockInputStream::KafkaBlockInputStream( , max_block_size(max_block_size_) , commit_in_suffix(commit_in_suffix_) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) - , virtual_header(storage.getSampleBlockForColumns( - {"_topic", "_key", "_offset", "_partition", "_timestamp", "_timestamp_ms", "_headers.name", "_headers.value"})) + , virtual_header(metadata_snapshot->getSampleBlockForColumns( + {"_topic", "_key", "_offset", "_partition", "_timestamp", "_timestamp_ms", "_headers.name", "_headers.value"}, storage.getVirtuals())) { } @@ -44,7 +44,7 @@ KafkaBlockInputStream::~KafkaBlockInputStream() Block KafkaBlockInputStream::getHeader() const { - return storage.getSampleBlockForColumns(column_names); + return metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals()); } void KafkaBlockInputStream::readPrefixImpl() diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index a2a3ca3a6cf..ec24c9ad652 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( Block header, const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, const PrewhereInfoPtr & prewhere_info_, UInt64 max_block_size_rows_, UInt64 preferred_block_size_bytes_, @@ -27,16 +28,16 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( const MergeTreeReaderSettings & reader_settings_, bool use_uncompressed_cache_, const Names & virt_column_names_) -: - SourceWithProgress(getHeader(std::move(header), prewhere_info_, virt_column_names_)), - storage(storage_), - prewhere_info(prewhere_info_), - max_block_size_rows(max_block_size_rows_), - preferred_block_size_bytes(preferred_block_size_bytes_), - preferred_max_column_in_block_size_bytes(preferred_max_column_in_block_size_bytes_), - reader_settings(reader_settings_), - use_uncompressed_cache(use_uncompressed_cache_), - virt_column_names(virt_column_names_) + : SourceWithProgress(getHeader(std::move(header), prewhere_info_, virt_column_names_)) + , storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , prewhere_info(prewhere_info_) + , max_block_size_rows(max_block_size_rows_) + , preferred_block_size_bytes(preferred_block_size_bytes_) + , preferred_max_column_in_block_size_bytes(preferred_max_column_in_block_size_bytes_) + , reader_settings(reader_settings_) + , use_uncompressed_cache(use_uncompressed_cache_) + , virt_column_names(virt_column_names_) { header_without_virtual_columns = getPort().getHeader(); diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 8fe8296381a..00ef131ae45 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -22,6 +22,7 @@ public: MergeTreeBaseSelectProcessor( Block header, const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, const PrewhereInfoPtr & prewhere_info_, UInt64 max_block_size_rows_, UInt64 preferred_block_size_bytes_, @@ -54,6 +55,7 @@ protected: protected: const MergeTreeData & storage; + StorageMetadataPtr metadata_snapshot; PrewhereInfoPtr prewhere_info; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 595370e7ecc..829f7cac528 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -579,7 +579,7 @@ public: /// parts should be sorted. MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( const FutureMergedMutatedPart & future_part, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, MergeList::Entry & merge_entry, TableStructureReadLockHolder &, time_t time_of_merge, @@ -712,7 +712,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor for (const auto & part : parts) { auto input = std::make_unique( - data, part, merging_column_names, read_with_direct_io, true); + data, metadata_snapshot, part, merging_column_names, read_with_direct_io, true); input->setProgressCallback( MergeProgressCallback(merge_entry, watch_prev_elapsed, horizontal_stage_progress)); @@ -898,7 +898,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor for (size_t part_num = 0; part_num < parts.size(); ++part_num) { auto column_part_source = std::make_shared( - data, parts[part_num], column_names, read_with_direct_io, true); + data, metadata_snapshot, parts[part_num], column_names, read_with_direct_io, true); column_part_source->setProgressCallback( MergeProgressCallback(merge_entry, watch_prev_elapsed, column_progress)); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c1fc8184206..ac2f4851185 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -147,6 +147,7 @@ static RelativeSize convertAbsoluteSampleSizeToRelative(const ASTPtr & node, siz Pipes MergeTreeDataSelectExecutor::read( const Names & column_names_to_return, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, const UInt64 max_block_size, @@ -154,13 +155,15 @@ Pipes MergeTreeDataSelectExecutor::read( const PartitionIdToMaxBlock * max_block_numbers_to_read) const { return readFromParts( - data.getDataPartsVector(), column_names_to_return, query_info, context, - max_block_size, num_streams, max_block_numbers_to_read); + data.getDataPartsVector(), column_names_to_return, metadata_snapshot, + query_info, context, max_block_size, num_streams, + max_block_numbers_to_read); } Pipes MergeTreeDataSelectExecutor::readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names_to_return, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, const UInt64 max_block_size, @@ -205,7 +208,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( } } - NamesAndTypesList available_real_columns = data.getColumns().getAllPhysical(); + NamesAndTypesList available_real_columns = metadata_snapshot->getColumns().getAllPhysical(); /// If there are only virtual columns in the query, you must request at least one non-virtual one. if (real_column_names.empty()) @@ -629,6 +632,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( std::move(parts_with_ranges), num_streams, column_names_to_read, + metadata_snapshot, max_block_size, settings.use_uncompressed_cache, query_info, @@ -650,6 +654,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( std::move(parts_with_ranges), num_streams, column_names_to_read, + metadata_snapshot, max_block_size, settings.use_uncompressed_cache, query_info, @@ -665,6 +670,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( std::move(parts_with_ranges), num_streams, column_names_to_read, + metadata_snapshot, max_block_size, settings.use_uncompressed_cache, query_info, @@ -727,6 +733,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( RangesInDataParts && parts, size_t num_streams, const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, UInt64 max_block_size, bool use_uncompressed_cache, const SelectQueryInfo & query_info, @@ -783,8 +790,18 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts.size()); MergeTreeReadPoolPtr pool = std::make_shared( - num_streams, sum_marks, min_marks_for_concurrent_read, parts, data, query_info.prewhere_info, true, - column_names, MergeTreeReadPool::BackoffSettings(settings), settings.preferred_block_size_bytes, false); + num_streams, + sum_marks, + min_marks_for_concurrent_read, + parts, + data, + metadata_snapshot, + query_info.prewhere_info, + true, + column_names, + MergeTreeReadPool::BackoffSettings(settings), + settings.preferred_block_size_bytes, + false); /// Let's estimate total number of rows for progress bar. LOG_TRACE(log, "Reading approx. {} rows with {} streams", total_rows, num_streams); @@ -792,8 +809,9 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( for (size_t i = 0; i < num_streams; ++i) { auto source = std::make_shared( - i, pool, min_marks_for_concurrent_read, max_block_size, settings.preferred_block_size_bytes, - settings.preferred_max_column_in_block_size_bytes, data, use_uncompressed_cache, + i, pool, min_marks_for_concurrent_read, max_block_size, + settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, + data, metadata_snapshot, use_uncompressed_cache, query_info.prewhere_info, reader_settings, virt_columns); if (i == 0) @@ -812,7 +830,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( for (const auto & part : parts) { auto source = std::make_shared( - data, part.data_part, max_block_size, settings.preferred_block_size_bytes, + data, metadata_snapshot, part.data_part, max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, column_names, part.ranges, use_uncompressed_cache, query_info.prewhere_info, true, reader_settings, virt_columns, part.part_index_in_query); @@ -845,6 +863,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( RangesInDataParts && parts, size_t num_streams, const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, UInt64 max_block_size, bool use_uncompressed_cache, const SelectQueryInfo & query_info, @@ -1004,18 +1023,38 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( if (input_order_info->direction == 1) { pipes.emplace_back(std::make_shared( - data, part.data_part, max_block_size, settings.preferred_block_size_bytes, - settings.preferred_max_column_in_block_size_bytes, column_names, ranges_to_get_from_part, - use_uncompressed_cache, query_info.prewhere_info, true, reader_settings, - virt_columns, part.part_index_in_query)); + data, + metadata_snapshot, + part.data_part, + max_block_size, + settings.preferred_block_size_bytes, + settings.preferred_max_column_in_block_size_bytes, + column_names, + ranges_to_get_from_part, + use_uncompressed_cache, + query_info.prewhere_info, + true, + reader_settings, + virt_columns, + part.part_index_in_query)); } else { pipes.emplace_back(std::make_shared( - data, part.data_part, max_block_size, settings.preferred_block_size_bytes, - settings.preferred_max_column_in_block_size_bytes, column_names, ranges_to_get_from_part, - use_uncompressed_cache, query_info.prewhere_info, true, reader_settings, - virt_columns, part.part_index_in_query)); + data, + metadata_snapshot, + part.data_part, + max_block_size, + settings.preferred_block_size_bytes, + settings.preferred_max_column_in_block_size_bytes, + column_names, + ranges_to_get_from_part, + use_uncompressed_cache, + query_info.prewhere_info, + true, + reader_settings, + virt_columns, + part.part_index_in_query)); pipes.back().addSimpleTransform(std::make_shared(pipes.back().getHeader())); } @@ -1050,6 +1089,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( RangesInDataParts && parts, size_t num_streams, const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, UInt64 max_block_size, bool use_uncompressed_cache, const SelectQueryInfo & query_info, @@ -1088,7 +1128,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( for (const auto & part : parts) { auto source_processor = std::make_shared( - data, part.data_part, max_block_size, settings.preferred_block_size_bytes, + data, metadata_snapshot, part.data_part, max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, column_names, part.ranges, use_uncompressed_cache, query_info.prewhere_info, true, reader_settings, virt_columns, part.part_index_in_query); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 85d69ead181..7811eb53b71 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -26,6 +26,7 @@ public: Pipes read( const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, UInt64 max_block_size, @@ -35,6 +36,7 @@ public: Pipes readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, UInt64 max_block_size, @@ -50,6 +52,7 @@ private: RangesInDataParts && parts, size_t num_streams, const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, UInt64 max_block_size, bool use_uncompressed_cache, const SelectQueryInfo & query_info, @@ -62,6 +65,7 @@ private: RangesInDataParts && parts, size_t num_streams, const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, UInt64 max_block_size, bool use_uncompressed_cache, const SelectQueryInfo & query_info, @@ -75,6 +79,7 @@ private: RangesInDataParts && parts, size_t num_streams, const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, UInt64 max_block_size, bool use_uncompressed_cache, const SelectQueryInfo & query_info, diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 9ca1446ef64..eb0b51235ad 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -17,17 +17,28 @@ namespace ErrorCodes namespace DB { - - MergeTreeReadPool::MergeTreeReadPool( - const size_t threads_, const size_t sum_marks_, const size_t min_marks_for_concurrent_read_, - RangesInDataParts parts_, const MergeTreeData & data_, const PrewhereInfoPtr & prewhere_info_, - const bool check_columns_, const Names & column_names_, - const BackoffSettings & backoff_settings_, size_t preferred_block_size_bytes_, + const size_t threads_, + const size_t sum_marks_, + const size_t min_marks_for_concurrent_read_, + RangesInDataParts parts_, + const MergeTreeData & data_, + const StorageMetadataPtr & metadata_snapshot_, + const PrewhereInfoPtr & prewhere_info_, + const bool check_columns_, + const Names & column_names_, + const BackoffSettings & backoff_settings_, + size_t preferred_block_size_bytes_, const bool do_not_steal_tasks_) - : backoff_settings{backoff_settings_}, backoff_state{threads_}, data{data_}, - column_names{column_names_}, do_not_steal_tasks{do_not_steal_tasks_}, - predict_block_size_bytes{preferred_block_size_bytes_ > 0}, prewhere_info{prewhere_info_}, parts_ranges{parts_} + : backoff_settings{backoff_settings_} + , backoff_state{threads_} + , data{data_} + , metadata_snapshot{metadata_snapshot_} + , column_names{column_names_} + , do_not_steal_tasks{do_not_steal_tasks_} + , predict_block_size_bytes{preferred_block_size_bytes_ > 0} + , prewhere_info{prewhere_info_} + , parts_ranges{parts_} { /// parts don't contain duplicate MergeTreeDataPart's. const auto per_part_sum_marks = fillPerPartInfo(parts_, check_columns_); @@ -139,7 +150,7 @@ MarkRanges MergeTreeReadPool::getRestMarks(const IMergeTreeDataPart & part, cons Block MergeTreeReadPool::getHeader() const { - return data.getSampleBlockForColumns(column_names); + return metadata_snapshot->getSampleBlockForColumns(column_names, data.getVirtuals()); } void MergeTreeReadPool::profileFeedback(const ReadBufferFromFileBase::ProfileInfo info) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index c43074f1962..c0b04c6a228 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -68,7 +68,7 @@ private: public: MergeTreeReadPool( const size_t threads_, const size_t sum_marks_, const size_t min_marks_for_concurrent_read_, - RangesInDataParts parts_, const MergeTreeData & data_, const PrewhereInfoPtr & prewhere_info_, + RangesInDataParts parts_, const MergeTreeData & data_, const StorageMetadataPtr & metadata_snapshot_, const PrewhereInfoPtr & prewhere_info_, const bool check_columns_, const Names & column_names_, const BackoffSettings & backoff_settings_, size_t preferred_block_size_bytes_, const bool do_not_steal_tasks_ = false); @@ -95,6 +95,7 @@ private: RangesInDataParts & parts, const size_t min_marks_for_concurrent_read); const MergeTreeData & data; + StorageMetadataPtr metadata_snapshot; Names column_names; bool do_not_steal_tasks; bool predict_block_size_bytes; diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp index 09bf784a293..81366614988 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp @@ -34,6 +34,7 @@ static Block replaceTypes(Block && header, const MergeTreeData::DataPartPtr & da MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor( const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, const MergeTreeData::DataPartPtr & owned_data_part_, UInt64 max_block_size_rows_, size_t preferred_block_size_bytes_, @@ -49,8 +50,8 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor( bool quiet) : MergeTreeBaseSelectProcessor{ - replaceTypes(storage_.getSampleBlockForColumns(required_columns_), owned_data_part_), - storage_, prewhere_info_, max_block_size_rows_, + replaceTypes(metadata_snapshot_->getSampleBlockForColumns(required_columns_, storage_.getVirtuals()), owned_data_part_), + storage_, metadata_snapshot_, prewhere_info_, max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, reader_settings_, use_uncompressed_cache_, virt_column_names_}, required_columns{std::move(required_columns_)}, diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h index ea603bd468f..c9fd06c5534 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h @@ -18,6 +18,7 @@ class MergeTreeReverseSelectProcessor : public MergeTreeBaseSelectProcessor public: MergeTreeReverseSelectProcessor( const MergeTreeData & storage, + const StorageMetadataPtr & metadata_snapshot, const MergeTreeData::DataPartPtr & owned_data_part, UInt64 max_block_size_rows, size_t preferred_block_size_bytes, diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index df471a8b8ec..e32fa70cb97 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -14,6 +14,7 @@ namespace ErrorCodes MergeTreeSelectProcessor::MergeTreeSelectProcessor( const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, const MergeTreeData::DataPartPtr & owned_data_part_, UInt64 max_block_size_rows_, size_t preferred_block_size_bytes_, @@ -29,8 +30,8 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( bool quiet) : MergeTreeBaseSelectProcessor{ - storage_.getSampleBlockForColumns(required_columns_), - storage_, prewhere_info_, max_block_size_rows_, + metadata_snapshot_->getSampleBlockForColumns(required_columns_, storage_.getVirtuals()), + storage_, metadata_snapshot_, prewhere_info_, max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, reader_settings_, use_uncompressed_cache_, virt_column_names_}, required_columns{std::move(required_columns_)}, diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index d2438e20192..dff4ebc2627 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -18,6 +18,7 @@ class MergeTreeSelectProcessor : public MergeTreeBaseSelectProcessor public: MergeTreeSelectProcessor( const MergeTreeData & storage, + const StorageMetadataPtr & metadata_snapshot, const MergeTreeData::DataPartPtr & owned_data_part, UInt64 max_block_size_rows, size_t preferred_block_size_bytes, diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 045962f44dd..dfd60bd50ef 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -11,13 +11,15 @@ namespace ErrorCodes MergeTreeSequentialSource::MergeTreeSequentialSource( const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, MergeTreeData::DataPartPtr data_part_, Names columns_to_read_, bool read_with_direct_io_, bool take_column_types_from_storage, bool quiet) - : SourceWithProgress(storage_.getSampleBlockForColumns(columns_to_read_)) + : SourceWithProgress(metadata_snapshot_->getSampleBlockForColumns(columns_to_read_, storage_.getVirtuals())) , storage(storage_) + , metadata_snapshot(metadata_snapshot_) , data_part(std::move(data_part_)) , columns_to_read(std::move(columns_to_read_)) , read_with_direct_io(read_with_direct_io_) @@ -41,7 +43,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( NamesAndTypesList columns_for_reader; if (take_column_types_from_storage) { - const NamesAndTypesList & physical_columns = storage.getColumns().getAllPhysical(); + const NamesAndTypesList & physical_columns = metadata_snapshot->getColumns().getAllPhysical(); columns_for_reader = physical_columns.addTypes(columns_to_read); } else diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index 6155fef200a..7eefdd9335b 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -14,12 +14,12 @@ class MergeTreeSequentialSource : public SourceWithProgress public: MergeTreeSequentialSource( const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, MergeTreeData::DataPartPtr data_part_, Names columns_to_read_, bool read_with_direct_io_, bool take_column_types_from_storage, - bool quiet = false - ); + bool quiet = false); ~MergeTreeSequentialSource() override; @@ -35,6 +35,7 @@ protected: private: const MergeTreeData & storage; + StorageMetadataPtr metadata_snapshot; /// Data part will not be removed if the pointer owns it MergeTreeData::DataPartPtr data_part; diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp index 0b09fad91d1..784c842d7d6 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp @@ -16,12 +16,15 @@ MergeTreeThreadSelectBlockInputProcessor::MergeTreeThreadSelectBlockInputProcess size_t preferred_block_size_bytes_, size_t preferred_max_column_in_block_size_bytes_, const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, const bool use_uncompressed_cache_, const PrewhereInfoPtr & prewhere_info_, const MergeTreeReaderSettings & reader_settings_, const Names & virt_column_names_) : - MergeTreeBaseSelectProcessor{pool_->getHeader(), storage_, prewhere_info_, max_block_size_rows_, + MergeTreeBaseSelectProcessor{ + pool_->getHeader(), storage_, metadata_snapshot_, prewhere_info_, + max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, reader_settings_, use_uncompressed_cache_, virt_column_names_}, thread{thread_}, diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h index e214696b705..d5a11f3d93b 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h +++ b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h @@ -22,6 +22,7 @@ public: size_t preferred_block_size_bytes_, size_t preferred_max_column_in_block_size_bytes_, const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, const bool use_uncompressed_cache_, const PrewhereInfoPtr & prewhere_info_, const MergeTreeReaderSettings & reader_settings_, diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 826af505b12..45ee947b81f 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -21,15 +21,15 @@ public: Pipes read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned num_streams) override { - return MergeTreeDataSelectExecutor(part->storage).readFromParts( - {part}, column_names, query_info, context, max_block_size, num_streams); + return MergeTreeDataSelectExecutor(part->storage) + .readFromParts({part}, column_names, metadata_snapshot, query_info, context, max_block_size, num_streams); } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 4754732159c..42eab838f32 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -88,9 +88,11 @@ StorageBuffer::StorageBuffer( class BufferSource : public SourceWithProgress { public: - BufferSource(const Names & column_names_, StorageBuffer::Buffer & buffer_, const StorageBuffer & storage) - : SourceWithProgress(storage.getSampleBlockForColumns(column_names_)) - , column_names(column_names_.begin(), column_names_.end()), buffer(buffer_) {} + BufferSource(const Names & column_names_, StorageBuffer::Buffer & buffer_, const StorageBuffer & storage, const StorageMetadataPtr & metadata_snapshot) + : SourceWithProgress( + metadata_snapshot->getSampleBlockForColumns(column_names_, storage.getVirtuals())) + , column_names(column_names_.begin(), column_names_.end()) + , buffer(buffer_) {} String getName() const override { return "Buffer"; } @@ -145,7 +147,7 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context Pipes StorageBuffer::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -236,7 +238,7 @@ Pipes StorageBuffer::read( Pipes pipes_from_buffers; pipes_from_buffers.reserve(num_shards); for (auto & buf : buffers) - pipes_from_buffers.emplace_back(std::make_shared(column_names, buf, *this)); + pipes_from_buffers.emplace_back(std::make_shared(column_names, buf, *this, metadata_snapshot)); /** If the sources from the table were processed before some non-initial stage of query execution, * then sources from the buffers must also be wrapped in the processing pipeline before the same stage. diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index f3719562af7..cce3911370d 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -1,7 +1,24 @@ #include +#include + namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int COLUMN_QUERIED_MORE_THAN_ONCE; + extern const int DUPLICATE_COLUMN; + extern const int EMPTY_LIST_OF_COLUMNS_PASSED; + extern const int EMPTY_LIST_OF_COLUMNS_QUERIED; + extern const int NO_SUCH_COLUMN_IN_TABLE; + extern const int NOT_FOUND_COLUMN_IN_BLOCK; + extern const int TYPE_MISMATCH; + extern const int TABLE_IS_DROPPED; + extern const int NOT_IMPLEMENTED; + extern const int DEADLOCK_AVOIDED; +} + StorageInMemoryMetadata::StorageInMemoryMetadata( const ColumnsDescription & columns_, @@ -240,4 +257,37 @@ Block StorageInMemoryMetadata::getSampleBlock() const return res; } + +Block StorageInMemoryMetadata::getSampleBlockForColumns(const Names & column_names, const NamesAndTypesList & virtuals) const +{ + Block res; + + std::unordered_map columns_map; + + NamesAndTypesList all_columns = getColumns().getAll(); + for (const auto & elem : all_columns) + columns_map.emplace(elem.name, elem.type); + + /// Virtual columns must be appended after ordinary, because user can + /// override them. + for (const auto & column : virtuals) + columns_map.emplace(column.name, column.type); + + for (const auto & name : column_names) + { + auto it = columns_map.find(name); + if (it != columns_map.end()) + { + res.insert({it->second->createColumn(), it->second, it->first}); + } + else + { + throw Exception( + "Column " + backQuote(name) + " not found in table " /*+ getStorageID().getNameForLogs() TODO(alesap)*/, + ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); + } + } + + return res; +} } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 2da766caacd..9f9154c48fb 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -110,6 +110,8 @@ struct StorageInMemoryMetadata Block getSampleBlock() const; /// ordinary + materialized. Block getSampleBlockNonMaterialized() const; /// ordinary. Block getSampleBlockWithVirtuals(const NamesAndTypesList & virtuals) const; /// ordinary + materialized + virtuals. + Block getSampleBlockForColumns( + const Names & column_names, const NamesAndTypesList & virtuals) const; /// ordinary + materialized + aliases + virtuals. }; using StorageMetadataPtr = std::shared_ptr; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 7ed4c1c110b..7d481395ef4 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -435,7 +435,7 @@ private: // TODO: multiple stream read and index read Pipes StorageJoin::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, @@ -445,7 +445,7 @@ Pipes StorageJoin::read( check(column_names); Pipes pipes; - pipes.emplace_back(std::make_shared(*join, max_block_size, getSampleBlockForColumns(column_names))); + pipes.emplace_back(std::make_shared(*join, max_block_size, metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals()))); return pipes; } diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index f9c39d78a05..442c5a3d67b 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -22,9 +22,19 @@ namespace ErrorCodes class MemorySource : public SourceWithProgress { public: - MemorySource(Names column_names_, BlocksList::iterator begin_, BlocksList::iterator end_, const StorageMemory & storage) - : SourceWithProgress(storage.getSampleBlockForColumns(column_names_)) - , column_names(std::move(column_names_)), begin(begin_), end(end_), it(begin) {} + MemorySource( + Names column_names_, + BlocksList::iterator begin_, + BlocksList::iterator end_, + const StorageMemory & storage, + const StorageMetadataPtr & metadata_snapshot) + : SourceWithProgress(metadata_snapshot->getSampleBlockForColumns(column_names_, storage.getVirtuals())) + , column_names(std::move(column_names_)) + , begin(begin_) + , end(end_) + , it(begin) + { + } String getName() const override { return "Memory"; } @@ -60,9 +70,14 @@ private: class MemoryBlockOutputStream : public IBlockOutputStream { public: - explicit MemoryBlockOutputStream(StorageMemory & storage_) : storage(storage_) {} + explicit MemoryBlockOutputStream( + StorageMemory & storage_, + const StorageMetadataPtr & metadata_snapshot_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + {} - Block getHeader() const override { return storage.getSampleBlock(); } + Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } void write(const Block & block) override { @@ -72,6 +87,7 @@ public: } private: StorageMemory & storage; + StorageMetadataPtr metadata_snapshot; }; @@ -87,7 +103,7 @@ StorageMemory::StorageMemory(const StorageID & table_id_, ColumnsDescription col Pipes StorageMemory::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, @@ -113,16 +129,16 @@ Pipes StorageMemory::read( std::advance(begin, stream * size / num_streams); std::advance(end, (stream + 1) * size / num_streams); - pipes.emplace_back(std::make_shared(column_names, begin, end, *this)); + pipes.emplace_back(std::make_shared(column_names, begin, end, *this, metadata_snapshot)); } return pipes; } -BlockOutputStreamPtr StorageMemory::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) +BlockOutputStreamPtr StorageMemory::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { - return std::make_shared(*this); + return std::make_shared(*this, metadata_snapshot); } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 6656e91189c..c5a3c20bb0c 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -129,7 +129,7 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & Pipes StorageMerge::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -157,7 +157,7 @@ Pipes StorageMerge::read( modified_context->setSetting("optimize_move_to_prewhere", false); /// What will be result structure depending on query processed stage in source tables? - Block header = getQueryHeader(column_names, query_info, context, processed_stage); + Block header = getQueryHeader(column_names, metadata_snapshot, query_info, context, processed_stage); /** First we make list of selected tables to find out its size. * This is necessary to correctly pass the recommended number of threads to each table. @@ -401,13 +401,17 @@ void StorageMerge::alter( } Block StorageMerge::getQueryHeader( - const Names & column_names, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage) + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage) { switch (processed_stage) { case QueryProcessingStage::FetchColumns: { - Block header = getSampleBlockForColumns(column_names); + Block header = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals()); if (query_info.prewhere_info) { query_info.prewhere_info->prewhere_actions->execute(header); @@ -420,7 +424,7 @@ Block StorageMerge::getQueryHeader( case QueryProcessingStage::WithMergeableState: case QueryProcessingStage::Complete: return materializeBlock(InterpreterSelectQuery( - query_info.query, context, std::make_shared(getSampleBlockForColumns(column_names)), + query_info.query, context, std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals())), SelectQueryOptions(processed_stage).analyze()).getSampleBlock()); } throw Exception("Logical Error: unknown processed stage.", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index a5d3b8d2667..350f7a124fe 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -74,8 +74,12 @@ protected: const String & table_name_regexp_, const Context & context_); - Block getQueryHeader(const Names & column_names, const SelectQueryInfo & query_info, - const Context & context, QueryProcessingStage::Enum processed_stage); + Block getQueryHeader( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage); Pipes createSources( const SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 1b00487c816..9c37cdd2b7c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -177,14 +177,15 @@ StorageMergeTree::~StorageMergeTree() Pipes StorageMergeTree::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t max_block_size, const unsigned num_streams) { - return reader.read(column_names, query_info, context, max_block_size, num_streams); + return reader.read(column_names, metadata_snapshot, query_info, + context, max_block_size, num_streams); } std::optional StorageMergeTree::totalRows() const diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index 72934d185c7..6bd102bdcda 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -24,7 +24,7 @@ public: Pipes read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo &, const Context & /*context*/, QueryProcessingStage::Enum /*processing_stage*/, @@ -32,13 +32,14 @@ public: unsigned) override { Pipes pipes; - pipes.emplace_back(std::make_shared(getSampleBlockForColumns(column_names))); + pipes.emplace_back( + std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals()))); return pipes; } - BlockOutputStreamPtr write(const ASTPtr &, const StorageMetadataPtr & /*metadata_snapshot*/, const Context &) override + BlockOutputStreamPtr write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &) override { - return std::make_shared(getSampleBlock()); + return std::make_shared(metadata_snapshot->getSampleBlock()); } void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 810a4fa5c97..a6f84ffe4df 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3387,7 +3387,7 @@ ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock StorageReplicatedMerg Pipes StorageReplicatedMergeTree::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -3402,10 +3402,10 @@ Pipes StorageReplicatedMergeTree::read( if (context.getSettingsRef().select_sequential_consistency) { auto max_added_blocks = getMaxAddedBlocks(); - return reader.read(column_names, query_info, context, max_block_size, num_streams, &max_added_blocks); + return reader.read(column_names, metadata_snapshot, query_info, context, max_block_size, num_streams, &max_added_blocks); } - return reader.read(column_names, query_info, context, max_block_size, num_streams); + return reader.read(column_names, metadata_snapshot, query_info, context, max_block_size, num_streams); } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 093f4450ecb..7f237fd551f 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -285,7 +285,7 @@ Strings listFilesWithRegexpMatching(Aws::S3::S3Client & client, const S3::URI & Pipes StorageS3::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -309,9 +309,9 @@ Pipes StorageS3::read( need_file_column, format_name, getName(), - getHeaderBlock(column_names), + metadata_snapshot->getSampleBlock(), context, - getColumns().getDefaults(), + metadata_snapshot->getColumns().getDefaults(), max_block_size, chooseCompressionMethod(uri.endpoint, compression_method), client, @@ -321,11 +321,11 @@ Pipes StorageS3::read( return narrowPipes(std::move(pipes), num_streams); } -BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) +BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { return std::make_shared( - format_name, min_upload_part_size, getSampleBlock(), context_global, - chooseCompressionMethod(uri.endpoint, compression_method), + format_name, min_upload_part_size, metadata_snapshot->getSampleBlock(), + context_global, chooseCompressionMethod(uri.endpoint, compression_method), client, uri.bucket, uri.key); } diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 665c00b8033..a468d69d223 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -41,11 +41,6 @@ public: return "S3"; } - Block getHeaderBlock(const Names & /*column_names*/) const - { - return getSampleBlock(); - } - Pipes read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index c320d0afb42..b0c5bcfd669 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -54,12 +54,13 @@ public: static Block getHeader( StorageStripeLog & storage, + const StorageMetadataPtr & metadata_snapshot, const Names & column_names, IndexForNativeFormat::Blocks::const_iterator index_begin, IndexForNativeFormat::Blocks::const_iterator index_end) { if (index_begin == index_end) - return storage.getSampleBlockForColumns(column_names); + return metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals()); /// TODO: check if possible to always return storage.getSampleBlock() @@ -74,13 +75,22 @@ public: return header; } - StripeLogSource(StorageStripeLog & storage_, const Names & column_names, size_t max_read_buffer_size_, + StripeLogSource( + StorageStripeLog & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const Names & column_names, + size_t max_read_buffer_size_, std::shared_ptr & index_, IndexForNativeFormat::Blocks::const_iterator index_begin_, IndexForNativeFormat::Blocks::const_iterator index_end_) - : SourceWithProgress(getHeader(storage_, column_names, index_begin_, index_end_)) - , storage(storage_), max_read_buffer_size(max_read_buffer_size_) - , index(index_), index_begin(index_begin_), index_end(index_end_) + : SourceWithProgress( + getHeader(storage_, metadata_snapshot_, column_names, index_begin_, index_end_)) + , storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , max_read_buffer_size(max_read_buffer_size_) + , index(index_) + , index_begin(index_begin_) + , index_end(index_end_) { } @@ -110,6 +120,7 @@ protected: private: StorageStripeLog & storage; + StorageMetadataPtr metadata_snapshot; size_t max_read_buffer_size; std::shared_ptr index; @@ -253,7 +264,7 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora Pipes StorageStripeLog::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -271,7 +282,7 @@ Pipes StorageStripeLog::read( String index_file = table_path + "index.mrk"; if (!disk->exists(index_file)) { - pipes.emplace_back(std::make_shared(getSampleBlockForColumns(column_names))); + pipes.emplace_back(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals()))); return pipes; } @@ -291,7 +302,7 @@ Pipes StorageStripeLog::read( std::advance(end, (stream + 1) * size / num_streams); pipes.emplace_back(std::make_shared( - *this, column_names, context.getSettingsRef().max_read_buffer_size, index, begin, end)); + *this, metadata_snapshot, column_names, context.getSettingsRef().max_read_buffer_size, index, begin, end)); } /// We do not keep read lock directly at the time of reading, because we read ranges of data that do not change. diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 0361718c616..6cea7115066 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -157,7 +157,7 @@ std::function IStorageURLBase::getReadPOSTDataCallback(con Pipes IStorageURLBase::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -170,14 +170,15 @@ Pipes IStorageURLBase::read( request_uri.addQueryParameter(param, value); Pipes pipes; - pipes.emplace_back(std::make_shared(request_uri, + pipes.emplace_back(std::make_shared( + request_uri, getReadMethod(), getReadPOSTDataCallback(column_names, query_info, context, processed_stage, max_block_size), format_name, getName(), - getHeaderBlock(column_names), + getHeaderBlock(column_names, metadata_snapshot), context, - getColumns().getDefaults(), + metadata_snapshot->getColumns().getDefaults(), max_block_size, ConnectionTimeouts::getHTTPTimeouts(context), chooseCompressionMethod(request_uri.getPath(), compression_method))); diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index ecd57024a44..04cbb278c37 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -62,7 +62,7 @@ private: QueryProcessingStage::Enum & processed_stage, size_t max_block_size) const; - virtual Block getHeaderBlock(const Names & column_names) const = 0; + virtual Block getHeaderBlock(const Names & column_names, const StorageMetadataPtr & metadata_snapshot) const = 0; }; class StorageURLBlockOutputStream : public IBlockOutputStream @@ -124,9 +124,9 @@ public: return "URL"; } - Block getHeaderBlock(const Names & /*column_names*/) const override + Block getHeaderBlock(const Names & /*column_names*/, const StorageMetadataPtr & metadata_snapshot) const override { - return getSampleBlock(); + return metadata_snapshot->getSampleBlock(); } }; } diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 52b7e8764d9..7e49580d6c2 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -54,7 +54,7 @@ StorageView::StorageView( Pipes StorageView::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -86,8 +86,9 @@ Pipes StorageView::read( /// And also convert to expected structure. pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, getSampleBlockForColumns(column_names), - ConvertingTransform::MatchColumnsMode::Name); + return std::make_shared( + header, metadata_snapshot->getSampleBlockForColumns( + column_names, getVirtuals()), ConvertingTransform::MatchColumnsMode::Name); }); pipes = std::move(pipeline).getPipes(); diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index c7fa8a88251..a837cf5b4ba 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -97,16 +97,16 @@ Pipes StorageXDBC::read( return IStorageURLBase::read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } -BlockOutputStreamPtr StorageXDBC::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) +BlockOutputStreamPtr StorageXDBC::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context) { bridge_helper->startBridgeSync(); NamesAndTypesList cols; Poco::URI request_uri = uri; request_uri.setPath("/write"); - for (const String & name : getSampleBlock().getNames()) + for (const String & name : metadata_snapshot->getSampleBlock().getNames()) { - auto column_data = getColumns().getPhysical(name); + auto column_data = metadata_snapshot->getColumns().getPhysical(name); cols.emplace_back(column_data.name, column_data.type); } auto url_params = bridge_helper->getURLParams(cols.toString(), 65536); @@ -117,14 +117,17 @@ BlockOutputStreamPtr StorageXDBC::write(const ASTPtr & /*query*/, const StorageM request_uri.addQueryParameter("format_name", format_name); return std::make_shared( - request_uri, format_name, getSampleBlock(), context, - ConnectionTimeouts::getHTTPTimeouts(context), - chooseCompressionMethod(uri.toString(), compression_method)); + request_uri, + format_name, + metadata_snapshot->getSampleBlock(), + context, + ConnectionTimeouts::getHTTPTimeouts(context), + chooseCompressionMethod(uri.toString(), compression_method)); } -Block StorageXDBC::getHeaderBlock(const Names & column_names) const +Block StorageXDBC::getHeaderBlock(const Names & column_names, const StorageMetadataPtr & metadata_snapshot) const { - return getSampleBlockForColumns(column_names); + return metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals()); } std::string StorageXDBC::getName() const diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index 4488122656d..0e227d7d432 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -24,11 +24,12 @@ public: size_t max_block_size, unsigned num_streams) override; - StorageXDBC(const StorageID & table_id_, - const std::string & remote_database_name, - const std::string & remote_table_name, - const ColumnsDescription & columns_, - const Context & context_, BridgeHelperPtr bridge_helper_); + StorageXDBC( + const StorageID & table_id_, + const std::string & remote_database_name, + const std::string & remote_table_name, + const ColumnsDescription & columns_, + const Context & context_, BridgeHelperPtr bridge_helper_); BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; @@ -42,19 +43,21 @@ private: std::string getReadMethod() const override; - std::vector> getReadURIParams(const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size) const override; + std::vector> getReadURIParams( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size) const override; - std::function getReadPOSTDataCallback(const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size) const override; + std::function getReadPOSTDataCallback( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size) const override; - Block getHeaderBlock(const Names & column_names) const override; + Block getHeaderBlock(const Names & column_names, const StorageMetadataPtr & metadata_snapshot) const override; std::string getName() const override; }; From 824d6667d97c945d72aafc87b531eba177305f9e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Jun 2020 17:27:44 +0300 Subject: [PATCH 068/211] Seems to be working getSampleBlockWithColumns in StorageInMemoryMetadata From 1ddeb3d149a19b21adc1c426287995dbfd4b3249 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Jun 2020 18:51:29 +0300 Subject: [PATCH 069/211] Buildable getSampleBlock in StorageInMemoryMetadata --- src/DataStreams/RemoteQueryExecutor.cpp | 3 +- src/Interpreters/InterpreterAlterQuery.cpp | 3 +- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Interpreters/executeQuery.cpp | 5 ++- src/Server/TCPHandler.cpp | 2 +- .../DistributedBlockOutputStream.cpp | 25 +++++++---- .../DistributedBlockOutputStream.h | 12 +++++- src/Storages/IStorage.cpp | 10 ----- src/Storages/IStorage.h | 4 +- .../MergeTree/IMergedBlockOutputStream.cpp | 5 ++- .../MergeTree/IMergedBlockOutputStream.h | 4 +- .../MergeTree/MergeTreeBlockOutputStream.cpp | 4 +- .../MergeTree/MergeTreeBlockOutputStream.h | 13 +++++- .../MergeTree/MergeTreeDataMergerMutator.cpp | 8 ++++ .../MergeTree/MergeTreeDataMergerMutator.h | 2 + .../MergeTree/MergeTreeDataWriter.cpp | 4 +- src/Storages/MergeTree/MergeTreeDataWriter.h | 2 +- src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 +- .../MergeTreeReverseSelectProcessor.cpp | 2 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 11 ++++- .../MergeTree/MergedBlockOutputStream.h | 4 +- .../MergedColumnOnlyOutputStream.cpp | 4 +- .../MergeTree/MergedColumnOnlyOutputStream.h | 1 + .../ReplicatedMergeTreeBlockOutputStream.cpp | 20 ++++++--- .../ReplicatedMergeTreeBlockOutputStream.h | 9 +++- src/Storages/StorageBuffer.cpp | 24 +++++++---- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageFile.cpp | 32 +++++++++------ src/Storages/StorageHDFS.cpp | 8 ++-- src/Storages/StorageInput.cpp | 4 +- src/Storages/StorageJoin.cpp | 12 ++++-- src/Storages/StorageLog.cpp | 17 ++++---- src/Storages/StorageMaterializedView.cpp | 5 ++- src/Storages/StorageMaterializedView.h | 2 +- src/Storages/StorageMerge.cpp | 36 +++++++++++----- src/Storages/StorageMerge.h | 18 +++++--- src/Storages/StorageMergeTree.cpp | 12 +++--- src/Storages/StorageMergeTree.h | 6 ++- src/Storages/StorageMySQL.cpp | 25 +++++++---- src/Storages/StorageReplicatedMergeTree.cpp | 17 ++++---- src/Storages/StorageReplicatedMergeTree.h | 8 +++- src/Storages/StorageSet.cpp | 41 ++++++++++++------- src/Storages/StorageStripeLog.cpp | 27 ++++++------ src/Storages/StorageTinyLog.cpp | 15 +++---- src/Storages/StorageURL.cpp | 4 +- src/Storages/System/IStorageSystemOneBlock.h | 4 +- src/Storages/System/StorageSystemColumns.cpp | 4 +- .../System/StorageSystemDetachedParts.cpp | 4 +- src/Storages/System/StorageSystemDisks.cpp | 4 +- .../System/StorageSystemPartsBase.cpp | 6 +-- src/Storages/System/StorageSystemReplicas.cpp | 8 ++-- .../System/StorageSystemStoragePolicies.cpp | 4 +- src/Storages/System/StorageSystemTables.cpp | 4 +- 54 files changed, 328 insertions(+), 188 deletions(-) diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index 071cb6e9aba..45ddd7c0893 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -333,7 +333,8 @@ void RemoteQueryExecutor::sendExternalTables() data->table_name = table.first; if (pipes.empty()) - data->pipe = std::make_unique(std::make_shared(cur->getSampleBlock(), Chunk())); + data->pipe = std::make_unique( + std::make_shared(metadata_snapshot->getSampleBlock(), Chunk())); else if (pipes.size() == 1) data->pipe = std::make_unique(std::move(pipes.front())); else diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index bd20d78279d..869c3ae98d3 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -43,6 +43,7 @@ BlockIO InterpreterAlterQuery::execute() context.checkAccess(getRequiredAccess()); auto table_id = context.resolveStorageID(alter, Context::ResolveOrdinary); StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); + auto metadata_snapshot = table->getInMemoryMetadataPtr(); /// Add default database to table identifiers that we can encounter in e.g. default expressions, /// mutation expression, etc. @@ -91,7 +92,7 @@ BlockIO InterpreterAlterQuery::execute() if (!partition_commands.empty()) { - table->alterPartition(query_ptr, partition_commands, context); + table->alterPartition(query_ptr, metadata_snapshot, partition_commands, context); } if (!live_view_commands.empty()) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index f61ef0e7381..443e2714ec7 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -88,7 +88,7 @@ Block InterpreterInsertQuery::getSampleBlock( return table_sample_non_materialized; } - Block table_sample = table->getSampleBlock(); + Block table_sample = metadata_snapshot->getSampleBlock(); /// Form the block based on the column names from the query Block res; for (const auto & identifier : query.columns->children) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index e1e2108c0fc..2b8ebf12a20 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -326,8 +326,9 @@ static std::tuple executeQueryImpl( { StoragePtr storage = context.executeTableFunction(input_function); auto & input_storage = dynamic_cast(*storage); - BlockInputStreamPtr input_stream = std::make_shared(ast, istr, - input_storage.getSampleBlock(), context, input_function); + auto input_metadata_snapshot = input_storage.getInMemoryMetadataPtr(); + BlockInputStreamPtr input_stream = std::make_shared( + ast, istr, input_metadata_snapshot->getSampleBlock(), context, input_function); input_storage.setInputStream(input_stream); } } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a01cc4fa0aa..009f7ad80f0 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -223,7 +223,7 @@ void TCPHandler::runImpl() } /// Send block to the client - input storage structure. - state.input_header = input_storage->getSampleBlock(); + state.input_header = input_storage->getInMemoryMetadataPtr()->getSampleBlock(); sendData(state.input_header); }); diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 4e28923ebfc..bf9efef1ba6 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -83,18 +83,29 @@ static void writeBlockConvert(const BlockOutputStreamPtr & out, const Block & bl DistributedBlockOutputStream::DistributedBlockOutputStream( - const Context & context_, StorageDistributed & storage_, const ASTPtr & query_ast_, const ClusterPtr & cluster_, - bool insert_sync_, UInt64 insert_timeout_) - : context(context_), storage(storage_), query_ast(query_ast_), query_string(queryToString(query_ast_)), - cluster(cluster_), insert_sync(insert_sync_), - insert_timeout(insert_timeout_), log(&Poco::Logger::get("DistributedBlockOutputStream")) + const Context & context_, + StorageDistributed & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const ASTPtr & query_ast_, + const ClusterPtr & cluster_, + bool insert_sync_, + UInt64 insert_timeout_) + : context(context_) + , storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , query_ast(query_ast_) + , query_string(queryToString(query_ast_)) + , cluster(cluster_) + , insert_sync(insert_sync_) + , insert_timeout(insert_timeout_) + , log(&Poco::Logger::get("DistributedBlockOutputStream")) { } Block DistributedBlockOutputStream::getHeader() const { - return storage.getSampleBlock(); + return metadata_snapshot->getSampleBlock(); } @@ -109,7 +120,7 @@ void DistributedBlockOutputStream::write(const Block & block) /* They are added by the AddingDefaultBlockOutputStream, and we will get * different number of columns eventually */ - for (const auto & col : storage.getColumns().getMaterialized()) + for (const auto & col : metadata_snapshot->getColumns().getMaterialized()) { if (ordinary_block.has(col.name)) { diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.h b/src/Storages/Distributed/DistributedBlockOutputStream.h index 17db955431c..53d71ffc424 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.h +++ b/src/Storages/Distributed/DistributedBlockOutputStream.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -36,8 +37,14 @@ class StorageDistributed; class DistributedBlockOutputStream : public IBlockOutputStream { public: - DistributedBlockOutputStream(const Context & context_, StorageDistributed & storage_, const ASTPtr & query_ast_, - const ClusterPtr & cluster_, bool insert_sync_, UInt64 insert_timeout_); + DistributedBlockOutputStream( + const Context & context_, + StorageDistributed & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const ASTPtr & query_ast_, + const ClusterPtr & cluster_, + bool insert_sync_, + UInt64 insert_timeout_); Block getHeader() const override; void write(const Block & block) override; @@ -79,6 +86,7 @@ private: private: const Context & context; StorageDistributed & storage; + StorageMetadataPtr metadata_snapshot; ASTPtr query_ast; String query_string; ClusterPtr cluster; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index d090dc9e51d..43e9a5dd040 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -52,16 +52,6 @@ const ConstraintsDescription & IStorage::getConstraints() const return metadata->constraints; } -Block IStorage::getSampleBlock() const -{ - Block res; - - for (const auto & column : getColumns().getAllPhysical()) - res.insert({column.type->createColumn(), column.type, column.name}); - - return res; -} - namespace { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index a4173c1c9fa..df1e1685a2e 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -158,8 +158,6 @@ public: /// thread-unsafe part. lockStructure must be acquired StorageMetadataPtr getInMemoryMetadataPtr() const { return metadata; } void setInMemoryMetadata(const StorageInMemoryMetadata & metadata_) { metadata = std::make_shared(metadata_); } - Block getSampleBlock() const; /// ordinary + materialized. - /// Verify that all the requested names are in the table and are set correctly: /// list of names is not empty and the names do not repeat. void check(const Names & column_names, bool include_virtuals = false) const; @@ -361,7 +359,7 @@ public: /** ALTER tables with regard to its partitions. * Should handle locks for each command on its own. */ - virtual void alterPartition(const ASTPtr & /* query */, const PartitionCommands & /* commands */, const Context & /* context */) + virtual void alterPartition(const ASTPtr & /* query */, const StorageMetadataPtr & /* metadata_snapshot */, const PartitionCommands & /* commands */, const Context & /* context */) { throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 329a8ee4508..cfb4d8ba4ba 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -5,10 +5,11 @@ namespace DB { - IMergedBlockOutputStream::IMergedBlockOutputStream( - const MergeTreeDataPartPtr & data_part) + const MergeTreeDataPartPtr & data_part, + const StorageMetadataPtr & metadata_snapshot_) : storage(data_part->storage) + , metadata_snapshot(metadata_snapshot_) , volume(data_part->volume) , part_path(data_part->getFullRelativePath()) { diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index 6a06d4b0c75..7ec9f85ff28 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -13,7 +13,8 @@ class IMergedBlockOutputStream : public IBlockOutputStream { public: IMergedBlockOutputStream( - const MergeTreeDataPartPtr & data_part); + const MergeTreeDataPartPtr & data_part, + const StorageMetadataPtr & metadata_snapshot_); using WrittenOffsetColumns = std::set; @@ -36,6 +37,7 @@ protected: protected: const MergeTreeData & storage; + StorageMetadataPtr metadata_snapshot; VolumePtr volume; String part_path; diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index b6376dd3779..1ea6b049bf6 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -8,7 +8,7 @@ namespace DB Block MergeTreeBlockOutputStream::getHeader() const { - return storage.getSampleBlock(); + return metadata_snapshot->getSampleBlock(); } @@ -21,7 +21,7 @@ void MergeTreeBlockOutputStream::write(const Block & block) { Stopwatch watch; - MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block); + MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, metadata_snapshot); storage.renameTempPartAndAdd(part, &storage.increment); PartLog::addNewPart(storage.global_context, part, watch.elapsed()); diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.h b/src/Storages/MergeTree/MergeTreeBlockOutputStream.h index 8f957d631d3..71e126b07ef 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.h +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -13,14 +14,22 @@ class StorageMergeTree; class MergeTreeBlockOutputStream : public IBlockOutputStream { public: - MergeTreeBlockOutputStream(StorageMergeTree & storage_, size_t max_parts_per_block_) - : storage(storage_), max_parts_per_block(max_parts_per_block_) {} + MergeTreeBlockOutputStream( + StorageMergeTree & storage_, + const StorageMetadataPtr metadata_snapshot_, + size_t max_parts_per_block_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , max_parts_per_block(max_parts_per_block_) + { + } Block getHeader() const override; void write(const Block & block) override; private: StorageMergeTree & storage; + StorageMetadataPtr metadata_snapshot; size_t max_parts_per_block; }; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 829f7cac528..39e6cfdc275 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -808,6 +808,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream to{ new_data_part, + metadata_snapshot, merging_columns, index_factory.getMany(data.getSecondaryIndices()), compression_codec, @@ -912,6 +913,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor MergedColumnOnlyOutputStream column_to( new_data_part, + metadata_snapshot, column_gathered_stream.getHeader(), compression_codec, /// we don't need to recalc indices here @@ -1085,6 +1087,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor auto part_indices = getIndicesForNewDataPart(data.getSecondaryIndices(), for_file_renames); mutateAllPartColumns( new_data_part, + metadata_snapshot, part_indices, in, time_of_mutation, @@ -1137,6 +1140,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor { mutateSomePartColumns( source_part, + metadata_snapshot, indices_to_recalc, updated_header, new_data_part, @@ -1582,6 +1586,7 @@ bool MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadataPtr & met void MergeTreeDataMergerMutator::mutateAllPartColumns( MergeTreeData::MutableDataPartPtr new_data_part, + const StorageMetadataPtr & metadata_snapshot, const MergeTreeIndices & skip_indices, BlockInputStreamPtr mutating_stream, time_t time_of_mutation, @@ -1603,6 +1608,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( MergedBlockOutputStream out{ new_data_part, + metadata_snapshot, new_data_part->getColumns(), skip_indices, compression_codec}; @@ -1629,6 +1635,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( void MergeTreeDataMergerMutator::mutateSomePartColumns( const MergeTreeDataPartPtr & source_part, + const StorageMetadataPtr & metadata_snapshot, const std::set & indices_to_recalc, const Block & mutation_header, MergeTreeData::MutableDataPartPtr new_data_part, @@ -1647,6 +1654,7 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets; MergedColumnOnlyOutputStream out( new_data_part, + metadata_snapshot, mutation_header, compression_codec, std::vector(indices_to_recalc.begin(), indices_to_recalc.end()), diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 185961972a8..3625c9bbe26 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -182,6 +182,7 @@ private: /// Override all columns of new part using mutating_stream void mutateAllPartColumns( MergeTreeData::MutableDataPartPtr new_data_part, + const StorageMetadataPtr & metadata_snapshot, const MergeTreeIndices & skip_indices, BlockInputStreamPtr mutating_stream, time_t time_of_mutation, @@ -192,6 +193,7 @@ private: /// Mutate some columns of source part with mutation_stream void mutateSomePartColumns( const MergeTreeDataPartPtr & source_part, + const StorageMetadataPtr & metadata_snapshot, const std::set & indices_to_recalc, const Block & mutation_header, MergeTreeData::MutableDataPartPtr new_data_part, diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index f3da98f0ba3..71501a0e19a 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -192,7 +192,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block return result; } -MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPartition & block_with_partition) +MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPartition & block_with_partition, const StorageMetadataPtr & metadata_snapshot) { Block & block = block_with_partition.block; @@ -302,7 +302,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa auto compression_codec = data.global_context.chooseCompressionCodec(0, 0); const auto & index_factory = MergeTreeIndexFactory::instance(); - MergedBlockOutputStream out(new_data_part, columns, index_factory.getMany(data.getSecondaryIndices()), compression_codec); + MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(data.getSecondaryIndices()), compression_codec); out.writePrefix(); out.writeWithPermutation(block, perm_ptr); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index ffaa227641e..dabdcbd2148 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -45,7 +45,7 @@ public: /** All rows must correspond to same partition. * Returns part with unique name starting with 'tmp_', yet not added to MergeTreeData. */ - MergeTreeData::MutableDataPartPtr writeTempPart(BlockWithPartition & block); + MergeTreeData::MutableDataPartPtr writeTempPart(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot); private: MergeTreeData & data; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index eb0b51235ad..8c73dc39dfb 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -197,7 +197,7 @@ std::vector MergeTreeReadPool::fillPerPartInfo( RangesInDataParts & parts, const bool check_columns) { std::vector per_part_sum_marks; - Block sample_block = data.getSampleBlock(); + Block sample_block = metadata_snapshot->getSampleBlock(); for (const auto i : ext::range(0, parts.size())) { diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp index 81366614988..c47dd7fb669 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp @@ -115,7 +115,7 @@ try auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr - : std::make_unique(data_part, ordered_names, data_part->storage.getSampleBlock()); + : std::make_unique(data_part, ordered_names, metadata_snapshot->getSampleBlock()); task = std::make_unique( data_part, mark_ranges_for_task, part_index_in_query, ordered_names, column_name_set, diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index e32fa70cb97..84c0f44c109 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -72,7 +72,7 @@ try auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr - : std::make_unique(data_part, ordered_names, data_part->storage.getSampleBlock()); + : std::make_unique(data_part, ordered_names, metadata_snapshot->getSampleBlock()); /// will be used to distinguish between PREWHERE and WHERE columns when applying filter const auto & column_names = task_columns.columns.getNames(); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index bce50918ac0..c768678c454 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -15,12 +15,18 @@ namespace ErrorCodes MergedBlockOutputStream::MergedBlockOutputStream( const MergeTreeDataPartPtr & data_part, + const StorageMetadataPtr & metadata_snapshot_, const NamesAndTypesList & columns_list_, const MergeTreeIndices & skip_indices, CompressionCodecPtr default_codec, bool blocks_are_granules_size) : MergedBlockOutputStream( - data_part, columns_list_, skip_indices, default_codec, {}, + data_part, + metadata_snapshot_, + columns_list_, + skip_indices, + default_codec, + {}, data_part->storage.global_context.getSettings().min_bytes_to_use_direct_io, blocks_are_granules_size) { @@ -28,13 +34,14 @@ MergedBlockOutputStream::MergedBlockOutputStream( MergedBlockOutputStream::MergedBlockOutputStream( const MergeTreeDataPartPtr & data_part, + const StorageMetadataPtr & metadata_snapshot_, const NamesAndTypesList & columns_list_, const MergeTreeIndices & skip_indices, CompressionCodecPtr default_codec, const MergeTreeData::DataPart::ColumnToSize & merged_column_to_size, size_t aio_threshold, bool blocks_are_granules_size) - : IMergedBlockOutputStream(data_part) + : IMergedBlockOutputStream(data_part, metadata_snapshot_) , columns_list(columns_list_) { MergeTreeWriterSettings writer_settings(data_part->storage.global_context.getSettings(), diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 5a92977640e..1a8bf9da822 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -15,6 +15,7 @@ class MergedBlockOutputStream final : public IMergedBlockOutputStream public: MergedBlockOutputStream( const MergeTreeDataPartPtr & data_part, + const StorageMetadataPtr & metadata_snapshot_, const NamesAndTypesList & columns_list_, const MergeTreeIndices & skip_indices, CompressionCodecPtr default_codec, @@ -22,6 +23,7 @@ public: MergedBlockOutputStream( const MergeTreeDataPartPtr & data_part, + const StorageMetadataPtr & metadata_snapshot_, const NamesAndTypesList & columns_list_, const MergeTreeIndices & skip_indices, CompressionCodecPtr default_codec, @@ -29,7 +31,7 @@ public: size_t aio_threshold, bool blocks_are_granules_size = false); - Block getHeader() const override { return storage.getSampleBlock(); } + Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } /// If the data is pre-sorted. void write(const Block & block) override; diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index b5eefbe3f0c..b74a8243437 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -10,13 +10,15 @@ namespace ErrorCodes MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( const MergeTreeDataPartPtr & data_part, + const StorageMetadataPtr & metadata_snapshot_, const Block & header_, CompressionCodecPtr default_codec, const std::vector & indices_to_recalc, WrittenOffsetColumns * offset_columns_, const MergeTreeIndexGranularity & index_granularity, const MergeTreeIndexGranularityInfo * index_granularity_info) - : IMergedBlockOutputStream(data_part), header(header_) + : IMergedBlockOutputStream(data_part, metadata_snapshot_) + , header(header_) { const auto & global_settings = data_part->storage.global_context.getSettings(); MergeTreeWriterSettings writer_settings( diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index 2c5024bbcfe..902138ced9d 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -15,6 +15,7 @@ public: /// if you want to serialize elements of Nested data structure in different instances of MergedColumnOnlyOutputStream. MergedColumnOnlyOutputStream( const MergeTreeDataPartPtr & data_part, + const StorageMetadataPtr & metadata_snapshot_, const Block & header_, CompressionCodecPtr default_codec_, const std::vector & indices_to_recalc_, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 1bbc56d940d..8319b0e018d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -31,9 +31,19 @@ namespace ErrorCodes ReplicatedMergeTreeBlockOutputStream::ReplicatedMergeTreeBlockOutputStream( - StorageReplicatedMergeTree & storage_, size_t quorum_, size_t quorum_timeout_ms_, size_t max_parts_per_block_, bool deduplicate_) - : storage(storage_), quorum(quorum_), quorum_timeout_ms(quorum_timeout_ms_), max_parts_per_block(max_parts_per_block_), deduplicate(deduplicate_), - log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)")) + StorageReplicatedMergeTree & storage_, + const StorageMetadataPtr & metadata_snapshot_, + size_t quorum_, + size_t quorum_timeout_ms_, + size_t max_parts_per_block_, + bool deduplicate_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , quorum(quorum_) + , quorum_timeout_ms(quorum_timeout_ms_) + , max_parts_per_block(max_parts_per_block_) + , deduplicate(deduplicate_) + , log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)")) { /// The quorum value `1` has the same meaning as if it is disabled. if (quorum == 1) @@ -43,7 +53,7 @@ ReplicatedMergeTreeBlockOutputStream::ReplicatedMergeTreeBlockOutputStream( Block ReplicatedMergeTreeBlockOutputStream::getHeader() const { - return storage.getSampleBlock(); + return metadata_snapshot->getSampleBlock(); } @@ -128,7 +138,7 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block) /// Write part to the filesystem under temporary name. Calculate a checksum. - MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block); + MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, metadata_snapshot); String block_id; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h index b8650c25c7d..ac169d248c2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h @@ -22,8 +22,12 @@ class StorageReplicatedMergeTree; class ReplicatedMergeTreeBlockOutputStream : public IBlockOutputStream { public: - ReplicatedMergeTreeBlockOutputStream(StorageReplicatedMergeTree & storage_, - size_t quorum_, size_t quorum_timeout_ms_, size_t max_parts_per_block_, + ReplicatedMergeTreeBlockOutputStream( + StorageReplicatedMergeTree & storage_, + const StorageMetadataPtr & metadata_snapshot_, + size_t quorum_, + size_t quorum_timeout_ms_, + size_t max_parts_per_block_, bool deduplicate_); Block getHeader() const override; @@ -55,6 +59,7 @@ private: void commitPart(zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id); StorageReplicatedMergeTree & storage; + StorageMetadataPtr metadata_snapshot; size_t quorum; size_t quorum_timeout_ms; size_t max_parts_per_block; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 42eab838f32..b08e4e93bed 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -167,10 +167,10 @@ Pipes StorageBuffer::read( auto destination_lock = destination->lockStructureForShare( false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - const bool dst_has_same_structure = std::all_of(column_names.begin(), column_names.end(), [this, destination](const String& column_name) + const bool dst_has_same_structure = std::all_of(column_names.begin(), column_names.end(), [metadata_snapshot, destination](const String& column_name) { const auto & dest_columns = destination->getColumns(); - const auto & our_columns = getColumns(); + const auto & our_columns = metadata_snapshot->getColumns(); return dest_columns.hasPhysical(column_name) && dest_columns.get(column_name).type->equals(*our_columns.get(column_name).type); }); @@ -188,7 +188,7 @@ Pipes StorageBuffer::read( else { /// There is a struct mismatch and we need to convert read blocks from the destination table. - const Block header = getSampleBlock(); + const Block header = metadata_snapshot->getSampleBlock(); Names columns_intersection = column_names; Block header_after_adding_defaults = header; const auto & dest_columns = destination->getColumns(); @@ -326,9 +326,14 @@ static void appendBlock(const Block & from, Block & to) class BufferBlockOutputStream : public IBlockOutputStream { public: - explicit BufferBlockOutputStream(StorageBuffer & storage_) : storage(storage_) {} + explicit BufferBlockOutputStream( + StorageBuffer & storage_, + const StorageMetadataPtr & metadata_snapshot_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + {} - Block getHeader() const override { return storage.getSampleBlock(); } + Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } void write(const Block & block) override { @@ -404,6 +409,7 @@ public: } private: StorageBuffer & storage; + StorageMetadataPtr metadata_snapshot; void insertIntoBuffer(const Block & block, StorageBuffer::Buffer & buffer) { @@ -434,9 +440,9 @@ private: }; -BlockOutputStreamPtr StorageBuffer::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) +BlockOutputStreamPtr StorageBuffer::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { - return std::make_shared(*this); + return std::make_shared(*this, metadata_snapshot); } @@ -654,8 +660,8 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl /** We will insert columns that are the intersection set of columns of the buffer table and the subordinate table. * This will support some of the cases (but not all) when the table structure does not match. */ - Block structure_of_destination_table - = allow_materialized ? table->getSampleBlock() : destination_metadata_snapshot->getSampleBlockNonMaterialized(); + Block structure_of_destination_table = allow_materialized ? destination_metadata_snapshot->getSampleBlock() + : destination_metadata_snapshot->getSampleBlockNonMaterialized(); Block block_to_write; for (size_t i : ext::range(0, structure_of_destination_table.columns())) { diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 66066ec3c18..6868f468f2e 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -536,7 +536,7 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const StorageMeta /// DistributedBlockOutputStream will not own cluster, but will own ConnectionPools of the cluster return std::make_shared( - context, *this, createInsertToRemoteTableQuery(remote_database, remote_table, metadata_snapshot->getSampleBlockNonMaterialized()), cluster, + context, *this, metadata_snapshot, createInsertToRemoteTableQuery(remote_database, remote_table, metadata_snapshot->getSampleBlockNonMaterialized()), cluster, insert_sync, timeout); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 07df2b4ec8a..65f36a48170 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -214,9 +214,9 @@ public: using FilesInfoPtr = std::shared_ptr; - static Block getHeader(StorageFile & storage, bool need_path_column, bool need_file_column) + static Block getHeader(const StorageMetadataPtr & metadata_snapshot, bool need_path_column, bool need_file_column) { - auto header = storage.getSampleBlock(); + auto header = metadata_snapshot->getSampleBlock(); /// Note: AddingDefaultsBlockInputStream doesn't change header. @@ -230,12 +230,14 @@ public: StorageFileSource( std::shared_ptr storage_, + const StorageMetadataPtr & metadata_snapshot_, const Context & context_, UInt64 max_block_size_, FilesInfoPtr files_info_, ColumnDefaults column_defaults_) - : SourceWithProgress(getHeader(*storage_, files_info_->need_path_column, files_info_->need_file_column)) + : SourceWithProgress(getHeader(metadata_snapshot_, files_info_->need_path_column, files_info_->need_file_column)) , storage(std::move(storage_)) + , metadata_snapshot(metadata_snapshot_) , files_info(std::move(files_info_)) , column_defaults(std::move(column_defaults_)) , context(context_) @@ -310,7 +312,7 @@ public: read_buf = wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method); reader = FormatFactory::instance().getInput( - storage->format_name, *read_buf, storage->getSampleBlock(), context, max_block_size); + storage->format_name, *read_buf, metadata_snapshot->getSampleBlock(), context, max_block_size); if (!column_defaults.empty()) reader = std::make_shared(reader, column_defaults, context); @@ -357,6 +359,7 @@ public: private: std::shared_ptr storage; + StorageMetadataPtr metadata_snapshot; FilesInfoPtr files_info; String current_path; Block sample_block; @@ -377,7 +380,7 @@ private: Pipes StorageFile::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -414,7 +417,7 @@ Pipes StorageFile::read( for (size_t i = 0; i < num_streams; ++i) pipes.emplace_back(std::make_shared( - this_ptr, context, max_block_size, files_info, getColumns().getDefaults())); + this_ptr, metadata_snapshot, context, max_block_size, files_info, getColumns().getDefaults())); return pipes; } @@ -423,10 +426,14 @@ Pipes StorageFile::read( class StorageFileBlockOutputStream : public IBlockOutputStream { public: - explicit StorageFileBlockOutputStream(StorageFile & storage_, + explicit StorageFileBlockOutputStream( + StorageFile & storage_, + const StorageMetadataPtr & metadata_snapshot_, const CompressionMethod compression_method, const Context & context) - : storage(storage_), lock(storage.rwlock) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , lock(storage.rwlock) { if (storage.use_table_fd) { @@ -446,10 +453,10 @@ public: compression_method, 3); } - writer = FormatFactory::instance().getOutput(storage.format_name, *write_buf, storage.getSampleBlock(), context); + writer = FormatFactory::instance().getOutput(storage.format_name, *write_buf, metadata_snapshot->getSampleBlock(), context); } - Block getHeader() const override { return storage.getSampleBlock(); } + Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } void write(const Block & block) override { @@ -473,6 +480,7 @@ public: private: StorageFile & storage; + StorageMetadataPtr metadata_snapshot; std::unique_lock lock; std::unique_ptr write_buf; BlockOutputStreamPtr writer; @@ -480,13 +488,13 @@ private: BlockOutputStreamPtr StorageFile::write( const ASTPtr & /*query*/, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const Context & context) { if (format_name == "Distributed") throw Exception("Method write is not implemented for Distributed format", ErrorCodes::NOT_IMPLEMENTED); - return std::make_shared(*this, + return std::make_shared(*this, metadata_snapshot, chooseCompressionMethod(paths[0], compression_method), context); } diff --git a/src/Storages/StorageHDFS.cpp b/src/Storages/StorageHDFS.cpp index 77afc4c47c8..ee5a426cedc 100644 --- a/src/Storages/StorageHDFS.cpp +++ b/src/Storages/StorageHDFS.cpp @@ -264,7 +264,7 @@ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, c Pipes StorageHDFS::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, const Context & context_, QueryProcessingStage::Enum /*processed_stage*/, @@ -296,16 +296,16 @@ Pipes StorageHDFS::read( for (size_t i = 0; i < num_streams; ++i) pipes.emplace_back(std::make_shared( - sources_info, uri_without_path, format_name, compression_method, getSampleBlock(), context_, max_block_size)); + sources_info, uri_without_path, format_name, compression_method, metadata_snapshot->getSampleBlock(), context_, max_block_size)); return pipes; } -BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) +BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { return std::make_shared(uri, format_name, - getSampleBlock(), + metadata_snapshot->getSampleBlock(), context, chooseCompressionMethod(uri, compression_method)); } diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index 4117a6b3a37..4430fb11186 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -60,7 +60,7 @@ void StorageInput::setInputStream(BlockInputStreamPtr input_stream_) Pipes StorageInput::read( const Names & /*column_names*/, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -74,7 +74,7 @@ Pipes StorageInput::read( { /// Send structure to the client. query_context.initializeInput(shared_from_this()); - pipes.emplace_back(std::make_shared(query_context, getSampleBlock())); + pipes.emplace_back(std::make_shared(query_context, metadata_snapshot->getSampleBlock())); return pipes; } diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 7d481395ef4..68b974c0dde 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -53,29 +53,33 @@ StorageJoin::StorageJoin( , strictness(strictness_) , overwrite(overwrite_) { + auto metadata_snapshot = getInMemoryMetadataPtr(); for (const auto & key : key_names) - if (!getColumns().hasPhysical(key)) + if (!metadata_snapshot->getColumns().hasPhysical(key)) throw Exception{"Key column (" + key + ") does not exist in table declaration.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE}; table_join = std::make_shared(limits, use_nulls, kind, strictness, key_names); - join = std::make_shared(table_join, getSampleBlock().sortColumns(), overwrite); + join = std::make_shared(table_join, metadata_snapshot->getSampleBlock().sortColumns(), overwrite); restore(); } void StorageJoin::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { + /// TODO(alesap) FIXME + auto metadata_snapshot = getInMemoryMetadataPtr(); Poco::File(path).remove(true); Poco::File(path).createDirectories(); Poco::File(path + "tmp/").createDirectories(); increment = 0; - join = std::make_shared(table_join, getSampleBlock().sortColumns(), overwrite); + join = std::make_shared(table_join, metadata_snapshot->getSampleBlock().sortColumns(), overwrite); } HashJoinPtr StorageJoin::getJoin(std::shared_ptr analyzed_join) const { + auto metadata_snapshot = getInMemoryMetadataPtr(); if (!analyzed_join->sameStrictnessAndKind(strictness, kind)) throw Exception("Table " + getStorageID().getNameForLogs() + " has incompatible type of JOIN.", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN); @@ -89,7 +93,7 @@ HashJoinPtr StorageJoin::getJoin(std::shared_ptr analyzed_join) const /// Some HACK to remove wrong names qualifiers: table.column -> column. analyzed_join->setRightKeys(key_names); - HashJoinPtr join_clone = std::make_shared(analyzed_join, getSampleBlock().sortColumns()); + HashJoinPtr join_clone = std::make_shared(analyzed_join, metadata_snapshot->getSampleBlock().sortColumns()); join_clone->reuseJoinedData(*join); return join_clone; } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index a09a99b30e1..79cc3e5bf68 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -114,10 +114,12 @@ private: class LogBlockOutputStream final : public IBlockOutputStream { public: - explicit LogBlockOutputStream(StorageLog & storage_) - : storage(storage_), - lock(storage.rwlock), - marks_stream(storage.disk->writeFile(storage.marks_file_path, 4096, WriteMode::Rewrite)) + explicit LogBlockOutputStream(StorageLog & storage_, const StorageMetadataPtr & metadata_snapshot_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , lock(storage.rwlock) + , marks_stream( + storage.disk->writeFile(storage.marks_file_path, 4096, WriteMode::Rewrite)) { } @@ -133,12 +135,13 @@ public: } } - Block getHeader() const override { return storage.getSampleBlock(); } + Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } void write(const Block & block) override; void writeSuffix() override; private: StorageLog & storage; + StorageMetadataPtr metadata_snapshot; std::unique_lock lock; bool done = false; @@ -621,10 +624,10 @@ Pipes StorageLog::read( return pipes; } -BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) +BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { loadMarks(); - return std::make_shared(*this); + return std::make_shared(*this, metadata_snapshot); } CheckResults StorageLog::checkData(const ASTPtr & /* query */, const Context & /* context */) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index a387eadabe0..e96a48efc9e 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -244,10 +244,11 @@ void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & command } } -void StorageMaterializedView::alterPartition(const ASTPtr & query, const PartitionCommands &commands, const Context &context) +void StorageMaterializedView::alterPartition( + const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) { checkStatementCanBeForwarded(); - getTargetTable()->alterPartition(query, commands, context); + getTargetTable()->alterPartition(query, metadata_snapshot, commands, context); } void StorageMaterializedView::mutate(const MutationCommands & commands, const Context & context) diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 42fe186a068..672be800c8f 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -43,7 +43,7 @@ public: void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override; - void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) override; + void alterPartition(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) override; void mutate(const MutationCommands & commands, const Context & context) override; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index c5a3c20bb0c..b3aed291735 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -168,7 +168,9 @@ Pipes StorageMerge::read( if (selected_tables.empty()) /// FIXME: do we support sampling in this case? return createSources( - query_info, processed_stage, max_block_size, header, {}, real_column_names, modified_context, 0, has_table_virtual_column); + metadata_snapshot, query_info, processed_stage, + max_block_size, header, {}, real_column_names, + modified_context, 0, has_table_virtual_column); size_t tables_count = selected_tables.size(); Float64 num_streams_multiplier = std::min(unsigned(tables_count), std::max(1U, unsigned(context.getSettingsRef().max_streams_multiplier_for_merge_tables))); @@ -207,8 +209,9 @@ Pipes StorageMerge::read( throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED); auto source_pipes = createSources( - query_info, processed_stage, max_block_size, header, table, real_column_names, modified_context, - current_streams, has_table_virtual_column); + metadata_snapshot, query_info, processed_stage, + max_block_size, header, table, real_column_names, modified_context, + current_streams, has_table_virtual_column); for (auto & pipe : source_pipes) res.emplace_back(std::move(pipe)); @@ -220,10 +223,17 @@ Pipes StorageMerge::read( return narrowPipes(std::move(res), num_streams); } -Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage, - const UInt64 max_block_size, const Block & header, const StorageWithLockAndName & storage_with_lock, +Pipes StorageMerge::createSources( + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const QueryProcessingStage::Enum & processed_stage, + const UInt64 max_block_size, + const Block & header, + const StorageWithLockAndName & storage_with_lock, Names & real_column_names, - const std::shared_ptr & modified_context, size_t streams_num, bool has_table_virtual_column, + const std::shared_ptr & modified_context, + size_t streams_num, + bool has_table_virtual_column, bool concat_streams) { const auto & [storage, struct_lock, table_name] = storage_with_lock; @@ -244,7 +254,6 @@ Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const Quer return pipes; } - auto metadata_snapshot = storage->getInMemoryMetadataPtr(); auto storage_stage = storage->getQueryProcessingStage(*modified_context, QueryProcessingStage::Complete, query_info.query); if (processed_stage <= storage_stage) { @@ -295,7 +304,7 @@ Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const Quer /// Subordinary tables could have different but convertible types, like numeric types of different width. /// We must return streams with structure equals to structure of Merge table. - convertingSourceStream(header, *modified_context, modified_query_info.query, pipe, processed_stage); + convertingSourceStream(header, metadata_snapshot, *modified_context, modified_query_info.query, pipe, processed_stage); pipe.addTableLock(struct_lock); pipe.addInterpreterContext(modified_context); @@ -430,8 +439,13 @@ Block StorageMerge::getQueryHeader( throw Exception("Logical Error: unknown processed stage.", ErrorCodes::LOGICAL_ERROR); } -void StorageMerge::convertingSourceStream(const Block & header, const Context & context, ASTPtr & query, - Pipe & pipe, QueryProcessingStage::Enum processed_stage) +void StorageMerge::convertingSourceStream( + const Block & header, + const StorageMetadataPtr & metadata_snapshot, + const Context & context, + ASTPtr & query, + Pipe & pipe, + QueryProcessingStage::Enum processed_stage) { Block before_block_header = pipe.getHeader(); pipe.addSimpleTransform(std::make_shared(before_block_header, header, ConvertingTransform::MatchColumnsMode::Name)); @@ -450,7 +464,7 @@ void StorageMerge::convertingSourceStream(const Block & header, const Context & /// So we need to throw exception. if (!header_column.type->equals(*before_column.type.get()) && processed_stage > QueryProcessingStage::FetchColumns) { - NamesAndTypesList source_columns = getSampleBlock().getNamesAndTypesList(); + NamesAndTypesList source_columns = metadata_snapshot->getSampleBlock().getNamesAndTypesList(); auto virtual_column = *getVirtuals().tryGetByName("_table"); source_columns.emplace_back(NameAndTypePair{virtual_column.name, virtual_column.type}); auto syntax_result = SyntaxAnalyzer(context).analyze(where_expression, source_columns); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 350f7a124fe..14bf83f8534 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -82,14 +82,22 @@ protected: QueryProcessingStage::Enum processed_stage); Pipes createSources( - const SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage, - const UInt64 max_block_size, const Block & header, const StorageWithLockAndName & storage_with_lock, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const QueryProcessingStage::Enum & processed_stage, + const UInt64 max_block_size, + const Block & header, + const StorageWithLockAndName & storage_with_lock, Names & real_column_names, - const std::shared_ptr & modified_context, size_t streams_num, bool has_table_virtual_column, + const std::shared_ptr & modified_context, + size_t streams_num, + bool has_table_virtual_column, bool concat_streams = false); - void convertingSourceStream(const Block & header, const Context & context, ASTPtr & query, - Pipe & pipe, QueryProcessingStage::Enum processed_stage); + void convertingSourceStream( + const Block & header, const StorageMetadataPtr & metadata_snapshot, + const Context & context, ASTPtr & query, + Pipe & pipe, QueryProcessingStage::Enum processed_stage); }; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9c37cdd2b7c..cfa5c34bece 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -198,9 +198,9 @@ std::optional StorageMergeTree::totalBytes() const return getTotalActiveSizeInBytes(); } -BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) +BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context) { - return std::make_shared(*this, context.getSettingsRef().max_partitions_per_insert_block); + return std::make_shared(*this, metadata_snapshot, context.getSettingsRef().max_partitions_per_insert_block); } void StorageMergeTree::checkTableCanBeDropped() const @@ -1017,7 +1017,8 @@ bool StorageMergeTree::optimize( return true; } -void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) +void StorageMergeTree::alterPartition( + const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) { for (const PartitionCommand & command : commands) { @@ -1085,7 +1086,7 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma break; default: - IStorage::alterPartition(query, commands, context); // should throw an exception. + IStorage::alterPartition(query, metadata_snapshot, commands, context); // should throw an exception. } } } @@ -1126,7 +1127,8 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons } -void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & context) +void StorageMergeTree::attachPartition( + const ASTPtr & partition, bool attach_part, const Context & context) { // TODO: should get some locks to prevent race with 'alter … modify column' diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 679726826d4..4b6da58572b 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -55,7 +55,11 @@ public: */ bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override; - void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) override; + void alterPartition( + const ASTPtr & query, + const StorageMetadataPtr & /* metadata_snapshot */, + const PartitionCommands & commands, + const Context & context) override; void mutate(const MutationCommands & commands, const Context & context) override; diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index dce9e0f38ec..b1262771d21 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -65,7 +65,7 @@ StorageMySQL::StorageMySQL( Pipes StorageMySQL::read( const Names & column_names_, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info_, const Context & context_, QueryProcessingStage::Enum /*processed_stage*/, @@ -74,12 +74,17 @@ Pipes StorageMySQL::read( { check(column_names_); String query = transformQueryForExternalDatabase( - query_info_, getColumns().getOrdinary(), IdentifierQuotingStyle::BackticksMySQL, remote_database_name, remote_table_name, context_); + query_info_, + metadata_snapshot->getColumns().getOrdinary(), + IdentifierQuotingStyle::BackticksMySQL, + remote_database_name, + remote_table_name, + context_); Block sample_block; for (const String & column_name : column_names_) { - auto column_data = getColumns().getPhysical(column_name); + auto column_data = metadata_snapshot->getColumns().getPhysical(column_name); sample_block.insert({ column_data.type, column_data.name }); } @@ -95,12 +100,15 @@ Pipes StorageMySQL::read( class StorageMySQLBlockOutputStream : public IBlockOutputStream { public: - explicit StorageMySQLBlockOutputStream(const StorageMySQL & storage_, + explicit StorageMySQLBlockOutputStream( + const StorageMySQL & storage_, + const StorageMetadataPtr & metadata_snapshot_, const std::string & remote_database_name_, const std::string & remote_table_name_, const mysqlxx::PoolWithFailover::Entry & entry_, const size_t & mysql_max_rows_to_insert) : storage{storage_} + , metadata_snapshot{metadata_snapshot_} , remote_database_name{remote_database_name_} , remote_table_name{remote_table_name_} , entry{entry_} @@ -108,7 +116,7 @@ public: { } - Block getHeader() const override { return storage.getSampleBlock(); } + Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } void write(const Block & block) override { @@ -136,7 +144,7 @@ public: sqlbuf << backQuoteMySQL(remote_database_name) << "." << backQuoteMySQL(remote_table_name); sqlbuf << " (" << dumpNamesWithBackQuote(block) << ") VALUES "; - auto writer = FormatFactory::instance().getOutput("Values", sqlbuf, storage.getSampleBlock(), storage.global_context); + auto writer = FormatFactory::instance().getOutput("Values", sqlbuf, metadata_snapshot->getSampleBlock(), storage.global_context); writer->write(block); if (!storage.on_duplicate_clause.empty()) @@ -192,6 +200,7 @@ public: private: const StorageMySQL & storage; + StorageMetadataPtr metadata_snapshot; std::string remote_database_name; std::string remote_table_name; mysqlxx::PoolWithFailover::Entry entry; @@ -199,9 +208,9 @@ private: }; -BlockOutputStreamPtr StorageMySQL::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) +BlockOutputStreamPtr StorageMySQL::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context) { - return std::make_shared(*this, remote_database_name, remote_table_name, pool.get(), context.getSettingsRef().mysql_max_rows_to_insert); + return std::make_shared(*this, metadata_snapshot, remote_database_name, remote_table_name, pool.get(), context.getSettingsRef().mysql_max_rows_to_insert); } void registerStorageMySQL(StorageFactory & factory) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a6f84ffe4df..8ae5a887013 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3449,7 +3449,7 @@ void StorageReplicatedMergeTree::assertNotReadonly() const } -BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) +BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context) { const auto storage_settings_ptr = getSettings(); assertNotReadonly(); @@ -3457,8 +3457,7 @@ BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const Settings & query_settings = context.getSettingsRef(); bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; - return std::make_shared(*this, - query_settings.insert_quorum, query_settings.insert_quorum_timeout.totalMilliseconds(), query_settings.max_partitions_per_insert_block, deduplicate); + return std::make_shared(*this, metadata_snapshot, query_settings.insert_quorum, query_settings.insert_quorum_timeout.totalMilliseconds(), query_settings.max_partitions_per_insert_block, deduplicate); } @@ -3830,7 +3829,11 @@ void StorageReplicatedMergeTree::alter( } } -void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & query_context) +void StorageReplicatedMergeTree::alterPartition( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + const PartitionCommands & commands, + const Context & query_context) { for (const PartitionCommand & command : commands) { @@ -3846,7 +3849,7 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part break; case PartitionCommand::ATTACH_PARTITION: - attachPartition(command.partition, command.part, query_context); + attachPartition(command.partition, metadata_snapshot, command.part, query_context); break; case PartitionCommand::MOVE_PARTITION: { @@ -4014,7 +4017,7 @@ void StorageReplicatedMergeTree::truncate(const ASTPtr & query, const Context & } -void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & query_context) +void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool attach_part, const Context & query_context) { // TODO: should get some locks to prevent race with 'alter … modify column' @@ -4023,7 +4026,7 @@ void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool PartsTemporaryRename renamed_parts(*this, "detached/"); MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts); - ReplicatedMergeTreeBlockOutputStream output(*this, 0, 0, 0, false); /// TODO Allow to use quorum here. + ReplicatedMergeTreeBlockOutputStream output(*this, metadata_snapshot, 0, 0, 0, false); /// TODO Allow to use quorum here. for (size_t i = 0; i < loaded_parts.size(); ++i) { String old_name = loaded_parts[i]->name; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 5fcfd98e71d..50530070d19 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -105,7 +105,11 @@ public: void alter(const AlterCommands & params, const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) override; - void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & query_context) override; + void alterPartition( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + const PartitionCommands & commands, + const Context & query_context) override; void mutate(const MutationCommands & commands, const Context & context) override; void waitMutation(const String & znode_name, size_t mutations_sync) const; @@ -527,7 +531,7 @@ private: // Partition helpers void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & query_context); - void attachPartition(const ASTPtr & partition, bool part, const Context & query_context); + void attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & query_context); void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & query_context); void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & query_context); void fetchPartition(const ASTPtr & partition, const String & from, const Context & query_context); diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index cddd4657cd1..235251c0761 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -32,15 +32,18 @@ namespace ErrorCodes class SetOrJoinBlockOutputStream : public IBlockOutputStream { public: - SetOrJoinBlockOutputStream(StorageSetOrJoinBase & table_, - const String & backup_path_, const String & backup_tmp_path_, const String & backup_file_name_); + SetOrJoinBlockOutputStream( + StorageSetOrJoinBase & table_, const StorageMetadataPtr & metadata_snapshot_, + const String & backup_path_, const String & backup_tmp_path_, + const String & backup_file_name_); - Block getHeader() const override { return table.getSampleBlock(); } + Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } void write(const Block & block) override; void writeSuffix() override; private: StorageSetOrJoinBase & table; + StorageMetadataPtr metadata_snapshot; String backup_path; String backup_tmp_path; String backup_file_name; @@ -50,14 +53,20 @@ private: }; -SetOrJoinBlockOutputStream::SetOrJoinBlockOutputStream(StorageSetOrJoinBase & table_, - const String & backup_path_, const String & backup_tmp_path_, const String & backup_file_name_) - : table(table_), - backup_path(backup_path_), backup_tmp_path(backup_tmp_path_), - backup_file_name(backup_file_name_), - backup_buf(backup_tmp_path + backup_file_name), - compressed_backup_buf(backup_buf), - backup_stream(compressed_backup_buf, 0, table.getSampleBlock()) +SetOrJoinBlockOutputStream::SetOrJoinBlockOutputStream( + StorageSetOrJoinBase & table_, + const StorageMetadataPtr & metadata_snapshot_, + const String & backup_path_, + const String & backup_tmp_path_, + const String & backup_file_name_) + : table(table_) + , metadata_snapshot(metadata_snapshot_) + , backup_path(backup_path_) + , backup_tmp_path(backup_tmp_path_) + , backup_file_name(backup_file_name_) + , backup_buf(backup_tmp_path + backup_file_name) + , compressed_backup_buf(backup_buf) + , backup_stream(compressed_backup_buf, 0, metadata_snapshot->getSampleBlock()) { } @@ -81,10 +90,10 @@ void SetOrJoinBlockOutputStream::writeSuffix() } -BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) +BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { UInt64 id = ++increment; - return std::make_shared(*this, path, path + "tmp/", toString(id) + ".bin"); + return std::make_shared(*this, metadata_snapshot, path, path + "tmp/", toString(id) + ".bin"); } @@ -119,7 +128,8 @@ StorageSet::StorageSet( : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_}, set(std::make_shared(SizeLimits(), false, true)) { - Block header = getSampleBlock(); + + Block header = getInMemoryMetadataPtr()->getSampleBlock(); header = header.sortColumns(); set->setHeader(header); @@ -134,11 +144,12 @@ size_t StorageSet::getSize() const { return set->getTotalRowCount(); } void StorageSet::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { + auto metadata_snapshot = getInMemoryMetadataPtr(); Poco::File(path).remove(true); Poco::File(path).createDirectories(); Poco::File(path + "tmp/").createDirectories(); - Block header = getSampleBlock(); + Block header = metadata_snapshot->getSampleBlock(); header = header.sortColumns(); increment = 0; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index b0c5bcfd669..4d9f08a60b7 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -155,15 +155,17 @@ private: class StripeLogBlockOutputStream final : public IBlockOutputStream { public: - explicit StripeLogBlockOutputStream(StorageStripeLog & storage_) - : storage(storage_), lock(storage.rwlock), - data_out_file(storage.table_path + "data.bin"), - data_out_compressed(storage.disk->writeFile(data_out_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append)), - data_out(*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), storage.max_compress_block_size), - index_out_file(storage.table_path + "index.mrk"), - index_out_compressed(storage.disk->writeFile(index_out_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append)), - index_out(*index_out_compressed), - block_out(data_out, 0, storage.getSampleBlock(), false, &index_out, storage.disk->getFileSize(data_out_file)) + explicit StripeLogBlockOutputStream(StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , lock(storage.rwlock) + , data_out_file(storage.table_path + "data.bin") + , data_out_compressed(storage.disk->writeFile(data_out_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append)) + , data_out(*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), storage.max_compress_block_size) + , index_out_file(storage.table_path + "index.mrk") + , index_out_compressed(storage.disk->writeFile(index_out_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append)) + , index_out(*index_out_compressed) + , block_out(data_out, 0, metadata_snapshot->getSampleBlock(), false, &index_out, storage.disk->getFileSize(data_out_file)) { } @@ -179,7 +181,7 @@ public: } } - Block getHeader() const override { return storage.getSampleBlock(); } + Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } void write(const Block & block) override { @@ -205,6 +207,7 @@ public: private: StorageStripeLog & storage; + StorageMetadataPtr metadata_snapshot; std::unique_lock lock; String data_out_file; @@ -311,9 +314,9 @@ Pipes StorageStripeLog::read( } -BlockOutputStreamPtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) +BlockOutputStreamPtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { - return std::make_shared(*this); + return std::make_shared(*this, metadata_snapshot); } diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 42b70f716f4..ba524c7761e 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -109,8 +109,8 @@ private: class TinyLogBlockOutputStream final : public IBlockOutputStream { public: - explicit TinyLogBlockOutputStream(StorageTinyLog & storage_) - : storage(storage_), lock(storage_.rwlock) + explicit TinyLogBlockOutputStream(StorageTinyLog & storage_, const StorageMetadataPtr & metadata_snapshot_) + : storage(storage_), metadata_snapshot(metadata_snapshot_), lock(storage_.rwlock) { } @@ -126,13 +126,14 @@ public: } } - Block getHeader() const override { return storage.getSampleBlock(); } + Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } void write(const Block & block) override; void writeSuffix() override; private: StorageTinyLog & storage; + StorageMetadataPtr metadata_snapshot; std::unique_lock lock; bool done = false; @@ -394,7 +395,7 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const Storage Pipes StorageTinyLog::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -408,15 +409,15 @@ Pipes StorageTinyLog::read( // When reading, we lock the entire storage, because we only have one file // per column and can't modify it concurrently. pipes.emplace_back(std::make_shared( - max_block_size, Nested::collect(getColumns().getAllPhysical().addTypes(column_names)), *this, context.getSettingsRef().max_read_buffer_size)); + max_block_size, Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)), *this, context.getSettingsRef().max_read_buffer_size)); return pipes; } -BlockOutputStreamPtr StorageTinyLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) +BlockOutputStreamPtr StorageTinyLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { - return std::make_shared(*this); + return std::make_shared(*this, metadata_snapshot); } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 6cea7115066..949d922b611 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -186,10 +186,10 @@ Pipes IStorageURLBase::read( return pipes; } -BlockOutputStreamPtr IStorageURLBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & /*context*/) +BlockOutputStreamPtr IStorageURLBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { return std::make_shared( - uri, format_name, getSampleBlock(), context_global, + uri, format_name, metadata_snapshot->getSampleBlock(), context_global, ConnectionTimeouts::getHTTPTimeouts(context_global), chooseCompressionMethod(uri.toString(), compression_method)); } diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 7644f62b96d..de7e1a0e933 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -30,7 +30,7 @@ public: Pipes read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -39,7 +39,7 @@ public: { check(column_names); - Block sample_block = getSampleBlock(); + Block sample_block = metadata_snapshot->getSampleBlock(); MutableColumns res_columns = sample_block.cloneEmptyColumns(); fillData(res_columns, context, query_info); diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 646a5434b64..14a59da1bf9 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -242,7 +242,7 @@ private: Pipes StorageSystemColumns::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -255,7 +255,7 @@ Pipes StorageSystemColumns::read( NameSet names_set(column_names.begin(), column_names.end()); - Block sample_block = getSampleBlock(); + Block sample_block = metadata_snapshot->getSampleBlock(); Block header; std::vector columns_mask(sample_block.columns()); diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 3d24d90bbef..7228651d140 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -47,7 +47,7 @@ protected: Pipes read( const Names & /* column_names */, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -57,7 +57,7 @@ protected: StoragesInfoStream stream(query_info, context); /// Create the result. - Block block = getSampleBlock(); + Block block = metadata_snapshot->getSampleBlock(); MutableColumns new_columns = block.cloneEmptyColumns(); while (StoragesInfo info = stream.next()) diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index 36fde616bd4..d13ea29804d 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -28,7 +28,7 @@ StorageSystemDisks::StorageSystemDisks(const std::string & name_) Pipes StorageSystemDisks::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -63,7 +63,7 @@ Pipes StorageSystemDisks::read( Chunk chunk(std::move(res_columns), num_rows); Pipes pipes; - pipes.emplace_back(std::make_shared(getSampleBlock(), std::move(chunk))); + pipes.emplace_back(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); return pipes; } diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index e599bbb19e3..4f99e1e8c6a 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -225,7 +225,7 @@ StoragesInfo StoragesInfoStream::next() Pipes StorageSystemPartsBase::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -238,7 +238,7 @@ Pipes StorageSystemPartsBase::read( /// Create the result. - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); + MutableColumns res_columns = metadata_snapshot->getSampleBlock().cloneEmptyColumns(); if (has_state_column) res_columns.push_back(ColumnString::create()); @@ -247,7 +247,7 @@ Pipes StorageSystemPartsBase::read( processNextStorage(res_columns, info, has_state_column); } - Block header = getSampleBlock(); + Block header = metadata_snapshot->getSampleBlock(); if (has_state_column) header.insert(ColumnWithTypeAndName(std::make_shared(), "_state")); diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 24861fcbd6a..8fb6a89ddd1 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -59,7 +59,7 @@ StorageSystemReplicas::StorageSystemReplicas(const std::string & name_) Pipes StorageSystemReplicas::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -146,7 +146,7 @@ Pipes StorageSystemReplicas::read( col_engine = filtered_block.getByName("engine").column; } - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); + MutableColumns res_columns = metadata_snapshot->getSampleBlock().cloneEmptyColumns(); for (size_t i = 0, size = col_database->size(); i < size; ++i) { @@ -187,7 +187,7 @@ Pipes StorageSystemReplicas::read( res_columns[col_num++]->insert(status.zookeeper_exception); } - Block header = getSampleBlock(); + Block header = metadata_snapshot->getSampleBlock(); Columns fin_columns; fin_columns.reserve(res_columns.size()); @@ -203,7 +203,7 @@ Pipes StorageSystemReplicas::read( Chunk chunk(std::move(fin_columns), num_rows); Pipes pipes; - pipes.emplace_back(std::make_shared(getSampleBlock(), std::move(chunk))); + pipes.emplace_back(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); return pipes; } diff --git a/src/Storages/System/StorageSystemStoragePolicies.cpp b/src/Storages/System/StorageSystemStoragePolicies.cpp index a80747c1fa1..44252a788b9 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.cpp +++ b/src/Storages/System/StorageSystemStoragePolicies.cpp @@ -32,7 +32,7 @@ StorageSystemStoragePolicies::StorageSystemStoragePolicies(const std::string & n Pipes StorageSystemStoragePolicies::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -78,7 +78,7 @@ Pipes StorageSystemStoragePolicies::read( Chunk chunk(std::move(res_columns), num_rows); Pipes pipes; - pipes.emplace_back(std::make_shared(getSampleBlock(), std::move(chunk))); + pipes.emplace_back(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); return pipes; } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index f04b3ea20c9..b33886ce179 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -448,7 +448,7 @@ private: Pipes StorageSystemTables::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, @@ -461,7 +461,7 @@ Pipes StorageSystemTables::read( NameSet names_set(column_names.begin(), column_names.end()); - Block sample_block = getSampleBlock(); + Block sample_block = metadata_snapshot->getSampleBlock(); Block res_block; std::vector columns_mask(sample_block.columns()); From ccc2bda66666f1ac548c04f28dcdd465b00d20d5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Jun 2020 19:55:04 +0300 Subject: [PATCH 070/211] getConstraints() in StorageInMemoryMetadata (suspicious commit, but pretend to work) --- src/Interpreters/InterpreterCreateQuery.cpp | 7 +-- src/Interpreters/InterpreterInsertQuery.cpp | 4 +- src/Storages/IStorage.cpp | 5 -- src/Storages/IStorage.h | 3 +- .../ReplicatedMergeTreeTableMetadata.cpp | 8 +-- .../ReplicatedMergeTreeTableMetadata.h | 2 +- src/Storages/StorageInMemoryMetadata.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 51 ++++++++++--------- src/Storages/StorageReplicatedMergeTree.h | 6 +-- 9 files changed, 43 insertions(+), 44 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 5d8c43aed0d..bb82c94a764 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -430,14 +430,15 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS /// as_storage->getColumns() and setEngine(...) must be called under structure lock of other_table for CREATE ... AS other_table. as_storage_lock = as_storage->lockStructureForShare( false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - properties.columns = as_storage->getColumns(); + auto as_storage_metadata = as_storage->getInMemoryMetadataPtr(); + properties.columns = as_storage_metadata->getColumns(); /// Secondary indices make sense only for MergeTree family of storage engines. /// We should not copy them for other storages. if (create.storage && endsWith(create.storage->engine->name, "MergeTree")) - properties.indices = as_storage->getSecondaryIndices(); + properties.indices = as_storage_metadata->getSecondaryIndices(); - properties.constraints = as_storage->getConstraints(); + properties.constraints = as_storage_metadata->getConstraints(); } else if (create.select) { diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 443e2714ec7..e7fdf80e297 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -237,9 +237,9 @@ BlockIO InterpreterInsertQuery::execute() /// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order. /// Checking constraints. It must be done after calculation of all defaults, so we can check them on calculated columns. - if (const auto & constraints = table->getConstraints(); !constraints.empty()) + if (const auto & constraints = metadata_snapshot->getConstraints(); !constraints.empty()) out = std::make_shared( - query.table_id, out, out->getHeader(), table->getConstraints(), context); + query.table_id, out, out->getHeader(), metadata_snapshot->getConstraints(), context); /// Actually we don't know structure of input blocks from query/table, /// because some clients break insertion protocol (columns != header) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 43e9a5dd040..1fb3e095229 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -47,11 +47,6 @@ bool IStorage::hasSecondaryIndices() const return !metadata->secondary_indices.empty(); } -const ConstraintsDescription & IStorage::getConstraints() const -{ - return metadata->constraints; -} - namespace { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index df1e1685a2e..e45d6a1128b 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -130,7 +130,7 @@ public: virtual bool hasEvenlyDistributedRead() const { return false; } /// Returns true if there is set table TTL, any column TTL or any move TTL. - virtual bool hasAnyTTL() const { return hasAnyColumnTTL() || hasAnyTableTTL(); } + bool hasAnyTTL() const { return hasAnyColumnTTL() || hasAnyTableTTL(); } /// Optional size information of each physical column. /// Currently it's only used by the MergeTree family for query optimizations. @@ -144,7 +144,6 @@ public: /// thread-unsafe part. lockStructure must be acquired /// Has at least one non primary index bool hasSecondaryIndices() const; - const ConstraintsDescription & getConstraints() const; /// Storage settings ASTPtr getSettingsChanges() const; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 2444affdbff..820f41326f1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -23,7 +23,7 @@ static String formattedAST(const ASTPtr & ast) return ss.str(); } -ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTreeData & data) +ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTreeData & data, const StorageMetadataPtr & metadata_snapshot) { if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos]; @@ -53,15 +53,15 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) partition_key = formattedAST(data.getPartitionKey().expression_list_ast); - ttl_table = formattedAST(data.getTableTTLs().definition_ast); + ttl_table = formattedAST(metadata_snapshot->getTableTTLs().definition_ast); - skip_indices = data.getSecondaryIndices().toString(); + skip_indices = metadata_snapshot->getSecondaryIndices().toString(); if (data.canUseAdaptiveGranularity()) index_granularity_bytes = data_settings->index_granularity_bytes; else index_granularity_bytes = 0; - constraints = data.getConstraints().toString(); + constraints = metadata_snapshot->getConstraints().toString(); } void ReplicatedMergeTreeTableMetadata::write(WriteBuffer & out) const diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h index 280a8c8b403..f7174140ee1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h @@ -32,7 +32,7 @@ struct ReplicatedMergeTreeTableMetadata UInt64 index_granularity_bytes; ReplicatedMergeTreeTableMetadata() = default; - explicit ReplicatedMergeTreeTableMetadata(const MergeTreeData & data); + explicit ReplicatedMergeTreeTableMetadata(const MergeTreeData & data, const StorageMetadataPtr & metadata_snapshot); void read(ReadBuffer & in); static ReplicatedMergeTreeTableMetadata parse(const String & s); diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index cce3911370d..359d561cd1f 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -290,4 +290,5 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns(const Names & column_nam return res; } + } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8ae5a887013..170a77e3508 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -248,6 +248,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( return; } + auto metadata_snapshot = getInMemoryMetadataPtr(); + if (!attach) { if (!getDataParts().empty()) @@ -255,21 +257,21 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( try { - bool is_first_replica = createTableIfNotExists(); + bool is_first_replica = createTableIfNotExists(metadata_snapshot); /// We have to check granularity on other replicas. If it's fixed we /// must create our new replica with fixed granularity and store this /// information in /replica/metadata. other_replicas_fixed_granularity = checkFixedGranualrityInZookeeper(); - checkTableStructure(zookeeper_path); + checkTableStructure(zookeeper_path, metadata_snapshot); Coordination::Stat metadata_stat; current_zookeeper->get(zookeeper_path + "/metadata", &metadata_stat); metadata_version = metadata_stat.version; if (!is_first_replica) - createReplica(); + createReplica(metadata_snapshot); } catch (...) { @@ -288,11 +290,11 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( { /// We have to check shared node granularity before we create ours. other_replicas_fixed_granularity = checkFixedGranualrityInZookeeper(); - ReplicatedMergeTreeTableMetadata current_metadata(*this); + ReplicatedMergeTreeTableMetadata current_metadata(*this, metadata_snapshot); current_zookeeper->createOrUpdate(replica_path + "/metadata", current_metadata.toString(), zkutil::CreateMode::Persistent); } - checkTableStructure(replica_path); + checkTableStructure(replica_path, metadata_snapshot); checkParts(skip_sanity_checks); if (current_zookeeper->exists(replica_path + "/metadata_version")) @@ -418,7 +420,7 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes() } -bool StorageReplicatedMergeTree::createTableIfNotExists() +bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot) { auto zookeeper = getZooKeeper(); zookeeper->createAncestors(zookeeper_path); @@ -483,7 +485,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists() LOG_DEBUG(log, "Creating table {}", zookeeper_path); /// We write metadata of table so that the replicas can check table parameters with them. - String metadata_str = ReplicatedMergeTreeTableMetadata(*this).toString(); + String metadata_str = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(); Coordination::Requests ops; ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); @@ -552,7 +554,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists() throw Exception("Cannot create table, because it is created concurrently every time or because of logical error", ErrorCodes::LOGICAL_ERROR); } -void StorageReplicatedMergeTree::createReplica() +void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metadata_snapshot) { auto zookeeper = getZooKeeper(); @@ -588,7 +590,7 @@ void StorageReplicatedMergeTree::createReplica() zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/is_lost", is_lost_value, zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata", ReplicatedMergeTreeTableMetadata(*this).toString(), + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata", ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(), zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/columns", getColumns().toString(), zkutil::CreateMode::Persistent)); @@ -728,11 +730,11 @@ void StorageReplicatedMergeTree::drop() /** Verify that list of columns and table storage_settings_ptr match those specified in ZK (/ metadata). * If not, throw an exception. */ -void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_prefix) +void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot) { auto zookeeper = getZooKeeper(); - ReplicatedMergeTreeTableMetadata old_metadata(*this); + ReplicatedMergeTreeTableMetadata old_metadata(*this, metadata_snapshot); Coordination::Stat metadata_stat; String metadata_str = zookeeper->get(zookeeper_prefix + "/metadata", &metadata_stat); @@ -3624,7 +3626,7 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally."); - auto metadata_diff = ReplicatedMergeTreeTableMetadata(*this).checkAndFindDiff(metadata_from_entry); + auto metadata_diff = ReplicatedMergeTreeTableMetadata(*this, getInMemoryMetadataPtr()).checkAndFindDiff(metadata_from_entry); setTableStructure(std::move(columns_from_entry), metadata_diff); metadata_version = entry.alter_version; @@ -3683,24 +3685,24 @@ void StorageReplicatedMergeTree::alter( throw Exception("Can't ALTER readonly table", ErrorCodes::TABLE_IS_READ_ONLY); - StorageInMemoryMetadata current_metadata = getInMemoryMetadata(); + auto current_metadata = getInMemoryMetadataPtr(); - StorageInMemoryMetadata future_metadata = current_metadata; + StorageInMemoryMetadata future_metadata = *current_metadata; params.apply(future_metadata, query_context); - ReplicatedMergeTreeTableMetadata future_metadata_in_zk(*this); - if (ast_to_str(future_metadata.sorting_key.definition_ast) != ast_to_str(current_metadata.sorting_key.definition_ast)) + ReplicatedMergeTreeTableMetadata future_metadata_in_zk(*this, current_metadata); + if (ast_to_str(future_metadata.sorting_key.definition_ast) != ast_to_str(current_metadata->sorting_key.definition_ast)) future_metadata_in_zk.sorting_key = serializeAST(*future_metadata.sorting_key.expression_list_ast); - if (ast_to_str(future_metadata.table_ttl.definition_ast) != ast_to_str(current_metadata.table_ttl.definition_ast)) + if (ast_to_str(future_metadata.table_ttl.definition_ast) != ast_to_str(current_metadata->table_ttl.definition_ast)) future_metadata_in_zk.ttl_table = serializeAST(*future_metadata.table_ttl.definition_ast); String new_indices_str = future_metadata.secondary_indices.toString(); - if (new_indices_str != current_metadata.secondary_indices.toString()) + if (new_indices_str != current_metadata->secondary_indices.toString()) future_metadata_in_zk.skip_indices = new_indices_str; String new_constraints_str = future_metadata.constraints.toString(); - if (new_constraints_str != current_metadata.constraints.toString()) + if (new_constraints_str != current_metadata->constraints.toString()) future_metadata_in_zk.constraints = new_constraints_str; Coordination::Requests ops; @@ -3711,14 +3713,15 @@ void StorageReplicatedMergeTree::alter( String new_columns_str = future_metadata.columns.toString(); ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/columns", new_columns_str, -1)); - if (ast_to_str(current_metadata.settings_changes) != ast_to_str(future_metadata.settings_changes)) + if (ast_to_str(current_metadata->settings_changes) != ast_to_str(future_metadata.settings_changes)) { lockStructureExclusively( table_lock_holder, query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); /// Just change settings - current_metadata.settings_changes = future_metadata.settings_changes; - changeSettings(current_metadata.settings_changes, table_lock_holder); - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id, current_metadata); + StorageInMemoryMetadata metadata_copy = *current_metadata; + metadata_copy.settings_changes = future_metadata.settings_changes; + changeSettings(metadata_copy.settings_changes, table_lock_holder); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id, metadata_copy); } /// We can be sure, that in case of successfull commit in zookeeper our @@ -3733,7 +3736,7 @@ void StorageReplicatedMergeTree::alter( alter_entry->create_time = time(nullptr); auto maybe_mutation_commands = params.getMutationCommands( - current_metadata, query_context.getSettingsRef().materialize_ttl_after_modify, query_context); + *current_metadata, query_context.getSettingsRef().materialize_ttl_after_modify, query_context); alter_entry->have_mutation = !maybe_mutation_commands.empty(); ops.emplace_back(zkutil::makeCreateRequest( diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 50530070d19..49dc09dbcf4 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -301,17 +301,17 @@ private: /** Creates the minimum set of nodes in ZooKeeper and create first replica. * Returns true if was created, false if exists. */ - bool createTableIfNotExists(); + bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); /** Creates a replica in ZooKeeper and adds to the queue all that it takes to catch up with the rest of the replicas. */ - void createReplica(); + void createReplica(const StorageMetadataPtr & metadata_snapshot); /** Create nodes in the ZK, which must always be, but which might not exist when older versions of the server are running. */ void createNewZooKeeperNodes(); - void checkTableStructure(const String & zookeeper_prefix); + void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot); /// A part of ALTER: apply metadata changes only (data parts are altered separately). /// Must be called under IStorage::lockStructureForAlter() lock. From fa60903620b46dccc310ecca309b0e67a53600a6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Jun 2020 21:41:11 +0300 Subject: [PATCH 071/211] Fix race condition --- src/Storages/StorageBuffer.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index b08e4e93bed..f8df14aa482 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -160,13 +160,14 @@ Pipes StorageBuffer::read( { auto destination = DatabaseCatalog::instance().getTable(destination_id, context); - auto destination_metadata_snapshot = destination->getInMemoryMetadataPtr(); if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); auto destination_lock = destination->lockStructureForShare( false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto destination_metadata_snapshot = destination->getInMemoryMetadataPtr(); + const bool dst_has_same_structure = std::all_of(column_names.begin(), column_names.end(), [metadata_snapshot, destination](const String& column_name) { const auto & dest_columns = destination->getColumns(); From 62f2c17a668d85cfb88a6137a14997cda1e35d5b Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jun 2020 12:38:47 +0300 Subject: [PATCH 072/211] Secondary indices in StorageInMemoryMetadata --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/ExpressionAnalyzer.h | 12 +++++++--- src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- src/Interpreters/MutationsInterpreter.cpp | 4 ++-- src/Storages/IStorage.cpp | 11 ---------- src/Storages/IStorage.h | 6 +---- src/Storages/MergeTree/MergeTreeData.cpp | 22 ++++++++++--------- src/Storages/MergeTree/MergeTreeData.h | 7 +++--- .../MergeTree/MergeTreeDataMergerMutator.cpp | 21 +++++++++--------- .../MergeTree/MergeTreeDataMergerMutator.h | 1 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 6 ++--- .../MergeTree/MergedBlockOutputStream.cpp | 2 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 5 +++-- src/Storages/StorageBuffer.cpp | 6 +++-- src/Storages/StorageBuffer.h | 2 +- src/Storages/StorageMaterializedView.h | 6 +++-- src/Storages/StorageMerge.cpp | 6 +++-- src/Storages/StorageMerge.h | 3 ++- 19 files changed, 66 insertions(+), 62 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 24c71e276d6..039001796cc 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -356,7 +356,7 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) const IAST & args = *func->arguments; const ASTPtr & left_in_operand = args.children.at(0); - if (storage()->mayBenefitFromIndexForIn(left_in_operand, context)) + if (storage()->mayBenefitFromIndexForIn(left_in_operand, context, metadata_snapshot)) { const ASTPtr & arg = args.children.at(1); if (arg->as() || arg->as()) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index c69cb61162f..1cc1b33bad1 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -11,7 +11,6 @@ #include #include - namespace DB { @@ -32,6 +31,9 @@ class ASTExpressionList; class ASTSelectQuery; struct ASTTablesInSelectQueryElement; +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; + /// Create columns in block or return false if not possible bool sanitizeBlock(Block & block); @@ -232,11 +234,14 @@ public: const ASTPtr & query_, const SyntaxAnalyzerResultPtr & syntax_analyzer_result_, const Context & context_, + const StorageMetadataPtr & metadata_snapshot_, const NameSet & required_result_columns_ = {}, bool do_global_ = false, const SelectQueryOptions & options_ = {}) - : ExpressionAnalyzer(query_, syntax_analyzer_result_, context_, options_.subquery_depth, do_global_) - , required_result_columns(required_result_columns_), query_options(options_) + : ExpressionAnalyzer(query_, syntax_analyzer_result_, context_, options_.subquery_depth, do_global_) + , metadata_snapshot(metadata_snapshot_) + , required_result_columns(required_result_columns_) + , query_options(options_) { } @@ -260,6 +265,7 @@ public: void appendProjectResult(ExpressionActionsChain & chain) const; private: + StorageMetadataPtr metadata_snapshot; /// If non-empty, ignore all expressions not from this list. NameSet required_result_columns; SelectQueryOptions query_options; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f73245179ce..331093b9d53 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -130,7 +130,7 @@ String InterpreterSelectQuery::generateFilterActions( /// Using separate expression analyzer to prevent any possible alias injection auto syntax_result = SyntaxAnalyzer(*context).analyzeSelect(query_ast, SyntaxAnalyzerResult({}, storage)); - SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, *context); + SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, *context, metadata_snapshot); actions = analyzer.simpleSelectActions(); return expr_list->children.at(0)->getColumnName(); @@ -336,7 +336,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( context->getQueryContext().addScalar(it.first, it.second); query_analyzer = std::make_unique( - query_ptr, syntax_analyzer_result, *context, + query_ptr, syntax_analyzer_result, *context, metadata_snapshot, NameSet(required_result_column_names.begin(), required_result_column_names.end()), !options.only_analyze, options); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index ce47ce6e476..7bf54d20a61 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -294,8 +294,8 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) throw Exception("Empty mutation commands list", ErrorCodes::LOGICAL_ERROR); - const ColumnsDescription & columns_desc = storage->getColumns(); - const IndicesDescription & indices_desc = storage->getSecondaryIndices(); + const ColumnsDescription & columns_desc = metadata_snapshot->getColumns(); + const IndicesDescription & indices_desc = metadata_snapshot->getSecondaryIndices(); NamesAndTypesList all_columns = columns_desc.getAllPhysical(); NameSet updated_columns; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 1fb3e095229..7d50025faff 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -37,17 +37,6 @@ const ColumnsDescription & IStorage::getColumns() const return metadata->columns; } -const IndicesDescription & IStorage::getSecondaryIndices() const -{ - return metadata->secondary_indices; -} - -bool IStorage::hasSecondaryIndices() const -{ - return !metadata->secondary_indices.empty(); -} - - namespace { #if !defined(ARCADIA_BUILD) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index e45d6a1128b..78d9b7d2013 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -140,10 +140,6 @@ public: public: /// thread-unsafe part. lockStructure must be acquired const ColumnsDescription & getColumns() const; /// returns combined set of columns - const IndicesDescription & getSecondaryIndices() const; - /// Has at least one non primary index - bool hasSecondaryIndices() const; - /// Storage settings ASTPtr getSettingsChanges() const; @@ -413,7 +409,7 @@ public: virtual bool supportsIndexForIn() const { return false; } /// Provides a hint that the storage engine may evaluate the IN-condition by using an index. - virtual bool mayBenefitFromIndexForIn(const ASTPtr & /* left_in_operand */, const Context & /* query_context */) const { return false; } + virtual bool mayBenefitFromIndexForIn(const ASTPtr & /* left_in_operand */, const Context & /* query_context */, const StorageMetadataPtr & /* metadata_snapshot */) const { return false; } /// Checks validity of the data virtual CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) { throw Exception("Check query is not supported for " + getName() + " storage", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8971b50a0fd..143ce44da5e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -408,14 +408,14 @@ ExpressionActionsPtr getCombinedIndicesExpression( } -ExpressionActionsPtr MergeTreeData::getPrimaryKeyAndSkipIndicesExpression() const +ExpressionActionsPtr MergeTreeData::getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const { - return getCombinedIndicesExpression(getPrimaryKey(), getSecondaryIndices(), getColumns(), global_context); + return getCombinedIndicesExpression(getPrimaryKey(), metadata_snapshot->getSecondaryIndices(), metadata_snapshot->getColumns(), global_context); } -ExpressionActionsPtr MergeTreeData::getSortingKeyAndSkipIndicesExpression() const +ExpressionActionsPtr MergeTreeData::getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const { - return getCombinedIndicesExpression(getSortingKey(), getSecondaryIndices(), getColumns(), global_context); + return getCombinedIndicesExpression(getSortingKey(), metadata_snapshot->getSecondaryIndices(), metadata_snapshot->getColumns(), global_context); } @@ -1237,9 +1237,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S { /// Check that needed transformations can be applied to the list of columns without considering type conversions. StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); + StorageInMemoryMetadata old_metadata = getInMemoryMetadata(); commands.apply(new_metadata, global_context); - if (getSecondaryIndices().empty() && !new_metadata.secondary_indices.empty() && - !settings.allow_experimental_data_skipping_indices) + if (old_metadata.getSecondaryIndices().empty() && !new_metadata.secondary_indices.empty() + && !settings.allow_experimental_data_skipping_indices) throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \ "before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS); @@ -1259,7 +1260,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S columns_alter_type_forbidden.insert(col); } - for (const auto & index : getSecondaryIndices()) + for (const auto & index : old_metadata.getSecondaryIndices()) { for (const String & col : index.expression->getRequiredColumns()) columns_alter_type_forbidden.insert(col); @@ -2932,7 +2933,8 @@ bool MergeTreeData::isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const A return false; } -bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context &) const +bool MergeTreeData::mayBenefitFromIndexForIn( + const ASTPtr & left_in_operand, const Context &, const StorageMetadataPtr & metadata_snapshot) const { /// Make sure that the left side of the IN operator contain part of the key. /// If there is a tuple on the left side of the IN operator, at least one item of the tuple @@ -2945,7 +2947,7 @@ bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, con { if (isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(item)) return true; - for (const auto & index : getSecondaryIndices()) + for (const auto & index : metadata_snapshot->getSecondaryIndices()) if (index_wrapper_factory.get(index)->mayBenefitFromIndexForIn(item)) return true; } @@ -2954,7 +2956,7 @@ bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, con } else { - for (const auto & index : getSecondaryIndices()) + for (const auto & index : metadata_snapshot->getSecondaryIndices()) if (index_wrapper_factory.get(index)->mayBenefitFromIndexForIn(left_in_operand)) return true; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 4be9f450535..22d2a9da79c 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -350,7 +350,8 @@ public: bool supportsSettings() const override { return true; } NamesAndTypesList getVirtuals() const override; - bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context &) const override; + bool + mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context &, const StorageMetadataPtr & metadata_snapshot) const override; /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks); @@ -643,8 +644,8 @@ public: Int64 minmax_idx_date_column_pos = -1; /// In a common case minmax index includes a date column. Int64 minmax_idx_time_column_pos = -1; /// In other cases, minmax index often includes a dateTime column. - ExpressionActionsPtr getPrimaryKeyAndSkipIndicesExpression() const; - ExpressionActionsPtr getSortingKeyAndSkipIndicesExpression() const; + ExpressionActionsPtr getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; + ExpressionActionsPtr getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; std::optional selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 3bff5c9f505..afd1586ac6c 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -612,7 +612,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor NamesAndTypesList merging_columns; Names gathering_column_names, merging_column_names; extractMergingAndGatheringColumns( - storage_columns, data.getSortingKey().expression, data.getSecondaryIndices(), + storage_columns, data.getSortingKey().expression, metadata_snapshot->getSecondaryIndices(), data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names); auto single_disk_volume = std::make_shared("volume_" + future_part.name, disk); @@ -798,10 +798,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor merged_stream = std::make_shared(merged_stream, data, new_data_part, time_of_merge, force_ttl); - if (data.hasSecondaryIndices()) + if (metadata_snapshot->hasSecondaryIndices()) { - const auto & indices = data.getSecondaryIndices(); - merged_stream = std::make_shared(merged_stream, indices.getSingleExpressionForIndices(data.getColumns(), data.global_context)); + const auto & indices = metadata_snapshot->getSecondaryIndices(); + merged_stream = std::make_shared(merged_stream, indices.getSingleExpressionForIndices(metadata_snapshot->getColumns(), data.global_context)); merged_stream = std::make_shared(merged_stream); } @@ -810,7 +810,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor new_data_part, metadata_snapshot, merging_columns, - index_factory.getMany(data.getSecondaryIndices()), + index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec, merged_column_to_size, data_settings->min_merge_bytes_to_use_direct_io, @@ -1084,7 +1084,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor /// All columns from part are changed and may be some more that were missing before in part if (isCompactPart(source_part) || source_part->getColumns().isSubsetOf(updated_header.getNamesAndTypesList())) { - auto part_indices = getIndicesForNewDataPart(data.getSecondaryIndices(), for_file_renames); + auto part_indices = getIndicesForNewDataPart(metadata_snapshot->getSecondaryIndices(), for_file_renames); mutateAllPartColumns( new_data_part, metadata_snapshot, @@ -1101,7 +1101,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor else /// TODO: check that we modify only non-key columns in this case. { /// We will modify only some of the columns. Other columns and key values can be copied as-is. - auto indices_to_recalc = getIndicesToRecalculate(in, updated_header.getNamesAndTypesList(), context); + auto indices_to_recalc = getIndicesToRecalculate(in, updated_header.getNamesAndTypesList(), metadata_snapshot, context); NameSet files_to_skip = collectFilesToSkip(updated_header, indices_to_recalc, mrk_extension); NameToNameVector files_to_rename = collectFilesForRenames(source_part, for_file_renames, mrk_extension); @@ -1524,6 +1524,7 @@ MergeTreeIndices MergeTreeDataMergerMutator::getIndicesForNewDataPart( std::set MergeTreeDataMergerMutator::getIndicesToRecalculate( BlockInputStreamPtr & input_stream, const NamesAndTypesList & updated_columns, + const StorageMetadataPtr & metadata_snapshot, const Context & context) const { /// Checks if columns used in skipping indexes modified. @@ -1532,7 +1533,7 @@ std::set MergeTreeDataMergerMutator::getIndicesToRecalculate( ASTPtr indices_recalc_expr_list = std::make_shared(); for (const auto & col : updated_columns.getNames()) { - const auto & indices = data.getSecondaryIndices(); + const auto & indices = metadata_snapshot->getSecondaryIndices(); for (size_t i = 0; i < indices.size(); ++i) { const auto & index = indices[i]; @@ -1597,9 +1598,9 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( if (mutating_stream == nullptr) throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR); - if (data.hasPrimaryKey() || data.hasSecondaryIndices()) + if (data.hasPrimaryKey() || metadata_snapshot->hasSecondaryIndices()) mutating_stream = std::make_shared( - std::make_shared(mutating_stream, data.getPrimaryKeyAndSkipIndicesExpression())); + std::make_shared(mutating_stream, data.getPrimaryKeyAndSkipIndicesExpression(metadata_snapshot))); if (need_remove_expired_values) mutating_stream = std::make_shared(mutating_stream, data, new_data_part, time_of_mutation, true); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 3625c9bbe26..7828f79ea33 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -177,6 +177,7 @@ private: std::set getIndicesToRecalculate( BlockInputStreamPtr & input_stream, const NamesAndTypesList & updated_columns, + const StorageMetadataPtr & metadata_snapshot, const Context & context) const; /// Override all columns of new part using mutating_stream diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ac2f4851185..9fd020d0317 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -550,7 +550,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( std::vector> useful_indices; - for (const auto & index : data.getSecondaryIndices()) + for (const auto & index : metadata_snapshot->getSecondaryIndices()) { auto index_helper = MergeTreeIndexFactory::instance().get(index); auto condition = index_helper->createIndexCondition(query_info, context); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 71501a0e19a..284ea02097b 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -262,8 +262,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->volume->getDisk()->createDirectories(full_path); /// If we need to calculate some columns to sort. - if (data.hasSortingKey() || data.hasSecondaryIndices()) - data.getSortingKeyAndSkipIndicesExpression()->execute(block); + if (data.hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) + data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot)->execute(block); Names sort_columns = data.getSortingKeyColumns(); SortDescription sort_description; @@ -302,7 +302,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa auto compression_codec = data.global_context.chooseCompressionCodec(0, 0); const auto & index_factory = MergeTreeIndexFactory::instance(); - MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(data.getSecondaryIndices()), compression_codec); + MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); out.writePrefix(); out.writeWithPermutation(block, perm_ptr); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index c768678c454..9cbdc338367 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -164,7 +164,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm return; std::unordered_set skip_indexes_column_names_set; - for (const auto & index : storage.getSecondaryIndices()) + for (const auto & index : metadata_snapshot->getSecondaryIndices()) std::copy(index.column_names.cbegin(), index.column_names.cend(), std::inserter(skip_indexes_column_names_set, skip_indexes_column_names_set.end())); Names skip_indexes_column_names(skip_indexes_column_names_set.begin(), skip_indexes_column_names_set.end()); diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 45ee947b81f..17891fde34a 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -35,9 +35,10 @@ public: bool supportsIndexForIn() const override { return true; } - bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override + bool mayBenefitFromIndexForIn( + const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & metadata_snapshot) const override { - return part->storage.mayBenefitFromIndexForIn(left_in_operand, query_context); + return part->storage.mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot); } NamesAndTypesList getVirtuals() const override diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index f8df14aa482..4882b5fdc1c 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -447,7 +447,8 @@ BlockOutputStreamPtr StorageBuffer::write(const ASTPtr & /*query*/, const Storag } -bool StorageBuffer::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const +bool StorageBuffer::mayBenefitFromIndexForIn( + const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & /*metadata_snapshot*/) const { if (!destination_id) return false; @@ -457,7 +458,8 @@ bool StorageBuffer::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, con if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); - return destination->mayBenefitFromIndexForIn(left_in_operand, query_context); + /// TODO alesap (check destination metadata) + return destination->mayBenefitFromIndexForIn(left_in_operand, query_context, destination->getInMemoryMetadataPtr()); } diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 7cd73dc556c..403b6c53172 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -84,7 +84,7 @@ public: bool supportsFinal() const override { return true; } bool supportsIndexForIn() const override { return true; } - bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override; + bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & metadata_snapshot) const override; void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override; diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 672be800c8f..ef895ff0165 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -26,9 +26,11 @@ public: bool supportsFinal() const override { return getTargetTable()->supportsFinal(); } bool supportsIndexForIn() const override { return getTargetTable()->supportsIndexForIn(); } bool supportsParallelInsert() const override { return getTargetTable()->supportsParallelInsert(); } - bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override + bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & /* metadata_snapshot */) const override { - return getTargetTable()->mayBenefitFromIndexForIn(left_in_operand, query_context); + auto target_table = getTargetTable(); + auto metadata_snapshot = target_table->getInMemoryMetadataPtr(); + return target_table->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot); } BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index e24e5986994..e47cde8de52 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -81,7 +81,7 @@ bool StorageMerge::isRemote() const } -bool StorageMerge::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const +bool StorageMerge::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & /*metadata_snapshot*/) const { /// It's beneficial if it is true for at least one table. StorageListWithLocks selected_tables = getSelectedTables( @@ -90,7 +90,9 @@ bool StorageMerge::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, cons size_t i = 0; for (const auto & table : selected_tables) { - if (std::get<0>(table)->mayBenefitFromIndexForIn(left_in_operand, query_context)) + auto storage_ptr = std::get<0>(table); + auto metadata_snapshot = storage_ptr->getInMemoryMetadataPtr(); + if (storage_ptr->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot)) return true; ++i; diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 14bf83f8534..1ad22869e39 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -44,7 +44,8 @@ public: /// the structure of sub-tables is not checked void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; - bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override; + bool mayBenefitFromIndexForIn( + const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & metadata_snapshot) const override; private: String source_database; From ab61abccc1eb3901dc3154010add03e58caf3958 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jun 2020 13:34:23 +0300 Subject: [PATCH 073/211] Partition key in StorageInMemoryMetadata --- src/Interpreters/MutationsInterpreter.cpp | 15 +++--- src/Storages/IStorage.cpp | 22 --------- src/Storages/IStorage.h | 12 ----- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 8 ++-- .../MergeTree/MergeTreeBlockOutputStream.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 46 ++++++++++--------- src/Storages/MergeTree/MergeTreeData.h | 12 ++--- .../MergeTree/MergeTreeDataWriter.cpp | 8 ++-- src/Storages/MergeTree/MergeTreeDataWriter.h | 2 +- src/Storages/MergeTree/MergeTreePartition.cpp | 11 +++-- .../ReplicatedMergeTreeBlockOutputStream.cpp | 2 +- .../ReplicatedMergeTreeTableMetadata.cpp | 2 +- src/Storages/StorageInMemoryMetadata.cpp | 21 +++++++++ src/Storages/StorageInMemoryMetadata.h | 11 +++++ src/Storages/StorageMergeTree.cpp | 14 ++++-- src/Storages/StorageReplicatedMergeTree.cpp | 19 ++++++-- src/Storages/System/StorageSystemColumns.cpp | 6 +-- src/Storages/System/StorageSystemTables.cpp | 3 +- 18 files changed, 118 insertions(+), 98 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 7bf54d20a61..50b68ba7ca3 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -214,7 +214,7 @@ MutationsInterpreter::MutationsInterpreter( select_interpreter = std::make_unique(mutation_ast, context, storage, limits); } -static NameSet getKeyColumns(const StoragePtr & storage) +static NameSet getKeyColumns(const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot) { const MergeTreeData * merge_tree_data = dynamic_cast(storage.get()); if (!merge_tree_data) @@ -222,7 +222,7 @@ static NameSet getKeyColumns(const StoragePtr & storage) NameSet key_columns; - for (const String & col : merge_tree_data->getColumnsRequiredForPartitionKey()) + for (const String & col : metadata_snapshot->getColumnsRequiredForPartitionKey()) key_columns.insert(col); for (const String & col : merge_tree_data->getColumnsRequiredForSortingKey()) @@ -239,15 +239,16 @@ static NameSet getKeyColumns(const StoragePtr & storage) } static void validateUpdateColumns( - const StoragePtr & storage, const NameSet & updated_columns, + const StoragePtr & storage, + const StorageMetadataPtr & metadata_snapshot, const NameSet & updated_columns, const std::unordered_map & column_to_affected_materialized) { - NameSet key_columns = getKeyColumns(storage); + NameSet key_columns = getKeyColumns(storage, metadata_snapshot); for (const String & column_name : updated_columns) { auto found = false; - for (const auto & col : storage->getColumns().getOrdinary()) + for (const auto & col : metadata_snapshot->getColumns().getOrdinary()) { if (col.name == column_name) { @@ -258,7 +259,7 @@ static void validateUpdateColumns( if (!found) { - for (const auto & col : storage->getColumns().getMaterialized()) + for (const auto & col : metadata_snapshot->getColumns().getMaterialized()) { if (col.name == column_name) throw Exception("Cannot UPDATE materialized column " + backQuote(column_name), ErrorCodes::CANNOT_UPDATE_COLUMN); @@ -326,7 +327,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } } - validateUpdateColumns(storage, updated_columns, column_to_affected_materialized); + validateUpdateColumns(storage, metadata_snapshot, updated_columns, column_to_affected_materialized); } /// Columns, that we need to read for calculation of skip indices or TTL expressions. diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 7d50025faff..84afd2fcf1c 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -319,28 +319,6 @@ NamesAndTypesList IStorage::getVirtuals() const return {}; } -const KeyDescription & IStorage::getPartitionKey() const -{ - return metadata->partition_key; -} - -bool IStorage::isPartitionKeyDefined() const -{ - return metadata->partition_key.definition_ast != nullptr; -} - -bool IStorage::hasPartitionKey() const -{ - return !metadata->partition_key.column_names.empty(); -} - -Names IStorage::getColumnsRequiredForPartitionKey() const -{ - if (hasPartitionKey()) - return metadata->partition_key.expression->getRequiredColumns(); - return {}; -} - const KeyDescription & IStorage::getSortingKey() const { return metadata->sorting_key; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 78d9b7d2013..5f08d48d4b0 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -427,18 +427,6 @@ public: /// Returns data paths if storage supports it, empty vector otherwise. virtual Strings getDataPaths() const { return {}; } - /// Returns structure with partition key. - const KeyDescription & getPartitionKey() const; - /// Returns ASTExpressionList of partition key expression for storage or nullptr if there is none. - ASTPtr getPartitionKeyAST() const { return metadata->partition_key.definition_ast; } - /// Storage has user-defined (in CREATE query) partition key. - bool isPartitionKeyDefined() const; - /// Storage has partition key. - bool hasPartitionKey() const; - /// Returns column names that need to be read to calculate partition key. - Names getColumnsRequiredForPartitionKey() const; - - /// Returns structure with sorting key. const KeyDescription & getSortingKey() const; /// Returns ASTExpressionList of sorting key expression for storage or nullptr if there is none. diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 287bf916c19..03b2dea23ba 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -496,7 +496,8 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() minmax_idx.load(storage, volume->getDisk(), path); } - String calculated_partition_id = partition.getID(storage.getPartitionKey().sample_block); + auto metadata_snapshot = storage.getInMemoryMetadataPtr(); + String calculated_partition_id = partition.getID(metadata_snapshot->getPartitionKey().sample_block); if (calculated_partition_id != info.partition_id) throw Exception( "While loading part " + getFullPath() + ": calculated partition ID: " + calculated_partition_id @@ -840,6 +841,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const { String path = getFullRelativePath(); + auto metadata_snapshot = storage.getInMemoryMetadataPtr(); const auto & pk = storage.getPrimaryKey(); if (!checksums.empty()) { @@ -851,7 +853,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const if (!checksums.files.count("count.txt")) throw Exception("No checksum for count.txt", ErrorCodes::NO_FILE_IN_DATA_PART); - if (storage.hasPartitionKey() && !checksums.files.count("partition.dat")) + if (metadata_snapshot->hasPartitionKey() && !checksums.files.count("partition.dat")) throw Exception("No checksum for partition.dat", ErrorCodes::NO_FILE_IN_DATA_PART); if (!isEmpty()) @@ -884,7 +886,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const { check_file_not_empty(volume->getDisk(), path + "count.txt"); - if (storage.hasPartitionKey()) + if (metadata_snapshot->hasPartitionKey()) check_file_not_empty(volume->getDisk(), path + "partition.dat"); for (const String & col_name : storage.minmax_idx_columns) diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index 1ea6b049bf6..5f774a97bce 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -16,7 +16,7 @@ void MergeTreeBlockOutputStream::write(const Block & block) { storage.delayInsertOrThrowIfNeeded(); - auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block); + auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot); for (auto & current_block : part_blocks) { Stopwatch watch; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 143ce44da5e..b7e152fe6b4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -169,7 +169,7 @@ MergeTreeData::MergeTreeData( min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING; } - setProperties(metadata_, attach); + setProperties(metadata_, metadata_, attach); const auto settings = getSettings(); /// NOTE: using the same columns list as is read when performing actual merges. @@ -184,7 +184,7 @@ MergeTreeData::MergeTreeData( } - setTTLExpressions(metadata_); + setTTLExpressions(metadata_, metadata_); /// format_file always contained on any data path PathWithDisk version_file; @@ -274,7 +274,7 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam } } -void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata, bool attach) const +void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & /*old_metadata*/, bool attach) const { if (!new_metadata.sorting_key.definition_ast) throw Exception("ORDER BY cannot be empty", ErrorCodes::BAD_ARGUMENTS); @@ -381,9 +381,9 @@ void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata } -void MergeTreeData::setProperties(const StorageInMemoryMetadata & new_metadata, bool attach) +void MergeTreeData::setProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach) { - checkProperties(new_metadata, attach); + checkProperties(new_metadata, old_metadata, attach); setInMemoryMetadata(new_metadata); } @@ -475,7 +475,7 @@ void MergeTreeData::initPartitionKey(const KeyDescription & new_partition_key) } -void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_metadata) const +void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata) const { auto new_column_ttls = new_metadata.column_ttls_by_name; @@ -483,8 +483,8 @@ void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_meta { NameSet columns_ttl_forbidden; - if (hasPartitionKey()) - for (const auto & col : getColumnsRequiredForPartitionKey()) + if (old_metadata.hasPartitionKey()) + for (const auto & col : old_metadata.getColumnsRequiredForPartitionKey()) columns_ttl_forbidden.insert(col); if (hasSortingKey()) @@ -517,9 +517,9 @@ void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_meta } /// Todo replace columns with TTL for columns -void MergeTreeData::setTTLExpressions(const StorageInMemoryMetadata & new_metadata) +void MergeTreeData::setTTLExpressions(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata) { - checkTTLExpressions(new_metadata); + checkTTLExpressions(new_metadata, old_metadata); //setColumnTTLs(new_metadata.column_ttls_by_name); //setTableTTLs(new_metadata.table_ttl); } @@ -1251,12 +1251,12 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S /// (and not as a part of some expression) and if the ALTER only affects column metadata. NameSet columns_alter_type_metadata_only; - if (hasPartitionKey()) + if (old_metadata.hasPartitionKey()) { /// Forbid altering partition key columns because it can change partition ID format. /// TODO: in some cases (e.g. adding an Enum value) a partition key column can still be ALTERed. /// We should allow it. - for (const String & col : getColumnsRequiredForPartitionKey()) + for (const String & col : old_metadata.getColumnsRequiredForPartitionKey()) columns_alter_type_forbidden.insert(col); } @@ -1284,7 +1284,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S columns_alter_type_forbidden.insert(merging_params.sign_column); std::map old_types; - for (const auto & column : getColumns().getAllPhysical()) + for (const auto & column : old_metadata.getColumns().getAllPhysical()) old_types.emplace(column.name, column.type.get()); @@ -1329,9 +1329,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S } } - checkProperties(new_metadata); + checkProperties(new_metadata, old_metadata); - checkTTLExpressions(new_metadata); + checkTTLExpressions(new_metadata, old_metadata); if (hasSettingsChanges()) { @@ -2450,7 +2450,8 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context /// Re-parse partition key fields using the information about expected field types. - size_t fields_count = getPartitionKey().sample_block.columns(); + auto metadata_snapshot = getInMemoryMetadataPtr(); + size_t fields_count = metadata_snapshot->getPartitionKey().sample_block.columns(); if (partition_ast.fields_count != fields_count) throw Exception( "Wrong number of fields in the partition expression: " + toString(partition_ast.fields_count) + @@ -2467,7 +2468,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context ReadBufferFromMemory right_paren_buf(")", 1); ConcatReadBuffer buf({&left_paren_buf, &fields_buf, &right_paren_buf}); - auto input_stream = FormatFactory::instance().getInput("Values", buf, getPartitionKey().sample_block, context, context.getSettingsRef().max_block_size); + auto input_stream = FormatFactory::instance().getInput("Values", buf, metadata_snapshot->getPartitionKey().sample_block, context, context.getSettingsRef().max_block_size); auto block = input_stream->read(); if (!block || !block.rows()) @@ -2964,7 +2965,7 @@ bool MergeTreeData::mayBenefitFromIndexForIn( } } -MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & source_table) const +MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const { MergeTreeData * src_data = dynamic_cast(&source_table); if (!src_data) @@ -2972,7 +2973,7 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & sour " supports attachPartitionFrom only for MergeTree family of table engines." " Got " + source_table.getName(), ErrorCodes::NOT_IMPLEMENTED); - if (getColumns().getAllPhysical().sizeOfDifference(src_data->getColumns().getAllPhysical())) + if (my_snapshot->getColumns().getAllPhysical().sizeOfDifference(src_snapshot->getColumns().getAllPhysical())) throw Exception("Tables have different structure", ErrorCodes::INCOMPATIBLE_COLUMNS); auto query_to_string = [] (const ASTPtr & ast) @@ -2983,7 +2984,7 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & sour if (query_to_string(getSortingKeyAST()) != query_to_string(src_data->getSortingKeyAST())) throw Exception("Tables have different ordering", ErrorCodes::BAD_ARGUMENTS); - if (query_to_string(getPartitionKeyAST()) != query_to_string(src_data->getPartitionKeyAST())) + if (query_to_string(my_snapshot->getPartitionKeyAST()) != query_to_string(src_snapshot->getPartitionKeyAST())) throw Exception("Tables have different partition key", ErrorCodes::BAD_ARGUMENTS); if (format_version != src_data->format_version) @@ -2992,9 +2993,10 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & sour return *src_data; } -MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(const StoragePtr & source_table) const +MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData( + const StoragePtr & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const { - return checkStructureAndGetMergeTreeData(*source_table); + return checkStructureAndGetMergeTreeData(*source_table, src_snapshot, my_snapshot); } MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(const MergeTreeData::DataPartPtr & src_part, diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 22d2a9da79c..863b5ba1644 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -556,8 +556,8 @@ public: /// Extracts MergeTreeData of other *MergeTree* storage /// and checks that their structure suitable for ALTER TABLE ATTACH PARTITION FROM /// Tables structure should be locked. - MergeTreeData & checkStructureAndGetMergeTreeData(const StoragePtr & source_table) const; - MergeTreeData & checkStructureAndGetMergeTreeData(IStorage & source_table) const; + MergeTreeData & checkStructureAndGetMergeTreeData(const StoragePtr & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const; + MergeTreeData & checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const; MergeTreeData::MutableDataPartPtr cloneAndLoadDataPartOnSameDisk( const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info); @@ -781,14 +781,14 @@ protected: /// The same for clearOldTemporaryDirectories. std::mutex clear_old_temporary_directories_mutex; - void checkProperties(const StorageInMemoryMetadata & new_metadata, bool attach = false) const; + void checkProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach = false) const; - void setProperties(const StorageInMemoryMetadata & new_metadata, bool attach = false); + void setProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach = false); void initPartitionKey(const KeyDescription & new_partition_key); - void checkTTLExpressions(const StorageInMemoryMetadata & new_metadata) const; - void setTTLExpressions(const StorageInMemoryMetadata & new_metadata); + void checkTTLExpressions(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata) const; + void setTTLExpressions(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata); void checkStoragePolicy(const StoragePolicyPtr & new_storage_policy) const; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 284ea02097b..f96c9b48c4d 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -132,7 +132,7 @@ void updateTTL( } -BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block, size_t max_parts) +BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot) { BlocksWithPartition result; if (!block || !block.rows()) @@ -140,14 +140,14 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block data.check(block, true); - if (!data.hasPartitionKey()) /// Table is not partitioned. + if (!metadata_snapshot->hasPartitionKey()) /// Table is not partitioned. { result.emplace_back(Block(block), Row()); return result; } Block block_copy = block; - const auto & partition_key = data.getPartitionKey(); + const auto & partition_key = metadata_snapshot->getPartitionKey(); partition_key.expression->execute(block_copy); ColumnRawPtrs partition_columns; @@ -206,7 +206,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa MergeTreePartition partition(std::move(block_with_partition.partition)); - MergeTreePartInfo new_part_info(partition.getID(data.getPartitionKey().sample_block), temp_index, temp_index, 0); + MergeTreePartInfo new_part_info(partition.getID(metadata_snapshot->getPartitionKey().sample_block), temp_index, temp_index, 0); String part_name; if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index dabdcbd2148..c04a09185c5 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -40,7 +40,7 @@ public: * (split rows by partition) * Works deterministically: if same block was passed, function will return same result in same order. */ - BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts); + BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot); /** All rows must correspond to same partition. * Returns part with unique name starting with 'tmp_', yet not added to MergeTreeData. diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 54e213fafac..cd73e9228fd 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -26,7 +26,7 @@ static std::unique_ptr openForReading(const DiskPtr & di String MergeTreePartition::getID(const MergeTreeData & storage) const { - return getID(storage.getPartitionKey().sample_block); + return getID(storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block); } /// NOTE: This ID is used to create part names which are then persisted in ZK and as directory names on the file system. @@ -89,7 +89,7 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const { - const auto & partition_key_sample = storage.getPartitionKey().sample_block; + const auto & partition_key_sample = storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block; size_t key_size = partition_key_sample.columns(); if (key_size == 0) @@ -124,10 +124,11 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path) { - if (!storage.hasPartitionKey()) + auto metadata_snapshot = storage.getInMemoryMetadataPtr(); + if (!metadata_snapshot->hasPartitionKey()) return; - const auto & partition_key_sample = storage.getPartitionKey().sample_block; + const auto & partition_key_sample = metadata_snapshot->getPartitionKey().sample_block; auto partition_file_path = part_path + "partition.dat"; auto file = openForReading(disk, partition_file_path); value.resize(partition_key_sample.columns()); @@ -137,7 +138,7 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis void MergeTreePartition::store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const { - store(storage.getPartitionKey().sample_block, disk, part_path, checksums); + store(storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block, disk, part_path, checksums); } void MergeTreePartition::store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 8319b0e018d..13df5ef23f1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -130,7 +130,7 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block) if (quorum) checkQuorumPrecondition(zookeeper); - auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block); + auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot); for (auto & current_block : part_blocks) { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 820f41326f1..1f62fba03a0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -51,7 +51,7 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr data_format_version = data.format_version; if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) - partition_key = formattedAST(data.getPartitionKey().expression_list_ast); + partition_key = formattedAST(metadata_snapshot->getPartitionKey().expression_list_ast); ttl_table = formattedAST(metadata_snapshot->getTableTTLs().definition_ast); diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 359d561cd1f..8fbe415ead6 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -291,4 +291,25 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns(const Names & column_nam return res; } +const KeyDescription & StorageInMemoryMetadata::getPartitionKey() const +{ + return partition_key; +} + +bool StorageInMemoryMetadata::isPartitionKeyDefined() const +{ + return partition_key.definition_ast != nullptr; +} + +bool StorageInMemoryMetadata::hasPartitionKey() const +{ + return !partition_key.column_names.empty(); +} + +Names StorageInMemoryMetadata::getColumnsRequiredForPartitionKey() const +{ + if (hasPartitionKey()) + return partition_key.expression->getRequiredColumns(); + return {}; +} } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 9f9154c48fb..8996f9fc1b9 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -112,6 +112,17 @@ struct StorageInMemoryMetadata Block getSampleBlockWithVirtuals(const NamesAndTypesList & virtuals) const; /// ordinary + materialized + virtuals. Block getSampleBlockForColumns( const Names & column_names, const NamesAndTypesList & virtuals) const; /// ordinary + materialized + aliases + virtuals. + + /// Returns structure with partition key. + const KeyDescription & getPartitionKey() const; + /// Returns ASTExpressionList of partition key expression for storage or nullptr if there is none. + ASTPtr getPartitionKeyAST() const { return partition_key.definition_ast; } + /// Storage has user-defined (in CREATE query) partition key. + bool isPartitionKeyDefined() const; + /// Storage has partition key. + bool hasPartitionKey() const; + /// Returns column names that need to be read to calculate partition key. + Names getColumnsRequiredForPartitionKey() const; }; using StorageMetadataPtr = std::shared_ptr; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index cfa5c34bece..c13070fa4a5 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -259,6 +259,7 @@ void StorageMergeTree::alter( auto table_id = getStorageID(); StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); + StorageInMemoryMetadata old_metadata = getInMemoryMetadata(); auto maybe_mutation_commands = commands.getMutationCommands(new_metadata, context.getSettingsRef().materialize_ttl_after_modify, context); String mutation_file_name; Int64 mutation_version = -1; @@ -282,8 +283,8 @@ void StorageMergeTree::alter( changeSettings(new_metadata.settings_changes, table_lock_holder); /// Reinitialize primary key because primary key column types might have changed. - setProperties(new_metadata); - setTTLExpressions(new_metadata); + setProperties(new_metadata, old_metadata); + setTTLExpressions(new_metadata, old_metadata); DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata); @@ -1151,9 +1152,11 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con { auto lock1 = lockStructureForShare(false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto lock2 = source_table->lockStructureForShare(false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto source_metadata_snapshot = source_table->getInMemoryMetadataPtr(); + auto my_metadata_snapshot = getInMemoryMetadataPtr(); Stopwatch watch; - MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table); + MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, my_metadata_snapshot); String partition_id = getPartitionIDFromQuery(partition, context); DataPartsVector src_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); @@ -1232,9 +1235,12 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const " should have the same storage policy of source table " + getStorageID().getNameForLogs() + ". " + getStorageID().getNameForLogs() + ": " + this->getStoragePolicy()->getName() + ", " + dest_table_storage->getStorageID().getNameForLogs() + ": " + dest_table_storage->getStoragePolicy()->getName(), ErrorCodes::LOGICAL_ERROR); + + auto dest_metadata_snapshot = dest_table->getInMemoryMetadataPtr(); + auto metadata_snapshot = getInMemoryMetadataPtr(); Stopwatch watch; - MergeTreeData & src_data = dest_table_storage->checkStructureAndGetMergeTreeData(*this); + MergeTreeData & src_data = dest_table_storage->checkStructureAndGetMergeTreeData(*this, metadata_snapshot, dest_metadata_snapshot); String partition_id = getPartitionIDFromQuery(partition, context); DataPartsVector src_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a7ddf96d08c..fc1bd8538e5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -758,6 +758,7 @@ void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_pr void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff) { StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); + StorageInMemoryMetadata old_metadata = getInMemoryMetadata(); if (new_columns != new_metadata.columns) { new_metadata.columns = new_columns; @@ -820,8 +821,8 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column /// Even if the primary/sorting keys didn't change we must reinitialize it /// because primary key column types might have changed. - setProperties(new_metadata); - setTTLExpressions(new_metadata); + setProperties(new_metadata, old_metadata); + setTTLExpressions(new_metadata, old_metadata); } @@ -1794,6 +1795,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) auto table_lock_holder_dst_table = lockStructureForShare( false, RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto dst_metadata_snapshot = getInMemoryMetadataPtr(); for (size_t i = 0; i < entry_replace.new_part_names.size(); ++i) { @@ -1843,10 +1845,11 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) return 0; } + auto src_metadata_snapshot = source_table->getInMemoryMetadataPtr(); MergeTreeData * src_data = nullptr; try { - src_data = &checkStructureAndGetMergeTreeData(source_table); + src_data = &checkStructureAndGetMergeTreeData(source_table, src_metadata_snapshot, dst_metadata_snapshot); } catch (Exception &) { @@ -5212,8 +5215,11 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ auto lock1 = lockStructureForShare(true, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto lock2 = source_table->lockStructureForShare(false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto source_metadata_snapshot = source_table->getInMemoryMetadataPtr(); + auto metadata_snapshot = getInMemoryMetadataPtr(); + Stopwatch watch; - MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table); + MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); String partition_id = getPartitionIDFromQuery(partition, context); DataPartsVector src_all_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); @@ -5405,8 +5411,11 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta getStorageID().getNameForLogs() + ": " + this->getStoragePolicy()->getName() + ", " + getStorageID().getNameForLogs() + ": " + dest_table_storage->getStoragePolicy()->getName(), ErrorCodes::LOGICAL_ERROR); + auto dest_metadata_snapshot = dest_table->getInMemoryMetadataPtr(); + auto metadata_snapshot = getInMemoryMetadataPtr(); + Stopwatch watch; - MergeTreeData & src_data = dest_table_storage->checkStructureAndGetMergeTreeData(*this); + MergeTreeData & src_data = dest_table_storage->checkStructureAndGetMergeTreeData(*this, metadata_snapshot, dest_metadata_snapshot); auto src_data_id = src_data.getStorageID(); String partition_id = getPartitionIDFromQuery(partition, query_context); diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 14a59da1bf9..5860facc814 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -122,13 +122,13 @@ protected: throw; } - columns = storage->getColumns(); + auto metadadata_snapshot = storage->getInMemoryMetadataPtr(); + columns = metadadata_snapshot->getColumns(); - cols_required_for_partition_key = storage->getColumnsRequiredForPartitionKey(); + cols_required_for_partition_key = metadadata_snapshot->getColumnsRequiredForPartitionKey(); cols_required_for_sorting_key = storage->getColumnsRequiredForSortingKey(); cols_required_for_primary_key = storage->getColumnsRequiredForPrimaryKey(); cols_required_for_sampling = storage->getColumnsRequiredForSampling(); - column_sizes = storage->getColumnSizes(); } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index b33886ce179..78346d1968f 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -267,6 +267,7 @@ protected: throw; } } + auto metadata_snapshot = table->getInMemoryMetadataPtr(); ++rows_count; @@ -365,7 +366,7 @@ protected: if (columns_mask[src_index++]) { assert(table != nullptr); - if ((expression_ptr = table->getPartitionKeyAST())) + if ((expression_ptr = metadata_snapshot->getPartitionKeyAST())) res_columns[res_index++]->insert(queryToString(expression_ptr)); else res_columns[res_index++]->insertDefault(); From ba04d02f1e23c6830a8750530d95ad3b73545bb8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jun 2020 14:05:11 +0300 Subject: [PATCH 074/211] Compilable sorting key in metadata --- src/Interpreters/ExpressionAnalyzer.cpp | 3 +- src/Interpreters/ExpressionAnalyzer.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 6 +++- src/Interpreters/MutationsInterpreter.cpp | 4 +-- src/Storages/IStorage.cpp | 29 ------------------ src/Storages/IStorage.h | 16 ---------- src/Storages/MergeTree/MergeTreeData.cpp | 20 ++++++------- .../MergeTree/MergeTreeDataMergerMutator.cpp | 16 ++++++---- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 10 +++---- .../MergeTree/MergeTreeDataWriter.cpp | 4 +-- .../ReplicatedMergeTreeTableMetadata.cpp | 4 +-- src/Storages/ReadInOrderOptimizer.cpp | 10 +++---- src/Storages/ReadInOrderOptimizer.h | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageInMemoryMetadata.cpp | 30 +++++++++++++++++++ src/Storages/StorageInMemoryMetadata.h | 17 +++++++++++ src/Storages/StorageMaterializedView.cpp | 2 +- src/Storages/StorageMerge.cpp | 4 ++- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/System/StorageSystemColumns.cpp | 8 ++--- src/Storages/System/StorageSystemTables.cpp | 2 +- 21 files changed, 104 insertions(+), 88 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 039001796cc..28aa42877d6 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1011,6 +1011,7 @@ ExpressionActionsPtr SelectQueryExpressionAnalyzer::simpleSelectActions() ExpressionAnalysisResult::ExpressionAnalysisResult( SelectQueryExpressionAnalyzer & query_analyzer, + const StorageMetadataPtr & metadata_snapshot, bool first_stage_, bool second_stage_, bool only_types, @@ -1068,7 +1069,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (storage && query.final()) { - Names columns_for_final = storage->getColumnsRequiredForFinal(); + Names columns_for_final = metadata_snapshot->getColumnsRequiredForFinal(); additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), columns_for_final.begin(), columns_for_final.end()); } diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 1cc1b33bad1..cd0b837b4ec 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -204,6 +204,7 @@ struct ExpressionAnalysisResult ExpressionAnalysisResult( SelectQueryExpressionAnalyzer & query_analyzer, + const StorageMetadataPtr & metadata_snapshot, bool first_stage, bool second_stage, bool only_types, diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 331093b9d53..038448ef353 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -492,8 +492,12 @@ Block InterpreterSelectQuery::getSampleBlockImpl() bool second_stage = from_stage <= QueryProcessingStage::WithMergeableState && options.to_stage > QueryProcessingStage::WithMergeableState; + Names columns_required_for_sampling; + Names columns_required_for_; + analysis_result = ExpressionAnalysisResult( *query_analyzer, + metadata_snapshot, first_stage, second_stage, options.only_analyze, @@ -1329,7 +1333,7 @@ void InterpreterSelectQuery::executeFetchColumns( getSortDescriptionFromGroupBy(query), query_info.syntax_analyzer_result); - query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage); + query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage, metadata_snapshot); } Pipes pipes = storage->read(required_columns, metadata_snapshot, query_info, *context, processing_stage, max_block_size, max_streams); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 50b68ba7ca3..0c0227d476f 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -225,7 +225,7 @@ static NameSet getKeyColumns(const StoragePtr & storage, const StorageMetadataPt for (const String & col : metadata_snapshot->getColumnsRequiredForPartitionKey()) key_columns.insert(col); - for (const String & col : merge_tree_data->getColumnsRequiredForSortingKey()) + for (const String & col : metadata_snapshot->getColumnsRequiredForSortingKey()) key_columns.insert(col); /// We don't process sample_by_ast separately because it must be among the primary key columns. @@ -731,7 +731,7 @@ size_t MutationsInterpreter::evaluateCommandsSize() std::optional MutationsInterpreter::getStorageSortDescriptionIfPossible(const Block & header) const { - Names sort_columns = storage->getSortingKeyColumns(); + Names sort_columns = metadata_snapshot->getSortingKeyColumns(); SortDescription sort_description; size_t sort_columns_size = sort_columns.size(); sort_description.reserve(sort_columns_size); diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 84afd2fcf1c..0c2ddc09da8 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -319,35 +319,6 @@ NamesAndTypesList IStorage::getVirtuals() const return {}; } -const KeyDescription & IStorage::getSortingKey() const -{ - return metadata->sorting_key; -} - -bool IStorage::isSortingKeyDefined() const -{ - return metadata->sorting_key.definition_ast != nullptr; -} - -bool IStorage::hasSortingKey() const -{ - return !metadata->sorting_key.column_names.empty(); -} - -Names IStorage::getColumnsRequiredForSortingKey() const -{ - if (hasSortingKey()) - return metadata->sorting_key.expression->getRequiredColumns(); - return {}; -} - -Names IStorage::getSortingKeyColumns() const -{ - if (hasSortingKey()) - return metadata->sorting_key.column_names; - return {}; -} - const KeyDescription & IStorage::getPrimaryKey() const { return metadata->primary_key; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 5f08d48d4b0..4e1ca81dd10 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -427,20 +427,6 @@ public: /// Returns data paths if storage supports it, empty vector otherwise. virtual Strings getDataPaths() const { return {}; } - /// Returns structure with sorting key. - const KeyDescription & getSortingKey() const; - /// Returns ASTExpressionList of sorting key expression for storage or nullptr if there is none. - ASTPtr getSortingKeyAST() const { return metadata->sorting_key.definition_ast; } - /// Storage has user-defined (in CREATE query) sorting key. - bool isSortingKeyDefined() const; - /// Storage has sorting key. It means, that it contains at least one column. - bool hasSortingKey() const; - /// Returns column names that need to be read to calculate sorting key. - Names getColumnsRequiredForSortingKey() const; - /// Returns columns names in sorting key specified by user in ORDER BY - /// expression. For example: 'a', 'x * y', 'toStartOfMonth(date)', etc. - Names getSortingKeyColumns() const; - /// Returns structure with primary key. const KeyDescription & getPrimaryKey() const; /// Returns ASTExpressionList of primary key expression for storage or nullptr if there is none. @@ -467,8 +453,6 @@ public: /// Returns column names that need to be read to calculate sampling key. Names getColumnsRequiredForSampling() const; - /// Returns column names that need to be read for FINAL to work. - Names getColumnsRequiredForFinal() const { return getColumnsRequiredForSortingKey(); } /// Returns storage policy if storage supports it. virtual StoragePolicyPtr getStoragePolicy() const { return {}; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b7e152fe6b4..a1487b3a1fe 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -274,7 +274,7 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam } } -void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & /*old_metadata*/, bool attach) const +void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach) const { if (!new_metadata.sorting_key.definition_ast) throw Exception("ORDER BY cannot be empty", ErrorCodes::BAD_ARGUMENTS); @@ -312,7 +312,7 @@ void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata auto all_columns = new_metadata.columns.getAllPhysical(); /// Order by check AST - if (hasSortingKey()) + if (old_metadata.hasSortingKey()) { /// This is ALTER, not CREATE/ATTACH TABLE. Let us check that all new columns used in the sorting key /// expression have just been added (so that the sorting order is guaranteed to be valid with the new key). @@ -321,7 +321,7 @@ void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata Names new_sorting_key_columns = new_sorting_key.column_names; ASTPtr added_key_column_expr_list = std::make_shared(); - const auto & old_sorting_key_columns = getSortingKeyColumns(); + const auto & old_sorting_key_columns = old_metadata.getSortingKeyColumns(); for (size_t new_i = 0, old_i = 0; new_i < sorting_key_size; ++new_i) { if (old_i < old_sorting_key_columns.size()) @@ -342,7 +342,7 @@ void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata NamesAndTypesList deleted_columns; NamesAndTypesList added_columns; - getColumns().getAllPhysical().getDifference(all_columns, deleted_columns, added_columns); + old_metadata.getColumns().getAllPhysical().getDifference(all_columns, deleted_columns, added_columns); for (const String & col : used_columns) { @@ -415,7 +415,7 @@ ExpressionActionsPtr MergeTreeData::getPrimaryKeyAndSkipIndicesExpression(const ExpressionActionsPtr MergeTreeData::getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const { - return getCombinedIndicesExpression(getSortingKey(), metadata_snapshot->getSecondaryIndices(), metadata_snapshot->getColumns(), global_context); + return getCombinedIndicesExpression(metadata_snapshot->getSortingKey(), metadata_snapshot->getSecondaryIndices(), metadata_snapshot->getColumns(), global_context); } @@ -487,8 +487,8 @@ void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_meta for (const auto & col : old_metadata.getColumnsRequiredForPartitionKey()) columns_ttl_forbidden.insert(col); - if (hasSortingKey()) - for (const auto & col : getColumnsRequiredForSortingKey()) + if (old_metadata.hasSortingKey()) + for (const auto & col : old_metadata.getColumnsRequiredForSortingKey()) columns_ttl_forbidden.insert(col); for (const auto & [name, ttl_description] : new_column_ttls) @@ -1266,9 +1266,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S columns_alter_type_forbidden.insert(col); } - if (hasSortingKey()) + if (old_metadata.hasSortingKey()) { - auto sorting_key_expr = getSortingKey().expression; + auto sorting_key_expr = old_metadata.getSortingKey().expression; for (const ExpressionAction & action : sorting_key_expr->getActions()) { auto action_columns = action.getNeededColumns(); @@ -2981,7 +2981,7 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & sour return ast ? queryToString(ast) : ""; }; - if (query_to_string(getSortingKeyAST()) != query_to_string(src_data->getSortingKeyAST())) + if (query_to_string(my_snapshot->getSortingKeyAST()) != query_to_string(src_snapshot->getSortingKeyAST())) throw Exception("Tables have different ordering", ErrorCodes::BAD_ARGUMENTS); if (query_to_string(my_snapshot->getPartitionKeyAST()) != query_to_string(src_snapshot->getPartitionKeyAST())) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index afd1586ac6c..fb08f379ee3 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -612,8 +612,14 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor NamesAndTypesList merging_columns; Names gathering_column_names, merging_column_names; extractMergingAndGatheringColumns( - storage_columns, data.getSortingKey().expression, metadata_snapshot->getSecondaryIndices(), - data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names); + storage_columns, + metadata_snapshot->getSortingKey().expression, + metadata_snapshot->getSecondaryIndices(), + data.merging_params, + gathering_columns, + gathering_column_names, + merging_columns, + merging_column_names); auto single_disk_volume = std::make_shared("volume_" + future_part.name, disk); MergeTreeData::MutableDataPartPtr new_data_part = data.createPart( @@ -719,16 +725,16 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor Pipe pipe(std::move(input)); - if (data.hasSortingKey()) + if (metadata_snapshot->hasSortingKey()) { - auto expr = std::make_shared(pipe.getHeader(), data.getSortingKey().expression); + auto expr = std::make_shared(pipe.getHeader(), metadata_snapshot->getSortingKey().expression); pipe.addSimpleTransform(std::move(expr)); } pipes.emplace_back(std::move(pipe)); } - Names sort_columns = data.getSortingKeyColumns(); + Names sort_columns = metadata_snapshot->getSortingKeyColumns(); SortDescription sort_description; size_t sort_columns_size = sort_columns.size(); sort_description.reserve(sort_columns_size); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 9fd020d0317..9d198c7ed65 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -617,7 +617,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( if (select.final()) { /// Add columns needed to calculate the sorting expression and the sign. - std::vector add_columns = data.getColumnsRequiredForSortingKey(); + std::vector add_columns = metadata_snapshot->getColumnsRequiredForSortingKey(); column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); if (!data.merging_params.sign_column.empty()) @@ -644,7 +644,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( else if ((settings.optimize_read_in_order || settings.optimize_aggregation_in_order) && query_info.input_order_info) { size_t prefix_size = query_info.input_order_info->order_key_prefix_descr.size(); - auto order_key_prefix_ast = data.getSortingKey().expression_list_ast->clone(); + auto order_key_prefix_ast = metadata_snapshot->getSortingKey().expression_list_ast->clone(); order_key_prefix_ast->children.resize(prefix_size); auto syntax_result = SyntaxAnalyzer(context).analyze(order_key_prefix_ast, data.getColumns().getAllPhysical()); @@ -1064,7 +1064,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( { SortDescription sort_description; for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) - sort_description.emplace_back(data.getSortingKey().column_names[j], + sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j], input_order_info->direction, 1); /// Drop temporary columns, added by 'sorting_key_prefix_expr' @@ -1138,11 +1138,11 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( if (!out_projection) out_projection = createProjection(pipe, data); - pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), data.getSortingKey().expression)); + pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), metadata_snapshot->getSortingKey().expression)); pipes.emplace_back(std::move(pipe)); } - Names sort_columns = data.getSortingKeyColumns(); + Names sort_columns = metadata_snapshot->getSortingKeyColumns(); SortDescription sort_description; size_t sort_columns_size = sort_columns.size(); sort_description.reserve(sort_columns_size); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index f96c9b48c4d..c31cfd3da6f 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -262,10 +262,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->volume->getDisk()->createDirectories(full_path); /// If we need to calculate some columns to sort. - if (data.hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) + if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot)->execute(block); - Names sort_columns = data.getSortingKeyColumns(); + Names sort_columns = metadata_snapshot->getSortingKeyColumns(); SortDescription sort_description; size_t sort_columns_size = sort_columns.size(); sort_description.reserve(sort_columns_size); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 1f62fba03a0..81366db5b2a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -41,11 +41,11 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr /// - When we have only ORDER BY, than store it in "primary key:" row of /metadata /// - When we have both, than store PRIMARY KEY in "primary key:" row and ORDER BY in "sorting key:" row of /metadata if (!data.isPrimaryKeyDefined()) - primary_key = formattedAST(data.getSortingKey().expression_list_ast); + primary_key = formattedAST(metadata_snapshot->getSortingKey().expression_list_ast); else { primary_key = formattedAST(data.getPrimaryKey().expression_list_ast); - sorting_key = formattedAST(data.getSortingKey().expression_list_ast); + sorting_key = formattedAST(metadata_snapshot->getSortingKey().expression_list_ast); } data_format_version = data.format_version; diff --git a/src/Storages/ReadInOrderOptimizer.cpp b/src/Storages/ReadInOrderOptimizer.cpp index bfdbd7ef557..a6cc6211788 100644 --- a/src/Storages/ReadInOrderOptimizer.cpp +++ b/src/Storages/ReadInOrderOptimizer.cpp @@ -30,20 +30,20 @@ ReadInOrderOptimizer::ReadInOrderOptimizer( forbidden_columns.insert(elem.first); } -InputOrderInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & storage) const +InputOrderInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot) const { Names sorting_key_columns; if (const auto * merge_tree = dynamic_cast(storage.get())) { - if (!merge_tree->hasSortingKey()) + if (!metadata_snapshot->hasSortingKey()) return {}; - sorting_key_columns = merge_tree->getSortingKeyColumns(); + sorting_key_columns = metadata_snapshot->getSortingKeyColumns(); } else if (const auto * part = dynamic_cast(storage.get())) { - if (!part->hasSortingKey()) + if (!metadata_snapshot->hasSortingKey()) return {}; - sorting_key_columns = part->getSortingKeyColumns(); + sorting_key_columns = metadata_snapshot->getSortingKeyColumns(); } else /// Inapplicable storage type { diff --git a/src/Storages/ReadInOrderOptimizer.h b/src/Storages/ReadInOrderOptimizer.h index de858e8fd92..f2a3e448f50 100644 --- a/src/Storages/ReadInOrderOptimizer.h +++ b/src/Storages/ReadInOrderOptimizer.h @@ -20,7 +20,7 @@ public: const SortDescription & required_sort_description, const SyntaxAnalyzerResultPtr & syntax_result); - InputOrderInfoPtr getInputOrder(const StoragePtr & storage) const; + InputOrderInfoPtr getInputOrder(const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot) const; private: /// Actions for every element of order expression to analyze functions for monotonicity diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 4882b5fdc1c..5eaaf98d397 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -179,7 +179,7 @@ Pipes StorageBuffer::read( if (dst_has_same_structure) { if (query_info.order_optimizer) - query_info.input_order_info = query_info.order_optimizer->getInputOrder(destination); + query_info.input_order_info = query_info.order_optimizer->getInputOrder(destination, metadata_snapshot); /// The destination table has the same structure of the requested columns and we can simply read blocks from there. pipes_from_dst = destination->read( diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 8fbe415ead6..3c7f474c482 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -312,4 +312,34 @@ Names StorageInMemoryMetadata::getColumnsRequiredForPartitionKey() const return partition_key.expression->getRequiredColumns(); return {}; } + + +const KeyDescription & StorageInMemoryMetadata::getSortingKey() const +{ + return sorting_key; +} + +bool StorageInMemoryMetadata::isSortingKeyDefined() const +{ + return sorting_key.definition_ast != nullptr; +} + +bool StorageInMemoryMetadata::hasSortingKey() const +{ + return !sorting_key.column_names.empty(); +} + +Names StorageInMemoryMetadata::getColumnsRequiredForSortingKey() const +{ + if (hasSortingKey()) + return sorting_key.expression->getRequiredColumns(); + return {}; +} + +Names StorageInMemoryMetadata::getSortingKeyColumns() const +{ + if (hasSortingKey()) + return sorting_key.column_names; + return {}; +} } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 8996f9fc1b9..f7370057410 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -123,6 +123,23 @@ struct StorageInMemoryMetadata bool hasPartitionKey() const; /// Returns column names that need to be read to calculate partition key. Names getColumnsRequiredForPartitionKey() const; + + /// Returns structure with sorting key. + const KeyDescription & getSortingKey() const; + /// Returns ASTExpressionList of sorting key expression for storage or nullptr if there is none. + ASTPtr getSortingKeyAST() const { return sorting_key.definition_ast; } + /// Storage has user-defined (in CREATE query) sorting key. + bool isSortingKeyDefined() const; + /// Storage has sorting key. It means, that it contains at least one column. + bool hasSortingKey() const; + /// Returns column names that need to be read to calculate sorting key. + Names getColumnsRequiredForSortingKey() const; + /// Returns columns names in sorting key specified by user in ORDER BY + /// expression. For example: 'a', 'x * y', 'toStartOfMonth(date)', etc. + Names getSortingKeyColumns() const; + + /// Returns column names that need to be read for FINAL to work. + Names getColumnsRequiredForFinal() const { return getColumnsRequiredForSortingKey(); } }; using StorageMetadataPtr = std::shared_ptr; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index e96a48efc9e..4eba4d6a165 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -121,7 +121,7 @@ Pipes StorageMaterializedView::read( auto metadata_snapshot = storage->getInMemoryMetadataPtr(); if (query_info.order_optimizer) - query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage); + query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage, metadata_snapshot); Pipes pipes = storage->read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index e47cde8de52..9765db35fc3 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -184,7 +184,9 @@ Pipes StorageMerge::read( { for (auto it = selected_tables.begin(); it != selected_tables.end(); ++it) { - auto current_info = query_info.order_optimizer->getInputOrder(std::get<0>(*it)); + auto storage_ptr = std::get<0>(*it); + auto storage_metadata_snapshot = storage_ptr->getInMemoryMetadataPtr(); + auto current_info = query_info.order_optimizer->getInputOrder(storage_ptr, storage_metadata_snapshot); if (it == selected_tables.begin()) input_sorting_info = current_info; else if (!current_info || (input_sorting_info && *current_info != *input_sorting_info)) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fc1bd8538e5..061be8b2821 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -796,7 +796,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column { /// Primary and sorting key become independent after this ALTER so we have to /// save the old ORDER BY expression as the new primary key. - auto old_sorting_key_ast = getSortingKey().definition_ast; + auto old_sorting_key_ast = old_metadata.getSortingKey().definition_ast; primary_key = KeyDescription::getKeyFromAST( old_sorting_key_ast, new_metadata.columns, global_context); } diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 5860facc814..016d52ffdcb 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -122,11 +122,11 @@ protected: throw; } - auto metadadata_snapshot = storage->getInMemoryMetadataPtr(); - columns = metadadata_snapshot->getColumns(); + auto metadata_snapshot = storage->getInMemoryMetadataPtr(); + columns = metadata_snapshot->getColumns(); - cols_required_for_partition_key = metadadata_snapshot->getColumnsRequiredForPartitionKey(); - cols_required_for_sorting_key = storage->getColumnsRequiredForSortingKey(); + cols_required_for_partition_key = metadata_snapshot->getColumnsRequiredForPartitionKey(); + cols_required_for_sorting_key = metadata_snapshot->getColumnsRequiredForSortingKey(); cols_required_for_primary_key = storage->getColumnsRequiredForPrimaryKey(); cols_required_for_sampling = storage->getColumnsRequiredForSampling(); column_sizes = storage->getColumnSizes(); diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 78346d1968f..50982e3c4b7 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -375,7 +375,7 @@ protected: if (columns_mask[src_index++]) { assert(table != nullptr); - if ((expression_ptr = table->getSortingKey().expression_list_ast)) + if ((expression_ptr = metadata_snapshot->getSortingKey().expression_list_ast)) res_columns[res_index++]->insert(queryToString(expression_ptr)); else res_columns[res_index++]->insertDefault(); From eca6e9087ebae69b79d667fc5e74f1190338ab0f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jun 2020 14:52:19 +0300 Subject: [PATCH 075/211] Fix race condition --- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 10 +++++++--- src/Interpreters/InterpreterSelectQuery.h | 6 ++++-- src/Interpreters/MutationsInterpreter.cpp | 10 ++++++---- src/Interpreters/MutationsInterpreter.h | 10 ++++++++-- src/Storages/LiveView/StorageLiveView.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 4 ++-- src/Storages/StorageMergeTree.cpp | 3 ++- 8 files changed, 32 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 869c3ae98d3..3736b482ddf 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -86,7 +86,7 @@ BlockIO InterpreterAlterQuery::execute() auto table_lock_holder = table->lockStructureForShare( false /* because mutation is executed asyncronously */, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - MutationsInterpreter(table, mutation_commands, context, false).validate(table_lock_holder); + MutationsInterpreter(table, metadata_snapshot, mutation_commands, context, false).validate(table_lock_holder); table->mutate(mutation_commands, context); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 038448ef353..42ce69f34c7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -165,8 +165,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, const Context & context_, const StoragePtr & storage_, + const StorageMetadataPtr & metadata_snapshot_, const SelectQueryOptions & options_) - : InterpreterSelectQuery(query_ptr_, context_, nullptr, std::nullopt, storage_, options_.copy().noSubquery()) + : InterpreterSelectQuery(query_ptr_, context_, nullptr, std::nullopt, storage_, options_.copy().noSubquery(), {}, metadata_snapshot_) {} InterpreterSelectQuery::~InterpreterSelectQuery() = default; @@ -214,7 +215,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( std::optional input_pipe_, const StoragePtr & storage_, const SelectQueryOptions & options_, - const Names & required_result_column_names) + const Names & required_result_column_names, + const StorageMetadataPtr & metadata_snapshot_) : options(options_) /// NOTE: the query almost always should be cloned because it will be modified during analysis. , query_ptr(options.modify_inplace ? query_ptr_ : query_ptr_->clone()) @@ -223,6 +225,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( , input(input_) , input_pipe(std::move(input_pipe_)) , log(&Poco::Logger::get("InterpreterSelectQuery")) + , metadata_snapshot(metadata_snapshot_) { checkStackSize(); @@ -255,7 +258,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( table_lock = storage->lockStructureForShare( false, context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); table_id = storage->getStorageID(); - metadata_snapshot = storage->getInMemoryMetadataPtr(); + if (metadata_snapshot == nullptr) + metadata_snapshot = storage->getInMemoryMetadataPtr(); } if (has_input || !joined_tables.resolveTables()) diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 8f7237ffd7e..2f0faa2ba72 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -70,6 +70,7 @@ public: const ASTPtr & query_ptr_, const Context & context_, const StoragePtr & storage_, + const StorageMetadataPtr & metadata_snapshot_ = nullptr, const SelectQueryOptions & = {}); ~InterpreterSelectQuery() override; @@ -98,7 +99,8 @@ private: std::optional input_pipe, const StoragePtr & storage_, const SelectQueryOptions &, - const Names & required_result_column_names = {}); + const Names & required_result_column_names = {}, + const StorageMetadataPtr & metadata_snapshot_= nullptr); ASTSelectQuery & getSelectQuery() { return query_ptr->as(); } @@ -184,13 +186,13 @@ private: StoragePtr storage; StorageID table_id = StorageID::createEmpty(); /// Will be initialized if storage is not nullptr TableStructureReadLockHolder table_lock; - StorageMetadataPtr metadata_snapshot; /// Used when we read from prepared input, not table or subquery. BlockInputStreamPtr input; std::optional input_pipe; Poco::Logger * log; + StorageMetadataPtr metadata_snapshot; }; } diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 0c0227d476f..1a38fcf40f3 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -163,6 +163,7 @@ ColumnDependencies getAllColumnDependencies(const StorageMetadataPtr & metadata_ bool isStorageTouchedByMutations( StoragePtr storage, + const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, Context context_copy) { @@ -183,7 +184,7 @@ bool isStorageTouchedByMutations( /// Interpreter must be alive, when we use result of execute() method. /// For some reason it may copy context and and give it into ExpressionBlockInputStream /// after that we will use context from destroyed stack frame in our stream. - InterpreterSelectQuery interpreter(select_query, context_copy, storage, SelectQueryOptions().ignoreLimits()); + InterpreterSelectQuery interpreter(select_query, context_copy, storage, metadata_snapshot, SelectQueryOptions().ignoreLimits()); BlockInputStreamPtr in = interpreter.execute().getInputStream(); Block block = in->read(); @@ -200,18 +201,19 @@ bool isStorageTouchedByMutations( MutationsInterpreter::MutationsInterpreter( StoragePtr storage_, + const StorageMetadataPtr & metadata_snapshot_, MutationCommands commands_, const Context & context_, bool can_execute_) : storage(std::move(storage_)) - , metadata_snapshot(storage->getInMemoryMetadataPtr()) + , metadata_snapshot(metadata_snapshot_) , commands(std::move(commands_)) , context(context_) , can_execute(can_execute_) { mutation_ast = prepare(!can_execute); SelectQueryOptions limits = SelectQueryOptions().analyze(!can_execute).ignoreLimits(); - select_interpreter = std::make_unique(mutation_ast, context, storage, limits); + select_interpreter = std::make_unique(mutation_ast, context, storage, metadata_snapshot_, limits); } static NameSet getKeyColumns(const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot) @@ -504,7 +506,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } const ASTPtr select_query = prepareInterpreterSelectQuery(stages_copy, /* dry_run = */ true); - InterpreterSelectQuery interpreter{select_query, context, storage, SelectQueryOptions().analyze(/* dry_run = */ false).ignoreLimits()}; + InterpreterSelectQuery interpreter{select_query, context, storage, metadata_snapshot, SelectQueryOptions().analyze(/* dry_run = */ false).ignoreLimits()}; auto first_stage_header = interpreter.getSampleBlock(); auto in = std::make_shared(first_stage_header); diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 158ed8d55af..3c3175c1856 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -15,7 +15,8 @@ namespace DB class Context; /// Return false if the data isn't going to be changed by mutations. -bool isStorageTouchedByMutations(StoragePtr storage, const std::vector & commands, Context context_copy); +bool isStorageTouchedByMutations( + StoragePtr storage, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, Context context_copy); /// Create an input stream that will read data from storage and apply mutation commands (UPDATEs, DELETEs, MATERIALIZEs) /// to this data. @@ -24,7 +25,12 @@ class MutationsInterpreter public: /// Storage to mutate, array of mutations commands and context. If you really want to execute mutation /// use can_execute = true, in other cases (validation, amount of commands) it can be false - MutationsInterpreter(StoragePtr storage_, MutationCommands commands_, const Context & context_, bool can_execute_); + MutationsInterpreter( + StoragePtr storage_, + const StorageMetadataPtr & metadata_snapshot_, + MutationCommands commands_, + const Context & context_, + bool can_execute_); void validate(TableStructureReadLockHolder & table_lock_holder); diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index cb4964f3c55..f1b9459b3d3 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -147,7 +147,7 @@ BlockInputStreamPtr StorageLiveView::completeQuery(Pipes pipes) }; block_context->addExternalTable(getBlocksTableName(), TemporaryTableHolder(global_context, creator)); - InterpreterSelectQuery select(getInnerBlocksQuery(), *block_context, StoragePtr(), SelectQueryOptions(QueryProcessingStage::Complete)); + InterpreterSelectQuery select(getInnerBlocksQuery(), *block_context, StoragePtr(), nullptr, SelectQueryOptions(QueryProcessingStage::Complete)); BlockInputStreamPtr data = std::make_shared(select.execute().getInputStream()); /// Squashing is needed here because the view query can generate a lot of blocks @@ -214,7 +214,7 @@ void StorageLiveView::writeIntoLiveView( }; TemporaryTableHolder blocks_storage(context, creator); - InterpreterSelectQuery select_block(mergeable_query, context, blocks_storage.getTable(), + InterpreterSelectQuery select_block(mergeable_query, context, blocks_storage.getTable(), blocks_storage.getTable()->getInMemoryMetadataPtr(), QueryProcessingStage::WithMergeableState); auto data_mergeable_stream = std::make_shared( diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index fb08f379ee3..02279fe3298 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1019,7 +1019,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor commands_for_part.emplace_back(command); } - if (!isStorageTouchedByMutations(storage_from_source_part, commands_for_part, context_for_reading)) + if (!isStorageTouchedByMutations(storage_from_source_part, metadata_snapshot, commands_for_part, context_for_reading)) { LOG_TRACE(log, "Part {} doesn't change up to mutation version {}", source_part->name, future_part.part_info.mutation); return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info); @@ -1045,7 +1045,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor if (!for_interpreter.empty()) { - interpreter.emplace(storage_from_source_part, for_interpreter, context_for_reading, true); + interpreter.emplace(storage_from_source_part, metadata_snapshot, for_interpreter, context_for_reading, true); in = interpreter->execute(table_lock_holder); updated_header = interpreter->getUpdatedHeader(); in->setProgressCallback(MergeProgressCallback(merge_entry, watch_prev_elapsed, stage_progress)); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c13070fa4a5..f259d74b9ea 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -789,7 +789,8 @@ bool StorageMergeTree::tryMutatePart() if (!commands_for_size_validation.empty()) { - MutationsInterpreter interpreter(shared_from_this(), commands_for_size_validation, global_context, false); + MutationsInterpreter interpreter( + shared_from_this(), metadata_snapshot, commands_for_size_validation, global_context, false); commands_size += interpreter.evaluateCommandsSize(); } From 1da393b2180bedc1827bdd7b8c6356e06db7a993 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jun 2020 15:07:09 +0300 Subject: [PATCH 076/211] Sampling key in StorageInMemoryMetadata --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Storages/IStorage.cpp | 23 ------------------- src/Storages/IStorage.h | 14 +---------- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 ++-- .../ReplicatedMergeTreeTableMetadata.cpp | 2 +- src/Storages/StorageInMemoryMetadata.cpp | 23 +++++++++++++++++++ src/Storages/StorageInMemoryMetadata.h | 11 +++++++++ src/Storages/System/StorageSystemColumns.cpp | 2 +- src/Storages/System/StorageSystemTables.cpp | 2 +- 9 files changed, 41 insertions(+), 42 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 28aa42877d6..9d9a403ab59 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1062,7 +1062,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1)) { - Names columns_for_sampling = storage->getColumnsRequiredForSampling(); + Names columns_for_sampling = metadata_snapshot->getColumnsRequiredForSampling(); additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), columns_for_sampling.begin(), columns_for_sampling.end()); } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 0c2ddc09da8..0a50cf95559 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -348,35 +348,12 @@ Names IStorage::getPrimaryKeyColumns() const return {}; } -const KeyDescription & IStorage::getSamplingKey() const -{ - return metadata->sampling_key; -} - -bool IStorage::isSamplingKeyDefined() const -{ - return metadata->sampling_key.definition_ast != nullptr; -} - -bool IStorage::hasSamplingKey() const -{ - return !metadata->sampling_key.column_names.empty(); -} - -Names IStorage::getColumnsRequiredForSampling() const -{ - if (hasSamplingKey()) - return metadata->sampling_key.expression->getRequiredColumns(); - return {}; -} - TTLTableDescription IStorage::getTableTTLs() const { std::lock_guard lock(ttl_mutex); return metadata->table_ttl; } - bool IStorage::hasAnyTableTTL() const { return hasAnyMoveTTL() || hasRowsTTL(); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 4e1ca81dd10..5aa595b1375 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -101,7 +101,7 @@ public: virtual bool isView() const { return false; } /// Returns true if the storage supports queries with the SAMPLE section. - virtual bool supportsSampling() const { return hasSamplingKey(); } + virtual bool supportsSampling() const { return getInMemoryMetadataPtr()->hasSamplingKey(); } /// Returns true if the storage supports queries with the FINAL section. virtual bool supportsFinal() const { return false; } @@ -442,18 +442,6 @@ public: /// * y', 'toStartOfMonth(date)', etc. Names getPrimaryKeyColumns() const; - /// Returns structure with sampling key. - const KeyDescription & getSamplingKey() const; - /// Returns sampling expression AST for storage or nullptr if there is none. - ASTPtr getSamplingKeyAST() const { return metadata->sampling_key.definition_ast; } - /// Storage has user-defined (in CREATE query) sampling key. - bool isSamplingKeyDefined() const; - /// Storage has sampling key. - bool hasSamplingKey() const; - /// Returns column names that need to be read to calculate sampling key. - Names getColumnsRequiredForSampling() const; - - /// Returns storage policy if storage supports it. virtual StoragePolicyPtr getStoragePolicy() const { return {}; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 9d198c7ed65..58214bae5ca 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -390,7 +390,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); RelativeSize size_of_universum = 0; - const auto & sampling_key = data.getSamplingKey(); + const auto & sampling_key = metadata_snapshot->getSamplingKey(); DataTypePtr sampling_column_type = sampling_key.data_types[0]; if (typeid_cast(sampling_column_type.get())) @@ -459,7 +459,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( /// If sample and final are used together no need to calculate sampling expression twice. /// The first time it was calculated for final, because sample key is a part of the PK. /// So, assume that we already have calculated column. - ASTPtr sampling_key_ast = data.getSamplingKeyAST(); + ASTPtr sampling_key_ast = metadata_snapshot->getSamplingKeyAST(); if (select.final()) { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 81366db5b2a..c3d91f4a5a9 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -29,7 +29,7 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos]; const auto data_settings = data.getSettings(); - sampling_expression = formattedAST(data.getSamplingKeyAST()); + sampling_expression = formattedAST(metadata_snapshot->getSamplingKeyAST()); index_granularity = data_settings->index_granularity; merging_params_mode = static_cast(data.merging_params.mode); sign_column = data.merging_params.sign_column; diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 3c7f474c482..6c5429fc556 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -342,4 +342,27 @@ Names StorageInMemoryMetadata::getSortingKeyColumns() const return sorting_key.column_names; return {}; } + +const KeyDescription & StorageInMemoryMetadata::getSamplingKey() const +{ + return sampling_key; +} + +bool StorageInMemoryMetadata::isSamplingKeyDefined() const +{ + return sampling_key.definition_ast != nullptr; +} + +bool StorageInMemoryMetadata::hasSamplingKey() const +{ + return !sampling_key.column_names.empty(); +} + +Names StorageInMemoryMetadata::getColumnsRequiredForSampling() const +{ + if (hasSamplingKey()) + return sampling_key.expression->getRequiredColumns(); + return {}; +} + } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index f7370057410..1abea7d250c 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -140,6 +140,17 @@ struct StorageInMemoryMetadata /// Returns column names that need to be read for FINAL to work. Names getColumnsRequiredForFinal() const { return getColumnsRequiredForSortingKey(); } + + /// Returns structure with sampling key. + const KeyDescription & getSamplingKey() const; + /// Returns sampling expression AST for storage or nullptr if there is none. + ASTPtr getSamplingKeyAST() const { return sampling_key.definition_ast; } + /// Storage has user-defined (in CREATE query) sampling key. + bool isSamplingKeyDefined() const; + /// Storage has sampling key. + bool hasSamplingKey() const; + /// Returns column names that need to be read to calculate sampling key. + Names getColumnsRequiredForSampling() const; }; using StorageMetadataPtr = std::shared_ptr; diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 016d52ffdcb..f998dc27cab 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -128,7 +128,7 @@ protected: cols_required_for_partition_key = metadata_snapshot->getColumnsRequiredForPartitionKey(); cols_required_for_sorting_key = metadata_snapshot->getColumnsRequiredForSortingKey(); cols_required_for_primary_key = storage->getColumnsRequiredForPrimaryKey(); - cols_required_for_sampling = storage->getColumnsRequiredForSampling(); + cols_required_for_sampling = metadata_snapshot->getColumnsRequiredForSampling(); column_sizes = storage->getColumnSizes(); } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 50982e3c4b7..0852a96e8ba 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -393,7 +393,7 @@ protected: if (columns_mask[src_index++]) { assert(table != nullptr); - if ((expression_ptr = table->getSamplingKeyAST())) + if ((expression_ptr = metadata_snapshot->getSamplingKeyAST())) res_columns[res_index++]->insert(queryToString(expression_ptr)); else res_columns[res_index++]->insertDefault(); From 1afdebeebdb8dcb0559ff51eacd259ea6094a409 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jun 2020 15:39:20 +0300 Subject: [PATCH 077/211] Primary key in storage metadata --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Storages/IStorage.cpp | 29 ------------------- src/Storages/IStorage.h | 15 ---------- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 ++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + .../MergeTree/IMergeTreeDataPartWriter.cpp | 6 ++-- .../MergeTree/IMergeTreeDataPartWriter.h | 2 ++ src/Storages/MergeTree/MergeTreeData.cpp | 19 ++++++------ src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 5 ++-- .../MergeTree/MergeTreeDataPartCompact.h | 1 + .../MergeTree/MergeTreeDataPartWide.cpp | 3 +- .../MergeTree/MergeTreeDataPartWide.h | 1 + .../MergeTreeDataPartWriterCompact.cpp | 8 ++--- .../MergeTreeDataPartWriterCompact.h | 1 + .../MergeTree/MergeTreeDataPartWriterWide.cpp | 8 ++--- .../MergeTree/MergeTreeDataPartWriterWide.h | 1 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 22 +++++++++----- .../MergeTree/MergeTreeDataSelectExecutor.h | 2 ++ .../MergeTree/MergeTreeWhereOptimizer.cpp | 13 +++++---- .../MergeTree/MergeTreeWhereOptimizer.h | 3 ++ .../MergeTree/MergedBlockOutputStream.cpp | 4 +-- .../MergedColumnOnlyOutputStream.cpp | 1 + .../ReplicatedMergeTreeTableMetadata.cpp | 4 +-- src/Storages/StorageInMemoryMetadata.cpp | 28 ++++++++++++++++++ src/Storages/StorageInMemoryMetadata.h | 15 ++++++++++ src/Storages/System/StorageSystemTables.cpp | 2 +- 28 files changed, 114 insertions(+), 91 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 42ce69f34c7..41b2abc33c6 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -330,7 +330,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( current_info.query = query_ptr; current_info.syntax_analyzer_result = syntax_analyzer_result; - MergeTreeWhereOptimizer{current_info, *context, *merge_tree, syntax_analyzer_result->requiredSourceColumns(), log}; + MergeTreeWhereOptimizer{current_info, *context, *merge_tree, metadata_snapshot, syntax_analyzer_result->requiredSourceColumns(), log}; } } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 0a50cf95559..f9b7f41f139 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -319,35 +319,6 @@ NamesAndTypesList IStorage::getVirtuals() const return {}; } -const KeyDescription & IStorage::getPrimaryKey() const -{ - return metadata->primary_key; -} - -bool IStorage::isPrimaryKeyDefined() const -{ - return metadata->primary_key.definition_ast != nullptr; -} - -bool IStorage::hasPrimaryKey() const -{ - return !metadata->primary_key.column_names.empty(); -} - -Names IStorage::getColumnsRequiredForPrimaryKey() const -{ - if (hasPrimaryKey()) - return metadata->primary_key.expression->getRequiredColumns(); - return {}; -} - -Names IStorage::getPrimaryKeyColumns() const -{ - if (!metadata->primary_key.column_names.empty()) - return metadata->primary_key.column_names; - return {}; -} - TTLTableDescription IStorage::getTableTTLs() const { std::lock_guard lock(ttl_mutex); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 5aa595b1375..787b96c9197 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -427,21 +427,6 @@ public: /// Returns data paths if storage supports it, empty vector otherwise. virtual Strings getDataPaths() const { return {}; } - /// Returns structure with primary key. - const KeyDescription & getPrimaryKey() const; - /// Returns ASTExpressionList of primary key expression for storage or nullptr if there is none. - ASTPtr getPrimaryKeyAST() const { return metadata->primary_key.definition_ast; } - /// Storage has user-defined (in CREATE query) sorting key. - bool isPrimaryKeyDefined() const; - /// Storage has primary key (maybe part of some other key). It means, that - /// it contains at least one column. - bool hasPrimaryKey() const; - /// Returns column names that need to be read to calculate primary key. - Names getColumnsRequiredForPrimaryKey() const; - /// Returns columns names in sorting key specified by. For example: 'a', 'x - /// * y', 'toStartOfMonth(date)', etc. - Names getPrimaryKeyColumns() const; - /// Returns storage policy if storage supports it. virtual StoragePolicyPtr getStoragePolicy() const { return {}; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 03b2dea23ba..17ff2259436 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -437,7 +437,8 @@ void IMergeTreeDataPart::loadIndex() if (!index_granularity.isInitialized()) throw Exception("Index granularity is not loaded before index loading", ErrorCodes::LOGICAL_ERROR); - const auto & primary_key = storage.getPrimaryKey(); + auto metadata_snapshot = storage.getInMemoryMetadataPtr(); + const auto & primary_key = metadata_snapshot->getPrimaryKey(); size_t key_size = primary_key.column_names.size(); if (key_size) @@ -842,7 +843,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const String path = getFullRelativePath(); auto metadata_snapshot = storage.getInMemoryMetadataPtr(); - const auto & pk = storage.getPrimaryKey(); + const auto & pk = metadata_snapshot->getPrimaryKey(); if (!checksums.empty()) { if (!pk.column_names.empty() && !checksums.files.count("primary.idx")) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index f5ca0fee070..04babece83e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -86,6 +86,7 @@ public: virtual MergeTreeWriterPtr getWriter( const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & writer_settings, diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index cfda613d31d..73ac7fc0064 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -65,6 +65,7 @@ void IMergeTreeDataPartWriter::Stream::addToChecksums(MergeTreeData::DataPart::C IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( const MergeTreeData::DataPartPtr & data_part_, const NamesAndTypesList & columns_list_, + const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, @@ -73,6 +74,7 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( : data_part(data_part_) , part_path(data_part_->getFullRelativePath()) , storage(data_part_->storage) + , metadata_snapshot(metadata_snapshot_) , columns_list(columns_list_) , marks_file_extension(marks_file_extension_) , index_granularity(index_granularity_) @@ -162,7 +164,7 @@ void IMergeTreeDataPartWriter::fillIndexGranularity(size_t index_granularity_for void IMergeTreeDataPartWriter::initPrimaryIndex() { - if (storage.hasPrimaryKey()) + if (metadata_snapshot->hasPrimaryKey()) { index_file_stream = data_part->volume->getDisk()->writeFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); index_stream = std::make_unique(*index_file_stream); @@ -221,7 +223,7 @@ void IMergeTreeDataPartWriter::calculateAndSerializePrimaryIndex(const Block & p while (index_mark < total_marks && current_row < rows) { - if (storage.hasPrimaryKey()) + if (metadata_snapshot->hasPrimaryKey()) { for (size_t j = 0; j < primary_columns_num; ++j) { diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 149aeaa2f0d..2f849e7c895 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -63,6 +63,7 @@ public: IMergeTreeDataPartWriter( const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc, const String & marks_file_extension, const CompressionCodecPtr & default_codec, @@ -119,6 +120,7 @@ protected: MergeTreeData::DataPartPtr data_part; String part_path; const MergeTreeData & storage; + StorageMetadataPtr metadata_snapshot; NamesAndTypesList columns_list; const String marks_file_extension; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a1487b3a1fe..ff38a21a15f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -173,11 +173,11 @@ MergeTreeData::MergeTreeData( const auto settings = getSettings(); /// NOTE: using the same columns list as is read when performing actual merges. - merging_params.check(getColumns().getAllPhysical()); + merging_params.check(metadata_.getColumns().getAllPhysical()); if (metadata_.sampling_key.definition_ast != nullptr) { - const auto & pk_sample_block = getPrimaryKey().sample_block; + const auto & pk_sample_block = metadata_.getPrimaryKey().sample_block; if (!pk_sample_block.has(metadata_.sampling_key.column_names[0]) && !attach && !settings->compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility. throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS); @@ -410,7 +410,7 @@ ExpressionActionsPtr getCombinedIndicesExpression( ExpressionActionsPtr MergeTreeData::getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const { - return getCombinedIndicesExpression(getPrimaryKey(), metadata_snapshot->getSecondaryIndices(), metadata_snapshot->getColumns(), global_context); + return getCombinedIndicesExpression(metadata_snapshot->getPrimaryKey(), metadata_snapshot->getSecondaryIndices(), metadata_snapshot->getColumns(), global_context); } ExpressionActionsPtr MergeTreeData::getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const @@ -2915,11 +2915,12 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData: return total_covered_parts; } -bool MergeTreeData::isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const +bool MergeTreeData::isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions( + const ASTPtr & node, const StorageMetadataPtr & metadata_snapshot) const { const String column_name = node->getColumnName(); - for (const auto & name : getPrimaryKeyColumns()) + for (const auto & name : metadata_snapshot->getPrimaryKeyColumns()) if (column_name == name) return true; @@ -2929,7 +2930,7 @@ bool MergeTreeData::isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const A if (const auto * func = node->as()) if (func->arguments->children.size() == 1) - return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(func->arguments->children.front()); + return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(func->arguments->children.front(), metadata_snapshot); return false; } @@ -2946,14 +2947,14 @@ bool MergeTreeData::mayBenefitFromIndexForIn( { for (const auto & item : left_in_operand_tuple->arguments->children) { - if (isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(item)) + if (isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(item, metadata_snapshot)) return true; for (const auto & index : metadata_snapshot->getSecondaryIndices()) if (index_wrapper_factory.get(index)->mayBenefitFromIndexForIn(item)) return true; } /// The tuple itself may be part of the primary key, so check that as a last resort. - return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand); + return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand, metadata_snapshot); } else { @@ -2961,7 +2962,7 @@ bool MergeTreeData::mayBenefitFromIndexForIn( if (index_wrapper_factory.get(index)->mayBenefitFromIndexForIn(left_in_operand)) return true; - return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand); + return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand, metadata_snapshot); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 863b5ba1644..fdbe36d10ec 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -812,7 +812,7 @@ protected: DataPartsLock & data_parts_lock) const; /// Checks whether the column is in the primary key, possibly wrapped in a chain of functions with single argument. - bool isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const; + bool isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node, const StorageMetadataPtr & metadata_snapshot) const; /// Common part for |freezePartition()| and |freezeAll()|. using MatcherFn = std::function; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 02279fe3298..fad65e492c0 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1604,7 +1604,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( if (mutating_stream == nullptr) throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR); - if (data.hasPrimaryKey() || metadata_snapshot->hasSecondaryIndices()) + if (metadata_snapshot->hasPrimaryKey() || metadata_snapshot->hasSecondaryIndices()) mutating_stream = std::make_shared( std::make_shared(mutating_stream, data.getPrimaryKeyAndSkipIndicesExpression(metadata_snapshot))); diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 32acc266e42..d45aa882b2a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -54,6 +54,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec, const MergeTreeWriterSettings & writer_settings, @@ -68,8 +69,8 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( { return *getColumnPosition(lhs.name) < *getColumnPosition(rhs.name); }); return std::make_unique( - shared_from_this(), ordered_columns_list, indices_to_recalc, - index_granularity_info.marks_file_extension, + shared_from_this(), ordered_columns_list, metadata_snapshot, + indices_to_recalc, index_granularity_info.marks_file_extension, default_codec, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index b4a2b5fa797..0b27dd53339 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -46,6 +46,7 @@ public: MergeTreeWriterPtr getWriter( const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & writer_settings, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index d96b61b4bea..e4901b1f74d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -53,13 +53,14 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter( const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const { return std::make_unique( - shared_from_this(), columns_list, indices_to_recalc, + shared_from_this(), columns_list, metadata_snapshot, indices_to_recalc, index_granularity_info.marks_file_extension, default_codec, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index ba9e0765510..144dfa86cfb 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -39,6 +39,7 @@ public: MergeTreeWriterPtr getWriter( const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & writer_settings, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 1a7a757c149..f7a3ad75cf5 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -3,19 +3,17 @@ namespace DB { - - MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const MergeTreeData::DataPartPtr & data_part_, const NamesAndTypesList & columns_list_, + const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) - : IMergeTreeDataPartWriter(data_part_, columns_list_, - indices_to_recalc_, marks_file_extension_, - default_codec_, settings_, index_granularity_) + : IMergeTreeDataPartWriter( + data_part_, columns_list_, metadata_snapshot_, indices_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_) { using DataPart = MergeTreeDataPartCompact; String data_file_name = DataPart::DATA_FILE_NAME; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 07caba94712..8183c038c4c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -10,6 +10,7 @@ public: MergeTreeDataPartWriterCompact( const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc, const String & marks_file_extension, const CompressionCodecPtr & default_codec, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 1ab10b55409..e71ea4d4b94 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -16,16 +16,16 @@ namespace MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( const MergeTreeData::DataPartPtr & data_part_, const NamesAndTypesList & columns_list_, + const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) - : IMergeTreeDataPartWriter(data_part_, columns_list_, - indices_to_recalc_, marks_file_extension_, - default_codec_, settings_, index_granularity_) + : IMergeTreeDataPartWriter( + data_part_, columns_list_, metadata_snapshot_, indices_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_) { - const auto & columns = storage.getColumns(); + const auto & columns = metadata_snapshot->getColumns(); for (const auto & it : columns_list) addStreams(it.name, *it.type, columns.getCodecOrDefault(it.name, default_codec), settings.estimated_size); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index acd7f749d00..f5a9d17f63c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -13,6 +13,7 @@ public: MergeTreeDataPartWriterWide( const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const String & marks_file_extension, const CompressionCodecPtr & default_codec, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 58214bae5ca..2cc5fc7dd5b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -100,7 +100,10 @@ static Block getBlockWithPartColumn(const MergeTreeData::DataPartsVector & parts size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead( - const MergeTreeData::DataPartsVector & parts, const KeyCondition & key_condition, const Settings & settings) const + const MergeTreeData::DataPartsVector & parts, + const StorageMetadataPtr & metadata_snapshot, + const KeyCondition & key_condition, + const Settings & settings) const { size_t rows_count = 0; @@ -109,7 +112,7 @@ size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead( for (const auto & part : parts) { - MarkRanges ranges = markRangesFromPKRange(part, key_condition, settings); + MarkRanges ranges = markRangesFromPKRange(part, metadata_snapshot, key_condition, settings); /** In order to get a lower bound on the number of rows that match the condition on PK, * consider only guaranteed full marks. @@ -224,7 +227,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( data.check(real_column_names); const Settings & settings = context.getSettingsRef(); - const auto & primary_key = data.getPrimaryKey(); + const auto & primary_key = metadata_snapshot->getPrimaryKey(); Names primary_key_columns = primary_key.column_names; KeyCondition key_condition(query_info, context, primary_key_columns, primary_key.expression); @@ -326,7 +329,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( /// Convert absolute value of the sampling (in form `SAMPLE 1000000` - how many rows to read) into the relative `SAMPLE 0.1` (how much data to read). size_t approx_total_rows = 0; if (relative_sample_size > 1 || relative_sample_offset > 1) - approx_total_rows = getApproximateTotalRowsToRead(parts, key_condition, settings); + approx_total_rows = getApproximateTotalRowsToRead(parts, metadata_snapshot, key_condition, settings); if (relative_sample_size > 1) { @@ -565,8 +568,8 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( { RangesInDataPart ranges(part, part_index++); - if (data.hasPrimaryKey()) - ranges.ranges = markRangesFromPKRange(part, key_condition, settings); + if (metadata_snapshot->hasPrimaryKey()) + ranges.ranges = markRangesFromPKRange(part, metadata_snapshot, key_condition, settings); else { size_t total_marks_count = part->getMarksCount(); @@ -1297,7 +1300,10 @@ void MergeTreeDataSelectExecutor::createPositiveSignCondition( /// Calculates a set of mark ranges, that could possibly contain keys, required by condition. /// In other words, it removes subranges from whole range, that definitely could not contain required keys. MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( - const MergeTreeData::DataPartPtr & part, const KeyCondition & key_condition, const Settings & settings) const + const MergeTreeData::DataPartPtr & part, + const StorageMetadataPtr & metadata_snapshot, + const KeyCondition & key_condition, + const Settings & settings) const { MarkRanges res; @@ -1335,7 +1341,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( std::function create_field_ref; /// If there are no monotonic functions, there is no need to save block reference. /// Passing explicit field to FieldRef allows to optimize ranges and shows better performance. - const auto & primary_key = data.getPrimaryKey(); + const auto & primary_key = metadata_snapshot->getPrimaryKey(); if (key_condition.hasMonotonicFunctionsChain()) { auto index_block = std::make_shared(); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 7811eb53b71..ba0613a832d 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -91,6 +91,7 @@ private: /// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index. size_t getApproximateTotalRowsToRead( const MergeTreeData::DataPartsVector & parts, + const StorageMetadataPtr & metadata_snapshot, const KeyCondition & key_condition, const Settings & settings) const; @@ -102,6 +103,7 @@ private: MarkRanges markRangesFromPKRange( const MergeTreeData::DataPartPtr & part, + const StorageMetadataPtr & metadata_snapshot, const KeyCondition & key_condition, const Settings & settings) const; diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 61f99ac6d88..7e51bcff27d 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -31,15 +31,16 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( SelectQueryInfo & query_info, const Context & context, const MergeTreeData & data, + const StorageMetadataPtr & metadata_snapshot, const Names & queried_columns_, Poco::Logger * log_) - : table_columns{ext::map(data.getColumns().getAllPhysical(), - [] (const NameAndTypePair & col) { return col.name; })}, - queried_columns{queried_columns_}, - block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)}, - log{log_} + : table_columns{ext::map( + metadata_snapshot->getColumns().getAllPhysical(), [](const NameAndTypePair & col) { return col.name; })} + , queried_columns{queried_columns_} + , block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)} + , log{log_} { - const auto & primary_key = data.getPrimaryKey(); + const auto & primary_key = metadata_snapshot->getPrimaryKey(); if (!primary_key.column_names.empty()) first_primary_key_column = primary_key.column_names[0]; diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index f9fdc01812b..866d0a8754e 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -16,6 +16,8 @@ namespace DB class ASTSelectQuery; class ASTFunction; class MergeTreeData; +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; /** Identifies WHERE expressions that can be placed in PREWHERE by calculating respective * sizes of columns used in particular expression and identifying "good" conditions of @@ -31,6 +33,7 @@ public: SelectQueryInfo & query_info, const Context & context, const MergeTreeData & data, + const StorageMetadataPtr & metadata_snapshot, const Names & queried_columns_, Poco::Logger * log_); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 9cbdc338367..e776a35f21f 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -59,7 +59,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( volume->getDisk()->createDirectories(part_path); - writer = data_part->getWriter(columns_list, skip_indices, default_codec, writer_settings); + writer = data_part->getWriter(columns_list, metadata_snapshot, skip_indices, default_codec, writer_settings); writer->initPrimaryIndex(); writer->initSkipIndices(); } @@ -169,7 +169,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm std::inserter(skip_indexes_column_names_set, skip_indexes_column_names_set.end())); Names skip_indexes_column_names(skip_indexes_column_names_set.begin(), skip_indexes_column_names_set.end()); - Block primary_key_block = getBlockAndPermute(block, storage.getPrimaryKeyColumns(), permutation); + Block primary_key_block = getBlockAndPermute(block, metadata_snapshot->getPrimaryKeyColumns(), permutation); Block skip_indexes_block = getBlockAndPermute(block, skip_indexes_column_names, permutation); writer->write(block, permutation, primary_key_block, skip_indexes_block); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index b74a8243437..1faadd0d720 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -28,6 +28,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( writer = data_part->getWriter( header.getNamesAndTypesList(), + metadata_snapshot_, indices_to_recalc, default_codec, std::move(writer_settings), diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index c3d91f4a5a9..cbb0a665b88 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -40,11 +40,11 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr /// So rules in zookeeper metadata is following: /// - When we have only ORDER BY, than store it in "primary key:" row of /metadata /// - When we have both, than store PRIMARY KEY in "primary key:" row and ORDER BY in "sorting key:" row of /metadata - if (!data.isPrimaryKeyDefined()) + if (!metadata_snapshot->isPrimaryKeyDefined()) primary_key = formattedAST(metadata_snapshot->getSortingKey().expression_list_ast); else { - primary_key = formattedAST(data.getPrimaryKey().expression_list_ast); + primary_key = formattedAST(metadata_snapshot->getPrimaryKey().expression_list_ast); sorting_key = formattedAST(metadata_snapshot->getSortingKey().expression_list_ast); } diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 6c5429fc556..404baa6677f 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -365,4 +365,32 @@ Names StorageInMemoryMetadata::getColumnsRequiredForSampling() const return {}; } +const KeyDescription & StorageInMemoryMetadata::getPrimaryKey() const +{ + return primary_key; +} + +bool StorageInMemoryMetadata::isPrimaryKeyDefined() const +{ + return primary_key.definition_ast != nullptr; +} + +bool StorageInMemoryMetadata::hasPrimaryKey() const +{ + return !primary_key.column_names.empty(); +} + +Names StorageInMemoryMetadata::getColumnsRequiredForPrimaryKey() const +{ + if (hasPrimaryKey()) + return primary_key.expression->getRequiredColumns(); + return {}; +} + +Names StorageInMemoryMetadata::getPrimaryKeyColumns() const +{ + if (!primary_key.column_names.empty()) + return primary_key.column_names; + return {}; +} } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 1abea7d250c..51036403e1f 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -151,6 +151,21 @@ struct StorageInMemoryMetadata bool hasSamplingKey() const; /// Returns column names that need to be read to calculate sampling key. Names getColumnsRequiredForSampling() const; + + /// Returns structure with primary key. + const KeyDescription & getPrimaryKey() const; + /// Returns ASTExpressionList of primary key expression for storage or nullptr if there is none. + ASTPtr getPrimaryKeyAST() const { return primary_key.definition_ast; } + /// Storage has user-defined (in CREATE query) sorting key. + bool isPrimaryKeyDefined() const; + /// Storage has primary key (maybe part of some other key). It means, that + /// it contains at least one column. + bool hasPrimaryKey() const; + /// Returns column names that need to be read to calculate primary key. + Names getColumnsRequiredForPrimaryKey() const; + /// Returns columns names in sorting key specified by. For example: 'a', 'x + /// * y', 'toStartOfMonth(date)', etc. + Names getPrimaryKeyColumns() const; }; using StorageMetadataPtr = std::shared_ptr; diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 0852a96e8ba..f8f40026940 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -384,7 +384,7 @@ protected: if (columns_mask[src_index++]) { assert(table != nullptr); - if ((expression_ptr = table->getPrimaryKey().expression_list_ast)) + if ((expression_ptr = metadata_snapshot->getPrimaryKey().expression_list_ast)) res_columns[res_index++]->insert(queryToString(expression_ptr)); else res_columns[res_index++]->insertDefault(); From 7064a366e2638588aa8352ebe105c76b86bd9e92 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jun 2020 15:40:27 +0300 Subject: [PATCH 078/211] Missed change for primary key --- src/Storages/System/StorageSystemColumns.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index f998dc27cab..83178870ba9 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -127,7 +127,7 @@ protected: cols_required_for_partition_key = metadata_snapshot->getColumnsRequiredForPartitionKey(); cols_required_for_sorting_key = metadata_snapshot->getColumnsRequiredForSortingKey(); - cols_required_for_primary_key = storage->getColumnsRequiredForPrimaryKey(); + cols_required_for_primary_key = metadata_snapshot->getColumnsRequiredForPrimaryKey(); cols_required_for_sampling = metadata_snapshot->getColumnsRequiredForSampling(); column_sizes = storage->getColumnSizes(); } From ed8f3b2fc42f73a82ca133240ff0642c990ab129 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jun 2020 16:39:26 +0300 Subject: [PATCH 079/211] TTL in storage in memory metadata --- src/DataStreams/TTLBlockInputStream.cpp | 32 +++++++------ src/DataStreams/TTLBlockInputStream.h | 2 + src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Interpreters/InterpreterOptimizeQuery.cpp | 3 +- src/Interpreters/MutationsInterpreter.cpp | 2 +- src/Storages/IStorage.cpp | 47 ------------------- src/Storages/IStorage.h | 27 +++-------- src/Storages/MergeTree/MergeTreeData.cpp | 6 ++- src/Storages/MergeTree/MergeTreeData.h | 20 ++++---- .../MergeTree/MergeTreeDataMergerMutator.cpp | 8 ++-- .../MergeTree/MergeTreeDataWriter.cpp | 10 ++-- src/Storages/StorageBuffer.cpp | 15 ++++-- src/Storages/StorageBuffer.h | 2 +- src/Storages/StorageInMemoryMetadata.h | 3 ++ src/Storages/StorageMaterializedView.cpp | 12 ++++- src/Storages/StorageMaterializedView.h | 8 +++- src/Storages/StorageMergeTree.cpp | 9 +++- src/Storages/StorageMergeTree.h | 8 +++- src/Storages/StorageReplicatedMergeTree.cpp | 9 +++- src/Storages/StorageReplicatedMergeTree.h | 2 +- 20 files changed, 109 insertions(+), 118 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index c79abff98cd..65e01a73f95 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -20,10 +20,12 @@ namespace ErrorCodes TTLBlockInputStream::TTLBlockInputStream( const BlockInputStreamPtr & input_, const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, const MergeTreeData::MutableDataPartPtr & data_part_, time_t current_time_, bool force_) : storage(storage_) + , metadata_snapshot(metadata_snapshot_) , data_part(data_part_) , current_time(current_time_) , force(force_) @@ -34,11 +36,11 @@ TTLBlockInputStream::TTLBlockInputStream( children.push_back(input_); header = children.at(0)->getHeader(); - const auto & storage_columns = storage.getColumns(); + const auto & storage_columns = metadata_snapshot->getColumns(); const auto & column_defaults = storage_columns.getDefaults(); ASTPtr default_expr_list = std::make_shared(); - for (const auto & [name, _] : storage.getColumnTTLs()) + for (const auto & [name, _] : metadata_snapshot->getColumnTTLs()) { auto it = column_defaults.find(name); if (it != column_defaults.end()) @@ -65,13 +67,12 @@ TTLBlockInputStream::TTLBlockInputStream( if (!default_expr_list->children.empty()) { - auto syntax_result = SyntaxAnalyzer(storage.global_context).analyze( - default_expr_list, storage.getColumns().getAllPhysical()); + auto syntax_result = SyntaxAnalyzer(storage.global_context).analyze(default_expr_list, metadata_snapshot->getColumns().getAllPhysical()); defaults_expression = ExpressionAnalyzer{default_expr_list, syntax_result, storage.global_context}.getActions(true); } - auto storage_rows_ttl = storage.getRowsTTL(); - if (storage.hasRowsTTL() && storage_rows_ttl.mode == TTLMode::GROUP_BY) + auto storage_rows_ttl = metadata_snapshot->getRowsTTL(); + if (metadata_snapshot->hasRowsTTL() && storage_rows_ttl.mode == TTLMode::GROUP_BY) { current_key_value.resize(storage_rows_ttl.group_by_keys.size()); @@ -106,14 +107,15 @@ bool TTLBlockInputStream::isTTLExpired(time_t ttl) const Block TTLBlockInputStream::readImpl() { /// Skip all data if table ttl is expired for part - auto storage_rows_ttl = storage.getRowsTTL(); - if (storage.hasRowsTTL() && !storage_rows_ttl.where_expression && - storage_rows_ttl.mode != TTLMode::GROUP_BY && isTTLExpired(old_ttl_infos.table_ttl.max)) + auto storage_rows_ttl = metadata_snapshot->getRowsTTL(); + if (metadata_snapshot->hasRowsTTL() && !storage_rows_ttl.where_expression && storage_rows_ttl.mode != TTLMode::GROUP_BY + && isTTLExpired(old_ttl_infos.table_ttl.max)) { rows_removed = data_part->rows_count; return {}; } + Block block = children.at(0)->read(); if (!block) { @@ -127,7 +129,7 @@ Block TTLBlockInputStream::readImpl() return block; } - if (storage.hasRowsTTL() && (force || isTTLExpired(old_ttl_infos.table_ttl.min))) + if (metadata_snapshot->hasRowsTTL() && (force || isTTLExpired(old_ttl_infos.table_ttl.min))) removeRowsWithExpiredTableTTL(block); removeValuesWithExpiredColumnTTL(block); @@ -153,7 +155,7 @@ void TTLBlockInputStream::readSuffixImpl() void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) { - auto rows_ttl = storage.getRowsTTL(); + auto rows_ttl = metadata_snapshot->getRowsTTL(); rows_ttl.expression->execute(block); if (rows_ttl.where_expression) @@ -201,7 +203,7 @@ void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) size_t rows_aggregated = 0; size_t current_key_start = 0; size_t rows_with_current_key = 0; - auto storage_rows_ttl = storage.getRowsTTL(); + auto storage_rows_ttl = metadata_snapshot->getRowsTTL(); for (size_t i = 0; i < block.rows(); ++i) { UInt32 cur_ttl = getTimestampByIndex(ttl_column, i); @@ -278,7 +280,7 @@ void TTLBlockInputStream::finalizeAggregates(MutableColumns & result_columns) if (!agg_result.empty()) { auto aggregated_res = aggregator->convertToBlocks(agg_result, true, 1); - auto storage_rows_ttl = storage.getRowsTTL(); + auto storage_rows_ttl = metadata_snapshot->getRowsTTL(); for (auto & agg_block : aggregated_res) { for (const auto & it : storage_rows_ttl.set_parts) @@ -310,7 +312,7 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) } std::vector columns_to_remove; - for (const auto & [name, ttl_entry] : storage.getColumnTTLs()) + for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs()) { /// If we read not all table columns. E.g. while mutation. if (!block.has(name)) @@ -371,7 +373,7 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) void TTLBlockInputStream::updateMovesTTL(Block & block) { std::vector columns_to_remove; - for (const auto & ttl_entry : storage.getMoveTTLs()) + for (const auto & ttl_entry : metadata_snapshot->getMoveTTLs()) { auto & new_ttl_info = new_ttl_infos.moves_ttl[ttl_entry.result_column]; diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/DataStreams/TTLBlockInputStream.h index 060306f7d2d..3f37f35426c 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/DataStreams/TTLBlockInputStream.h @@ -16,6 +16,7 @@ public: TTLBlockInputStream( const BlockInputStreamPtr & input_, const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, const MergeTreeData::MutableDataPartPtr & data_part_, time_t current_time, bool force_ @@ -33,6 +34,7 @@ protected: private: const MergeTreeData & storage; + StorageMetadataPtr metadata_snapshot; /// ttl_infos and empty_columns are updating while reading const MergeTreeData::MutableDataPartPtr & data_part; diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 3736b482ddf..958291d5882 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -69,7 +69,7 @@ BlockIO InterpreterAlterQuery::execute() } else if (auto mut_command = MutationCommand::parse(command_ast)) { - if (mut_command->type == MutationCommand::MATERIALIZE_TTL && !table->hasAnyTTL()) + if (mut_command->type == MutationCommand::MATERIALIZE_TTL && !metadata_snapshot->hasAnyTTL()) throw Exception("Cannot MATERIALIZE TTL as there is no TTL set for table " + table->getStorageID().getNameForLogs(), ErrorCodes::INCORRECT_QUERY); diff --git a/src/Interpreters/InterpreterOptimizeQuery.cpp b/src/Interpreters/InterpreterOptimizeQuery.cpp index c47fe1160cf..680dd9b803b 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -26,7 +26,8 @@ BlockIO InterpreterOptimizeQuery::execute() auto table_id = context.resolveStorageID(ast, Context::ResolveOrdinary); StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); - table->optimize(query_ptr, ast.partition, ast.final, ast.deduplicate, context); + auto metadata_snapshot = table->getInMemoryMetadataPtr(); + table->optimize(query_ptr, metadata_snapshot, ast.partition, ast.final, ast.deduplicate, context); return {}; } diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 1a38fcf40f3..694e114af7a 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -411,7 +411,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } else if (command.type == MutationCommand::MATERIALIZE_TTL) { - if (storage->hasRowsTTL()) + if (metadata_snapshot->hasRowsTTL()) { for (const auto & column : all_columns) dependencies.emplace(column.name, ColumnDependency::TTL_TARGET); diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index f9b7f41f139..3741a06fc4a 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -319,53 +319,6 @@ NamesAndTypesList IStorage::getVirtuals() const return {}; } -TTLTableDescription IStorage::getTableTTLs() const -{ - std::lock_guard lock(ttl_mutex); - return metadata->table_ttl; -} - -bool IStorage::hasAnyTableTTL() const -{ - return hasAnyMoveTTL() || hasRowsTTL(); -} - -TTLColumnsDescription IStorage::getColumnTTLs() const -{ - std::lock_guard lock(ttl_mutex); - return metadata->column_ttls_by_name; -} - -bool IStorage::hasAnyColumnTTL() const -{ - std::lock_guard lock(ttl_mutex); - return !metadata->column_ttls_by_name.empty(); -} - -TTLDescription IStorage::getRowsTTL() const -{ - std::lock_guard lock(ttl_mutex); - return metadata->table_ttl.rows_ttl; -} - -bool IStorage::hasRowsTTL() const -{ - std::lock_guard lock(ttl_mutex); - return metadata->table_ttl.rows_ttl.expression != nullptr; -} - -TTLDescriptions IStorage::getMoveTTLs() const -{ - std::lock_guard lock(ttl_mutex); - return metadata->table_ttl.move_ttl; -} - -bool IStorage::hasAnyMoveTTL() const -{ - std::lock_guard lock(ttl_mutex); - return !metadata->table_ttl.move_ttl.empty(); -} - ASTPtr IStorage::getSettingsChanges() const { if (metadata->settings_changes) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 787b96c9197..4eea343db5d 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -129,8 +129,6 @@ public: /// Example is StorageSystemNumbers. virtual bool hasEvenlyDistributedRead() const { return false; } - /// Returns true if there is set table TTL, any column TTL or any move TTL. - bool hasAnyTTL() const { return hasAnyColumnTTL() || hasAnyTableTTL(); } /// Optional size information of each physical column. /// Currently it's only used by the MergeTree family for query optimizations. @@ -362,7 +360,13 @@ public: /** Perform any background work. For example, combining parts in a MergeTree type table. * Returns whether any work has been done. */ - virtual bool optimize(const ASTPtr & /*query*/, const ASTPtr & /*partition*/, bool /*final*/, bool /*deduplicate*/, const Context & /*context*/) + virtual bool optimize( + const ASTPtr & /*query*/, + const StorageMetadataPtr & /*metadata_snapshot*/, + const ASTPtr & /*partition*/, + bool /*final*/, + bool /*deduplicate*/, + const Context & /*context*/) { throw Exception("Method optimize is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } @@ -430,23 +434,6 @@ public: /// Returns storage policy if storage supports it. virtual StoragePolicyPtr getStoragePolicy() const { return {}; } - /// Common tables TTLs (for rows and moves). - TTLTableDescription getTableTTLs() const; - bool hasAnyTableTTL() const; - - /// Separate TTLs for columns. - TTLColumnsDescription getColumnTTLs() const; - bool hasAnyColumnTTL() const; - - /// Just wrapper for table TTLs, return rows part of table TTLs. - TTLDescription getRowsTTL() const; - bool hasRowsTTL() const; - - /// Just wrapper for table TTLs, return moves (to disks or volumes) parts of - /// table TTL. - TTLDescriptions getMoveTTLs() const; - bool hasAnyMoveTTL() const; - /// If it is possible to quickly determine exact number of rows in the table at this moment of time, then return it. /// Used for: /// - Simple count() opimization diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ff38a21a15f..8c94325cd4b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2801,8 +2801,9 @@ MergeTreeData::selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & tt { time_t max_max_ttl = 0; TTLDescriptions::const_iterator best_entry_it; + auto metadata_snapshot = getInMemoryMetadataPtr(); - const auto & move_ttl_entries = getMoveTTLs(); + const auto & move_ttl_entries = metadata_snapshot->getMoveTTLs(); for (auto ttl_entry_it = move_ttl_entries.begin(); ttl_entry_it != move_ttl_entries.end(); ++ttl_entry_it) { auto ttl_info_it = ttl_infos.moves_ttl.find(ttl_entry_it->result_column); @@ -3235,11 +3236,12 @@ bool MergeTreeData::selectPartsAndMove() bool MergeTreeData::areBackgroundMovesNeeded() const { auto policy = getStoragePolicy(); + auto metadata_snapshot = getInMemoryMetadataPtr(); if (policy->getVolumes().size() > 1) return true; - return policy->getVolumes().size() == 1 && policy->getVolumes()[0]->getDisks().size() > 1 && hasAnyMoveTTL(); + return policy->getVolumes().size() == 1 && policy->getVolumes()[0]->getDisks().size() > 1 && metadata_snapshot->hasAnyMoveTTL(); } bool MergeTreeData::movePartsToSpace(const DataPartsVector & parts, SpacePtr space) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index fdbe36d10ec..af6bee4936c 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -607,14 +607,18 @@ public: static ReservationPtr tryReserveSpace(UInt64 expected_size, SpacePtr space); /// Reserves space at least 1MB preferring best destination according to `ttl_infos`. - ReservationPtr reserveSpacePreferringTTLRules(UInt64 expected_size, - const IMergeTreeDataPart::TTLInfos & ttl_infos, - time_t time_of_move, - size_t min_volume_index = 0) const; - ReservationPtr tryReserveSpacePreferringTTLRules(UInt64 expected_size, - const IMergeTreeDataPart::TTLInfos & ttl_infos, - time_t time_of_move, - size_t min_volume_index = 0) const; + ReservationPtr reserveSpacePreferringTTLRules( + UInt64 expected_size, + const IMergeTreeDataPart::TTLInfos & ttl_infos, + time_t time_of_move, + size_t min_volume_index = 0) const; + + ReservationPtr tryReserveSpacePreferringTTLRules( + UInt64 expected_size, + const IMergeTreeDataPart::TTLInfos & ttl_infos, + time_t time_of_move, + size_t min_volume_index = 0) const; + /// Choose disk with max available free space /// Reserves 0 bytes ReservationPtr makeEmptyReservationOnLargestDisk() { return getStoragePolicy()->makeEmptyReservationOnLargestDisk(); } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index fad65e492c0..3024adafb27 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -801,7 +801,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor merged_stream = std::make_shared(merged_stream, SizeLimits(), 0 /*limit_hint*/, Names()); if (need_remove_expired_values) - merged_stream = std::make_shared(merged_stream, data, new_data_part, time_of_merge, force_ttl); + merged_stream = std::make_shared(merged_stream, data, metadata_snapshot, new_data_part, time_of_merge, force_ttl); if (metadata_snapshot->hasSecondaryIndices()) @@ -1576,7 +1576,7 @@ std::set MergeTreeDataMergerMutator::getIndicesToRecalculate( bool MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const Names & columns, const MutationCommands & commands) const { - if (!data.hasAnyTTL()) + if (!metadata_snapshot->hasAnyTTL()) return false; for (const auto & command : commands) @@ -1609,7 +1609,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( std::make_shared(mutating_stream, data.getPrimaryKeyAndSkipIndicesExpression(metadata_snapshot))); if (need_remove_expired_values) - mutating_stream = std::make_shared(mutating_stream, data, new_data_part, time_of_mutation, true); + mutating_stream = std::make_shared(mutating_stream, data, metadata_snapshot, new_data_part, time_of_mutation, true); IMergeTreeDataPart::MinMaxIndex minmax_idx; @@ -1656,7 +1656,7 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR); if (need_remove_expired_values) - mutating_stream = std::make_shared(mutating_stream, data, new_data_part, time_of_mutation, true); + mutating_stream = std::make_shared(mutating_stream, data, metadata_snapshot, new_data_part, time_of_mutation, true); IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets; MergedColumnOnlyOutputStream out( diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index c31cfd3da6f..5974f366b66 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -230,11 +230,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa size_t expected_size = block.bytes(); DB::IMergeTreeDataPart::TTLInfos move_ttl_infos; - const auto & move_ttl_entries = data.getMoveTTLs(); + const auto & move_ttl_entries = metadata_snapshot->getMoveTTLs(); for (const auto & ttl_entry : move_ttl_entries) updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); - NamesAndTypesList columns = data.getColumns().getAllPhysical().filter(block.getNames()); + NamesAndTypesList columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr)); VolumePtr volume = data.getStoragePolicy()->getVolume(0); @@ -289,10 +289,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocksAlreadySorted); } - if (data.hasRowsTTL()) - updateTTL(data.getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true); + if (metadata_snapshot->hasRowsTTL()) + updateTTL(metadata_snapshot->getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true); - for (const auto & [name, ttl_entry] : data.getColumnTTLs()) + for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs()) updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true); new_data_part->ttl_infos.update(move_ttl_infos); diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 5eaaf98d397..13b37980c56 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -484,7 +484,7 @@ void StorageBuffer::shutdown() try { - optimize(nullptr /*query*/, {} /*partition*/, false /*final*/, false /*deduplicate*/, global_context); + optimize(nullptr /*query*/, getInMemoryMetadataPtr(), {} /*partition*/, false /*final*/, false /*deduplicate*/, global_context); } catch (...) { @@ -503,7 +503,13 @@ void StorageBuffer::shutdown() * * This kind of race condition make very hard to implement proper tests. */ -bool StorageBuffer::optimize(const ASTPtr & /*query*/, const ASTPtr & partition, bool final, bool deduplicate, const Context & /*context*/) +bool StorageBuffer::optimize( + const ASTPtr & /*query*/, + const StorageMetadataPtr & /*metadata_snapshot*/, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Context & /*context*/) { if (partition) throw Exception("Partition cannot be specified when optimizing table of type Buffer", ErrorCodes::NOT_IMPLEMENTED); @@ -793,11 +799,12 @@ void StorageBuffer::alter(const AlterCommands & params, const Context & context, auto table_id = getStorageID(); checkAlterIsPossible(params, context.getSettingsRef()); + auto metadata_snapshot = getInMemoryMetadataPtr(); /// So that no blocks of the old structure remain. - optimize({} /*query*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context); + optimize({} /*query*/, metadata_snapshot, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context); - StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); + StorageInMemoryMetadata new_metadata = *metadata_snapshot; params.apply(new_metadata, context); DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata); setInMemoryMetadata(new_metadata); diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 403b6c53172..ceedbd25a0c 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -69,7 +69,7 @@ public: void startup() override; /// Flush all buffers into the subordinate table and stop background thread. void shutdown() override; - bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override; + bool optimize(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override; bool supportsSampling() const override { return true; } bool supportsPrewhere() const override diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 51036403e1f..cc6fcfbe083 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -86,6 +86,9 @@ struct StorageInMemoryMetadata const ConstraintsDescription & getConstraints() const; + /// Returns true if there is set table TTL, any column TTL or any move TTL. + bool hasAnyTTL() const { return hasAnyColumnTTL() || hasAnyTableTTL(); } + /// Common tables TTLs (for rows and moves). TTLTableDescription getTableTTLs() const; bool hasAnyTableTTL() const; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 4eba4d6a165..319e1631ae6 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -187,10 +187,18 @@ void StorageMaterializedView::checkStatementCanBeForwarded() const + "Execute the statement directly on it.", ErrorCodes::INCORRECT_QUERY); } -bool StorageMaterializedView::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) +bool StorageMaterializedView::optimize( + const ASTPtr & query, + const StorageMetadataPtr & /*metadata_snapshot*/, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Context & context) { checkStatementCanBeForwarded(); - return getTargetTable()->optimize(query, partition, final, deduplicate, context); + auto storage_ptr = getTargetTable(); + auto metadata_snapshot = storage_ptr->getInMemoryMetadataPtr(); + return getTargetTable()->optimize(query, metadata_snapshot, partition, final, deduplicate, context); } void StorageMaterializedView::alter( diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index ef895ff0165..6f462c2cccc 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -39,7 +39,13 @@ public: void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; - bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override; + bool optimize( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Context & context) override; void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index f259d74b9ea..40b3aeffb8a 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -693,7 +693,7 @@ bool StorageMergeTree::merge( { /// Force filter by TTL in 'OPTIMIZE ... FINAL' query to remove expired values from old parts /// without TTL infos or with outdated TTL infos, e.g. after 'ALTER ... MODIFY TTL' query. - bool force_ttl = (final && hasAnyTTL()); + bool force_ttl = (final && metadata_snapshot->hasAnyTTL()); new_part = merger_mutator.mergePartsToTemporaryPart( future_part, metadata_snapshot, *merge_entry, table_lock_holder, time(nullptr), @@ -965,7 +965,12 @@ void StorageMergeTree::clearOldMutations(bool truncate) } bool StorageMergeTree::optimize( - const ASTPtr & /*query*/, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) + const ASTPtr & /*query*/, + const StorageMetadataPtr & /*metadata_snapshot*/, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Context & context) { String disable_reason; if (!partition && final) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 4b6da58572b..69ee6714164 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -53,7 +53,13 @@ public: /** Perform the next step in combining the parts. */ - bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override; + bool optimize( + const ASTPtr & query, + const StorageMetadataPtr & /*metadata_snapshot*/, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Context & context) override; void alterPartition( const ASTPtr & query, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 061be8b2821..b0a7e550233 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3475,7 +3475,12 @@ BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, bool StorageReplicatedMergeTree::optimize( - const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & query_context) + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Context & query_context) { assertNotReadonly(); @@ -3498,7 +3503,7 @@ bool StorageReplicatedMergeTree::optimize( return false; }; - bool force_ttl = (final && hasAnyTTL()); + bool force_ttl = (final && metadata_snapshot->hasAnyTTL()); const auto storage_settings_ptr = getSettings(); if (!partition && final) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index fe1b052b717..c98fcb0ae3d 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -101,7 +101,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; - bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & query_context) override; + bool optimize(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const ASTPtr & partition, bool final, bool deduplicate, const Context & query_context) override; void alter(const AlterCommands & params, const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) override; From eaaef837426ac9dc61d7c8c257261f82ba896744 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jun 2020 16:46:01 +0300 Subject: [PATCH 080/211] Settings changes in StorageInMemoryMetadata --- src/Storages/IStorage.cpp | 7 ------- src/Storages/IStorage.h | 5 ----- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- src/Storages/StorageInMemoryMetadata.cpp | 8 ++++++++ src/Storages/StorageInMemoryMetadata.h | 4 ++++ 5 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 3741a06fc4a..6b3aafa1784 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -319,13 +319,6 @@ NamesAndTypesList IStorage::getVirtuals() const return {}; } -ASTPtr IStorage::getSettingsChanges() const -{ - if (metadata->settings_changes) - return metadata->settings_changes->clone(); - return nullptr; -} - const SelectQueryDescription & IStorage::getSelectQuery() const { return metadata->select; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 4eea343db5d..d9e9aa247c2 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -138,11 +138,6 @@ public: public: /// thread-unsafe part. lockStructure must be acquired const ColumnsDescription & getColumns() const; /// returns combined set of columns - - /// Storage settings - ASTPtr getSettingsChanges() const; - bool hasSettingsChanges() const { return metadata->settings_changes != nullptr; } - /// Select query for *View storages. const SelectQueryDescription & getSelectQuery() const; bool hasSelectQuery() const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8c94325cd4b..115e0b78bf0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1333,10 +1333,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S checkTTLExpressions(new_metadata, old_metadata); - if (hasSettingsChanges()) + if (old_metadata.hasSettingsChanges()) { - const auto current_changes = getSettingsChanges()->as().changes; + const auto current_changes = old_metadata.getSettingsChanges()->as().changes; const auto & new_changes = new_metadata.settings_changes->as().changes; for (const auto & changed_setting : new_changes) { diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 404baa6677f..5f8c83d6e14 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -393,4 +393,12 @@ Names StorageInMemoryMetadata::getPrimaryKeyColumns() const return primary_key.column_names; return {}; } + +ASTPtr StorageInMemoryMetadata::getSettingsChanges() const +{ + if (settings_changes) + return settings_changes->clone(); + return nullptr; +} + } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index cc6fcfbe083..1de17d768ae 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -169,6 +169,10 @@ struct StorageInMemoryMetadata /// Returns columns names in sorting key specified by. For example: 'a', 'x /// * y', 'toStartOfMonth(date)', etc. Names getPrimaryKeyColumns() const; + + /// Storage settings + ASTPtr getSettingsChanges() const; + bool hasSettingsChanges() const { return settings_changes != nullptr; } }; using StorageMetadataPtr = std::shared_ptr; From 31abbe5dbd3b43c2968bb558c747598a6904f326 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jun 2020 17:06:22 +0300 Subject: [PATCH 081/211] Select query in metadata --- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Storages/IStorage.cpp | 10 ---------- src/Storages/IStorage.h | 3 --- src/Storages/StorageInMemoryMetadata.cpp | 9 +++++++++ src/Storages/StorageInMemoryMetadata.h | 4 ++++ src/Storages/StorageMaterializedView.cpp | 11 +++++++---- src/Storages/StorageView.cpp | 2 +- src/Storages/StorageView.h | 4 ++-- 9 files changed, 25 insertions(+), 22 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 2e02c26d38c..e6e368f78e9 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -79,7 +79,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( StoragePtr inner_table = materialized_view->getTargetTable(); auto inner_table_id = inner_table->getStorageID(); - query = materialized_view->getSelectQuery().inner_query; + query = dependent_metadata_snapshot->getSelectQuery().inner_query; std::unique_ptr insert = std::make_unique(); insert->table_id = inner_table_id; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 41b2abc33c6..509825e75e4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -308,7 +308,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it. ASTPtr view_table; if (view) - view->replaceWithSubquery(getSelectQuery(), view_table); + view->replaceWithSubquery(getSelectQuery(), view_table, metadata_snapshot); syntax_analyzer_result = SyntaxAnalyzer(*context).analyzeSelect( query_ptr, SyntaxAnalyzerResult(source_header.getNamesAndTypesList(), storage), diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 6b3aafa1784..a67229d6231 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -319,14 +319,4 @@ NamesAndTypesList IStorage::getVirtuals() const return {}; } -const SelectQueryDescription & IStorage::getSelectQuery() const -{ - return metadata->select; -} - -bool IStorage::hasSelectQuery() const -{ - return metadata->select.select_query != nullptr; -} - } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index d9e9aa247c2..4b712853b53 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -138,9 +138,6 @@ public: public: /// thread-unsafe part. lockStructure must be acquired const ColumnsDescription & getColumns() const; /// returns combined set of columns - /// Select query for *View storages. - const SelectQueryDescription & getSelectQuery() const; - bool hasSelectQuery() const; StorageInMemoryMetadata getInMemoryMetadata() const { return *metadata; } StorageMetadataPtr getInMemoryMetadataPtr() const { return metadata; } diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 5f8c83d6e14..3b72dd97089 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -400,5 +400,14 @@ ASTPtr StorageInMemoryMetadata::getSettingsChanges() const return settings_changes->clone(); return nullptr; } +const SelectQueryDescription & StorageInMemoryMetadata::getSelectQuery() const +{ + return select; +} + +bool StorageInMemoryMetadata::hasSelectQuery() const +{ + return select.select_query != nullptr; +} } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 1de17d768ae..efda4377dfc 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -173,6 +173,10 @@ struct StorageInMemoryMetadata /// Storage settings ASTPtr getSettingsChanges() const; bool hasSettingsChanges() const { return settings_changes != nullptr; } + + /// Select query for *View storages. + const SelectQueryDescription & getSelectQuery() const; + bool hasSelectQuery() const; }; using StorageMetadataPtr = std::shared_ptr; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 319e1631ae6..3d3137fe1a6 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -165,7 +165,7 @@ static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, void StorageMaterializedView::drop() { auto table_id = getStorageID(); - const auto & select_query = getSelectQuery(); + const auto & select_query = getInMemoryMetadataPtr()->getSelectQuery(); if (!select_query.select_table_id.empty()) DatabaseCatalog::instance().removeDependency(select_query.select_table_id, table_id); @@ -209,13 +209,14 @@ void StorageMaterializedView::alter( lockStructureExclusively(table_lock_holder, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto table_id = getStorageID(); StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); + StorageInMemoryMetadata old_metadata = getInMemoryMetadata(); params.apply(new_metadata, context); /// start modify query if (context.getSettingsRef().allow_experimental_alter_materialized_view_structure) { const auto & new_select = new_metadata.select; - const auto & old_select = getSelectQuery(); + const auto & old_select = old_metadata.getSelectQuery(); DatabaseCatalog::instance().updateDependency(old_select.select_table_id, table_id, new_select.select_table_id, table_id); @@ -268,6 +269,7 @@ void StorageMaterializedView::mutate(const MutationCommands & commands, const Co void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) { auto old_table_id = getStorageID(); + auto metadata_snapshot = getInMemoryMetadataPtr(); bool from_atomic_to_atomic_database = old_table_id.hasUUID() && new_table_id.hasUUID(); if (has_inner_table && tryGetTargetTable() && !from_atomic_to_atomic_database) @@ -293,14 +295,15 @@ void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) } IStorage::renameInMemory(new_table_id); - const auto & select_query = getSelectQuery(); + const auto & select_query = metadata_snapshot->getSelectQuery(); // TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated DatabaseCatalog::instance().updateDependency(select_query.select_table_id, old_table_id, select_query.select_table_id, getStorageID()); } void StorageMaterializedView::shutdown() { - const auto & select_query = getSelectQuery(); + auto metadata_snapshot = getInMemoryMetadataPtr(); + const auto & select_query = metadata_snapshot->getSelectQuery(); /// Make sure the dependency is removed after DETACH TABLE if (!select_query.select_table_id.empty()) DatabaseCatalog::instance().removeDependency(select_query.select_table_id, getStorageID()); diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 7e49580d6c2..006b1b3caec 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -63,7 +63,7 @@ Pipes StorageView::read( { Pipes pipes; - ASTPtr current_inner_query = getSelectQuery().inner_query; + ASTPtr current_inner_query = metadata_snapshot->getSelectQuery().inner_query; if (query_info.view_query) { diff --git a/src/Storages/StorageView.h b/src/Storages/StorageView.h index 143ed3c06c4..9de1f3f0bd8 100644 --- a/src/Storages/StorageView.h +++ b/src/Storages/StorageView.h @@ -30,9 +30,9 @@ public: size_t max_block_size, unsigned num_streams) override; - void replaceWithSubquery(ASTSelectQuery & select_query, ASTPtr & view_name) const + void replaceWithSubquery(ASTSelectQuery & select_query, ASTPtr & view_name, const StorageMetadataPtr & metadata_snapshot) const { - replaceWithSubquery(select_query, getSelectQuery().inner_query->clone(), view_name); + replaceWithSubquery(select_query, metadata_snapshot->getSelectQuery().inner_query->clone(), view_name); } static void replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_query, ASTPtr & view_name); From 33c27de54d535fdc3f0445db55638025644cfbad Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jun 2020 17:32:25 +0300 Subject: [PATCH 082/211] Check methods in metadata --- src/Storages/IStorage.cpp | 170 ----------------- src/Storages/IStorage.h | 15 -- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 2 +- .../ReplicatedMergeTreeBlockOutputStream.cpp | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageGenerateRandom.cpp | 4 +- src/Storages/StorageInMemoryMetadata.cpp | 172 ++++++++++++++++++ src/Storages/StorageInMemoryMetadata.h | 15 ++ src/Storages/StorageJoin.cpp | 2 +- src/Storages/StorageLog.cpp | 6 +- src/Storages/StorageMemory.cpp | 4 +- src/Storages/StorageMySQL.cpp | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- src/Storages/StorageTinyLog.cpp | 4 +- src/Storages/StorageValues.cpp | 4 +- src/Storages/StorageXDBC.cpp | 2 +- src/Storages/System/IStorageSystemOneBlock.h | 2 +- src/Storages/System/StorageSystemColumns.cpp | 2 +- src/Storages/System/StorageSystemDisks.cpp | 2 +- src/Storages/System/StorageSystemNumbers.cpp | 4 +- src/Storages/System/StorageSystemOne.cpp | 4 +- .../System/StorageSystemPartsBase.cpp | 6 +- src/Storages/System/StorageSystemPartsBase.h | 4 +- src/Storages/System/StorageSystemReplicas.cpp | 2 +- .../System/StorageSystemStoragePolicies.cpp | 2 +- src/Storages/System/StorageSystemTables.cpp | 2 +- src/Storages/System/StorageSystemZeros.cpp | 4 +- 28 files changed, 223 insertions(+), 221 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index a67229d6231..38fdaa832bd 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -37,176 +37,6 @@ const ColumnsDescription & IStorage::getColumns() const return metadata->columns; } -namespace -{ -#if !defined(ARCADIA_BUILD) - using NamesAndTypesMap = google::dense_hash_map; - using UniqueStrings = google::dense_hash_set; -#else - using NamesAndTypesMap = google::sparsehash::dense_hash_map; - using UniqueStrings = google::sparsehash::dense_hash_set; -#endif - - String listOfColumns(const NamesAndTypesList & available_columns) - { - std::stringstream ss; - for (auto it = available_columns.begin(); it != available_columns.end(); ++it) - { - if (it != available_columns.begin()) - ss << ", "; - ss << it->name; - } - return ss.str(); - } - - NamesAndTypesMap getColumnsMap(const NamesAndTypesList & columns) - { - NamesAndTypesMap res; - res.set_empty_key(StringRef()); - - for (const auto & column : columns) - res.insert({column.name, column.type.get()}); - - return res; - } - - UniqueStrings initUniqueStrings() - { - UniqueStrings strings; - strings.set_empty_key(StringRef()); - return strings; - } -} - -void IStorage::check(const Names & column_names, bool include_virtuals) const -{ - NamesAndTypesList available_columns = getColumns().getAllPhysical(); - if (include_virtuals) - { - auto virtuals = getVirtuals(); - available_columns.insert(available_columns.end(), virtuals.begin(), virtuals.end()); - } - - const String list_of_columns = listOfColumns(available_columns); - - if (column_names.empty()) - throw Exception("Empty list of columns queried. There are columns: " + list_of_columns, ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED); - - const auto columns_map = getColumnsMap(available_columns); - - auto unique_names = initUniqueStrings(); - for (const auto & name : column_names) - { - if (columns_map.end() == columns_map.find(name)) - throw Exception( - "There is no column with name " + backQuote(name) + " in table " + getStorageID().getNameForLogs() + ". There are columns: " + list_of_columns, - ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); - - if (unique_names.end() != unique_names.find(name)) - throw Exception("Column " + name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE); - unique_names.insert(name); - } -} - -void IStorage::check(const NamesAndTypesList & provided_columns) const -{ - const NamesAndTypesList & available_columns = getColumns().getAllPhysical(); - const auto columns_map = getColumnsMap(available_columns); - - auto unique_names = initUniqueStrings(); - for (const NameAndTypePair & column : provided_columns) - { - auto it = columns_map.find(column.name); - if (columns_map.end() == it) - throw Exception( - "There is no column with name " + column.name + ". There are columns: " + listOfColumns(available_columns), - ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); - - if (!column.type->equals(*it->second)) - throw Exception( - "Type mismatch for column " + column.name + ". Column has type " + it->second->getName() + ", got type " - + column.type->getName(), - ErrorCodes::TYPE_MISMATCH); - - if (unique_names.end() != unique_names.find(column.name)) - throw Exception("Column " + column.name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE); - unique_names.insert(column.name); - } -} - -void IStorage::check(const NamesAndTypesList & provided_columns, const Names & column_names) const -{ - const NamesAndTypesList & available_columns = getColumns().getAllPhysical(); - const auto available_columns_map = getColumnsMap(available_columns); - const auto & provided_columns_map = getColumnsMap(provided_columns); - - if (column_names.empty()) - throw Exception( - "Empty list of columns queried. There are columns: " + listOfColumns(available_columns), - ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED); - - auto unique_names = initUniqueStrings(); - for (const String & name : column_names) - { - auto it = provided_columns_map.find(name); - if (provided_columns_map.end() == it) - continue; - - auto jt = available_columns_map.find(name); - if (available_columns_map.end() == jt) - throw Exception( - "There is no column with name " + name + ". There are columns: " + listOfColumns(available_columns), - ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); - - if (!it->second->equals(*jt->second)) - throw Exception( - "Type mismatch for column " + name + ". Column has type " + jt->second->getName() + ", got type " + it->second->getName(), - ErrorCodes::TYPE_MISMATCH); - - if (unique_names.end() != unique_names.find(name)) - throw Exception("Column " + name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE); - unique_names.insert(name); - } -} - -void IStorage::check(const Block & block, bool need_all) const -{ - const NamesAndTypesList & available_columns = getColumns().getAllPhysical(); - const auto columns_map = getColumnsMap(available_columns); - - NameSet names_in_block; - - block.checkNumberOfRows(); - - for (const auto & column : block) - { - if (names_in_block.count(column.name)) - throw Exception("Duplicate column " + column.name + " in block", ErrorCodes::DUPLICATE_COLUMN); - - names_in_block.insert(column.name); - - auto it = columns_map.find(column.name); - if (columns_map.end() == it) - throw Exception( - "There is no column with name " + column.name + ". There are columns: " + listOfColumns(available_columns), - ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); - - if (!column.type->equals(*it->second)) - throw Exception( - "Type mismatch for column " + column.name + ". Column has type " + it->second->getName() + ", got type " - + column.type->getName(), - ErrorCodes::TYPE_MISMATCH); - } - - if (need_all && names_in_block.size() < columns_map.size()) - { - for (const auto & available_column : available_columns) - { - if (!names_in_block.count(available_column.name)) - throw Exception("Expected column " + available_column.name, ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); - } - } -} bool IStorage::isVirtualColumn(const String & column_name) const { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 4b712853b53..bb4bf2ed09b 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -138,25 +138,10 @@ public: public: /// thread-unsafe part. lockStructure must be acquired const ColumnsDescription & getColumns() const; /// returns combined set of columns - StorageInMemoryMetadata getInMemoryMetadata() const { return *metadata; } StorageMetadataPtr getInMemoryMetadataPtr() const { return metadata; } void setInMemoryMetadata(const StorageInMemoryMetadata & metadata_) { metadata = std::make_shared(metadata_); } - /// Verify that all the requested names are in the table and are set correctly: - /// list of names is not empty and the names do not repeat. - void check(const Names & column_names, bool include_virtuals = false) const; - - /// Check that all the requested names are in the table and have the correct types. - void check(const NamesAndTypesList & columns) const; - - /// Check that all names from the intersection of `names` and `columns` are in the table and have the same types. - void check(const NamesAndTypesList & columns, const Names & column_names) const; - - /// Check that the data block contains all the columns of the table with the correct types, - /// contains only the columns of the table, and all the columns are different. - /// If |need_all| is set, then checks that all the columns of the table are in the block. - void check(const Block & block, bool need_all = false) const; /// Return list of virtual columns (like _part, _table, etc). In the vast /// majority of cases virtual columns are static constant part of Storage diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 2cc5fc7dd5b..b72c46afca3 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -224,7 +224,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( std::multiset part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); - data.check(real_column_names); + metadata_snapshot->check(real_column_names, data.getVirtuals()); const Settings & settings = context.getSettingsRef(); const auto & primary_key = metadata_snapshot->getPrimaryKey(); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 5974f366b66..099480aca2f 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -138,7 +138,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block if (!block || !block.rows()) return result; - data.check(block, true); + metadata_snapshot->check(block, true); if (!metadata_snapshot->hasPartitionKey()) /// Table is not partitioned. { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 13df5ef23f1..bdefc5f1b14 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -211,7 +211,7 @@ void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::Muta void ReplicatedMergeTreeBlockOutputStream::commitPart( zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id) { - storage.check(part->getColumns()); + metadata_snapshot->check(part->getColumns()); assertSessionIsNotExpired(zookeeper); /// Obtain incremental block number and lock it. The lock holds our intention to add the block to the filesystem. diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 13b37980c56..2ce258a2d25 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -342,7 +342,7 @@ public: return; // Check table structure. - storage.check(block, true); + metadata_snapshot->check(block, true); size_t rows = block.rows(); if (!rows) diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index f1d97a4e5c4..bcebeec09dd 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -429,14 +429,14 @@ void registerStorageGenerateRandom(StorageFactory & factory) Pipes StorageGenerateRandom::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned num_streams) { - check(column_names, true); + metadata_snapshot->check(column_names, getVirtuals()); Pipes pipes; pipes.reserve(num_streams); diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 3b72dd97089..a394e196eac 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -1,6 +1,10 @@ #include +#include +#include #include +#include + namespace DB { @@ -410,4 +414,172 @@ bool StorageInMemoryMetadata::hasSelectQuery() const return select.select_query != nullptr; } +namespace +{ +#if !defined(ARCADIA_BUILD) + using NamesAndTypesMap = google::dense_hash_map; + using UniqueStrings = google::dense_hash_set; +#else + using NamesAndTypesMap = google::sparsehash::dense_hash_map; + using UniqueStrings = google::sparsehash::dense_hash_set; +#endif + + String listOfColumns(const NamesAndTypesList & available_columns) + { + std::stringstream ss; + for (auto it = available_columns.begin(); it != available_columns.end(); ++it) + { + if (it != available_columns.begin()) + ss << ", "; + ss << it->name; + } + return ss.str(); + } + + NamesAndTypesMap getColumnsMap(const NamesAndTypesList & columns) + { + NamesAndTypesMap res; + res.set_empty_key(StringRef()); + + for (const auto & column : columns) + res.insert({column.name, column.type.get()}); + + return res; + } + + UniqueStrings initUniqueStrings() + { + UniqueStrings strings; + strings.set_empty_key(StringRef()); + return strings; + } +} + +void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTypesList & virtuals) const +{ + NamesAndTypesList available_columns = getColumns().getAllPhysical(); + available_columns.insert(available_columns.end(), virtuals.begin(), virtuals.end()); + + const String list_of_columns = listOfColumns(available_columns); + + if (column_names.empty()) + throw Exception("Empty list of columns queried. There are columns: " + list_of_columns, ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED); + + const auto columns_map = getColumnsMap(available_columns); + + auto unique_names = initUniqueStrings(); + for (const auto & name : column_names) + { + if (columns_map.end() == columns_map.find(name)) + throw Exception( + "There is no column with name " + backQuote(name) + " in table " + /* TODO alesap getStorageID().getNameForLogs() +*/ ". There are columns: " + list_of_columns, + ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + + if (unique_names.end() != unique_names.find(name)) + throw Exception("Column " + name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE); + unique_names.insert(name); + } +} + +void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns) const +{ + const NamesAndTypesList & available_columns = getColumns().getAllPhysical(); + const auto columns_map = getColumnsMap(available_columns); + + auto unique_names = initUniqueStrings(); + for (const NameAndTypePair & column : provided_columns) + { + auto it = columns_map.find(column.name); + if (columns_map.end() == it) + throw Exception( + "There is no column with name " + column.name + ". There are columns: " + listOfColumns(available_columns), + ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + + if (!column.type->equals(*it->second)) + throw Exception( + "Type mismatch for column " + column.name + ". Column has type " + it->second->getName() + ", got type " + + column.type->getName(), + ErrorCodes::TYPE_MISMATCH); + + if (unique_names.end() != unique_names.find(column.name)) + throw Exception("Column " + column.name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE); + unique_names.insert(column.name); + } +} + +void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns, const Names & column_names) const +{ + const NamesAndTypesList & available_columns = getColumns().getAllPhysical(); + const auto available_columns_map = getColumnsMap(available_columns); + const auto & provided_columns_map = getColumnsMap(provided_columns); + + if (column_names.empty()) + throw Exception( + "Empty list of columns queried. There are columns: " + listOfColumns(available_columns), + ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED); + + auto unique_names = initUniqueStrings(); + for (const String & name : column_names) + { + auto it = provided_columns_map.find(name); + if (provided_columns_map.end() == it) + continue; + + auto jt = available_columns_map.find(name); + if (available_columns_map.end() == jt) + throw Exception( + "There is no column with name " + name + ". There are columns: " + listOfColumns(available_columns), + ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + + if (!it->second->equals(*jt->second)) + throw Exception( + "Type mismatch for column " + name + ". Column has type " + jt->second->getName() + ", got type " + it->second->getName(), + ErrorCodes::TYPE_MISMATCH); + + if (unique_names.end() != unique_names.find(name)) + throw Exception("Column " + name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE); + unique_names.insert(name); + } +} + +void StorageInMemoryMetadata::check(const Block & block, bool need_all) const +{ + const NamesAndTypesList & available_columns = getColumns().getAllPhysical(); + const auto columns_map = getColumnsMap(available_columns); + + NameSet names_in_block; + + block.checkNumberOfRows(); + + for (const auto & column : block) + { + if (names_in_block.count(column.name)) + throw Exception("Duplicate column " + column.name + " in block", ErrorCodes::DUPLICATE_COLUMN); + + names_in_block.insert(column.name); + + auto it = columns_map.find(column.name); + if (columns_map.end() == it) + throw Exception( + "There is no column with name " + column.name + ". There are columns: " + listOfColumns(available_columns), + ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + + if (!column.type->equals(*it->second)) + throw Exception( + "Type mismatch for column " + column.name + ". Column has type " + it->second->getName() + ", got type " + + column.type->getName(), + ErrorCodes::TYPE_MISMATCH); + } + + if (need_all && names_in_block.size() < columns_map.size()) + { + for (const auto & available_column : available_columns) + { + if (!names_in_block.count(available_column.name)) + throw Exception("Expected column " + available_column.name, ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); + } + } +} + + } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index efda4377dfc..e4755bb0464 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -177,6 +177,21 @@ struct StorageInMemoryMetadata /// Select query for *View storages. const SelectQueryDescription & getSelectQuery() const; bool hasSelectQuery() const; + + /// Verify that all the requested names are in the table and are set correctly: + /// list of names is not empty and the names do not repeat. + void check(const Names & column_names, const NamesAndTypesList & virtuals) const; + + /// Check that all the requested names are in the table and have the correct types. + void check(const NamesAndTypesList & columns) const; + + /// Check that all names from the intersection of `names` and `columns` are in the table and have the same types. + void check(const NamesAndTypesList & columns, const Names & column_names) const; + + /// Check that the data block contains all the columns of the table with the correct types, + /// contains only the columns of the table, and all the columns are different. + /// If |need_all| is set, then checks that all the columns of the table are in the block. + void check(const Block & block, bool need_all = false) const; }; using StorageMetadataPtr = std::shared_ptr; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 68b974c0dde..300ab400a46 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -446,7 +446,7 @@ Pipes StorageJoin::read( size_t max_block_size, unsigned /*num_streams*/) { - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); Pipes pipes; pipes.emplace_back(std::make_shared(*join, max_block_size, metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals()))); diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 79cc3e5bf68..fcae9c9aa82 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -276,7 +276,7 @@ void LogSource::readData(const String & name, const IDataType & type, IColumn & void LogBlockOutputStream::write(const Block & block) { - storage.check(block, true); + metadata_snapshot->check(block, true); /// The set of written offset columns so that you do not write shared offsets of columns for nested structures multiple times WrittenStreams written_streams; @@ -580,14 +580,14 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount() const Pipes StorageLog::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned num_streams) { - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); loadMarks(); NamesAndTypesList all_columns = Nested::collect(getColumns().getAllPhysical().addTypes(column_names)); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 442c5a3d67b..f9ef3cfcc98 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -81,7 +81,7 @@ public: void write(const Block & block) override { - storage.check(block, true); + metadata_snapshot->check(block, true); std::lock_guard lock(storage.mutex); storage.data.push_back(block); } @@ -110,7 +110,7 @@ Pipes StorageMemory::read( size_t /*max_block_size*/, unsigned num_streams) { - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); std::lock_guard lock(mutex); diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index b1262771d21..3e9b48e976b 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -72,7 +72,7 @@ Pipes StorageMySQL::read( size_t max_block_size_, unsigned) { - check(column_names_); + metadata_snapshot->check(column_names_, getVirtuals()); String query = transformQueryForExternalDatabase( query_info_, metadata_snapshot->getColumns().getOrdinary(), diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 4d9f08a60b7..407c9b164ff 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -276,7 +276,7 @@ Pipes StorageStripeLog::read( { std::shared_lock lock(rwlock); - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); NameSet column_names_set(column_names.begin(), column_names.end()); diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index ba524c7761e..4015d8ca574 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -309,7 +309,7 @@ void TinyLogBlockOutputStream::writeSuffix() void TinyLogBlockOutputStream::write(const Block & block) { - storage.check(block, true); + metadata_snapshot->check(block, true); /// The set of written offset columns so that you do not write shared columns for nested structures multiple times WrittenStreams written_streams; @@ -402,7 +402,7 @@ Pipes StorageTinyLog::read( const size_t max_block_size, const unsigned /*num_streams*/) { - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); Pipes pipes; diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index bb29b4a0932..063cd3d5224 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -23,14 +23,14 @@ StorageValues::StorageValues( Pipes StorageValues::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, unsigned /*num_streams*/) { - check(column_names, true); + metadata_snapshot->check(column_names, getVirtuals()); Pipes pipes; diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index c3c62ea1f0a..ab8b37db7db 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -90,7 +90,7 @@ Pipes StorageXDBC::read( size_t max_block_size, unsigned num_streams) { - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); bridge_helper->startBridgeSync(); return IStorageURLBase::read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index de7e1a0e933..b3a2a6fe53b 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -37,7 +37,7 @@ public: size_t /*max_block_size*/, unsigned /*num_streams*/) override { - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); Block sample_block = metadata_snapshot->getSampleBlock(); MutableColumns res_columns = sample_block.cloneEmptyColumns(); diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 83178870ba9..319ef257d6d 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -249,7 +249,7 @@ Pipes StorageSystemColumns::read( const size_t max_block_size, const unsigned /*num_streams*/) { - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); /// Create a mask of what columns are needed in the result. diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index d13ea29804d..fbcdd78988a 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -35,7 +35,7 @@ Pipes StorageSystemDisks::read( const size_t /*max_block_size*/, const unsigned /*num_streams*/) { - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); MutableColumnPtr col_name = ColumnString::create(); MutableColumnPtr col_path = ColumnString::create(); diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index fd7e04cfb1f..50921c53fb6 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -125,14 +125,14 @@ StorageSystemNumbers::StorageSystemNumbers(const StorageID & table_id, bool mult Pipes StorageSystemNumbers::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo &, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned num_streams) { - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); if (limit && *limit < max_block_size) { diff --git a/src/Storages/System/StorageSystemOne.cpp b/src/Storages/System/StorageSystemOne.cpp index af736c215b5..20d61d5da1b 100644 --- a/src/Storages/System/StorageSystemOne.cpp +++ b/src/Storages/System/StorageSystemOne.cpp @@ -22,14 +22,14 @@ StorageSystemOne::StorageSystemOne(const std::string & name_) Pipes StorageSystemOne::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo &, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, const unsigned /*num_streams*/) { - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); Block header{ColumnWithTypeAndName( DataTypeUInt8().createColumn(), diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 4f99e1e8c6a..b998b60c02d 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes extern const int TABLE_IS_DROPPED; } -bool StorageSystemPartsBase::hasStateColumn(const Names & column_names) const +bool StorageSystemPartsBase::hasStateColumn(const Names & column_names, const StorageMetadataPtr & metadata_snapshot) const { bool has_state_column = false; Names real_column_names; @@ -41,7 +41,7 @@ bool StorageSystemPartsBase::hasStateColumn(const Names & column_names) const /// Do not check if only _state column is requested if (!(has_state_column && real_column_names.empty())) - check(real_column_names); + metadata_snapshot->check(real_column_names, {}); return has_state_column; } @@ -232,7 +232,7 @@ Pipes StorageSystemPartsBase::read( const size_t /*max_block_size*/, const unsigned /*num_streams*/) { - bool has_state_column = hasStateColumn(column_names); + bool has_state_column = hasStateColumn(column_names, metadata_snapshot); StoragesInfoStream stream(query_info, context); diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index a46cecec9dd..8af1f46d8a7 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -57,7 +57,7 @@ class StorageSystemPartsBase : public IStorage public: Pipes read( const Names & column_names, - const StorageMetadataPtr & metadata_, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -67,7 +67,7 @@ public: NamesAndTypesList getVirtuals() const override; private: - bool hasStateColumn(const Names & column_names) const; + bool hasStateColumn(const Names & column_names, const StorageMetadataPtr & metadata_snapshot) const; protected: const FormatSettings format_settings; diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 8fb6a89ddd1..f79e9138500 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -66,7 +66,7 @@ Pipes StorageSystemReplicas::read( const size_t /*max_block_size*/, const unsigned /*num_streams*/) { - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); const auto access = context.getAccess(); const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); diff --git a/src/Storages/System/StorageSystemStoragePolicies.cpp b/src/Storages/System/StorageSystemStoragePolicies.cpp index 44252a788b9..a6092a28a47 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.cpp +++ b/src/Storages/System/StorageSystemStoragePolicies.cpp @@ -39,7 +39,7 @@ Pipes StorageSystemStoragePolicies::read( const size_t /*max_block_size*/, const unsigned /*num_streams*/) { - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); MutableColumnPtr col_policy_name = ColumnString::create(); MutableColumnPtr col_volume_name = ColumnString::create(); diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index f8f40026940..84635acb887 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -456,7 +456,7 @@ Pipes StorageSystemTables::read( const size_t max_block_size, const unsigned /*num_streams*/) { - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); /// Create a mask of what columns are needed in the result. diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index 2bc53b5093e..d325840091e 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -92,14 +92,14 @@ StorageSystemZeros::StorageSystemZeros(const StorageID & table_id_, bool multith Pipes StorageSystemZeros::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo &, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned num_streams) { - check(column_names); + metadata_snapshot->check(column_names, getVirtuals()); bool use_multiple_streams = multithreaded; From ef8781cce77dddff57f44aaa1005b0f88e30dcdf Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jun 2020 17:37:21 +0300 Subject: [PATCH 083/211] Better getVirtuals method --- src/Storages/IStorage.cpp | 4 ++-- src/Storages/IStorage.h | 2 +- src/Storages/StorageDistributed.cpp | 4 ++-- src/Storages/StorageMerge.cpp | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 38fdaa832bd..884982c93b9 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -38,10 +38,10 @@ const ColumnsDescription & IStorage::getColumns() const } -bool IStorage::isVirtualColumn(const String & column_name) const +bool IStorage::isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const { /// Virtual column maybe overriden by real column - return !getColumns().has(column_name) && getVirtuals().contains(column_name); + return !metadata_snapshot->getColumns().has(column_name) && getVirtuals().contains(column_name); } RWLockImpl::LockHolder IStorage::tryLockTimed( diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index bb4bf2ed09b..a125c6f8310 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -159,7 +159,7 @@ protected: /// Returns whether the column is virtual - by default all columns are real. /// Initially reserved virtual column name may be shadowed by real column. - bool isVirtualColumn(const String & column_name) const; + bool isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const; private: diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 6868f468f2e..ce4fcbb3513 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -464,7 +464,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Con Pipes StorageDistributed::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -497,7 +497,7 @@ Pipes StorageDistributed::read( const Scalars & scalars = context.hasQueryContext() ? context.getQueryContext().getScalars() : Scalars{}; bool has_virtual_shard_num_column = std::find(column_names.begin(), column_names.end(), "_shard_num") != column_names.end(); - if (has_virtual_shard_num_column && !isVirtualColumn("_shard_num")) + if (has_virtual_shard_num_column && !isVirtualColumn("_shard_num", metadata_snapshot)) has_virtual_shard_num_column = false; ClusterProxy::SelectStreamFactory select_stream_factory = remote_table_function_ptr diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 9765db35fc3..92e965c420e 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -146,7 +146,7 @@ Pipes StorageMerge::read( for (const auto & column_name : column_names) { - if (column_name == "_table" && isVirtualColumn(column_name)) + if (column_name == "_table" && isVirtualColumn(column_name, metadata_snapshot)) has_table_virtual_column = true; else real_column_names.push_back(column_name); From dffdece3501fce6ef74b1ae7c970f3f477024bb6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jun 2020 19:39:58 +0300 Subject: [PATCH 084/211] getColumns in StorageInMemoryMetadta (only compilable) --- src/Core/iostream_debug_helpers.cpp | 2 +- .../InputStreamFromASTInsertQuery.cpp | 9 +++++-- .../InputStreamFromASTInsertQuery.h | 5 +++- .../PushingToViewsBlockOutputStream.cpp | 5 ++-- src/DataStreams/RemoteQueryExecutor.cpp | 2 +- src/Databases/DatabaseMySQL.cpp | 3 ++- src/Functions/hasColumnInTable.cpp | 3 ++- src/Interpreters/InterpreterDescribeQuery.cpp | 3 ++- src/Interpreters/InterpreterInsertQuery.cpp | 4 +-- src/Interpreters/InterpreterSelectQuery.cpp | 13 +++++----- src/Interpreters/InterpreterWatchQuery.cpp | 2 +- src/Interpreters/JoinedTables.cpp | 4 +-- src/Interpreters/JoinedTables.h | 4 ++- src/Interpreters/MutationsInterpreter.cpp | 4 +-- src/Interpreters/SyntaxAnalyzer.cpp | 11 +++++--- src/Interpreters/SyntaxAnalyzer.h | 17 ++++++++++-- src/Interpreters/getTableExpressions.cpp | 6 +++-- src/Interpreters/interpretSubquery.cpp | 4 +-- src/Server/MySQLHandler.cpp | 3 ++- src/Server/TCPHandler.cpp | 10 ++++--- src/Storages/IStorage.cpp | 6 ----- src/Storages/IStorage.h | 1 - src/Storages/LiveView/StorageLiveView.cpp | 12 ++++++--- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 7 ++--- src/Storages/MergeTree/IMergeTreeDataPart.h | 3 ++- src/Storages/MergeTree/IMergeTreeReader.cpp | 26 +++++++++++++------ src/Storages/MergeTree/IMergeTreeReader.h | 5 +++- .../MergeTree/MergeTreeBlockReadUtils.cpp | 21 +++++++++------ .../MergeTree/MergeTreeBlockReadUtils.h | 11 +++++--- .../MergeTree/MergeTreeDataMergerMutator.cpp | 6 ++--- .../MergeTree/MergeTreeDataPartCompact.cpp | 3 ++- .../MergeTree/MergeTreeDataPartCompact.h | 1 + .../MergeTree/MergeTreeDataPartWide.cpp | 3 ++- .../MergeTree/MergeTreeDataPartWide.h | 1 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 25 +----------------- .../MergeTree/MergeTreeDataSelectExecutor.h | 6 ----- src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 +- .../MergeTree/MergeTreeReaderCompact.cpp | 25 ++++++++++++------ .../MergeTree/MergeTreeReaderCompact.h | 1 + .../MergeTree/MergeTreeReaderWide.cpp | 11 ++++++-- src/Storages/MergeTree/MergeTreeReaderWide.h | 1 + .../MergeTreeReverseSelectProcessor.cpp | 9 ++++--- .../MergeTree/MergeTreeSelectProcessor.cpp | 8 +++--- .../MergeTree/MergeTreeSequentialSource.cpp | 4 +-- ...rgeTreeThreadSelectBlockInputProcessor.cpp | 8 +++--- src/Storages/StorageBuffer.cpp | 10 +++---- src/Storages/StorageDistributed.cpp | 19 +++++++++----- src/Storages/StorageDistributed.h | 4 +-- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageGenerateRandom.cpp | 2 +- src/Storages/StorageLog.cpp | 17 ++++++------ src/Storages/StorageLog.h | 4 +-- src/Storages/StorageMerge.cpp | 10 ++++--- src/Storages/StorageReplicatedMergeTree.cpp | 8 +++--- src/Storages/StorageTinyLog.cpp | 21 ++++++++------- src/Storages/StorageURL.cpp | 14 +++++++--- src/Storages/StorageURL.h | 15 ++++++----- src/Storages/StorageXDBC.cpp | 12 ++++++--- src/Storages/StorageXDBC.h | 5 +++- .../System/StorageSystemPartsColumns.cpp | 2 +- src/Storages/TTLDescription.cpp | 4 +-- src/Storages/getStructureOfRemoteTable.cpp | 8 ++++-- src/Storages/tests/gtest_storage_log.cpp | 2 +- src/TableFunctions/TableFunctionMerge.cpp | 2 +- 64 files changed, 285 insertions(+), 196 deletions(-) diff --git a/src/Core/iostream_debug_helpers.cpp b/src/Core/iostream_debug_helpers.cpp index 3a77b1f42be..8683bb14db6 100644 --- a/src/Core/iostream_debug_helpers.cpp +++ b/src/Core/iostream_debug_helpers.cpp @@ -49,7 +49,7 @@ std::ostream & operator<<(std::ostream & stream, const IStorage & what) { auto table_id = what.getStorageID(); stream << "IStorage(name = " << what.getName() << ", tableName = " << table_id.table_name << ") {" - << what.getColumns().getAllPhysical().toString() << "}"; + << what.getInMemoryMetadataPtr()->getColumns().getAllPhysical().toString() << "}"; return stream; } diff --git a/src/DataStreams/InputStreamFromASTInsertQuery.cpp b/src/DataStreams/InputStreamFromASTInsertQuery.cpp index 47b61294da3..19c6fe41eca 100644 --- a/src/DataStreams/InputStreamFromASTInsertQuery.cpp +++ b/src/DataStreams/InputStreamFromASTInsertQuery.cpp @@ -21,7 +21,11 @@ namespace ErrorCodes InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery( - const ASTPtr & ast, ReadBuffer * input_buffer_tail_part, const Block & header, const Context & context, const ASTPtr & input_function) + const ASTPtr & ast, + ReadBuffer * input_buffer_tail_part, + const Block & header, + const Context & context, + const ASTPtr & input_function) { const auto * ast_insert_query = ast->as(); @@ -59,7 +63,8 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery( if (context.getSettingsRef().input_format_defaults_for_omitted_fields && ast_insert_query->table_id && !input_function) { StoragePtr storage = DatabaseCatalog::instance().getTable(ast_insert_query->table_id, context); - auto column_defaults = storage->getColumns().getDefaults(); + auto metadata_snapshot = storage->getInMemoryMetadataPtr(); + auto column_defaults = metadata_snapshot->getColumns().getDefaults(); if (!column_defaults.empty()) res_stream = std::make_shared(res_stream, column_defaults, context); } diff --git a/src/DataStreams/InputStreamFromASTInsertQuery.h b/src/DataStreams/InputStreamFromASTInsertQuery.h index a57e9199603..0604f011e28 100644 --- a/src/DataStreams/InputStreamFromASTInsertQuery.h +++ b/src/DataStreams/InputStreamFromASTInsertQuery.h @@ -11,6 +11,8 @@ namespace DB struct BlockIO; class Context; +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; /** Prepares an input stream which produce data containing in INSERT query * Head of inserting data could be stored in INSERT ast directly @@ -19,7 +21,8 @@ class Context; class InputStreamFromASTInsertQuery : public IBlockInputStream { public: - InputStreamFromASTInsertQuery(const ASTPtr & ast, + InputStreamFromASTInsertQuery( + const ASTPtr & ast, ReadBuffer * input_buffer_tail_part, const Block & header, const Context & context, diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index e6e368f78e9..72de6b889f1 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -79,6 +79,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( StoragePtr inner_table = materialized_view->getTargetTable(); auto inner_table_id = inner_table->getStorageID(); + auto inner_metadata_snapshot = inner_table->getInMemoryMetadataPtr(); query = dependent_metadata_snapshot->getSelectQuery().inner_query; std::unique_ptr insert = std::make_unique(); @@ -90,7 +91,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( /// Insert only columns returned by select. auto list = std::make_shared(); - const auto & inner_table_columns = inner_table->getColumns(); + const auto & inner_table_columns = inner_metadata_snapshot->getColumns(); for (auto & column : header) /// But skip columns which storage doesn't have. if (inner_table_columns.hasPhysical(column.name)) @@ -323,7 +324,7 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n Context local_context = *select_context; local_context.addViewSource( StorageValues::create( - storage->getStorageID(), storage->getColumns(), block, storage->getVirtuals())); + storage->getStorageID(), metadata_snapshot->getColumns(), block, storage->getVirtuals())); select.emplace(view.query, local_context, SelectQueryOptions()); in = std::make_shared(select->execute().getInputStream()); diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index 45ddd7c0893..be09cd94814 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -325,7 +325,7 @@ void RemoteQueryExecutor::sendExternalTables() Pipes pipes; pipes = cur->read( - cur->getColumns().getNamesOfPhysical(), + metadata_snapshot->getColumns().getNamesOfPhysical(), metadata_snapshot, {}, context, read_from_table_stage, DEFAULT_BLOCK_SIZE, 1); diff --git a/src/Databases/DatabaseMySQL.cpp b/src/Databases/DatabaseMySQL.cpp index 5d4b81014f9..a73fbafb7f5 100644 --- a/src/Databases/DatabaseMySQL.cpp +++ b/src/Databases/DatabaseMySQL.cpp @@ -139,7 +139,8 @@ static ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr create_table_query->table = table_id.table_name; create_table_query->database = table_id.database_name; - for (const auto & column_type_and_name : storage->getColumns().getOrdinary()) + auto metadata_snapshot = storage->getInMemoryMetadataPtr(); + for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary()) { const auto & column_declaration = std::make_shared(); column_declaration->name = column_type_and_name.name; diff --git a/src/Functions/hasColumnInTable.cpp b/src/Functions/hasColumnInTable.cpp index b9ec2b84837..ef447070e7a 100644 --- a/src/Functions/hasColumnInTable.cpp +++ b/src/Functions/hasColumnInTable.cpp @@ -114,7 +114,8 @@ void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers & if (host_name.empty()) { const StoragePtr & table = DatabaseCatalog::instance().getTable({database_name, table_name}, global_context); - has_column = table->getColumns().hasPhysical(column_name); + auto table_metadata = table->getInMemoryMetadataPtr(); + has_column = table_metadata->getColumns().hasPhysical(column_name); } else { diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index d457fefed6a..535a4280b45 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -91,7 +91,8 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() auto table_lock = table->lockStructureForShare( false, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); - columns = table->getColumns(); + auto metadata_snapshot = table->getInMemoryMetadataPtr(); + columns = metadata_snapshot->getColumns(); } Block sample_block = getSampleBlock(); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index e7fdf80e297..a39e8961970 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -244,7 +244,7 @@ BlockIO InterpreterInsertQuery::execute() /// Actually we don't know structure of input blocks from query/table, /// because some clients break insertion protocol (columns != header) out = std::make_shared( - out, query_sample_block, out->getHeader(), table->getColumns().getDefaults(), context); + out, query_sample_block, out->getHeader(), metadata_snapshot->getColumns().getDefaults(), context); /// It's important to squash blocks as early as possible (before other transforms), /// because other transforms may work inefficient if block size is small. @@ -295,7 +295,7 @@ BlockIO InterpreterInsertQuery::execute() if (!allow_materialized) { - for (const auto & column : table->getColumns()) + for (const auto & column : metadata_snapshot->getColumns()) if (column.default_desc.kind == ColumnDefaultKind::Materialized && header.has(column.name)) throw Exception("Cannot insert column " + column.name + ", because it is MATERIALIZED column.", ErrorCodes::ILLEGAL_COLUMN); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 509825e75e4..f601ca74112 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -129,7 +129,7 @@ String InterpreterSelectQuery::generateFilterActions( table_expr->children.push_back(table_expr->database_and_table_name); /// Using separate expression analyzer to prevent any possible alias injection - auto syntax_result = SyntaxAnalyzer(*context).analyzeSelect(query_ast, SyntaxAnalyzerResult({}, storage)); + auto syntax_result = SyntaxAnalyzer(*context).analyzeSelect(query_ast, SyntaxAnalyzerResult({}, storage, metadata_snapshot)); SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, *context, metadata_snapshot); actions = analyzer.simpleSelectActions(); @@ -263,7 +263,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( } if (has_input || !joined_tables.resolveTables()) - joined_tables.makeFakeTable(storage, source_header); + joined_tables.makeFakeTable(storage, metadata_snapshot, source_header); /// Rewrite JOINs if (!has_input && joined_tables.tablesCount() > 1) @@ -311,8 +311,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( view->replaceWithSubquery(getSelectQuery(), view_table, metadata_snapshot); syntax_analyzer_result = SyntaxAnalyzer(*context).analyzeSelect( - query_ptr, SyntaxAnalyzerResult(source_header.getNamesAndTypesList(), storage), - options, joined_tables.tablesWithColumns(), required_result_column_names, table_join); + query_ptr, + SyntaxAnalyzerResult(source_header.getNamesAndTypesList(), storage, metadata_snapshot), + options, joined_tables.tablesWithColumns(), required_result_column_names, table_join); if (view) { @@ -1087,7 +1088,7 @@ void InterpreterSelectQuery::executeFetchColumns( /// Detect, if ALIAS columns are required for query execution auto alias_columns_required = false; - const ColumnsDescription & storage_columns = storage->getColumns(); + const ColumnsDescription & storage_columns = metadata_snapshot->getColumns(); for (const auto & column_name : required_columns) { auto column_default = storage_columns.getDefault(column_name); @@ -1210,7 +1211,7 @@ void InterpreterSelectQuery::executeFetchColumns( prewhere_info->prewhere_actions = std::move(new_actions); auto analyzed_result - = SyntaxAnalyzer(*context).analyze(required_columns_from_prewhere_expr, storage->getColumns().getAllPhysical()); + = SyntaxAnalyzer(*context).analyze(required_columns_from_prewhere_expr, metadata_snapshot->getColumns().getAllPhysical()); prewhere_info->alias_actions = ExpressionAnalyzer(required_columns_from_prewhere_expr, analyzed_result, *context).getActions(true, false); diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index 489be488b4d..71ec1609046 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -47,7 +47,7 @@ BlockIO InterpreterWatchQuery::execute() ErrorCodes::UNKNOWN_TABLE); /// List of columns to read to execute the query. - Names required_columns = storage->getColumns().getNamesOfPhysical(); + Names required_columns = storage->getInMemoryMetadataPtr()->getColumns().getNamesOfPhysical(); context.checkAccess(AccessType::SELECT, table_id, required_columns); /// Get context settings for this query diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index 7450890952a..127df9b5eac 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -207,11 +207,11 @@ bool JoinedTables::resolveTables() return !tables_with_columns.empty(); } -void JoinedTables::makeFakeTable(StoragePtr storage, const Block & source_header) +void JoinedTables::makeFakeTable(StoragePtr storage, const StorageMetadataPtr & metadata_snapshot, const Block & source_header) { if (storage) { - const ColumnsDescription & storage_columns = storage->getColumns(); + const ColumnsDescription & storage_columns = metadata_snapshot->getColumns(); tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, storage_columns.getOrdinary()); auto & table = tables_with_columns.back(); diff --git a/src/Interpreters/JoinedTables.h b/src/Interpreters/JoinedTables.h index 2591b49527b..cff86c5a535 100644 --- a/src/Interpreters/JoinedTables.h +++ b/src/Interpreters/JoinedTables.h @@ -13,6 +13,8 @@ namespace DB class ASTSelectQuery; class TableJoin; struct SelectQueryOptions; +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; /// Joined tables' columns resolver. /// We want to get each table structure at most once per table occurance. Or even better once per table. @@ -31,7 +33,7 @@ public: bool resolveTables(); /// Make fake tables_with_columns[0] in case we have predefined input in InterpreterSelectQuery - void makeFakeTable(StoragePtr storage, const Block & source_header); + void makeFakeTable(StoragePtr storage, const StorageMetadataPtr & metadata_snapshot, const Block & source_header); std::shared_ptr makeTableJoin(const ASTSelectQuery & select_query); const TablesWithColumns & tablesWithColumns() const { return tables_with_columns; } diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 694e114af7a..3ad813a15b7 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -419,7 +419,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) else { NameSet new_updated_columns; - auto column_ttls = storage->getColumns().getColumnTTLs(); + auto column_ttls = metadata_snapshot->getColumns().getColumnTTLs(); for (const auto & elem : column_ttls) { dependencies.emplace(elem.first, ColumnDependency::TTL_TARGET); @@ -528,7 +528,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & prepared_stages, bool dry_run) { - NamesAndTypesList all_columns = storage->getColumns().getAllPhysical(); + NamesAndTypesList all_columns = metadata_snapshot->getColumns().getAllPhysical(); /// Next, for each stage calculate columns changed by this and previous stages. diff --git a/src/Interpreters/SyntaxAnalyzer.cpp b/src/Interpreters/SyntaxAnalyzer.cpp index 2dc2943d36d..9e927e1eb5a 100644 --- a/src/Interpreters/SyntaxAnalyzer.cpp +++ b/src/Interpreters/SyntaxAnalyzer.cpp @@ -681,7 +681,7 @@ void SyntaxAnalyzerResult::collectSourceColumns(bool add_special) { if (storage) { - const ColumnsDescription & columns = storage->getColumns(); + const ColumnsDescription & columns = metadata_snapshot->getColumns(); auto columns_from_storage = add_special ? columns.getAll() : columns.getAllPhysical(); if (source_columns.empty()) @@ -962,14 +962,19 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyzeSelect( return std::make_shared(result); } -SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(ASTPtr & query, const NamesAndTypesList & source_columns, ConstStoragePtr storage, bool allow_aggregations) const +SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( + ASTPtr & query, + const NamesAndTypesList & source_columns, + ConstStoragePtr storage, + const StorageMetadataPtr & metadata_snapshot, + bool allow_aggregations) const { if (query->as()) throw Exception("Not select analyze for select asts.", ErrorCodes::LOGICAL_ERROR); const auto & settings = context.getSettingsRef(); - SyntaxAnalyzerResult result(source_columns, storage, false); + SyntaxAnalyzerResult result(source_columns, storage, metadata_snapshot, false); normalize(query, result.aliases, settings); diff --git a/src/Interpreters/SyntaxAnalyzer.h b/src/Interpreters/SyntaxAnalyzer.h index 175c2db295a..4308b70c45a 100644 --- a/src/Interpreters/SyntaxAnalyzer.h +++ b/src/Interpreters/SyntaxAnalyzer.h @@ -16,10 +16,13 @@ class Context; struct Settings; struct SelectQueryOptions; using Scalars = std::map; +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; struct SyntaxAnalyzerResult { ConstStoragePtr storage; + StorageMetadataPtr metadata_snapshot; std::shared_ptr analyzed_join; NamesAndTypesList source_columns; @@ -51,8 +54,13 @@ struct SyntaxAnalyzerResult /// Results of scalar sub queries Scalars scalars; - SyntaxAnalyzerResult(const NamesAndTypesList & source_columns_, ConstStoragePtr storage_ = {}, bool add_special = true) + SyntaxAnalyzerResult( + const NamesAndTypesList & source_columns_, + ConstStoragePtr storage_ = {}, + const StorageMetadataPtr & metadata_snapshot_ = {}, + bool add_special = true) : storage(storage_) + , metadata_snapshot(metadata_snapshot_) , source_columns(source_columns_) { collectSourceColumns(add_special); @@ -86,7 +94,12 @@ public: {} /// Analyze and rewrite not select query - SyntaxAnalyzerResultPtr analyze(ASTPtr & query, const NamesAndTypesList & source_columns_, ConstStoragePtr storage = {}, bool allow_aggregations = false) const; + SyntaxAnalyzerResultPtr analyze( + ASTPtr & query, + const NamesAndTypesList & source_columns_, + ConstStoragePtr storage = {}, + const StorageMetadataPtr & metadata_snapshot = {}, + bool allow_aggregations = false) const; /// Analyze and rewrite select query SyntaxAnalyzerResultPtr analyzeSelect( diff --git a/src/Interpreters/getTableExpressions.cpp b/src/Interpreters/getTableExpressions.cpp index 6e3fd516e1c..56ca614dc2d 100644 --- a/src/Interpreters/getTableExpressions.cpp +++ b/src/Interpreters/getTableExpressions.cpp @@ -87,7 +87,8 @@ static NamesAndTypesList getColumnsFromTableExpression(const ASTTableExpression const auto table_function = table_expression.table_function; auto * query_context = const_cast(&context.getQueryContext()); const auto & function_storage = query_context->executeTableFunction(table_function); - const auto & columns = function_storage->getColumns(); + auto function_metadata_snapshot = function_storage->getInMemoryMetadataPtr(); + const auto & columns = function_metadata_snapshot->getColumns(); names_and_type_list = columns.getOrdinary(); materialized = columns.getMaterialized(); aliases = columns.getAliases(); @@ -97,7 +98,8 @@ static NamesAndTypesList getColumnsFromTableExpression(const ASTTableExpression { auto table_id = context.resolveStorageID(table_expression.database_and_table_name); const auto & table = DatabaseCatalog::instance().getTable(table_id, context); - const auto & columns = table->getColumns(); + auto table_metadata_snapshot = table->getInMemoryMetadataPtr(); + const auto & columns = table_metadata_snapshot->getColumns(); names_and_type_list = columns.getOrdinary(); materialized = columns.getMaterialized(); aliases = columns.getAliases(); diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index c94759897f5..cf343a4fda2 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -90,14 +90,14 @@ std::shared_ptr interpretSubquery( { auto * query_context = const_cast(&context.getQueryContext()); const auto & storage = query_context->executeTableFunction(table_expression); - columns = storage->getColumns().getOrdinary(); + columns = storage->getInMemoryMetadataPtr()->getColumns().getOrdinary(); select_query->addTableFunction(*const_cast(&table_expression)); // XXX: const_cast should be avoided! } else { auto table_id = context.resolveStorageID(table_expression); const auto & storage = DatabaseCatalog::instance().getTable(table_id, context); - columns = storage->getColumns().getOrdinary(); + columns = storage->getInMemoryMetadataPtr()->getColumns().getOrdinary(); select_query->replaceDatabaseAndTable(table_id); } diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 51b3d7eaef5..68f1bb8efff 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -254,7 +254,8 @@ void MySQLHandler::comFieldList(ReadBuffer & payload) packet.readPayload(payload); String database = connection_context.getCurrentDatabase(); StoragePtr table_ptr = DatabaseCatalog::instance().getTable({database, packet.table}, connection_context); - for (const NameAndTypePair & column: table_ptr->getColumns().getAll()) + auto metadata_snapshot = table_ptr->getInMemoryMetadataPtr(); + for (const NameAndTypePair & column : metadata_snapshot->getColumns().getAll()) { ColumnDefinition column_definition( database, packet.table, packet.table, column.name, column.name, CharacterSet::binary, 100, ColumnType::MYSQL_TYPE_STRING, 0, 0 diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 009f7ad80f0..056234af45d 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -213,17 +213,18 @@ void TCPHandler::runImpl() if (&context != &query_context.value()) throw Exception("Unexpected context in Input initializer", ErrorCodes::LOGICAL_ERROR); + auto metadata_snapshot = input_storage->getInMemoryMetadataPtr(); state.need_receive_data_for_input = true; /// Send ColumnsDescription for input storage. if (client_revision >= DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA && query_context->getSettingsRef().input_format_defaults_for_omitted_fields) { - sendTableColumns(input_storage->getColumns()); + sendTableColumns(metadata_snapshot->getColumns()); } /// Send block to the client - input storage structure. - state.input_header = input_storage->getInMemoryMetadataPtr()->getSampleBlock(); + state.input_header = metadata_snapshot->getSampleBlock(); sendData(state.input_header); }); @@ -474,7 +475,10 @@ void TCPHandler::processInsertQuery(const Settings & connection_settings) if (query_context->getSettingsRef().input_format_defaults_for_omitted_fields) { if (!table_id.empty()) - sendTableColumns(DatabaseCatalog::instance().getTable(table_id, *query_context)->getColumns()); + { + auto storage_ptr = DatabaseCatalog::instance().getTable(table_id, *query_context); + sendTableColumns(storage_ptr->getInMemoryMetadataPtr()->getColumns()); + } } } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 884982c93b9..8ee9561466a 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -32,12 +32,6 @@ namespace ErrorCodes extern const int DEADLOCK_AVOIDED; } -const ColumnsDescription & IStorage::getColumns() const -{ - return metadata->columns; -} - - bool IStorage::isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const { /// Virtual column maybe overriden by real column diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index a125c6f8310..ba1945d5c79 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -137,7 +137,6 @@ public: public: /// thread-unsafe part. lockStructure must be acquired - const ColumnsDescription & getColumns() const; /// returns combined set of columns StorageInMemoryMetadata getInMemoryMetadata() const { return *metadata; } StorageMetadataPtr getInMemoryMetadataPtr() const { return metadata; } void setInMemoryMetadata(const StorageInMemoryMetadata & metadata_) { metadata = std::make_shared(metadata_); } diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index f1b9459b3d3..ac6bd48f534 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -142,8 +142,10 @@ BlockInputStreamPtr StorageLiveView::completeQuery(Pipes pipes) auto creator = [&](const StorageID & blocks_id_global) { - return StorageBlocks::createStorage(blocks_id_global, getParentStorage()->getColumns(), - std::move(pipes), QueryProcessingStage::WithMergeableState); + auto parent_table_metadata = getParentStorage()->getInMemoryMetadataPtr(); + return StorageBlocks::createStorage( + blocks_id_global, parent_table_metadata->getColumns(), + std::move(pipes), QueryProcessingStage::WithMergeableState); }; block_context->addExternalTable(getBlocksTableName(), TemporaryTableHolder(global_context, creator)); @@ -209,8 +211,10 @@ void StorageLiveView::writeIntoLiveView( auto creator = [&](const StorageID & blocks_id_global) { - return StorageBlocks::createStorage(blocks_id_global, live_view.getParentStorage()->getColumns(), - std::move(pipes), QueryProcessingStage::FetchColumns); + auto parent_metadata = live_view.getParentStorage()->getInMemoryMetadataPtr(); + return StorageBlocks::createStorage( + blocks_id_global, parent_metadata->getColumns(), + std::move(pipes), QueryProcessingStage::FetchColumns); }; TemporaryTableHolder blocks_storage(context, creator); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 17ff2259436..61dfeed6b7c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -352,9 +352,9 @@ size_t IMergeTreeDataPart::getFileSizeOrZero(const String & file_name) const return checksum->second.file_size; } -String IMergeTreeDataPart::getColumnNameWithMinumumCompressedSize() const +String IMergeTreeDataPart::getColumnNameWithMinumumCompressedSize(const StorageMetadataPtr & metadata_snapshot) const { - const auto & storage_columns = storage.getColumns().getAllPhysical(); + const auto & storage_columns = metadata_snapshot->getColumns().getAllPhysical(); auto alter_conversions = storage.getAlterConversionsForPart(shared_from_this()); std::optional minimum_size_column; @@ -613,6 +613,7 @@ void IMergeTreeDataPart::loadTTLInfos() void IMergeTreeDataPart::loadColumns(bool require) { String path = getFullRelativePath() + "columns.txt"; + auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (!volume->getDisk()->exists(path)) { /// We can get list of columns only from columns.txt in compact parts. @@ -620,7 +621,7 @@ void IMergeTreeDataPart::loadColumns(bool require) throw Exception("No columns.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); /// If there is no file with a list of columns, write it down. - for (const NameAndTypePair & column : storage.getColumns().getAllPhysical()) + for (const NameAndTypePair & column : metadata_snapshot->getColumns().getAllPhysical()) if (volume->getDisk()->exists(getFullRelativePath() + getFileNameForColumn(column) + ".bin")) columns.push_back(column); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 04babece83e..0e73b1370c5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -77,6 +77,7 @@ public: virtual MergeTreeReaderPtr getReader( const NamesAndTypesList & columns_, + const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, @@ -143,7 +144,7 @@ public: /// Returns the name of a column with minimum compressed size (as returned by getColumnSize()). /// If no checksums are present returns the name of the first physically existing column. - String getColumnNameWithMinumumCompressedSize() const; + String getColumnNameWithMinumumCompressedSize(const StorageMetadataPtr & metadata_snapshot) const; bool contains(const IMergeTreeDataPart & other) const { return info.contains(other.info); } diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 5d2a5ac3616..53dc47d8068 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -22,13 +22,23 @@ namespace ErrorCodes } -IMergeTreeReader::IMergeTreeReader(const MergeTreeData::DataPartPtr & data_part_, - const NamesAndTypesList & columns_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, - const MarkRanges & all_mark_ranges_, const MergeTreeReaderSettings & settings_, +IMergeTreeReader::IMergeTreeReader( + const MergeTreeData::DataPartPtr & data_part_, + const NamesAndTypesList & columns_, + const StorageMetadataPtr & metadata_snapshot_, + UncompressedCache * uncompressed_cache_, + MarkCache * mark_cache_, + const MarkRanges & all_mark_ranges_, + const MergeTreeReaderSettings & settings_, const ValueSizeMap & avg_value_size_hints_) - : data_part(data_part_), avg_value_size_hints(avg_value_size_hints_) - , columns(columns_), uncompressed_cache(uncompressed_cache_), mark_cache(mark_cache_) - , settings(settings_), storage(data_part_->storage) + : data_part(data_part_) + , avg_value_size_hints(avg_value_size_hints_) + , columns(columns_) + , uncompressed_cache(uncompressed_cache_) + , mark_cache(mark_cache_) + , settings(settings_) + , storage(data_part_->storage) + , metadata_snapshot(metadata_snapshot_) , all_mark_ranges(all_mark_ranges_) , alter_conversions(storage.getAlterConversionsForPart(data_part)) { @@ -112,7 +122,7 @@ void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_e if (res_columns[i] == nullptr) { - if (storage.getColumns().hasDefault(name)) + if (metadata_snapshot->getColumns().hasDefault(name)) { should_evaluate_missing_defaults = true; continue; @@ -170,7 +180,7 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns additional_columns.insert({res_columns[pos], name_and_type->type, name_and_type->name}); } - DB::evaluateMissingDefaults(additional_columns, columns, storage.getColumns().getDefaults(), storage.global_context); + DB::evaluateMissingDefaults(additional_columns, columns, metadata_snapshot->getColumns().getDefaults(), storage.global_context); /// Move columns from block. name_and_type = columns.begin(); diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 02d8f67f9d0..6e9922b29ed 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -18,8 +18,10 @@ public: using ValueSizeMap = std::map; using DeserializeBinaryBulkStateMap = std::map; - IMergeTreeReader(const MergeTreeData::DataPartPtr & data_part_, + IMergeTreeReader( + const MergeTreeData::DataPartPtr & data_part_, const NamesAndTypesList & columns_, + const StorageMetadataPtr & metadata_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, const MarkRanges & all_mark_ranges_, @@ -75,6 +77,7 @@ protected: MergeTreeReaderSettings settings; const MergeTreeData & storage; + StorageMetadataPtr metadata_snapshot; MarkRanges all_mark_ranges; friend class MergeTreeRangeReader::DelayedStream; diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 310c566fb19..03235742a68 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -13,14 +13,14 @@ namespace ErrorCodes } -NameSet injectRequiredColumns(const MergeTreeData & storage, const MergeTreeData::DataPartPtr & part, Names & columns) +NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData::DataPartPtr & part, Names & columns) { NameSet required_columns{std::begin(columns), std::end(columns)}; NameSet injected_columns; auto all_column_files_missing = true; - const auto & storage_columns = storage.getColumns(); + const auto & storage_columns = metadata_snapshot->getColumns(); auto alter_conversions = storage.getAlterConversionsForPart(part); for (size_t i = 0; i < columns.size(); ++i) { @@ -66,7 +66,7 @@ NameSet injectRequiredColumns(const MergeTreeData & storage, const MergeTreeData */ if (all_column_files_missing) { - const auto minimum_size_column_name = part->getColumnNameWithMinumumCompressedSize(); + const auto minimum_size_column_name = part->getColumnNameWithMinumumCompressedSize(metadata_snapshot); columns.push_back(minimum_size_column_name); /// correctly report added column injected_columns.insert(columns.back()); @@ -214,14 +214,19 @@ void MergeTreeBlockSizePredictor::update(const Block & sample_block, const Colum } -MergeTreeReadTaskColumns getReadTaskColumns(const MergeTreeData & storage, const MergeTreeData::DataPartPtr & data_part, - const Names & required_columns, const PrewhereInfoPtr & prewhere_info, bool check_columns) +MergeTreeReadTaskColumns getReadTaskColumns( + const MergeTreeData & storage, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreeData::DataPartPtr & data_part, + const Names & required_columns, + const PrewhereInfoPtr & prewhere_info, + bool check_columns) { Names column_names = required_columns; Names pre_column_names; /// inject columns required for defaults evaluation - bool should_reorder = !injectRequiredColumns(storage, data_part, column_names).empty(); + bool should_reorder = !injectRequiredColumns(storage, metadata_snapshot, data_part, column_names).empty(); if (prewhere_info) { @@ -233,7 +238,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(const MergeTreeData & storage, const if (pre_column_names.empty()) pre_column_names.push_back(column_names[0]); - const auto injected_pre_columns = injectRequiredColumns(storage, data_part, pre_column_names); + const auto injected_pre_columns = injectRequiredColumns(storage, metadata_snapshot, data_part, pre_column_names); if (!injected_pre_columns.empty()) should_reorder = true; @@ -251,7 +256,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(const MergeTreeData & storage, const if (check_columns) { - const NamesAndTypesList & physical_columns = storage.getColumns().getAllPhysical(); + const NamesAndTypesList & physical_columns = metadata_snapshot->getColumns().getAllPhysical(); result.pre_columns = physical_columns.addTypes(pre_column_names); result.columns = physical_columns.addTypes(column_names); } diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 108742e1101..31d609e4242 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -22,7 +22,7 @@ using MergeTreeBlockSizePredictorPtr = std::unique_ptrgetColumns().getNamesOfPhysical(); + NamesAndTypesList storage_columns = metadata_snapshot->getColumns().getAllPhysical(); const auto data_settings = data.getSettings(); NamesAndTypesList gathering_columns; @@ -1041,7 +1041,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor UInt64 watch_prev_elapsed = 0; MergeStageProgress stage_progress(1.0); - NamesAndTypesList storage_columns = data.getColumns().getAllPhysical(); + NamesAndTypesList storage_columns = metadata_snapshot->getColumns().getAllPhysical(); if (!for_interpreter.empty()) { diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index d45aa882b2a..65577eb4ca1 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -38,6 +38,7 @@ MergeTreeDataPartCompact::MergeTreeDataPartCompact( IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( const NamesAndTypesList & columns_to_read, + const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, @@ -47,7 +48,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( { auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - ptr, columns_to_read, uncompressed_cache, + ptr, columns_to_read, metadata_snapshot, uncompressed_cache, mark_cache, mark_ranges, reader_settings, avg_value_size_hints, profile_callback); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 0b27dd53339..7d5c8628570 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -37,6 +37,7 @@ public: MergeTreeReaderPtr getReader( const NamesAndTypesList & columns, + const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index e4901b1f74d..e9383bc917d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -37,6 +37,7 @@ MergeTreeDataPartWide::MergeTreeDataPartWide( IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( const NamesAndTypesList & columns_to_read, + const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, @@ -46,7 +47,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( { auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - ptr, columns_to_read, uncompressed_cache, + ptr, columns_to_read, metadata_snapshot, uncompressed_cache, mark_cache, mark_ranges, reader_settings, avg_value_size_hints, profile_callback); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 144dfa86cfb..8d8b6fa678b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -30,6 +30,7 @@ public: MergeTreeReaderPtr getReader( const NamesAndTypesList & columns, + const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index b72c46afca3..7f7fd203297 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -650,7 +650,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( auto order_key_prefix_ast = metadata_snapshot->getSortingKey().expression_list_ast->clone(); order_key_prefix_ast->children.resize(prefix_size); - auto syntax_result = SyntaxAnalyzer(context).analyze(order_key_prefix_ast, data.getColumns().getAllPhysical()); + auto syntax_result = SyntaxAnalyzer(context).analyze(order_key_prefix_ast, metadata_snapshot->getColumns().getAllPhysical()); auto sorting_key_prefix_expr = ExpressionAnalyzer(order_key_prefix_ast, syntax_result, context).getActions(false); res = spreadMarkRangesAmongStreamsWithOrder( @@ -1274,29 +1274,6 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( return pipes; } - -void MergeTreeDataSelectExecutor::createPositiveSignCondition( - ExpressionActionsPtr & out_expression, String & out_column, const Context & context) const -{ - auto function = std::make_shared(); - auto arguments = std::make_shared(); - auto sign = std::make_shared(data.merging_params.sign_column); - auto one = std::make_shared(1); - - function->name = "equals"; - function->arguments = arguments; - function->children.push_back(arguments); - - arguments->children.push_back(sign); - arguments->children.push_back(one); - - ASTPtr query = function; - auto syntax_result = SyntaxAnalyzer(context).analyze(query, data.getColumns().getAllPhysical()); - out_expression = ExpressionAnalyzer(query, syntax_result, context).getActions(false); - out_column = function->getColumnName(); -} - - /// Calculates a set of mark ranges, that could possibly contain keys, required by condition. /// In other words, it removes subranges from whole range, that definitely could not contain required keys. MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index ba0613a832d..5669e8708b6 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -95,12 +95,6 @@ private: const KeyCondition & key_condition, const Settings & settings) const; - /// Create the expression "Sign == 1". - void createPositiveSignCondition( - ExpressionActionsPtr & out_expression, - String & out_column, - const Context & context) const; - MarkRanges markRangesFromPKRange( const MergeTreeData::DataPartPtr & part, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 8c73dc39dfb..fdf3908d21e 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -211,7 +211,7 @@ std::vector MergeTreeReadPool::fillPerPartInfo( per_part_sum_marks.push_back(sum_marks); auto [required_columns, required_pre_columns, should_reorder] = - getReadTaskColumns(data, part.data_part, column_names, prewhere_info, check_columns); + getReadTaskColumns(data, metadata_snapshot, part.data_part, column_names, prewhere_info, check_columns); /// will be used to distinguish between PREWHERE and WHERE columns when applying filter const auto & required_column_names = required_columns.getNames(); diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index c4a05a8bfac..64e9deec744 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -17,6 +17,7 @@ namespace ErrorCodes MergeTreeReaderCompact::MergeTreeReaderCompact( DataPartCompactPtr data_part_, NamesAndTypesList columns_, + const StorageMetadataPtr & metadata_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, MarkRanges mark_ranges_, @@ -24,15 +25,23 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( ValueSizeMap avg_value_size_hints_, const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_) - : IMergeTreeReader(std::move(data_part_), std::move(columns_), - uncompressed_cache_, mark_cache_, std::move(mark_ranges_), - std::move(settings_), std::move(avg_value_size_hints_)) + : IMergeTreeReader( + std::move(data_part_), + std::move(columns_), + metadata_snapshot_, + uncompressed_cache_, + mark_cache_, + std::move(mark_ranges_), + std::move(settings_), + std::move(avg_value_size_hints_)) , marks_loader( - data_part->volume->getDisk(), - mark_cache, - data_part->index_granularity_info.getMarksFilePath(data_part->getFullRelativePath() + MergeTreeDataPartCompact::DATA_FILE_NAME), - data_part->getMarksCount(), data_part->index_granularity_info, - settings.save_marks_in_cache, data_part->getColumns().size()) + data_part->volume->getDisk(), + mark_cache, + data_part->index_granularity_info.getMarksFilePath(data_part->getFullRelativePath() + MergeTreeDataPartCompact::DATA_FILE_NAME), + data_part->getMarksCount(), + data_part->index_granularity_info, + settings.save_marks_in_cache, + data_part->getColumns().size()) { size_t buffer_size = settings.max_read_buffer_size; const String full_data_path = data_part->getFullRelativePath() + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index 827306cd983..584d8ed2ff0 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -17,6 +17,7 @@ public: MergeTreeReaderCompact( DataPartCompactPtr data_part_, NamesAndTypesList columns_, + const StorageMetadataPtr & metadata_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, MarkRanges mark_ranges_, diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 34bf095e57e..2326c012fee 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -28,6 +28,7 @@ namespace ErrorCodes MergeTreeReaderWide::MergeTreeReaderWide( DataPartWidePtr data_part_, NamesAndTypesList columns_, + const StorageMetadataPtr & metadata_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, MarkRanges mark_ranges_, @@ -36,8 +37,14 @@ MergeTreeReaderWide::MergeTreeReaderWide( const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_) : IMergeTreeReader( - std::move(data_part_), std::move(columns_), uncompressed_cache_, std::move(mark_cache_), - std::move(mark_ranges_), std::move(settings_), std::move(avg_value_size_hints_)) + std::move(data_part_), + std::move(columns_), + metadata_snapshot_, + uncompressed_cache_, + std::move(mark_cache_), + std::move(mark_ranges_), + std::move(settings_), + std::move(avg_value_size_hints_)) { try { diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index 7684d69f0a5..69652d1e954 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -17,6 +17,7 @@ public: MergeTreeReaderWide( DataPartWidePtr data_part_, NamesAndTypesList columns_, + const StorageMetadataPtr & metadata_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, MarkRanges mark_ranges_, diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp index c47dd7fb669..b71c343614b 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp @@ -76,7 +76,7 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor( ordered_names = header_without_virtual_columns.getNames(); - task_columns = getReadTaskColumns(storage, data_part, required_columns, prewhere_info, check_columns); + task_columns = getReadTaskColumns(storage, metadata_snapshot, data_part, required_columns, prewhere_info, check_columns); /// will be used to distinguish between PREWHERE and WHERE columns when applying filter const auto & column_names = task_columns.columns.getNames(); @@ -87,11 +87,12 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor( owned_mark_cache = storage.global_context.getMarkCache(); - reader = data_part->getReader(task_columns.columns, all_mark_ranges, - owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings); + reader = data_part->getReader(task_columns.columns, metadata_snapshot, + all_mark_ranges, owned_uncompressed_cache.get(), + owned_mark_cache.get(), reader_settings); if (prewhere_info) - pre_reader = data_part->getReader(task_columns.pre_columns, all_mark_ranges, + pre_reader = data_part->getReader(task_columns.pre_columns, metadata_snapshot, all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings); } diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 84c0f44c109..b46b414bfe8 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -68,7 +68,9 @@ try } is_first_task = false; - task_columns = getReadTaskColumns(storage, data_part, required_columns, prewhere_info, check_columns); + task_columns = getReadTaskColumns( + storage, metadata_snapshot, data_part, + required_columns, prewhere_info, check_columns); auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr @@ -90,11 +92,11 @@ try owned_mark_cache = storage.global_context.getMarkCache(); - reader = data_part->getReader(task_columns.columns, all_mark_ranges, + reader = data_part->getReader(task_columns.columns, metadata_snapshot, all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings); if (prewhere_info) - pre_reader = data_part->getReader(task_columns.pre_columns, all_mark_ranges, + pre_reader = data_part->getReader(task_columns.pre_columns, metadata_snapshot, all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings); } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index dfd60bd50ef..f8e31db2b5a 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -39,7 +39,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( addTotalRowsApprox(data_part->rows_count); /// Add columns because we don't want to read empty blocks - injectRequiredColumns(storage, data_part, columns_to_read); + injectRequiredColumns(storage, metadata_snapshot, data_part, columns_to_read); NamesAndTypesList columns_for_reader; if (take_column_types_from_storage) { @@ -60,7 +60,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( .save_marks_in_cache = false }; - reader = data_part->getReader(columns_for_reader, + reader = data_part->getReader(columns_for_reader, metadata_snapshot, MarkRanges{MarkRange(0, data_part->getMarksCount())}, /* uncompressed_cache = */ nullptr, mark_cache.get(), reader_settings); } diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp index 784c842d7d6..c332685799c 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp @@ -74,12 +74,12 @@ bool MergeTreeThreadSelectBlockInputProcessor::getNewTask() owned_uncompressed_cache = storage.global_context.getUncompressedCache(); owned_mark_cache = storage.global_context.getMarkCache(); - reader = task->data_part->getReader(task->columns, rest_mark_ranges, + reader = task->data_part->getReader(task->columns, metadata_snapshot, rest_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, IMergeTreeReader::ValueSizeMap{}, profile_callback); if (prewhere_info) - pre_reader = task->data_part->getReader(task->pre_columns, rest_mark_ranges, + pre_reader = task->data_part->getReader(task->pre_columns, metadata_snapshot, rest_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, IMergeTreeReader::ValueSizeMap{}, profile_callback); } @@ -90,12 +90,12 @@ bool MergeTreeThreadSelectBlockInputProcessor::getNewTask() { auto rest_mark_ranges = pool->getRestMarks(*task->data_part, task->mark_ranges[0]); /// retain avg_value_size_hints - reader = task->data_part->getReader(task->columns, rest_mark_ranges, + reader = task->data_part->getReader(task->columns, metadata_snapshot, rest_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, reader->getAvgValueSizeHints(), profile_callback); if (prewhere_info) - pre_reader = task->data_part->getReader(task->pre_columns, rest_mark_ranges, + pre_reader = task->data_part->getReader(task->pre_columns, metadata_snapshot, rest_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, reader->getAvgValueSizeHints(), profile_callback); } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 2ce258a2d25..4e659a8e7b1 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -168,9 +168,9 @@ Pipes StorageBuffer::read( auto destination_metadata_snapshot = destination->getInMemoryMetadataPtr(); - const bool dst_has_same_structure = std::all_of(column_names.begin(), column_names.end(), [metadata_snapshot, destination](const String& column_name) + const bool dst_has_same_structure = std::all_of(column_names.begin(), column_names.end(), [metadata_snapshot, destination_metadata_snapshot](const String& column_name) { - const auto & dest_columns = destination->getColumns(); + const auto & dest_columns = destination_metadata_snapshot->getColumns(); const auto & our_columns = metadata_snapshot->getColumns(); return dest_columns.hasPhysical(column_name) && dest_columns.get(column_name).type->equals(*our_columns.get(column_name).type); @@ -192,8 +192,8 @@ Pipes StorageBuffer::read( const Block header = metadata_snapshot->getSampleBlock(); Names columns_intersection = column_names; Block header_after_adding_defaults = header; - const auto & dest_columns = destination->getColumns(); - const auto & our_columns = getColumns(); + const auto & dest_columns = destination_metadata_snapshot->getColumns(); + const auto & our_columns = metadata_snapshot->getColumns(); for (const String & column_name : column_names) { if (!dest_columns.hasPhysical(column_name)) @@ -224,7 +224,7 @@ Pipes StorageBuffer::read( for (auto & pipe : pipes_from_dst) { pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), header_after_adding_defaults, getColumns().getDefaults(), context)); + pipe.getHeader(), header_after_adding_defaults, metadata_snapshot->getColumns().getDefaults(), context)); pipe.addSimpleTransform(std::make_shared( pipe.getHeader(), header, ConvertingTransform::MatchColumnsMode::Name)); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index ce4fcbb3513..238623c1576 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -290,7 +290,7 @@ StorageDistributed::StorageDistributed( if (sharding_key_) { - sharding_key_expr = buildShardingKeyExpression(sharding_key_, *global_context, getColumns().getAllPhysical(), false); + sharding_key_expr = buildShardingKeyExpression(sharding_key_, *global_context, metadata_.getColumns().getAllPhysical(), false); sharding_key_column_name = sharding_key_->getColumnName(); } @@ -447,6 +447,7 @@ bool StorageDistributed::canForceGroupByNoMerge(const Context &context, QueryPro QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context &context, QueryProcessingStage::Enum to_stage, const ASTPtr & query_ptr) const { const auto & settings = context.getSettingsRef(); + auto metadata_snapshot = getInMemoryMetadataPtr(); if (canForceGroupByNoMerge(context, to_stage, query_ptr)) return QueryProcessingStage::Complete; @@ -454,7 +455,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Con ClusterPtr cluster = getCluster(); if (settings.optimize_skip_unused_shards) { - ClusterPtr optimized_cluster = getOptimizedCluster(context, query_ptr); + ClusterPtr optimized_cluster = getOptimizedCluster(context, metadata_snapshot, query_ptr); if (optimized_cluster) cluster = optimized_cluster; } @@ -476,7 +477,7 @@ Pipes StorageDistributed::read( ClusterPtr cluster = getCluster(); if (settings.optimize_skip_unused_shards) { - ClusterPtr optimized_cluster = getOptimizedCluster(context, query_info.query); + ClusterPtr optimized_cluster = getOptimizedCluster(context, metadata_snapshot, query_info.query); if (optimized_cluster) { LOG_DEBUG(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}", makeFormattedListOfShards(optimized_cluster)); @@ -683,14 +684,14 @@ ClusterPtr StorageDistributed::getCluster() const return owned_cluster ? owned_cluster : global_context->getCluster(cluster_name); } -ClusterPtr StorageDistributed::getOptimizedCluster(const Context & context, const ASTPtr & query_ptr) const +ClusterPtr StorageDistributed::getOptimizedCluster(const Context & context, const StorageMetadataPtr & metadata_snapshot, const ASTPtr & query_ptr) const { ClusterPtr cluster = getCluster(); const Settings & settings = context.getSettingsRef(); if (has_sharding_key) { - ClusterPtr optimized = skipUnusedShards(cluster, query_ptr, context); + ClusterPtr optimized = skipUnusedShards(cluster, query_ptr, metadata_snapshot, context); if (optimized) return optimized; } @@ -751,7 +752,11 @@ IColumn::Selector StorageDistributed::createSelector(const ClusterPtr cluster, c /// Returns a new cluster with fewer shards if constant folding for `sharding_key_expr` is possible /// using constraints from "PREWHERE" and "WHERE" conditions, otherwise returns `nullptr` -ClusterPtr StorageDistributed::skipUnusedShards(ClusterPtr cluster, const ASTPtr & query_ptr, const Context & context) const +ClusterPtr StorageDistributed::skipUnusedShards( + ClusterPtr cluster, + const ASTPtr & query_ptr, + const StorageMetadataPtr & metadata_snapshot, + const Context & context) const { const auto & select = query_ptr->as(); @@ -770,7 +775,7 @@ ClusterPtr StorageDistributed::skipUnusedShards(ClusterPtr cluster, const ASTPtr condition_ast = select.prewhere() ? select.prewhere()->clone() : select.where()->clone(); } - replaceConstantExpressions(condition_ast, context, getColumns().getAll(), shared_from_this()); + replaceConstantExpressions(condition_ast, context, metadata_snapshot->getColumns().getAll(), shared_from_this()); const auto blocks = evaluateExpressionOverConstantCondition(condition_ast, sharding_key_expr); // Can't get definite answer if we can skip any shards diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 3f148cfff01..af508a80646 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -120,8 +120,8 @@ public: /// Apply the following settings: /// - optimize_skip_unused_shards /// - force_optimize_skip_unused_shards - ClusterPtr getOptimizedCluster(const Context &, const ASTPtr & query_ptr) const; - ClusterPtr skipUnusedShards(ClusterPtr cluster, const ASTPtr & query_ptr, const Context & context) const; + ClusterPtr getOptimizedCluster(const Context &, const StorageMetadataPtr & metadata_snapshot, const ASTPtr & query_ptr) const; + ClusterPtr skipUnusedShards(ClusterPtr cluster, const ASTPtr & query_ptr, const StorageMetadataPtr & metadata_snapshot, const Context & context) const; ActionLock getActionLock(StorageActionBlockType type) override; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 65f36a48170..8083a8b2145 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -417,7 +417,7 @@ Pipes StorageFile::read( for (size_t i = 0; i < num_streams; ++i) pipes.emplace_back(std::make_shared( - this_ptr, metadata_snapshot, context, max_block_size, files_info, getColumns().getDefaults())); + this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults())); return pipes; } diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index bcebeec09dd..dad323f7b72 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -441,7 +441,7 @@ Pipes StorageGenerateRandom::read( Pipes pipes; pipes.reserve(num_streams); - const ColumnsDescription & our_columns = getColumns(); + const ColumnsDescription & our_columns = metadata_snapshot->getColumns(); Block block_header; for (const auto & name : column_names) { diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index fcae9c9aa82..e0953283a17 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -358,7 +358,7 @@ void LogBlockOutputStream::writeData(const String & name, const IDataType & type if (written_streams.count(stream_name)) return; - const auto & columns = storage.getColumns(); + const auto & columns = metadata_snapshot->getColumns(); streams.try_emplace( stream_name, storage.disk, @@ -445,7 +445,7 @@ StorageLog::StorageLog( /// create directories if they do not exist disk->createDirectories(table_path); - for (const auto & column : getColumns().getAllPhysical()) + for (const auto & column : metadata_.getColumns().getAllPhysical()) addFiles(column.name, *column.type); marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME; @@ -539,13 +539,14 @@ void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLo { std::shared_lock lock(rwlock); + auto metadata_snapshot = getInMemoryMetadataPtr(); files.clear(); file_count = 0; loaded_marks = false; disk->clearDirectory(table_path); - for (const auto & column : getColumns().getAllPhysical()) + for (const auto & column : metadata_snapshot->getColumns().getAllPhysical()) addFiles(column.name, *column.type); file_checker = FileChecker{disk, table_path + "sizes.json"}; @@ -553,11 +554,11 @@ void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLo } -const StorageLog::Marks & StorageLog::getMarksWithRealRowCount() const +const StorageLog::Marks & StorageLog::getMarksWithRealRowCount(const StorageMetadataPtr & metadata_snapshot) const { /// There should be at least one physical column - const String column_name = getColumns().getAllPhysical().begin()->name; - const auto column_type = getColumns().getAllPhysical().begin()->type; + const String column_name = metadata_snapshot->getColumns().getAllPhysical().begin()->name; + const auto column_type = metadata_snapshot->getColumns().getAllPhysical().begin()->type; String filename; /** We take marks from first column. @@ -590,13 +591,13 @@ Pipes StorageLog::read( metadata_snapshot->check(column_names, getVirtuals()); loadMarks(); - NamesAndTypesList all_columns = Nested::collect(getColumns().getAllPhysical().addTypes(column_names)); + NamesAndTypesList all_columns = Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)); std::shared_lock lock(rwlock); Pipes pipes; - const Marks & marks = getMarksWithRealRowCount(); + const Marks & marks = getMarksWithRealRowCount(metadata_snapshot); size_t marks_size = marks.size(); if (num_streams > marks_size) diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 60f885ce45c..90d0799e1a8 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -26,7 +26,7 @@ public: Pipes read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -112,7 +112,7 @@ private: * * Return the first group of marks that contain the number of rows, but not the internals of the arrays. */ - const Marks & getMarksWithRealRowCount() const; + const Marks & getMarksWithRealRowCount(const StorageMetadataPtr & metadata_snapshot) const; }; } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 92e965c420e..228cec99357 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -250,9 +250,11 @@ Pipes StorageMerge::createSources( if (!storage) { - auto pipe = InterpreterSelectQuery(modified_query_info.query, *modified_context, - std::make_shared(header), - SelectQueryOptions(processed_stage).analyze()).execute().pipeline.getPipe(); + auto pipe = InterpreterSelectQuery( + modified_query_info.query, *modified_context, + std::make_shared(header), + SelectQueryOptions(processed_stage).analyze()).execute().pipeline.getPipe(); + pipe.addInterpreterContext(modified_context); pipes.emplace_back(std::move(pipe)); return pipes; @@ -263,7 +265,7 @@ Pipes StorageMerge::createSources( { /// If there are only virtual columns in query, you must request at least one other column. if (real_column_names.empty()) - real_column_names.push_back(ExpressionActions::getSmallestColumn(storage->getColumns().getAllPhysical())); + real_column_names.push_back(ExpressionActions::getSmallestColumn(metadata_snapshot->getColumns().getAllPhysical())); pipes = storage->read(real_column_names, metadata_snapshot, modified_query_info, *modified_context, processed_stage, max_block_size, UInt32(streams_num)); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b0a7e550233..e45d54a8c64 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -500,7 +500,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata_str, zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/columns", getColumns().toString(), + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log", "", zkutil::CreateMode::Persistent)); @@ -535,7 +535,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata", metadata_str, zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/columns", getColumns().toString(), + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", std::to_string(metadata_version), zkutil::CreateMode::Persistent)); @@ -596,7 +596,7 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata", ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(), zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/columns", getColumns().toString(), + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", std::to_string(metadata_version), zkutil::CreateMode::Persistent)); @@ -748,7 +748,7 @@ void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_pr Coordination::Stat columns_stat; auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_prefix + "/columns", &columns_stat)); - const ColumnsDescription & old_columns = getColumns(); + const ColumnsDescription & old_columns = metadata_snapshot->getColumns(); if (columns_from_zk != old_columns) { throw Exception("Table columns structure in ZooKeeper is different from local table structure", ErrorCodes::INCOMPATIBLE_COLUMNS); diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 4015d8ca574..4578a82f650 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -237,8 +237,9 @@ void TinyLogSource::readData(const String & name, const IDataType & type, IColum } -IDataType::OutputStreamGetter TinyLogBlockOutputStream::createStreamGetter(const String & name, - WrittenStreams & written_streams) +IDataType::OutputStreamGetter TinyLogBlockOutputStream::createStreamGetter( + const String & name, + WrittenStreams & written_streams) { return [&] (const IDataType::SubstreamPath & path) -> WriteBuffer * { @@ -247,12 +248,13 @@ IDataType::OutputStreamGetter TinyLogBlockOutputStream::createStreamGetter(const if (!written_streams.insert(stream_name).second) return nullptr; - const auto & columns = storage.getColumns(); + const auto & columns = metadata_snapshot->getColumns(); if (!streams.count(stream_name)) - streams[stream_name] = std::make_unique(storage.disk, - storage.files[stream_name].data_file_path, - columns.getCodecOrDefault(name), - storage.max_compress_block_size); + streams[stream_name] = std::make_unique( + storage.disk, + storage.files[stream_name].data_file_path, + columns.getCodecOrDefault(name), + storage.max_compress_block_size); return &streams[stream_name]->compressed; }; @@ -351,7 +353,7 @@ StorageTinyLog::StorageTinyLog( disk->createDirectories(table_path); } - for (const auto & col : getColumns().getAllPhysical()) + for (const auto & col : metadata_.getColumns().getAllPhysical()) addFiles(col.name, *col.type); } @@ -430,13 +432,14 @@ CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context void StorageTinyLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { std::unique_lock lock(rwlock); + auto metadata_snapshot = getInMemoryMetadataPtr(); disk->clearDirectory(table_path); files.clear(); file_checker = FileChecker{disk, table_path + "sizes.json"}; - for (const auto &column : getColumns().getAllPhysical()) + for (const auto & column : metadata_snapshot->getColumns().getAllPhysical()) addFiles(column.name, *column.type); } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 949d922b611..802ad0571a8 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -136,7 +136,9 @@ std::string IStorageURLBase::getReadMethod() const return Poco::Net::HTTPRequest::HTTP_GET; } -std::vector> IStorageURLBase::getReadURIParams(const Names & /*column_names*/, +std::vector> IStorageURLBase::getReadURIParams( + const Names & /*column_names*/, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum & /*processed_stage*/, @@ -145,7 +147,9 @@ std::vector> IStorageURLBase::getReadURIPara return {}; } -std::function IStorageURLBase::getReadPOSTDataCallback(const Names & /*column_names*/, +std::function IStorageURLBase::getReadPOSTDataCallback( + const Names & /*column_names*/, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum & /*processed_stage*/, @@ -165,7 +169,7 @@ Pipes IStorageURLBase::read( unsigned /*num_streams*/) { auto request_uri = uri; - auto params = getReadURIParams(column_names, query_info, context, processed_stage, max_block_size); + auto params = getReadURIParams(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size); for (const auto & [param, value] : params) request_uri.addQueryParameter(param, value); @@ -173,7 +177,9 @@ Pipes IStorageURLBase::read( pipes.emplace_back(std::make_shared( request_uri, getReadMethod(), - getReadPOSTDataCallback(column_names, query_info, context, processed_stage, max_block_size), + getReadPOSTDataCallback( + column_names, metadata_snapshot, query_info, + context, processed_stage, max_block_size), format_name, getName(), getHeaderBlock(column_names, metadata_snapshot), diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 04cbb278c37..67ad95d2f91 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -50,6 +50,7 @@ private: virtual std::vector> getReadURIParams( const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum & processed_stage, @@ -57,6 +58,7 @@ private: virtual std::function getReadPOSTDataCallback( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum & processed_stage, @@ -68,12 +70,13 @@ private: class StorageURLBlockOutputStream : public IBlockOutputStream { public: - StorageURLBlockOutputStream(const Poco::URI & uri, - const String & format, - const Block & sample_block_, - const Context & context, - const ConnectionTimeouts & timeouts, - const CompressionMethod compression_method); + StorageURLBlockOutputStream( + const Poco::URI & uri, + const String & format, + const Block & sample_block_, + const Context & context, + const ConnectionTimeouts & timeouts, + const CompressionMethod compression_method); Block getHeader() const override { diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index ab8b37db7db..05cf4ed5abf 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -50,7 +50,9 @@ std::string StorageXDBC::getReadMethod() const return Poco::Net::HTTPRequest::HTTP_POST; } -std::vector> StorageXDBC::getReadURIParams(const Names & column_names, +std::vector> StorageXDBC::getReadURIParams( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum & /*processed_stage*/, @@ -59,20 +61,22 @@ std::vector> StorageXDBC::getReadURIParams(c NamesAndTypesList cols; for (const String & name : column_names) { - auto column_data = getColumns().getPhysical(name); + auto column_data = metadata_snapshot->getColumns().getPhysical(name); cols.emplace_back(column_data.name, column_data.type); } return bridge_helper->getURLParams(cols.toString(), max_block_size); } -std::function StorageXDBC::getReadPOSTDataCallback(const Names & /*column_names*/, +std::function StorageXDBC::getReadPOSTDataCallback( + const Names & /*column_names*/, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum & /*processed_stage*/, size_t /*max_block_size*/) const { String query = transformQueryForExternalDatabase(query_info, - getColumns().getOrdinary(), + metadata_snapshot->getColumns().getOrdinary(), bridge_helper->getIdentifierQuotingStyle(), remote_database_name, remote_table_name, diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index 0e227d7d432..44931af4643 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -29,7 +29,8 @@ public: const std::string & remote_database_name, const std::string & remote_table_name, const ColumnsDescription & columns_, - const Context & context_, BridgeHelperPtr bridge_helper_); + const Context & context_, + BridgeHelperPtr bridge_helper_); BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; @@ -45,6 +46,7 @@ private: std::vector> getReadURIParams( const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum & processed_stage, @@ -52,6 +54,7 @@ private: std::function getReadPOSTDataCallback( const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum & processed_stage, diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index 479621fd47f..4631bb9c4c5 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -71,7 +71,7 @@ void StorageSystemPartsColumns::processNextStorage(MutableColumns & columns_, co }; std::unordered_map columns_info; - for (const auto & column : info.storage->getColumns()) + for (const auto & column : info.storage->getInMemoryMetadataPtr()->getColumns()) { ColumnInfo column_info; if (column.default_desc.expression) diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index ea6b3e64aff..6e0d323e8a0 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -220,7 +220,7 @@ TTLDescription TTLDescription::getTTLFromAST( if (value->as()) { - auto syntax_result = SyntaxAnalyzer(context).analyze(value, columns.getAllPhysical(), {}, true); + auto syntax_result = SyntaxAnalyzer(context).analyze(value, columns.getAllPhysical(), {}, {}, true); auto expr_actions = ExpressionAnalyzer(value, syntax_result, context).getActions(false); for (const auto & column : expr_actions->getRequiredColumns()) { @@ -249,7 +249,7 @@ TTLDescription TTLDescription::getTTLFromAST( for (auto [name, value] : aggregations) { - auto syntax_result = SyntaxAnalyzer(context).analyze(value, columns.getAllPhysical(), {}, true); + auto syntax_result = SyntaxAnalyzer(context).analyze(value, columns.getAllPhysical(), {}, {}, true); auto expr_analyzer = ExpressionAnalyzer(value, syntax_result, context); TTLAggregateDescription set_part; diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 19d1172f1ff..aca5456d85e 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -75,7 +75,8 @@ ColumnsDescription getStructureOfRemoteTableInShard( { const auto * table_function = table_func_ptr->as(); TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_function->name, context); - return table_function_ptr->execute(table_func_ptr, context, table_function_ptr->getName())->getColumns(); + auto storage_ptr = table_function_ptr->execute(table_func_ptr, context, table_function_ptr->getName()); + return storage_ptr->getInMemoryMetadataPtr()->getColumns(); } auto table_func_name = queryToString(table_func_ptr); @@ -84,7 +85,10 @@ ColumnsDescription getStructureOfRemoteTableInShard( else { if (shard_info.isLocal()) - return DatabaseCatalog::instance().getTable(table_id, context)->getColumns(); + { + auto storage_ptr = DatabaseCatalog::instance().getTable(table_id, context); + return storage_ptr->getInMemoryMetadataPtr()->getColumns(); + } /// Request for a table description query = "DESC TABLE " + table_id.getFullTableName(); diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 618d524987b..c97adaf118d 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -78,7 +78,7 @@ std::string writeData(int rows, DB::StoragePtr & table, const DB::Context & cont Block block; { - const auto & storage_columns = table->getColumns(); + const auto & storage_columns = metadata_snapshot->getColumns(); ColumnWithTypeAndName column; column.name = "a"; column.type = storage_columns.getPhysical("a").type; diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index ee447a13174..7c0c1fb233f 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -42,7 +42,7 @@ static NamesAndTypesList chooseColumns(const String & source_database, const Str throw Exception("Error while executing table function merge. In database " + source_database + " no one matches regular expression: " + table_name_regexp_, ErrorCodes::UNKNOWN_TABLE); - return any_table->getColumns().getAllPhysical(); + return any_table->getInMemoryMetadataPtr()->getColumns().getAllPhysical(); } From 51a47560e0a980344fb34b0d06d99885a74948f9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jun 2020 19:59:26 +0300 Subject: [PATCH 085/211] Fix segmentation fault --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f601ca74112..d832bcb7dc0 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1180,7 +1180,7 @@ void InterpreterSelectQuery::executeFetchColumns( = ext::map(required_columns_after_prewhere, [](const auto & it) { return it.name; }); } - auto syntax_result = SyntaxAnalyzer(*context).analyze(required_columns_all_expr, required_columns_after_prewhere, storage); + auto syntax_result = SyntaxAnalyzer(*context).analyze(required_columns_all_expr, required_columns_after_prewhere, storage, metadata_snapshot); alias_actions = ExpressionAnalyzer(required_columns_all_expr, syntax_result, *context).getActions(true); /// The set of required columns could be added as a result of adding an action to calculate ALIAS. From f4037b8f54840ca0b449e86e0bb575de9c01e1be Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 12:00:43 +0300 Subject: [PATCH 086/211] Fix build --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 9f8859385d3..0e9a2bf7ec4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -743,7 +743,7 @@ void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_pr Coordination::Stat metadata_stat; String metadata_str = zookeeper->get(zookeeper_prefix + "/metadata", &metadata_stat); auto metadata_from_zk = ReplicatedMergeTreeTableMetadata::parse(metadata_str); - old_metadata.checkEquals(metadata_from_zk, getColumns(), global_context); + old_metadata.checkEquals(metadata_from_zk, metadata_snapshot->getColumns(), global_context); Coordination::Stat columns_stat; auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_prefix + "/columns", &columns_stat)); From 760e9a8488f0a5ee24a4ccce141593195c3ffcef Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 12:08:24 +0300 Subject: [PATCH 087/211] Fix crash --- src/Storages/StorageDistributed.cpp | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index f853a6a8673..53342c754ed 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -215,9 +215,14 @@ public: } }; -void replaceConstantExpressions(ASTPtr & node, const Context & context, const NamesAndTypesList & columns, ConstStoragePtr storage) +void replaceConstantExpressions( + ASTPtr & node, + const Context & context, + const NamesAndTypesList & columns, + ConstStoragePtr storage, + const StorageMetadataPtr & metadata_snapshot) { - auto syntax_result = SyntaxAnalyzer(context).analyze(node, columns, storage); + auto syntax_result = SyntaxAnalyzer(context).analyze(node, columns, storage, metadata_snapshot); Block block_with_constants = KeyCondition::getBlockWithConstants(node, syntax_result, context); InDepthNodeVisitor visitor(block_with_constants); @@ -777,7 +782,7 @@ ClusterPtr StorageDistributed::skipUnusedShards( condition_ast = select.prewhere() ? select.prewhere()->clone() : select.where()->clone(); } - replaceConstantExpressions(condition_ast, context, metadata_snapshot->getColumns().getAll(), shared_from_this()); + replaceConstantExpressions(condition_ast, context, metadata_snapshot->getColumns().getAll(), shared_from_this(), metadata_snapshot); const auto blocks = evaluateExpressionOverConstantCondition(condition_ast, sharding_key_expr); // Can't get definite answer if we can skip any shards From 35ce47951d7e7f2d9490b97af16899933af4285e Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 12:22:54 +0300 Subject: [PATCH 088/211] Fix storage merge --- src/Storages/StorageMerge.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 228cec99357..235f78505e0 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -170,9 +170,7 @@ Pipes StorageMerge::read( if (selected_tables.empty()) /// FIXME: do we support sampling in this case? return createSources( - metadata_snapshot, query_info, processed_stage, - max_block_size, header, {}, real_column_names, - modified_context, 0, has_table_virtual_column); + {}, query_info, processed_stage, max_block_size, header, {}, real_column_names, modified_context, 0, has_table_virtual_column); size_t tables_count = selected_tables.size(); Float64 num_streams_multiplier = std::min(unsigned(tables_count), std::max(1U, unsigned(context.getSettingsRef().max_streams_multiplier_for_merge_tables))); @@ -212,8 +210,10 @@ Pipes StorageMerge::read( if (query_info.query->as()->sampleSize() && !storage->supportsSampling()) throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED); + auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); + auto source_pipes = createSources( - metadata_snapshot, query_info, processed_stage, + storage_metadata_snapshot, query_info, processed_stage, max_block_size, header, table, real_column_names, modified_context, current_streams, has_table_virtual_column); @@ -267,6 +267,7 @@ Pipes StorageMerge::createSources( if (real_column_names.empty()) real_column_names.push_back(ExpressionActions::getSmallestColumn(metadata_snapshot->getColumns().getAllPhysical())); + pipes = storage->read(real_column_names, metadata_snapshot, modified_query_info, *modified_context, processed_stage, max_block_size, UInt32(streams_num)); } else if (processed_stage > storage_stage) From aab4ce6394c9bd72622f29f749e5bb5f56a6851a Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 13:29:13 +0300 Subject: [PATCH 089/211] Truncate with metadata --- src/Interpreters/InterpreterDropQuery.cpp | 6 ++++-- src/Storages/IStorage.h | 6 +++++- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageDistributed.h | 2 +- src/Storages/StorageFile.cpp | 6 +++++- src/Storages/StorageFile.h | 6 +++++- src/Storages/StorageJoin.cpp | 5 ++--- src/Storages/StorageJoin.h | 2 +- src/Storages/StorageLog.cpp | 3 +-- src/Storages/StorageLog.h | 2 +- src/Storages/StorageMaterializedView.cpp | 2 +- src/Storages/StorageMaterializedView.h | 2 +- src/Storages/StorageMemory.cpp | 3 ++- src/Storages/StorageMemory.h | 4 ++-- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/StorageSet.cpp | 3 +-- src/Storages/StorageSet.h | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- src/Storages/StorageStripeLog.h | 2 +- src/Storages/StorageTinyLog.cpp | 4 ++-- src/Storages/StorageTinyLog.h | 2 +- 24 files changed, 44 insertions(+), 31 deletions(-) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 5ffce2fc3ec..15f19b585de 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -105,8 +105,9 @@ BlockIO InterpreterDropQuery::executeToTable( table->checkTableCanBeDropped(); auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto metadata_snapshot = table->getInMemoryMetadataPtr(); /// Drop table data, don't touch metadata - table->truncate(query_ptr, context, table_lock); + table->truncate(query_ptr, metadata_snapshot, context, table_lock); } else if (query.kind == ASTDropQuery::Kind::Drop) { @@ -187,7 +188,8 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(const String & table_name, { auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); /// Drop table data, don't touch metadata - table->truncate(query_ptr, context, table_lock); + auto metadata_snapshot = table->getInMemoryMetadataPtr(); + table->truncate(query_ptr, metadata_snapshot, context, table_lock); } else if (kind == ASTDropQuery::Kind::Drop) { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index ba1945d5c79..ec13e26ff43 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -292,7 +292,11 @@ public: /** Clear the table data and leave it empty. * Must be called under lockForAlter. */ - virtual void truncate(const ASTPtr & /*query*/, const Context & /* context */, TableStructureWriteLockHolder &) + virtual void truncate( + const ASTPtr & /*query*/, + const StorageMetadataPtr & /* metadata_snapshot */, + const Context & /* context */, + TableStructureWriteLockHolder &) { throw Exception("Truncate is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 53342c754ed..2e07a393b04 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -619,7 +619,7 @@ Strings StorageDistributed::getDataPaths() const return paths; } -void StorageDistributed::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) +void StorageDistributed::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) { std::lock_guard lock(cluster_nodes_mutex); diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index aeb5f1875e9..c952ccde8ac 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -82,7 +82,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; /// Removes temporary data in local filesystem. - void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; void renameOnDisk(const String & new_path_to_table_data); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 8083a8b2145..8fb09d0a41e 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -523,7 +523,11 @@ void StorageFile::rename(const String & new_path_to_table_data, const StorageID renameInMemory(new_table_id); } -void StorageFile::truncate(const ASTPtr & /*query*/, const Context & /* context */, TableStructureWriteLockHolder &) +void StorageFile::truncate( + const ASTPtr & /*query*/, + const StorageMetadataPtr & /* metadata_snapshot */, + const Context & /* context */, + TableStructureWriteLockHolder &) { if (paths.size() != 1) throw Exception("Can't truncate table '" + getStorageID().getNameForLogs() + "' in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 65589d245b9..05b4d7aea8a 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -38,7 +38,11 @@ public: const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; - void truncate(const ASTPtr & /*query*/, const Context & /* context */, TableStructureWriteLockHolder &) override; + void truncate( + const ASTPtr & /*query*/, + const StorageMetadataPtr & /* metadata_snapshot */, + const Context & /* context */, + TableStructureWriteLockHolder &) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 300ab400a46..5000dcd8b18 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -64,10 +64,9 @@ StorageJoin::StorageJoin( } -void StorageJoin::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) +void StorageJoin::truncate( + const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) { - /// TODO(alesap) FIXME - auto metadata_snapshot = getInMemoryMetadataPtr(); Poco::File(path).remove(true); Poco::File(path).createDirectories(); Poco::File(path + "tmp/").createDirectories(); diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 40dbf1b44dd..4d4d1a81da2 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -27,7 +27,7 @@ class StorageJoin final : public ext::shared_ptr_helper, public Sto public: String getName() const override { return "Join"; } - void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) override; /// Access the innards. HashJoinPtr & getJoin() { return join; } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index e0953283a17..45d55938db3 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -535,11 +535,10 @@ void StorageLog::rename(const String & new_path_to_table_data, const StorageID & renameInMemory(new_table_id); } -void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) +void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) { std::shared_lock lock(rwlock); - auto metadata_snapshot = getInMemoryMetadataPtr(); files.clear(); file_count = 0; loaded_marks = false; diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 90d0799e1a8..670e2777d44 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -39,7 +39,7 @@ public: CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override; - void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) override; Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 3d3137fe1a6..2c0d5727b31 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -173,7 +173,7 @@ void StorageMaterializedView::drop() executeDropQuery(ASTDropQuery::Kind::Drop, global_context, target_table_id); } -void StorageMaterializedView::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) +void StorageMaterializedView::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) { if (has_inner_table) executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, target_table_id); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 6f462c2cccc..e2111a15f5c 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -37,7 +37,7 @@ public: void drop() override; - void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) override; bool optimize( const ASTPtr & query, diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index f9ef3cfcc98..3bae29ac96c 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -148,7 +148,8 @@ void StorageMemory::drop() data.clear(); } -void StorageMemory::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) +void StorageMemory::truncate( + const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) { std::lock_guard lock(mutex); data.clear(); diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 3c583533462..842c7dc3790 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -37,11 +37,11 @@ public: size_t max_block_size, unsigned num_streams) override; - BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const Context & context) override; void drop() override; - void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) override; std::optional totalRows() const override; std::optional totalBytes() const override; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 40b3aeffb8a..45f8ecf0ef9 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -231,7 +231,7 @@ void StorageMergeTree::drop() dropAllData(); } -void StorageMergeTree::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) +void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) { { /// Asks to complete merges and does not allow them to start. diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 69ee6714164..cf3eccc0c0b 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -75,7 +75,7 @@ public: CancellationCode killMutation(const String & mutation_id) override; void drop() override; - void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) override; void alter(const AlterCommands & commands, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0e9a2bf7ec4..2e64c54112f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4011,7 +4011,8 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPt } -void StorageReplicatedMergeTree::truncate(const ASTPtr & query, const Context & query_context, TableStructureWriteLockHolder & table_lock) +void StorageReplicatedMergeTree::truncate( + const ASTPtr & query, const StorageMetadataPtr &, const Context & query_context, TableStructureWriteLockHolder & table_lock) { table_lock.release(); /// Truncate is done asynchronously. diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index c98fcb0ae3d..c1ba737d849 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -120,7 +120,7 @@ public: */ void drop() override; - void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 235251c0761..93b288569c2 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -142,9 +142,8 @@ void StorageSet::finishInsert() { set->finishInsert(); } size_t StorageSet::getSize() const { return set->getTotalRowCount(); } -void StorageSet::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) +void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) { - auto metadata_snapshot = getInMemoryMetadataPtr(); Poco::File(path).remove(true); Poco::File(path).createDirectories(); Poco::File(path + "tmp/").createDirectories(); diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index b7785aadc6a..2685fa26ba6 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -67,7 +67,7 @@ public: /// Access the insides. SetPtr & getSet() { return set; } - void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) override; private: SetPtr set; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 407c9b164ff..4b95a389f2c 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -326,7 +326,7 @@ CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Conte return file_checker.check(); } -void StorageStripeLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) +void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) { std::shared_lock lock(rwlock); diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index d06758a60e8..381be7762df 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -42,7 +42,7 @@ public: Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } - void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) override; protected: StorageStripeLog( diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 4578a82f650..4beb44405d7 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -429,10 +429,10 @@ CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context return file_checker.check(); } -void StorageTinyLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) +void StorageTinyLog::truncate( + const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) { std::unique_lock lock(rwlock); - auto metadata_snapshot = getInMemoryMetadataPtr(); disk->clearDirectory(table_path); diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index a55bf6d0dcf..ae124e5e958 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -41,7 +41,7 @@ public: Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } - void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) override; void drop() override; From 4de5331b0df17163e05fe11893ea4c2da35ad985 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 13:39:33 +0300 Subject: [PATCH 090/211] Fix SystemTables --- src/Storages/System/StorageSystemTables.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 84635acb887..df8df75ad6d 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -267,7 +267,6 @@ protected: throw; } } - auto metadata_snapshot = table->getInMemoryMetadataPtr(); ++rows_count; @@ -362,10 +361,14 @@ protected: else src_index += 2; + StorageMetadataPtr metadata_snapshot; + if (table != nullptr) + metadata_snapshot = table->getInMemoryMetadataPtr(); + ASTPtr expression_ptr; if (columns_mask[src_index++]) { - assert(table != nullptr); + assert(metadata_snapshot != nullptr); if ((expression_ptr = metadata_snapshot->getPartitionKeyAST())) res_columns[res_index++]->insert(queryToString(expression_ptr)); else @@ -374,7 +377,7 @@ protected: if (columns_mask[src_index++]) { - assert(table != nullptr); + assert(metadata_snapshot != nullptr); if ((expression_ptr = metadata_snapshot->getSortingKey().expression_list_ast)) res_columns[res_index++]->insert(queryToString(expression_ptr)); else @@ -383,7 +386,7 @@ protected: if (columns_mask[src_index++]) { - assert(table != nullptr); + assert(metadata_snapshot != nullptr); if ((expression_ptr = metadata_snapshot->getPrimaryKey().expression_list_ast)) res_columns[res_index++]->insert(queryToString(expression_ptr)); else @@ -392,7 +395,7 @@ protected: if (columns_mask[src_index++]) { - assert(table != nullptr); + assert(metadata_snapshot != nullptr); if ((expression_ptr = metadata_snapshot->getSamplingKeyAST())) res_columns[res_index++]->insert(queryToString(expression_ptr)); else @@ -401,7 +404,7 @@ protected: if (columns_mask[src_index++]) { - assert(table != nullptr); + assert(metadata_snapshot != nullptr); auto policy = table->getStoragePolicy(); if (policy) res_columns[res_index++]->insert(policy->getName()); From d4c49816ab140f1ca0d73c173d1cd1a62b8003fd Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 14:02:31 +0300 Subject: [PATCH 091/211] Really atomic metadata --- src/DataStreams/InputStreamFromASTInsertQuery.h | 2 +- src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/JoinedTables.h | 2 +- src/Interpreters/SyntaxAnalyzer.h | 2 +- src/Storages/IStorage.h | 15 ++++++++------- src/Storages/MergeTree/MergeTreeWhereOptimizer.h | 2 +- src/Storages/StorageInMemoryMetadata.h | 2 +- 7 files changed, 14 insertions(+), 13 deletions(-) diff --git a/src/DataStreams/InputStreamFromASTInsertQuery.h b/src/DataStreams/InputStreamFromASTInsertQuery.h index 0604f011e28..d4c6443c77d 100644 --- a/src/DataStreams/InputStreamFromASTInsertQuery.h +++ b/src/DataStreams/InputStreamFromASTInsertQuery.h @@ -12,7 +12,7 @@ namespace DB struct BlockIO; class Context; struct StorageInMemoryMetadata; -using StorageMetadataPtr = std::shared_ptr; +using StorageMetadataPtr = std::shared_ptr; /** Prepares an input stream which produce data containing in INSERT query * Head of inserting data could be stored in INSERT ast directly diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index cd0b837b4ec..31939f5016b 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -32,7 +32,7 @@ class ASTSelectQuery; struct ASTTablesInSelectQueryElement; struct StorageInMemoryMetadata; -using StorageMetadataPtr = std::shared_ptr; +using StorageMetadataPtr = std::shared_ptr; /// Create columns in block or return false if not possible bool sanitizeBlock(Block & block); diff --git a/src/Interpreters/JoinedTables.h b/src/Interpreters/JoinedTables.h index cff86c5a535..f150de83a94 100644 --- a/src/Interpreters/JoinedTables.h +++ b/src/Interpreters/JoinedTables.h @@ -14,7 +14,7 @@ class ASTSelectQuery; class TableJoin; struct SelectQueryOptions; struct StorageInMemoryMetadata; -using StorageMetadataPtr = std::shared_ptr; +using StorageMetadataPtr = std::shared_ptr; /// Joined tables' columns resolver. /// We want to get each table structure at most once per table occurance. Or even better once per table. diff --git a/src/Interpreters/SyntaxAnalyzer.h b/src/Interpreters/SyntaxAnalyzer.h index 4308b70c45a..dd3c49a0f1a 100644 --- a/src/Interpreters/SyntaxAnalyzer.h +++ b/src/Interpreters/SyntaxAnalyzer.h @@ -17,7 +17,7 @@ struct Settings; struct SelectQueryOptions; using Scalars = std::map; struct StorageInMemoryMetadata; -using StorageMetadataPtr = std::shared_ptr; +using StorageMetadataPtr = std::shared_ptr; struct SyntaxAnalyzerResult { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index ec13e26ff43..375ab90aee4 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -82,7 +82,7 @@ public: IStorage() = delete; /// Storage fields should be initialized in separate methods like setColumns /// or setTableTTLs. - explicit IStorage(StorageID storage_id_) : storage_id(std::move(storage_id_)), metadata(std::make_shared()) {} //-V730 + explicit IStorage(StorageID storage_id_) : storage_id(std::move(storage_id_)), metadata(std::make_unique()) {} //-V730 virtual ~IStorage() = default; IStorage(const IStorage &) = delete; @@ -137,9 +137,12 @@ public: public: /// thread-unsafe part. lockStructure must be acquired - StorageInMemoryMetadata getInMemoryMetadata() const { return *metadata; } - StorageMetadataPtr getInMemoryMetadataPtr() const { return metadata; } - void setInMemoryMetadata(const StorageInMemoryMetadata & metadata_) { metadata = std::make_shared(metadata_); } + StorageInMemoryMetadata getInMemoryMetadata() const { return *metadata.get(); } + StorageMetadataPtr getInMemoryMetadataPtr() const { return metadata.get(); } + void setInMemoryMetadata(const StorageInMemoryMetadata & metadata_) + { + metadata.set(std::make_unique(metadata_)); + } /// Return list of virtual columns (like _part, _table, etc). In the vast @@ -165,9 +168,7 @@ private: StorageID storage_id; mutable std::mutex id_mutex; - /// TODO (alesap) just use multiversion for atomic metadata - mutable std::mutex ttl_mutex; - StorageMetadataPtr metadata; + MultiVersionStorageMetadataPtr metadata; private: RWLockImpl::LockHolder tryLockTimed( const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const SettingSeconds & acquire_timeout) const; diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 866d0a8754e..cb2f8939cb5 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -17,7 +17,7 @@ class ASTSelectQuery; class ASTFunction; class MergeTreeData; struct StorageInMemoryMetadata; -using StorageMetadataPtr = std::shared_ptr; +using StorageMetadataPtr = std::shared_ptr; /** Identifies WHERE expressions that can be placed in PREWHERE by calculating respective * sizes of columns used in particular expression and identifying "good" conditions of diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index e4755bb0464..bda48bc19cb 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -194,7 +194,7 @@ struct StorageInMemoryMetadata void check(const Block & block, bool need_all = false) const; }; -using StorageMetadataPtr = std::shared_ptr; +using StorageMetadataPtr = std::shared_ptr; using MultiVersionStorageMetadataPtr = MultiVersion; } From 1a69c3234ae26dd158d1102adf7223f2dd8ff0d3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 14:09:55 +0300 Subject: [PATCH 092/211] Fix style --- src/Storages/IStorage.cpp | 7 ------- src/Storages/StorageInMemoryMetadata.cpp | 5 ----- 2 files changed, 12 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 8ee9561466a..3a4559f94dc 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -20,13 +20,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int COLUMN_QUERIED_MORE_THAN_ONCE; - extern const int DUPLICATE_COLUMN; - extern const int EMPTY_LIST_OF_COLUMNS_PASSED; - extern const int EMPTY_LIST_OF_COLUMNS_QUERIED; - extern const int NO_SUCH_COLUMN_IN_TABLE; - extern const int NOT_FOUND_COLUMN_IN_BLOCK; - extern const int TYPE_MISMATCH; extern const int TABLE_IS_DROPPED; extern const int NOT_IMPLEMENTED; extern const int DEADLOCK_AVOIDED; diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index a394e196eac..81d1f387424 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -10,17 +10,12 @@ namespace DB { namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int COLUMN_QUERIED_MORE_THAN_ONCE; extern const int DUPLICATE_COLUMN; - extern const int EMPTY_LIST_OF_COLUMNS_PASSED; extern const int EMPTY_LIST_OF_COLUMNS_QUERIED; extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int NOT_FOUND_COLUMN_IN_BLOCK; extern const int TYPE_MISMATCH; - extern const int TABLE_IS_DROPPED; - extern const int NOT_IMPLEMENTED; - extern const int DEADLOCK_AVOIDED; } From c8a58299ac20beb96f17dabb340524bb40dfb789 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 14:42:48 +0300 Subject: [PATCH 093/211] Fix storage buffer metadata --- src/Storages/StorageBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 065dfaa27bf..88619f5bc42 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -179,7 +179,7 @@ Pipes StorageBuffer::read( if (dst_has_same_structure) { if (query_info.order_optimizer) - query_info.input_order_info = query_info.order_optimizer->getInputOrder(destination, metadata_snapshot); + query_info.input_order_info = query_info.order_optimizer->getInputOrder(destination, destination_metadata_snapshot); /// The destination table has the same structure of the requested columns and we can simply read blocks from there. pipes_from_dst = destination->read( From d79982f4973ccf8f0a9cc6198f23ec1f150494d6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 19:10:47 +0300 Subject: [PATCH 094/211] Better locks in Storages --- src/Core/iostream_debug_helpers.cpp | 2 +- src/Core/iostream_debug_helpers.h | 3 - src/DataStreams/IBlockInputStream.h | 4 +- src/DataStreams/IBlockOutputStream.h | 4 +- .../PushingToViewsBlockOutputStream.cpp | 5 +- src/Databases/DatabaseMySQL.cpp | 2 +- src/Databases/DatabaseOnDisk.cpp | 2 +- src/Databases/IDatabase.h | 1 - src/Functions/FunctionJoinGet.cpp | 3 +- src/Functions/FunctionJoinGet.h | 4 +- src/Interpreters/InterpreterAlterQuery.cpp | 10 +-- src/Interpreters/InterpreterCreateQuery.cpp | 5 +- src/Interpreters/InterpreterDescribeQuery.cpp | 3 +- src/Interpreters/InterpreterDropQuery.cpp | 4 +- src/Interpreters/InterpreterInsertQuery.cpp | 3 +- src/Interpreters/InterpreterSelectQuery.cpp | 5 +- src/Interpreters/InterpreterSelectQuery.h | 2 +- src/Interpreters/MutationsInterpreter.cpp | 4 +- src/Interpreters/MutationsInterpreter.h | 4 +- src/Processors/Pipe.h | 6 +- src/Processors/QueryPipeline.h | 9 ++- src/Storages/IStorage.cpp | 37 +++------- src/Storages/IStorage.h | 39 +++-------- src/Storages/LiveView/StorageLiveView.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 6 +- src/Storages/MergeTree/MergeTreeData.h | 10 +-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 6 +- .../MergeTree/MergeTreeDataMergerMutator.h | 4 +- .../ReplicatedMergeTreeCleanupThread.cpp | 4 +- .../ReplicatedMergeTreePartCheckThread.cpp | 3 +- src/Storages/ReadInOrderOptimizer.cpp | 4 +- src/Storages/StorageBuffer.cpp | 7 +- src/Storages/StorageBuffer.h | 2 +- src/Storages/StorageDistributed.cpp | 5 +- src/Storages/StorageDistributed.h | 4 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageFile.h | 2 +- src/Storages/StorageJoin.cpp | 2 +- src/Storages/StorageJoin.h | 2 +- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageLog.h | 2 +- src/Storages/StorageMaterializedView.cpp | 11 ++- src/Storages/StorageMaterializedView.h | 4 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMemory.h | 2 +- src/Storages/StorageMerge.cpp | 7 +- src/Storages/StorageMerge.h | 4 +- src/Storages/StorageMergeTree.cpp | 42 ++++------- src/Storages/StorageMergeTree.h | 4 +- src/Storages/StorageNull.cpp | 4 +- src/Storages/StorageNull.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 69 ++++++++----------- src/Storages/StorageReplicatedMergeTree.h | 6 +- src/Storages/StorageSet.cpp | 2 +- src/Storages/StorageSet.h | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- src/Storages/StorageStripeLog.h | 2 +- src/Storages/StorageTinyLog.cpp | 2 +- src/Storages/StorageTinyLog.h | 2 +- src/Storages/System/StorageSystemColumns.cpp | 4 +- .../System/StorageSystemPartsBase.cpp | 2 +- src/Storages/System/StorageSystemPartsBase.h | 2 +- src/Storages/System/StorageSystemTables.cpp | 5 +- src/Storages/TableStructureLockHolder.h | 37 ++-------- 64 files changed, 162 insertions(+), 287 deletions(-) diff --git a/src/Core/iostream_debug_helpers.cpp b/src/Core/iostream_debug_helpers.cpp index 8683bb14db6..08477770c53 100644 --- a/src/Core/iostream_debug_helpers.cpp +++ b/src/Core/iostream_debug_helpers.cpp @@ -53,7 +53,7 @@ std::ostream & operator<<(std::ostream & stream, const IStorage & what) return stream; } -std::ostream & operator<<(std::ostream & stream, const TableStructureReadLock &) +std::ostream & operator<<(std::ostream & stream, const TableLockHolder &) { stream << "TableStructureReadLock()"; return stream; diff --git a/src/Core/iostream_debug_helpers.h b/src/Core/iostream_debug_helpers.h index b9e5efa5d95..8abffd4fe58 100644 --- a/src/Core/iostream_debug_helpers.h +++ b/src/Core/iostream_debug_helpers.h @@ -22,9 +22,6 @@ std::ostream & operator<<(std::ostream & stream, const IDataType & what); class IStorage; std::ostream & operator<<(std::ostream & stream, const IStorage & what); -class TableStructureReadLock; -std::ostream & operator<<(std::ostream & stream, const TableStructureReadLock & what); - class IFunctionOverloadResolver; std::ostream & operator<<(std::ostream & stream, const IFunctionOverloadResolver & what); diff --git a/src/DataStreams/IBlockInputStream.h b/src/DataStreams/IBlockInputStream.h index 66f3e68d601..68850a822e8 100644 --- a/src/DataStreams/IBlockInputStream.h +++ b/src/DataStreams/IBlockInputStream.h @@ -109,7 +109,7 @@ public: size_t checkDepth(size_t max_depth) const { return checkDepthImpl(max_depth, max_depth); } /// Do not allow to change the table while the blocks stream and its children are alive. - void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); } + void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } /// Get information about execution speed. const BlockStreamProfileInfo & getProfileInfo() const { return info; } @@ -229,7 +229,7 @@ public: protected: /// Order is important: `table_locks` must be destroyed after `children` so that tables from /// which child streams read are protected by the locks during the lifetime of the child streams. - std::vector table_locks; + std::vector table_locks; BlockInputStreams children; std::shared_mutex children_mutex; diff --git a/src/DataStreams/IBlockOutputStream.h b/src/DataStreams/IBlockOutputStream.h index 060438ba457..bb62d0183f9 100644 --- a/src/DataStreams/IBlockOutputStream.h +++ b/src/DataStreams/IBlockOutputStream.h @@ -61,10 +61,10 @@ public: /** Don't let to alter table while instance of stream is alive. */ - void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); } + void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } private: - std::vector table_locks; + std::vector table_locks; }; } diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 72de6b889f1..2d2d678bff6 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -33,7 +33,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( * but it's clear that here is not the best place for this functionality. */ addTableLock( - storage->lockStructureForShare(true, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout)); + storage->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout)); /// If the "root" table deduplactes blocks, there are no need to make deduplication for children /// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks @@ -74,8 +74,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( if (auto * materialized_view = dynamic_cast(dependent_table.get())) { addTableLock( - materialized_view->lockStructureForShare( - true, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout)); + materialized_view->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout)); StoragePtr inner_table = materialized_view->getTargetTable(); auto inner_table_id = inner_table->getStorageID(); diff --git a/src/Databases/DatabaseMySQL.cpp b/src/Databases/DatabaseMySQL.cpp index a73fbafb7f5..6e5837257f0 100644 --- a/src/Databases/DatabaseMySQL.cpp +++ b/src/Databases/DatabaseMySQL.cpp @@ -362,7 +362,7 @@ void DatabaseMySQL::cleanOutdatedTables() ++iterator; else { - const auto table_lock = (*iterator)->lockAlterIntention(RWLockImpl::NO_QUERY, lock_acquire_timeout); + const auto table_lock = (*iterator)->lockExclusively(RWLockImpl::NO_QUERY, lock_acquire_timeout); (*iterator)->shutdown(); (*iterator)->is_dropped = true; diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 364c9d50c48..0a16b6eacff 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -266,7 +266,7 @@ void DatabaseOnDisk::renameTable( } auto table_data_relative_path = getTableDataPath(table_name); - TableStructureWriteLockHolder table_lock; + TableExclusiveLockHolder table_lock; String table_metadata_path; ASTPtr attach_query; /// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case. diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index b9a7a907f73..3d8d5c74ceb 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -22,7 +22,6 @@ class Context; struct Settings; struct ConstraintsDescription; struct IndicesDescription; -struct TableStructureWriteLockHolder; class ASTCreateQuery; using Dictionaries = std::vector; diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index a4569684e7f..a33b70684a5 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -67,8 +67,7 @@ FunctionBaseImplPtr JoinGetOverloadResolver::build(const ColumnsWithTyp auto join = storage_join->getJoin(); DataTypes data_types(arguments.size()); - auto table_lock = storage_join->lockStructureForShare( - false, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto table_lock = storage_join->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); for (size_t i = 0; i < arguments.size(); ++i) data_types[i] = arguments[i].type; diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h index f233ccd8a4f..af95686c207 100644 --- a/src/Functions/FunctionJoinGet.h +++ b/src/Functions/FunctionJoinGet.h @@ -37,7 +37,7 @@ class FunctionJoinGet final : public IFunctionBaseImpl public: static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet"; - FunctionJoinGet(TableStructureReadLockHolder table_lock_, StoragePtr storage_join_, + FunctionJoinGet(TableLockHolder table_lock_, StoragePtr storage_join_, HashJoinPtr join_, String attr_name_, DataTypes argument_types_, DataTypePtr return_type_) : table_lock(std::move(table_lock_)) @@ -57,7 +57,7 @@ public: ExecutableFunctionImplPtr prepare(const Block & sample_block, const ColumnNumbers & arguments, size_t result) const override; private: - TableStructureReadLockHolder table_lock; + TableLockHolder table_lock; StoragePtr storage_join; HashJoinPtr join; const String attr_name; diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 958291d5882..61277b8160c 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -43,6 +43,7 @@ BlockIO InterpreterAlterQuery::execute() context.checkAccess(getRequiredAccess()); auto table_id = context.resolveStorageID(alter, Context::ResolveOrdinary); StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); + auto alter_lock = table->lockForAlter(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = table->getInMemoryMetadataPtr(); /// Add default database to table identifiers that we can encounter in e.g. default expressions, @@ -83,10 +84,7 @@ BlockIO InterpreterAlterQuery::execute() if (!mutation_commands.empty()) { - auto table_lock_holder = table->lockStructureForShare( - false /* because mutation is executed asyncronously */, - context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - MutationsInterpreter(table, metadata_snapshot, mutation_commands, context, false).validate(table_lock_holder); + MutationsInterpreter(table, metadata_snapshot, mutation_commands, context, false).validate(); table->mutate(mutation_commands, context); } @@ -112,13 +110,11 @@ BlockIO InterpreterAlterQuery::execute() if (!alter_commands.empty()) { - auto table_lock_holder = table->lockAlterIntention( - context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); StorageInMemoryMetadata metadata = table->getInMemoryMetadata(); alter_commands.validate(metadata, context); alter_commands.prepare(metadata); table->checkAlterIsPossible(alter_commands, context.getSettingsRef()); - table->alter(alter_commands, context, table_lock_holder); + table->alter(alter_commands, context, alter_lock); } return {}; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index bb82c94a764..3e09d728c4c 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -405,7 +405,7 @@ ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(const A InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(ASTCreateQuery & create) const { TableProperties properties; - TableStructureReadLockHolder as_storage_lock; + TableLockHolder as_storage_lock; if (create.columns_list) { @@ -428,8 +428,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS StoragePtr as_storage = DatabaseCatalog::instance().getTable({as_database_name, create.as_table}, context); /// as_storage->getColumns() and setEngine(...) must be called under structure lock of other_table for CREATE ... AS other_table. - as_storage_lock = as_storage->lockStructureForShare( - false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + as_storage_lock = as_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto as_storage_metadata = as_storage->getInMemoryMetadataPtr(); properties.columns = as_storage_metadata->getColumns(); diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 535a4280b45..94fa748ea15 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -89,8 +89,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() table = DatabaseCatalog::instance().getTable(table_id, context); } - auto table_lock = table->lockStructureForShare( - false, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto table_lock = table->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = table->getInMemoryMetadataPtr(); columns = metadata_snapshot->getColumns(); } diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 15f19b585de..e6853a8af4c 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -93,7 +93,7 @@ BlockIO InterpreterDropQuery::executeToTable( { context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id); table->shutdown(); - TableStructureWriteLockHolder table_lock; + TableExclusiveLockHolder table_lock; if (database->getEngineName() != "Atomic") table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); /// Drop table from memory, don't touch data and metadata @@ -116,7 +116,7 @@ BlockIO InterpreterDropQuery::executeToTable( table->shutdown(); - TableStructureWriteLockHolder table_lock; + TableExclusiveLockHolder table_lock; if (database->getEngineName() != "Atomic") table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index a39e8961970..554907d37d3 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -118,8 +118,7 @@ BlockIO InterpreterInsertQuery::execute() BlockIO res; StoragePtr table = getTable(query); - auto table_lock = table->lockStructureForShare( - true, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto table_lock = table->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = table->getInMemoryMetadataPtr(); auto query_sample_block = getSampleBlock(query, table, metadata_snapshot); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d832bcb7dc0..e0d5adf92b8 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -255,8 +255,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (storage) { - table_lock = storage->lockStructureForShare( - false, context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); + table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); table_id = storage->getStorageID(); if (metadata_snapshot == nullptr) metadata_snapshot = storage->getInMemoryMetadataPtr(); @@ -277,7 +276,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( { /// Rewritten with subquery. Free storage locks here. storage = {}; - table_lock.release(); + table_lock.reset(); table_id = StorageID::createEmpty(); } } diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 2f0faa2ba72..e274175eb30 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -185,7 +185,7 @@ private: /// Table from where to read data, if not subquery. StoragePtr storage; StorageID table_id = StorageID::createEmpty(); /// Will be initialized if storage is not nullptr - TableStructureReadLockHolder table_lock; + TableLockHolder table_lock; /// Used when we read from prepared input, not table or subquery. BlockInputStreamPtr input; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 3ad813a15b7..51b0cf92484 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -671,7 +671,7 @@ BlockInputStreamPtr MutationsInterpreter::addStreamsForLaterStages(const std::ve return in; } -void MutationsInterpreter::validate(TableStructureReadLockHolder &) +void MutationsInterpreter::validate() { const Settings & settings = context.getSettingsRef(); @@ -696,7 +696,7 @@ void MutationsInterpreter::validate(TableStructureReadLockHolder &) addStreamsForLaterStages(stages, in)->getHeader(); } -BlockInputStreamPtr MutationsInterpreter::execute(TableStructureReadLockHolder &) +BlockInputStreamPtr MutationsInterpreter::execute() { if (!can_execute) throw Exception("Cannot execute mutations interpreter because can_execute flag set to false", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 3c3175c1856..894d135a099 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -32,12 +32,12 @@ public: const Context & context_, bool can_execute_); - void validate(TableStructureReadLockHolder & table_lock_holder); + void validate(); size_t evaluateCommandsSize(); /// The resulting stream will return blocks containing only changed columns and columns, that we need to recalculate indices. - BlockInputStreamPtr execute(TableStructureReadLockHolder & table_lock_holder); + BlockInputStreamPtr execute(); /// Only changed columns. const Block & getUpdatedHeader() const; diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index ec5514915a7..085016c3588 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -62,12 +62,12 @@ public: /// Do not allow to change the table while the processors of pipe are alive. /// TODO: move it to pipeline. - void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); } + void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } /// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible. void addInterpreterContext(std::shared_ptr context) { interpreter_context.emplace_back(std::move(context)); } void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); } - const std::vector & getTableLocks() const { return table_locks; } + const std::vector & getTableLocks() const { return table_locks; } const std::vector> & getContexts() const { return interpreter_context; } const std::vector & getStorageHolders() const { return storage_holders; } @@ -80,7 +80,7 @@ private: /// It is the max number of processors which can be executed in parallel for each step. See QueryPipeline::Streams. size_t max_parallel_streams = 0; - std::vector table_locks; + std::vector table_locks; /// Some processors may implicitly use Context or temporary Storage created by Interpreter. /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 129b7f5ae3c..9f632a7180a 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -7,14 +7,13 @@ #include #include +#include namespace DB { -class TableStructureReadLock; -using TableStructureReadLockPtr = std::shared_ptr; -using TableStructureReadLocks = std::vector; +using TableLockHolders = std::vector; class Context; class IOutputFormat; @@ -146,7 +145,7 @@ public: const Block & getHeader() const { return current_header; } - void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); } + void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } void addInterpreterContext(std::shared_ptr context) { interpreter_context.emplace_back(std::move(context)); } void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); } @@ -180,7 +179,7 @@ private: /// because QueryPipeline is alive until query is finished. std::vector> interpreter_context; std::vector storage_holders; - TableStructureReadLocks table_locks; + TableLockHolders table_locks; /// Common header for each stream. Block current_header; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 3a4559f94dc..42224ec01ac 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -47,58 +47,43 @@ RWLockImpl::LockHolder IStorage::tryLockTimed( return lock_holder; } -TableStructureReadLockHolder IStorage::lockStructureForShare(bool will_add_new_data, const String & query_id, const SettingSeconds & acquire_timeout) +TableLockHolder IStorage::lockForShare(const String & query_id, const SettingSeconds & acquire_timeout) { - TableStructureReadLockHolder result; - if (will_add_new_data) - result.new_data_structure_lock = tryLockTimed(new_data_structure_lock, RWLockImpl::Read, query_id, acquire_timeout); - result.structure_lock = tryLockTimed(structure_lock, RWLockImpl::Read, query_id, acquire_timeout); + TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); + return result; } -TableStructureWriteLockHolder IStorage::lockAlterIntention(const String & query_id, const SettingSeconds & acquire_timeout) +TableLockHolder IStorage::lockForAlter(const String & query_id, const SettingSeconds & acquire_timeout) { - TableStructureWriteLockHolder result; - result.alter_intention_lock = tryLockTimed(alter_intention_lock, RWLockImpl::Write, query_id, acquire_timeout); + TableLockHolder result = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); + return result; } -void IStorage::lockStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id, const SettingSeconds & acquire_timeout) -{ - if (!lock_holder.alter_intention_lock) - throw Exception("Alter intention lock for table " + getStorageID().getNameForLogs() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR); - if (!lock_holder.new_data_structure_lock) - lock_holder.new_data_structure_lock = tryLockTimed(new_data_structure_lock, RWLockImpl::Write, query_id, acquire_timeout); - lock_holder.structure_lock = tryLockTimed(structure_lock, RWLockImpl::Write, query_id, acquire_timeout); -} - -TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id, const SettingSeconds & acquire_timeout) +TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, const SettingSeconds & acquire_timeout) { - TableStructureWriteLockHolder result; - result.alter_intention_lock = tryLockTimed(alter_intention_lock, RWLockImpl::Write, query_id, acquire_timeout); + TableExclusiveLockHolder result; + result.alter_lock = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); - result.new_data_structure_lock = tryLockTimed(new_data_structure_lock, RWLockImpl::Write, query_id, acquire_timeout); - result.structure_lock = tryLockTimed(structure_lock, RWLockImpl::Write, query_id, acquire_timeout); + result.drop_lock = tryLockTimed(drop_lock, RWLockImpl::Write, query_id, acquire_timeout); return result; } void IStorage::alter( - const AlterCommands & params, - const Context & context, - TableStructureWriteLockHolder & table_lock_holder) + const AlterCommands & params, const Context & context, TableLockHolder &) { - lockStructureExclusively(table_lock_holder, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto table_id = getStorageID(); StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); params.apply(new_metadata, context); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 375ab90aee4..1309b727a74 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -135,7 +135,7 @@ public: using ColumnSizeByName = std::unordered_map; virtual ColumnSizeByName getColumnSizes() const { return {}; } -public: /// thread-unsafe part. lockStructure must be acquired +public: StorageInMemoryMetadata getInMemoryMetadata() const { return *metadata.get(); } StorageMetadataPtr getInMemoryMetadataPtr() const { return metadata.get(); } @@ -174,21 +174,11 @@ private: const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const SettingSeconds & acquire_timeout) const; public: - /// Acquire this lock if you need the table structure to remain constant during the execution of - /// the query. If will_add_new_data is true, this means that the query will add new data to the table - /// (INSERT or a parts merge). - TableStructureReadLockHolder lockStructureForShare(bool will_add_new_data, const String & query_id, const SettingSeconds & acquire_timeout); + TableLockHolder lockForShare(const String & query_id, const SettingSeconds & acquire_timeout); - /// Acquire this lock at the start of ALTER to lock out other ALTERs and make sure that only you - /// can modify the table structure. It can later be upgraded to the exclusive lock. - TableStructureWriteLockHolder lockAlterIntention(const String & query_id, const SettingSeconds & acquire_timeout); + TableLockHolder lockForAlter(const String & query_id, const SettingSeconds & acquire_timeout); - /// Upgrade alter intention lock to the full exclusive structure lock. This is done by ALTER queries - /// to ensure that no other query uses the table structure and it can be safely changed. - void lockStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id, const SettingSeconds & acquire_timeout); - - /// Acquire the full exclusive lock immediately. No other queries can run concurrently. - TableStructureWriteLockHolder lockExclusively(const String & query_id, const SettingSeconds & acquire_timeout); + TableExclusiveLockHolder lockExclusively(const String & query_id, const SettingSeconds & acquire_timeout); /** Returns stage to which query is going to be processed in read() function. * (Normally, the function only reads the columns from the list, but in other cases, @@ -297,7 +287,7 @@ public: const ASTPtr & /*query*/, const StorageMetadataPtr & /* metadata_snapshot */, const Context & /* context */, - TableStructureWriteLockHolder &) + TableExclusiveLockHolder &) { throw Exception("Truncate is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } @@ -323,7 +313,7 @@ public: * This method must fully execute the ALTER query, taking care of the locks itself. * To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata-> */ - virtual void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder); + virtual void alter(const AlterCommands & params, const Context & context, TableLockHolder & alter_lock_holder); /** Checks that alter commands can be applied to storage. For example, columns can be modified, * or primary key can be changes, etc. @@ -441,22 +431,9 @@ public: } private: - /// You always need to take the next three locks in this order. + mutable RWLock alter_lock = RWLockImpl::create(); - /// If you hold this lock exclusively, you can be sure that no other structure modifying queries - /// (e.g. ALTER, DROP) are concurrently executing. But queries that only read table structure - /// (e.g. SELECT, INSERT) can continue to execute. - mutable RWLock alter_intention_lock = RWLockImpl::create(); - - /// It is taken for share for the entire INSERT query and the entire merge of the parts (for MergeTree). - /// ALTER COLUMN queries acquire an exclusive lock to ensure that no new parts with the old structure - /// are added to the table and thus the set of parts to modify doesn't change. - mutable RWLock new_data_structure_lock = RWLockImpl::create(); - - /// Lock for the table column structure (names, types, etc.) and data path. - /// It is taken in exclusive mode by queries that modify them (e.g. RENAME, ALTER and DROP) - /// and in share mode by other queries. - mutable RWLock structure_lock = RWLockImpl::create(); + mutable RWLock drop_lock = RWLockImpl::create(); }; } diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index ac6bd48f534..0abb01d7dc7 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -514,7 +514,7 @@ void StorageLiveView::drop() void StorageLiveView::refresh(const Context & context) { - auto alter_lock = lockAlterIntention(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto alter_lock = lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); { std::lock_guard lock(mutex); if (getNewBlocks()) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 115e0b78bf0..0ed4e98e864 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1436,7 +1436,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createPart( void MergeTreeData::changeSettings( const ASTPtr & new_settings, - TableStructureWriteLockHolder & /* table_lock_holder */) + TableLockHolder & /* table_lock_holder */) { if (new_settings) { @@ -1481,7 +1481,7 @@ void MergeTreeData::changeSettings( } } -void MergeTreeData::freezeAll(const String & with_name, const Context & context, TableStructureReadLockHolder &) +void MergeTreeData::freezeAll(const String & with_name, const Context & context, TableLockHolder &) { freezePartitionsByMatcher([] (const DataPartPtr &){ return true; }, with_name, context); } @@ -2289,7 +2289,7 @@ void MergeTreeData::removePartContributionToColumnSizes(const DataPartPtr & part } -void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String & with_name, const Context & context, TableStructureReadLockHolder &) +void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String & with_name, const Context & context, TableLockHolder &) { std::optional prefix; String partition_id; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index af6bee4936c..ca6928cbb01 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -477,7 +477,7 @@ public: /// Delete all directories which names begin with "tmp" /// Set non-negative parameter value to override MergeTreeSettings temporary_directories_lifetime - /// Must be called with locked lockStructureForShare(). + /// Must be called with locked lockForShare(). void clearOldTemporaryDirectories(ssize_t custom_directories_lifetime_seconds = -1); /// After the call to dropAllData() no method can be called. @@ -489,7 +489,7 @@ public: /// Moves the entire data directory. /// Flushes the uncompressed blocks cache and the marks cache. - /// Must be called with locked lockStructureForAlter(). + /// Must be called with locked lockForShare(). void rename(const String & new_table_path, const StorageID & new_table_id) override; /// Check if the ALTER can be performed: @@ -502,10 +502,10 @@ public: /// Change MergeTreeSettings void changeSettings( const ASTPtr & new_settings, - TableStructureWriteLockHolder & table_lock_holder); + TableLockHolder & table_lock_holder); /// Freezes all parts. - void freezeAll(const String & with_name, const Context & context, TableStructureReadLockHolder & table_lock_holder); + void freezeAll(const String & with_name, const Context & context, TableLockHolder & table_lock_holder); /// Should be called if part data is suspected to be corrupted. void reportBrokenPart(const String & name) const @@ -527,7 +527,7 @@ public: * Backup is created in directory clickhouse_dir/shadow/i/, where i - incremental number, * or if 'with_name' is specified - backup is created in directory with specified name. */ - void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context, TableStructureReadLockHolder & table_lock_holder); + void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context, TableLockHolder & table_lock_holder); public: diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 732c97c6dcc..d52154002fe 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -581,7 +581,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor const FutureMergedMutatedPart & future_part, const StorageMetadataPtr & metadata_snapshot, MergeList::Entry & merge_entry, - TableStructureReadLockHolder &, + TableLockHolder &, time_t time_of_merge, const ReservationPtr & space_reservation, bool deduplicate, @@ -995,7 +995,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor time_t time_of_mutation, const Context & context, const ReservationPtr & space_reservation, - TableStructureReadLockHolder & table_lock_holder) + TableLockHolder &) { checkOperationIsNotCanceled(merge_entry); @@ -1046,7 +1046,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor if (!for_interpreter.empty()) { interpreter.emplace(storage_from_source_part, metadata_snapshot, for_interpreter, context_for_reading, true); - in = interpreter->execute(table_lock_holder); + in = interpreter->execute(); updated_header = interpreter->getUpdatedHeader(); in->setProgressCallback(MergeProgressCallback(merge_entry, watch_prev_elapsed, stage_progress)); } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 7828f79ea33..d62587bef5f 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -107,7 +107,7 @@ public: const FutureMergedMutatedPart & future_part, const StorageMetadataPtr & metadata_snapshot, MergeListEntry & merge_entry, - TableStructureReadLockHolder & table_lock_holder, + TableLockHolder & table_lock_holder, time_t time_of_merge, const ReservationPtr & space_reservation, bool deduplicate, @@ -122,7 +122,7 @@ public: time_t time_of_mutation, const Context & context, const ReservationPtr & space_reservation, - TableStructureReadLockHolder & table_lock_holder); + TableLockHolder & table_lock_holder); MergeTreeData::DataPartPtr renameMergedTemporaryPart( MergeTreeData::MutableDataPartPtr & new_data_part, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 0870c0fdf72..f7fa957e997 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -58,9 +58,7 @@ void ReplicatedMergeTreeCleanupThread::iterate() storage.clearOldPartsAndRemoveFromZK(); { - /// TODO: Implement tryLockStructureForShare. - auto lock = storage.lockStructureForShare( - false, RWLockImpl::NO_QUERY, storage.getSettings()->lock_acquire_timeout_for_background_operations); + auto lock = storage.lockForShare(RWLockImpl::NO_QUERY, storage.getSettings()->lock_acquire_timeout_for_background_operations); storage.clearOldTemporaryDirectories(); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 0d824fa2dd8..75a3c463061 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -201,8 +201,7 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na { auto zookeeper = storage.getZooKeeper(); - auto table_lock = storage.lockStructureForShare( - false, RWLockImpl::NO_QUERY, storage.getSettings()->lock_acquire_timeout_for_background_operations); + auto table_lock = storage.lockForShare(RWLockImpl::NO_QUERY, storage.getSettings()->lock_acquire_timeout_for_background_operations); auto local_part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksums( part->getColumns(), part->checksums); diff --git a/src/Storages/ReadInOrderOptimizer.cpp b/src/Storages/ReadInOrderOptimizer.cpp index a6cc6211788..bc220bc33ce 100644 --- a/src/Storages/ReadInOrderOptimizer.cpp +++ b/src/Storages/ReadInOrderOptimizer.cpp @@ -33,13 +33,13 @@ ReadInOrderOptimizer::ReadInOrderOptimizer( InputOrderInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot) const { Names sorting_key_columns; - if (const auto * merge_tree = dynamic_cast(storage.get())) + if (dynamic_cast(storage.get())) { if (!metadata_snapshot->hasSortingKey()) return {}; sorting_key_columns = metadata_snapshot->getSortingKeyColumns(); } - else if (const auto * part = dynamic_cast(storage.get())) + else if (dynamic_cast(storage.get())) { if (!metadata_snapshot->hasSortingKey()) return {}; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 88619f5bc42..b4d6b66ebe7 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -163,8 +163,7 @@ Pipes StorageBuffer::read( if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); - auto destination_lock = destination->lockStructureForShare( - false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto destination_lock = destination->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto destination_metadata_snapshot = destination->getInMemoryMetadataPtr(); @@ -804,10 +803,8 @@ std::optional StorageBuffer::totalBytes() const return bytes; } -void StorageBuffer::alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) +void StorageBuffer::alter(const AlterCommands & params, const Context & context, TableLockHolder &) { - lockStructureExclusively(table_lock_holder, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - auto table_id = getStorageID(); checkAlterIsPossible(params, context.getSettingsRef()); auto metadata_snapshot = getInMemoryMetadataPtr(); diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index ceedbd25a0c..e168f79293e 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -89,7 +89,7 @@ public: void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override; /// The structure of the subordinate table is not checked and does not change. - void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override; std::optional totalRows() const override; std::optional totalBytes() const override; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 2e07a393b04..9c20e3f8e11 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -564,9 +564,8 @@ void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, co } } -void StorageDistributed::alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) +void StorageDistributed::alter(const AlterCommands & params, const Context & context, TableLockHolder &) { - lockStructureExclusively(table_lock_holder, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto table_id = getStorageID(); checkAlterIsPossible(params, context.getSettingsRef()); @@ -619,7 +618,7 @@ Strings StorageDistributed::getDataPaths() const return paths; } -void StorageDistributed::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) +void StorageDistributed::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) { std::lock_guard lock(cluster_nodes_mutex); diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index c952ccde8ac..006f2bb580a 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -82,7 +82,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; /// Removes temporary data in local filesystem. - void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; void renameOnDisk(const String & new_path_to_table_data); @@ -91,7 +91,7 @@ public: /// in the sub-tables, you need to manually add and delete columns /// the structure of the sub-table is not checked - void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override; void startup() override; void shutdown() override; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 8fb09d0a41e..4867a0bc215 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -527,7 +527,7 @@ void StorageFile::truncate( const ASTPtr & /*query*/, const StorageMetadataPtr & /* metadata_snapshot */, const Context & /* context */, - TableStructureWriteLockHolder &) + TableExclusiveLockHolder &) { if (paths.size() != 1) throw Exception("Can't truncate table '" + getStorageID().getNameForLogs() + "' in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 05b4d7aea8a..2c4a43eb979 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -42,7 +42,7 @@ public: const ASTPtr & /*query*/, const StorageMetadataPtr & /* metadata_snapshot */, const Context & /* context */, - TableStructureWriteLockHolder &) override; + TableExclusiveLockHolder &) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 5000dcd8b18..21e4370c28b 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -65,7 +65,7 @@ StorageJoin::StorageJoin( void StorageJoin::truncate( - const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) + const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder&) { Poco::File(path).remove(true); Poco::File(path).createDirectories(); diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 4d4d1a81da2..fb8ffc1c353 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -27,7 +27,7 @@ class StorageJoin final : public ext::shared_ptr_helper, public Sto public: String getName() const override { return "Join"; } - void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; /// Access the innards. HashJoinPtr & getJoin() { return join; } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 45d55938db3..542fb507d83 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -535,7 +535,7 @@ void StorageLog::rename(const String & new_path_to_table_data, const StorageID & renameInMemory(new_table_id); } -void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) +void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) { std::shared_lock lock(rwlock); diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 670e2777d44..d020f906609 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -39,7 +39,7 @@ public: CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override; - void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 2c0d5727b31..976b3c80dec 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -116,8 +116,7 @@ Pipes StorageMaterializedView::read( const unsigned num_streams) { auto storage = getTargetTable(); - auto lock = storage->lockStructureForShare( - false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto lock = storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = storage->getInMemoryMetadataPtr(); if (query_info.order_optimizer) @@ -134,8 +133,7 @@ Pipes StorageMaterializedView::read( BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) { auto storage = getTargetTable(); - auto lock = storage->lockStructureForShare( - true, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto lock = storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = storage->getInMemoryMetadataPtr(); auto stream = storage->write(query, metadata_snapshot, context); @@ -173,7 +171,7 @@ void StorageMaterializedView::drop() executeDropQuery(ASTDropQuery::Kind::Drop, global_context, target_table_id); } -void StorageMaterializedView::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) +void StorageMaterializedView::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) { if (has_inner_table) executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, target_table_id); @@ -204,9 +202,8 @@ bool StorageMaterializedView::optimize( void StorageMaterializedView::alter( const AlterCommands & params, const Context & context, - TableStructureWriteLockHolder & table_lock_holder) + TableLockHolder &) { - lockStructureExclusively(table_lock_holder, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto table_id = getStorageID(); StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); StorageInMemoryMetadata old_metadata = getInMemoryMetadata(); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index e2111a15f5c..f45d9203bad 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -37,7 +37,7 @@ public: void drop() override; - void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override; bool optimize( const ASTPtr & query, @@ -47,7 +47,7 @@ public: bool deduplicate, const Context & context) override; - void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override; void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override; diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 3bae29ac96c..05b37ecf32e 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -149,7 +149,7 @@ void StorageMemory::drop() } void StorageMemory::truncate( - const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) + const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) { std::lock_guard lock(mutex); data.clear(); diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 842c7dc3790..851fe7fc70a 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -41,7 +41,7 @@ public: void drop() override; - void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override; std::optional totalRows() const override; std::optional totalBytes() const override; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 235f78505e0..f4030ed573f 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -333,7 +333,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const String const auto & table = iterator->table(); if (table && table.get() != this) selected_tables.emplace_back( - table, table->lockStructureForShare(false, query_id, settings.lock_acquire_timeout), iterator->name()); + table, table->lockForShare(query_id, settings.lock_acquire_timeout), iterator->name()); iterator->next(); } @@ -362,7 +362,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( if (storage.get() != this) { selected_tables.emplace_back( - storage, storage->lockStructureForShare(false, query_id, settings.lock_acquire_timeout), iterator->name()); + storage, storage->lockForShare(query_id, settings.lock_acquire_timeout), iterator->name()); virtual_column->insert(iterator->name()); } @@ -405,9 +405,8 @@ void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, const Se } void StorageMerge::alter( - const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) + const AlterCommands & params, const Context & context, TableLockHolder &) { - lockStructureExclusively(table_lock_holder, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto table_id = getStorageID(); StorageInMemoryMetadata storage_metadata = getInMemoryMetadata(); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 1ad22869e39..f2af25d3f3d 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -42,7 +42,7 @@ public: /// you need to add and remove columns in the sub-tables manually /// the structure of sub-tables is not checked - void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override; bool mayBenefitFromIndexForIn( const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & metadata_snapshot) const override; @@ -52,7 +52,7 @@ private: OptimizedRegularExpression table_name_regexp; Context global_context; - using StorageWithLockAndName = std::tuple; + using StorageWithLockAndName = std::tuple; using StorageListWithLocks = std::list; StorageListWithLocks getSelectedTables(const String & query_id, const Settings & settings) const; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 45f8ecf0ef9..9042afe1b2d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -231,7 +231,7 @@ void StorageMergeTree::drop() dropAllData(); } -void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) +void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) { { /// Asks to complete merges and does not allow them to start. @@ -254,7 +254,7 @@ void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, cons void StorageMergeTree::alter( const AlterCommands & commands, const Context & context, - TableStructureWriteLockHolder & table_lock_holder) + TableLockHolder & table_lock_holder) { auto table_id = getStorageID(); @@ -268,8 +268,6 @@ void StorageMergeTree::alter( /// This alter can be performed at new_metadata level only if (commands.isSettingsAlter()) { - lockStructureExclusively(table_lock_holder, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - changeSettings(new_metadata.settings_changes, table_lock_holder); DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata); @@ -277,10 +275,6 @@ void StorageMergeTree::alter( else { { - /// TODO (relax this lock and remove this action lock) - auto merges_block = getActionLock(ActionLocks::PartsMerge); - lockStructureExclusively(table_lock_holder, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - changeSettings(new_metadata.settings_changes, table_lock_holder); /// Reinitialize primary key because primary key column types might have changed. setProperties(new_metadata, old_metadata); @@ -290,9 +284,6 @@ void StorageMergeTree::alter( if (!maybe_mutation_commands.empty()) mutation_version = startMutation(maybe_mutation_commands, mutation_file_name); - /// We release all locks except alter_intention_lock which allows - /// to execute alter queries sequentially - table_lock_holder.releaseAllExceptAlterIntention(); } /// Always execute required mutations synchronously, because alters @@ -591,8 +582,7 @@ bool StorageMergeTree::merge( bool deduplicate, String * out_disable_reason) { - auto table_lock_holder = lockStructureForShare( - true, RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto metadata_snapshot = getInMemoryMetadataPtr(); FutureMergedMutatedPart future_part; @@ -740,8 +730,7 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::movePartsTask() bool StorageMergeTree::tryMutatePart() { - auto table_lock_holder = lockStructureForShare( - true, RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); StorageMetadataPtr metadata_snapshot = getInMemoryMetadataPtr(); size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements; @@ -876,13 +865,8 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() /// Clear old parts. It is unnecessary to do it more than once a second. if (auto lock = time_after_previous_cleanup.compareAndRestartDeferred(1)) { - { - /// TODO: Implement tryLockStructureForShare. - auto lock_structure = lockStructureForShare( - false, RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); - clearOldPartsFromFilesystem(); - clearOldTemporaryDirectories(); - } + clearOldPartsFromFilesystem(); + clearOldTemporaryDirectories(); clearOldMutations(); } @@ -1078,16 +1062,14 @@ void StorageMergeTree::alterPartition( case PartitionCommand::FREEZE_PARTITION: { - auto lock = lockStructureForShare( - false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto lock = lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); freezePartition(command.partition, command.with_name, context, lock); } break; case PartitionCommand::FREEZE_ALL_PARTITIONS: { - auto lock = lockStructureForShare( - false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto lock = lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); freezeAll(command.with_name, context, lock); } break; @@ -1156,8 +1138,8 @@ void StorageMergeTree::attachPartition( void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) { - auto lock1 = lockStructureForShare(false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - auto lock2 = source_table->lockStructureForShare(false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto lock1 = lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto lock2 = source_table->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto source_metadata_snapshot = source_table->getInMemoryMetadataPtr(); auto my_metadata_snapshot = getInMemoryMetadataPtr(); @@ -1229,8 +1211,8 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & context) { - auto lock1 = lockStructureForShare(false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - auto lock2 = dest_table->lockStructureForShare(false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto lock1 = lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto lock2 = dest_table->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto dest_table_storage = std::dynamic_pointer_cast(dest_table); if (!dest_table_storage) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index cf3eccc0c0b..9a45fd285dc 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -75,9 +75,9 @@ public: CancellationCode killMutation(const String & mutation_id) override; void drop() override; - void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override; - void alter(const AlterCommands & commands, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + void alter(const AlterCommands & commands, const Context & context, TableLockHolder & table_lock_holder) override; void checkTableCanBeDropped() const override; diff --git a/src/Storages/StorageNull.cpp b/src/Storages/StorageNull.cpp index 7589c4b44dc..499f7329cd9 100644 --- a/src/Storages/StorageNull.cpp +++ b/src/Storages/StorageNull.cpp @@ -45,10 +45,8 @@ void StorageNull::checkAlterIsPossible(const AlterCommands & commands, const Set } -void StorageNull::alter( - const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) +void StorageNull::alter(const AlterCommands & params, const Context & context, TableLockHolder &) { - lockStructureExclusively(table_lock_holder, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto table_id = getStorageID(); StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index 6bd102bdcda..e79174c2565 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -44,7 +44,7 @@ public: void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override; - void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override; std::optional totalRows() const override { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2e64c54112f..b62f6bbd198 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1309,8 +1309,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) ReservationPtr reserved_space = reserveSpacePreferringTTLRules(estimated_space_for_merge, ttl_infos, time(nullptr), max_volume_index); - auto table_lock = lockStructureForShare( - false, RWLockImpl::NO_QUERY, storage_settings_ptr->lock_acquire_timeout_for_background_operations); + auto table_lock = lockForShare(RWLockImpl::NO_QUERY, storage_settings_ptr->lock_acquire_timeout_for_background_operations); StorageMetadataPtr metadata_snapshot = getInMemoryMetadataPtr(); FutureMergedMutatedPart future_merged_part(parts, entry.new_part_type); @@ -1436,8 +1435,8 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM /// Can throw an exception. ReservationPtr reserved_space = reserveSpace(estimated_space_for_result, source_part->volume); - auto table_lock = lockStructureForShare( - false, RWLockImpl::NO_QUERY, storage_settings_ptr->lock_acquire_timeout_for_background_operations); + auto table_lock = lockForShare( + RWLockImpl::NO_QUERY, storage_settings_ptr->lock_acquire_timeout_for_background_operations); StorageMetadataPtr metadata_snapshot = getInMemoryMetadataPtr(); MutableDataPartPtr new_part; @@ -1793,8 +1792,8 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) PartDescriptions parts_to_add; DataPartsVector parts_to_remove; - auto table_lock_holder_dst_table = lockStructureForShare( - false, RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto table_lock_holder_dst_table = lockForShare( + RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto dst_metadata_snapshot = getInMemoryMetadataPtr(); for (size_t i = 0; i < entry_replace.new_part_names.size(); ++i) @@ -1833,7 +1832,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } StoragePtr source_table; - TableStructureReadLockHolder table_lock_holder_src_table; + TableLockHolder table_lock_holder_src_table; StorageID source_table_id{entry_replace.from_database, entry_replace.from_table}; auto clone_data_parts_from_source_table = [&] () -> size_t @@ -1857,11 +1856,11 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) return 0; } - table_lock_holder_src_table = source_table->lockStructureForShare( - false, RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + table_lock_holder_src_table = source_table->lockForShare( + RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); - DataPartStates valid_states{MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, - MergeTreeDataPartState::Outdated}; + DataPartStates valid_states{ + MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}; size_t num_clonable_parts = 0; for (PartDescriptionPtr & part_desc : parts_to_add) @@ -3092,10 +3091,9 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin LOG_DEBUG(log, "Fetching part {} from {}", part_name, source_replica_path); - TableStructureReadLockHolder table_lock_holder; + TableLockHolder table_lock_holder; if (!to_detached) - table_lock_holder = lockStructureForShare( - true, RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); /// Logging Stopwatch stopwatch; @@ -3636,10 +3634,7 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer zookeeper->multi(requests); { - /// TODO (relax this lock and remove this action locks) - auto merges_block = getActionLock(ActionLocks::PartsMerge); - auto fetchers_block = getActionLock(ActionLocks::PartsFetch); - auto table_lock = lockExclusively(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto alter_lock = lockForAlter(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally."); @@ -3658,7 +3653,7 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer void StorageReplicatedMergeTree::alter( - const AlterCommands & params, const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) + const AlterCommands & params, const Context & query_context, TableLockHolder & table_lock_holder) { assertNotReadonly(); @@ -3666,8 +3661,6 @@ void StorageReplicatedMergeTree::alter( if (params.isSettingsAlter()) { - lockStructureExclusively( - table_lock_holder, query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); /// We don't replicate storage_settings_ptr ALTER. It's local operation. /// Also we don't upgrade alter lock to table structure lock. StorageInMemoryMetadata future_metadata = getInMemoryMetadata(); @@ -3732,8 +3725,6 @@ void StorageReplicatedMergeTree::alter( if (ast_to_str(current_metadata->settings_changes) != ast_to_str(future_metadata.settings_changes)) { - lockStructureExclusively( - table_lock_holder, query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); /// Just change settings StorageInMemoryMetadata metadata_copy = *current_metadata; metadata_copy.settings_changes = future_metadata.settings_changes; @@ -3824,7 +3815,7 @@ void StorageReplicatedMergeTree::alter( } - table_lock_holder.release(); + table_lock_holder.reset(); std::vector unwaited; if (query_context.getSettingsRef().replication_alter_partitions_sync == 2) @@ -3908,16 +3899,14 @@ void StorageReplicatedMergeTree::alterPartition( case PartitionCommand::FREEZE_PARTITION: { - auto lock = lockStructureForShare( - false, query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); + auto lock = lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); freezePartition(command.partition, command.with_name, query_context, lock); } break; case PartitionCommand::FREEZE_ALL_PARTITIONS: { - auto lock = lockStructureForShare( - false, query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); + auto lock = lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); freezeAll(command.with_name, query_context, lock); } break; @@ -4012,7 +4001,7 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPt void StorageReplicatedMergeTree::truncate( - const ASTPtr & query, const StorageMetadataPtr &, const Context & query_context, TableStructureWriteLockHolder & table_lock) + const ASTPtr & query, const StorageMetadataPtr &, const Context & query_context, TableExclusiveLockHolder & table_lock) { table_lock.release(); /// Truncate is done asynchronously. @@ -4925,10 +4914,8 @@ CancellationCode StorageReplicatedMergeTree::killMutation(const String & mutatio void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() { - /// Critical section is not required (since grabOldParts() returns unique part set on each call) - - auto table_lock = lockStructureForShare( - false, RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto table_lock = lockForShare( + RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto zookeeper = getZooKeeper(); DataPartsVector parts = grabOldParts(); @@ -5219,8 +5206,8 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ const Context & context) { /// First argument is true, because we possibly will add new data to current table. - auto lock1 = lockStructureForShare(true, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - auto lock2 = source_table->lockStructureForShare(false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto lock1 = lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto lock2 = source_table->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); auto source_metadata_snapshot = source_table->getInMemoryMetadataPtr(); auto metadata_snapshot = getInMemoryMetadataPtr(); @@ -5397,16 +5384,16 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ /// If necessary, wait until the operation is performed on all replicas. if (context.getSettingsRef().replication_alter_partitions_sync > 1) { - lock2.release(); - lock1.release(); + lock2.reset(); + lock1.reset(); waitForAllReplicasToProcessLogEntry(entry); } } void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & query_context) { - auto lock1 = lockStructureForShare(false, query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); - auto lock2 = dest_table->lockStructureForShare(false, query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); + auto lock1 = lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); + auto lock2 = dest_table->lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); auto dest_table_storage = std::dynamic_pointer_cast(dest_table); if (!dest_table_storage) @@ -5583,7 +5570,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta if (query_context.getSettingsRef().replication_alter_partitions_sync > 1) { - lock2.release(); + lock2.reset(); dest_table_storage->waitForAllReplicasToProcessLogEntry(entry); } @@ -5600,7 +5587,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta if (query_context.getSettingsRef().replication_alter_partitions_sync > 1) { - lock1.release(); + lock1.reset(); waitForAllReplicasToProcessLogEntry(entry_delete); } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index c1ba737d849..e340de88749 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -103,7 +103,7 @@ public: bool optimize(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const ASTPtr & partition, bool final, bool deduplicate, const Context & query_context) override; - void alter(const AlterCommands & params, const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, const Context & query_context, TableLockHolder & table_lock_holder) override; void alterPartition( const ASTPtr & query, @@ -120,7 +120,7 @@ public: */ void drop() override; - void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; @@ -315,7 +315,7 @@ private: void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot); /// A part of ALTER: apply metadata changes only (data parts are altered separately). - /// Must be called under IStorage::lockStructureForAlter() lock. + /// Must be called under IStorage::lockForAlter() lock. void setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff); /** Check that the set of parts corresponds to that in ZK (/replicas/me/parts/). diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 93b288569c2..58d5226c91a 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -142,7 +142,7 @@ void StorageSet::finishInsert() { set->finishInsert(); } size_t StorageSet::getSize() const { return set->getTotalRowCount(); } -void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) +void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) { Poco::File(path).remove(true); Poco::File(path).createDirectories(); diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index 2685fa26ba6..de7c65bbc3e 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -67,7 +67,7 @@ public: /// Access the insides. SetPtr & getSet() { return set; } - void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; private: SetPtr set; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 4b95a389f2c..3086e971121 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -326,7 +326,7 @@ CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Conte return file_checker.check(); } -void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) +void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) { std::shared_lock lock(rwlock); diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 381be7762df..dfdf4c381b6 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -42,7 +42,7 @@ public: Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } - void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder&) override; protected: StorageStripeLog( diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 4beb44405d7..7a399f35c9c 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -430,7 +430,7 @@ CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context } void StorageTinyLog::truncate( - const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) + const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) { std::unique_lock lock(rwlock); diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index ae124e5e958..60dacf6e162 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -41,7 +41,7 @@ public: Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } - void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; void drop() override; diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 319ef257d6d..85d0f679708 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -103,11 +103,11 @@ protected: { StoragePtr storage = storages.at(std::make_pair(database_name, table_name)); - TableStructureReadLockHolder table_lock; + TableLockHolder table_lock; try { - table_lock = storage->lockStructureForShare(false, query_id, lock_acquire_timeout); + table_lock = storage->lockForShare(query_id, lock_acquire_timeout); } catch (const Exception & e) { diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index b998b60c02d..b48f8a3cb6b 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -196,7 +196,7 @@ StoragesInfo StoragesInfoStream::next() try { /// For table not to be dropped and set of columns to remain constant. - info.table_lock = info.storage->lockStructureForShare(false, query_id, settings.lock_acquire_timeout); + info.table_lock = info.storage->lockForShare(query_id, settings.lock_acquire_timeout); } catch (const Exception & e) { diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 8af1f46d8a7..56c9a8fb0d0 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -14,7 +14,7 @@ class Context; struct StoragesInfo { StoragePtr storage = nullptr; - TableStructureReadLockHolder table_lock; + TableLockHolder table_lock; String database; String table; diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index df8df75ad6d..b7f029945d8 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -245,7 +245,7 @@ protected: continue; StoragePtr table = nullptr; - TableStructureReadLockHolder lock; + TableLockHolder lock; if (need_lock_structure) { @@ -257,8 +257,7 @@ protected: } try { - lock = table->lockStructureForShare( - false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + lock = table->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); } catch (const Exception & e) { diff --git a/src/Storages/TableStructureLockHolder.h b/src/Storages/TableStructureLockHolder.h index b5fc0c620ad..946f9ee545c 100644 --- a/src/Storages/TableStructureLockHolder.h +++ b/src/Storages/TableStructureLockHolder.h @@ -5,44 +5,17 @@ namespace DB { -/// Structs that hold table structure (columns, their types, default values etc.) locks when executing queries. -/// See IStorage::lock* methods for comments. - -struct TableStructureWriteLockHolder +struct TableExclusiveLockHolder { - void release() - { - *this = TableStructureWriteLockHolder(); - } - - void releaseAllExceptAlterIntention() - { - new_data_structure_lock.reset(); - structure_lock.reset(); - } + void release() { *this = TableExclusiveLockHolder(); } private: friend class IStorage; /// Order is important. - RWLockImpl::LockHolder alter_intention_lock; - RWLockImpl::LockHolder new_data_structure_lock; - RWLockImpl::LockHolder structure_lock; -}; - -struct TableStructureReadLockHolder -{ - void release() - { - *this = TableStructureReadLockHolder(); - } - -private: - friend class IStorage; - - /// Order is important. - RWLockImpl::LockHolder new_data_structure_lock; - RWLockImpl::LockHolder structure_lock; + RWLockImpl::LockHolder alter_lock; + RWLockImpl::LockHolder drop_lock; }; +using TableLockHolder = RWLockImpl::LockHolder; } From 54e5fe7dbc6336da462812549ae2cad1911cbf66 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 19:19:40 +0300 Subject: [PATCH 095/211] Less locks --- src/Storages/StorageMergeTree.cpp | 2 -- src/Storages/StorageReplicatedMergeTree.cpp | 14 +++++--------- 2 files changed, 5 insertions(+), 11 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9042afe1b2d..16e921f5294 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1086,8 +1086,6 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons /// Asks to complete merges and does not allow them to start. /// This protects against "revival" of data for a removed partition after completion of merge. auto merge_blocker = merger_mutator.merges_blocker.cancel(); - /// Waits for completion of merge and does not start new ones. - auto lock = lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); String partition_id = getPartitionIDFromQuery(partition, context); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b62f6bbd198..c1eb8183a32 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3633,17 +3633,13 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer zookeeper->multi(requests); - { - auto alter_lock = lockForAlter(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally."); - LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally."); + auto metadata_diff = ReplicatedMergeTreeTableMetadata(*this, getInMemoryMetadataPtr()).checkAndFindDiff(metadata_from_entry); + setTableStructure(std::move(columns_from_entry), metadata_diff); + metadata_version = entry.alter_version; - auto metadata_diff = ReplicatedMergeTreeTableMetadata(*this, getInMemoryMetadataPtr()).checkAndFindDiff(metadata_from_entry); - setTableStructure(std::move(columns_from_entry), metadata_diff); - metadata_version = entry.alter_version; - - LOG_INFO(log, "Applied changes to the metadata of the table. Current metadata version: {}", metadata_version); - } + LOG_INFO(log, "Applied changes to the metadata of the table. Current metadata version: {}", metadata_version); /// This transaction may not happen, but it's OK, because on the next retry we will eventually create/update this node zookeeper->createOrUpdate(replica_path + "/metadata_version", std::to_string(metadata_version), zkutil::CreateMode::Persistent); From b3ee8967dce4f9b6e9e99e447ff3b63a4b2d7e63 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 19:28:20 +0300 Subject: [PATCH 096/211] Fix style --- src/Storages/IStorage.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 42224ec01ac..919464a6a5d 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -19,7 +19,6 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int TABLE_IS_DROPPED; extern const int NOT_IMPLEMENTED; extern const int DEADLOCK_AVOIDED; From e888dafdc25f9a498feefdc3eec2776c93731892 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 20:03:42 +0300 Subject: [PATCH 097/211] Remove unused method --- src/Storages/MergeTree/MergeTreeData.cpp | 11 ----------- src/Storages/MergeTree/MergeTreeData.h | 1 - src/Storages/StorageMergeTree.cpp | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 7 +++---- 4 files changed, 3 insertions(+), 17 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0ed4e98e864..779a6a7ebea 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -183,9 +183,6 @@ MergeTreeData::MergeTreeData( throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS); } - - setTTLExpressions(metadata_, metadata_); - /// format_file always contained on any data path PathWithDisk version_file; /// Creating directories, if not exist. @@ -516,14 +513,6 @@ void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_meta } } -/// Todo replace columns with TTL for columns -void MergeTreeData::setTTLExpressions(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata) -{ - checkTTLExpressions(new_metadata, old_metadata); - //setColumnTTLs(new_metadata.column_ttls_by_name); - //setTableTTLs(new_metadata.table_ttl); -} - void MergeTreeData::checkStoragePolicy(const StoragePolicyPtr & new_storage_policy) const { diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index ca6928cbb01..2e6c0bfc903 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -792,7 +792,6 @@ protected: void initPartitionKey(const KeyDescription & new_partition_key); void checkTTLExpressions(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata) const; - void setTTLExpressions(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata); void checkStoragePolicy(const StoragePolicyPtr & new_storage_policy) const; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 16e921f5294..324c61ae419 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -278,7 +278,6 @@ void StorageMergeTree::alter( changeSettings(new_metadata.settings_changes, table_lock_holder); /// Reinitialize primary key because primary key column types might have changed. setProperties(new_metadata, old_metadata); - setTTLExpressions(new_metadata, old_metadata); DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c1eb8183a32..352e0cbe802 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -816,13 +816,12 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column } } - auto table_id = getStorageID(); - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(global_context, table_id, new_metadata); - /// Even if the primary/sorting keys didn't change we must reinitialize it /// because primary key column types might have changed. setProperties(new_metadata, old_metadata); - setTTLExpressions(new_metadata, old_metadata); + + auto table_id = getStorageID(); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(global_context, table_id, new_metadata); } From c666763cc05048477fea6099c0b20280bc3946cd Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 20:09:06 +0300 Subject: [PATCH 098/211] Remove unused method better --- src/Storages/MergeTree/MergeTreeData.cpp | 3 +++ src/Storages/StorageMergeTree.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 1 + 3 files changed, 5 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 779a6a7ebea..72937bd7102 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -183,6 +183,9 @@ MergeTreeData::MergeTreeData( throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS); } + + checkTTLExpressions(metadata_, metadata_); + /// format_file always contained on any data path PathWithDisk version_file; /// Creating directories, if not exist. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 324c61ae419..1c497a6b62c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -277,6 +277,7 @@ void StorageMergeTree::alter( { changeSettings(new_metadata.settings_changes, table_lock_holder); /// Reinitialize primary key because primary key column types might have changed. + checkTTLExpressions(new_metadata, old_metadata); setProperties(new_metadata, old_metadata); DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 352e0cbe802..30b2644fd04 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -818,6 +818,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column /// Even if the primary/sorting keys didn't change we must reinitialize it /// because primary key column types might have changed. + checkTTLExpressions(new_metadata, old_metadata); setProperties(new_metadata, old_metadata); auto table_id = getStorageID(); From 8b131e2079c07b8d5c46cc758055e46d8e029d61 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 20:19:11 +0300 Subject: [PATCH 099/211] Remove int contention --- src/Storages/StorageReplicatedMergeTree.cpp | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 30b2644fd04..c1534d9eed6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3633,13 +3633,16 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer zookeeper->multi(requests); - LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally."); + { + auto alter_lock = lockExclusively(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally."); - auto metadata_diff = ReplicatedMergeTreeTableMetadata(*this, getInMemoryMetadataPtr()).checkAndFindDiff(metadata_from_entry); - setTableStructure(std::move(columns_from_entry), metadata_diff); - metadata_version = entry.alter_version; + auto metadata_diff = ReplicatedMergeTreeTableMetadata(*this, getInMemoryMetadataPtr()).checkAndFindDiff(metadata_from_entry); + setTableStructure(std::move(columns_from_entry), metadata_diff); + metadata_version = entry.alter_version; - LOG_INFO(log, "Applied changes to the metadata of the table. Current metadata version: {}", metadata_version); + LOG_INFO(log, "Applied changes to the metadata of the table. Current metadata version: {}", metadata_version); + } /// This transaction may not happen, but it's OK, because on the next retry we will eventually create/update this node zookeeper->createOrUpdate(replica_path + "/metadata_version", std::to_string(metadata_version), zkutil::CreateMode::Persistent); From 009977c20c1651ccad992240428aa8388850358c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Jun 2020 22:45:37 +0300 Subject: [PATCH 100/211] Fix locks --- src/Storages/LiveView/StorageLiveView.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 0abb01d7dc7..f4f3c6b8642 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -514,7 +514,7 @@ void StorageLiveView::drop() void StorageLiveView::refresh(const Context & context) { - auto alter_lock = lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto table_lock = lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); { std::lock_guard lock(mutex); if (getNewBlocks()) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c1534d9eed6..c256a79dfe3 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3634,7 +3634,7 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer zookeeper->multi(requests); { - auto alter_lock = lockExclusively(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto lock = lockForAlter(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally."); auto metadata_diff = ReplicatedMergeTreeTableMetadata(*this, getInMemoryMetadataPtr()).checkAndFindDiff(metadata_from_entry); From 4b6db63fff8d78c273867d54663a9e8c6f1c24f5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Jun 2020 13:53:20 +0300 Subject: [PATCH 101/211] Fix alter key when materialized column passed --- src/Storages/AlterCommands.cpp | 4 ++++ src/Storages/IStorage.cpp | 2 +- src/Storages/IndicesDescription.cpp | 5 +++++ src/Storages/IndicesDescription.h | 4 ++++ tests/integration/test_alter_codec/test.py | 6 ++---- 5 files changed, 16 insertions(+), 5 deletions(-) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index c0fc53aa8e3..436862a3804 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -728,6 +728,10 @@ void AlterCommands::apply(StorageInMemoryMetadata & metadata, const Context & co metadata_copy.primary_key.definition_ast = nullptr; } + /// Changes in columns may lead to changes in secondary indices + for (auto & index : metadata_copy.secondary_indices) + index.recalculateWithNewColumns(metadata_copy.columns, context); + /// Changes in columns may lead to changes in TTL expressions. auto column_ttl_asts = metadata_copy.columns.getColumnTTLs(); for (const auto & [name, ast] : column_ttl_asts) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index a244f836f5c..01f6c709833 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -289,7 +289,7 @@ void IStorage::check(const Block & block, bool need_all) const void IStorage::setColumns(ColumnsDescription columns_) { - if (columns_.getOrdinary().empty()) + if (columns_.getAllPhysical().empty()) throw Exception("Empty list of columns passed", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); metadata.columns = std::move(columns_); } diff --git a/src/Storages/IndicesDescription.cpp b/src/Storages/IndicesDescription.cpp index ee9a9681e61..69b5d50dcb6 100644 --- a/src/Storages/IndicesDescription.cpp +++ b/src/Storages/IndicesDescription.cpp @@ -117,6 +117,10 @@ IndexDescription IndexDescription::getIndexFromAST(const ASTPtr & definition_ast return result; } +void IndexDescription::recalculateWithNewColumns(const ColumnsDescription & new_columns, const Context & context) +{ + *this = getIndexFromAST(definition_ast, new_columns, context); +} bool IndicesDescription::has(const String & name) const { @@ -154,6 +158,7 @@ IndicesDescription IndicesDescription::parse(const String & str, const ColumnsDe return result; } + ExpressionActionsPtr IndicesDescription::getSingleExpressionForIndices(const ColumnsDescription & columns, const Context & context) const { ASTPtr combined_expr_list = std::make_shared(); diff --git a/src/Storages/IndicesDescription.h b/src/Storages/IndicesDescription.h index b4d225c6511..18b41b05fbb 100644 --- a/src/Storages/IndicesDescription.h +++ b/src/Storages/IndicesDescription.h @@ -55,6 +55,10 @@ struct IndexDescription /// unintentionaly share AST variables and modify them. IndexDescription(const IndexDescription & other); IndexDescription & operator=(const IndexDescription & other); + + /// Recalculate index with new columns because index expression may change + /// if something change in columns. + void recalculateWithNewColumns(const ColumnsDescription & new_columns, const Context & context); }; /// All secondary indices in storage diff --git a/tests/integration/test_alter_codec/test.py b/tests/integration/test_alter_codec/test.py index 573fe7f0961..e6ad8d664a8 100644 --- a/tests/integration/test_alter_codec/test.py +++ b/tests/integration/test_alter_codec/test.py @@ -46,8 +46,7 @@ def test_alter_codec_pk(started_cluster): with pytest.raises(QueryRuntimeException): node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 ALIAS 3 CODEC(Delta, LZ4)".format(name=name)) - with pytest.raises(QueryRuntimeException): - node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 MATERIALIZED 3 CODEC(Delta, LZ4)".format(name=name)) + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 MATERIALIZED 3 CODEC(Delta, LZ4)".format(name=name)) node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64".format(name=name)) @@ -76,8 +75,7 @@ def test_alter_codec_index(started_cluster): with pytest.raises(QueryRuntimeException): node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 ALIAS 3 CODEC(Delta, LZ4)".format(name=name)) - with pytest.raises(QueryRuntimeException): - node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 MATERIALIZED 3 CODEC(Delta, LZ4)".format(name=name)) + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 MATERIALIZED 3 CODEC(Delta, LZ4)".format(name=name)) node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64".format(name=name)) From 016ee6316faa9bf1523e1a1f65a0aa3ece449234 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Jun 2020 15:05:29 +0300 Subject: [PATCH 102/211] Add missed check --- src/Storages/StorageInMemoryMetadata.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 81d1f387424..c33361ec7bf 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -16,6 +16,7 @@ namespace ErrorCodes extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int NOT_FOUND_COLUMN_IN_BLOCK; extern const int TYPE_MISMATCH; + extern const int EMPTY_LIST_OF_COLUMNS_PASSED; } @@ -69,6 +70,8 @@ StorageInMemoryMetadata & StorageInMemoryMetadata::operator=(const StorageInMemo void StorageInMemoryMetadata::setColumns(ColumnsDescription columns_) { + if (columns_.getAllPhysical().empty()) + throw Exception("Empty list of columns passed", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); columns = std::move(columns_); } From b504fc6a907937b5c400249c31b859aaa4838b7d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Jun 2020 16:37:27 +0300 Subject: [PATCH 103/211] Fix error code --- .../0_stateless/01213_alter_rename_primary_key_zookeeper.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql b/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql index daec7666e2d..b7102fab230 100644 --- a/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql +++ b/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql @@ -51,6 +51,6 @@ ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key2 TO renamed_key2 ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key3 TO renamed_key3; --{serverError 47} -ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN value1 TO renamed_value1; --{serverError 524} +ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN value1 TO renamed_value1; --{serverError 47} DROP TABLE IF EXISTS table_for_rename_with_primary_key; From 3c47faa9daa9ac8bc9445ac914d6a7d266d5e239 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Jun 2020 18:14:08 +0300 Subject: [PATCH 104/211] Move partition key initialization into registerMergeTree --- src/Storages/IStorage.h | 2 ++ src/Storages/MergeTree/MergeTreeData.cpp | 6 +----- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/MergeTree/registerStorageMergeTree.cpp | 6 ++++++ src/Storages/StorageInMemoryMetadata.h | 3 --- 5 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 1309b727a74..eb69264c6c8 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -138,7 +138,9 @@ public: public: StorageInMemoryMetadata getInMemoryMetadata() const { return *metadata.get(); } + StorageMetadataPtr getInMemoryMetadataPtr() const { return metadata.get(); } + void setInMemoryMetadata(const StorageInMemoryMetadata & metadata_) { metadata.set(std::make_unique(metadata_)); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 72937bd7102..dfc7636b3e4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -118,7 +118,7 @@ const char * DELETE_ON_DESTROY_MARKER_PATH = "delete-on-destroy.txt"; MergeTreeData::MergeTreeData( const StorageID & table_id_, const String & relative_data_path_, - StorageInMemoryMetadata metadata_, + const StorageInMemoryMetadata & metadata_, Context & context_, const String & date_column_name, const MergingParams & merging_params_, @@ -143,15 +143,11 @@ MergeTreeData::MergeTreeData( throw Exception("MergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME); MergeTreeDataFormatVersion min_format_version(0); - /// TODO(alesap) Move to register methods if (!date_column_name.empty()) { try { - auto partition_by_ast = makeASTFunction("toYYYYMM", std::make_shared(date_column_name)); - metadata_.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, metadata_.columns, global_context); initPartitionKey(metadata_.partition_key); - if (minmax_idx_date_column_pos == -1) throw Exception("Could not find Date column", ErrorCodes::BAD_TYPE_OF_FIELD); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 2e6c0bfc903..c6721658d78 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -324,7 +324,7 @@ public: /// attach - whether the existing table is attached or the new table is created. MergeTreeData(const StorageID & table_id_, const String & relative_data_path_, - StorageInMemoryMetadata metadata_, + const StorageInMemoryMetadata & metadata_, Context & context_, const String & date_column_name, const MergingParams & merging_params_, diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 98884de985e..1ecac8f413d 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -570,6 +570,12 @@ static StoragePtr create(const StorageFactory::Arguments & args) throw Exception( "Date column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS); + + auto partition_by_ast = makeASTFunction("toYYYYMM", std::make_shared(date_column_name)); + + metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, metadata.columns, args.context); + + ++arg_num; /// If there is an expression for sampling diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index bda48bc19cb..756b11150fb 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -14,9 +14,6 @@ namespace DB { -/// Structure represent table metadata stored in memory. -/// Only one storage engine support all fields -- MergeTree. -/// Complete table AST can be recreated from this struct. struct StorageInMemoryMetadata { /// Columns of table with their names, types, From a2e7e9f232edbb4e09a24a962b9aef2f2b8b9236 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Jun 2020 18:21:48 +0300 Subject: [PATCH 105/211] Remove unused constructor and rename method --- src/Storages/MergeTree/MergeTreeData.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageInMemoryMetadata.cpp | 11 ----------- src/Storages/StorageInMemoryMetadata.h | 1 - 4 files changed, 4 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index dfc7636b3e4..f1567a1d18d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -147,7 +147,7 @@ MergeTreeData::MergeTreeData( { try { - initPartitionKey(metadata_.partition_key); + checkPartitionKeyAndInitMinMax(metadata_.partition_key); if (minmax_idx_date_column_pos == -1) throw Exception("Could not find Date column", ErrorCodes::BAD_TYPE_OF_FIELD); } @@ -161,7 +161,7 @@ MergeTreeData::MergeTreeData( else { is_custom_partitioned = true; - initPartitionKey(metadata_.partition_key); + checkPartitionKeyAndInitMinMax(metadata_.partition_key); min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING; } @@ -415,7 +415,7 @@ ExpressionActionsPtr MergeTreeData::getSortingKeyAndSkipIndicesExpression(const } -void MergeTreeData::initPartitionKey(const KeyDescription & new_partition_key) +void MergeTreeData::checkPartitionKeyAndInitMinMax(const KeyDescription & new_partition_key) { if (new_partition_key.expression_list_ast->children.empty()) return; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index c6721658d78..1b970d470ba 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -789,7 +789,7 @@ protected: void setProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach = false); - void initPartitionKey(const KeyDescription & new_partition_key); + void checkPartitionKeyAndInitMinMax(const KeyDescription & new_partition_key); void checkTTLExpressions(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata) const; diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index c33361ec7bf..e39dff6472e 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -19,17 +19,6 @@ namespace ErrorCodes extern const int EMPTY_LIST_OF_COLUMNS_PASSED; } - -StorageInMemoryMetadata::StorageInMemoryMetadata( - const ColumnsDescription & columns_, - const IndicesDescription & secondary_indices_, - const ConstraintsDescription & constraints_) - : columns(columns_) - , secondary_indices(secondary_indices_) - , constraints(constraints_) -{ -} - StorageInMemoryMetadata::StorageInMemoryMetadata(const StorageInMemoryMetadata & other) : columns(other.columns) , secondary_indices(other.secondary_indices) diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 756b11150fb..83f9180dcbb 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -42,7 +42,6 @@ struct StorageInMemoryMetadata SelectQueryDescription select; StorageInMemoryMetadata() = default; - StorageInMemoryMetadata(const ColumnsDescription & columns_, const IndicesDescription & secondary_indices_, const ConstraintsDescription & constraints_); StorageInMemoryMetadata(const StorageInMemoryMetadata & other); StorageInMemoryMetadata & operator=(const StorageInMemoryMetadata & other); From c9fa5d2ec3e1ad7612702049e048a0193fdd991c Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Jun 2020 18:39:41 +0300 Subject: [PATCH 106/211] Better naming --- src/Storages/IStorage.h | 3 +-- src/Storages/Kafka/StorageKafka.cpp | 6 +++--- src/Storages/LiveView/StorageLiveView.cpp | 6 +++--- src/Storages/StorageBuffer.cpp | 8 ++++---- src/Storages/StorageDictionary.cpp | 6 +++--- src/Storages/StorageDistributed.cpp | 10 +++++----- src/Storages/StorageFile.cpp | 14 +++++++------- src/Storages/StorageGenerateRandom.cpp | 6 +++--- src/Storages/StorageHDFS.cpp | 8 ++++---- src/Storages/StorageInput.cpp | 6 +++--- src/Storages/StorageLog.cpp | 10 +++++----- src/Storages/StorageMaterializedView.cpp | 8 ++++---- src/Storages/StorageMemory.cpp | 8 ++++---- src/Storages/StorageMerge.cpp | 6 +++--- src/Storages/StorageMySQL.cpp | 8 ++++---- src/Storages/StorageS3.cpp | 8 ++++---- src/Storages/StorageSet.cpp | 8 ++++---- src/Storages/StorageStripeLog.cpp | 8 ++++---- src/Storages/StorageTinyLog.cpp | 10 +++++----- src/Storages/StorageURL.cpp | 8 ++++---- src/Storages/StorageValues.cpp | 6 +++--- src/Storages/StorageView.cpp | 8 ++++---- src/Storages/System/StorageSystemColumns.cpp | 6 +++--- src/Storages/System/StorageSystemDetachedParts.cpp | 6 +++--- src/Storages/System/StorageSystemDisks.cpp | 6 +++--- src/Storages/System/StorageSystemNumbers.cpp | 6 +++--- src/Storages/System/StorageSystemOne.cpp | 6 +++--- src/Storages/System/StorageSystemPartsBase.cpp | 6 +++--- src/Storages/System/StorageSystemReplicas.cpp | 6 +++--- .../System/StorageSystemStoragePolicies.cpp | 6 +++--- src/Storages/System/StorageSystemTables.cpp | 6 +++--- src/Storages/System/StorageSystemZeros.cpp | 6 +++--- 32 files changed, 114 insertions(+), 115 deletions(-) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index eb69264c6c8..e980bad889c 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -80,8 +80,7 @@ class IStorage : public std::enable_shared_from_this, public TypePromo { public: IStorage() = delete; - /// Storage fields should be initialized in separate methods like setColumns - /// or setTableTTLs. + /// Storage metadata can be set separately in setInMemoryMetadata method explicit IStorage(StorageID storage_id_) : storage_id(std::move(storage_id_)), metadata(std::make_unique()) {} //-V730 virtual ~IStorage() = default; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index b46cf0579ec..e0949cd9420 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -137,9 +137,9 @@ StorageKafka::StorageKafka( , intermediate_commit(kafka_settings->kafka_commit_every_batch.value) , settings_adjustments(createSettingsAdjustments()) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + setInMemoryMetadata(storage_metadata); task = global_context.getSchedulePool().createTask(log->name(), [this]{ threadFunc(); }); task->deactivate(); diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index f4f3c6b8642..efd0a71b18f 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -255,9 +255,9 @@ StorageLiveView::StorageLiveView( live_view_context = std::make_unique(global_context); live_view_context->makeQueryContext(); - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + setInMemoryMetadata(storage_metadata); if (!query.select) throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index b4d6b66ebe7..bd3945c26e0 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -77,10 +77,10 @@ StorageBuffer::StorageBuffer( , log(&Poco::Logger::get("StorageBuffer (" + table_id_.getFullTableName() + ")")) , bg_pool(global_context.getBufferFlushSchedulePool()) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - metadata_.setConstraints(constraints_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 25126ad951d..83a093d5635 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -100,9 +100,9 @@ StorageDictionary::StorageDictionary( : IStorage(table_id_) , dictionary_name(dictionary_name_) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(ColumnsDescription{getNamesAndTypes(dictionary_structure_)}); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription{getNamesAndTypes(dictionary_structure_)}); + setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 9c20e3f8e11..bf9e7f126c5 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -290,14 +290,14 @@ StorageDistributed::StorageDistributed( , storage_policy(storage_policy_) , relative_data_path(relative_data_path_) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - metadata_.setConstraints(constraints_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); if (sharding_key_) { - sharding_key_expr = buildShardingKeyExpression(sharding_key_, *global_context, metadata_.getColumns().getAllPhysical(), false); + sharding_key_expr = buildShardingKeyExpression(sharding_key_, *global_context, storage_metadata.getColumns().getAllPhysical(), false); sharding_key_column_name = sharding_key_->getColumnName(); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 4867a0bc215..c7671fd8759 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -167,9 +167,9 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us Block header = StorageDistributedDirectoryMonitor::createStreamFromFile(first_path)->getHeader(); - StorageInMemoryMetadata metadata_; - metadata_.setColumns(ColumnsDescription(header.getNamesAndTypesList())); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription(header.getNamesAndTypesList())); + setInMemoryMetadata(storage_metadata); } } } @@ -191,12 +191,12 @@ StorageFile::StorageFile(CommonArguments args) , compression_method(args.compression_method) , base_path(args.context.getPath()) { - StorageInMemoryMetadata metadata_; + StorageInMemoryMetadata storage_metadata; if (args.format_name != "Distributed") - metadata_.setColumns(args.columns); + storage_metadata.setColumns(args.columns); - metadata_.setConstraints(args.constraints); - setInMemoryMetadata(metadata_); + storage_metadata.setConstraints(args.constraints); + setInMemoryMetadata(storage_metadata); } class StorageFileSource : public SourceWithProgress diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index dad323f7b72..6d923f7678c 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -388,9 +388,9 @@ StorageGenerateRandom::StorageGenerateRandom(const StorageID & table_id_, const : IStorage(table_id_), max_array_length(max_array_length_), max_string_length(max_string_length_) { random_seed = random_seed_ ? sipHash64(*random_seed_) : randomSeed(); - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/StorageHDFS.cpp b/src/Storages/StorageHDFS.cpp index ee5a426cedc..ce492017a09 100644 --- a/src/Storages/StorageHDFS.cpp +++ b/src/Storages/StorageHDFS.cpp @@ -50,10 +50,10 @@ StorageHDFS::StorageHDFS(const String & uri_, { context.getRemoteHostFilter().checkURL(Poco::URI(uri)); - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - metadata_.setConstraints(constraints_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); } namespace diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index 4430fb11186..dc8d7ec1581 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -21,9 +21,9 @@ namespace ErrorCodes StorageInput::StorageInput(const StorageID & table_id, const ColumnsDescription & columns_) : IStorage(table_id) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 542fb507d83..a655373ce6e 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -434,10 +434,10 @@ StorageLog::StorageLog( , max_compress_block_size(max_compress_block_size_) , file_checker(disk, table_path + "sizes.json") { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - metadata_.setConstraints(constraints_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); if (relative_path_.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); @@ -445,7 +445,7 @@ StorageLog::StorageLog( /// create directories if they do not exist disk->createDirectories(table_path); - for (const auto & column : metadata_.getColumns().getAllPhysical()) + for (const auto & column : storage_metadata.getColumns().getAllPhysical()) addFiles(column.name, *column.type); marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 976b3c80dec..9e6245c3ced 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -50,8 +50,8 @@ StorageMaterializedView::StorageMaterializedView( bool attach_) : IStorage(table_id_), global_context(local_context.getGlobalContext()) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); if (!query.select) throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); @@ -67,8 +67,8 @@ StorageMaterializedView::StorageMaterializedView( throw Exception("UNION is not supported for MATERIALIZED VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW); auto select = SelectQueryDescription::getSelectQueryFromASTForMatView(query.select->clone(), local_context); - metadata_.setSelectQuery(select); - setInMemoryMetadata(metadata_); + storage_metadata.setSelectQuery(select); + setInMemoryMetadata(storage_metadata); if (!has_inner_table) target_table_id = query.to_table_id; diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 05b37ecf32e..c77fddb2372 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -94,10 +94,10 @@ private: StorageMemory::StorageMemory(const StorageID & table_id_, ColumnsDescription columns_description_, ConstraintsDescription constraints_) : IStorage(table_id_) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(std::move(columns_description_)); - metadata_.setConstraints(std::move(constraints_)); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(std::move(columns_description_)); + storage_metadata.setConstraints(std::move(constraints_)); + setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index f4030ed573f..8a28387d24d 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -51,9 +51,9 @@ StorageMerge::StorageMerge( , table_name_regexp(table_name_regexp_) , global_context(context_) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + setInMemoryMetadata(storage_metadata); } template diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 3e9b48e976b..919acd79fdd 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -56,10 +56,10 @@ StorageMySQL::StorageMySQL( , pool(std::move(pool_)) , global_context(context_) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - metadata_.setConstraints(constraints_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 7f237fd551f..f8d3de3a238 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -206,10 +206,10 @@ StorageS3::StorageS3( , compression_method(compression_method_) { context_global.getRemoteHostFilter().checkURL(uri_.uri); - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - metadata_.setConstraints(constraints_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); auto settings = context_.getStorageS3Settings().getSettings(uri.endpoint); Aws::Auth::AWSCredentials credentials(access_key_id_, secret_access_key_); diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 58d5226c91a..f2946afbbfd 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -105,10 +105,10 @@ StorageSetOrJoinBase::StorageSetOrJoinBase( const Context & context_) : IStorage(table_id_) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - metadata_.setConstraints(constraints_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); if (relative_path_.empty()) diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 3086e971121..f773ab2ca1e 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -237,10 +237,10 @@ StorageStripeLog::StorageStripeLog( , file_checker(disk, table_path + "sizes.json") , log(&Poco::Logger::get("StorageStripeLog")) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - metadata_.setConstraints(constraints_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); if (relative_path_.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 7a399f35c9c..9f19f44d16b 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -339,10 +339,10 @@ StorageTinyLog::StorageTinyLog( , file_checker(disk, table_path + "sizes.json") , log(&Poco::Logger::get("StorageTinyLog")) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - metadata_.setConstraints(constraints_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); if (relative_path_.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); @@ -353,7 +353,7 @@ StorageTinyLog::StorageTinyLog( disk->createDirectories(table_path); } - for (const auto & col : metadata_.getColumns().getAllPhysical()) + for (const auto & col : storage_metadata.getColumns().getAllPhysical()) addFiles(col.name, *col.type); } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 802ad0571a8..fd9dde695ff 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -44,10 +44,10 @@ IStorageURLBase::IStorageURLBase( { context_global.getRemoteHostFilter().checkURL(uri); - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - metadata_.setConstraints(constraints_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); } namespace diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index 063cd3d5224..e99aeb52018 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -16,9 +16,9 @@ StorageValues::StorageValues( const NamesAndTypesList & virtuals_) : IStorage(table_id_), res_block(res_block_), virtuals(virtuals_) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + setInMemoryMetadata(storage_metadata); } Pipes StorageValues::read( diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 006b1b3caec..21d353f6bed 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -38,8 +38,8 @@ StorageView::StorageView( const ColumnsDescription & columns_) : IStorage(table_id_) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(columns_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); if (!query.select) throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); @@ -47,8 +47,8 @@ StorageView::StorageView( SelectQueryDescription description; description.inner_query = query.select->ptr(); - metadata_.setSelectQuery(description); - setInMemoryMetadata(metadata_); + storage_metadata.setSelectQuery(description); + setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 85d0f679708..6f5c8bc673c 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -26,8 +26,8 @@ namespace ErrorCodes StorageSystemColumns::StorageSystemColumns(const std::string & name_) : IStorage({"system", name_}) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(ColumnsDescription( + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription( { { "database", std::make_shared() }, { "table", std::make_shared() }, @@ -46,7 +46,7 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_) { "is_in_sampling_key", std::make_shared() }, { "compression_codec", std::make_shared() }, })); - setInMemoryMetadata(metadata_); + setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 7228651d140..c325df1251d 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -30,8 +30,8 @@ protected: explicit StorageSystemDetachedParts() : IStorage({"system", "detached_parts"}) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(ColumnsDescription{{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription{{ {"database", std::make_shared()}, {"table", std::make_shared()}, {"partition_id", std::make_shared(std::make_shared())}, @@ -42,7 +42,7 @@ protected: {"max_block_number", std::make_shared(std::make_shared())}, {"level", std::make_shared(std::make_shared())} }}); - setInMemoryMetadata(metadata_); + setInMemoryMetadata(storage_metadata); } Pipes read( diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index fbcdd78988a..a270a96b8f7 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -14,8 +14,8 @@ namespace ErrorCodes StorageSystemDisks::StorageSystemDisks(const std::string & name_) : IStorage({"system", name_}) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(ColumnsDescription( + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription( { {"name", std::make_shared()}, {"path", std::make_shared()}, @@ -23,7 +23,7 @@ StorageSystemDisks::StorageSystemDisks(const std::string & name_) {"total_space", std::make_shared()}, {"keep_free_space", std::make_shared()}, })); - setInMemoryMetadata(metadata_); + setInMemoryMetadata(storage_metadata); } Pipes StorageSystemDisks::read( diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index 50921c53fb6..404eb1af99c 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -118,9 +118,9 @@ private: StorageSystemNumbers::StorageSystemNumbers(const StorageID & table_id, bool multithreaded_, std::optional limit_, UInt64 offset_, bool even_distribution_) : IStorage(table_id), multithreaded(multithreaded_), even_distribution(even_distribution_), limit(limit_), offset(offset_) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(ColumnsDescription({{"number", std::make_shared()}})); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription({{"number", std::make_shared()}})); + setInMemoryMetadata(storage_metadata); } Pipes StorageSystemNumbers::read( diff --git a/src/Storages/System/StorageSystemOne.cpp b/src/Storages/System/StorageSystemOne.cpp index 20d61d5da1b..f6fba0d302c 100644 --- a/src/Storages/System/StorageSystemOne.cpp +++ b/src/Storages/System/StorageSystemOne.cpp @@ -14,9 +14,9 @@ namespace DB StorageSystemOne::StorageSystemOne(const std::string & name_) : IStorage({"system", name_}) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(ColumnsDescription({{"dummy", std::make_shared()}})); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription({{"dummy", std::make_shared()}})); + setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index b48f8a3cb6b..928b146247d 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -278,9 +278,9 @@ StorageSystemPartsBase::StorageSystemPartsBase(std::string name_, NamesAndTypesL add_alias("bytes", "bytes_on_disk"); add_alias("marks_size", "marks_bytes"); - StorageInMemoryMetadata metadata_; - metadata_.setColumns(tmp_columns); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(tmp_columns); + setInMemoryMetadata(storage_metadata); } NamesAndTypesList StorageSystemPartsBase::getVirtuals() const diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index f79e9138500..26076d09914 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -19,8 +19,8 @@ namespace DB StorageSystemReplicas::StorageSystemReplicas(const std::string & name_) : IStorage({"system", name_}) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(ColumnsDescription({ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription({ { "database", std::make_shared() }, { "table", std::make_shared() }, { "engine", std::make_shared() }, @@ -53,7 +53,7 @@ StorageSystemReplicas::StorageSystemReplicas(const std::string & name_) { "active_replicas", std::make_shared() }, { "zookeeper_exception", std::make_shared() }, })); - setInMemoryMetadata(metadata_); + setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/System/StorageSystemStoragePolicies.cpp b/src/Storages/System/StorageSystemStoragePolicies.cpp index a6092a28a47..e589955c861 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.cpp +++ b/src/Storages/System/StorageSystemStoragePolicies.cpp @@ -17,8 +17,8 @@ namespace ErrorCodes StorageSystemStoragePolicies::StorageSystemStoragePolicies(const std::string & name_) : IStorage({"system", name_}) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns( + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns( ColumnsDescription({ {"policy_name", std::make_shared()}, {"volume_name", std::make_shared()}, @@ -27,7 +27,7 @@ StorageSystemStoragePolicies::StorageSystemStoragePolicies(const std::string & n {"max_data_part_size", std::make_shared()}, {"move_factor", std::make_shared()} })); - setInMemoryMetadata(metadata_); + setInMemoryMetadata(storage_metadata); } Pipes StorageSystemStoragePolicies::read( diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index b7f029945d8..deb8f0551ea 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -33,8 +33,8 @@ namespace ErrorCodes StorageSystemTables::StorageSystemTables(const std::string & name_) : IStorage({"system", name_}) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(ColumnsDescription( + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription( { {"database", std::make_shared()}, {"name", std::make_shared()}, @@ -56,7 +56,7 @@ StorageSystemTables::StorageSystemTables(const std::string & name_) {"total_rows", std::make_shared(std::make_shared())}, {"total_bytes", std::make_shared(std::make_shared())}, })); - setInMemoryMetadata(metadata_); + setInMemoryMetadata(storage_metadata); } diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index d325840091e..9489ff249a5 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -84,9 +84,9 @@ private: StorageSystemZeros::StorageSystemZeros(const StorageID & table_id_, bool multithreaded_, std::optional limit_) : IStorage(table_id_), multithreaded(multithreaded_), limit(limit_) { - StorageInMemoryMetadata metadata_; - metadata_.setColumns(ColumnsDescription({{"zero", std::make_shared()}})); - setInMemoryMetadata(metadata_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription({{"zero", std::make_shared()}})); + setInMemoryMetadata(storage_metadata); } From 4c0879ae300c40ca0d1aa20e5f4e4856fb96c401 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Jun 2020 20:17:13 +0300 Subject: [PATCH 107/211] Better logging in storages --- src/Interpreters/InterpreterSelectQuery.cpp | 8 +++++--- src/Storages/IStorage.h | 4 +++- src/Storages/Kafka/KafkaBlockInputStream.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 +- .../MergeTree/MergeTreeReverseSelectProcessor.cpp | 2 +- src/Storages/MergeTree/MergeTreeSelectProcessor.cpp | 2 +- src/Storages/MergeTree/MergeTreeSequentialSource.cpp | 2 +- src/Storages/StorageBuffer.cpp | 3 +-- src/Storages/StorageGenerateRandom.cpp | 2 +- src/Storages/StorageInMemoryMetadata.cpp | 9 +++++---- src/Storages/StorageInMemoryMetadata.h | 4 ++-- src/Storages/StorageJoin.cpp | 4 ++-- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageMemory.cpp | 4 ++-- src/Storages/StorageMerge.cpp | 4 ++-- src/Storages/StorageMySQL.cpp | 2 +- src/Storages/StorageNull.h | 2 +- src/Storages/StorageStripeLog.cpp | 6 +++--- src/Storages/StorageTinyLog.cpp | 2 +- src/Storages/StorageValues.cpp | 2 +- src/Storages/StorageView.cpp | 2 +- src/Storages/StorageXDBC.cpp | 4 ++-- src/Storages/System/IStorageSystemOneBlock.h | 2 +- src/Storages/System/StorageSystemColumns.cpp | 2 +- src/Storages/System/StorageSystemDisks.cpp | 2 +- src/Storages/System/StorageSystemNumbers.cpp | 2 +- src/Storages/System/StorageSystemOne.cpp | 2 +- src/Storages/System/StorageSystemPartsBase.cpp | 2 +- src/Storages/System/StorageSystemReplicas.cpp | 2 +- src/Storages/System/StorageSystemStoragePolicies.cpp | 2 +- src/Storages/System/StorageSystemTables.cpp | 2 +- src/Storages/System/StorageSystemZeros.cpp | 2 +- 33 files changed, 51 insertions(+), 47 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e0d5adf92b8..187fdeb2b19 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -381,14 +381,15 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (storage) { - source_header = metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals()); + source_header = metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID()); /// Fix source_header for filter actions. if (row_policy_filter) { filter_info = std::make_shared(); filter_info->column_name = generateFilterActions(filter_info->actions, row_policy_filter, required_columns); - source_header = metadata_snapshot->getSampleBlockForColumns(filter_info->actions->getRequiredColumns(), storage->getVirtuals()); + source_header = metadata_snapshot->getSampleBlockForColumns( + filter_info->actions->getRequiredColumns(), storage->getVirtuals(), storage->getStorageID()); } } @@ -1344,7 +1345,8 @@ void InterpreterSelectQuery::executeFetchColumns( if (pipes.empty()) { - Pipe pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals()))); + Pipe pipe(std::make_shared( + metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID()))); if (query_info.prewhere_info) { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index e980bad889c..6abf310442e 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -81,7 +81,9 @@ class IStorage : public std::enable_shared_from_this, public TypePromo public: IStorage() = delete; /// Storage metadata can be set separately in setInMemoryMetadata method - explicit IStorage(StorageID storage_id_) : storage_id(std::move(storage_id_)), metadata(std::make_unique()) {} //-V730 + explicit IStorage(StorageID storage_id_) + : storage_id(std::move(storage_id_)) + , metadata(std::make_unique()) {} //-V730 virtual ~IStorage() = default; IStorage(const IStorage &) = delete; diff --git a/src/Storages/Kafka/KafkaBlockInputStream.cpp b/src/Storages/Kafka/KafkaBlockInputStream.cpp index 847b0d915cd..9634cded7c8 100644 --- a/src/Storages/Kafka/KafkaBlockInputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockInputStream.cpp @@ -27,7 +27,7 @@ KafkaBlockInputStream::KafkaBlockInputStream( , commit_in_suffix(commit_in_suffix_) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) , virtual_header(metadata_snapshot->getSampleBlockForColumns( - {"_topic", "_key", "_offset", "_partition", "_timestamp", "_timestamp_ms", "_headers.name", "_headers.value"}, storage.getVirtuals())) + {"_topic", "_key", "_offset", "_partition", "_timestamp", "_timestamp_ms", "_headers.name", "_headers.value"}, storage.getVirtuals(), storage.getStorageID())) { } @@ -44,7 +44,7 @@ KafkaBlockInputStream::~KafkaBlockInputStream() Block KafkaBlockInputStream::getHeader() const { - return metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals()); + return metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()); } void KafkaBlockInputStream::readPrefixImpl() diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 7f7fd203297..fa91a9190e5 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -224,7 +224,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( std::multiset part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); - metadata_snapshot->check(real_column_names, data.getVirtuals()); + metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID()); const Settings & settings = context.getSettingsRef(); const auto & primary_key = metadata_snapshot->getPrimaryKey(); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index fdf3908d21e..d78f72d1dd0 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -150,7 +150,7 @@ MarkRanges MergeTreeReadPool::getRestMarks(const IMergeTreeDataPart & part, cons Block MergeTreeReadPool::getHeader() const { - return metadata_snapshot->getSampleBlockForColumns(column_names, data.getVirtuals()); + return metadata_snapshot->getSampleBlockForColumns(column_names, data.getVirtuals(), data.getStorageID()); } void MergeTreeReadPool::profileFeedback(const ReadBufferFromFileBase::ProfileInfo info) diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp index b71c343614b..1e6352824ef 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp @@ -50,7 +50,7 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor( bool quiet) : MergeTreeBaseSelectProcessor{ - replaceTypes(metadata_snapshot_->getSampleBlockForColumns(required_columns_, storage_.getVirtuals()), owned_data_part_), + replaceTypes(metadata_snapshot_->getSampleBlockForColumns(required_columns_, storage_.getVirtuals(), storage_.getStorageID()), owned_data_part_), storage_, metadata_snapshot_, prewhere_info_, max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, reader_settings_, use_uncompressed_cache_, virt_column_names_}, diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index b46b414bfe8..a9ba6f7836b 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -30,7 +30,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( bool quiet) : MergeTreeBaseSelectProcessor{ - metadata_snapshot_->getSampleBlockForColumns(required_columns_, storage_.getVirtuals()), + metadata_snapshot_->getSampleBlockForColumns(required_columns_, storage_.getVirtuals(), storage_.getStorageID()), storage_, metadata_snapshot_, prewhere_info_, max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, reader_settings_, use_uncompressed_cache_, virt_column_names_}, diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index f8e31db2b5a..edd63aadd29 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -17,7 +17,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( bool read_with_direct_io_, bool take_column_types_from_storage, bool quiet) - : SourceWithProgress(metadata_snapshot_->getSampleBlockForColumns(columns_to_read_, storage_.getVirtuals())) + : SourceWithProgress(metadata_snapshot_->getSampleBlockForColumns(columns_to_read_, storage_.getVirtuals(), storage_.getStorageID())) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , data_part(std::move(data_part_)) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index bd3945c26e0..e0bd19feba9 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -90,7 +90,7 @@ class BufferSource : public SourceWithProgress public: BufferSource(const Names & column_names_, StorageBuffer::Buffer & buffer_, const StorageBuffer & storage, const StorageMetadataPtr & metadata_snapshot) : SourceWithProgress( - metadata_snapshot->getSampleBlockForColumns(column_names_, storage.getVirtuals())) + metadata_snapshot->getSampleBlockForColumns(column_names_, storage.getVirtuals(), storage.getStorageID())) , column_names(column_names_.begin(), column_names_.end()) , buffer(buffer_) {} @@ -468,7 +468,6 @@ bool StorageBuffer::mayBenefitFromIndexForIn( if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); - /// TODO alesap (check destination metadata) return destination->mayBenefitFromIndexForIn(left_in_operand, query_context, destination->getInMemoryMetadataPtr()); } diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 6d923f7678c..1f227265d19 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -436,7 +436,7 @@ Pipes StorageGenerateRandom::read( size_t max_block_size, unsigned num_streams) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); Pipes pipes; pipes.reserve(num_streams); diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index e39dff6472e..b7f4565a55a 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -249,7 +249,8 @@ Block StorageInMemoryMetadata::getSampleBlock() const return res; } -Block StorageInMemoryMetadata::getSampleBlockForColumns(const Names & column_names, const NamesAndTypesList & virtuals) const +Block StorageInMemoryMetadata::getSampleBlockForColumns( + const Names & column_names, const NamesAndTypesList & virtuals, const StorageID & storage_id) const { Block res; @@ -274,7 +275,7 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns(const Names & column_nam else { throw Exception( - "Column " + backQuote(name) + " not found in table " /*+ getStorageID().getNameForLogs() TODO(alesap)*/, + "Column " + backQuote(name) + " not found in table " + storage_id.getNameForLogs(), ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); } } @@ -442,7 +443,7 @@ namespace } } -void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTypesList & virtuals) const +void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTypesList & virtuals, const StorageID & storage_id) const { NamesAndTypesList available_columns = getColumns().getAllPhysical(); available_columns.insert(available_columns.end(), virtuals.begin(), virtuals.end()); @@ -459,7 +460,7 @@ void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTy { if (columns_map.end() == columns_map.find(name)) throw Exception( - "There is no column with name " + backQuote(name) + " in table " + /* TODO alesap getStorageID().getNameForLogs() +*/ ". There are columns: " + list_of_columns, + "There is no column with name " + backQuote(name) + " in table " + storage_id.getNameForLogs() + ". There are columns: " + list_of_columns, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); if (unique_names.end() != unique_names.find(name)) diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 83f9180dcbb..1d392c2d228 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -110,7 +110,7 @@ struct StorageInMemoryMetadata Block getSampleBlockNonMaterialized() const; /// ordinary. Block getSampleBlockWithVirtuals(const NamesAndTypesList & virtuals) const; /// ordinary + materialized + virtuals. Block getSampleBlockForColumns( - const Names & column_names, const NamesAndTypesList & virtuals) const; /// ordinary + materialized + aliases + virtuals. + const Names & column_names, const NamesAndTypesList & virtuals, const StorageID & storage_id) const; /// ordinary + materialized + aliases + virtuals. /// Returns structure with partition key. const KeyDescription & getPartitionKey() const; @@ -176,7 +176,7 @@ struct StorageInMemoryMetadata /// Verify that all the requested names are in the table and are set correctly: /// list of names is not empty and the names do not repeat. - void check(const Names & column_names, const NamesAndTypesList & virtuals) const; + void check(const Names & column_names, const NamesAndTypesList & virtuals, const StorageID & storage_id) const; /// Check that all the requested names are in the table and have the correct types. void check(const NamesAndTypesList & columns) const; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 21e4370c28b..af1a8bf10d7 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -445,10 +445,10 @@ Pipes StorageJoin::read( size_t max_block_size, unsigned /*num_streams*/) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); Pipes pipes; - pipes.emplace_back(std::make_shared(*join, max_block_size, metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals()))); + pipes.emplace_back(std::make_shared(*join, max_block_size, metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); return pipes; } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index a655373ce6e..39fa1d1af70 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -587,7 +587,7 @@ Pipes StorageLog::read( size_t max_block_size, unsigned num_streams) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); loadMarks(); NamesAndTypesList all_columns = Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index c77fddb2372..44413caaa57 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -28,7 +28,7 @@ public: BlocksList::iterator end_, const StorageMemory & storage, const StorageMetadataPtr & metadata_snapshot) - : SourceWithProgress(metadata_snapshot->getSampleBlockForColumns(column_names_, storage.getVirtuals())) + : SourceWithProgress(metadata_snapshot->getSampleBlockForColumns(column_names_, storage.getVirtuals(), storage.getStorageID())) , column_names(std::move(column_names_)) , begin(begin_) , end(end_) @@ -110,7 +110,7 @@ Pipes StorageMemory::read( size_t /*max_block_size*/, unsigned num_streams) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); std::lock_guard lock(mutex); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 8a28387d24d..0ef4e415ff3 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -426,7 +426,7 @@ Block StorageMerge::getQueryHeader( { case QueryProcessingStage::FetchColumns: { - Block header = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals()); + Block header = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); if (query_info.prewhere_info) { query_info.prewhere_info->prewhere_actions->execute(header); @@ -438,7 +438,7 @@ Block StorageMerge::getQueryHeader( case QueryProcessingStage::WithMergeableState: case QueryProcessingStage::Complete: return InterpreterSelectQuery( - query_info.query, context, std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals())), + query_info.query, context, std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())), SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); } throw Exception("Logical Error: unknown processed stage.", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 919acd79fdd..6f57ea196d0 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -72,7 +72,7 @@ Pipes StorageMySQL::read( size_t max_block_size_, unsigned) { - metadata_snapshot->check(column_names_, getVirtuals()); + metadata_snapshot->check(column_names_, getVirtuals(), getStorageID()); String query = transformQueryForExternalDatabase( query_info_, metadata_snapshot->getColumns().getOrdinary(), diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index e79174c2565..072a5e3bc32 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -33,7 +33,7 @@ public: { Pipes pipes; pipes.emplace_back( - std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals()))); + std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); return pipes; } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index f773ab2ca1e..e55cc190f80 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -60,7 +60,7 @@ public: IndexForNativeFormat::Blocks::const_iterator index_end) { if (index_begin == index_end) - return metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals()); + return metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()); /// TODO: check if possible to always return storage.getSampleBlock() @@ -276,7 +276,7 @@ Pipes StorageStripeLog::read( { std::shared_lock lock(rwlock); - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); NameSet column_names_set(column_names.begin(), column_names.end()); @@ -285,7 +285,7 @@ Pipes StorageStripeLog::read( String index_file = table_path + "index.mrk"; if (!disk->exists(index_file)) { - pipes.emplace_back(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals()))); + pipes.emplace_back(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); return pipes; } diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 9f19f44d16b..ef8c30cacbe 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -404,7 +404,7 @@ Pipes StorageTinyLog::read( const size_t max_block_size, const unsigned /*num_streams*/) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); Pipes pipes; diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index e99aeb52018..d5585edde3b 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -30,7 +30,7 @@ Pipes StorageValues::read( size_t /*max_block_size*/, unsigned /*num_streams*/) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); Pipes pipes; diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 21d353f6bed..6e66b770486 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -88,7 +88,7 @@ Pipes StorageView::read( { return std::make_shared( header, metadata_snapshot->getSampleBlockForColumns( - column_names, getVirtuals()), ConvertingTransform::MatchColumnsMode::Name); + column_names, getVirtuals(), getStorageID()), ConvertingTransform::MatchColumnsMode::Name); }); pipes = std::move(pipeline).getPipes(); diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index 05cf4ed5abf..fc4bbefe74c 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -94,7 +94,7 @@ Pipes StorageXDBC::read( size_t max_block_size, unsigned num_streams) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); bridge_helper->startBridgeSync(); return IStorageURLBase::read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); @@ -130,7 +130,7 @@ BlockOutputStreamPtr StorageXDBC::write(const ASTPtr & /*query*/, const StorageM Block StorageXDBC::getHeaderBlock(const Names & column_names, const StorageMetadataPtr & metadata_snapshot) const { - return metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals()); + return metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); } std::string StorageXDBC::getName() const diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index b3a2a6fe53b..7c2ef85f158 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -37,7 +37,7 @@ public: size_t /*max_block_size*/, unsigned /*num_streams*/) override { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); Block sample_block = metadata_snapshot->getSampleBlock(); MutableColumns res_columns = sample_block.cloneEmptyColumns(); diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 6f5c8bc673c..beb01bc6192 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -249,7 +249,7 @@ Pipes StorageSystemColumns::read( const size_t max_block_size, const unsigned /*num_streams*/) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); /// Create a mask of what columns are needed in the result. diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index a270a96b8f7..cf00bbb5254 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -35,7 +35,7 @@ Pipes StorageSystemDisks::read( const size_t /*max_block_size*/, const unsigned /*num_streams*/) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); MutableColumnPtr col_name = ColumnString::create(); MutableColumnPtr col_path = ColumnString::create(); diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index 404eb1af99c..c70446ddeba 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -132,7 +132,7 @@ Pipes StorageSystemNumbers::read( size_t max_block_size, unsigned num_streams) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); if (limit && *limit < max_block_size) { diff --git a/src/Storages/System/StorageSystemOne.cpp b/src/Storages/System/StorageSystemOne.cpp index f6fba0d302c..3329cbb035e 100644 --- a/src/Storages/System/StorageSystemOne.cpp +++ b/src/Storages/System/StorageSystemOne.cpp @@ -29,7 +29,7 @@ Pipes StorageSystemOne::read( const size_t /*max_block_size*/, const unsigned /*num_streams*/) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); Block header{ColumnWithTypeAndName( DataTypeUInt8().createColumn(), diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 928b146247d..168b305605d 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -41,7 +41,7 @@ bool StorageSystemPartsBase::hasStateColumn(const Names & column_names, const St /// Do not check if only _state column is requested if (!(has_state_column && real_column_names.empty())) - metadata_snapshot->check(real_column_names, {}); + metadata_snapshot->check(real_column_names, {}, getStorageID()); return has_state_column; } diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 26076d09914..27a9cd0c4bb 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -66,7 +66,7 @@ Pipes StorageSystemReplicas::read( const size_t /*max_block_size*/, const unsigned /*num_streams*/) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); const auto access = context.getAccess(); const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); diff --git a/src/Storages/System/StorageSystemStoragePolicies.cpp b/src/Storages/System/StorageSystemStoragePolicies.cpp index e589955c861..ec771ec2421 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.cpp +++ b/src/Storages/System/StorageSystemStoragePolicies.cpp @@ -39,7 +39,7 @@ Pipes StorageSystemStoragePolicies::read( const size_t /*max_block_size*/, const unsigned /*num_streams*/) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); MutableColumnPtr col_policy_name = ColumnString::create(); MutableColumnPtr col_volume_name = ColumnString::create(); diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index deb8f0551ea..4b78416dabb 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -458,7 +458,7 @@ Pipes StorageSystemTables::read( const size_t max_block_size, const unsigned /*num_streams*/) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); /// Create a mask of what columns are needed in the result. diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index 9489ff249a5..3839439794b 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -99,7 +99,7 @@ Pipes StorageSystemZeros::read( size_t max_block_size, unsigned num_streams) { - metadata_snapshot->check(column_names, getVirtuals()); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); bool use_multiple_streams = multithreaded; From 85070ea2fd5e6a42d6ef5c4d3ab3aff909ce9652 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Jun 2020 22:41:24 +0300 Subject: [PATCH 108/211] Remove unused variable --- src/Interpreters/InterpreterSelectQuery.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 187fdeb2b19..c9715523174 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -497,9 +497,6 @@ Block InterpreterSelectQuery::getSampleBlockImpl() bool second_stage = from_stage <= QueryProcessingStage::WithMergeableState && options.to_stage > QueryProcessingStage::WithMergeableState; - Names columns_required_for_sampling; - Names columns_required_for_; - analysis_result = ExpressionAnalysisResult( *query_analyzer, metadata_snapshot, From 2a5a8e70f4cc32e8628d5b500f5c314ad7ece29d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 14 Jun 2020 01:01:58 +0300 Subject: [PATCH 109/211] Drop unused PoolWithFailoverBase<>::reportError() --- src/Common/PoolWithFailoverBase.h | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 4d9e52364b9..cb598163ab7 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -113,8 +113,6 @@ public: const GetPriorityFunc & get_priority = GetPriorityFunc(), bool fallback_to_stale_replicas = true); - void reportError(const Entry & entry); - protected: struct PoolState; @@ -295,22 +293,6 @@ PoolWithFailoverBase::getMany( return try_results; } -template -void PoolWithFailoverBase::reportError(const Entry & entry) -{ - for (size_t i = 0; i < nested_pools.size(); ++i) - { - if (nested_pools[i]->contains(entry)) - { - std::lock_guard lock(pool_states_mutex); - auto & pool_state = shared_pool_states[i]; - pool_state.error_count = std::min(max_error_cap, pool_state.error_count + 1); - return; - } - } - throw DB::Exception("Can't find pool to report error", DB::ErrorCodes::LOGICAL_ERROR); -} - template struct PoolWithFailoverBase::PoolState { From 687eb24f50d8ee660575ca007e95d72fb2a1e84c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 13 Jun 2020 16:39:21 +0300 Subject: [PATCH 110/211] Fix divizion by zero for distributed_replica_error_half_life=0 --- src/Common/PoolWithFailoverBase.h | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index cb598163ab7..458a4d7783c 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -336,14 +336,17 @@ PoolWithFailoverBase::updatePoolStates() if (delta >= 0) { + const UInt64 MAX_BITS = sizeof(UInt64) * CHAR_BIT; + size_t shift_amount = MAX_BITS; /// Divide error counts by 2 every decrease_error_period seconds. - size_t shift_amount = delta / decrease_error_period; + if (decrease_error_period) + shift_amount = delta / decrease_error_period; /// Update time but don't do it more often than once a period. /// Else if the function is called often enough, error count will never decrease. if (shift_amount) last_error_decrease_time = current_time; - if (shift_amount >= sizeof(UInt64) * CHAR_BIT) + if (shift_amount >= MAX_BITS) { for (auto & state : shared_pool_states) state.error_count = 0; From caa195c0348f2d1753f4dd67c80982c330062dec Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 13 Jun 2020 20:32:21 +0300 Subject: [PATCH 111/211] Add number of errors to ignore while choosing replicas (distributed_replica_max_ignored_errors) This will allow avoid switching to another replica in case of error (since error can be temporary). --- docs/en/operations/settings/settings.md | 22 ++++++++++++++++++++++ src/Client/ConnectionPoolWithFailover.cpp | 5 ++++- src/Common/PoolWithFailoverBase.h | 17 ++++++++++++----- src/Core/Defines.h | 2 ++ src/Core/Settings.h | 1 + 5 files changed, 41 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ce98291e700..34d5f04a381 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -821,6 +821,10 @@ ClickHouse supports the following algorithms of choosing replicas: - [First or random](#load_balancing-first_or_random) - [Round robin](#load_balancing-round_robin) +See also: + +- [distributed\_replica\_max\_ignored\_errors](#settings-distributed_replica_max_ignored_errors) + ### Random (by Default) {#load_balancing-random} ``` sql @@ -1170,8 +1174,10 @@ Controls how fast errors in distributed tables are zeroed. If a replica is unava See also: +- [load\_balancing](#load_balancing-round_robin) - [Table engine Distributed](../../engines/table-engines/special/distributed.md) - [distributed\_replica\_error\_cap](#settings-distributed_replica_error_cap) +- [distributed\_replica\_max\_ignored\_errors](#settings-distributed_replica_max_ignored_errors) ## distributed\_replica\_error\_cap {#settings-distributed_replica_error_cap} @@ -1182,8 +1188,24 @@ Error count of each replica is capped at this value, preventing a single replica See also: +- [load\_balancing](#load_balancing-round_robin) - [Table engine Distributed](../../engines/table-engines/special/distributed.md) - [distributed\_replica\_error\_half\_life](#settings-distributed_replica_error_half_life) +- [distributed\_replica\_max\_ignored\_errors](#settings-distributed_replica_max_ignored_errors) + +## distributed\_replica\_max\_ignored\_errors {#settings-distributed_replica_max_ignored_errors} + +- Type: unsigned int +- Default value: 0 + +Number of errors that will be ignored while choosing replicas (according to `load_balancing` algorithm). + +See also: + +- [load\_balancing](#load_balancing-round_robin) +- [Table engine Distributed](../../engines/table-engines/special/distributed.md) +- [distributed\_replica\_error\_cap](#settings-distributed_replica_error_cap) +- [distributed\_replica\_error\_half\_life](#settings-distributed_replica_error_half_life) ## distributed\_directory\_monitor\_sleep\_time\_ms {#distributed_directory_monitor_sleep_time_ms} diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index 92e7d483b06..c2a0dcad668 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -208,7 +208,10 @@ std::vector ConnectionPoolWithFailover::g bool fallback_to_stale_replicas = settings ? bool(settings->fallback_to_stale_replicas_for_distributed_queries) : true; - return Base::getMany(min_entries, max_entries, max_tries, try_get_entry, get_priority, fallback_to_stale_replicas); + return Base::getMany(min_entries, max_entries, max_tries, + try_get_entry, get_priority, + fallback_to_stale_replicas, + settings ? settings->distributed_replica_max_ignored_errors.value : 0); } ConnectionPoolWithFailover::TryResult diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 458a4d7783c..b17e35996a0 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -111,7 +111,8 @@ public: size_t min_entries, size_t max_entries, size_t max_tries, const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority = GetPriorityFunc(), - bool fallback_to_stale_replicas = true); + bool fallback_to_stale_replicas = true, + size_t max_ignored_errors = 0); protected: struct PoolState; @@ -119,7 +120,7 @@ protected: using PoolStates = std::vector; /// This function returns a copy of pool states to avoid race conditions when modifying shared pool states. - PoolStates updatePoolStates(); + PoolStates updatePoolStates(size_t max_ignored_errors); PoolStates getPoolStates() const; NestedPools nested_pools; @@ -153,10 +154,11 @@ PoolWithFailoverBase::getMany( size_t min_entries, size_t max_entries, size_t max_tries, const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority, - bool fallback_to_stale_replicas) + bool fallback_to_stale_replicas, + size_t max_ignored_errors) { /// Update random numbers and error counts. - PoolStates pool_states = updatePoolStates(); + PoolStates pool_states = updatePoolStates(max_ignored_errors); if (get_priority) { for (size_t i = 0; i < pool_states.size(); ++i) @@ -317,7 +319,7 @@ private: template typename PoolWithFailoverBase::PoolStates -PoolWithFailoverBase::updatePoolStates() +PoolWithFailoverBase::updatePoolStates(size_t max_ignored_errors) { PoolStates result; result.reserve(nested_pools.size()); @@ -363,6 +365,11 @@ PoolWithFailoverBase::updatePoolStates() result.assign(shared_pool_states.begin(), shared_pool_states.end()); } + + /// distributed_replica_max_ignored_errors + for (auto & state : result) + state.error_count = std::max(0, state.error_count - max_ignored_errors); + return result; } diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 8b26f486c9d..bff782ea1b5 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -45,6 +45,8 @@ #define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD 60 /// replica error max cap, this is to prevent replica from accumulating too many errors and taking to long to recover. #define DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT 1000 +/// Number of errors that will be ignored while choosing replicas +#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_ERROR_IGNORE 0 #define DBMS_MIN_REVISION_WITH_CLIENT_INFO 54032 #define DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE 54058 diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1f3a8f42400..cf298996c58 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -348,6 +348,7 @@ struct Settings : public SettingsCollection \ M(SettingSeconds, distributed_replica_error_half_life, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD, "Time period reduces replica error counter by 2 times.", 0) \ M(SettingUInt64, distributed_replica_error_cap, DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT, "Max number of errors per replica, prevents piling up an incredible amount of errors if replica was offline for some time and allows it to be reconsidered in a shorter amount of time.", 0) \ + M(SettingUInt64, distributed_replica_max_ignored_errors, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_ERROR_IGNORE, "Number of errors that will be ignored while choosing replicas", 0) \ \ M(SettingBool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \ M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.", 0) \ From 9bfda6513431c4f0a752dab1c5dcb014d7631fcf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 13 Jun 2020 23:01:17 +0300 Subject: [PATCH 112/211] Add a test for distributed_replica_max_ignored_errors --- .../test_distributed_load_balancing/test.py | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/tests/integration/test_distributed_load_balancing/test.py b/tests/integration/test_distributed_load_balancing/test.py index fa6dfb20a88..4f760c705c3 100644 --- a/tests/integration/test_distributed_load_balancing/test.py +++ b/tests/integration/test_distributed_load_balancing/test.py @@ -112,3 +112,39 @@ def test_load_balancing_round_robin(): unique_nodes.add(get_node(n1, settings={'load_balancing': 'round_robin'})) assert len(unique_nodes) == nodes, unique_nodes assert unique_nodes == set(['n1', 'n2', 'n3']) + +def test_distributed_replica_max_ignored_errors(): + settings = { + 'load_balancing': 'in_order', + 'prefer_localhost_replica': 0, + 'connect_timeout': 2, + 'receive_timeout': 2, + 'send_timeout': 2, + 'idle_connection_timeout': 2, + 'tcp_keep_alive_timeout': 2, + + 'distributed_replica_max_ignored_errors': 0, + 'distributed_replica_error_half_life': 60, + } + + # initiate connection (if started only this test) + n2.query('SELECT * FROM dist', settings=settings) + cluster.pause_container('n1') + + # n1 paused -- skipping, and increment error_count for n1 + # but the query succeeds, no need in query_and_get_error() + n2.query('SELECT * FROM dist', settings=settings) + # XXX: due to config reloading we need second time (sigh) + n2.query('SELECT * FROM dist', settings=settings) + # check error_count for n1 + assert int(n2.query(""" + SELECT errors_count FROM system.clusters + WHERE cluster = 'replicas_cluster' AND host_name = 'n1' + """, settings=settings)) == 1 + + cluster.unpause_container('n1') + # still n2 + assert get_node(n2, settings=settings) == 'n2' + # now n1 + settings['distributed_replica_max_ignored_errors'] = 1 + assert get_node(n2, settings=settings) == 'n1' From dd054380bf9043f74531d77efe06d39ae496f632 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 Jun 2020 11:13:25 +0300 Subject: [PATCH 113/211] Drop default values for some arguments of PoolWithFailoverBase::getMany --- src/Client/ConnectionPoolWithFailover.cpp | 8 ++++---- src/Common/PoolWithFailoverBase.h | 14 +++++++------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index c2a0dcad668..d6457cf4024 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -206,12 +206,12 @@ std::vector ConnectionPoolWithFailover::g break; } - bool fallback_to_stale_replicas = settings ? bool(settings->fallback_to_stale_replicas_for_distributed_queries) : true; + bool fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries.value : true; + UInt64 max_ignored_errors = settings ? settings->distributed_replica_max_ignored_errors.value : 0; return Base::getMany(min_entries, max_entries, max_tries, - try_get_entry, get_priority, - fallback_to_stale_replicas, - settings ? settings->distributed_replica_max_ignored_errors.value : 0); + max_ignored_errors, fallback_to_stale_replicas, + try_get_entry, get_priority); } ConnectionPoolWithFailover::TryResult diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index b17e35996a0..2a8117f6f2d 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -109,10 +109,10 @@ public: /// if fallback_to_stale_replicas is false and it is unable to get min_entries connections to up-to-date replicas. std::vector getMany( size_t min_entries, size_t max_entries, size_t max_tries, + size_t max_ignored_errors, + bool fallback_to_stale_replicas, const TryGetEntryFunc & try_get_entry, - const GetPriorityFunc & get_priority = GetPriorityFunc(), - bool fallback_to_stale_replicas = true, - size_t max_ignored_errors = 0); + const GetPriorityFunc & get_priority = GetPriorityFunc()); protected: struct PoolState; @@ -140,7 +140,7 @@ template typename TNestedPool::Entry PoolWithFailoverBase::get(const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority) { - std::vector results = getMany(1, 1, 1, try_get_entry, get_priority); + std::vector results = getMany(1, 1, 1, 0, true, try_get_entry, get_priority); if (results.empty() || results[0].entry.isNull()) throw DB::Exception( "PoolWithFailoverBase::getMany() returned less than min_entries entries.", @@ -152,10 +152,10 @@ template std::vector::TryResult> PoolWithFailoverBase::getMany( size_t min_entries, size_t max_entries, size_t max_tries, - const TryGetEntryFunc & try_get_entry, - const GetPriorityFunc & get_priority, + size_t max_ignored_errors, bool fallback_to_stale_replicas, - size_t max_ignored_errors) + const TryGetEntryFunc & try_get_entry, + const GetPriorityFunc & get_priority) { /// Update random numbers and error counts. PoolStates pool_states = updatePoolStates(max_ignored_errors); From de011a6edc13eddffd0c1300b364858defec2c0b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 Jun 2020 11:39:21 +0300 Subject: [PATCH 114/211] Mark PoolWithFailoverBase::get() protected --- src/Common/PoolWithFailoverBase.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 2a8117f6f2d..189fa05e894 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -100,9 +100,6 @@ public: /// this functor. The pools with lower result value will be tried first. using GetPriorityFunc = std::function; - /// Returns a single connection. - Entry get(const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority = GetPriorityFunc()); - /// Returns at least min_entries and at most max_entries connections (at most one connection per nested pool). /// The method will throw if it is unable to get min_entries alive connections or @@ -119,6 +116,9 @@ protected: using PoolStates = std::vector; + /// Returns a single connection. + Entry get(const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority = GetPriorityFunc()); + /// This function returns a copy of pool states to avoid race conditions when modifying shared pool states. PoolStates updatePoolStates(size_t max_ignored_errors); PoolStates getPoolStates() const; From b8ee2ea7d9e447284f28da564cd24ca025d6ebdf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 Jun 2020 11:44:48 +0300 Subject: [PATCH 115/211] Pass max_ignored_errors/fallback_to_stale_replicas to PoolWithFailoverBase::get() too --- src/Client/ConnectionPoolWithFailover.cpp | 7 +++++-- src/Common/PoolWithFailoverBase.h | 11 ++++++++--- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index d6457cf4024..128a4836a42 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -84,7 +84,10 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts break; } - return Base::get(try_get_entry, get_priority); + UInt64 max_ignored_errors = settings ? settings->distributed_replica_max_ignored_errors.value : 0; + bool fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries.value : true; + + return Base::get(max_ignored_errors, fallback_to_stale_replicas, try_get_entry, get_priority); } ConnectionPoolWithFailover::Status ConnectionPoolWithFailover::getStatus() const @@ -206,8 +209,8 @@ std::vector ConnectionPoolWithFailover::g break; } - bool fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries.value : true; UInt64 max_ignored_errors = settings ? settings->distributed_replica_max_ignored_errors.value : 0; + bool fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries.value : true; return Base::getMany(min_entries, max_entries, max_tries, max_ignored_errors, fallback_to_stale_replicas, diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 189fa05e894..5b4e736a907 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -117,7 +117,8 @@ protected: using PoolStates = std::vector; /// Returns a single connection. - Entry get(const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority = GetPriorityFunc()); + Entry get(size_t max_ignored_errors, bool fallback_to_stale_replicas, + const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority = GetPriorityFunc()); /// This function returns a copy of pool states to avoid race conditions when modifying shared pool states. PoolStates updatePoolStates(size_t max_ignored_errors); @@ -138,9 +139,13 @@ protected: template typename TNestedPool::Entry -PoolWithFailoverBase::get(const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority) +PoolWithFailoverBase::get(size_t max_ignored_errors, bool fallback_to_stale_replicas, + const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority) { - std::vector results = getMany(1, 1, 1, 0, true, try_get_entry, get_priority); + std::vector results = getMany( + 1 /* min entries */, 1 /* max entries */, 1 /* max tries */, + max_ignored_errors, fallback_to_stale_replicas, + try_get_entry, get_priority); if (results.empty() || results[0].entry.isNull()) throw DB::Exception( "PoolWithFailoverBase::getMany() returned less than min_entries entries.", From ead4a2cfd97720e0e363430b833aaf381b0cc637 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jun 2020 12:07:05 +0300 Subject: [PATCH 116/211] Print build id in crash messages --- base/daemon/BaseDaemon.cpp | 46 +++++++++++++++++++++----------------- base/daemon/BaseDaemon.h | 2 ++ src/Common/SymbolIndex.cpp | 28 ++++++++++++++++++++--- src/Common/SymbolIndex.h | 5 +++++ 4 files changed, 57 insertions(+), 24 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 2a36777218e..4f74de2e853 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -50,6 +50,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include @@ -236,7 +237,8 @@ private: void onTerminate(const std::string & message, UInt32 thread_num) const { - LOG_FATAL(log, "(version {}{}) (from thread {}) {}", VERSION_STRING, VERSION_OFFICIAL, thread_num, message); + LOG_FATAL(log, "(version {}{}, {}) (from thread {}) {}", + VERSION_STRING, VERSION_OFFICIAL, daemon.build_id_info, thread_num, message); } void onFault( @@ -249,17 +251,15 @@ private: { LOG_FATAL(log, "########################################"); + if (query_id.empty()) { - std::stringstream message; - message << "(version " << VERSION_STRING << VERSION_OFFICIAL << ")"; - message << " (from thread " << thread_num << ")"; - if (query_id.empty()) - message << " (no query)"; - else - message << " (query_id: " << query_id << ")"; - message << " Received signal " << strsignal(sig) << " (" << sig << ")."; - - LOG_FATAL(log, message.str()); + LOG_FATAL(log, "(version {}{}, {}) (from thread {}) (no query) Received signal {} ({})", + VERSION_STRING, VERSION_OFFICIAL, daemon.build_id_info, thread_num, strsignal(sig), sig); + } + else + { + LOG_FATAL(log, "(version {}{}, {}) (from thread {}) (query_id: {}) Received signal {} ({})", + VERSION_STRING, VERSION_OFFICIAL, daemon.build_id_info, thread_num, query_id, strsignal(sig), sig); } LOG_FATAL(log, signalToErrorMessage(sig, info, context)); @@ -292,17 +292,15 @@ static void sanitizerDeathCallback() StringRef query_id = DB::CurrentThread::getQueryId(); /// This is signal safe. + if (query_id.empty()) { - std::stringstream message; - message << "(version " << VERSION_STRING << VERSION_OFFICIAL << ")"; - message << " (from thread " << getThreadId() << ")"; - if (query_id.size == 0) - message << " (no query)"; - else - message << " (query_id: " << query_id << ")"; - message << " Sanitizer trap."; - - LOG_FATAL(log, message.str()); + LOG_FATAL(log, "(version {}{}, {}) (from thread {}) (no query) Sanitizer trap.", + VERSION_STRING, VERSION_OFFICIAL, daemon.build_id_info, thread_num); + } + else + { + LOG_FATAL(log, "(version {}{}, {}) (from thread {}) (query_id: {}) Sanitizer trap.", + VERSION_STRING, VERSION_OFFICIAL, daemon.build_id_info, thread_num, query_id); } /// Just in case print our own stack trace. In case when llvm-symbolizer does not work. @@ -711,6 +709,12 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() signal_listener = std::make_unique(*this); signal_listener_thread.start(*signal_listener); + + String build_id_hex = DB::SymbolIndex::instance().getBuildIDHex(); + if (build_id_hex.empty()) + build_id_info = "no build id"; + else + build_id_info = "build id: " + build_id_hex; } void BaseDaemon::logRevision() const diff --git a/base/daemon/BaseDaemon.h b/base/daemon/BaseDaemon.h index 39332cfe963..2a3262dd26f 100644 --- a/base/daemon/BaseDaemon.h +++ b/base/daemon/BaseDaemon.h @@ -198,6 +198,8 @@ protected: std::string config_path; DB::ConfigProcessor::LoadedConfig loaded_config; Poco::Util::AbstractConfiguration * last_configuration = nullptr; + + String build_id_info; }; diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index 54789695dd1..6f4bef0a3f9 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -1,6 +1,7 @@ #if defined(__ELF__) && !defined(__FreeBSD__) #include +#include #include #include @@ -276,15 +277,21 @@ bool searchAndCollectSymbolsFromELFSymbolTable( void collectSymbolsFromELF(dl_phdr_info * info, std::vector & symbols, - std::vector & objects) + std::vector & objects, + String & build_id) { std::string object_name = info->dlpi_name; + String our_build_id = getBuildIDFromProgramHeaders(info); + /// If the name is empty - it's main executable. /// Find a elf file for the main executable. if (object_name.empty()) + { object_name = "/proc/self/exe"; + build_id = our_build_id; + } std::error_code ec; std::filesystem::path canonical_path = std::filesystem::canonical(object_name, ec); @@ -298,7 +305,6 @@ void collectSymbolsFromELF(dl_phdr_info * info, object_name = std::filesystem::exists(debug_info_path) ? debug_info_path : canonical_path; /// But we have to compare Build ID to check that debug info corresponds to the same executable. - String our_build_id = getBuildIDFromProgramHeaders(info); SymbolIndex::Object object; object.elf = std::make_unique(object_name); @@ -343,7 +349,7 @@ int collectSymbols(dl_phdr_info * info, size_t, void * data_ptr) SymbolIndex::Data & data = *reinterpret_cast(data_ptr); collectSymbolsFromProgramHeaders(info, data.symbols); - collectSymbolsFromELF(info, data.symbols, data.objects); + collectSymbolsFromELF(info, data.symbols, data.objects, data.build_id); /* Continue iterations */ return 0; @@ -396,6 +402,22 @@ const SymbolIndex::Object * SymbolIndex::findObject(const void * address) const return find(address, data.objects); } +String SymbolIndex::getBuildIDHex() const +{ + String build_id_binary = getBuildID(); + String build_id_hex; + build_id_hex.resize(build_id_binary.size() * 2); + + char * pos = build_id_hex.data(); + for (auto c : build_id_binary) + { + writeHexByteUppercase(c, pos); + pos += 2; + } + + return build_id_hex; +} + SymbolIndex & SymbolIndex::instance() { static SymbolIndex instance; diff --git a/src/Common/SymbolIndex.h b/src/Common/SymbolIndex.h index 1e762780dad..8ef0a949515 100644 --- a/src/Common/SymbolIndex.h +++ b/src/Common/SymbolIndex.h @@ -45,10 +45,15 @@ public: const std::vector & symbols() const { return data.symbols; } const std::vector & objects() const { return data.objects; } + /// The BuildID that is generated by compiler. + String getBuildID() const { return data.build_id; } + String getBuildIDHex() const; + struct Data { std::vector symbols; std::vector objects; + String build_id; }; private: Data data; From 7fcdfa1f88d2858842e7d3f860c6b9b703778ecd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jun 2020 12:13:11 +0300 Subject: [PATCH 117/211] Print build id on startup --- base/daemon/BaseDaemon.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 4f74de2e853..6862f064b6e 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -721,6 +721,7 @@ void BaseDaemon::logRevision() const { Poco::Logger::root().information("Starting " + std::string{VERSION_FULL} + " with revision " + std::to_string(ClickHouseRevision::get()) + + ", " + build_id_info + ", PID " + std::to_string(getpid())); } From 2897a5407204acb6c16323be1a84302644bca924 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jun 2020 12:17:31 +0300 Subject: [PATCH 118/211] Fix build --- base/daemon/BaseDaemon.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 6862f064b6e..3c4d3d2f74f 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -710,11 +710,15 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() signal_listener = std::make_unique(*this); signal_listener_thread.start(*signal_listener); +#if defined(__ELF__) && !defined(__FreeBSD__) String build_id_hex = DB::SymbolIndex::instance().getBuildIDHex(); if (build_id_hex.empty()) build_id_info = "no build id"; else build_id_info = "build id: " + build_id_hex; +#else + build_id_info = "no build id"; +#endif } void BaseDaemon::logRevision() const From 5c2e9da5836f810d6961533157784cf85e45adf1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jun 2020 12:17:43 +0300 Subject: [PATCH 119/211] Added a function "buildId" --- src/Functions/buildId.cpp | 53 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 53 insertions(+) create mode 100644 src/Functions/buildId.cpp diff --git a/src/Functions/buildId.cpp b/src/Functions/buildId.cpp new file mode 100644 index 00000000000..f0745cc34c1 --- /dev/null +++ b/src/Functions/buildId.cpp @@ -0,0 +1,53 @@ +#if defined(__ELF__) && !defined(__FreeBSD__) + +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** buildId() - returns the compiler build id of the running binary. + */ +class FunctionVersion : public IFunction +{ +public: + static constexpr auto name = "buildId"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, SymbolIndex::instance().getBuildIDHex()); + } +}; + + +void registerFunctionVersion(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + +#endif From d0b948dc3fa4b070114bccce6fd946ca795172ed Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jun 2020 12:18:53 +0300 Subject: [PATCH 120/211] Added a test --- tests/queries/0_stateless/01326_build_id.reference | 1 + tests/queries/0_stateless/01326_build_id.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/01326_build_id.reference create mode 100644 tests/queries/0_stateless/01326_build_id.sql diff --git a/tests/queries/0_stateless/01326_build_id.reference b/tests/queries/0_stateless/01326_build_id.reference new file mode 100644 index 00000000000..b6a7d89c68e --- /dev/null +++ b/tests/queries/0_stateless/01326_build_id.reference @@ -0,0 +1 @@ +16 diff --git a/tests/queries/0_stateless/01326_build_id.sql b/tests/queries/0_stateless/01326_build_id.sql new file mode 100644 index 00000000000..f9bf2731374 --- /dev/null +++ b/tests/queries/0_stateless/01326_build_id.sql @@ -0,0 +1 @@ +SELECT length(buildId()); From 9f42e81ef99e314f56eb5f8d72d14c5f71aaf0e3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jun 2020 13:10:18 +0300 Subject: [PATCH 121/211] Send logs on fatal errors (development) --- base/daemon/BaseDaemon.cpp | 26 +++++++++++++++++++++++--- src/Functions/trap.cpp | 4 +++- 2 files changed, 26 insertions(+), 4 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 2a36777218e..d37d1d26fd5 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -83,7 +83,8 @@ static const size_t signal_pipe_buf_size = + sizeof(ucontext_t) + sizeof(StackTrace) + sizeof(UInt32) - + max_query_id_size + 1; /// query_id + varint encoded length + + max_query_id_size + 1 /// query_id + varint encoded length + + sizeof(void*); using signal_function = void(int, siginfo_t*, void*); @@ -133,6 +134,7 @@ static void signalHandler(int sig, siginfo_t * info, void * context) DB::writePODBinary(stack_trace, out); DB::writeBinary(UInt32(getThreadId()), out); DB::writeStringBinary(query_id, out); +// DB::writePODBinary(DB::current_thread, out); out.next(); @@ -140,6 +142,9 @@ static void signalHandler(int sig, siginfo_t * info, void * context) { /// The time that is usually enough for separate thread to print info into log. ::sleep(10); + + //std::cerr << "signalHandler: " << static_cast(DB::current_thread) << ", " << !!DB::current_thread->getInternalTextLogsQueue() << "\n"; + call_default_signal_handler(sig); } @@ -216,16 +221,20 @@ public: StackTrace stack_trace(NoCapture{}); UInt32 thread_num; std::string query_id; + const DB::CurrentThread * thread_ptr{}; DB::readPODBinary(info, in); DB::readPODBinary(context, in); DB::readPODBinary(stack_trace, in); DB::readBinary(thread_num, in); DB::readBinary(query_id, in); +// DB::readPODBinary(thread_ptr, in); + + std::cerr << "Read " << static_cast(thread_ptr) << "\n"; /// This allows to receive more signals if failure happens inside onFault function. /// Example: segfault while symbolizing stack trace. - std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_num, query_id); }).detach(); + std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_num, query_id, thread_ptr); }).detach(); } } } @@ -245,8 +254,19 @@ private: const ucontext_t & context, const StackTrace & stack_trace, UInt32 thread_num, - const std::string & query_id) const + const std::string & query_id, + const DB::CurrentThread * /*thread_ptr*/) const { + /// Send logs from this thread to client if possible. + /// It will allow client to see failure messages directly. +/* if (thread_ptr) + { + std::cerr << static_cast(thread_ptr) << ", " << !!thread_ptr->getInternalTextLogsQueue() << "\n"; + + if (auto logs_queue = thread_ptr->getInternalTextLogsQueue()) + DB::CurrentThread::attachInternalTextLogsQueue(logs_queue, DB::LogsLevel::trace); + }*/ + LOG_FATAL(log, "########################################"); { diff --git a/src/Functions/trap.cpp b/src/Functions/trap.cpp index 94aa56919b4..65aa9b8d3e3 100644 --- a/src/Functions/trap.cpp +++ b/src/Functions/trap.cpp @@ -1,4 +1,4 @@ -#if 0 +#if 1 #include #include @@ -63,6 +63,8 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { + std::cerr << "executeImpl: " << !!CurrentThread::getInternalTextLogsQueue() << "\n"; + if (const ColumnConst * column = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get())) { String mode = column->getValue(); From 826fdf44ae5b6f923f23ea95ec83ea44b5fac50f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jun 2020 14:28:49 +0300 Subject: [PATCH 122/211] Set send_logs_level to fatal by default --- src/Core/Settings.h | 2 +- src/Core/SettingsCollection.cpp | 1 + src/Core/SettingsCollection.h | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1f3a8f42400..5c499a45c8f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -316,7 +316,7 @@ struct Settings : public SettingsCollection M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.", 0) \ M(SettingBool, log_query_settings, true, "Log query settings into the query_log.", 0) \ M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \ - M(SettingLogsLevel, send_logs_level, LogsLevel::none, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'", 0) \ + M(SettingLogsLevel, send_logs_level, LogsLevel::fatal, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \ M(SettingBool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.", 0) \ M(SettingBool, enable_optimize_predicate_expression_to_final_subquery, 1, "Allow push predicate to final subquery.", 0) \ \ diff --git a/src/Core/SettingsCollection.cpp b/src/Core/SettingsCollection.cpp index b36884fad22..32bf1f29c90 100644 --- a/src/Core/SettingsCollection.cpp +++ b/src/Core/SettingsCollection.cpp @@ -542,6 +542,7 @@ IMPLEMENT_SETTING_ENUM(FormatSettings::DateTimeInputFormat, DATE_TIME_INPUT_FORM #define LOGS_LEVEL_LIST_OF_NAMES(M) \ M(none, "none") \ + M(fatal, "fatal") \ M(error, "error") \ M(warning, "warning") \ M(information, "information") \ diff --git a/src/Core/SettingsCollection.h b/src/Core/SettingsCollection.h index 71a308fb37e..b1378e01c32 100644 --- a/src/Core/SettingsCollection.h +++ b/src/Core/SettingsCollection.h @@ -302,6 +302,7 @@ using SettingDateTimeInputFormat = SettingEnum Date: Sat, 20 Jun 2020 14:29:01 +0300 Subject: [PATCH 123/211] Update tests --- tests/queries/0_stateless/00002_system_numbers.sql | 2 +- .../queries/0_stateless/00158_buffer_and_nonexistent_table.sql | 2 +- tests/queries/0_stateless/00183_skip_unavailable_shards.sql | 2 +- tests/queries/0_stateless/00205_scalar_subqueries.sql | 2 +- tests/queries/0_stateless/00233_position_function_family.sql | 2 +- tests/queries/0_stateless/00492_drop_temporary_table.sql | 2 +- tests/queries/0_stateless/00502_sum_map.sql | 2 +- tests/queries/0_stateless/00597_push_down_predicate.sql | 2 +- tests/queries/0_stateless/00647_multiply_aggregation_state.sql | 2 +- tests/queries/0_stateless/00692_if_exception_code.sql | 2 +- tests/queries/0_stateless/00694_max_block_size_zero.sql | 2 +- .../0_stateless/00698_validate_array_sizes_for_nested.sql | 2 +- .../00698_validate_array_sizes_for_nested_kshvakov.sql | 2 +- tests/queries/0_stateless/00701_join_default_strictness.sql | 2 +- tests/queries/0_stateless/00705_aggregate_states_addition.sql | 2 +- tests/queries/0_stateless/00714_alter_uuid.sql | 2 +- tests/queries/0_stateless/00716_allow_ddl.sql | 2 +- tests/queries/0_stateless/00717_merge_and_distributed.sql | 2 +- tests/queries/0_stateless/00718_format_datetime.sql | 2 +- tests/queries/0_stateless/00729_prewhere_array_join.sql | 2 +- tests/queries/0_stateless/00732_base64_functions.sql | 2 +- .../00732_quorum_insert_have_data_before_quorum_zookeeper.sql | 2 +- .../00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql | 2 +- .../0_stateless/00732_quorum_insert_lost_part_zookeeper.sql | 2 +- ...insert_select_with_old_data_and_without_quorum_zookeeper.sql | 2 +- .../00732_quorum_insert_simple_test_1_parts_zookeeper.sql | 2 +- .../00732_quorum_insert_simple_test_2_parts_zookeeper.sql | 2 +- tests/queries/0_stateless/00735_conditional.sql | 2 +- tests/queries/0_stateless/00742_require_join_strictness.sql | 2 +- .../0_stateless/00754_alter_modify_column_partitions.sql | 2 +- tests/queries/0_stateless/00754_alter_modify_order_by.sql | 2 +- .../00754_alter_modify_order_by_replicated_zookeeper.sql | 2 +- tests/queries/0_stateless/00758_array_reverse.sql | 2 +- tests/queries/0_stateless/00762_date_comparsion.sql | 2 +- tests/queries/0_stateless/00765_sql_compatibility_aliases.sql | 2 +- .../queries/0_stateless/00804_test_alter_compression_codecs.sql | 2 +- .../0_stateless/00804_test_custom_compression_codecs.sql | 2 +- .../00804_test_custom_compression_codes_log_storages.sql | 2 +- .../queries/0_stateless/00804_test_delta_codec_compression.sql | 2 +- .../0_stateless/00804_test_delta_codec_no_type_alter.sql | 2 +- tests/queries/0_stateless/00805_round_down.sql | 2 +- tests/queries/0_stateless/00808_array_enumerate_segfault.sql | 2 +- tests/queries/0_stateless/00808_not_optimize_predicate.sql | 2 +- tests/queries/0_stateless/00809_add_days_segfault.sql | 2 +- tests/queries/0_stateless/00810_in_operators_segfault.sql | 2 +- tests/queries/0_stateless/00811_garbage.sql | 2 +- .../00910_zookeeper_custom_compression_codecs_replicated.sql | 2 +- .../00910_zookeeper_test_alter_compression_codecs.sql | 2 +- .../queries/0_stateless/00926_adaptive_index_granularity_pk.sql | 2 +- tests/queries/0_stateless/00929_multi_match_edit_distance.sql | 2 +- tests/queries/0_stateless/00933_alter_ttl.sql | 2 +- tests/queries/0_stateless/00933_ttl_simple.sql | 2 +- tests/queries/0_stateless/00949_format.sql | 2 +- tests/queries/0_stateless/00955_test_final_mark.sql | 2 +- tests/queries/0_stateless/01018_ddl_dictionaries_create.sql | 2 +- tests/queries/0_stateless/01018_ddl_dictionaries_select.sql | 2 +- tests/queries/0_stateless/01018_ddl_dictionaries_special.sql | 2 +- .../0_stateless/01018_dictionaries_from_dictionaries.sql | 2 +- tests/queries/0_stateless/01033_dictionaries_lifetime.sql | 2 +- tests/queries/0_stateless/01037_polygon_dict_multi_polygons.sql | 2 +- .../queries/0_stateless/01037_polygon_dict_simple_polygons.sql | 2 +- .../0_stateless/01037_zookeeper_check_table_empty_pk.sql | 2 +- tests/queries/0_stateless/01038_array_of_unnamed_tuples.sql | 2 +- tests/queries/0_stateless/01115_join_with_dictionary.sql | 2 +- .../01247_dist_on_dist_group_by_sharding_key_optimization.sql | 2 +- ...ibuted_group_by_no_merge_GROUP_BY_injective_sharding_key.sql | 2 +- 66 files changed, 66 insertions(+), 66 deletions(-) diff --git a/tests/queries/0_stateless/00002_system_numbers.sql b/tests/queries/0_stateless/00002_system_numbers.sql index 8f1580e9127..95f75573201 100644 --- a/tests/queries/0_stateless/00002_system_numbers.sql +++ b/tests/queries/0_stateless/00002_system_numbers.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT * FROM system.numbers LIMIT 3; SELECT sys_num.number FROM system.numbers AS sys_num WHERE number > 2 LIMIT 2; diff --git a/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql b/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql index 5fc41e30889..8d5d297b629 100644 --- a/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql +++ b/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql @@ -2,7 +2,7 @@ CREATE DATABASE IF NOT EXISTS test2_00158; DROP TABLE IF EXISTS test2_00158.mt_buffer_00158; DROP TABLE IF EXISTS test2_00158.mt_00158; CREATE TABLE test2_00158.mt_buffer_00158 (d Date DEFAULT today(), x UInt64) ENGINE = Buffer(test2_00158, mt_00158, 16, 100, 100, 1000000, 1000000, 1000000000, 1000000000); -SET send_logs_level = 'none'; -- Supress "Destination table test2.mt doesn't exist. Block of data is discarded." +SET send_logs_level = 'fatal'; -- Supress "Destination table test2.mt doesn't exist. Block of data is discarded." INSERT INTO test2_00158.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 100000; INSERT INTO test2_00158.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 1000000; DROP TABLE IF EXISTS test2_00158.mt_buffer_00158; diff --git a/tests/queries/0_stateless/00183_skip_unavailable_shards.sql b/tests/queries/0_stateless/00183_skip_unavailable_shards.sql index 2f2fe407400..4aa7cc72605 100644 --- a/tests/queries/0_stateless/00183_skip_unavailable_shards.sql +++ b/tests/queries/0_stateless/00183_skip_unavailable_shards.sql @@ -1,3 +1,3 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT count() FROM remote('{127,1}.0.0.{2,3}', system.one) SETTINGS skip_unavailable_shards = 1; SELECT count() FROM remote('{1,127}.0.0.{2,3}', system.one) SETTINGS skip_unavailable_shards = 1; diff --git a/tests/queries/0_stateless/00205_scalar_subqueries.sql b/tests/queries/0_stateless/00205_scalar_subqueries.sql index 35febd5a75b..14244377e5f 100644 --- a/tests/queries/0_stateless/00205_scalar_subqueries.sql +++ b/tests/queries/0_stateless/00205_scalar_subqueries.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT (SELECT (SELECT (SELECT (SELECT (SELECT count() FROM (SELECT * FROM system.numbers LIMIT 10)))))) = (SELECT 10), ((SELECT 1, 'Hello', [1, 2]).3)[1]; SELECT toUInt64((SELECT 9)) IN (SELECT number FROM system.numbers LIMIT 10); diff --git a/tests/queries/0_stateless/00233_position_function_family.sql b/tests/queries/0_stateless/00233_position_function_family.sql index 874e093d3c5..1bfd17310e1 100644 --- a/tests/queries/0_stateless/00233_position_function_family.sql +++ b/tests/queries/0_stateless/00233_position_function_family.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; select 1 = position('', ''); select 1 = position('abc', ''); select 0 = position('', 'abc'); diff --git a/tests/queries/0_stateless/00492_drop_temporary_table.sql b/tests/queries/0_stateless/00492_drop_temporary_table.sql index 5bc3652a411..a065b7f225d 100644 --- a/tests/queries/0_stateless/00492_drop_temporary_table.sql +++ b/tests/queries/0_stateless/00492_drop_temporary_table.sql @@ -2,7 +2,7 @@ DROP TEMPORARY TABLE IF EXISTS temp_tab; CREATE TEMPORARY TABLE temp_tab (number UInt64); INSERT INTO temp_tab SELECT number FROM system.numbers LIMIT 1; SELECT number FROM temp_tab; -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; EXISTS TEMPORARY TABLE temp_tab; DROP TABLE temp_tab; EXISTS TEMPORARY TABLE temp_tab; diff --git a/tests/queries/0_stateless/00502_sum_map.sql b/tests/queries/0_stateless/00502_sum_map.sql index 6a4035a3782..021aaf3cd3b 100644 --- a/tests/queries/0_stateless/00502_sum_map.sql +++ b/tests/queries/0_stateless/00502_sum_map.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS sum_map; CREATE TABLE sum_map(date Date, timeslot DateTime, statusMap Nested(status UInt16, requests UInt64)) ENGINE = Log; diff --git a/tests/queries/0_stateless/00597_push_down_predicate.sql b/tests/queries/0_stateless/00597_push_down_predicate.sql index 774c855c369..7fd30a82d8d 100644 --- a/tests/queries/0_stateless/00597_push_down_predicate.sql +++ b/tests/queries/0_stateless/00597_push_down_predicate.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET any_join_distinct_right_table_keys = 1; SET joined_subquery_requires_alias = 0; diff --git a/tests/queries/0_stateless/00647_multiply_aggregation_state.sql b/tests/queries/0_stateless/00647_multiply_aggregation_state.sql index ebf0c22d9d0..b0361458221 100644 --- a/tests/queries/0_stateless/00647_multiply_aggregation_state.sql +++ b/tests/queries/0_stateless/00647_multiply_aggregation_state.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT countMerge(x) AS y FROM ( SELECT countState() * 2 AS x FROM ( SELECT 1 )); SELECT countMerge(x) AS y FROM ( SELECT countState() * 0 AS x FROM ( SELECT 1 UNION ALL SELECT 2)); diff --git a/tests/queries/0_stateless/00692_if_exception_code.sql b/tests/queries/0_stateless/00692_if_exception_code.sql index c37b877648d..f9d06f2e3a5 100644 --- a/tests/queries/0_stateless/00692_if_exception_code.sql +++ b/tests/queries/0_stateless/00692_if_exception_code.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT if(); -- { serverError 42 } SELECT if(1); -- { serverError 42 } diff --git a/tests/queries/0_stateless/00694_max_block_size_zero.sql b/tests/queries/0_stateless/00694_max_block_size_zero.sql index d0dc826160f..ba5b513bb5d 100644 --- a/tests/queries/0_stateless/00694_max_block_size_zero.sql +++ b/tests/queries/0_stateless/00694_max_block_size_zero.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET max_block_size = 0; SELECT number FROM system.numbers; -- { serverError 12 } diff --git a/tests/queries/0_stateless/00698_validate_array_sizes_for_nested.sql b/tests/queries/0_stateless/00698_validate_array_sizes_for_nested.sql index 8bdc7c4c1a1..ec238797dca 100644 --- a/tests/queries/0_stateless/00698_validate_array_sizes_for_nested.sql +++ b/tests/queries/0_stateless/00698_validate_array_sizes_for_nested.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS mergetree_00698; CREATE TABLE mergetree_00698 (k UInt32, `n.x` Array(UInt64), `n.y` Array(UInt64)) ENGINE = MergeTree ORDER BY k; diff --git a/tests/queries/0_stateless/00698_validate_array_sizes_for_nested_kshvakov.sql b/tests/queries/0_stateless/00698_validate_array_sizes_for_nested_kshvakov.sql index 2d8dcc1f4f3..010d53dbcac 100644 --- a/tests/queries/0_stateless/00698_validate_array_sizes_for_nested_kshvakov.sql +++ b/tests/queries/0_stateless/00698_validate_array_sizes_for_nested_kshvakov.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS Issue_2231_Invalid_Nested_Columns_Size; CREATE TABLE Issue_2231_Invalid_Nested_Columns_Size ( diff --git a/tests/queries/0_stateless/00701_join_default_strictness.sql b/tests/queries/0_stateless/00701_join_default_strictness.sql index 873911a944d..6a10b4b88ad 100644 --- a/tests/queries/0_stateless/00701_join_default_strictness.sql +++ b/tests/queries/0_stateless/00701_join_default_strictness.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS a1; DROP TABLE IF EXISTS a2; -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; CREATE TABLE a1(a UInt8, b UInt8) ENGINE=Memory; CREATE TABLE a2(a UInt8, b UInt8) ENGINE=Memory; diff --git a/tests/queries/0_stateless/00705_aggregate_states_addition.sql b/tests/queries/0_stateless/00705_aggregate_states_addition.sql index a2d9660955d..661197d15ef 100644 --- a/tests/queries/0_stateless/00705_aggregate_states_addition.sql +++ b/tests/queries/0_stateless/00705_aggregate_states_addition.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS add_aggregate; CREATE TABLE add_aggregate(a UInt32, b UInt32) ENGINE = Memory; diff --git a/tests/queries/0_stateless/00714_alter_uuid.sql b/tests/queries/0_stateless/00714_alter_uuid.sql index 00a8f793e0e..ab08e943175 100644 --- a/tests/queries/0_stateless/00714_alter_uuid.sql +++ b/tests/queries/0_stateless/00714_alter_uuid.sql @@ -37,7 +37,7 @@ ENGINE = MergeTree PARTITION BY toDate(created_at) ORDER BY (created_at, id0, id1); -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; ALTER TABLE uuid MODIFY COLUMN id0 UUID; -- { serverError 524 } ALTER TABLE uuid MODIFY COLUMN id1 UUID; -- { serverError 524 } diff --git a/tests/queries/0_stateless/00716_allow_ddl.sql b/tests/queries/0_stateless/00716_allow_ddl.sql index c060bf37290..d33d8b7eec5 100755 --- a/tests/queries/0_stateless/00716_allow_ddl.sql +++ b/tests/queries/0_stateless/00716_allow_ddl.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET allow_ddl = 0; CREATE DATABASE some_db; -- { serverError 392 } diff --git a/tests/queries/0_stateless/00717_merge_and_distributed.sql b/tests/queries/0_stateless/00717_merge_and_distributed.sql index 908df53db44..f0d34b5165f 100644 --- a/tests/queries/0_stateless/00717_merge_and_distributed.sql +++ b/tests/queries/0_stateless/00717_merge_and_distributed.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS test_local_1; DROP TABLE IF EXISTS test_local_2; diff --git a/tests/queries/0_stateless/00718_format_datetime.sql b/tests/queries/0_stateless/00718_format_datetime.sql index 21dc504985d..cd679fe9735 100644 --- a/tests/queries/0_stateless/00718_format_datetime.sql +++ b/tests/queries/0_stateless/00718_format_datetime.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT formatDateTime(); -- { serverError 42 } SELECT formatDateTime('not a datetime', 'IGNORED'); -- { serverError 43 } diff --git a/tests/queries/0_stateless/00729_prewhere_array_join.sql b/tests/queries/0_stateless/00729_prewhere_array_join.sql index 2fdd66c11f7..ba10dd38bd2 100644 --- a/tests/queries/0_stateless/00729_prewhere_array_join.sql +++ b/tests/queries/0_stateless/00729_prewhere_array_join.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; drop table if exists t1_00729; create table t1_00729 (id UInt64, val Array(String),nid UInt64, eDate Date)ENGINE = MergeTree(eDate, (id, eDate), 8192); diff --git a/tests/queries/0_stateless/00732_base64_functions.sql b/tests/queries/0_stateless/00732_base64_functions.sql index a2dd4c4b41b..0d3b7e43645 100644 --- a/tests/queries/0_stateless/00732_base64_functions.sql +++ b/tests/queries/0_stateless/00732_base64_functions.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT base64Encode(val) FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar']) val); SELECT base64Decode(val) FROM (select arrayJoin(['', 'Zg==', 'Zm8=', 'Zm9v', 'Zm9vYg==', 'Zm9vYmE=', 'Zm9vYmFy']) val); SELECT base64Decode(base64Encode('foo')) = 'foo', base64Encode(base64Decode('Zm9v')) == 'Zm9v'; diff --git a/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql index 9348c94c981..6346dde9b76 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql index 5bfd54c3768..bf9f63ee3c1 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql index 001932c852e..7d2ee796b48 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; diff --git a/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql index 1644856dc8d..0f66598fc51 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql index 69fd709add8..397aa7853df 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql index 74e625cafa4..a77f3377202 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; diff --git a/tests/queries/0_stateless/00735_conditional.sql b/tests/queries/0_stateless/00735_conditional.sql index ce49c26ca3d..001a4aa7430 100644 --- a/tests/queries/0_stateless/00735_conditional.sql +++ b/tests/queries/0_stateless/00735_conditional.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT 'value vs value'; diff --git a/tests/queries/0_stateless/00742_require_join_strictness.sql b/tests/queries/0_stateless/00742_require_join_strictness.sql index 958d349aa51..5659a0f6833 100644 --- a/tests/queries/0_stateless/00742_require_join_strictness.sql +++ b/tests/queries/0_stateless/00742_require_join_strictness.sql @@ -1,3 +1,3 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET join_default_strictness = ''; SELECT * FROM system.one INNER JOIN (SELECT number AS k FROM system.numbers) js2 ON dummy = k; -- { serverError 417 } diff --git a/tests/queries/0_stateless/00754_alter_modify_column_partitions.sql b/tests/queries/0_stateless/00754_alter_modify_column_partitions.sql index d2fffabb8c6..4aa7ab8ad64 100644 --- a/tests/queries/0_stateless/00754_alter_modify_column_partitions.sql +++ b/tests/queries/0_stateless/00754_alter_modify_column_partitions.sql @@ -1,6 +1,6 @@ -- check ALTER MODIFY COLUMN with partitions -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS alter_column; diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by.sql b/tests/queries/0_stateless/00754_alter_modify_order_by.sql index 6cf3da0e29f..f8c584ed052 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by.sql +++ b/tests/queries/0_stateless/00754_alter_modify_order_by.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS old_style; CREATE TABLE old_style(d Date, x UInt32) ENGINE MergeTree(d, x, 8192); diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql index c94c0f3c55b..e973259d4b7 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS old_style; CREATE TABLE old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/old_style', 'r1', d, x, 8192); diff --git a/tests/queries/0_stateless/00758_array_reverse.sql b/tests/queries/0_stateless/00758_array_reverse.sql index 8a2417e68b0..11192535dc1 100644 --- a/tests/queries/0_stateless/00758_array_reverse.sql +++ b/tests/queries/0_stateless/00758_array_reverse.sql @@ -11,5 +11,5 @@ SELECT reverse(NULL); SELECT reverse([]); SELECT reverse([[[[]]]]); -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT '[RE7', ( SELECT '\0' ) AS riwwq, ( SELECT reverse([( SELECT bitTestAll(NULL) ) , ( SELECT '\0' ) AS ddfweeuy]) ) AS xuvv, '', ( SELECT * FROM file() ) AS wqgdswyc, ( SELECT * FROM file() ); -- { serverError 42 } diff --git a/tests/queries/0_stateless/00762_date_comparsion.sql b/tests/queries/0_stateless/00762_date_comparsion.sql index b874cb0b0b3..cc054bc7047 100644 --- a/tests/queries/0_stateless/00762_date_comparsion.sql +++ b/tests/queries/0_stateless/00762_date_comparsion.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; select today() < 2018-11-14; -- { serverError 43 } select toDate('2018-01-01') < '2018-11-14'; diff --git a/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql b/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql index a7f1f3ad98a..4e16768b373 100644 --- a/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql +++ b/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; select lcase('FOO'); select ucase('foo'); diff --git a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql index 8a4cb9f81d7..a9e6c12735c 100644 --- a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS alter_compression_codec; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 2c7b3b39469..61298de2548 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; DROP TABLE IF EXISTS compression_codec; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql index f61adccefd5..fba6a216762 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; -- copy-paste for storage log diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql index 5f4e3d9be79..ad104eff92c 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET joined_subquery_requires_alias = 0; DROP TABLE IF EXISTS delta_codec_synthetic; diff --git a/tests/queries/0_stateless/00804_test_delta_codec_no_type_alter.sql b/tests/queries/0_stateless/00804_test_delta_codec_no_type_alter.sql index f49b8b2ee32..4cb34b6384e 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_no_type_alter.sql +++ b/tests/queries/0_stateless/00804_test_delta_codec_no_type_alter.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; DROP TABLE IF EXISTS delta_codec_for_alter; diff --git a/tests/queries/0_stateless/00805_round_down.sql b/tests/queries/0_stateless/00805_round_down.sql index bf25f7be952..6d59cb0af1a 100644 --- a/tests/queries/0_stateless/00805_round_down.sql +++ b/tests/queries/0_stateless/00805_round_down.sql @@ -4,7 +4,7 @@ SELECT toInt32(number) as x, roundDown(x, [e(), pi(), pi(), e()]) FROM system.nu SELECT number as x, roundDown(x, [6, 5, 4]) FROM system.numbers LIMIT 10; SELECT 1 as x, roundDown(x, [6, 5, 4]); -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT 1 as x, roundDown(x, []); -- { serverError 43 } SELECT 1 as x, roundDown(x, emptyArrayUInt8()); -- { serverError 44 } SELECT roundDown(number, [number]) FROM system.numbers LIMIT 10; -- { serverError 44 } diff --git a/tests/queries/0_stateless/00808_array_enumerate_segfault.sql b/tests/queries/0_stateless/00808_array_enumerate_segfault.sql index b492d3114f8..88f9b821685 100644 --- a/tests/queries/0_stateless/00808_array_enumerate_segfault.sql +++ b/tests/queries/0_stateless/00808_array_enumerate_segfault.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT arrayEnumerateUniq(anyHeavy([]), []); SELECT arrayEnumerateDense([], [sequenceCount(NULL)]); -- { serverError 190 } SELECT arrayEnumerateDense([STDDEV_SAMP(NULL, 910947.571364)], [NULL]); diff --git a/tests/queries/0_stateless/00808_not_optimize_predicate.sql b/tests/queries/0_stateless/00808_not_optimize_predicate.sql index b4204f0c3bb..ad3df16e4bb 100644 --- a/tests/queries/0_stateless/00808_not_optimize_predicate.sql +++ b/tests/queries/0_stateless/00808_not_optimize_predicate.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS test_00808; CREATE TABLE test_00808(date Date, id Int8, name String, value Int64, sign Int8) ENGINE = CollapsingMergeTree(sign) ORDER BY (id, date); diff --git a/tests/queries/0_stateless/00809_add_days_segfault.sql b/tests/queries/0_stateless/00809_add_days_segfault.sql index 3d2e11ece77..6791439708a 100644 --- a/tests/queries/0_stateless/00809_add_days_segfault.sql +++ b/tests/queries/0_stateless/00809_add_days_segfault.sql @@ -4,7 +4,7 @@ SELECT ignore(subtractDays(toDateTime(0), 1)); SELECT ignore(addDays(toDate(0), -1)); SELECT ignore(subtractDays(toDate(0), 1)); -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT ignore(addDays((CAST((96.338) AS DateTime)), -3)); SELECT ignore(subtractDays((CAST((-5263074.47) AS DateTime)), -737895)); diff --git a/tests/queries/0_stateless/00810_in_operators_segfault.sql b/tests/queries/0_stateless/00810_in_operators_segfault.sql index 8e4a4723608..99ce7231780 100644 --- a/tests/queries/0_stateless/00810_in_operators_segfault.sql +++ b/tests/queries/0_stateless/00810_in_operators_segfault.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT globalNotIn(['"wh'], [NULL]); SELECT globalIn([''], [NULL]); diff --git a/tests/queries/0_stateless/00811_garbage.sql b/tests/queries/0_stateless/00811_garbage.sql index 65db1c736fa..93d9ec963c2 100644 --- a/tests/queries/0_stateless/00811_garbage.sql +++ b/tests/queries/0_stateless/00811_garbage.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT truncate(895, -16); SELECT ( SELECT toDecimal128([], rowNumberInBlock()) ) , lcm('', [[(CAST(('>A') AS String))]]); -- { serverError 44 } diff --git a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql index be83f040e04..68823386853 100644 --- a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql +++ b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; DROP TABLE IF EXISTS test.compression_codec_replicated1; diff --git a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql index cec9ff86cd0..cd0512d6be8 100644 --- a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS alter_compression_codec1; diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_pk.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_pk.sql index 53e640a4032..58e336e03c7 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_pk.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_pk.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT '----00489----'; DROP TABLE IF EXISTS pk; diff --git a/tests/queries/0_stateless/00929_multi_match_edit_distance.sql b/tests/queries/0_stateless/00929_multi_match_edit_distance.sql index c0f39a4f201..50f2359a685 100644 --- a/tests/queries/0_stateless/00929_multi_match_edit_distance.sql +++ b/tests/queries/0_stateless/00929_multi_match_edit_distance.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; select 0 = multiFuzzyMatchAny('abc', 0, ['a1c']) from system.numbers limit 5; select 1 = multiFuzzyMatchAny('abc', 1, ['a1c']) from system.numbers limit 5; diff --git a/tests/queries/0_stateless/00933_alter_ttl.sql b/tests/queries/0_stateless/00933_alter_ttl.sql index 4e0fde00952..1aacb9a6694 100644 --- a/tests/queries/0_stateless/00933_alter_ttl.sql +++ b/tests/queries/0_stateless/00933_alter_ttl.sql @@ -1,4 +1,4 @@ -set send_logs_level = 'none'; +set send_logs_level = 'fatal'; drop table if exists ttl; diff --git a/tests/queries/0_stateless/00933_ttl_simple.sql b/tests/queries/0_stateless/00933_ttl_simple.sql index 6c750f8749d..b924faad3f5 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/tests/queries/0_stateless/00933_ttl_simple.sql @@ -83,7 +83,7 @@ select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1; -set send_logs_level = 'none'; +set send_logs_level = 'fatal'; drop table if exists ttl_00933_1; diff --git a/tests/queries/0_stateless/00949_format.sql b/tests/queries/0_stateless/00949_format.sql index 433ababde9d..0683b2b6952 100644 --- a/tests/queries/0_stateless/00949_format.sql +++ b/tests/queries/0_stateless/00949_format.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; select format('Hello {1} World {0}', materialize('first'), materialize('second')) from system.numbers limit 1; select format('Hello {0} World {1}', materialize('first'), materialize('second')) from system.numbers limit 2; diff --git a/tests/queries/0_stateless/00955_test_final_mark.sql b/tests/queries/0_stateless/00955_test_final_mark.sql index d58bdec7472..f77f1b7b30a 100644 --- a/tests/queries/0_stateless/00955_test_final_mark.sql +++ b/tests/queries/0_stateless/00955_test_final_mark.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS mt_with_pk; diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql index d420ecdd445..d7d7c02baa8 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS database_for_dict; diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql index 7f2e1d08671..f4de269e774 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS database_for_dict; diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql index 2cea0c8e61d..6d9b499a247 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS database_for_dict; diff --git a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql index 029dc9755d8..4d2cd6351b5 100644 --- a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql +++ b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS database_for_dict; diff --git a/tests/queries/0_stateless/01033_dictionaries_lifetime.sql b/tests/queries/0_stateless/01033_dictionaries_lifetime.sql index 8b16c401afe..57776e1fec1 100644 --- a/tests/queries/0_stateless/01033_dictionaries_lifetime.sql +++ b/tests/queries/0_stateless/01033_dictionaries_lifetime.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS database_for_dict; diff --git a/tests/queries/0_stateless/01037_polygon_dict_multi_polygons.sql b/tests/queries/0_stateless/01037_polygon_dict_multi_polygons.sql index 4c446c28b51..9750ec04daf 100644 --- a/tests/queries/0_stateless/01037_polygon_dict_multi_polygons.sql +++ b/tests/queries/0_stateless/01037_polygon_dict_multi_polygons.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS test_01037; diff --git a/tests/queries/0_stateless/01037_polygon_dict_simple_polygons.sql b/tests/queries/0_stateless/01037_polygon_dict_simple_polygons.sql index 56e5865a5a1..9ffbac78bbb 100644 --- a/tests/queries/0_stateless/01037_polygon_dict_simple_polygons.sql +++ b/tests/queries/0_stateless/01037_polygon_dict_simple_polygons.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS test_01037; diff --git a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql index 1d195b0388f..d8f5356d145 100644 --- a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql +++ b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql @@ -1,5 +1,5 @@ SET check_query_single_value_result = 0; -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS mt_without_pk; diff --git a/tests/queries/0_stateless/01038_array_of_unnamed_tuples.sql b/tests/queries/0_stateless/01038_array_of_unnamed_tuples.sql index c9805599549..5da319f1a25 100644 --- a/tests/queries/0_stateless/01038_array_of_unnamed_tuples.sql +++ b/tests/queries/0_stateless/01038_array_of_unnamed_tuples.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS array_of_tuples; diff --git a/tests/queries/0_stateless/01115_join_with_dictionary.sql b/tests/queries/0_stateless/01115_join_with_dictionary.sql index 65704f2b3eb..f1477df7df2 100644 --- a/tests/queries/0_stateless/01115_join_with_dictionary.sql +++ b/tests/queries/0_stateless/01115_join_with_dictionary.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS db_01115; CREATE DATABASE db_01115 Engine = Ordinary; diff --git a/tests/queries/0_stateless/01247_dist_on_dist_group_by_sharding_key_optimization.sql b/tests/queries/0_stateless/01247_dist_on_dist_group_by_sharding_key_optimization.sql index 29e45ffdb80..b4852793e7c 100644 --- a/tests/queries/0_stateless/01247_dist_on_dist_group_by_sharding_key_optimization.sql +++ b/tests/queries/0_stateless/01247_dist_on_dist_group_by_sharding_key_optimization.sql @@ -1,7 +1,7 @@ -- TODO: correct testing with real unique shards -- Avoid "Connection failed at try №1" messages. -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET connect_timeout_with_failover_ms = 5000; set optimize_distributed_group_by_sharding_key=1; diff --git a/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql b/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql index d4753e95f79..4a8842ca71f 100644 --- a/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql +++ b/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql @@ -1,7 +1,7 @@ -- TODO: correct testing with real unique shards -- Avoid "Connection failed at try №1" messages. -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET connect_timeout_with_failover_ms = 5000; set optimize_distributed_group_by_sharding_key=1; From 7fcf942d24cc76aedb2c0c1a67aa2ffaf53175f1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jun 2020 14:29:23 +0300 Subject: [PATCH 124/211] Turn off traps --- src/Functions/trap.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Functions/trap.cpp b/src/Functions/trap.cpp index 65aa9b8d3e3..94aa56919b4 100644 --- a/src/Functions/trap.cpp +++ b/src/Functions/trap.cpp @@ -1,4 +1,4 @@ -#if 1 +#if 0 #include #include @@ -63,8 +63,6 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { - std::cerr << "executeImpl: " << !!CurrentThread::getInternalTextLogsQueue() << "\n"; - if (const ColumnConst * column = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get())) { String mode = column->getValue(); From b2d1e08dd6221bc8233148e10030e2e0fe122d98 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jun 2020 14:31:38 +0300 Subject: [PATCH 125/211] Fix build --- base/daemon/BaseDaemon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 3c4d3d2f74f..7df5eb55e17 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -292,7 +292,7 @@ static void sanitizerDeathCallback() StringRef query_id = DB::CurrentThread::getQueryId(); /// This is signal safe. - if (query_id.empty()) + if (query_id.size == 0) { LOG_FATAL(log, "(version {}{}, {}) (from thread {}) (no query) Sanitizer trap.", VERSION_STRING, VERSION_OFFICIAL, daemon.build_id_info, thread_num); From 733e34f890399f416b5ff7f0658cdbede039d20e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jun 2020 14:33:09 +0300 Subject: [PATCH 126/211] Fix build --- src/Functions/buildId.cpp | 8 ++++---- src/Functions/registerFunctionsMiscellaneous.cpp | 2 ++ 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Functions/buildId.cpp b/src/Functions/buildId.cpp index f0745cc34c1..73e39d5f480 100644 --- a/src/Functions/buildId.cpp +++ b/src/Functions/buildId.cpp @@ -12,13 +12,13 @@ namespace DB /** buildId() - returns the compiler build id of the running binary. */ -class FunctionVersion : public IFunction +class FunctionBuildId : public IFunction { public: static constexpr auto name = "buildId"; static FunctionPtr create(const Context &) { - return std::make_shared(); + return std::make_shared(); } String getName() const override @@ -43,9 +43,9 @@ public: }; -void registerFunctionVersion(FunctionFactory & factory) +void registerFunctionBuildId(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 3e6099ee25a..720aea1077f 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -38,6 +38,7 @@ void registerFunctionIsNaN(FunctionFactory &); void registerFunctionIfNotFinite(FunctionFactory &); void registerFunctionThrowIf(FunctionFactory &); void registerFunctionVersion(FunctionFactory &); +void registerFunctionBuildId(FunctionFactory &); void registerFunctionUptime(FunctionFactory &); void registerFunctionTimeZone(FunctionFactory &); void registerFunctionRunningAccumulate(FunctionFactory &); @@ -94,6 +95,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionIfNotFinite(factory); registerFunctionThrowIf(factory); registerFunctionVersion(factory); + registerFunctionBuildId(factory); registerFunctionUptime(factory); registerFunctionTimeZone(factory); registerFunctionRunningAccumulate(factory); From 420b943cf10a31ec89a26b21a0bd8066bfdfe12c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jun 2020 21:03:40 +0300 Subject: [PATCH 127/211] Auto-generate ya.make --- src/Functions/ya.make | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/ya.make b/src/Functions/ya.make index ea66828c942..082efc07681 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -122,6 +122,7 @@ SRCS( blockNumber.cpp blockSerializedSize.cpp blockSize.cpp + buildId.cpp caseWithExpression.cpp cbrt.cpp coalesce.cpp @@ -146,6 +147,7 @@ SRCS( exp10.cpp exp2.cpp exp.cpp + extractAllGroups.cpp extractAllGroupsHorizontal.cpp extractAllGroupsVertical.cpp extract.cpp From 581d1a4470da6ad0e72a84b9630c4f5492962c11 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jun 2020 21:05:44 +0300 Subject: [PATCH 128/211] Fix build --- base/daemon/BaseDaemon.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 7df5eb55e17..761f6c160d7 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -294,13 +294,13 @@ static void sanitizerDeathCallback() if (query_id.size == 0) { - LOG_FATAL(log, "(version {}{}, {}) (from thread {}) (no query) Sanitizer trap.", - VERSION_STRING, VERSION_OFFICIAL, daemon.build_id_info, thread_num); + LOG_FATAL(log, "(version {}{}) (from thread {}) (no query) Sanitizer trap.", + VERSION_STRING, VERSION_OFFICIAL, thread_num); } else { - LOG_FATAL(log, "(version {}{}, {}) (from thread {}) (query_id: {}) Sanitizer trap.", - VERSION_STRING, VERSION_OFFICIAL, daemon.build_id_info, thread_num, query_id); + LOG_FATAL(log, "(version {}{}) (from thread {}) (query_id: {}) Sanitizer trap.", + VERSION_STRING, VERSION_OFFICIAL, thread_num, query_id); } /// Just in case print our own stack trace. In case when llvm-symbolizer does not work. From bd45592539a2351c1547c5e3ef8951bcb13dd1a0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 21 Jun 2020 11:59:54 +0300 Subject: [PATCH 129/211] Fix test_distributed_load_balancing flaps (due to config reload) At startup, server loads configuration files. However ConfigReloader does not know about already loaded files (files is empty()), hence it will always reload the configuration just after server starts (+ 2 seconds, reload timeout). And on configuration reload the clusters will be re-created, so some internal stuff will be reseted: - error_count - last_used (round_robing) And if the reload will happen during round_robin test it will start querying from the beginning, so let's issue config reload just after start to avoid reload in the middle of the test execution. --- .../test_distributed_load_balancing/test.py | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_distributed_load_balancing/test.py b/tests/integration/test_distributed_load_balancing/test.py index 4f760c705c3..07986de6a85 100644 --- a/tests/integration/test_distributed_load_balancing/test.py +++ b/tests/integration/test_distributed_load_balancing/test.py @@ -16,8 +16,23 @@ n3 = cluster.add_instance('n3', main_configs=['configs/remote_servers.xml']) nodes = len(cluster.instances) queries = nodes*5 -def create_tables(): +def bootstrap(): for n in cluster.instances.values(): + # At startup, server loads configuration files. + # + # However ConfigReloader does not know about already loaded files + # (files is empty()), hence it will always reload the configuration + # just after server starts (+ 2 seconds, reload timeout). + # + # And on configuration reload the clusters will be re-created, so some + # internal stuff will be reseted: + # - error_count + # - last_used (round_robing) + # + # And if the reload will happen during round_robin test it will start + # querying from the beginning, so let's issue config reload just after + # start to avoid reload in the middle of the test execution. + n.query('SYSTEM RELOAD CONFIG') n.query('DROP TABLE IF EXISTS data') n.query('DROP TABLE IF EXISTS dist') n.query('CREATE TABLE data (key Int) Engine=Memory()') @@ -36,7 +51,7 @@ def make_uuid(): def start_cluster(): try: cluster.start() - create_tables() + bootstrap() yield cluster finally: cluster.shutdown() From f430893adc51d7f188566cb616fd59a185545695 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jun 2020 13:45:26 +0300 Subject: [PATCH 130/211] Fix Arcadia --- src/Functions/ya.make | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 082efc07681..dfd7faca9c7 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -147,7 +147,6 @@ SRCS( exp10.cpp exp2.cpp exp.cpp - extractAllGroups.cpp extractAllGroupsHorizontal.cpp extractAllGroupsVertical.cpp extract.cpp From ea358294810f0d19944b53e74d011ca69925c3e6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jun 2020 13:46:41 +0300 Subject: [PATCH 131/211] Fix build --- base/daemon/BaseDaemon.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index c9169d55ee6..e40884a44b5 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -296,12 +296,12 @@ static void sanitizerDeathCallback() if (query_id.size == 0) { LOG_FATAL(log, "(version {}{}) (from thread {}) (no query) Sanitizer trap.", - VERSION_STRING, VERSION_OFFICIAL, thread_num); + VERSION_STRING, VERSION_OFFICIAL, getThreadId()); } else { LOG_FATAL(log, "(version {}{}) (from thread {}) (query_id: {}) Sanitizer trap.", - VERSION_STRING, VERSION_OFFICIAL, thread_num, query_id); + VERSION_STRING, VERSION_OFFICIAL, getThreadId(), query_id); } /// Just in case print our own stack trace. In case when llvm-symbolizer does not work. From 844960195d5e8d6a42f7cf4d48adcb18bcb4a304 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 21 Jun 2020 16:55:18 +0300 Subject: [PATCH 132/211] Fix memory accounting via HTTP interface function perf_test() { time yes '127.1:8123/?wait_end_of_query=1' | head -n10000 | xargs -P10000 curl -s -d 'select 1' | grep -x -c 1 } function server() { local limit=$1 && shift clickhouse-server "$@" -- --max_server_memory_usage=$limit } perf_test: - before this patch with 1G limit: succeed only ~800 queries - after this patch with 1G limit: succeed ~8000 queries - before this patch with 10G limit: succeed only ~3000 queries - after this patch with 10G limit: succeed ~10000 queries Fixes: #11153 --- src/Server/HTTPHandler.cpp | 12 +++++++----- src/Server/HTTPHandler.h | 1 + 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index e866af2f49b..cd26f9e88fc 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -232,15 +232,12 @@ HTTPHandler::HTTPHandler(IServer & server_, const std::string & name) void HTTPHandler::processQuery( + Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params, Poco::Net::HTTPServerResponse & response, Output & used_output) { - Context context = server.context(); - - CurrentThread::QueryScope query_scope(context); - LOG_TRACE(log, "Request URI: {}", request.getURI()); std::istream & istr = request.stream(); @@ -683,6 +680,11 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne setThreadName("HTTPHandler"); ThreadStatus thread_status; + /// Should be initialized before anything, + /// For correct memory accounting. + Context context = server.context(); + CurrentThread::QueryScope query_scope(context); + Output used_output; /// In case of exception, send stack trace to client. @@ -706,7 +708,7 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne throw Exception("The Transfer-Encoding is not chunked and there is no Content-Length header for POST request", ErrorCodes::HTTP_LENGTH_REQUIRED); } - processQuery(request, params, response, used_output); + processQuery(context, request, params, response, used_output); LOG_INFO(log, "Done processing query"); } catch (...) diff --git a/src/Server/HTTPHandler.h b/src/Server/HTTPHandler.h index b1a6355d281..be3fa6d4c4a 100644 --- a/src/Server/HTTPHandler.h +++ b/src/Server/HTTPHandler.h @@ -72,6 +72,7 @@ private: /// Also initializes 'used_output'. void processQuery( + Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params, Poco::Net::HTTPServerResponse & response, From bffd4cf576c396cd8a23684d3c70d1f016162799 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 21 Jun 2020 17:34:34 +0300 Subject: [PATCH 133/211] Add a test for memory tracking via HTTP interface --- .../01238_http_memory_tracking.reference | 1 + .../0_stateless/01238_http_memory_tracking.sh | 26 +++++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/01238_http_memory_tracking.reference create mode 100755 tests/queries/0_stateless/01238_http_memory_tracking.sh diff --git a/tests/queries/0_stateless/01238_http_memory_tracking.reference b/tests/queries/0_stateless/01238_http_memory_tracking.reference new file mode 100644 index 00000000000..5caff40c4a0 --- /dev/null +++ b/tests/queries/0_stateless/01238_http_memory_tracking.reference @@ -0,0 +1 @@ +10000 diff --git a/tests/queries/0_stateless/01238_http_memory_tracking.sh b/tests/queries/0_stateless/01238_http_memory_tracking.sh new file mode 100755 index 00000000000..07c731c1ca1 --- /dev/null +++ b/tests/queries/0_stateless/01238_http_memory_tracking.sh @@ -0,0 +1,26 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +# TODO: the test can be way more optimal + +set -o pipefail + +function execute_null() +{ + ${CLICKHOUSE_CLIENT} --format Null -n "$@" +} + +# This is needed to keep at least one running query for user for the time of test. +# (10k http queries takes 10seconds, let's run for 3x more to avoid flaps) +execute_null <<<'SELECT sleepEachRow(1) FROM numbers(30)' & + +# ignore "yes: standard output: Broken pipe" +yes 'SELECT 1' 2>/dev/null | { + head -n10000 +} | { + xargs -P10000 -i ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&max_memory_usage_for_user=$((10<<30))" -d '{}' +} | grep -x -c 1 + +wait From 8a3610bad8049abae2e08478d48073a7b2174311 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jun 2020 19:08:41 +0300 Subject: [PATCH 134/211] Fix build --- src/Functions/buildId.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Functions/buildId.cpp b/src/Functions/buildId.cpp index 73e39d5f480..d75deae4480 100644 --- a/src/Functions/buildId.cpp +++ b/src/Functions/buildId.cpp @@ -50,4 +50,8 @@ void registerFunctionBuildId(FunctionFactory & factory) } +#else + +void registerFunctionBuildId(FunctionFactory &) {} + #endif From fac2c6050c7e4bc0ee20b4f7d62f5464a06d816b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jun 2020 19:10:38 +0300 Subject: [PATCH 135/211] Fix UBSan --- src/Common/Elf.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/Elf.cpp b/src/Common/Elf.cpp index ee32586d95e..0c2359b3418 100644 --- a/src/Common/Elf.cpp +++ b/src/Common/Elf.cpp @@ -2,6 +2,7 @@ #include #include +#include #include @@ -135,7 +136,7 @@ String Elf::getBuildID(const char * nhdr_pos, size_t size) while (nhdr_pos < nhdr_end) { - const ElfNhdr & nhdr = *reinterpret_cast(nhdr_pos); + ElfNhdr nhdr = unalignedLoad(nhdr_pos); nhdr_pos += sizeof(ElfNhdr) + nhdr.n_namesz; if (nhdr.n_type == NT_GNU_BUILD_ID) From 55c219b10766fa7b3882370c709bed269567ef02 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jun 2020 20:28:40 +0300 Subject: [PATCH 136/211] Slightly loose the validation of ODBC connection string --- .../odbc-bridge/validateODBCConnectionString.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/programs/odbc-bridge/validateODBCConnectionString.cpp b/programs/odbc-bridge/validateODBCConnectionString.cpp index e564cebdeee..a54021431ef 100644 --- a/programs/odbc-bridge/validateODBCConnectionString.cpp +++ b/programs/odbc-bridge/validateODBCConnectionString.cpp @@ -65,7 +65,11 @@ std::string validateODBCConnectionString(const std::string & connection_string) else throw Exception("ODBC connection string parameter name doesn't begin with valid identifier character", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - while (pos < end && isWordCharASCII(*pos)) + /// Additionally allow dash and dot symbols in names. + /// Strictly speaking, the name with that characters should be escaped. + /// But some ODBC drivers (e.g.) Postgres don't like escaping. + + while (pos < end && (isWordCharASCII(*pos) || *pos == '-' || *pos == '.')) ++pos; return std::string(begin, pos); @@ -213,7 +217,11 @@ std::string validateODBCConnectionString(const std::string & connection_string) auto write_value = [&](const std::string & value) { - if (std::all_of(value.begin(), value.end(), isWordCharASCII)) + /// Additionally allow dash and dot symbols - for hostnames. + /// Strictly speaking, hostname with that characters should be escaped. + /// But some ODBC drivers (e.g.) Postgres don't like escaping. + + if (std::all_of(value.begin(), value.end(), [](char c) { return isWordCharASCII(c) || c == '.' || c == '-'; })) write_plain_value(value); else write_escaped_value(value); From dcda56181521cc89db078ce1496c9dc2927c165f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 21 Jun 2020 21:31:19 +0300 Subject: [PATCH 137/211] Update synthetic_hardware_benchmark.xml --- tests/performance/synthetic_hardware_benchmark.xml | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/performance/synthetic_hardware_benchmark.xml b/tests/performance/synthetic_hardware_benchmark.xml index deae39ab80f..f15a015927f 100644 --- a/tests/performance/synthetic_hardware_benchmark.xml +++ b/tests/performance/synthetic_hardware_benchmark.xml @@ -1,6 +1,4 @@ - - + 30000000000 From 045553d00f6255f3e06a3ec64d887afb1fd74d13 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 21 Jun 2020 21:31:31 +0300 Subject: [PATCH 138/211] Update synthetic_hardware_benchmark.xml --- tests/performance/synthetic_hardware_benchmark.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/synthetic_hardware_benchmark.xml b/tests/performance/synthetic_hardware_benchmark.xml index f15a015927f..2233bfeca8d 100644 --- a/tests/performance/synthetic_hardware_benchmark.xml +++ b/tests/performance/synthetic_hardware_benchmark.xml @@ -1,4 +1,4 @@ - + 30000000000 From d4fce9d0f38dc7035720a862c6b60ff410744119 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 21 Jun 2020 21:31:58 +0300 Subject: [PATCH 139/211] Update great_circle_dist.xml --- tests/performance/great_circle_dist.xml | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/performance/great_circle_dist.xml b/tests/performance/great_circle_dist.xml index a57097bcbe7..13f9e6fde56 100644 --- a/tests/performance/great_circle_dist.xml +++ b/tests/performance/great_circle_dist.xml @@ -1,6 +1,4 @@ - - + SELECT count() FROM numbers(1000000) WHERE NOT ignore(greatCircleDistance((rand(1) % 360) * 1. - 180, (number % 150) * 1.2 - 90, (number % 360) + toFloat64(rand(2)) / 4294967296 - 180, (rand(3) % 180) * 1. - 90)) From 746676d762d25aaafdc8119c1bd3cfa0b870ba4b Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 21 Jun 2020 21:32:12 +0300 Subject: [PATCH 140/211] Update general_purpose_hashes.xml --- tests/performance/general_purpose_hashes.xml | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/performance/general_purpose_hashes.xml b/tests/performance/general_purpose_hashes.xml index ada1df439fe..31a1bd65835 100644 --- a/tests/performance/general_purpose_hashes.xml +++ b/tests/performance/general_purpose_hashes.xml @@ -1,6 +1,4 @@ - - + gp_hash_func From 8100423b7ed96b91df2f1367fa48467e9b5a8b4a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jun 2020 21:47:16 +0300 Subject: [PATCH 141/211] Style --- src/Functions/FunctionsRandom.cpp | 12 +++++----- src/Functions/PerformanceAdaptors.h | 20 ++++++++--------- src/Functions/TargetSpecific.cpp | 8 +++---- src/Functions/TargetSpecific.h | 34 ++++++++++++++--------------- 4 files changed, 37 insertions(+), 37 deletions(-) diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index e77bab9c0a5..9436ce0f5e2 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -101,7 +101,7 @@ using namespace VectorExtension; /* Takes 2 vectors with LinearCongruentialGenerator states and combines them into vector with random values. * From every rand-state we use only bits 15...47 to generate random vector. */ -inline UInt64x4 CombineValues(UInt64x4 a, UInt64x4 b) +inline UInt64x4 combineValues(UInt64x4 a, UInt64x4 b) { auto xa = reinterpret_cast<__m256i>(a); auto xb = reinterpret_cast<__m256i>(b); @@ -156,16 +156,16 @@ void RandImpl::execute(char * output, size_t size) { gens1 = gens1 * a + c; gens2 = gens2 * a + c; - unalignedStore(output, CombineValues(gens1, gens2)); + unalignedStore(output, combineValues(gens1, gens2)); gens3 = gens3 * a + c; gens4 = gens4 * a + c; - unalignedStore(output + sizeof(UInt64x4), CombineValues(gens3, gens4)); + unalignedStore(output + sizeof(UInt64x4), combineValues(gens3, gens4)); gens1 = gens1 * a + c; gens2 = gens2 * a + c; - unalignedStore(output + 2 * sizeof(UInt64x4), CombineValues(gens1, gens2)); + unalignedStore(output + 2 * sizeof(UInt64x4), combineValues(gens1, gens2)); gens3 = gens3 * a + c; gens4 = gens4 * a + c; - unalignedStore(output + 3 * sizeof(UInt64x4), CombineValues(gens3, gens4)); + unalignedStore(output + 3 * sizeof(UInt64x4), combineValues(gens3, gens4)); output += bytes_per_write; } @@ -174,7 +174,7 @@ void RandImpl::execute(char * output, size_t size) { gens1 = gens1 * a + c; gens2 = gens2 * a + c; - UInt64x4 values = CombineValues(gens1, gens2); + UInt64x4 values = combineValues(gens1, gens2); for (int i = 0; i < vec_size && (end - output) > 0; ++i) { unalignedStore(output, values[i]); diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index bbe50d2e994..419faf57e30 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -149,25 +149,25 @@ namespace detail String getImplementationTag(TargetArch arch) { if constexpr (has_implementation_tag) - return ToString(arch) + "_" + T::getImplementationTag(); + return toString(arch) + "_" + T::getImplementationTag(); else - return ToString(arch); + return toString(arch); } } /* Class which is used to store implementations for the function and to select the best one to run * based on processor architecture and statistics from previous runs. - * + * * FunctionInterface is typically IFunction or IExecutableFunctionImpl, but practically it can be * any interface that contains "execute" method (IFunction is an exception and is supported as well). - * + * * Example of usage: - * + * * class MyDefaulImpl : public IFunction {...}; * DECLARE_AVX2_SPECIFIC_CODE( * class MyAVX2Impl : public IFunction {...}; * ) - * + * * /// All methods but execute/executeImpl are usually not bottleneck, so just use them from * /// default implementation. * class MyFunction : public MyDefaultImpl @@ -232,17 +232,17 @@ public: /* Register new implementation for function. * - * Arch - required instruction set for running the implementation. It's guarantied that no one method would + * Arch - required instruction set for running the implementation. It's guarantied that no method would * be called (even the constructor and static methods) if the processor doesn't support this instruction set. - * + * * FunctionImpl - implementation, should be inherited from template argument FunctionInterface. - * + * * All function arguments will be forwarded to the implementation constructor. */ template void registerImplementation(Args&&... args) { - if (IsArchSupported(Arch)) + if (isArchSupported(Arch)) { // TODO(dakovalkov): make this option better. const auto & choose_impl = context.getSettingsRef().function_implementation.value; diff --git a/src/Functions/TargetSpecific.cpp b/src/Functions/TargetSpecific.cpp index 65f8641ee8e..830611fea7a 100644 --- a/src/Functions/TargetSpecific.cpp +++ b/src/Functions/TargetSpecific.cpp @@ -5,7 +5,7 @@ namespace DB { -UInt32 GetSupportedArches() +UInt32 getSupportedArchs() { UInt32 result = 0; if (Cpu::CpuFlagsCache::have_SSE42) @@ -19,13 +19,13 @@ UInt32 GetSupportedArches() return result; } -bool IsArchSupported(TargetArch arch) +bool isArchSupported(TargetArch arch) { - static UInt32 arches = GetSupportedArches(); + static UInt32 arches = getSupportedArchs(); return arch == TargetArch::Default || (arches & static_cast(arch)); } -String ToString(TargetArch arch) +String toString(TargetArch arch) { switch (arch) { diff --git a/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h index ed9c0d3c244..df7473686fd 100644 --- a/src/Functions/TargetSpecific.h +++ b/src/Functions/TargetSpecific.h @@ -3,54 +3,54 @@ #include /* This file contains macros and helpers for writing platform-dependent code. - * - * Macros DECLARE__SPECIFIC_CODE will wrap code inside it into the + * + * Macros DECLARE__SPECIFIC_CODE will wrap code inside it into the * namespace TargetSpecific:: and enable Arch-specific compile options. * Thus, it's allowed to call functions inside these namespaces only after - * checking platform in runtime (see IsArchSupported() below). + * checking platform in runtime (see isArchSupported() below). * * If compiler is not gcc/clang or target isn't x86_64 or ENABLE_MULTITARGET_CODE * was set to OFF in cmake, all code inside these macroses will be removed and * USE_MUTLITARGE_CODE will be set to 0. Use #if USE_MUTLITARGE_CODE whenever you * use anything from this namespaces. - * + * * For similarities there is a macros DECLARE_DEFAULT_CODE, which wraps code * into the namespace TargetSpecific::Default but dosn't specify any additional * copile options. Functions and classes inside this macros are available regardless * of USE_MUTLITARGE_CODE. - * + * * Example of usage: - * + * * DECLARE_DEFAULT_CODE ( * int funcImpl() { * return 1; * } * ) // DECLARE_DEFAULT_CODE - * + * * DECLARE_AVX2_SPECIFIC_CODE ( * int funcImpl() { * return 2; * } * ) // DECLARE_DEFAULT_CODE - * + * * int func() { * #if USE_MULTITARGET_CODE - * if (IsArchSupported(TargetArch::AVX2)) + * if (isArchSupported(TargetArch::AVX2)) * return TargetSpecifc::AVX2::funcImpl(); * #endif * return TargetSpecifc::Default::funcImpl(); - * } - * + * } + * * Sometimes code may benefit from compiling with different options. * For these purposes use DECLARE_MULTITARGET_CODE macros. It will create a copy * of the code for every supported target and compile it with different options. * These copies are available via TargetSpecifc namespaces described above. - * - * Inside every TargetSpecific namespace there is a constexpr variable BuildArch, + * + * Inside every TargetSpecific namespace there is a constexpr variable BuildArch, * which indicates the target platform for current code. - * + * * Example: - * + * * DECLARE_MULTITARGET_CODE( * int funcImpl(int size, ...) { * int iteration_size = 1; @@ -83,9 +83,9 @@ enum class TargetArch : UInt32 }; /// Runtime detection. -bool IsArchSupported(TargetArch arch); +bool isArchSupported(TargetArch arch); -String ToString(TargetArch arch); +String toString(TargetArch arch); #if ENABLE_MULTITARGET_CODE && defined(__GNUC__) && defined(__x86_64__) From 8cc411c5b5be01d8967ee03c7b2413294d031027 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jun 2020 21:50:02 +0300 Subject: [PATCH 142/211] Clarify setting --- src/Core/Settings.h | 2 +- src/Functions/PerformanceAdaptors.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9d2b5db7d33..dc50a6a6957 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -377,7 +377,7 @@ struct Settings : public SettingsCollection M(SettingBool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \ M(SettingSeconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \ M(SettingBool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \ - M(SettingString, function_implementation, "", "Choose implementation. If empty enable all of them.", 0) \ + M(SettingString, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \ \ M(SettingBool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ M(SettingBool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index 419faf57e30..b17be0619a9 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -240,7 +240,7 @@ public: * All function arguments will be forwarded to the implementation constructor. */ template - void registerImplementation(Args&&... args) + void registerImplementation(Args &&... args) { if (isArchSupported(Arch)) { From a8e739b191bf6de04cccb001a0f476b8279c19fc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jun 2020 21:52:58 +0300 Subject: [PATCH 143/211] Fix build --- src/Functions/buildId.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/buildId.cpp b/src/Functions/buildId.cpp index d75deae4480..ef36c0065af 100644 --- a/src/Functions/buildId.cpp +++ b/src/Functions/buildId.cpp @@ -52,6 +52,7 @@ void registerFunctionBuildId(FunctionFactory & factory) #else +class FunctionFactory; void registerFunctionBuildId(FunctionFactory &) {} #endif From 318b35cd85c8e2cbfb1304554a1e8cbb72a5a810 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jun 2020 23:22:34 +0300 Subject: [PATCH 144/211] generate-ya-make --- src/Functions/ya.make | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/ya.make b/src/Functions/ya.make index ea66828c942..cbd0c0bf947 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -363,6 +363,7 @@ SRCS( subtractYears.cpp tan.cpp tanh.cpp + TargetSpecific.cpp tgamma.cpp throwIf.cpp timeSlot.cpp From e9fb277e6c180f10f3d8786f182ce4e0525d8f99 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jun 2020 23:37:55 +0300 Subject: [PATCH 145/211] Make LIVE VIEW test less wrong (it was depending on the timing of progress messages) --- .../test_live_view_over_distributed/test.py | 60 +++++++++++++++---- 1 file changed, 48 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_live_view_over_distributed/test.py b/tests/integration/test_live_view_over_distributed/test.py index c7b9c452725..1ae9a4b5199 100644 --- a/tests/integration/test_live_view_over_distributed/test.py +++ b/tests/integration/test_live_view_over_distributed/test.py @@ -111,15 +111,28 @@ node2\t1\t11 client1.expect(prompt) client1.send("WATCH lv FORMAT CSV") - client1.expect('"node1",0,0,1\r\n.*"node1",1,1,1\r\n.*"node2",0,10,1\r\n.*"node2",1,11,1\r\n') + client1.expect('"node1",0,0,1') + client1.expect('"node1",1,1,1') + client1.expect('"node2",0,10,1') + client1.expect('"node2",1,11,1') client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) - client1.expect('"node1",0,0,2\r\n.*"node1",1,1,2\r\n.*"node1",2,2,2\r\n.*"node2",0,10,2\r\n.*"node2",1,11,2\r\n') + client1.expect('"node1",0,0,2') + client1.expect('"node1",1,1,2') + client1.expect('"node1",2,2,2') + client1.expect('"node2",0,10,2') + client1.expect('"node2",1,11,2') client2.send("INSERT INTO distributed_table VALUES ('node1', 0, 3), ('node3', 3, 3)") client2.expect(prompt) - client1.expect('"node1",0,0,3\r\n.*"node1",0,3,3\r\n.*"node1",1,1,3\r\n.*"node1",2,2,3\r\n.*"node2",0,10,3\r\n.*"node2",1,11,3\r\n.*"node3",3,3,3\r\n') + client1.expect('"node1",0,0,3') + client1.expect('"node1",0,3,3') + client1.expect('"node1",1,1,3') + client1.expect('"node1",2,2,3') + client1.expect('"node2",0,10,3') + client1.expect('"node2",1,11,3') + client1.expect('"node3",3,3,3') def test_watch_live_view_order_by_key(self, started_cluster, node, source): log = sys.stdout @@ -141,15 +154,28 @@ node2\t1\t11 client1.expect(prompt) client1.send("WATCH lv FORMAT CSV") - client1.expect('"node1",0,0,1\r\n.*"node2",0,10,1\r\n.*"node1",1,1,1\r\n.*"node2",1,11,1\r\n') + client1.expect('"node1",0,0,1') + client1.expect('"node2",0,10,1') + client1.expect('"node1",1,1,1') + client1.expect('"node2",1,11,1') client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) - client1.expect('"node1",0,0,2\r\n.*"node2",0,10,2\r\n.*"node1",1,1,2\r\n.*"node2",1,11,2\r\n.*"node1",2,2,2\r\n') + client1.expect('"node1",0,0,2') + client1.expect('"node2",0,10,2') + client1.expect('"node1",1,1,2') + client1.expect('"node2",1,11,2') + client1.expect('"node1",2,2,2') client2.send("INSERT INTO distributed_table VALUES ('node1', 0, 3), ('node3', 3, 3)") client2.expect(prompt) - client1.expect('"node1",0,0,3\r\n.*"node1",0,3,3\r\n.*"node2",0,10,3\r\n.*"node1",1,1,3\r\n.*"node2",1,11,3\r\n.*"node1",2,2,3\r\n.*"node3",3,3,3\r\n') + client1.expect('"node1",0,0,3') + client1.expect('"node1",0,3,3') + client1.expect('"node2",0,10,3') + client1.expect('"node1",1,1,3') + client1.expect('"node2",1,11,3') + client1.expect('"node1",2,2,3') + client1.expect('"node3",3,3,3') def test_watch_live_view_group_by_node(self, started_cluster, node, source): log = sys.stdout @@ -171,15 +197,19 @@ node2\t1\t11 client1.expect(prompt) client1.send("WATCH lv FORMAT CSV") - client1.expect('"node1",1,1\r\n.*"node2",21,1\r\n') + client1.expect('"node1",1,1') + client1.expect('"node2",21,1') client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) - client1.expect('"node1",3,2\r\n.*"node2",21,2\r\n') + client1.expect('"node1",3,2') + client1.expect('"node2",21,2') client2.send("INSERT INTO distributed_table VALUES ('node1', 0, 3), ('node3', 3, 3)") client2.expect(prompt) - client1.expect('"node1",6,3\r\n.*"node2",21,3\r\n.*"node3",3,3\r\n') + client1.expect('"node1",6,3') + client1.expect('"node2",21,3') + client1.expect('"node3",3,3') def test_watch_live_view_group_by_key(self, started_cluster, node, source): log = sys.stdout @@ -201,15 +231,21 @@ node2\t1\t11 client1.expect(prompt) client1.send("WATCH lv FORMAT CSV") - client1.expect("0,10,1\r\n.*1,12,1\r\n") + client1.expect('0,10,1') + client1.expect('1,12,1') client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) - client1.expect("0,10,2\r\n.*1,12,2\r\n.*2,2,2\r\n") + client1.expect('0,10,2') + client1.expect('1,12,2') + client1.expect('2,2,2') client2.send("INSERT INTO distributed_table VALUES ('node1', 0, 3), ('node1', 3, 3)") client2.expect(prompt) - client1.expect("0,13,3\r\n.*1,12,3\r\n.*2,2,3\r\n.*3,3,3\r\n") + client1.expect('0,13,3') + client1.expect('1,12,3') + client1.expect('2,2,3') + client1.expect('3,3,3') def test_watch_live_view_sum(self, started_cluster, node, source): From 4dcc0a0f86ee64cde4367386f5f23932dd6cda16 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jun 2020 00:07:06 +0300 Subject: [PATCH 146/211] Better diagnostics for logical error triggered by test 00992 --- .../CheckSortedBlockInputStream.cpp | 49 +++++++++---------- 1 file changed, 24 insertions(+), 25 deletions(-) diff --git a/src/DataStreams/CheckSortedBlockInputStream.cpp b/src/DataStreams/CheckSortedBlockInputStream.cpp index a56738a9a9f..a047d22751c 100644 --- a/src/DataStreams/CheckSortedBlockInputStream.cpp +++ b/src/DataStreams/CheckSortedBlockInputStream.cpp @@ -1,6 +1,9 @@ #include +#include +#include #include + namespace DB { @@ -35,17 +38,16 @@ CheckSortedBlockInputStream::addPositionsToSortDescriptions(const SortDescriptio return result; } -/// Compares values in columns. Columns must have equal types. -struct SortingLessOrEqualComparator + +Block CheckSortedBlockInputStream::readImpl() { - const SortDescriptionsWithPositions & sort_description; + Block block = children.back()->read(); + if (!block || block.rows() == 0) + return block; - explicit SortingLessOrEqualComparator(const SortDescriptionsWithPositions & sort_description_) - : sort_description(sort_description_) {} - - bool operator()(const Columns & left, size_t left_index, const Columns & right, size_t right_index) const + auto check = [this](const Columns & left, size_t left_index, const Columns & right, size_t right_index) { - for (const auto & elem : sort_description) + for (const auto & elem : sort_description_map) { size_t column_number = elem.column_number; @@ -54,30 +56,27 @@ struct SortingLessOrEqualComparator int res = elem.direction * left_col->compareAt(left_index, right_index, *right_col, elem.nulls_direction); if (res < 0) - return true; + { + return; + } else if (res > 0) - return false; + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Sort order of blocks violated for column {}, left: {}, right: {}.", + backQuoteIfNeed(elem.column_name), + applyVisitor(FieldVisitorDump(), (*left_col)[left_index]), + applyVisitor(FieldVisitorDump(), (*right_col)[right_index])); + } } - return true; - } -}; - -Block CheckSortedBlockInputStream::readImpl() -{ - Block block = children.back()->read(); - if (!block || block.rows() == 0) - return block; - - SortingLessOrEqualComparator less(sort_description_map); + }; auto block_columns = block.getColumns(); - if (!last_row.empty() && !less(last_row, 0, block_columns, 0)) - throw Exception("Sort order of blocks violated", ErrorCodes::LOGICAL_ERROR); + if (!last_row.empty()) + check(last_row, 0, block_columns, 0); size_t rows = block.rows(); for (size_t i = 1; i < rows; ++i) - if (!less(block_columns, i - 1, block_columns, i)) - throw Exception("Sort order of blocks violated", ErrorCodes::LOGICAL_ERROR); + check(block_columns, i - 1, block_columns, i); last_row.clear(); for (size_t i = 0; i < block.columns(); ++i) From 69008521f6486baef5e3473e38d53acf5842c21f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jun 2020 00:17:19 +0300 Subject: [PATCH 147/211] Fix build --- src/Functions/buildId.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Functions/buildId.cpp b/src/Functions/buildId.cpp index ef36c0065af..6daeed16e8b 100644 --- a/src/Functions/buildId.cpp +++ b/src/Functions/buildId.cpp @@ -52,7 +52,10 @@ void registerFunctionBuildId(FunctionFactory & factory) #else +namespace DB +{ class FunctionFactory; void registerFunctionBuildId(FunctionFactory &) {} +} #endif From 8cce416921736d27e3c302f1451c3aaf4c2b7f0e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jun 2020 01:12:35 +0300 Subject: [PATCH 148/211] Added a test with another hostname --- tests/integration/helpers/cluster.py | 2 ++ tests/integration/test_odbc_interaction/test.py | 8 ++++++-- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 1c1f758b291..2006d017bc2 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -668,6 +668,8 @@ services: - {env_file} security_opt: - label:disable + links: + - postgres1:postgre-sql.local {networks} {app_net} {ipv4_address} diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index c6b4e0eb5b0..7bd9a756702 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -222,11 +222,15 @@ def test_postgres_odbc_hached_dictionary_no_tty_pipe_overflow(started_cluster): def test_postgres_insert(started_cluster): conn = get_postgres_conn() conn.cursor().execute("truncate table clickhouse.test_table") - node1.query("create table pg_insert (column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;', 'clickhouse', 'test_table')") + + # Also test with Servername containing '.' and '-' symbols (see links in docker-compose template in helpers/cluster.py) + # This is needed to check parsing, validation and reconstruction of connection string. + + node1.query("create table pg_insert (column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')") node1.query("insert into pg_insert values (1, 'hello'), (2, 'world')") assert node1.query("select * from pg_insert") == '1\thello\n2\tworld\n' node1.query("insert into table function odbc('DSN=postgresql_odbc;', 'clickhouse', 'test_table') format CSV 3,test") - node1.query("insert into table function odbc('DSN=postgresql_odbc;', 'clickhouse', 'test_table') select number, 's' || toString(number) from numbers (4, 7)") + node1.query("insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table') select number, 's' || toString(number) from numbers (4, 7)") assert node1.query("select sum(column1), count(column1) from pg_insert") == "55\t10\n" assert node1.query("select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc;', 'clickhouse', 'test_table')))") == "55\t10\n" From c758567e4e23859500f2e82c3d67524fa7e1b822 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 22 Jun 2020 02:42:11 +0300 Subject: [PATCH 149/211] Do not run 01238_http_memory_tracking under sanitizers --- tests/queries/0_stateless/01238_http_memory_tracking.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/01238_http_memory_tracking.sh b/tests/queries/0_stateless/01238_http_memory_tracking.sh index 07c731c1ca1..e2d6646e20a 100755 --- a/tests/queries/0_stateless/01238_http_memory_tracking.sh +++ b/tests/queries/0_stateless/01238_http_memory_tracking.sh @@ -3,6 +3,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh +# Don't even try to do run under sanitizers, since they are too slow. +${CLICKHOUSE_LOCAL} --query "SELECT max(value LIKE '%sanitize%') FROM system.build_options" | grep -q '1' && echo 'Skip test for sanitizer build' && exit + # TODO: the test can be way more optimal set -o pipefail From 37985185431f799f8ff7f72d800f4269d389b9ca Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 22 Jun 2020 11:01:14 +0300 Subject: [PATCH 150/211] Update README.md --- README.md | 5 ----- 1 file changed, 5 deletions(-) diff --git a/README.md b/README.md index 991b15f60aa..ef39a163807 100644 --- a/README.md +++ b/README.md @@ -13,8 +13,3 @@ ClickHouse is an open-source column-oriented database management system that all * [Yandex.Messenger channel](https://yandex.ru/chat/#/join/20e380d9-c7be-4123-ab06-e95fb946975e) shares announcements and useful links in Russian. * [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any. * You can also [fill this form](https://clickhouse.tech/#meet) to meet Yandex ClickHouse team in person. - -## Upcoming Events - -* [ClickHouse Workshop in Novosibirsk](https://2020.codefest.ru/lecture/1628) on TBD date. -* [Yandex C++ Open-Source Sprints in Moscow](https://events.yandex.ru/events/otkrytyj-kod-v-yandek-28-03-2020) on TBD date. From 3510040bc7a1f92a8c8a56fbe57bc200df3eb1ae Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 22 Jun 2020 11:45:22 +0300 Subject: [PATCH 151/211] [docs] fix "zh" build (#11854) --- docs/zh/commercial/support.md | 24 +- docs/zh/operations/monitoring.md | 2 +- .../sampling-query-profiler.md | 2 +- .../settings.md | 16 +- docs/zh/operations/settings/settings.md | 6 +- docs/zh/operations/system-tables.md | 1168 ----------------- .../system-tables/asynchronous_metric_log.md | 8 + .../system-tables/asynchronous_metrics.md | 41 + docs/zh/operations/system-tables/clusters.md | 29 + docs/zh/operations/system-tables/columns.md | 27 + .../operations/system-tables/contributors.md | 45 + .../system-tables/data_type_families.md | 39 + docs/zh/operations/system-tables/databases.md | 12 + .../system-tables/detached_parts.md | 14 + .../operations/system-tables/dictionaries.md | 66 + docs/zh/operations/system-tables/disks.md | 31 + docs/zh/operations/system-tables/events.md | 37 + docs/zh/operations/system-tables/functions.md | 13 + .../system-tables/graphite_retentions.md | 20 + docs/zh/operations/system-tables/index.md | 50 + .../system-tables/merge_tree_settings.md | 16 + docs/zh/operations/system-tables/merges.md | 24 + .../zh/operations/system-tables/metric_log.md | 60 + docs/zh/operations/system-tables/metrics.md | 44 + docs/zh/operations/system-tables/mutations.md | 30 + docs/zh/operations/system-tables/numbers.md | 12 + .../zh/operations/system-tables/numbers_mt.md | 10 + docs/zh/operations/system-tables/one.md | 12 + docs/zh/operations/system-tables/part_log.md | 37 + docs/zh/operations/system-tables/parts.md | 85 ++ docs/zh/operations/system-tables/processes.md | 20 + docs/zh/operations/system-tables/query_log.md | 143 ++ .../system-tables/query_thread_log.md | 118 ++ docs/zh/operations/system-tables/replicas.md | 126 ++ docs/zh/operations/system-tables/settings.md | 55 + .../system-tables/storage_policies.md | 19 + .../operations/system-tables/table_engines.md | 40 + docs/zh/operations/system-tables/tables.md | 54 + docs/zh/operations/system-tables/text_log.md | 31 + docs/zh/operations/system-tables/trace_log.md | 53 + docs/zh/operations/system-tables/zookeeper.md | 75 ++ .../external-dictionaries/external-dicts.md | 2 +- .../sql-reference/functions/introspection.md | 2 +- docs/zh/sql-reference/operators/in.md | 205 ++- docs/zh/sql-reference/statements/alter.md | 2 +- docs/zh/sql-reference/statements/misc.md | 2 +- .../statements/select/array-join.md | 284 +++- .../statements/select/distinct.md | 65 +- .../sql-reference/statements/select/format.md | 20 +- .../sql-reference/statements/select/from.md | 46 +- .../statements/select/group-by.md | 134 +- .../sql-reference/statements/select/having.md | 16 +- .../sql-reference/statements/select/index.md | 165 ++- .../statements/select/into-outfile.md | 16 +- .../sql-reference/statements/select/join.md | 199 ++- .../statements/select/limit-by.md | 73 +- .../sql-reference/statements/select/limit.md | 16 +- .../statements/select/order-by.md | 74 +- .../statements/select/prewhere.md | 24 +- .../sql-reference/statements/select/sample.md | 115 +- .../statements/select/union-all.md | 37 +- .../sql-reference/statements/select/where.md | 17 +- .../sql-reference/statements/select/with.md | 82 +- docs/zh/sql-reference/syntax.md | 2 +- 64 files changed, 3107 insertions(+), 1205 deletions(-) mode change 120000 => 100644 docs/zh/commercial/support.md delete mode 100644 docs/zh/operations/system-tables.md create mode 100644 docs/zh/operations/system-tables/asynchronous_metric_log.md create mode 100644 docs/zh/operations/system-tables/asynchronous_metrics.md create mode 100644 docs/zh/operations/system-tables/clusters.md create mode 100644 docs/zh/operations/system-tables/columns.md create mode 100644 docs/zh/operations/system-tables/contributors.md create mode 100644 docs/zh/operations/system-tables/data_type_families.md create mode 100644 docs/zh/operations/system-tables/databases.md create mode 100644 docs/zh/operations/system-tables/detached_parts.md create mode 100644 docs/zh/operations/system-tables/dictionaries.md create mode 100644 docs/zh/operations/system-tables/disks.md create mode 100644 docs/zh/operations/system-tables/events.md create mode 100644 docs/zh/operations/system-tables/functions.md create mode 100644 docs/zh/operations/system-tables/graphite_retentions.md create mode 100644 docs/zh/operations/system-tables/index.md create mode 100644 docs/zh/operations/system-tables/merge_tree_settings.md create mode 100644 docs/zh/operations/system-tables/merges.md create mode 100644 docs/zh/operations/system-tables/metric_log.md create mode 100644 docs/zh/operations/system-tables/metrics.md create mode 100644 docs/zh/operations/system-tables/mutations.md create mode 100644 docs/zh/operations/system-tables/numbers.md create mode 100644 docs/zh/operations/system-tables/numbers_mt.md create mode 100644 docs/zh/operations/system-tables/one.md create mode 100644 docs/zh/operations/system-tables/part_log.md create mode 100644 docs/zh/operations/system-tables/parts.md create mode 100644 docs/zh/operations/system-tables/processes.md create mode 100644 docs/zh/operations/system-tables/query_log.md create mode 100644 docs/zh/operations/system-tables/query_thread_log.md create mode 100644 docs/zh/operations/system-tables/replicas.md create mode 100644 docs/zh/operations/system-tables/settings.md create mode 100644 docs/zh/operations/system-tables/storage_policies.md create mode 100644 docs/zh/operations/system-tables/table_engines.md create mode 100644 docs/zh/operations/system-tables/tables.md create mode 100644 docs/zh/operations/system-tables/text_log.md create mode 100644 docs/zh/operations/system-tables/trace_log.md create mode 100644 docs/zh/operations/system-tables/zookeeper.md mode change 120000 => 100644 docs/zh/sql-reference/operators/in.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/array-join.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/distinct.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/format.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/from.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/group-by.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/having.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/index.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/into-outfile.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/join.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/limit-by.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/limit.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/order-by.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/prewhere.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/sample.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/union-all.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/where.md mode change 120000 => 100644 docs/zh/sql-reference/statements/select/with.md diff --git a/docs/zh/commercial/support.md b/docs/zh/commercial/support.md deleted file mode 120000 index 1eb20ccf36a..00000000000 --- a/docs/zh/commercial/support.md +++ /dev/null @@ -1 +0,0 @@ -../../en/commercial/support.md \ No newline at end of file diff --git a/docs/zh/commercial/support.md b/docs/zh/commercial/support.md new file mode 100644 index 00000000000..e8462fc962e --- /dev/null +++ b/docs/zh/commercial/support.md @@ -0,0 +1,23 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_priority: 3 +toc_title: "\u788C\u83BD\u7984Support:" +--- + +# ClickHouse商业支持服务提供商 {#clickhouse-commercial-support-service-providers} + +!!! info "信息" + 如果您已经推出ClickHouse商业支持服务,请随时 [打开拉取请求](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/commercial/support.md) 将其添加到以下列表。 + +## 敏锐性 {#altinity} + +隆隆隆隆路虏脢..陇.貌.垄拢卢虏禄and陇.貌路.隆拢脳枚脢虏 隆隆隆隆路虏脢..陇.貌.垄拢卢虏禄.陇 访问 [www.altinity.com](https://www.altinity.com/) 欲了解更多信息. + +## Mafiree {#mafiree} + +[服务说明](http://mafiree.com/clickhouse-analytics-services.php) + +## MinervaDB {#minervadb} + +[服务说明](https://minervadb.com/index.php/clickhouse-consulting-and-support-by-minervadb/) diff --git a/docs/zh/operations/monitoring.md b/docs/zh/operations/monitoring.md index 0bf8556a870..ee913f998ca 100644 --- a/docs/zh/operations/monitoring.md +++ b/docs/zh/operations/monitoring.md @@ -33,7 +33,7 @@ ClickHouse 收集的指标项: - 服务用于计算的资源占用的各种指标。 - 关于查询处理的常见统计信息。 -可以在 [系统指标](system-tables.md#system_tables-metrics) ,[系统事件](system-tables.md#system_tables-events) 以及[系统异步指标](system-tables.md#system_tables-asynchronous_metrics) 等系统表查看所有的指标项。 +可以在 [系统指标](system-tables/metrics.md#system_tables-metrics) ,[系统事件](system-tables/events.md#system_tables-events) 以及[系统异步指标](system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) 等系统表查看所有的指标项。 可以配置ClickHouse 往 [石墨](https://github.com/graphite-project)导入指标。 参考 [石墨部分](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) 配置文件。在配置指标导出之前,需要参考Graphite[官方教程](https://graphite.readthedocs.io/en/latest/install.html)搭建服务。 diff --git a/docs/zh/operations/optimizing-performance/sampling-query-profiler.md b/docs/zh/operations/optimizing-performance/sampling-query-profiler.md index e4242199713..a1d75fda0c3 100644 --- a/docs/zh/operations/optimizing-performance/sampling-query-profiler.md +++ b/docs/zh/operations/optimizing-performance/sampling-query-profiler.md @@ -13,7 +13,7 @@ ClickHouse运行允许分析查询执行的采样探查器。 使用探查器, - 设置 [trace\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) 服务器配置部分。 - 本节配置 [trace\_log](../../operations/system-tables.md#system_tables-trace_log) 系统表包含探查器运行的结果。 它是默认配置的。 请记住,此表中的数据仅对正在运行的服务器有效。 服务器重新启动后,ClickHouse不会清理表,所有存储的虚拟内存地址都可能无效。 + 本节配置 [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) 系统表包含探查器运行的结果。 它是默认配置的。 请记住,此表中的数据仅对正在运行的服务器有效。 服务器重新启动后,ClickHouse不会清理表,所有存储的虚拟内存地址都可能无效。 - 设置 [query\_profiler\_cpu\_time\_period\_ns](../settings/settings.md#query_profiler_cpu_time_period_ns) 或 [query\_profiler\_real\_time\_period\_ns](../settings/settings.md#query_profiler_real_time_period_ns) 设置。 这两种设置可以同时使用。 diff --git a/docs/zh/operations/server-configuration-parameters/settings.md b/docs/zh/operations/server-configuration-parameters/settings.md index 2c9d611b6a7..252323351b2 100644 --- a/docs/zh/operations/server-configuration-parameters/settings.md +++ b/docs/zh/operations/server-configuration-parameters/settings.md @@ -145,10 +145,10 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly - interval – The interval for sending, in seconds. - timeout – The timeout for sending data, in seconds. - root\_path – Prefix for keys. -- metrics – Sending data from the [系统。指标](../../operations/system-tables.md#system_tables-metrics) 桌子 -- events – Sending deltas data accumulated for the time period from the [系统。活动](../../operations/system-tables.md#system_tables-events) 桌子 -- events\_cumulative – Sending cumulative data from the [系统。活动](../../operations/system-tables.md#system_tables-events) 桌子 -- asynchronous\_metrics – Sending data from the [系统。asynchronous\_metrics](../../operations/system-tables.md#system_tables-asynchronous_metrics) 桌子 +- metrics – Sending data from the [系统。指标](../../operations/system-tables/metrics.md#system_tables-metrics) 桌子 +- events – Sending deltas data accumulated for the time period from the [系统。活动](../../operations/system-tables/events.md#system_tables-events) 桌子 +- events\_cumulative – Sending cumulative data from the [系统。活动](../../operations/system-tables/events.md#system_tables-events) 桌子 +- asynchronous\_metrics – Sending data from the [系统。asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) 桌子 您可以配置多个 `` 条款 例如,您可以使用它以不同的时间间隔发送不同的数据。 @@ -503,7 +503,7 @@ SSL客户端/服务器配置。 记录与之关联的事件 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). 例如,添加或合并数据。 您可以使用日志来模拟合并算法并比较它们的特征。 您可以可视化合并过程。 -查询记录在 [系统。part\_log](../../operations/system-tables.md#system_tables-part-log) 表,而不是在一个单独的文件。 您可以在以下命令中配置此表的名称 `table` 参数(见下文)。 +查询记录在 [系统。part\_log](../../operations/system-tables/part_log.md#system_tables-part-log) 表,而不是在一个单独的文件。 您可以在以下命令中配置此表的名称 `table` 参数(见下文)。 使用以下参数配置日志记录: @@ -540,7 +540,7 @@ SSL客户端/服务器配置。 用于记录接收到的查询的设置 [log\_queries=1](../settings/settings.md) 设置。 -查询记录在 [系统。query\_log](../../operations/system-tables.md#system_tables-query_log) 表,而不是在一个单独的文件。 您可以更改表的名称 `table` 参数(见下文)。 +查询记录在 [系统。query\_log](../../operations/system-tables/query_log.md#system_tables-query_log) 表,而不是在一个单独的文件。 您可以更改表的名称 `table` 参数(见下文)。 使用以下参数配置日志记录: @@ -566,7 +566,7 @@ SSL客户端/服务器配置。 设置用于记录接收到的查询的线程 [log\_query\_threads=1](../settings/settings.md#settings-log-query-threads) 设置。 -查询记录在 [系统。query\_thread\_log](../../operations/system-tables.md#system_tables-query-thread-log) 表,而不是在一个单独的文件。 您可以更改表的名称 `table` 参数(见下文)。 +查询记录在 [系统。query\_thread\_log](../../operations/system-tables/query_thread_log.md#system_tables-query-thread-log) 表,而不是在一个单独的文件。 您可以更改表的名称 `table` 参数(见下文)。 使用以下参数配置日志记录: @@ -590,7 +590,7 @@ SSL客户端/服务器配置。 ## trace\_log {#server_configuration_parameters-trace_log} -设置为 [trace\_log](../../operations/system-tables.md#system_tables-trace_log) 系统表操作。 +设置为 [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) 系统表操作。 参数: diff --git a/docs/zh/operations/settings/settings.md b/docs/zh/operations/settings/settings.md index 07362dcaceb..d6c411c70fb 100644 --- a/docs/zh/operations/settings/settings.md +++ b/docs/zh/operations/settings/settings.md @@ -1165,7 +1165,7 @@ ClickHouse生成异常 另请参阅: -- 系统表 [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- 系统表 [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) ## query\_profiler\_cpu\_time\_period\_ns {#query_profiler_cpu_time_period_ns} @@ -1188,7 +1188,7 @@ ClickHouse生成异常 另请参阅: -- 系统表 [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- 系统表 [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) ## allow\_introspection\_functions {#settings-allow_introspection_functions} @@ -1204,7 +1204,7 @@ ClickHouse生成异常 **另请参阅** - [采样查询探查器](../optimizing-performance/sampling-query-profiler.md) -- 系统表 [trace\_log](../../operations/system-tables.md#system_tables-trace_log) +- 系统表 [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) ## input\_format\_parallel\_parsing {#input-format-parallel-parsing} diff --git a/docs/zh/operations/system-tables.md b/docs/zh/operations/system-tables.md deleted file mode 100644 index 5070d3455ab..00000000000 --- a/docs/zh/operations/system-tables.md +++ /dev/null @@ -1,1168 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 52 -toc_title: "\u7CFB\u7EDF\u8868" ---- - -# 系统表 {#system-tables} - -系统表用于实现系统的部分功能,并提供对有关系统如何工作的信息的访问。 -您无法删除系统表(但可以执行分离)。 -系统表没有包含磁盘上数据的文件或包含元数据的文件。 服务器在启动时创建所有系统表。 -系统表是只读的。 -它们位于 ‘system’ 数据库。 - -## 系统。asynchronous\_metrics {#system_tables-asynchronous_metrics} - -包含在后台定期计算的指标。 例如,在使用的RAM量。 - -列: - -- `metric` ([字符串](../sql-reference/data-types/string.md)) — Metric name. -- `value` ([Float64](../sql-reference/data-types/float.md)) — Metric value. - -**示例** - -``` sql -SELECT * FROM system.asynchronous_metrics LIMIT 10 -``` - -``` text -┌─metric──────────────────────────────────┬──────value─┐ -│ jemalloc.background_thread.run_interval │ 0 │ -│ jemalloc.background_thread.num_runs │ 0 │ -│ jemalloc.background_thread.num_threads │ 0 │ -│ jemalloc.retained │ 422551552 │ -│ jemalloc.mapped │ 1682989056 │ -│ jemalloc.resident │ 1656446976 │ -│ jemalloc.metadata_thp │ 0 │ -│ jemalloc.metadata │ 10226856 │ -│ UncompressedCacheCells │ 0 │ -│ MarkCacheFiles │ 0 │ -└─────────────────────────────────────────┴────────────┘ -``` - -**另请参阅** - -- [监测](monitoring.md) — Base concepts of ClickHouse monitoring. -- [系统。指标](#system_tables-metrics) — Contains instantly calculated metrics. -- [系统。活动](#system_tables-events) — Contains a number of events that have occurred. -- [系统。metric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - -## 系统。集群 {#system-clusters} - -包含有关配置文件中可用的集群及其中的服务器的信息。 - -列: - -- `cluster` (String) — The cluster name. -- `shard_num` (UInt32) — The shard number in the cluster, starting from 1. -- `shard_weight` (UInt32) — The relative weight of the shard when writing data. -- `replica_num` (UInt32) — The replica number in the shard, starting from 1. -- `host_name` (String) — The host name, as specified in the config. -- `host_address` (String) — The host IP address obtained from DNS. -- `port` (UInt16) — The port to use for connecting to the server. -- `user` (String) — The name of the user for connecting to the server. -- `errors_count` (UInt32)-此主机无法到达副本的次数。 -- `estimated_recovery_time` (UInt32)-剩下的秒数,直到副本错误计数归零,它被认为是恢复正常。 - -请注意 `errors_count` 每个查询集群更新一次,但 `estimated_recovery_time` 按需重新计算。 所以有可能是非零的情况 `errors_count` 和零 `estimated_recovery_time`,下一个查询将为零 `errors_count` 并尝试使用副本,就好像它没有错误。 - -**另请参阅** - -- [表引擎分布式](../engines/table-engines/special/distributed.md) -- [distributed\_replica\_error\_cap设置](settings/settings.md#settings-distributed_replica_error_cap) -- [distributed\_replica\_error\_half\_life设置](settings/settings.md#settings-distributed_replica_error_half_life) - -## 系统。列 {#system-columns} - -包含有关所有表中列的信息。 - -您可以使用此表获取类似于以下内容的信息 [DESCRIBE TABLE](../sql-reference/statements/misc.md#misc-describe-table) 查询,但对于多个表一次。 - -该 `system.columns` 表包含以下列(列类型显示在括号中): - -- `database` (String) — Database name. -- `table` (String) — Table name. -- `name` (String) — Column name. -- `type` (String) — Column type. -- `default_kind` (String) — Expression type (`DEFAULT`, `MATERIALIZED`, `ALIAS`)为默认值,如果没有定义,则为空字符串。 -- `default_expression` (String) — Expression for the default value, or an empty string if it is not defined. -- `data_compressed_bytes` (UInt64) — The size of compressed data, in bytes. -- `data_uncompressed_bytes` (UInt64) — The size of decompressed data, in bytes. -- `marks_bytes` (UInt64) — The size of marks, in bytes. -- `comment` (String) — Comment on the column, or an empty string if it is not defined. -- `is_in_partition_key` (UInt8) — Flag that indicates whether the column is in the partition expression. -- `is_in_sorting_key` (UInt8) — Flag that indicates whether the column is in the sorting key expression. -- `is_in_primary_key` (UInt8) — Flag that indicates whether the column is in the primary key expression. -- `is_in_sampling_key` (UInt8) — Flag that indicates whether the column is in the sampling key expression. - -## 系统。贡献者 {#system-contributors} - -包含有关贡献者的信息。 按随机顺序排列所有构造。 该顺序在查询执行时是随机的。 - -列: - -- `name` (String) — Contributor (author) name from git log. - -**示例** - -``` sql -SELECT * FROM system.contributors LIMIT 10 -``` - -``` text -┌─name─────────────┐ -│ Olga Khvostikova │ -│ Max Vetrov │ -│ LiuYangkuan │ -│ svladykin │ -│ zamulla │ -│ Šimon Podlipský │ -│ BayoNet │ -│ Ilya Khomutov │ -│ Amy Krishnevsky │ -│ Loud_Scream │ -└──────────────────┘ -``` - -要在表中找出自己,请使用查询: - -``` sql -SELECT * FROM system.contributors WHERE name='Olga Khvostikova' -``` - -``` text -┌─name─────────────┐ -│ Olga Khvostikova │ -└──────────────────┘ -``` - -## 系统。数据库 {#system-databases} - -此表包含一个名为"字符串"的列 ‘name’ – the name of a database. -服务器知道的每个数据库在表中都有相应的条目。 -该系统表用于实现 `SHOW DATABASES` 查询。 - -## 系统。detached\_parts {#system_tables-detached_parts} - -包含有关分离部分的信息 [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) 桌子 该 `reason` 列指定分离部件的原因。 对于用户分离的部件,原因是空的。 这些部件可以附加 [ALTER TABLE ATTACH PARTITION\|PART](../sql-reference/statements/alter.md#alter_attach-partition) 指挥部 有关其他列的说明,请参阅 [系统。零件](#system_tables-parts). 如果部件名称无效,某些列的值可能为 `NULL`. 这些部分可以删除 [ALTER TABLE DROP DETACHED PART](../sql-reference/statements/alter.md#alter_drop-detached). - -## 系统。字典 {#system_tables-dictionaries} - -包含以下信息 [外部字典](../sql-reference/dictionaries/external-dictionaries/external-dicts.md). - -列: - -- `database` ([字符串](../sql-reference/data-types/string.md)) — Name of the database containing the dictionary created by DDL query. Empty string for other dictionaries. -- `name` ([字符串](../sql-reference/data-types/string.md)) — [字典名称](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md). -- `status` ([枚举8](../sql-reference/data-types/enum.md)) — Dictionary status. Possible values: - - `NOT_LOADED` — Dictionary was not loaded because it was not used. - - `LOADED` — Dictionary loaded successfully. - - `FAILED` — Unable to load the dictionary as a result of an error. - - `LOADING` — Dictionary is loading now. - - `LOADED_AND_RELOADING` — Dictionary is loaded successfully, and is being reloaded right now (frequent reasons: [SYSTEM RELOAD DICTIONARY](../sql-reference/statements/system.md#query_language-system-reload-dictionary) 查询,超时,字典配置已更改)。 - - `FAILED_AND_RELOADING` — Could not load the dictionary as a result of an error and is loading now. -- `origin` ([字符串](../sql-reference/data-types/string.md)) — Path to the configuration file that describes the dictionary. -- `type` ([字符串](../sql-reference/data-types/string.md)) — Type of a dictionary allocation. [在内存中存储字典](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md). -- `key` — [密钥类型](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-key):数字键 ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) or Сomposite key ([字符串](../sql-reference/data-types/string.md)) — form “(type 1, type 2, …, type n)”. -- `attribute.names` ([阵列](../sql-reference/data-types/array.md)([字符串](../sql-reference/data-types/string.md))) — Array of [属性名称](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes) 由字典提供。 -- `attribute.types` ([阵列](../sql-reference/data-types/array.md)([字符串](../sql-reference/data-types/string.md))) — Corresponding array of [属性类型](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes) 这是由字典提供。 -- `bytes_allocated` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Amount of RAM allocated for the dictionary. -- `query_count` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of queries since the dictionary was loaded or since the last successful reboot. -- `hit_rate` ([Float64](../sql-reference/data-types/float.md)) — For cache dictionaries, the percentage of uses for which the value was in the cache. -- `element_count` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of items stored in the dictionary. -- `load_factor` ([Float64](../sql-reference/data-types/float.md)) — Percentage filled in the dictionary (for a hashed dictionary, the percentage filled in the hash table). -- `source` ([字符串](../sql-reference/data-types/string.md)) — Text describing the [数据源](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md) 为了字典 -- `lifetime_min` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Minimum [使用寿命](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) 在内存中的字典,之后ClickHouse尝试重新加载字典(如果 `invalidate_query` 被设置,那么只有当它已经改变)。 在几秒钟内设置。 -- `lifetime_max` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Maximum [使用寿命](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) 在内存中的字典,之后ClickHouse尝试重新加载字典(如果 `invalidate_query` 被设置,那么只有当它已经改变)。 在几秒钟内设置。 -- `loading_start_time` ([日期时间](../sql-reference/data-types/datetime.md)) — Start time for loading the dictionary. -- `last_successful_update_time` ([日期时间](../sql-reference/data-types/datetime.md)) — End time for loading or updating the dictionary. Helps to monitor some troubles with external sources and investigate causes. -- `loading_duration` ([Float32](../sql-reference/data-types/float.md)) — Duration of a dictionary loading. -- `last_exception` ([字符串](../sql-reference/data-types/string.md)) — Text of the error that occurs when creating or reloading the dictionary if the dictionary couldn't be created. - -**示例** - -配置字典。 - -``` sql -CREATE DICTIONARY dictdb.dict -( - `key` Int64 DEFAULT -1, - `value_default` String DEFAULT 'world', - `value_expression` String DEFAULT 'xxx' EXPRESSION 'toString(127 * 172)' -) -PRIMARY KEY key -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dicttbl' DB 'dictdb')) -LIFETIME(MIN 0 MAX 1) -LAYOUT(FLAT()) -``` - -确保字典已加载。 - -``` sql -SELECT * FROM system.dictionaries -``` - -``` text -┌─database─┬─name─┬─status─┬─origin──────┬─type─┬─key────┬─attribute.names──────────────────────┬─attribute.types─────┬─bytes_allocated─┬─query_count─┬─hit_rate─┬─element_count─┬───────────load_factor─┬─source─────────────────────┬─lifetime_min─┬─lifetime_max─┬──loading_start_time─┌──last_successful_update_time─┬──────loading_duration─┬─last_exception─┐ -│ dictdb │ dict │ LOADED │ dictdb.dict │ Flat │ UInt64 │ ['value_default','value_expression'] │ ['String','String'] │ 74032 │ 0 │ 1 │ 1 │ 0.0004887585532746823 │ ClickHouse: dictdb.dicttbl │ 0 │ 1 │ 2020-03-04 04:17:34 │ 2020-03-04 04:30:34 │ 0.002 │ │ -└──────────┴──────┴────────┴─────────────┴──────┴────────┴──────────────────────────────────────┴─────────────────────┴─────────────────┴─────────────┴──────────┴───────────────┴───────────────────────┴────────────────────────────┴──────────────┴──────────────┴─────────────────────┴──────────────────────────────┘───────────────────────┴────────────────┘ -``` - -## 系统。活动 {#system_tables-events} - -包含有关系统中发生的事件数的信息。 例如,在表中,您可以找到多少 `SELECT` 自ClickHouse服务器启动以来已处理查询。 - -列: - -- `event` ([字符串](../sql-reference/data-types/string.md)) — Event name. -- `value` ([UInt64](../sql-reference/data-types/int-uint.md)) — Number of events occurred. -- `description` ([字符串](../sql-reference/data-types/string.md)) — Event description. - -**示例** - -``` sql -SELECT * FROM system.events LIMIT 5 -``` - -``` text -┌─event─────────────────────────────────┬─value─┬─description────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ Query │ 12 │ Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries. │ -│ SelectQuery │ 8 │ Same as Query, but only for SELECT queries. │ -│ FileOpen │ 73 │ Number of files opened. │ -│ ReadBufferFromFileDescriptorRead │ 155 │ Number of reads (read/pread) from a file descriptor. Does not include sockets. │ -│ ReadBufferFromFileDescriptorReadBytes │ 9931 │ Number of bytes read from file descriptors. If the file is compressed, this will show the compressed data size. │ -└───────────────────────────────────────┴───────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -**另请参阅** - -- [系统。asynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. -- [系统。指标](#system_tables-metrics) — Contains instantly calculated metrics. -- [系统。metric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. -- [监测](monitoring.md) — Base concepts of ClickHouse monitoring. - -## 系统。功能 {#system-functions} - -包含有关正常函数和聚合函数的信息。 - -列: - -- `name`(`String`) – The name of the function. -- `is_aggregate`(`UInt8`) — Whether the function is aggregate. - -## 系统。graphite\_retentions {#system-graphite-retentions} - -包含有关参数的信息 [graphite\_rollup](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) 这是在表中使用 [\*GraphiteMergeTree](../engines/table-engines/mergetree-family/graphitemergetree.md) 引擎 - -列: - -- `config_name` (字符串) - `graphite_rollup` 参数名称。 -- `regexp` (String)-指标名称的模式。 -- `function` (String)-聚合函数的名称。 -- `age` (UInt64)-以秒为单位的数据的最小期限。 -- `precision` (UInt64)-如何精确地定义以秒为单位的数据的年龄。 -- `priority` (UInt16)-模式优先级。 -- `is_default` (UInt8)-模式是否为默认值。 -- `Tables.database` (Array(String))-使用数据库表名称的数组 `config_name` 参数。 -- `Tables.table` (Array(String))-使用表名称的数组 `config_name` 参数。 - -## 系统。合并 {#system-merges} - -包含有关MergeTree系列中表当前正在进行的合并和部件突变的信息。 - -列: - -- `database` (String) — The name of the database the table is in. -- `table` (String) — Table name. -- `elapsed` (Float64) — The time elapsed (in seconds) since the merge started. -- `progress` (Float64) — The percentage of completed work from 0 to 1. -- `num_parts` (UInt64) — The number of pieces to be merged. -- `result_part_name` (String) — The name of the part that will be formed as the result of merging. -- `is_mutation` (UInt8)-1如果这个过程是一个部分突变. -- `total_size_bytes_compressed` (UInt64) — The total size of the compressed data in the merged chunks. -- `total_size_marks` (UInt64) — The total number of marks in the merged parts. -- `bytes_read_uncompressed` (UInt64) — Number of bytes read, uncompressed. -- `rows_read` (UInt64) — Number of rows read. -- `bytes_written_uncompressed` (UInt64) — Number of bytes written, uncompressed. -- `rows_written` (UInt64) — Number of rows written. - -## 系统。指标 {#system_tables-metrics} - -包含可以立即计算或具有当前值的指标。 例如,同时处理的查询的数量或当前副本的延迟。 此表始终是最新的。 - -列: - -- `metric` ([字符串](../sql-reference/data-types/string.md)) — Metric name. -- `value` ([Int64](../sql-reference/data-types/int-uint.md)) — Metric value. -- `description` ([字符串](../sql-reference/data-types/string.md)) — Metric description. - -支持的指标列表,您可以在 [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp) ClickHouse的源文件。 - -**示例** - -``` sql -SELECT * FROM system.metrics LIMIT 10 -``` - -``` text -┌─metric─────────────────────┬─value─┬─description──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ Query │ 1 │ Number of executing queries │ -│ Merge │ 0 │ Number of executing background merges │ -│ PartMutation │ 0 │ Number of mutations (ALTER DELETE/UPDATE) │ -│ ReplicatedFetch │ 0 │ Number of data parts being fetched from replicas │ -│ ReplicatedSend │ 0 │ Number of data parts being sent to replicas │ -│ ReplicatedChecks │ 0 │ Number of data parts checking for consistency │ -│ BackgroundPoolTask │ 0 │ Number of active tasks in BackgroundProcessingPool (merges, mutations, fetches, or replication queue bookkeeping) │ -│ BackgroundSchedulePoolTask │ 0 │ Number of active tasks in BackgroundSchedulePool. This pool is used for periodic ReplicatedMergeTree tasks, like cleaning old data parts, altering data parts, replica re-initialization, etc. │ -│ DiskSpaceReservedForMerge │ 0 │ Disk space reserved for currently running background merges. It is slightly more than the total size of currently merging parts. │ -│ DistributedSend │ 0 │ Number of connections to remote servers sending data that was INSERTed into Distributed tables. Both synchronous and asynchronous mode. │ -└────────────────────────────┴───────┴──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -**另请参阅** - -- [系统。asynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. -- [系统。活动](#system_tables-events) — Contains a number of events that occurred. -- [系统。metric\_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. -- [监测](monitoring.md) — Base concepts of ClickHouse monitoring. - -## 系统。metric\_log {#system_tables-metric_log} - -包含表中度量值的历史记录 `system.metrics` 和 `system.events`,定期刷新到磁盘。 -打开指标历史记录收集 `system.metric_log`,创建 `/etc/clickhouse-server/config.d/metric_log.xml` 具有以下内容: - -``` xml - - - system - metric_log
- 7500 - 1000 -
-
-``` - -**示例** - -``` sql -SELECT * FROM system.metric_log LIMIT 1 FORMAT Vertical; -``` - -``` text -Row 1: -────── -event_date: 2020-02-18 -event_time: 2020-02-18 07:15:33 -milliseconds: 554 -ProfileEvent_Query: 0 -ProfileEvent_SelectQuery: 0 -ProfileEvent_InsertQuery: 0 -ProfileEvent_FileOpen: 0 -ProfileEvent_Seek: 0 -ProfileEvent_ReadBufferFromFileDescriptorRead: 1 -ProfileEvent_ReadBufferFromFileDescriptorReadFailed: 0 -ProfileEvent_ReadBufferFromFileDescriptorReadBytes: 0 -ProfileEvent_WriteBufferFromFileDescriptorWrite: 1 -ProfileEvent_WriteBufferFromFileDescriptorWriteFailed: 0 -ProfileEvent_WriteBufferFromFileDescriptorWriteBytes: 56 -... -CurrentMetric_Query: 0 -CurrentMetric_Merge: 0 -CurrentMetric_PartMutation: 0 -CurrentMetric_ReplicatedFetch: 0 -CurrentMetric_ReplicatedSend: 0 -CurrentMetric_ReplicatedChecks: 0 -... -``` - -**另请参阅** - -- [系统。asynchronous\_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. -- [系统。活动](#system_tables-events) — Contains a number of events that occurred. -- [系统。指标](#system_tables-metrics) — Contains instantly calculated metrics. -- [监测](monitoring.md) — Base concepts of ClickHouse monitoring. - -## 系统。数字 {#system-numbers} - -此表包含一个名为UInt64的列 ‘number’ 它包含几乎所有从零开始的自然数。 -您可以使用此表进行测试,或者如果您需要进行暴力搜索。 -从此表中读取的内容不是并行的。 - -## 系统。numbers\_mt {#system-numbers-mt} - -一样的 ‘system.numbers’ 但读取是并行的。 这些数字可以以任何顺序返回。 -用于测试。 - -## 系统。一 {#system-one} - -此表包含一行,其中包含一行 ‘dummy’ UInt8列包含值0。 -如果SELECT查询未指定FROM子句,则使用此表。 -这与其他Dbms中的双表类似。 - -## 系统。零件 {#system_tables-parts} - -包含有关的部分信息 [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) 桌子 - -每行描述一个数据部分。 - -列: - -- `partition` (String) – The partition name. To learn what a partition is, see the description of the [ALTER](../sql-reference/statements/alter.md#query_language_queries_alter) 查询。 - - 格式: - - - `YYYYMM` 用于按月自动分区。 - - `any_string` 手动分区时。 - -- `name` (`String`) – Name of the data part. - -- `active` (`UInt8`) – Flag that indicates whether the data part is active. If a data part is active, it's used in a table. Otherwise, it's deleted. Inactive data parts remain after merging. - -- `marks` (`UInt64`) – The number of marks. To get the approximate number of rows in a data part, multiply `marks` 通过索引粒度(通常为8192)(此提示不适用于自适应粒度)。 - -- `rows` (`UInt64`) – The number of rows. - -- `bytes_on_disk` (`UInt64`) – Total size of all the data part files in bytes. - -- `data_compressed_bytes` (`UInt64`) – Total size of compressed data in the data part. All the auxiliary files (for example, files with marks) are not included. - -- `data_uncompressed_bytes` (`UInt64`) – Total size of uncompressed data in the data part. All the auxiliary files (for example, files with marks) are not included. - -- `marks_bytes` (`UInt64`) – The size of the file with marks. - -- `modification_time` (`DateTime`) – The time the directory with the data part was modified. This usually corresponds to the time of data part creation.\| - -- `remove_time` (`DateTime`) – The time when the data part became inactive. - -- `refcount` (`UInt32`) – The number of places where the data part is used. A value greater than 2 indicates that the data part is used in queries or merges. - -- `min_date` (`Date`) – The minimum value of the date key in the data part. - -- `max_date` (`Date`) – The maximum value of the date key in the data part. - -- `min_time` (`DateTime`) – The minimum value of the date and time key in the data part. - -- `max_time`(`DateTime`) – The maximum value of the date and time key in the data part. - -- `partition_id` (`String`) – ID of the partition. - -- `min_block_number` (`UInt64`) – The minimum number of data parts that make up the current part after merging. - -- `max_block_number` (`UInt64`) – The maximum number of data parts that make up the current part after merging. - -- `level` (`UInt32`) – Depth of the merge tree. Zero means that the current part was created by insert rather than by merging other parts. - -- `data_version` (`UInt64`) – Number that is used to determine which mutations should be applied to the data part (mutations with a version higher than `data_version`). - -- `primary_key_bytes_in_memory` (`UInt64`) – The amount of memory (in bytes) used by primary key values. - -- `primary_key_bytes_in_memory_allocated` (`UInt64`) – The amount of memory (in bytes) reserved for primary key values. - -- `is_frozen` (`UInt8`) – Flag that shows that a partition data backup exists. 1, the backup exists. 0, the backup doesn't exist. For more details, see [FREEZE PARTITION](../sql-reference/statements/alter.md#alter_freeze-partition) - -- `database` (`String`) – Name of the database. - -- `table` (`String`) – Name of the table. - -- `engine` (`String`) – Name of the table engine without parameters. - -- `path` (`String`) – Absolute path to the folder with data part files. - -- `disk` (`String`) – Name of a disk that stores the data part. - -- `hash_of_all_files` (`String`) – [sipHash128](../sql-reference/functions/hash-functions.md#hash_functions-siphash128) 的压缩文件。 - -- `hash_of_uncompressed_files` (`String`) – [sipHash128](../sql-reference/functions/hash-functions.md#hash_functions-siphash128) 未压缩的文件(带标记的文件,索引文件等。). - -- `uncompressed_hash_of_compressed_files` (`String`) – [sipHash128](../sql-reference/functions/hash-functions.md#hash_functions-siphash128) 压缩文件中的数据,就好像它们是未压缩的。 - -- `bytes` (`UInt64`) – Alias for `bytes_on_disk`. - -- `marks_size` (`UInt64`) – Alias for `marks_bytes`. - -## 系统。part\_log {#system_tables-part-log} - -该 `system.part_log` 表只有当创建 [part\_log](server-configuration-parameters/settings.md#server_configuration_parameters-part-log) 指定了服务器设置。 - -此表包含与以下情况发生的事件有关的信息 [数据部分](../engines/table-engines/mergetree-family/custom-partitioning-key.md) 在 [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) 家庭表,例如添加或合并数据。 - -该 `system.part_log` 表包含以下列: - -- `event_type` (Enum) — Type of the event that occurred with the data part. Can have one of the following values: - - `NEW_PART` — Inserting of a new data part. - - `MERGE_PARTS` — Merging of data parts. - - `DOWNLOAD_PART` — Downloading a data part. - - `REMOVE_PART` — Removing or detaching a data part using [DETACH PARTITION](../sql-reference/statements/alter.md#alter_detach-partition). - - `MUTATE_PART` — Mutating of a data part. - - `MOVE_PART` — Moving the data part from the one disk to another one. -- `event_date` (Date) — Event date. -- `event_time` (DateTime) — Event time. -- `duration_ms` (UInt64) — Duration. -- `database` (String) — Name of the database the data part is in. -- `table` (String) — Name of the table the data part is in. -- `part_name` (String) — Name of the data part. -- `partition_id` (String) — ID of the partition that the data part was inserted to. The column takes the ‘all’ 值,如果分区是由 `tuple()`. -- `rows` (UInt64) — The number of rows in the data part. -- `size_in_bytes` (UInt64) — Size of the data part in bytes. -- `merged_from` (Array(String)) — An array of names of the parts which the current part was made up from (after the merge). -- `bytes_uncompressed` (UInt64) — Size of uncompressed bytes. -- `read_rows` (UInt64) — The number of rows was read during the merge. -- `read_bytes` (UInt64) — The number of bytes was read during the merge. -- `error` (UInt16) — The code number of the occurred error. -- `exception` (String) — Text message of the occurred error. - -该 `system.part_log` 表的第一个插入数据到后创建 `MergeTree` 桌子 - -## 系统。流程 {#system_tables-processes} - -该系统表用于实现 `SHOW PROCESSLIST` 查询。 - -列: - -- `user` (String) – The user who made the query. Keep in mind that for distributed processing, queries are sent to remote servers under the `default` 用户。 该字段包含特定查询的用户名,而不是此查询启动的查询的用户名。 -- `address` (String) – The IP address the request was made from. The same for distributed processing. To track where a distributed query was originally made from, look at `system.processes` 查询请求者服务器上。 -- `elapsed` (Float64) – The time in seconds since request execution started. -- `rows_read` (UInt64) – The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. -- `bytes_read` (UInt64) – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. -- `total_rows_approx` (UInt64) – The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known. -- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max\_memory\_usage](../operations/settings/query-complexity.md#settings_max_memory_usage) 设置。 -- `query` (String) – The query text. For `INSERT`,它不包括要插入的数据。 -- `query_id` (String) – Query ID, if defined. - -## 系统。text\_log {#system-tables-text-log} - -包含日志记录条目。 进入该表的日志记录级别可以通过以下方式进行限制 `text_log.level` 服务器设置。 - -列: - -- `event_date` (`Date`)-条目的日期。 -- `event_time` (`DateTime`)-条目的时间。 -- `microseconds` (`UInt32`)-条目的微秒。 -- `thread_name` (String) — Name of the thread from which the logging was done. -- `thread_id` (UInt64) — OS thread ID. -- `level` (`Enum8`)-入门级。 - - `'Fatal' = 1` - - `'Critical' = 2` - - `'Error' = 3` - - `'Warning' = 4` - - `'Notice' = 5` - - `'Information' = 6` - - `'Debug' = 7` - - `'Trace' = 8` -- `query_id` (`String`)-查询的ID。 -- `logger_name` (`LowCardinality(String)`) - Name of the logger (i.e. `DDLWorker`) -- `message` (`String`)-消息本身。 -- `revision` (`UInt32`)-ClickHouse修订。 -- `source_file` (`LowCardinality(String)`)-从中完成日志记录的源文件。 -- `source_line` (`UInt64`)-从中完成日志记录的源代码行。 - -## 系统。query\_log {#system_tables-query_log} - -包含有关查询执行的信息。 对于每个查询,您可以看到处理开始时间,处理持续时间,错误消息和其他信息。 - -!!! note "注" - 该表不包含以下内容的输入数据 `INSERT` 查询。 - -ClickHouse仅在以下情况下创建此表 [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) 指定服务器参数。 此参数设置日志记录规则,例如日志记录间隔或将记录查询的表的名称。 - -要启用查询日志记录,请设置 [log\_queries](settings/settings.md#settings-log-queries) 参数为1。 有关详细信息,请参阅 [设置](settings/settings.md) 科。 - -该 `system.query_log` 表注册两种查询: - -1. 客户端直接运行的初始查询。 -2. 由其他查询启动的子查询(用于分布式查询执行)。 对于这些类型的查询,有关父查询的信息显示在 `initial_*` 列。 - -列: - -- `type` (`Enum8`) — Type of event that occurred when executing the query. Values: - - `'QueryStart' = 1` — Successful start of query execution. - - `'QueryFinish' = 2` — Successful end of query execution. - - `'ExceptionBeforeStart' = 3` — Exception before the start of query execution. - - `'ExceptionWhileProcessing' = 4` — Exception during the query execution. -- `event_date` (Date) — Query starting date. -- `event_time` (DateTime) — Query starting time. -- `query_start_time` (DateTime) — Start time of query execution. -- `query_duration_ms` (UInt64) — Duration of query execution. -- `read_rows` (UInt64) — Number of read rows. -- `read_bytes` (UInt64) — Number of read bytes. -- `written_rows` (UInt64) — For `INSERT` 查询,写入的行数。 对于其他查询,列值为0。 -- `written_bytes` (UInt64) — For `INSERT` 查询时,写入的字节数。 对于其他查询,列值为0。 -- `result_rows` (UInt64) — Number of rows in the result. -- `result_bytes` (UInt64) — Number of bytes in the result. -- `memory_usage` (UInt64) — Memory consumption by the query. -- `query` (String) — Query string. -- `exception` (String) — Exception message. -- `stack_trace` (String) — Stack trace (a list of methods called before the error occurred). An empty string, if the query is completed successfully. -- `is_initial_query` (UInt8) — Query type. Possible values: - - 1 — Query was initiated by the client. - - 0 — Query was initiated by another query for distributed query execution. -- `user` (String) — Name of the user who initiated the current query. -- `query_id` (String) — ID of the query. -- `address` (IPv6) — IP address that was used to make the query. -- `port` (UInt16) — The client port that was used to make the query. -- `initial_user` (String) — Name of the user who ran the initial query (for distributed query execution). -- `initial_query_id` (String) — ID of the initial query (for distributed query execution). -- `initial_address` (IPv6) — IP address that the parent query was launched from. -- `initial_port` (UInt16) — The client port that was used to make the parent query. -- `interface` (UInt8) — Interface that the query was initiated from. Possible values: - - 1 — TCP. - - 2 — HTTP. -- `os_user` (String) — OS's username who runs [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md). -- `client_hostname` (String) — Hostname of the client machine where the [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md) 或者运行另一个TCP客户端。 -- `client_name` (String) — The [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md) 或另一个TCP客户端名称。 -- `client_revision` (UInt32) — Revision of the [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md) 或另一个TCP客户端。 -- `client_version_major` (UInt32) — Major version of the [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md) 或另一个TCP客户端。 -- `client_version_minor` (UInt32) — Minor version of the [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md) 或另一个TCP客户端。 -- `client_version_patch` (UInt32) — Patch component of the [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md) 或另一个TCP客户端版本。 -- `http_method` (UInt8) — HTTP method that initiated the query. Possible values: - - 0 — The query was launched from the TCP interface. - - 1 — `GET` 方法被使用。 - - 2 — `POST` 方法被使用。 -- `http_user_agent` (String) — The `UserAgent` http请求中传递的标头。 -- `quota_key` (String) — The “quota key” 在指定 [配额](quotas.md) 设置(见 `keyed`). -- `revision` (UInt32) — ClickHouse revision. -- `thread_numbers` (Array(UInt32)) — Number of threads that are participating in query execution. -- `ProfileEvents.Names` (Array(String)) — Counters that measure different metrics. The description of them could be found in the table [系统。活动](#system_tables-events) -- `ProfileEvents.Values` (Array(UInt64)) — Values of metrics that are listed in the `ProfileEvents.Names` 列。 -- `Settings.Names` (Array(String)) — Names of settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` 参数为1。 -- `Settings.Values` (Array(String)) — Values of settings that are listed in the `Settings.Names` 列。 - -每个查询创建一个或两个行中 `query_log` 表,具体取决于查询的状态: - -1. 如果查询执行成功,将创建两个类型为1和2的事件(请参阅 `type` 列)。 -2. 如果在查询处理过程中发生错误,将创建两个类型为1和4的事件。 -3. 如果在启动查询之前发生错误,将创建类型为3的单个事件。 - -默认情况下,日志以7.5秒的间隔添加到表中。 您可以在设置此时间间隔 [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) 服务器设置(请参阅 `flush_interval_milliseconds` 参数)。 要强制将日志从内存缓冲区刷新到表中,请使用 `SYSTEM FLUSH LOGS` 查询。 - -当手动删除表时,它将自动动态创建。 请注意,所有以前的日志将被删除。 - -!!! note "注" - 日志的存储周期是无限的。 日志不会自动从表中删除。 您需要自己组织删除过时的日志。 - -您可以指定一个任意的分区键 `system.query_log` 表中的 [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) 服务器设置(请参阅 `partition_by` 参数)。 - -## 系统。query\_thread\_log {#system_tables-query-thread-log} - -该表包含有关每个查询执行线程的信息。 - -ClickHouse仅在以下情况下创建此表 [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) 指定服务器参数。 此参数设置日志记录规则,例如日志记录间隔或将记录查询的表的名称。 - -要启用查询日志记录,请设置 [log\_query\_threads](settings/settings.md#settings-log-query-threads) 参数为1。 有关详细信息,请参阅 [设置](settings/settings.md) 科。 - -列: - -- `event_date` (Date) — the date when the thread has finished execution of the query. -- `event_time` (DateTime) — the date and time when the thread has finished execution of the query. -- `query_start_time` (DateTime) — Start time of query execution. -- `query_duration_ms` (UInt64) — Duration of query execution. -- `read_rows` (UInt64) — Number of read rows. -- `read_bytes` (UInt64) — Number of read bytes. -- `written_rows` (UInt64) — For `INSERT` 查询,写入的行数。 对于其他查询,列值为0。 -- `written_bytes` (UInt64) — For `INSERT` 查询时,写入的字节数。 对于其他查询,列值为0。 -- `memory_usage` (Int64) — The difference between the amount of allocated and freed memory in context of this thread. -- `peak_memory_usage` (Int64) — The maximum difference between the amount of allocated and freed memory in context of this thread. -- `thread_name` (String) — Name of the thread. -- `thread_number` (UInt32) — Internal thread ID. -- `os_thread_id` (Int32) — OS thread ID. -- `master_thread_id` (UInt64) — OS initial ID of initial thread. -- `query` (String) — Query string. -- `is_initial_query` (UInt8) — Query type. Possible values: - - 1 — Query was initiated by the client. - - 0 — Query was initiated by another query for distributed query execution. -- `user` (String) — Name of the user who initiated the current query. -- `query_id` (String) — ID of the query. -- `address` (IPv6) — IP address that was used to make the query. -- `port` (UInt16) — The client port that was used to make the query. -- `initial_user` (String) — Name of the user who ran the initial query (for distributed query execution). -- `initial_query_id` (String) — ID of the initial query (for distributed query execution). -- `initial_address` (IPv6) — IP address that the parent query was launched from. -- `initial_port` (UInt16) — The client port that was used to make the parent query. -- `interface` (UInt8) — Interface that the query was initiated from. Possible values: - - 1 — TCP. - - 2 — HTTP. -- `os_user` (String) — OS's username who runs [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md). -- `client_hostname` (String) — Hostname of the client machine where the [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md) 或者运行另一个TCP客户端。 -- `client_name` (String) — The [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md) 或另一个TCP客户端名称。 -- `client_revision` (UInt32) — Revision of the [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md) 或另一个TCP客户端。 -- `client_version_major` (UInt32) — Major version of the [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md) 或另一个TCP客户端。 -- `client_version_minor` (UInt32) — Minor version of the [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md) 或另一个TCP客户端。 -- `client_version_patch` (UInt32) — Patch component of the [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md) 或另一个TCP客户端版本。 -- `http_method` (UInt8) — HTTP method that initiated the query. Possible values: - - 0 — The query was launched from the TCP interface. - - 1 — `GET` 方法被使用。 - - 2 — `POST` 方法被使用。 -- `http_user_agent` (String) — The `UserAgent` http请求中传递的标头。 -- `quota_key` (String) — The “quota key” 在指定 [配额](quotas.md) 设置(见 `keyed`). -- `revision` (UInt32) — ClickHouse revision. -- `ProfileEvents.Names` (Array(String)) — Counters that measure different metrics for this thread. The description of them could be found in the table [系统。活动](#system_tables-events) -- `ProfileEvents.Values` (Array(UInt64)) — Values of metrics for this thread that are listed in the `ProfileEvents.Names` 列。 - -默认情况下,日志以7.5秒的间隔添加到表中。 您可以在设置此时间间隔 [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) 服务器设置(请参阅 `flush_interval_milliseconds` 参数)。 要强制将日志从内存缓冲区刷新到表中,请使用 `SYSTEM FLUSH LOGS` 查询。 - -当手动删除表时,它将自动动态创建。 请注意,所有以前的日志将被删除。 - -!!! note "注" - 日志的存储周期是无限的。 日志不会自动从表中删除。 您需要自己组织删除过时的日志。 - -您可以指定一个任意的分区键 `system.query_thread_log` 表中的 [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) 服务器设置(请参阅 `partition_by` 参数)。 - -## 系统。trace\_log {#system_tables-trace_log} - -包含采样查询探查器收集的堆栈跟踪。 - -ClickHouse创建此表时 [trace\_log](server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) 服务器配置部分被设置。 也是 [query\_profiler\_real\_time\_period\_ns](settings/settings.md#query_profiler_real_time_period_ns) 和 [query\_profiler\_cpu\_time\_period\_ns](settings/settings.md#query_profiler_cpu_time_period_ns) 应设置设置。 - -要分析日志,请使用 `addressToLine`, `addressToSymbol` 和 `demangle` 内省功能。 - -列: - -- `event_date` ([日期](../sql-reference/data-types/date.md)) — Date of sampling moment. - -- `event_time` ([日期时间](../sql-reference/data-types/datetime.md)) — Timestamp of the sampling moment. - -- `timestamp_ns` ([UInt64](../sql-reference/data-types/int-uint.md)) — Timestamp of the sampling moment in nanoseconds. - -- `revision` ([UInt32](../sql-reference/data-types/int-uint.md)) — ClickHouse server build revision. - - 通过以下方式连接到服务器 `clickhouse-client`,你看到的字符串类似于 `Connected to ClickHouse server version 19.18.1 revision 54429.`. 该字段包含 `revision`,但不是 `version` 的服务器。 - -- `timer_type` ([枚举8](../sql-reference/data-types/enum.md)) — Timer type: - - - `Real` 表示挂钟时间。 - - `CPU` 表示CPU时间。 - -- `thread_number` ([UInt32](../sql-reference/data-types/int-uint.md)) — Thread identifier. - -- `query_id` ([字符串](../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query\_log](#system_tables-query_log) 系统表. - -- `trace` ([数组(UInt64)](../sql-reference/data-types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process. - -**示例** - -``` sql -SELECT * FROM system.trace_log LIMIT 1 \G -``` - -``` text -Row 1: -────── -event_date: 2019-11-15 -event_time: 2019-11-15 15:09:38 -revision: 54428 -timer_type: Real -thread_number: 48 -query_id: acc4d61f-5bd1-4a3e-bc91-2180be37c915 -trace: [94222141367858,94222152240175,94222152325351,94222152329944,94222152330796,94222151449980,94222144088167,94222151682763,94222144088167,94222151682763,94222144088167,94222144058283,94222144059248,94222091840750,94222091842302,94222091831228,94222189631488,140509950166747,140509942945935] -``` - -## 系统。副本 {#system_tables-replicas} - -包含驻留在本地服务器上的复制表的信息和状态。 -此表可用于监视。 该表对于每个已复制的\*表都包含一行。 - -示例: - -``` sql -SELECT * -FROM system.replicas -WHERE table = 'visits' -FORMAT Vertical -``` - -``` text -Row 1: -────── -database: merge -table: visits -engine: ReplicatedCollapsingMergeTree -is_leader: 1 -can_become_leader: 1 -is_readonly: 0 -is_session_expired: 0 -future_parts: 1 -parts_to_check: 0 -zookeeper_path: /clickhouse/tables/01-06/visits -replica_name: example01-06-1.yandex.ru -replica_path: /clickhouse/tables/01-06/visits/replicas/example01-06-1.yandex.ru -columns_version: 9 -queue_size: 1 -inserts_in_queue: 0 -merges_in_queue: 1 -part_mutations_in_queue: 0 -queue_oldest_time: 2020-02-20 08:34:30 -inserts_oldest_time: 0000-00-00 00:00:00 -merges_oldest_time: 2020-02-20 08:34:30 -part_mutations_oldest_time: 0000-00-00 00:00:00 -oldest_part_to_get: -oldest_part_to_merge_to: 20200220_20284_20840_7 -oldest_part_to_mutate_to: -log_max_index: 596273 -log_pointer: 596274 -last_queue_update: 2020-02-20 08:34:32 -absolute_delay: 0 -total_replicas: 2 -active_replicas: 2 -``` - -列: - -- `database` (`String`)-数据库名称 -- `table` (`String`)-表名 -- `engine` (`String`)-表引擎名称 -- `is_leader` (`UInt8`)-副本是否是领导者。 - 一次只有一个副本可以成为领导者。 领导者负责选择要执行的后台合并。 - 请注意,可以对任何可用且在ZK中具有会话的副本执行写操作,而不管该副本是否为leader。 -- `can_become_leader` (`UInt8`)-副本是否可以当选为领导者。 -- `is_readonly` (`UInt8`)-副本是否处于只读模式。 - 如果配置没有ZooKeeper的部分,如果在ZooKeeper中重新初始化会话时发生未知错误,以及在ZooKeeper中重新初始化会话时发生未知错误,则此模式将打开。 -- `is_session_expired` (`UInt8`)-与ZooKeeper的会话已经过期。 基本上一样 `is_readonly`. -- `future_parts` (`UInt32`)-由于尚未完成的插入或合并而显示的数据部分的数量。 -- `parts_to_check` (`UInt32`)-队列中用于验证的数据部分的数量。 如果怀疑零件可能已损坏,则将其放入验证队列。 -- `zookeeper_path` (`String`)-在ZooKeeper中的表数据路径。 -- `replica_name` (`String`)-在动物园管理员副本名称. 同一表的不同副本具有不同的名称。 -- `replica_path` (`String`)-在ZooKeeper中的副本数据的路径。 与连接相同 ‘zookeeper\_path/replicas/replica\_path’. -- `columns_version` (`Int32`)-表结构的版本号。 指示执行ALTER的次数。 如果副本有不同的版本,这意味着一些副本还没有做出所有的改变。 -- `queue_size` (`UInt32`)-等待执行的操作的队列大小。 操作包括插入数据块、合并和某些其他操作。 它通常与 `future_parts`. -- `inserts_in_queue` (`UInt32`)-需要插入数据块的数量。 插入通常复制得相当快。 如果这个数字很大,这意味着有什么不对劲。 -- `merges_in_queue` (`UInt32`)-等待进行合并的数量。 有时合并时间很长,因此此值可能长时间大于零。 -- `part_mutations_in_queue` (`UInt32`)-等待进行的突变的数量。 -- `queue_oldest_time` (`DateTime`)-如果 `queue_size` 大于0,显示何时将最旧的操作添加到队列中。 -- `inserts_oldest_time` (`DateTime`)-看 `queue_oldest_time` -- `merges_oldest_time` (`DateTime`)-看 `queue_oldest_time` -- `part_mutations_oldest_time` (`DateTime`)-看 `queue_oldest_time` - -接下来的4列只有在有ZK活动会话的情况下才具有非零值。 - -- `log_max_index` (`UInt64`)-一般活动日志中的最大条目数。 -- `log_pointer` (`UInt64`)-副本复制到其执行队列的常规活动日志中的最大条目数加一。 如果 `log_pointer` 比 `log_max_index`,有点不对劲。 -- `last_queue_update` (`DateTime`)-上次更新队列时。 -- `absolute_delay` (`UInt64`)-当前副本有多大滞后秒。 -- `total_replicas` (`UInt8`)-此表的已知副本总数。 -- `active_replicas` (`UInt8`)-在ZooKeeper中具有会话的此表的副本的数量(即正常运行的副本的数量)。 - -如果您请求所有列,表可能会工作得有点慢,因为每行都会从ZooKeeper进行几次读取。 -如果您没有请求最后4列(log\_max\_index,log\_pointer,total\_replicas,active\_replicas),表工作得很快。 - -例如,您可以检查一切是否正常工作,如下所示: - -``` sql -SELECT - database, - table, - is_leader, - is_readonly, - is_session_expired, - future_parts, - parts_to_check, - columns_version, - queue_size, - inserts_in_queue, - merges_in_queue, - log_max_index, - log_pointer, - total_replicas, - active_replicas -FROM system.replicas -WHERE - is_readonly - OR is_session_expired - OR future_parts > 20 - OR parts_to_check > 10 - OR queue_size > 20 - OR inserts_in_queue > 10 - OR log_max_index - log_pointer > 10 - OR total_replicas < 2 - OR active_replicas < total_replicas -``` - -如果这个查询没有返回任何东西,这意味着一切都很好。 - -## 系统。设置 {#system-tables-system-settings} - -包含有关当前用户的会话设置的信息。 - -列: - -- `name` ([字符串](../sql-reference/data-types/string.md)) — Setting name. -- `value` ([字符串](../sql-reference/data-types/string.md)) — Setting value. -- `changed` ([UInt8](../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether a setting is changed from its default value. -- `description` ([字符串](../sql-reference/data-types/string.md)) — Short setting description. -- `min` ([可为空](../sql-reference/data-types/nullable.md)([字符串](../sql-reference/data-types/string.md))) — Minimum value of the setting, if any is set via [制约因素](settings/constraints-on-settings.md#constraints-on-settings). 如果设置没有最小值,则包含 [NULL](../sql-reference/syntax.md#null-literal). -- `max` ([可为空](../sql-reference/data-types/nullable.md)([字符串](../sql-reference/data-types/string.md))) — Maximum value of the setting, if any is set via [制约因素](settings/constraints-on-settings.md#constraints-on-settings). 如果设置没有最大值,则包含 [NULL](../sql-reference/syntax.md#null-literal). -- `readonly` ([UInt8](../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether the current user can change the setting: - - `0` — Current user can change the setting. - - `1` — Current user can't change the setting. - -**示例** - -下面的示例演示如何获取有关名称包含的设置的信息 `min_i`. - -``` sql -SELECT * -FROM system.settings -WHERE name LIKE '%min_i%' -``` - -``` text -┌─name────────────────────────────────────────┬─value─────┬─changed─┬─description───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─min──┬─max──┬─readonly─┐ -│ min_insert_block_size_rows │ 1048576 │ 0 │ Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough. │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ -│ min_insert_block_size_bytes │ 268435456 │ 0 │ Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough. │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ -│ read_backoff_min_interval_between_events_ms │ 1000 │ 0 │ Settings to reduce the number of threads in case of slow reads. Do not pay attention to the event, if the previous one has passed less than a certain amount of time. │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ -└─────────────────────────────────────────────┴───────────┴─────────┴───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴──────┴──────┴──────────┘ -``` - -使用 `WHERE changed` 可以是有用的,例如,当你想检查: - -- 配置文件中的设置是否正确加载并正在使用。 -- 在当前会话中更改的设置。 - - - -``` sql -SELECT * FROM system.settings WHERE changed AND name='load_balancing' -``` - -**另请参阅** - -- [设置](settings/index.md#session-settings-intro) -- [查询权限](settings/permissions-for-queries.md#settings_readonly) -- [对设置的限制](settings/constraints-on-settings.md) - -## 系统。表\_engines {#system.table_engines} - -``` text -┌─name───────────────────┬─value───────┐ -│ max_threads │ 8 │ -│ use_uncompressed_cache │ 0 │ -│ load_balancing │ random │ -│ max_memory_usage │ 10000000000 │ -└────────────────────────┴─────────────┘ -``` - -## 系统。merge\_tree\_settings {#system-merge_tree_settings} - -包含有关以下设置的信息 `MergeTree` 桌子 - -列: - -- `name` (String) — Setting name. -- `value` (String) — Setting value. -- `description` (String) — Setting description. -- `type` (String) — Setting type (implementation specific string value). -- `changed` (UInt8) — Whether the setting was explicitly defined in the config or explicitly changed. - -## 系统。表\_engines {#system-table-engines} - -包含服务器支持的表引擎的描述及其功能支持信息。 - -此表包含以下列(列类型显示在括号中): - -- `name` (String) — The name of table engine. -- `supports_settings` (UInt8) — Flag that indicates if table engine supports `SETTINGS` 条款 -- `supports_skipping_indices` (UInt8) — Flag that indicates if table engine supports [跳过索引](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-data_skipping-indexes). -- `supports_ttl` (UInt8) — Flag that indicates if table engine supports [TTL](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). -- `supports_sort_order` (UInt8) — Flag that indicates if table engine supports clauses `PARTITION_BY`, `PRIMARY_KEY`, `ORDER_BY` 和 `SAMPLE_BY`. -- `supports_replication` (UInt8) — Flag that indicates if table engine supports [数据复制](../engines/table-engines/mergetree-family/replication.md). -- `supports_duduplication` (UInt8) — Flag that indicates if table engine supports data deduplication. - -示例: - -``` sql -SELECT * -FROM system.table_engines -WHERE name in ('Kafka', 'MergeTree', 'ReplicatedCollapsingMergeTree') -``` - -``` text -┌─name──────────────────────────┬─supports_settings─┬─supports_skipping_indices─┬─supports_sort_order─┬─supports_ttl─┬─supports_replication─┬─supports_deduplication─┐ -│ Kafka │ 1 │ 0 │ 0 │ 0 │ 0 │ 0 │ -│ MergeTree │ 1 │ 1 │ 1 │ 1 │ 0 │ 0 │ -│ ReplicatedCollapsingMergeTree │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ -└───────────────────────────────┴───────────────────┴───────────────────────────┴─────────────────────┴──────────────┴──────────────────────┴────────────────────────┘ -``` - -**另请参阅** - -- 梅树家族 [查询子句](../engines/table-engines/mergetree-family/mergetree.md#mergetree-query-clauses) -- 卡夫卡 [设置](../engines/table-engines/integrations/kafka.md#table_engine-kafka-creating-a-table) -- 加入我们 [设置](../engines/table-engines/special/join.md#join-limitations-and-settings) - -## 系统。表 {#system-tables} - -包含服务器知道的每个表的元数据。 分离的表不显示在 `system.tables`. - -此表包含以下列(列类型显示在括号中): - -- `database` (String) — The name of the database the table is in. - -- `name` (String) — Table name. - -- `engine` (String) — Table engine name (without parameters). - -- `is_temporary` (UInt8)-指示表是否是临时的标志。 - -- `data_path` (String)-文件系统中表数据的路径。 - -- `metadata_path` (String)-文件系统中表元数据的路径。 - -- `metadata_modification_time` (DateTime)-表元数据的最新修改时间。 - -- `dependencies_database` (数组(字符串))-数据库依赖关系. - -- `dependencies_table` (数组(字符串))-表依赖关系 ([MaterializedView](../engines/table-engines/special/materializedview.md) 基于当前表的表)。 - -- `create_table_query` (String)-用于创建表的查询。 - -- `engine_full` (String)-表引擎的参数。 - -- `partition_key` (String)-表中指定的分区键表达式。 - -- `sorting_key` (String)-表中指定的排序键表达式。 - -- `primary_key` (String)-表中指定的主键表达式。 - -- `sampling_key` (String)-表中指定的采样键表达式。 - -- `storage_policy` (字符串)-存储策略: - - - [MergeTree](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) - - [分布](../engines/table-engines/special/distributed.md#distributed) - -- `total_rows` (Nullable(UInt64))-总行数,如果可以快速确定表中的确切行数,否则 `Null` (包括内衣 `Buffer` 表)。 - -- `total_bytes` (Nullable(UInt64))-总字节数,如果可以快速确定存储表的确切字节数,否则 `Null` (**不** 包括任何底层存储)。 - - - If the table stores data on disk, returns used space on disk (i.e. compressed). - - 如果表在内存中存储数据,返回在内存中使用的近似字节数. - -该 `system.tables` 表中使用 `SHOW TABLES` 查询实现。 - -## 系统。动物园管理员 {#system-zookeeper} - -如果未配置ZooKeeper,则表不存在。 允许从配置中定义的ZooKeeper集群读取数据。 -查询必须具有 ‘path’ WHERE子句中的平等条件。 这是ZooKeeper中您想要获取数据的孩子的路径。 - -查询 `SELECT * FROM system.zookeeper WHERE path = '/clickhouse'` 输出对所有孩子的数据 `/clickhouse` 节点。 -要输出所有根节点的数据,write path= ‘/’. -如果在指定的路径 ‘path’ 不存在,将引发异常。 - -列: - -- `name` (String) — The name of the node. -- `path` (String) — The path to the node. -- `value` (String) — Node value. -- `dataLength` (Int32) — Size of the value. -- `numChildren` (Int32) — Number of descendants. -- `czxid` (Int64) — ID of the transaction that created the node. -- `mzxid` (Int64) — ID of the transaction that last changed the node. -- `pzxid` (Int64) — ID of the transaction that last deleted or added descendants. -- `ctime` (DateTime) — Time of node creation. -- `mtime` (DateTime) — Time of the last modification of the node. -- `version` (Int32) — Node version: the number of times the node was changed. -- `cversion` (Int32) — Number of added or removed descendants. -- `aversion` (Int32) — Number of changes to the ACL. -- `ephemeralOwner` (Int64) — For ephemeral nodes, the ID of the session that owns this node. - -示例: - -``` sql -SELECT * -FROM system.zookeeper -WHERE path = '/clickhouse/tables/01-08/visits/replicas' -FORMAT Vertical -``` - -``` text -Row 1: -────── -name: example01-08-1.yandex.ru -value: -czxid: 932998691229 -mzxid: 932998691229 -ctime: 2015-03-27 16:49:51 -mtime: 2015-03-27 16:49:51 -version: 0 -cversion: 47 -aversion: 0 -ephemeralOwner: 0 -dataLength: 0 -numChildren: 7 -pzxid: 987021031383 -path: /clickhouse/tables/01-08/visits/replicas - -Row 2: -────── -name: example01-08-2.yandex.ru -value: -czxid: 933002738135 -mzxid: 933002738135 -ctime: 2015-03-27 16:57:01 -mtime: 2015-03-27 16:57:01 -version: 0 -cversion: 37 -aversion: 0 -ephemeralOwner: 0 -dataLength: 0 -numChildren: 7 -pzxid: 987021252247 -path: /clickhouse/tables/01-08/visits/replicas -``` - -## 系统。突变 {#system_tables-mutations} - -该表包含以下信息 [突变](../sql-reference/statements/alter.md#alter-mutations) MergeTree表及其进展。 每个突变命令由一行表示。 该表具有以下列: - -**数据库**, **表** -应用突变的数据库和表的名称。 - -**mutation\_id** -变异的ID 对于复制的表,这些Id对应于znode中的名称 `/mutations/` 动物园管理员的目录。 对于未复制的表,Id对应于表的数据目录中的文件名。 - -**命令** -Mutation命令字符串(查询后的部分 `ALTER TABLE [db.]table`). - -**create\_time** -当这个突变命令被提交执行。 - -**block\_numbers.partition\_id**, **block\_numbers.编号** -嵌套列。 对于复制表的突变,它包含每个分区的一条记录:分区ID和通过突变获取的块编号(在每个分区中,只有包含编号小于该分区中突变获取的块编号的块的 在非复制表中,所有分区中的块编号形成一个序列。 这意味着对于非复制表的突变,该列将包含一条记录,其中包含由突变获取的单个块编号。 - -**parts\_to\_do** -为了完成突变,需要突变的数据部分的数量。 - -**is\_done** -变异完成了?? 请注意,即使 `parts_to_do = 0` 由于长时间运行的INSERT将创建需要突变的新数据部分,因此可能尚未完成复制表的突变。 - -如果在改变某些部分时出现问题,以下列将包含其他信息: - -**latest\_failed\_part** -不能变异的最新部分的名称。 - -**latest\_fail\_time** -最近的部分突变失败的时间。 - -**latest\_fail\_reason** -导致最近部件变异失败的异常消息。 - -## 系统。磁盘 {#system_tables-disks} - -包含有关在定义的磁盘信息 [服务器配置](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). - -列: - -- `name` ([字符串](../sql-reference/data-types/string.md)) — Name of a disk in the server configuration. -- `path` ([字符串](../sql-reference/data-types/string.md)) — Path to the mount point in the file system. -- `free_space` ([UInt64](../sql-reference/data-types/int-uint.md)) — Free space on disk in bytes. -- `total_space` ([UInt64](../sql-reference/data-types/int-uint.md)) — Disk volume in bytes. -- `keep_free_space` ([UInt64](../sql-reference/data-types/int-uint.md)) — Amount of disk space that should stay free on disk in bytes. Defined in the `keep_free_space_bytes` 磁盘配置参数。 - -## 系统。storage\_policies {#system_tables-storage_policies} - -包含有关存储策略和卷中定义的信息 [服务器配置](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). - -列: - -- `policy_name` ([字符串](../sql-reference/data-types/string.md)) — Name of the storage policy. -- `volume_name` ([字符串](../sql-reference/data-types/string.md)) — Volume name defined in the storage policy. -- `volume_priority` ([UInt64](../sql-reference/data-types/int-uint.md)) — Volume order number in the configuration. -- `disks` ([数组(字符串)](../sql-reference/data-types/array.md)) — Disk names, defined in the storage policy. -- `max_data_part_size` ([UInt64](../sql-reference/data-types/int-uint.md)) — Maximum size of a data part that can be stored on volume disks (0 — no limit). -- `move_factor` ([Float64](../sql-reference/data-types/float.md)) — Ratio of free disk space. When the ratio exceeds the value of configuration parameter, ClickHouse start to move data to the next volume in order. - -如果存储策略包含多个卷,则每个卷的信息将存储在表的单独行中。 - -[原始文章](https://clickhouse.tech/docs/en/operations/system_tables/) diff --git a/docs/zh/operations/system-tables/asynchronous_metric_log.md b/docs/zh/operations/system-tables/asynchronous_metric_log.md new file mode 100644 index 00000000000..9f6c697a18e --- /dev/null +++ b/docs/zh/operations/system-tables/asynchronous_metric_log.md @@ -0,0 +1,8 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +## 系统。asynchronous\_metric\_log {#system-tables-async-log} + +包含以下内容的历史值 `system.asynchronous_log` (见 [系统。asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics)) diff --git a/docs/zh/operations/system-tables/asynchronous_metrics.md b/docs/zh/operations/system-tables/asynchronous_metrics.md new file mode 100644 index 00000000000..2bd615085a8 --- /dev/null +++ b/docs/zh/operations/system-tables/asynchronous_metrics.md @@ -0,0 +1,41 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。asynchronous\_metrics {#system_tables-asynchronous_metrics} + +包含在后台定期计算的指标。 例如,在使用的RAM量。 + +列: + +- `metric` ([字符串](../../sql-reference/data-types/string.md)) — Metric name. +- `value` ([Float64](../../sql-reference/data-types/float.md)) — Metric value. + +**示例** + +``` sql +SELECT * FROM system.asynchronous_metrics LIMIT 10 +``` + +``` text +┌─metric──────────────────────────────────┬──────value─┐ +│ jemalloc.background_thread.run_interval │ 0 │ +│ jemalloc.background_thread.num_runs │ 0 │ +│ jemalloc.background_thread.num_threads │ 0 │ +│ jemalloc.retained │ 422551552 │ +│ jemalloc.mapped │ 1682989056 │ +│ jemalloc.resident │ 1656446976 │ +│ jemalloc.metadata_thp │ 0 │ +│ jemalloc.metadata │ 10226856 │ +│ UncompressedCacheCells │ 0 │ +│ MarkCacheFiles │ 0 │ +└─────────────────────────────────────────┴────────────┘ +``` + +**另请参阅** + +- [监测](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. +- [系统。指标](../../operations/system-tables/metrics.md#system_tables-metrics) — Contains instantly calculated metrics. +- [系统。活动](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that have occurred. +- [系统。metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. diff --git a/docs/zh/operations/system-tables/clusters.md b/docs/zh/operations/system-tables/clusters.md new file mode 100644 index 00000000000..4bc8d4210ff --- /dev/null +++ b/docs/zh/operations/system-tables/clusters.md @@ -0,0 +1,29 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。集群 {#system-clusters} + +包含有关配置文件中可用的集群及其中的服务器的信息。 + +列: + +- `cluster` (String) — The cluster name. +- `shard_num` (UInt32) — The shard number in the cluster, starting from 1. +- `shard_weight` (UInt32) — The relative weight of the shard when writing data. +- `replica_num` (UInt32) — The replica number in the shard, starting from 1. +- `host_name` (String) — The host name, as specified in the config. +- `host_address` (String) — The host IP address obtained from DNS. +- `port` (UInt16) — The port to use for connecting to the server. +- `user` (String) — The name of the user for connecting to the server. +- `errors_count` (UInt32)-此主机无法到达副本的次数。 +- `estimated_recovery_time` (UInt32)-剩下的秒数,直到副本错误计数归零,它被认为是恢复正常。 + +请注意 `errors_count` 每个查询集群更新一次,但 `estimated_recovery_time` 按需重新计算。 所以有可能是非零的情况 `errors_count` 和零 `estimated_recovery_time`,下一个查询将为零 `errors_count` 并尝试使用副本,就好像它没有错误。 + +**另请参阅** + +- [表引擎分布式](../../engines/table-engines/special/distributed.md) +- [distributed\_replica\_error\_cap设置](../../operations/settings/settings.md#settings-distributed_replica_error_cap) +- [distributed\_replica\_error\_half\_life设置](../../operations/settings/settings.md#settings-distributed_replica_error_half_life) diff --git a/docs/zh/operations/system-tables/columns.md b/docs/zh/operations/system-tables/columns.md new file mode 100644 index 00000000000..24296dc715c --- /dev/null +++ b/docs/zh/operations/system-tables/columns.md @@ -0,0 +1,27 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。列 {#system-columns} + +包含有关所有表中列的信息。 + +您可以使用此表获取类似于以下内容的信息 [DESCRIBE TABLE](../../sql-reference/statements/misc.md#misc-describe-table) 查询,但对于多个表一次。 + +该 `system.columns` 表包含以下列(列类型显示在括号中): + +- `database` (String) — Database name. +- `table` (String) — Table name. +- `name` (String) — Column name. +- `type` (String) — Column type. +- `default_kind` (String) — Expression type (`DEFAULT`, `MATERIALIZED`, `ALIAS`)为默认值,如果没有定义,则为空字符串。 +- `default_expression` (String) — Expression for the default value, or an empty string if it is not defined. +- `data_compressed_bytes` (UInt64) — The size of compressed data, in bytes. +- `data_uncompressed_bytes` (UInt64) — The size of decompressed data, in bytes. +- `marks_bytes` (UInt64) — The size of marks, in bytes. +- `comment` (String) — Comment on the column, or an empty string if it is not defined. +- `is_in_partition_key` (UInt8) — Flag that indicates whether the column is in the partition expression. +- `is_in_sorting_key` (UInt8) — Flag that indicates whether the column is in the sorting key expression. +- `is_in_primary_key` (UInt8) — Flag that indicates whether the column is in the primary key expression. +- `is_in_sampling_key` (UInt8) — Flag that indicates whether the column is in the sampling key expression. diff --git a/docs/zh/operations/system-tables/contributors.md b/docs/zh/operations/system-tables/contributors.md new file mode 100644 index 00000000000..e9374a7dc9c --- /dev/null +++ b/docs/zh/operations/system-tables/contributors.md @@ -0,0 +1,45 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。贡献者 {#system-contributors} + +包含有关贡献者的信息。 该顺序在查询执行时是随机的。 + +列: + +- `name` (String) — Contributor (author) name from git log. + +**示例** + +``` sql +SELECT * FROM system.contributors LIMIT 10 +``` + +``` text +┌─name─────────────┐ +│ Olga Khvostikova │ +│ Max Vetrov │ +│ LiuYangkuan │ +│ svladykin │ +│ zamulla │ +│ Šimon Podlipský │ +│ BayoNet │ +│ Ilya Khomutov │ +│ Amy Krishnevsky │ +│ Loud_Scream │ +└──────────────────┘ +``` + +要在表中找出自己,请使用查询: + +``` sql +SELECT * FROM system.contributors WHERE name = 'Olga Khvostikova' +``` + +``` text +┌─name─────────────┐ +│ Olga Khvostikova │ +└──────────────────┘ +``` diff --git a/docs/zh/operations/system-tables/data_type_families.md b/docs/zh/operations/system-tables/data_type_families.md new file mode 100644 index 00000000000..e6ec3fdbfce --- /dev/null +++ b/docs/zh/operations/system-tables/data_type_families.md @@ -0,0 +1,39 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。data\_type\_families {#system_tables-data_type_families} + +包含有关受支持的信息 [数据类型](../../sql-reference/data-types/). + +列: + +- `name` ([字符串](../../sql-reference/data-types/string.md)) — Data type name. +- `case_insensitive` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Property that shows whether you can use a data type name in a query in case insensitive manner or not. For example, `Date` 和 `date` 都是有效的。 +- `alias_to` ([字符串](../../sql-reference/data-types/string.md)) — Data type name for which `name` 是个化名 + +**示例** + +``` sql +SELECT * FROM system.data_type_families WHERE alias_to = 'String' +``` + +``` text +┌─name───────┬─case_insensitive─┬─alias_to─┐ +│ LONGBLOB │ 1 │ String │ +│ LONGTEXT │ 1 │ String │ +│ TINYTEXT │ 1 │ String │ +│ TEXT │ 1 │ String │ +│ VARCHAR │ 1 │ String │ +│ MEDIUMBLOB │ 1 │ String │ +│ BLOB │ 1 │ String │ +│ TINYBLOB │ 1 │ String │ +│ CHAR │ 1 │ String │ +│ MEDIUMTEXT │ 1 │ String │ +└────────────┴──────────────────┴──────────┘ +``` + +**另请参阅** + +- [语法](../../sql-reference/syntax.md) — Information about supported syntax. diff --git a/docs/zh/operations/system-tables/databases.md b/docs/zh/operations/system-tables/databases.md new file mode 100644 index 00000000000..134b8ebc7ab --- /dev/null +++ b/docs/zh/operations/system-tables/databases.md @@ -0,0 +1,12 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。数据库 {#system-databases} + +此表包含一个名为"字符串"的列 ‘name’ – the name of a database. + +服务器知道的每个数据库在表中都有相应的条目。 + +该系统表用于实现 `SHOW DATABASES` 查询。 diff --git a/docs/zh/operations/system-tables/detached_parts.md b/docs/zh/operations/system-tables/detached_parts.md new file mode 100644 index 00000000000..dd561dec6f3 --- /dev/null +++ b/docs/zh/operations/system-tables/detached_parts.md @@ -0,0 +1,14 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。detached\_parts {#system_tables-detached_parts} + +包含有关分离部分的信息 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 桌子 该 `reason` 列指定分离部件的原因。 + +对于用户分离的部件,原因是空的。 这些部件可以附加 [ALTER TABLE ATTACH PARTITION\|PART](../../sql-reference/statements/alter.md#alter_attach-partition) 指挥部 + +有关其他列的说明,请参阅 [系统。零件](../../operations/system-tables/parts.md#system_tables-parts). + +如果部件名称无效,某些列的值可能为 `NULL`. 这些部分可以删除 [ALTER TABLE DROP DETACHED PART](../../sql-reference/statements/alter.md#alter_drop-detached). diff --git a/docs/zh/operations/system-tables/dictionaries.md b/docs/zh/operations/system-tables/dictionaries.md new file mode 100644 index 00000000000..6cfe71de3cb --- /dev/null +++ b/docs/zh/operations/system-tables/dictionaries.md @@ -0,0 +1,66 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。字典 {#system_tables-dictionaries} + +包含以下信息 [外部字典](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). + +列: + +- `database` ([字符串](../../sql-reference/data-types/string.md)) — Name of the database containing the dictionary created by DDL query. Empty string for other dictionaries. +- `name` ([字符串](../../sql-reference/data-types/string.md)) — [字典名称](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md). +- `status` ([枚举8](../../sql-reference/data-types/enum.md)) — Dictionary status. Possible values: + - `NOT_LOADED` — Dictionary was not loaded because it was not used. + - `LOADED` — Dictionary loaded successfully. + - `FAILED` — Unable to load the dictionary as a result of an error. + - `LOADING` — Dictionary is loading now. + - `LOADED_AND_RELOADING` — Dictionary is loaded successfully, and is being reloaded right now (frequent reasons: [SYSTEM RELOAD DICTIONARY](../../sql-reference/statements/system.md#query_language-system-reload-dictionary) 查询,超时,字典配置已更改)。 + - `FAILED_AND_RELOADING` — Could not load the dictionary as a result of an error and is loading now. +- `origin` ([字符串](../../sql-reference/data-types/string.md)) — Path to the configuration file that describes the dictionary. +- `type` ([字符串](../../sql-reference/data-types/string.md)) — Type of a dictionary allocation. [在内存中存储字典](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md). +- `key` — [密钥类型](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-key):数字键 ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) or Сomposite key ([字符串](../../sql-reference/data-types/string.md)) — form “(type 1, type 2, …, type n)”. +- `attribute.names` ([阵列](../../sql-reference/data-types/array.md)([字符串](../../sql-reference/data-types/string.md))) — Array of [属性名称](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes) 由字典提供。 +- `attribute.types` ([阵列](../../sql-reference/data-types/array.md)([字符串](../../sql-reference/data-types/string.md))) — Corresponding array of [属性类型](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes) 这是由字典提供。 +- `bytes_allocated` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Amount of RAM allocated for the dictionary. +- `query_count` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of queries since the dictionary was loaded or since the last successful reboot. +- `hit_rate` ([Float64](../../sql-reference/data-types/float.md)) — For cache dictionaries, the percentage of uses for which the value was in the cache. +- `element_count` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of items stored in the dictionary. +- `load_factor` ([Float64](../../sql-reference/data-types/float.md)) — Percentage filled in the dictionary (for a hashed dictionary, the percentage filled in the hash table). +- `source` ([字符串](../../sql-reference/data-types/string.md)) — Text describing the [数据源](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md) 为了字典 +- `lifetime_min` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Minimum [使用寿命](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) 在内存中的字典,之后ClickHouse尝试重新加载字典(如果 `invalidate_query` 被设置,那么只有当它已经改变)。 在几秒钟内设置。 +- `lifetime_max` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Maximum [使用寿命](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) 在内存中的字典,之后ClickHouse尝试重新加载字典(如果 `invalidate_query` 被设置,那么只有当它已经改变)。 在几秒钟内设置。 +- `loading_start_time` ([日期时间](../../sql-reference/data-types/datetime.md)) — Start time for loading the dictionary. +- `last_successful_update_time` ([日期时间](../../sql-reference/data-types/datetime.md)) — End time for loading or updating the dictionary. Helps to monitor some troubles with external sources and investigate causes. +- `loading_duration` ([Float32](../../sql-reference/data-types/float.md)) — Duration of a dictionary loading. +- `last_exception` ([字符串](../../sql-reference/data-types/string.md)) — Text of the error that occurs when creating or reloading the dictionary if the dictionary couldn't be created. + +**示例** + +配置字典。 + +``` sql +CREATE DICTIONARY dictdb.dict +( + `key` Int64 DEFAULT -1, + `value_default` String DEFAULT 'world', + `value_expression` String DEFAULT 'xxx' EXPRESSION 'toString(127 * 172)' +) +PRIMARY KEY key +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dicttbl' DB 'dictdb')) +LIFETIME(MIN 0 MAX 1) +LAYOUT(FLAT()) +``` + +确保字典已加载。 + +``` sql +SELECT * FROM system.dictionaries +``` + +``` text +┌─database─┬─name─┬─status─┬─origin──────┬─type─┬─key────┬─attribute.names──────────────────────┬─attribute.types─────┬─bytes_allocated─┬─query_count─┬─hit_rate─┬─element_count─┬───────────load_factor─┬─source─────────────────────┬─lifetime_min─┬─lifetime_max─┬──loading_start_time─┌──last_successful_update_time─┬──────loading_duration─┬─last_exception─┐ +│ dictdb │ dict │ LOADED │ dictdb.dict │ Flat │ UInt64 │ ['value_default','value_expression'] │ ['String','String'] │ 74032 │ 0 │ 1 │ 1 │ 0.0004887585532746823 │ ClickHouse: dictdb.dicttbl │ 0 │ 1 │ 2020-03-04 04:17:34 │ 2020-03-04 04:30:34 │ 0.002 │ │ +└──────────┴──────┴────────┴─────────────┴──────┴────────┴──────────────────────────────────────┴─────────────────────┴─────────────────┴─────────────┴──────────┴───────────────┴───────────────────────┴────────────────────────────┴──────────────┴──────────────┴─────────────────────┴──────────────────────────────┘───────────────────────┴────────────────┘ +``` diff --git a/docs/zh/operations/system-tables/disks.md b/docs/zh/operations/system-tables/disks.md new file mode 100644 index 00000000000..39cacccb4db --- /dev/null +++ b/docs/zh/operations/system-tables/disks.md @@ -0,0 +1,31 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。磁盘 {#system_tables-disks} + +包含有关在定义的磁盘信息 [服务器配置](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). + +列: + +- `name` ([字符串](../../sql-reference/data-types/string.md)) — Name of a disk in the server configuration. +- `path` ([字符串](../../sql-reference/data-types/string.md)) — Path to the mount point in the file system. +- `free_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Free space on disk in bytes. +- `total_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Disk volume in bytes. +- `keep_free_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Amount of disk space that should stay free on disk in bytes. Defined in the `keep_free_space_bytes` 磁盘配置参数。 + +## 系统。storage\_policies {#system_tables-storage_policies} + +包含有关存储策略和卷中定义的信息 [服务器配置](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). + +列: + +- `policy_name` ([字符串](../../sql-reference/data-types/string.md)) — Name of the storage policy. +- `volume_name` ([字符串](../../sql-reference/data-types/string.md)) — Volume name defined in the storage policy. +- `volume_priority` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Volume order number in the configuration. +- `disks` ([数组(字符串)](../../sql-reference/data-types/array.md)) — Disk names, defined in the storage policy. +- `max_data_part_size` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Maximum size of a data part that can be stored on volume disks (0 — no limit). +- `move_factor` ([Float64](../../sql-reference/data-types/float.md)) — Ratio of free disk space. When the ratio exceeds the value of configuration parameter, ClickHouse start to move data to the next volume in order. + +如果存储策略包含多个卷,则每个卷的信息将存储在表的单独行中。 diff --git a/docs/zh/operations/system-tables/events.md b/docs/zh/operations/system-tables/events.md new file mode 100644 index 00000000000..21b787c6064 --- /dev/null +++ b/docs/zh/operations/system-tables/events.md @@ -0,0 +1,37 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。活动 {#system_tables-events} + +包含有关系统中发生的事件数的信息。 例如,在表中,您可以找到多少 `SELECT` 自ClickHouse服务器启动以来已处理查询。 + +列: + +- `event` ([字符串](../../sql-reference/data-types/string.md)) — Event name. +- `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of events occurred. +- `description` ([字符串](../../sql-reference/data-types/string.md)) — Event description. + +**示例** + +``` sql +SELECT * FROM system.events LIMIT 5 +``` + +``` text +┌─event─────────────────────────────────┬─value─┬─description────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ Query │ 12 │ Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries. │ +│ SelectQuery │ 8 │ Same as Query, but only for SELECT queries. │ +│ FileOpen │ 73 │ Number of files opened. │ +│ ReadBufferFromFileDescriptorRead │ 155 │ Number of reads (read/pread) from a file descriptor. Does not include sockets. │ +│ ReadBufferFromFileDescriptorReadBytes │ 9931 │ Number of bytes read from file descriptors. If the file is compressed, this will show the compressed data size. │ +└───────────────────────────────────────┴───────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +**另请参阅** + +- [系统。asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [系统。指标](../../operations/system-tables/metrics.md#system_tables-metrics) — Contains instantly calculated metrics. +- [系统。metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [监测](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. diff --git a/docs/zh/operations/system-tables/functions.md b/docs/zh/operations/system-tables/functions.md new file mode 100644 index 00000000000..ff716b0bc6c --- /dev/null +++ b/docs/zh/operations/system-tables/functions.md @@ -0,0 +1,13 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。功能 {#system-functions} + +包含有关正常函数和聚合函数的信息。 + +列: + +- `name`(`String`) – The name of the function. +- `is_aggregate`(`UInt8`) — Whether the function is aggregate. diff --git a/docs/zh/operations/system-tables/graphite_retentions.md b/docs/zh/operations/system-tables/graphite_retentions.md new file mode 100644 index 00000000000..b2579541920 --- /dev/null +++ b/docs/zh/operations/system-tables/graphite_retentions.md @@ -0,0 +1,20 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。graphite\_retentions {#system-graphite-retentions} + +包含有关参数的信息 [graphite\_rollup](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) 这是在表中使用 [\*GraphiteMergeTree](../../engines/table-engines/mergetree-family/graphitemergetree.md) 引擎 + +列: + +- `config_name` (字符串) - `graphite_rollup` 参数名称。 +- `regexp` (String)-指标名称的模式。 +- `function` (String)-聚合函数的名称。 +- `age` (UInt64)-以秒为单位的数据的最小期限。 +- `precision` (UInt64)-如何精确地定义以秒为单位的数据的年龄。 +- `priority` (UInt16)-模式优先级。 +- `is_default` (UInt8)-模式是否为默认值。 +- `Tables.database` (Array(String))-使用数据库表名称的数组 `config_name` 参数。 +- `Tables.table` (Array(String))-使用表名称的数组 `config_name` 参数。 diff --git a/docs/zh/operations/system-tables/index.md b/docs/zh/operations/system-tables/index.md new file mode 100644 index 00000000000..73a57300de8 --- /dev/null +++ b/docs/zh/operations/system-tables/index.md @@ -0,0 +1,50 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_priority: 52 +toc_title: "\u7CFB\u7EDF\u8868" +--- + +# 系统表 {#system-tables} + +## 导言 {#system-tables-introduction} + +系统表提供以下信息: + +- 服务器状态、进程和环境。 +- 服务器的内部进程。 + +系统表: + +- 坐落于 `system` 数据库。 +- 仅适用于读取数据。 +- 不能删除或更改,但可以分离。 + +大多数系统表将数据存储在RAM中。 ClickHouse服务器在开始时创建此类系统表。 + +与其他系统表不同,系统表 [metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log), [query\_log](../../operations/system-tables/query_log.md#system_tables-query_log), [query\_thread\_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log), [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) 由 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 表引擎并将其数据存储在存储文件系统中。 如果从文件系统中删除表,ClickHouse服务器会在下一次写入数据时再次创建空表。 如果系统表架构在新版本中发生更改,则ClickHouse会重命名当前表并创建一个新表。 + +默认情况下,表增长是无限的。 要控制表的大小,可以使用 [TTL](../../sql-reference/statements/alter.md#manipulations-with-table-ttl) 删除过期日志记录的设置。 你也可以使用分区功能 `MergeTree`-发动机表。 + +## 系统指标的来源 {#system-tables-sources-of-system-metrics} + +用于收集ClickHouse服务器使用的系统指标: + +- `CAP_NET_ADMIN` 能力。 +- [procfs](https://en.wikipedia.org/wiki/Procfs) (仅在Linux中)。 + +**procfs** + +如果ClickHouse服务器没有 `CAP_NET_ADMIN` 能力,它试图回落到 `ProcfsMetricsProvider`. `ProcfsMetricsProvider` 允许收集每个查询系统指标(用于CPU和I/O)。 + +如果系统上支持并启用procfs,ClickHouse server将收集这些指标: + +- `OSCPUVirtualTimeMicroseconds` +- `OSCPUWaitMicroseconds` +- `OSIOWaitMicroseconds` +- `OSReadChars` +- `OSWriteChars` +- `OSReadBytes` +- `OSWriteBytes` + +[原始文章](https://clickhouse.tech/docs/en/operations/system-tables/) diff --git a/docs/zh/operations/system-tables/merge_tree_settings.md b/docs/zh/operations/system-tables/merge_tree_settings.md new file mode 100644 index 00000000000..d2a5f64ba21 --- /dev/null +++ b/docs/zh/operations/system-tables/merge_tree_settings.md @@ -0,0 +1,16 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。merge\_tree\_settings {#system-merge_tree_settings} + +包含有关以下设置的信息 `MergeTree` 桌子 + +列: + +- `name` (String) — Setting name. +- `value` (String) — Setting value. +- `description` (String) — Setting description. +- `type` (String) — Setting type (implementation specific string value). +- `changed` (UInt8) — Whether the setting was explicitly defined in the config or explicitly changed. diff --git a/docs/zh/operations/system-tables/merges.md b/docs/zh/operations/system-tables/merges.md new file mode 100644 index 00000000000..f5cf2a56118 --- /dev/null +++ b/docs/zh/operations/system-tables/merges.md @@ -0,0 +1,24 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。合并 {#system-merges} + +包含有关MergeTree系列中表当前正在进行的合并和部件突变的信息。 + +列: + +- `database` (String) — The name of the database the table is in. +- `table` (String) — Table name. +- `elapsed` (Float64) — The time elapsed (in seconds) since the merge started. +- `progress` (Float64) — The percentage of completed work from 0 to 1. +- `num_parts` (UInt64) — The number of pieces to be merged. +- `result_part_name` (String) — The name of the part that will be formed as the result of merging. +- `is_mutation` (UInt8)-1如果这个过程是一个部分突变. +- `total_size_bytes_compressed` (UInt64) — The total size of the compressed data in the merged chunks. +- `total_size_marks` (UInt64) — The total number of marks in the merged parts. +- `bytes_read_uncompressed` (UInt64) — Number of bytes read, uncompressed. +- `rows_read` (UInt64) — Number of rows read. +- `bytes_written_uncompressed` (UInt64) — Number of bytes written, uncompressed. +- `rows_written` (UInt64) — Number of rows written. diff --git a/docs/zh/operations/system-tables/metric_log.md b/docs/zh/operations/system-tables/metric_log.md new file mode 100644 index 00000000000..46b28f8d2f8 --- /dev/null +++ b/docs/zh/operations/system-tables/metric_log.md @@ -0,0 +1,60 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。metric\_log {#system_tables-metric_log} + +包含表中度量值的历史记录 `system.metrics` 和 `system.events`,定期刷新到磁盘。 +打开指标历史记录收集 `system.metric_log`,创建 `/etc/clickhouse-server/config.d/metric_log.xml` 具有以下内容: + +``` xml + + + system + metric_log
+ 7500 + 1000 +
+
+``` + +**示例** + +``` sql +SELECT * FROM system.metric_log LIMIT 1 FORMAT Vertical; +``` + +``` text +Row 1: +────── +event_date: 2020-02-18 +event_time: 2020-02-18 07:15:33 +milliseconds: 554 +ProfileEvent_Query: 0 +ProfileEvent_SelectQuery: 0 +ProfileEvent_InsertQuery: 0 +ProfileEvent_FileOpen: 0 +ProfileEvent_Seek: 0 +ProfileEvent_ReadBufferFromFileDescriptorRead: 1 +ProfileEvent_ReadBufferFromFileDescriptorReadFailed: 0 +ProfileEvent_ReadBufferFromFileDescriptorReadBytes: 0 +ProfileEvent_WriteBufferFromFileDescriptorWrite: 1 +ProfileEvent_WriteBufferFromFileDescriptorWriteFailed: 0 +ProfileEvent_WriteBufferFromFileDescriptorWriteBytes: 56 +... +CurrentMetric_Query: 0 +CurrentMetric_Merge: 0 +CurrentMetric_PartMutation: 0 +CurrentMetric_ReplicatedFetch: 0 +CurrentMetric_ReplicatedSend: 0 +CurrentMetric_ReplicatedChecks: 0 +... +``` + +**另请参阅** + +- [系统。asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [系统。活动](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that occurred. +- [系统。指标](../../operations/system-tables/metrics.md#system_tables-metrics) — Contains instantly calculated metrics. +- [监测](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. diff --git a/docs/zh/operations/system-tables/metrics.md b/docs/zh/operations/system-tables/metrics.md new file mode 100644 index 00000000000..1bf74524785 --- /dev/null +++ b/docs/zh/operations/system-tables/metrics.md @@ -0,0 +1,44 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。指标 {#system_tables-metrics} + +包含可以立即计算或具有当前值的指标。 例如,同时处理的查询的数量或当前副本的延迟。 此表始终是最新的。 + +列: + +- `metric` ([字符串](../../sql-reference/data-types/string.md)) — Metric name. +- `value` ([Int64](../../sql-reference/data-types/int-uint.md)) — Metric value. +- `description` ([字符串](../../sql-reference/data-types/string.md)) — Metric description. + +支持的指标列表,您可以在 [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp) ClickHouse的源文件。 + +**示例** + +``` sql +SELECT * FROM system.metrics LIMIT 10 +``` + +``` text +┌─metric─────────────────────┬─value─┬─description──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ Query │ 1 │ Number of executing queries │ +│ Merge │ 0 │ Number of executing background merges │ +│ PartMutation │ 0 │ Number of mutations (ALTER DELETE/UPDATE) │ +│ ReplicatedFetch │ 0 │ Number of data parts being fetched from replicas │ +│ ReplicatedSend │ 0 │ Number of data parts being sent to replicas │ +│ ReplicatedChecks │ 0 │ Number of data parts checking for consistency │ +│ BackgroundPoolTask │ 0 │ Number of active tasks in BackgroundProcessingPool (merges, mutations, fetches, or replication queue bookkeeping) │ +│ BackgroundSchedulePoolTask │ 0 │ Number of active tasks in BackgroundSchedulePool. This pool is used for periodic ReplicatedMergeTree tasks, like cleaning old data parts, altering data parts, replica re-initialization, etc. │ +│ DiskSpaceReservedForMerge │ 0 │ Disk space reserved for currently running background merges. It is slightly more than the total size of currently merging parts. │ +│ DistributedSend │ 0 │ Number of connections to remote servers sending data that was INSERTed into Distributed tables. Both synchronous and asynchronous mode. │ +└────────────────────────────┴───────┴──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +**另请参阅** + +- [系统。asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [系统。活动](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that occurred. +- [系统。metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [监测](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. diff --git a/docs/zh/operations/system-tables/mutations.md b/docs/zh/operations/system-tables/mutations.md new file mode 100644 index 00000000000..8e2d66a42e8 --- /dev/null +++ b/docs/zh/operations/system-tables/mutations.md @@ -0,0 +1,30 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。突变 {#system_tables-mutations} + +该表包含以下信息 [突变](../../sql-reference/statements/alter.md#alter-mutations) MergeTree表及其进展。 每个突变命令由一行表示。 该表具有以下列: + +**数据库**, **表** -应用突变的数据库和表的名称。 + +**mutation\_id** -变异的ID 对于复制的表,这些Id对应于znode中的名称 `/mutations/` 动物园管理员的目录。 对于未复制的表,Id对应于表的数据目录中的文件名。 + +**命令** -Mutation命令字符串(查询后的部分 `ALTER TABLE [db.]table`). + +**create\_time** -当这个突变命令被提交执行。 + +**block\_numbers.partition\_id**, **block\_numbers.编号** -嵌套列。 对于复制表的突变,它包含每个分区的一条记录:分区ID和通过突变获取的块编号(在每个分区中,只有包含编号小于该分区中突变获取的块编号的块的 在非复制表中,所有分区中的块编号形成一个序列。 这意味着对于非复制表的突变,该列将包含一条记录,其中包含由突变获取的单个块编号。 + +**parts\_to\_do** -为了完成突变,需要突变的数据部分的数量。 + +**is\_done** -变异完成了?? 请注意,即使 `parts_to_do = 0` 由于长时间运行的INSERT将创建需要突变的新数据部分,因此可能尚未完成复制表的突变。 + +如果在改变某些部分时出现问题,以下列将包含其他信息: + +**latest\_failed\_part** -不能变异的最新部分的名称。 + +**latest\_fail\_time** -最近的部分突变失败的时间。 + +**latest\_fail\_reason** -导致最近部件变异失败的异常消息。 diff --git a/docs/zh/operations/system-tables/numbers.md b/docs/zh/operations/system-tables/numbers.md new file mode 100644 index 00000000000..c42c87053ca --- /dev/null +++ b/docs/zh/operations/system-tables/numbers.md @@ -0,0 +1,12 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。数字 {#system-numbers} + +此表包含一个名为UInt64的列 `number` 它包含几乎所有从零开始的自然数。 + +您可以使用此表进行测试,或者如果您需要进行暴力搜索。 + +从此表中读取的内容不是并行的。 diff --git a/docs/zh/operations/system-tables/numbers_mt.md b/docs/zh/operations/system-tables/numbers_mt.md new file mode 100644 index 00000000000..90addea157d --- /dev/null +++ b/docs/zh/operations/system-tables/numbers_mt.md @@ -0,0 +1,10 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。numbers\_mt {#system-numbers-mt} + +一样的 [系统。数字](../../operations/system-tables/numbers.md) 但读取是并行的。 这些数字可以以任何顺序返回。 + +用于测试。 diff --git a/docs/zh/operations/system-tables/one.md b/docs/zh/operations/system-tables/one.md new file mode 100644 index 00000000000..a8dc64c18c7 --- /dev/null +++ b/docs/zh/operations/system-tables/one.md @@ -0,0 +1,12 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。一 {#system-one} + +此表包含一行,其中包含一行 `dummy` UInt8列包含值0。 + +如果使用此表 `SELECT` 查询不指定 `FROM` 条款 + +这类似于 `DUAL` 表在其他Dbms中找到。 diff --git a/docs/zh/operations/system-tables/part_log.md b/docs/zh/operations/system-tables/part_log.md new file mode 100644 index 00000000000..b8388455948 --- /dev/null +++ b/docs/zh/operations/system-tables/part_log.md @@ -0,0 +1,37 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。part\_log {#system_tables-part-log} + +该 `system.part_log` 表只有当创建 [part\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-part-log) 指定了服务器设置。 + +此表包含与以下情况发生的事件有关的信息 [数据部分](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) 在 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 家庭表,例如添加或合并数据。 + +该 `system.part_log` 表包含以下列: + +- `event_type` (Enum) — Type of the event that occurred with the data part. Can have one of the following values: + - `NEW_PART` — Inserting of a new data part. + - `MERGE_PARTS` — Merging of data parts. + - `DOWNLOAD_PART` — Downloading a data part. + - `REMOVE_PART` — Removing or detaching a data part using [DETACH PARTITION](../../sql-reference/statements/alter.md#alter_detach-partition). + - `MUTATE_PART` — Mutating of a data part. + - `MOVE_PART` — Moving the data part from the one disk to another one. +- `event_date` (Date) — Event date. +- `event_time` (DateTime) — Event time. +- `duration_ms` (UInt64) — Duration. +- `database` (String) — Name of the database the data part is in. +- `table` (String) — Name of the table the data part is in. +- `part_name` (String) — Name of the data part. +- `partition_id` (String) — ID of the partition that the data part was inserted to. The column takes the ‘all’ 值,如果分区是由 `tuple()`. +- `rows` (UInt64) — The number of rows in the data part. +- `size_in_bytes` (UInt64) — Size of the data part in bytes. +- `merged_from` (Array(String)) — An array of names of the parts which the current part was made up from (after the merge). +- `bytes_uncompressed` (UInt64) — Size of uncompressed bytes. +- `read_rows` (UInt64) — The number of rows was read during the merge. +- `read_bytes` (UInt64) — The number of bytes was read during the merge. +- `error` (UInt16) — The code number of the occurred error. +- `exception` (String) — Text message of the occurred error. + +该 `system.part_log` 表的第一个插入数据到后创建 `MergeTree` 桌子 diff --git a/docs/zh/operations/system-tables/parts.md b/docs/zh/operations/system-tables/parts.md new file mode 100644 index 00000000000..e924ee27df3 --- /dev/null +++ b/docs/zh/operations/system-tables/parts.md @@ -0,0 +1,85 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。零件 {#system_tables-parts} + +包含有关的部分信息 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 桌子 + +每行描述一个数据部分。 + +列: + +- `partition` (String) – The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter.md#query_language_queries_alter) 查询。 + + 格式: + + - `YYYYMM` 用于按月自动分区。 + - `any_string` 手动分区时。 + +- `name` (`String`) – Name of the data part. + +- `active` (`UInt8`) – Flag that indicates whether the data part is active. If a data part is active, it's used in a table. Otherwise, it's deleted. Inactive data parts remain after merging. + +- `marks` (`UInt64`) – The number of marks. To get the approximate number of rows in a data part, multiply `marks` 通过索引粒度(通常为8192)(此提示不适用于自适应粒度)。 + +- `rows` (`UInt64`) – The number of rows. + +- `bytes_on_disk` (`UInt64`) – Total size of all the data part files in bytes. + +- `data_compressed_bytes` (`UInt64`) – Total size of compressed data in the data part. All the auxiliary files (for example, files with marks) are not included. + +- `data_uncompressed_bytes` (`UInt64`) – Total size of uncompressed data in the data part. All the auxiliary files (for example, files with marks) are not included. + +- `marks_bytes` (`UInt64`) – The size of the file with marks. + +- `modification_time` (`DateTime`) – The time the directory with the data part was modified. This usually corresponds to the time of data part creation.\| + +- `remove_time` (`DateTime`) – The time when the data part became inactive. + +- `refcount` (`UInt32`) – The number of places where the data part is used. A value greater than 2 indicates that the data part is used in queries or merges. + +- `min_date` (`Date`) – The minimum value of the date key in the data part. + +- `max_date` (`Date`) – The maximum value of the date key in the data part. + +- `min_time` (`DateTime`) – The minimum value of the date and time key in the data part. + +- `max_time`(`DateTime`) – The maximum value of the date and time key in the data part. + +- `partition_id` (`String`) – ID of the partition. + +- `min_block_number` (`UInt64`) – The minimum number of data parts that make up the current part after merging. + +- `max_block_number` (`UInt64`) – The maximum number of data parts that make up the current part after merging. + +- `level` (`UInt32`) – Depth of the merge tree. Zero means that the current part was created by insert rather than by merging other parts. + +- `data_version` (`UInt64`) – Number that is used to determine which mutations should be applied to the data part (mutations with a version higher than `data_version`). + +- `primary_key_bytes_in_memory` (`UInt64`) – The amount of memory (in bytes) used by primary key values. + +- `primary_key_bytes_in_memory_allocated` (`UInt64`) – The amount of memory (in bytes) reserved for primary key values. + +- `is_frozen` (`UInt8`) – Flag that shows that a partition data backup exists. 1, the backup exists. 0, the backup doesn't exist. For more details, see [FREEZE PARTITION](../../sql-reference/statements/alter.md#alter_freeze-partition) + +- `database` (`String`) – Name of the database. + +- `table` (`String`) – Name of the table. + +- `engine` (`String`) – Name of the table engine without parameters. + +- `path` (`String`) – Absolute path to the folder with data part files. + +- `disk` (`String`) – Name of a disk that stores the data part. + +- `hash_of_all_files` (`String`) – [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128) 的压缩文件。 + +- `hash_of_uncompressed_files` (`String`) – [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128) 未压缩的文件(带标记的文件,索引文件等。). + +- `uncompressed_hash_of_compressed_files` (`String`) – [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128) 压缩文件中的数据,就好像它们是未压缩的。 + +- `bytes` (`UInt64`) – Alias for `bytes_on_disk`. + +- `marks_size` (`UInt64`) – Alias for `marks_bytes`. diff --git a/docs/zh/operations/system-tables/processes.md b/docs/zh/operations/system-tables/processes.md new file mode 100644 index 00000000000..c42b7e59827 --- /dev/null +++ b/docs/zh/operations/system-tables/processes.md @@ -0,0 +1,20 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。流程 {#system_tables-processes} + +该系统表用于实现 `SHOW PROCESSLIST` 查询。 + +列: + +- `user` (String) – The user who made the query. Keep in mind that for distributed processing, queries are sent to remote servers under the `default` 用户。 该字段包含特定查询的用户名,而不是此查询启动的查询的用户名。 +- `address` (String) – The IP address the request was made from. The same for distributed processing. To track where a distributed query was originally made from, look at `system.processes` 查询请求者服务器上。 +- `elapsed` (Float64) – The time in seconds since request execution started. +- `rows_read` (UInt64) – The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. +- `bytes_read` (UInt64) – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. +- `total_rows_approx` (UInt64) – The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known. +- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max\_memory\_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) 设置。 +- `query` (String) – The query text. For `INSERT`,它不包括要插入的数据。 +- `query_id` (String) – Query ID, if defined. diff --git a/docs/zh/operations/system-tables/query_log.md b/docs/zh/operations/system-tables/query_log.md new file mode 100644 index 00000000000..7658196b81b --- /dev/null +++ b/docs/zh/operations/system-tables/query_log.md @@ -0,0 +1,143 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。query\_log {#system_tables-query_log} + +包含有关已执行查询的信息,例如,开始时间、处理持续时间、错误消息。 + +!!! note "注" + 此表不包含以下内容的摄取数据 `INSERT` 查询。 + +您可以更改查询日志记录的设置 [query\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) 服务器配置部分。 + +您可以通过设置禁用查询日志记录 [log\_queries=0](../../operations/settings/settings.md#settings-log-queries). 我们不建议关闭日志记录,因为此表中的信息对于解决问题很重要。 + +数据的冲洗周期设置在 `flush_interval_milliseconds` 的参数 [query\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) 服务器设置部分。 要强制冲洗,请使用 [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) 查询。 + +ClickHouse不会自动从表中删除数据。 看 [导言](../../operations/system-tables/index.md#system-tables-introduction) 欲了解更多详情。 + +该 `system.query_log` 表注册两种查询: + +1. 客户端直接运行的初始查询。 +2. 由其他查询启动的子查询(用于分布式查询执行)。 对于这些类型的查询,有关父查询的信息显示在 `initial_*` 列。 + +每个查询创建一个或两个行中 `query_log` 表,这取决于状态(见 `type` 列)的查询: + +1. 如果查询执行成功,则两行具有 `QueryStart` 和 `QueryFinish` 创建类型。 +2. 如果在查询处理过程中发生错误,两个事件与 `QueryStart` 和 `ExceptionWhileProcessing` 创建类型。 +3. 如果在启动查询之前发生错误,则单个事件具有 `ExceptionBeforeStart` 创建类型。 + +列: + +- `type` ([枚举8](../../sql-reference/data-types/enum.md)) — Type of an event that occurred when executing the query. Values: + - `'QueryStart' = 1` — Successful start of query execution. + - `'QueryFinish' = 2` — Successful end of query execution. + - `'ExceptionBeforeStart' = 3` — Exception before the start of query execution. + - `'ExceptionWhileProcessing' = 4` — Exception during the query execution. +- `event_date` ([日期](../../sql-reference/data-types/date.md)) — Query starting date. +- `event_time` ([日期时间](../../sql-reference/data-types/datetime.md)) — Query starting time. +- `query_start_time` ([日期时间](../../sql-reference/data-types/datetime.md)) — Start time of query execution. +- `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution in milliseconds. +- `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Total number or rows read from all tables and table functions participated in query. It includes usual subqueries, subqueries for `IN` 和 `JOIN`. 对于分布式查询 `read_rows` 包括在所有副本上读取的行总数。 每个副本发送它的 `read_rows` 值,并且查询的服务器-发起方汇总所有接收到的和本地的值。 缓存卷不会影响此值。 +- `read_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Total number or bytes read from all tables and table functions participated in query. It includes usual subqueries, subqueries for `IN` 和 `JOIN`. 对于分布式查询 `read_bytes` 包括在所有副本上读取的行总数。 每个副本发送它的 `read_bytes` 值,并且查询的服务器-发起方汇总所有接收到的和本地的值。 缓存卷不会影响此值。 +- `written_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — For `INSERT` 查询,写入的行数。 对于其他查询,列值为0。 +- `written_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — For `INSERT` 查询时,写入的字节数。 对于其他查询,列值为0。 +- `result_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of rows in a result of the `SELECT` 查询,或者在一些行 `INSERT` 查询。 +- `result_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — RAM volume in bytes used to store a query result. +- `memory_usage` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Memory consumption by the query. +- `query` ([字符串](../../sql-reference/data-types/string.md)) — Query string. +- `exception` ([字符串](../../sql-reference/data-types/string.md)) — Exception message. +- `exception_code` ([Int32](../../sql-reference/data-types/int-uint.md)) — Code of an exception. +- `stack_trace` ([字符串](../../sql-reference/data-types/string.md)) — [堆栈跟踪](https://en.wikipedia.org/wiki/Stack_trace). 如果查询成功完成,则为空字符串。 +- `is_initial_query` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Query type. Possible values: + - 1 — Query was initiated by the client. + - 0 — Query was initiated by another query as part of distributed query execution. +- `user` ([字符串](../../sql-reference/data-types/string.md)) — Name of the user who initiated the current query. +- `query_id` ([字符串](../../sql-reference/data-types/string.md)) — ID of the query. +- `address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP address that was used to make the query. +- `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — The client port that was used to make the query. +- `initial_user` ([字符串](../../sql-reference/data-types/string.md)) — Name of the user who ran the initial query (for distributed query execution). +- `initial_query_id` ([字符串](../../sql-reference/data-types/string.md)) — ID of the initial query (for distributed query execution). +- `initial_address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP address that the parent query was launched from. +- `initial_port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — The client port that was used to make the parent query. +- `interface` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Interface that the query was initiated from. Possible values: + - 1 — TCP. + - 2 — HTTP. +- `os_user` ([字符串](../../sql-reference/data-types/string.md)) — Operating system username who runs [ツ环板clientョツ嘉ッツ偲](../../interfaces/cli.md). +- `client_hostname` ([字符串](../../sql-reference/data-types/string.md)) — Hostname of the client machine where the [ツ环板clientョツ嘉ッツ偲](../../interfaces/cli.md) 或者运行另一个TCP客户端。 +- `client_name` ([字符串](../../sql-reference/data-types/string.md)) — The [ツ环板clientョツ嘉ッツ偲](../../interfaces/cli.md) 或另一个TCP客户端名称。 +- `client_revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Revision of the [ツ环板clientョツ嘉ッツ偲](../../interfaces/cli.md) 或另一个TCP客户端。 +- `client_version_major` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Major version of the [ツ环板clientョツ嘉ッツ偲](../../interfaces/cli.md) 或另一个TCP客户端。 +- `client_version_minor` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Minor version of the [ツ环板clientョツ嘉ッツ偲](../../interfaces/cli.md) 或另一个TCP客户端。 +- `client_version_patch` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Patch component of the [ツ环板clientョツ嘉ッツ偲](../../interfaces/cli.md) 或另一个TCP客户端版本。 +- `http_method` (UInt8) — HTTP method that initiated the query. Possible values: + - 0 — The query was launched from the TCP interface. + - 1 — `GET` 方法被使用。 + - 2 — `POST` 方法被使用。 +- `http_user_agent` ([字符串](../../sql-reference/data-types/string.md)) — The `UserAgent` http请求中传递的标头。 +- `quota_key` ([字符串](../../sql-reference/data-types/string.md)) — The “quota key” 在指定 [配额](../../operations/quotas.md) 设置(见 `keyed`). +- `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse revision. +- `thread_numbers` ([数组(UInt32)](../../sql-reference/data-types/array.md)) — Number of threads that are participating in query execution. +- `ProfileEvents.Names` ([数组(字符串)](../../sql-reference/data-types/array.md)) — Counters that measure different metrics. The description of them could be found in the table [系统。活动](../../operations/system-tables/events.md#system_tables-events) +- `ProfileEvents.Values` ([数组(UInt64)](../../sql-reference/data-types/array.md)) — Values of metrics that are listed in the `ProfileEvents.Names` 列。 +- `Settings.Names` ([数组(字符串)](../../sql-reference/data-types/array.md)) — Names of settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` 参数为1。 +- `Settings.Values` ([数组(字符串)](../../sql-reference/data-types/array.md)) — Values of settings that are listed in the `Settings.Names` 列。 + +**示例** + +``` sql +SELECT * FROM system.query_log LIMIT 1 FORMAT Vertical; +``` + +``` text +Row 1: +────── +type: QueryStart +event_date: 2020-05-13 +event_time: 2020-05-13 14:02:28 +query_start_time: 2020-05-13 14:02:28 +query_duration_ms: 0 +read_rows: 0 +read_bytes: 0 +written_rows: 0 +written_bytes: 0 +result_rows: 0 +result_bytes: 0 +memory_usage: 0 +query: SELECT 1 +exception_code: 0 +exception: +stack_trace: +is_initial_query: 1 +user: default +query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a +address: ::ffff:127.0.0.1 +port: 57720 +initial_user: default +initial_query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a +initial_address: ::ffff:127.0.0.1 +initial_port: 57720 +interface: 1 +os_user: bayonet +client_hostname: clickhouse.ru-central1.internal +client_name: ClickHouse client +client_revision: 54434 +client_version_major: 20 +client_version_minor: 4 +client_version_patch: 1 +http_method: 0 +http_user_agent: +quota_key: +revision: 54434 +thread_ids: [] +ProfileEvents.Names: [] +ProfileEvents.Values: [] +Settings.Names: ['use_uncompressed_cache','load_balancing','log_queries','max_memory_usage'] +Settings.Values: ['0','random','1','10000000000'] +``` + +**另请参阅** + +- [系统。query\_thread\_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) — This table contains information about each query execution thread. diff --git a/docs/zh/operations/system-tables/query_thread_log.md b/docs/zh/operations/system-tables/query_thread_log.md new file mode 100644 index 00000000000..115e69ec93c --- /dev/null +++ b/docs/zh/operations/system-tables/query_thread_log.md @@ -0,0 +1,118 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。query\_thread\_log {#system_tables-query_thread_log} + +包含有关执行查询的线程的信息,例如,线程名称、线程开始时间、查询处理的持续时间。 + +开始记录: + +1. 在配置参数 [query\_thread\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) 科。 +2. 设置 [log\_query\_threads](../../operations/settings/settings.md#settings-log-query-threads) 到1。 + +数据的冲洗周期设置在 `flush_interval_milliseconds` 的参数 [query\_thread\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) 服务器设置部分。 要强制冲洗,请使用 [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) 查询。 + +ClickHouse不会自动从表中删除数据。 看 [导言](../../operations/system-tables/index.md#system-tables-introduction) 欲了解更多详情。 + +列: + +- `event_date` ([日期](../../sql-reference/data-types/date.md)) — The date when the thread has finished execution of the query. +- `event_time` ([日期时间](../../sql-reference/data-types/datetime.md)) — The date and time when the thread has finished execution of the query. +- `query_start_time` ([日期时间](../../sql-reference/data-types/datetime.md)) — Start time of query execution. +- `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution. +- `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of read rows. +- `read_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of read bytes. +- `written_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — For `INSERT` 查询,写入的行数。 对于其他查询,列值为0。 +- `written_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — For `INSERT` 查询时,写入的字节数。 对于其他查询,列值为0。 +- `memory_usage` ([Int64](../../sql-reference/data-types/int-uint.md)) — The difference between the amount of allocated and freed memory in context of this thread. +- `peak_memory_usage` ([Int64](../../sql-reference/data-types/int-uint.md)) — The maximum difference between the amount of allocated and freed memory in context of this thread. +- `thread_name` ([字符串](../../sql-reference/data-types/string.md)) — Name of the thread. +- `thread_number` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Internal thread ID. +- `thread_id` ([Int32](../../sql-reference/data-types/int-uint.md)) — thread ID. +- `master_thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — OS initial ID of initial thread. +- `query` ([字符串](../../sql-reference/data-types/string.md)) — Query string. +- `is_initial_query` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Query type. Possible values: + - 1 — Query was initiated by the client. + - 0 — Query was initiated by another query for distributed query execution. +- `user` ([字符串](../../sql-reference/data-types/string.md)) — Name of the user who initiated the current query. +- `query_id` ([字符串](../../sql-reference/data-types/string.md)) — ID of the query. +- `address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP address that was used to make the query. +- `port` ([UInt16](../../sql-reference/data-types/int-uint.md#uint-ranges)) — The client port that was used to make the query. +- `initial_user` ([字符串](../../sql-reference/data-types/string.md)) — Name of the user who ran the initial query (for distributed query execution). +- `initial_query_id` ([字符串](../../sql-reference/data-types/string.md)) — ID of the initial query (for distributed query execution). +- `initial_address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP address that the parent query was launched from. +- `initial_port` ([UInt16](../../sql-reference/data-types/int-uint.md#uint-ranges)) — The client port that was used to make the parent query. +- `interface` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Interface that the query was initiated from. Possible values: + - 1 — TCP. + - 2 — HTTP. +- `os_user` ([字符串](../../sql-reference/data-types/string.md)) — OS's username who runs [ツ环板clientョツ嘉ッツ偲](../../interfaces/cli.md). +- `client_hostname` ([字符串](../../sql-reference/data-types/string.md)) — Hostname of the client machine where the [ツ环板clientョツ嘉ッツ偲](../../interfaces/cli.md) 或者运行另一个TCP客户端。 +- `client_name` ([字符串](../../sql-reference/data-types/string.md)) — The [ツ环板clientョツ嘉ッツ偲](../../interfaces/cli.md) 或另一个TCP客户端名称。 +- `client_revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Revision of the [ツ环板clientョツ嘉ッツ偲](../../interfaces/cli.md) 或另一个TCP客户端。 +- `client_version_major` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Major version of the [ツ环板clientョツ嘉ッツ偲](../../interfaces/cli.md) 或另一个TCP客户端。 +- `client_version_minor` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Minor version of the [ツ环板clientョツ嘉ッツ偲](../../interfaces/cli.md) 或另一个TCP客户端。 +- `client_version_patch` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Patch component of the [ツ环板clientョツ嘉ッツ偲](../../interfaces/cli.md) 或另一个TCP客户端版本。 +- `http_method` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — HTTP method that initiated the query. Possible values: + - 0 — The query was launched from the TCP interface. + - 1 — `GET` 方法被使用。 + - 2 — `POST` 方法被使用。 +- `http_user_agent` ([字符串](../../sql-reference/data-types/string.md)) — The `UserAgent` http请求中传递的标头。 +- `quota_key` ([字符串](../../sql-reference/data-types/string.md)) — The “quota key” 在指定 [配额](../../operations/quotas.md) 设置(见 `keyed`). +- `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse revision. +- `ProfileEvents.Names` ([数组(字符串)](../../sql-reference/data-types/array.md)) — Counters that measure different metrics for this thread. The description of them could be found in the table [系统。活动](#system_tables-events). +- `ProfileEvents.Values` ([数组(UInt64)](../../sql-reference/data-types/array.md)) — Values of metrics for this thread that are listed in the `ProfileEvents.Names` 列。 + +**示例** + +``` sql + SELECT * FROM system.query_thread_log LIMIT 1 FORMAT Vertical +``` + +``` text +Row 1: +────── +event_date: 2020-05-13 +event_time: 2020-05-13 14:02:28 +query_start_time: 2020-05-13 14:02:28 +query_duration_ms: 0 +read_rows: 1 +read_bytes: 1 +written_rows: 0 +written_bytes: 0 +memory_usage: 0 +peak_memory_usage: 0 +thread_name: QueryPipelineEx +thread_id: 28952 +master_thread_id: 28924 +query: SELECT 1 +is_initial_query: 1 +user: default +query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a +address: ::ffff:127.0.0.1 +port: 57720 +initial_user: default +initial_query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a +initial_address: ::ffff:127.0.0.1 +initial_port: 57720 +interface: 1 +os_user: bayonet +client_hostname: clickhouse.ru-central1.internal +client_name: ClickHouse client +client_revision: 54434 +client_version_major: 20 +client_version_minor: 4 +client_version_patch: 1 +http_method: 0 +http_user_agent: +quota_key: +revision: 54434 +ProfileEvents.Names: ['ContextLock','RealTimeMicroseconds','UserTimeMicroseconds','OSCPUWaitMicroseconds','OSCPUVirtualTimeMicroseconds'] +ProfileEvents.Values: [1,97,81,5,81] +... +``` + +**另请参阅** + +- [系统。query\_log](../../operations/system-tables/query_log.md#system_tables-query_log) — Description of the `query_log` 系统表,其中包含有关查询执行的公共信息。 diff --git a/docs/zh/operations/system-tables/replicas.md b/docs/zh/operations/system-tables/replicas.md new file mode 100644 index 00000000000..4be74b118c5 --- /dev/null +++ b/docs/zh/operations/system-tables/replicas.md @@ -0,0 +1,126 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。副本 {#system_tables-replicas} + +包含驻留在本地服务器上的复制表的信息和状态。 +此表可用于监视。 该表对于每个已复制的\*表都包含一行。 + +示例: + +``` sql +SELECT * +FROM system.replicas +WHERE table = 'visits' +FORMAT Vertical +``` + +``` text +Row 1: +────── +database: merge +table: visits +engine: ReplicatedCollapsingMergeTree +is_leader: 1 +can_become_leader: 1 +is_readonly: 0 +is_session_expired: 0 +future_parts: 1 +parts_to_check: 0 +zookeeper_path: /clickhouse/tables/01-06/visits +replica_name: example01-06-1.yandex.ru +replica_path: /clickhouse/tables/01-06/visits/replicas/example01-06-1.yandex.ru +columns_version: 9 +queue_size: 1 +inserts_in_queue: 0 +merges_in_queue: 1 +part_mutations_in_queue: 0 +queue_oldest_time: 2020-02-20 08:34:30 +inserts_oldest_time: 0000-00-00 00:00:00 +merges_oldest_time: 2020-02-20 08:34:30 +part_mutations_oldest_time: 0000-00-00 00:00:00 +oldest_part_to_get: +oldest_part_to_merge_to: 20200220_20284_20840_7 +oldest_part_to_mutate_to: +log_max_index: 596273 +log_pointer: 596274 +last_queue_update: 2020-02-20 08:34:32 +absolute_delay: 0 +total_replicas: 2 +active_replicas: 2 +``` + +列: + +- `database` (`String`)-数据库名称 +- `table` (`String`)-表名 +- `engine` (`String`)-表引擎名称 +- `is_leader` (`UInt8`)-副本是否是领导者。 + 一次只有一个副本可以成为领导者。 领导者负责选择要执行的后台合并。 + 请注意,可以对任何可用且在ZK中具有会话的副本执行写操作,而不管该副本是否为leader。 +- `can_become_leader` (`UInt8`)-副本是否可以当选为领导者。 +- `is_readonly` (`UInt8`)-副本是否处于只读模式。 + 如果配置没有ZooKeeper的部分,如果在ZooKeeper中重新初始化会话时发生未知错误,以及在ZooKeeper中重新初始化会话时发生未知错误,则此模式将打开。 +- `is_session_expired` (`UInt8`)-与ZooKeeper的会话已经过期。 基本上一样 `is_readonly`. +- `future_parts` (`UInt32`)-由于尚未完成的插入或合并而显示的数据部分的数量。 +- `parts_to_check` (`UInt32`)-队列中用于验证的数据部分的数量。 如果怀疑零件可能已损坏,则将其放入验证队列。 +- `zookeeper_path` (`String`)-在ZooKeeper中的表数据路径。 +- `replica_name` (`String`)-在动物园管理员副本名称. 同一表的不同副本具有不同的名称。 +- `replica_path` (`String`)-在ZooKeeper中的副本数据的路径。 与连接相同 ‘zookeeper\_path/replicas/replica\_path’. +- `columns_version` (`Int32`)-表结构的版本号。 指示执行ALTER的次数。 如果副本有不同的版本,这意味着一些副本还没有做出所有的改变。 +- `queue_size` (`UInt32`)-等待执行的操作的队列大小。 操作包括插入数据块、合并和某些其他操作。 它通常与 `future_parts`. +- `inserts_in_queue` (`UInt32`)-需要插入数据块的数量。 插入通常复制得相当快。 如果这个数字很大,这意味着有什么不对劲。 +- `merges_in_queue` (`UInt32`)-等待进行合并的数量。 有时合并时间很长,因此此值可能长时间大于零。 +- `part_mutations_in_queue` (`UInt32`)-等待进行的突变的数量。 +- `queue_oldest_time` (`DateTime`)-如果 `queue_size` 大于0,显示何时将最旧的操作添加到队列中。 +- `inserts_oldest_time` (`DateTime`)-看 `queue_oldest_time` +- `merges_oldest_time` (`DateTime`)-看 `queue_oldest_time` +- `part_mutations_oldest_time` (`DateTime`)-看 `queue_oldest_time` + +接下来的4列只有在有ZK活动会话的情况下才具有非零值。 + +- `log_max_index` (`UInt64`)-一般活动日志中的最大条目数。 +- `log_pointer` (`UInt64`)-副本复制到其执行队列的常规活动日志中的最大条目数加一。 如果 `log_pointer` 比 `log_max_index`,有点不对劲。 +- `last_queue_update` (`DateTime`)-上次更新队列时。 +- `absolute_delay` (`UInt64`)-当前副本有多大滞后秒。 +- `total_replicas` (`UInt8`)-此表的已知副本总数。 +- `active_replicas` (`UInt8`)-在ZooKeeper中具有会话的此表的副本的数量(即正常运行的副本的数量)。 + +如果您请求所有列,表可能会工作得有点慢,因为每行都会从ZooKeeper进行几次读取。 +如果您没有请求最后4列(log\_max\_index,log\_pointer,total\_replicas,active\_replicas),表工作得很快。 + +例如,您可以检查一切是否正常工作,如下所示: + +``` sql +SELECT + database, + table, + is_leader, + is_readonly, + is_session_expired, + future_parts, + parts_to_check, + columns_version, + queue_size, + inserts_in_queue, + merges_in_queue, + log_max_index, + log_pointer, + total_replicas, + active_replicas +FROM system.replicas +WHERE + is_readonly + OR is_session_expired + OR future_parts > 20 + OR parts_to_check > 10 + OR queue_size > 20 + OR inserts_in_queue > 10 + OR log_max_index - log_pointer > 10 + OR total_replicas < 2 + OR active_replicas < total_replicas +``` + +如果这个查询没有返回任何东西,这意味着一切都很好。 diff --git a/docs/zh/operations/system-tables/settings.md b/docs/zh/operations/system-tables/settings.md new file mode 100644 index 00000000000..c717c8c9562 --- /dev/null +++ b/docs/zh/operations/system-tables/settings.md @@ -0,0 +1,55 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。设置 {#system-tables-system-settings} + +包含有关当前用户的会话设置的信息。 + +列: + +- `name` ([字符串](../../sql-reference/data-types/string.md)) — Setting name. +- `value` ([字符串](../../sql-reference/data-types/string.md)) — Setting value. +- `changed` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether a setting is changed from its default value. +- `description` ([字符串](../../sql-reference/data-types/string.md)) — Short setting description. +- `min` ([可为空](../../sql-reference/data-types/nullable.md)([字符串](../../sql-reference/data-types/string.md))) — Minimum value of the setting, if any is set via [制约因素](../../operations/settings/constraints-on-settings.md#constraints-on-settings). 如果设置没有最小值,则包含 [NULL](../../sql-reference/syntax.md#null-literal). +- `max` ([可为空](../../sql-reference/data-types/nullable.md)([字符串](../../sql-reference/data-types/string.md))) — Maximum value of the setting, if any is set via [制约因素](../../operations/settings/constraints-on-settings.md#constraints-on-settings). 如果设置没有最大值,则包含 [NULL](../../sql-reference/syntax.md#null-literal). +- `readonly` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether the current user can change the setting: + - `0` — Current user can change the setting. + - `1` — Current user can't change the setting. + +**示例** + +下面的示例演示如何获取有关名称包含的设置的信息 `min_i`. + +``` sql +SELECT * +FROM system.settings +WHERE name LIKE '%min_i%' +``` + +``` text +┌─name────────────────────────────────────────┬─value─────┬─changed─┬─description───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─min──┬─max──┬─readonly─┐ +│ min_insert_block_size_rows │ 1048576 │ 0 │ Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough. │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ +│ min_insert_block_size_bytes │ 268435456 │ 0 │ Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough. │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ +│ read_backoff_min_interval_between_events_ms │ 1000 │ 0 │ Settings to reduce the number of threads in case of slow reads. Do not pay attention to the event, if the previous one has passed less than a certain amount of time. │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ +└─────────────────────────────────────────────┴───────────┴─────────┴───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴──────┴──────┴──────────┘ +``` + +使用 `WHERE changed` 可以是有用的,例如,当你想检查: + +- 配置文件中的设置是否正确加载并正在使用。 +- 在当前会话中更改的设置。 + + + +``` sql +SELECT * FROM system.settings WHERE changed AND name='load_balancing' +``` + +**另请参阅** + +- [设置](../../operations/settings/index.md#session-settings-intro) +- [查询权限](../../operations/settings/permissions-for-queries.md#settings_readonly) +- [对设置的限制](../../operations/settings/constraints-on-settings.md) diff --git a/docs/zh/operations/system-tables/storage_policies.md b/docs/zh/operations/system-tables/storage_policies.md new file mode 100644 index 00000000000..29347aa36c2 --- /dev/null +++ b/docs/zh/operations/system-tables/storage_policies.md @@ -0,0 +1,19 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。storage\_policies {#system_tables-storage_policies} + +包含有关存储策略和卷中定义的信息 [服务器配置](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). + +列: + +- `policy_name` ([字符串](../../sql-reference/data-types/string.md)) — Name of the storage policy. +- `volume_name` ([字符串](../../sql-reference/data-types/string.md)) — Volume name defined in the storage policy. +- `volume_priority` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Volume order number in the configuration. +- `disks` ([数组(字符串)](../../sql-reference/data-types/array.md)) — Disk names, defined in the storage policy. +- `max_data_part_size` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Maximum size of a data part that can be stored on volume disks (0 — no limit). +- `move_factor` ([Float64](../../sql-reference/data-types/float.md)) — Ratio of free disk space. When the ratio exceeds the value of configuration parameter, ClickHouse start to move data to the next volume in order. + +如果存储策略包含多个卷,则每个卷的信息将存储在表的单独行中。 diff --git a/docs/zh/operations/system-tables/table_engines.md b/docs/zh/operations/system-tables/table_engines.md new file mode 100644 index 00000000000..401a3a8616d --- /dev/null +++ b/docs/zh/operations/system-tables/table_engines.md @@ -0,0 +1,40 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。表\_engines {#system-table-engines} + +包含服务器支持的表引擎的描述及其功能支持信息。 + +此表包含以下列(列类型显示在括号中): + +- `name` (String) — The name of table engine. +- `supports_settings` (UInt8) — Flag that indicates if table engine supports `SETTINGS` 条款 +- `supports_skipping_indices` (UInt8) — Flag that indicates if table engine supports [跳过索引](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-data_skipping-indexes). +- `supports_ttl` (UInt8) — Flag that indicates if table engine supports [TTL](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). +- `supports_sort_order` (UInt8) — Flag that indicates if table engine supports clauses `PARTITION_BY`, `PRIMARY_KEY`, `ORDER_BY` 和 `SAMPLE_BY`. +- `supports_replication` (UInt8) — Flag that indicates if table engine supports [数据复制](../../engines/table-engines/mergetree-family/replication.md). +- `supports_duduplication` (UInt8) — Flag that indicates if table engine supports data deduplication. + +示例: + +``` sql +SELECT * +FROM system.table_engines +WHERE name in ('Kafka', 'MergeTree', 'ReplicatedCollapsingMergeTree') +``` + +``` text +┌─name──────────────────────────┬─supports_settings─┬─supports_skipping_indices─┬─supports_sort_order─┬─supports_ttl─┬─supports_replication─┬─supports_deduplication─┐ +│ Kafka │ 1 │ 0 │ 0 │ 0 │ 0 │ 0 │ +│ MergeTree │ 1 │ 1 │ 1 │ 1 │ 0 │ 0 │ +│ ReplicatedCollapsingMergeTree │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ +└───────────────────────────────┴───────────────────┴───────────────────────────┴─────────────────────┴──────────────┴──────────────────────┴────────────────────────┘ +``` + +**另请参阅** + +- 梅树家族 [查询子句](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-query-clauses) +- 卡夫卡 [设置](../../engines/table-engines/integrations/kafka.md#table_engine-kafka-creating-a-table) +- 加入我们 [设置](../../engines/table-engines/special/join.md#join-limitations-and-settings) diff --git a/docs/zh/operations/system-tables/tables.md b/docs/zh/operations/system-tables/tables.md new file mode 100644 index 00000000000..a690e938a3a --- /dev/null +++ b/docs/zh/operations/system-tables/tables.md @@ -0,0 +1,54 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。表 {#system-tables} + +包含服务器知道的每个表的元数据。 分离的表不显示在 `system.tables`. + +此表包含以下列(列类型显示在括号中): + +- `database` (String) — The name of the database the table is in. + +- `name` (String) — Table name. + +- `engine` (String) — Table engine name (without parameters). + +- `is_temporary` (UInt8)-指示表是否是临时的标志。 + +- `data_path` (String)-文件系统中表数据的路径。 + +- `metadata_path` (String)-文件系统中表元数据的路径。 + +- `metadata_modification_time` (DateTime)-表元数据的最新修改时间。 + +- `dependencies_database` (数组(字符串))-数据库依赖关系. + +- `dependencies_table` (数组(字符串))-表依赖关系 ([MaterializedView](../../engines/table-engines/special/materializedview.md) 基于当前表的表)。 + +- `create_table_query` (String)-用于创建表的查询。 + +- `engine_full` (String)-表引擎的参数。 + +- `partition_key` (String)-表中指定的分区键表达式。 + +- `sorting_key` (String)-表中指定的排序键表达式。 + +- `primary_key` (String)-表中指定的主键表达式。 + +- `sampling_key` (String)-表中指定的采样键表达式。 + +- `storage_policy` (字符串)-存储策略: + + - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) + - [分布](../../engines/table-engines/special/distributed.md#distributed) + +- `total_rows` (Nullable(UInt64))-总行数,如果可以快速确定表中的确切行数,否则 `Null` (包括内衣 `Buffer` 表)。 + +- `total_bytes` (Nullable(UInt64))-总字节数,如果可以快速确定存储表的确切字节数,否则 `Null` (**不** 包括任何底层存储)。 + + - If the table stores data on disk, returns used space on disk (i.e. compressed). + - 如果表在内存中存储数据,返回在内存中使用的近似字节数. + +该 `system.tables` 表中使用 `SHOW TABLES` 查询实现。 diff --git a/docs/zh/operations/system-tables/text_log.md b/docs/zh/operations/system-tables/text_log.md new file mode 100644 index 00000000000..60c7a3ed90c --- /dev/null +++ b/docs/zh/operations/system-tables/text_log.md @@ -0,0 +1,31 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。text\_log {#system-tables-text-log} + +包含日志记录条目。 进入该表的日志记录级别可以通过以下方式进行限制 `text_log.level` 服务器设置。 + +列: + +- `event_date` (Date) — Date of the entry. +- `event_time` (DateTime) — Time of the entry. +- `microseconds` (UInt32) — Microseconds of the entry. +- `thread_name` (String) — Name of the thread from which the logging was done. +- `thread_id` (UInt64) — OS thread ID. +- `level` (`Enum8`) — Entry level. Possible values: + - `1` 或 `'Fatal'`. + - `2` 或 `'Critical'`. + - `3` 或 `'Error'`. + - `4` 或 `'Warning'`. + - `5` 或 `'Notice'`. + - `6` 或 `'Information'`. + - `7` 或 `'Debug'`. + - `8` 或 `'Trace'`. +- `query_id` (String) — ID of the query. +- `logger_name` (LowCardinality(String)) — Name of the logger (i.e. `DDLWorker`). +- `message` (String) — The message itself. +- `revision` (UInt32) — ClickHouse revision. +- `source_file` (LowCardinality(String)) — Source file from which the logging was done. +- `source_line` (UInt64) — Source line from which the logging was done. diff --git a/docs/zh/operations/system-tables/trace_log.md b/docs/zh/operations/system-tables/trace_log.md new file mode 100644 index 00000000000..3004dca707a --- /dev/null +++ b/docs/zh/operations/system-tables/trace_log.md @@ -0,0 +1,53 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。trace\_log {#system_tables-trace_log} + +包含采样查询探查器收集的堆栈跟踪。 + +ClickHouse创建此表时 [trace\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) 服务器配置部分被设置。 也是 [query\_profiler\_real\_time\_period\_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns) 和 [query\_profiler\_cpu\_time\_period\_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns) 应设置设置。 + +要分析日志,请使用 `addressToLine`, `addressToSymbol` 和 `demangle` 内省功能。 + +列: + +- `event_date` ([日期](../../sql-reference/data-types/date.md)) — Date of sampling moment. + +- `event_time` ([日期时间](../../sql-reference/data-types/datetime.md)) — Timestamp of the sampling moment. + +- `timestamp_ns` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Timestamp of the sampling moment in nanoseconds. + +- `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse server build revision. + + 通过以下方式连接到服务器 `clickhouse-client`,你看到的字符串类似于 `Connected to ClickHouse server version 19.18.1 revision 54429.`. 该字段包含 `revision`,但不是 `version` 的服务器。 + +- `timer_type` ([枚举8](../../sql-reference/data-types/enum.md)) — Timer type: + + - `Real` 表示挂钟时间。 + - `CPU` 表示CPU时间。 + +- `thread_number` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Thread identifier. + +- `query_id` ([字符串](../../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query\_log](#system_tables-query_log) 系统表. + +- `trace` ([数组(UInt64)](../../sql-reference/data-types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process. + +**示例** + +``` sql +SELECT * FROM system.trace_log LIMIT 1 \G +``` + +``` text +Row 1: +────── +event_date: 2019-11-15 +event_time: 2019-11-15 15:09:38 +revision: 54428 +timer_type: Real +thread_number: 48 +query_id: acc4d61f-5bd1-4a3e-bc91-2180be37c915 +trace: [94222141367858,94222152240175,94222152325351,94222152329944,94222152330796,94222151449980,94222144088167,94222151682763,94222144088167,94222151682763,94222144088167,94222144058283,94222144059248,94222091840750,94222091842302,94222091831228,94222189631488,140509950166747,140509942945935] +``` diff --git a/docs/zh/operations/system-tables/zookeeper.md b/docs/zh/operations/system-tables/zookeeper.md new file mode 100644 index 00000000000..b66e5262df3 --- /dev/null +++ b/docs/zh/operations/system-tables/zookeeper.md @@ -0,0 +1,75 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 系统。动物园管理员 {#system-zookeeper} + +如果未配置ZooKeeper,则表不存在。 允许从配置中定义的ZooKeeper集群读取数据。 +查询必须具有 ‘path’ WHERE子句中的平等条件。 这是ZooKeeper中您想要获取数据的孩子的路径。 + +查询 `SELECT * FROM system.zookeeper WHERE path = '/clickhouse'` 输出对所有孩子的数据 `/clickhouse` 节点。 +要输出所有根节点的数据,write path= ‘/’. +如果在指定的路径 ‘path’ 不存在,将引发异常。 + +列: + +- `name` (String) — The name of the node. +- `path` (String) — The path to the node. +- `value` (String) — Node value. +- `dataLength` (Int32) — Size of the value. +- `numChildren` (Int32) — Number of descendants. +- `czxid` (Int64) — ID of the transaction that created the node. +- `mzxid` (Int64) — ID of the transaction that last changed the node. +- `pzxid` (Int64) — ID of the transaction that last deleted or added descendants. +- `ctime` (DateTime) — Time of node creation. +- `mtime` (DateTime) — Time of the last modification of the node. +- `version` (Int32) — Node version: the number of times the node was changed. +- `cversion` (Int32) — Number of added or removed descendants. +- `aversion` (Int32) — Number of changes to the ACL. +- `ephemeralOwner` (Int64) — For ephemeral nodes, the ID of the session that owns this node. + +示例: + +``` sql +SELECT * +FROM system.zookeeper +WHERE path = '/clickhouse/tables/01-08/visits/replicas' +FORMAT Vertical +``` + +``` text +Row 1: +────── +name: example01-08-1.yandex.ru +value: +czxid: 932998691229 +mzxid: 932998691229 +ctime: 2015-03-27 16:49:51 +mtime: 2015-03-27 16:49:51 +version: 0 +cversion: 47 +aversion: 0 +ephemeralOwner: 0 +dataLength: 0 +numChildren: 7 +pzxid: 987021031383 +path: /clickhouse/tables/01-08/visits/replicas + +Row 2: +────── +name: example01-08-2.yandex.ru +value: +czxid: 933002738135 +mzxid: 933002738135 +ctime: 2015-03-27 16:57:01 +mtime: 2015-03-27 16:57:01 +version: 0 +cversion: 37 +aversion: 0 +ephemeralOwner: 0 +dataLength: 0 +numChildren: 7 +pzxid: 987021252247 +path: /clickhouse/tables/01-08/visits/replicas +``` diff --git a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts.md index c67deb55401..756eee31026 100644 --- a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -19,7 +19,7 @@ ClickHouse: 字典可以在服务器启动或首次使用时加载,具体取决于 [dictionaries\_lazy\_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) 设置。 -该 [字典](../../../operations/system-tables.md#system_tables-dictionaries) 系统表包含有关在服务器上配置的字典的信息。 对于每个字典,你可以在那里找到: +该 [字典](../../../operations/system-tables/dictionaries.md#system_tables-dictionaries) 系统表包含有关在服务器上配置的字典的信息。 对于每个字典,你可以在那里找到: - 字典的状态。 - 配置参数。 diff --git a/docs/zh/sql-reference/functions/introspection.md b/docs/zh/sql-reference/functions/introspection.md index 43d8b596dfb..4b4367f3dc8 100644 --- a/docs/zh/sql-reference/functions/introspection.md +++ b/docs/zh/sql-reference/functions/introspection.md @@ -20,7 +20,7 @@ toc_title: "\u81EA\u7701" For security reasons introspection functions are disabled by default. -ClickHouse将探查器报告保存到 [trace\_log](../../operations/system-tables.md#system_tables-trace_log) 系统表. 确保正确配置了表和探查器。 +ClickHouse将探查器报告保存到 [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) 系统表. 确保正确配置了表和探查器。 ## addressToLine {#addresstoline} diff --git a/docs/zh/sql-reference/operators/in.md b/docs/zh/sql-reference/operators/in.md deleted file mode 120000 index 3a2feda2f61..00000000000 --- a/docs/zh/sql-reference/operators/in.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/sql-reference/operators/in.md \ No newline at end of file diff --git a/docs/zh/sql-reference/operators/in.md b/docs/zh/sql-reference/operators/in.md new file mode 100644 index 00000000000..eaaa477fbe1 --- /dev/null +++ b/docs/zh/sql-reference/operators/in.md @@ -0,0 +1,204 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +--- + +# 在运营商 {#select-in-operators} + +该 `IN`, `NOT IN`, `GLOBAL IN`,和 `GLOBAL NOT IN` 运算符是单独复盖的,因为它们的功能相当丰富。 + +运算符的左侧是单列或元组。 + +例: + +``` sql +SELECT UserID IN (123, 456) FROM ... +SELECT (CounterID, UserID) IN ((34, 123), (101500, 456)) FROM ... +``` + +如果左侧是索引中的单列,而右侧是一组常量,则系统将使用索引处理查询。 + +Don't list too many values explicitly (i.e. millions). If a data set is large, put it in a temporary table (for example, see the section “External data for query processing”),然后使用子查询。 + +运算符的右侧可以是一组常量表达式、一组带有常量表达式的元组(如上面的示例所示),或括号中的数据库表或SELECT子查询的名称。 + +如果运算符的右侧是表的名称(例如, `UserID IN users`),这相当于子查询 `UserID IN (SELECT * FROM users)`. 使用与查询一起发送的外部数据时,请使用此选项。 例如,查询可以与一组用户Id一起发送到 ‘users’ 应过滤的临时表。 + +如果运算符的右侧是具有Set引擎的表名(始终位于RAM中的准备好的数据集),则不会为每个查询重新创建数据集。 + +子查询可以指定多个用于筛选元组的列。 +示例: + +``` sql +SELECT (CounterID, UserID) IN (SELECT CounterID, UserID FROM ...) FROM ... +``` + +IN运算符左侧和右侧的列应具有相同的类型。 + +IN运算符和子查询可能出现在查询的任何部分,包括聚合函数和lambda函数。 +示例: + +``` sql +SELECT + EventDate, + avg(UserID IN + ( + SELECT UserID + FROM test.hits + WHERE EventDate = toDate('2014-03-17') + )) AS ratio +FROM test.hits +GROUP BY EventDate +ORDER BY EventDate ASC +``` + +``` text +┌──EventDate─┬────ratio─┐ +│ 2014-03-17 │ 1 │ +│ 2014-03-18 │ 0.807696 │ +│ 2014-03-19 │ 0.755406 │ +│ 2014-03-20 │ 0.723218 │ +│ 2014-03-21 │ 0.697021 │ +│ 2014-03-22 │ 0.647851 │ +│ 2014-03-23 │ 0.648416 │ +└────────────┴──────────┘ +``` + +对于3月17日后的每一天,计算3月17日访问该网站的用户所做的浏览量百分比。 +IN子句中的子查询始终只在单个服务器上运行一次。 没有依赖子查询。 + +## 空处理 {#in-null-processing} + +在请求处理过程中, `IN` 运算符假定运算的结果 [NULL](../../sql-reference/syntax.md#null-literal) 总是等于 `0`,无论是否 `NULL` 位于操作员的右侧或左侧。 `NULL` 值不包含在任何数据集中,彼此不对应,并且在以下情况下无法进行比较 [transform\_null\_in=0](../../operations/settings/settings.md#transform_null_in). + +下面是一个例子 `t_null` 表: + +``` text +┌─x─┬────y─┐ +│ 1 │ ᴺᵁᴸᴸ │ +│ 2 │ 3 │ +└───┴──────┘ +``` + +运行查询 `SELECT x FROM t_null WHERE y IN (NULL,3)` 为您提供以下结果: + +``` text +┌─x─┐ +│ 2 │ +└───┘ +``` + +你可以看到,在其中的行 `y = NULL` 被抛出的查询结果。 这是因为ClickHouse无法决定是否 `NULL` 包含在 `(NULL,3)` 设置,返回 `0` 作为操作的结果,和 `SELECT` 从最终输出中排除此行。 + +``` sql +SELECT y IN (NULL, 3) +FROM t_null +``` + +``` text +┌─in(y, tuple(NULL, 3))─┐ +│ 0 │ +│ 1 │ +└───────────────────────┘ +``` + +## 分布式子查询 {#select-distributed-subqueries} + +带子查询的IN-s有两个选项(类似于连接):normal `IN` / `JOIN` 和 `GLOBAL IN` / `GLOBAL JOIN`. 它们在分布式查询处理的运行方式上有所不同。 + +!!! attention "注意" + 请记住,下面描述的算法可能会有不同的工作方式取决于 [设置](../../operations/settings/settings.md) `distributed_product_mode` 设置。 + +当使用常规IN时,查询被发送到远程服务器,并且它们中的每个服务器都在运行子查询 `IN` 或 `JOIN` 条款 + +使用时 `GLOBAL IN` / `GLOBAL JOINs`,首先所有的子查询都运行 `GLOBAL IN` / `GLOBAL JOINs`,并将结果收集在临时表中。 然后将临时表发送到每个远程服务器,其中使用此临时数据运行查询。 + +对于非分布式查询,请使用常规 `IN` / `JOIN`. + +在使用子查询时要小心 `IN` / `JOIN` 用于分布式查询处理的子句。 + +让我们来看看一些例子。 假设集群中的每个服务器都有一个正常的 **local\_table**. 每个服务器还具有 **distributed\_table** 表与 **分布** 类型,它查看群集中的所有服务器。 + +对于查询 **distributed\_table**,查询将被发送到所有远程服务器,并使用以下命令在其上运行 **local\_table**. + +例如,查询 + +``` sql +SELECT uniq(UserID) FROM distributed_table +``` + +将被发送到所有远程服务器 + +``` sql +SELECT uniq(UserID) FROM local_table +``` + +并且并行运行它们中的每一个,直到达到可以结合中间结果的阶段。 然后将中间结果返回给请求者服务器并在其上合并,并将最终结果发送给客户端。 + +现在让我们检查一个查询IN: + +``` sql +SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM local_table WHERE CounterID = 34) +``` + +- 计算两个网站的受众的交集。 + +此查询将以下列方式发送到所有远程服务器 + +``` sql +SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM local_table WHERE CounterID = 34) +``` + +换句话说,IN子句中的数据集将在每台服务器上独立收集,仅在每台服务器上本地存储的数据中收集。 + +如果您已经为此情况做好准备,并且已经将数据分散到群集服务器上,以便单个用户Id的数据完全驻留在单个服务器上,则这将正常和最佳地工作。 在这种情况下,所有必要的数据将在每台服务器上本地提供。 否则,结果将是不准确的。 我们将查询的这种变体称为 “local IN”. + +若要更正数据在群集服务器上随机传播时查询的工作方式,可以指定 **distributed\_table** 在子查询中。 查询如下所示: + +``` sql +SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) +``` + +此查询将以下列方式发送到所有远程服务器 + +``` sql +SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) +``` + +子查询将开始在每个远程服务器上运行。 由于子查询使用分布式表,因此每个远程服务器上的子查询将重新发送到每个远程服务器 + +``` sql +SELECT UserID FROM local_table WHERE CounterID = 34 +``` + +例如,如果您有100台服务器的集群,则执行整个查询将需要10,000个基本请求,这通常被认为是不可接受的。 + +在这种情况下,应始终使用GLOBAL IN而不是IN。 让我们来看看它是如何工作的查询 + +``` sql +SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID GLOBAL IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) +``` + +请求者服务器将运行子查询 + +``` sql +SELECT UserID FROM distributed_table WHERE CounterID = 34 +``` + +结果将被放在RAM中的临时表中。 然后请求将被发送到每个远程服务器 + +``` sql +SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID GLOBAL IN _data1 +``` + +和临时表 `_data1` 将通过查询发送到每个远程服务器(临时表的名称是实现定义的)。 + +这比使用正常IN更优化。 但是,请记住以下几点: + +1. 创建临时表时,数据不是唯一的。 要减少通过网络传输的数据量,请在子查询中指定DISTINCT。 (你不需要为正常人做这个。) +2. 临时表将被发送到所有远程服务器。 传输不考虑网络拓扑。 例如,如果10个远程服务器驻留在与请求者服务器非常远程的数据中心中,则数据将通过通道发送10次到远程数据中心。 使用GLOBAL IN时尽量避免使用大型数据集。 +3. 将数据传输到远程服务器时,无法配置网络带宽限制。 您可能会使网络过载。 +4. 尝试跨服务器分发数据,以便您不需要定期使用GLOBAL IN。 +5. 如果您需要经常使用GLOBAL IN,请规划ClickHouse集群的位置,以便单个副本组驻留在不超过一个数据中心中,并且它们之间具有快速网络,以便可以完全在单个数据中心内处理查询。 + +这也是有意义的,在指定一个本地表 `GLOBAL IN` 子句,以防此本地表仅在请求者服务器上可用,并且您希望在远程服务器上使用来自它的数据。 diff --git a/docs/zh/sql-reference/statements/alter.md b/docs/zh/sql-reference/statements/alter.md index c2176d9115b..26b5e66cc8a 100644 --- a/docs/zh/sql-reference/statements/alter.md +++ b/docs/zh/sql-reference/statements/alter.md @@ -113,7 +113,7 @@ MODIFY COLUMN [IF EXISTS] name [type] [default_expr] [TTL] - TTL -有关修改列TTL的示例,请参见 [Column TTL](../engines/table_engines/mergetree_family/mergetree.md#mergetree-column-ttl). +有关修改列TTL的示例,请参见 [Column TTL](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-column-ttl). 如果语句中包含 `IF EXISTS` ,遇到不存在的列,sql执行不会报错。 diff --git a/docs/zh/sql-reference/statements/misc.md b/docs/zh/sql-reference/statements/misc.md index aa10350280f..5320fbd0869 100644 --- a/docs/zh/sql-reference/statements/misc.md +++ b/docs/zh/sql-reference/statements/misc.md @@ -223,7 +223,7 @@ KILL MUTATION [ON CLUSTER cluster] [FORMAT format] ``` -尝试取消和删除 [突变](alter.md#alter-mutations) 当前正在执行。 要取消的突变选自 [`system.mutations`](../../operations/system-tables.md#system_tables-mutations) 表使用由指定的过滤器 `WHERE` 《公约》条款 `KILL` 查询。 +尝试取消和删除 [突变](alter.md#alter-mutations) 当前正在执行。 要取消的突变选自 [`system.mutations`](../../operations/system-tables/mutations.md#system_tables-mutations) 表使用由指定的过滤器 `WHERE` 《公约》条款 `KILL` 查询。 测试查询 (`TEST`)仅检查用户的权限并显示要停止的查询列表。 diff --git a/docs/zh/sql-reference/statements/select/array-join.md b/docs/zh/sql-reference/statements/select/array-join.md deleted file mode 120000 index c341801e419..00000000000 --- a/docs/zh/sql-reference/statements/select/array-join.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/array-join.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/array-join.md b/docs/zh/sql-reference/statements/select/array-join.md new file mode 100644 index 00000000000..e84682838f4 --- /dev/null +++ b/docs/zh/sql-reference/statements/select/array-join.md @@ -0,0 +1,283 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: ARRAY JOIN +--- + +# ARRAY JOIN子句 {#select-array-join-clause} + +对于包含数组列的表来说,这是一种常见的操作,用于生成一个新表,该表具有包含该初始列的每个单独数组元素的列,而其他列的值将被重复。 这是什么基本情况 `ARRAY JOIN` 子句有 + +它的名字来自这样一个事实,即它可以被视为执行 `JOIN` 具有数组或嵌套数据结构。 意图类似于 [arrayJoin](../../../sql-reference/functions/array-join.md#functions_arrayjoin) 功能,但该子句功能更广泛。 + +语法: + +``` sql +SELECT +FROM +[LEFT] ARRAY JOIN +[WHERE|PREWHERE ] +... +``` + +您只能指定一个 `ARRAY JOIN` a中的条款 `SELECT` 查询。 + +支持的类型 `ARRAY JOIN` 下面列出: + +- `ARRAY JOIN` -在基本情况下,空数组不包括在结果中 `JOIN`. +- `LEFT ARRAY JOIN` -的结果 `JOIN` 包含具有空数组的行。 空数组的值设置为数组元素类型的默认值(通常为0、空字符串或NULL)。 + +## 基本数组连接示例 {#basic-array-join-examples} + +下面的例子演示的用法 `ARRAY JOIN` 和 `LEFT ARRAY JOIN` 条款 让我们创建一个表 [阵列](../../../sql-reference/data-types/array.md) 键入column并在其中插入值: + +``` sql +CREATE TABLE arrays_test +( + s String, + arr Array(UInt8) +) ENGINE = Memory; + +INSERT INTO arrays_test +VALUES ('Hello', [1,2]), ('World', [3,4,5]), ('Goodbye', []); +``` + +``` text +┌─s───────────┬─arr─────┐ +│ Hello │ [1,2] │ +│ World │ [3,4,5] │ +│ Goodbye │ [] │ +└─────────────┴─────────┘ +``` + +下面的例子使用 `ARRAY JOIN` 条款: + +``` sql +SELECT s, arr +FROM arrays_test +ARRAY JOIN arr; +``` + +``` text +┌─s─────┬─arr─┐ +│ Hello │ 1 │ +│ Hello │ 2 │ +│ World │ 3 │ +│ World │ 4 │ +│ World │ 5 │ +└───────┴─────┘ +``` + +下一个示例使用 `LEFT ARRAY JOIN` 条款: + +``` sql +SELECT s, arr +FROM arrays_test +LEFT ARRAY JOIN arr; +``` + +``` text +┌─s───────────┬─arr─┐ +│ Hello │ 1 │ +│ Hello │ 2 │ +│ World │ 3 │ +│ World │ 4 │ +│ World │ 5 │ +│ Goodbye │ 0 │ +└─────────────┴─────┘ +``` + +## 使用别名 {#using-aliases} + +可以为数组中的别名指定 `ARRAY JOIN` 条款 在这种情况下,数组项目可以通过此别名访问,但数组本身可以通过原始名称访问。 示例: + +``` sql +SELECT s, arr, a +FROM arrays_test +ARRAY JOIN arr AS a; +``` + +``` text +┌─s─────┬─arr─────┬─a─┐ +│ Hello │ [1,2] │ 1 │ +│ Hello │ [1,2] │ 2 │ +│ World │ [3,4,5] │ 3 │ +│ World │ [3,4,5] │ 4 │ +│ World │ [3,4,5] │ 5 │ +└───────┴─────────┴───┘ +``` + +使用别名,您可以执行 `ARRAY JOIN` 与外部阵列。 例如: + +``` sql +SELECT s, arr_external +FROM arrays_test +ARRAY JOIN [1, 2, 3] AS arr_external; +``` + +``` text +┌─s───────────┬─arr_external─┐ +│ Hello │ 1 │ +│ Hello │ 2 │ +│ Hello │ 3 │ +│ World │ 1 │ +│ World │ 2 │ +│ World │ 3 │ +│ Goodbye │ 1 │ +│ Goodbye │ 2 │ +│ Goodbye │ 3 │ +└─────────────┴──────────────┘ +``` + +多个数组可以在逗号分隔 `ARRAY JOIN` 条款 在这种情况下, `JOIN` 与它们同时执行(直接和,而不是笛卡尔积)。 请注意,所有数组必须具有相同的大小。 示例: + +``` sql +SELECT s, arr, a, num, mapped +FROM arrays_test +ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(x -> x + 1, arr) AS mapped; +``` + +``` text +┌─s─────┬─arr─────┬─a─┬─num─┬─mapped─┐ +│ Hello │ [1,2] │ 1 │ 1 │ 2 │ +│ Hello │ [1,2] │ 2 │ 2 │ 3 │ +│ World │ [3,4,5] │ 3 │ 1 │ 4 │ +│ World │ [3,4,5] │ 4 │ 2 │ 5 │ +│ World │ [3,4,5] │ 5 │ 3 │ 6 │ +└───────┴─────────┴───┴─────┴────────┘ +``` + +下面的例子使用 [arrayEnumerate](../../../sql-reference/functions/array-functions.md#array_functions-arrayenumerate) 功能: + +``` sql +SELECT s, arr, a, num, arrayEnumerate(arr) +FROM arrays_test +ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num; +``` + +``` text +┌─s─────┬─arr─────┬─a─┬─num─┬─arrayEnumerate(arr)─┐ +│ Hello │ [1,2] │ 1 │ 1 │ [1,2] │ +│ Hello │ [1,2] │ 2 │ 2 │ [1,2] │ +│ World │ [3,4,5] │ 3 │ 1 │ [1,2,3] │ +│ World │ [3,4,5] │ 4 │ 2 │ [1,2,3] │ +│ World │ [3,4,5] │ 5 │ 3 │ [1,2,3] │ +└───────┴─────────┴───┴─────┴─────────────────────┘ +``` + +## 具有嵌套数据结构的数组连接 {#array-join-with-nested-data-structure} + +`ARRAY JOIN` 也适用于 [嵌套数据结构](../../../sql-reference/data-types/nested-data-structures/nested.md): + +``` sql +CREATE TABLE nested_test +( + s String, + nest Nested( + x UInt8, + y UInt32) +) ENGINE = Memory; + +INSERT INTO nested_test +VALUES ('Hello', [1,2], [10,20]), ('World', [3,4,5], [30,40,50]), ('Goodbye', [], []); +``` + +``` text +┌─s───────┬─nest.x──┬─nest.y─────┐ +│ Hello │ [1,2] │ [10,20] │ +│ World │ [3,4,5] │ [30,40,50] │ +│ Goodbye │ [] │ [] │ +└─────────┴─────────┴────────────┘ +``` + +``` sql +SELECT s, `nest.x`, `nest.y` +FROM nested_test +ARRAY JOIN nest; +``` + +``` text +┌─s─────┬─nest.x─┬─nest.y─┐ +│ Hello │ 1 │ 10 │ +│ Hello │ 2 │ 20 │ +│ World │ 3 │ 30 │ +│ World │ 4 │ 40 │ +│ World │ 5 │ 50 │ +└───────┴────────┴────────┘ +``` + +当指定嵌套数据结构的名称 `ARRAY JOIN`,意思是一样的 `ARRAY JOIN` 它包含的所有数组元素。 下面列出了示例: + +``` sql +SELECT s, `nest.x`, `nest.y` +FROM nested_test +ARRAY JOIN `nest.x`, `nest.y`; +``` + +``` text +┌─s─────┬─nest.x─┬─nest.y─┐ +│ Hello │ 1 │ 10 │ +│ Hello │ 2 │ 20 │ +│ World │ 3 │ 30 │ +│ World │ 4 │ 40 │ +│ World │ 5 │ 50 │ +└───────┴────────┴────────┘ +``` + +这种变化也是有道理的: + +``` sql +SELECT s, `nest.x`, `nest.y` +FROM nested_test +ARRAY JOIN `nest.x`; +``` + +``` text +┌─s─────┬─nest.x─┬─nest.y─────┐ +│ Hello │ 1 │ [10,20] │ +│ Hello │ 2 │ [10,20] │ +│ World │ 3 │ [30,40,50] │ +│ World │ 4 │ [30,40,50] │ +│ World │ 5 │ [30,40,50] │ +└───────┴────────┴────────────┘ +``` + +可以将别名用于嵌套数据结构,以便选择 `JOIN` 结果或源数组。 示例: + +``` sql +SELECT s, `n.x`, `n.y`, `nest.x`, `nest.y` +FROM nested_test +ARRAY JOIN nest AS n; +``` + +``` text +┌─s─────┬─n.x─┬─n.y─┬─nest.x──┬─nest.y─────┐ +│ Hello │ 1 │ 10 │ [1,2] │ [10,20] │ +│ Hello │ 2 │ 20 │ [1,2] │ [10,20] │ +│ World │ 3 │ 30 │ [3,4,5] │ [30,40,50] │ +│ World │ 4 │ 40 │ [3,4,5] │ [30,40,50] │ +│ World │ 5 │ 50 │ [3,4,5] │ [30,40,50] │ +└───────┴─────┴─────┴─────────┴────────────┘ +``` + +使用的例子 [arrayEnumerate](../../../sql-reference/functions/array-functions.md#array_functions-arrayenumerate) 功能: + +``` sql +SELECT s, `n.x`, `n.y`, `nest.x`, `nest.y`, num +FROM nested_test +ARRAY JOIN nest AS n, arrayEnumerate(`nest.x`) AS num; +``` + +``` text +┌─s─────┬─n.x─┬─n.y─┬─nest.x──┬─nest.y─────┬─num─┐ +│ Hello │ 1 │ 10 │ [1,2] │ [10,20] │ 1 │ +│ Hello │ 2 │ 20 │ [1,2] │ [10,20] │ 2 │ +│ World │ 3 │ 30 │ [3,4,5] │ [30,40,50] │ 1 │ +│ World │ 4 │ 40 │ [3,4,5] │ [30,40,50] │ 2 │ +│ World │ 5 │ 50 │ [3,4,5] │ [30,40,50] │ 3 │ +└───────┴─────┴─────┴─────────┴────────────┴─────┘ +``` + +## 实施细节 {#implementation-details} + +运行时优化查询执行顺序 `ARRAY JOIN`. 虽然 `ARRAY JOIN` 必须始终之前指定 [WHERE](../../../sql-reference/statements/select/where.md)/[PREWHERE](../../../sql-reference/statements/select/prewhere.md) 子句中的查询,从技术上讲,它们可以以任何顺序执行,除非结果 `ARRAY JOIN` 用于过滤。 处理顺序由查询优化器控制。 diff --git a/docs/zh/sql-reference/statements/select/distinct.md b/docs/zh/sql-reference/statements/select/distinct.md deleted file mode 120000 index 59319557dc1..00000000000 --- a/docs/zh/sql-reference/statements/select/distinct.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/distinct.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/distinct.md b/docs/zh/sql-reference/statements/select/distinct.md new file mode 100644 index 00000000000..ea430e8602f --- /dev/null +++ b/docs/zh/sql-reference/statements/select/distinct.md @@ -0,0 +1,64 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: DISTINCT +--- + +# DISTINCT子句 {#select-distinct} + +如果 `SELECT DISTINCT` 如果指定,则查询结果中只保留唯一行。 因此,在结果中所有完全匹配的行集合中,只有一行将保留。 + +## 空处理 {#null-processing} + +`DISTINCT` 适用于 [NULL](../../../sql-reference/syntax.md#null-literal) 就好像 `NULL` 是一个特定的值,并且 `NULL==NULL`. 换句话说,在 `DISTINCT` 结果,不同的组合 `NULL` 仅发生一次。 它不同于 `NULL` 在大多数其他上下文中进行处理。 + +## 替代办法 {#alternatives} + +通过应用可以获得相同的结果 [GROUP BY](../../../sql-reference/statements/select/group-by.md) 在同一组值指定为 `SELECT` 子句,而不使用任何聚合函数。 但有几个区别 `GROUP BY` 方法: + +- `DISTINCT` 可以一起应用 `GROUP BY`. +- 当 [ORDER BY](../../../sql-reference/statements/select/order-by.md) 省略和 [LIMIT](../../../sql-reference/statements/select/limit.md) 定义时,查询在读取所需数量的不同行后立即停止运行。 +- 数据块在处理时输出,而无需等待整个查询完成运行。 + +## 限制 {#limitations} + +`DISTINCT` 如果不支持 `SELECT` 具有至少一个数组列。 + +## 例 {#examples} + +ClickHouse支持使用 `DISTINCT` 和 `ORDER BY` 一个查询中不同列的子句。 该 `DISTINCT` 子句之前执行 `ORDER BY` 条款 + +示例表: + +``` text +┌─a─┬─b─┐ +│ 2 │ 1 │ +│ 1 │ 2 │ +│ 3 │ 3 │ +│ 2 │ 4 │ +└───┴───┘ +``` + +当与选择数据 `SELECT DISTINCT a FROM t1 ORDER BY b ASC` 查询,我们得到以下结果: + +``` text +┌─a─┐ +│ 2 │ +│ 1 │ +│ 3 │ +└───┘ +``` + +如果我们改变排序方向 `SELECT DISTINCT a FROM t1 ORDER BY b DESC`,我们得到以下结果: + +``` text +┌─a─┐ +│ 3 │ +│ 1 │ +│ 2 │ +└───┘ +``` + +行 `2, 4` 分拣前被切割。 + +在编程查询时考虑这种实现特异性。 diff --git a/docs/zh/sql-reference/statements/select/format.md b/docs/zh/sql-reference/statements/select/format.md deleted file mode 120000 index 106b2d9ebbc..00000000000 --- a/docs/zh/sql-reference/statements/select/format.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/format.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/format.md b/docs/zh/sql-reference/statements/select/format.md new file mode 100644 index 00000000000..014aec3b72e --- /dev/null +++ b/docs/zh/sql-reference/statements/select/format.md @@ -0,0 +1,19 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: FORMAT +--- + +# 格式子句 {#format-clause} + +ClickHouse支持广泛的 [序列化格式](../../../interfaces/formats.md) 可用于查询结果等。 有多种方法可以选择以下格式 `SELECT` 输出,其中之一是指定 `FORMAT format` 在查询结束时以任何特定格式获取结果数据。 + +特定的格式可以用于方便使用,与其他系统集成或性能增益。 + +## 默认格式 {#default-format} + +如果 `FORMAT` 省略子句,使用默认格式,这取决于用于访问ClickHouse服务器的设置和接口。 为 [HTTP接口](../../../interfaces/http.md) 和 [命令行客户端](../../../interfaces/cli.md) 在批处理模式下,默认格式为 `TabSeparated`. 对于交互模式下的命令行客户端,默认格式为 `PrettyCompact` (它生成紧凑的人类可读表)。 + +## 实施细节 {#implementation-details} + +使用命令行客户端时,数据始终以内部高效格式通过网络传递 (`Native`). 客户端独立解释 `FORMAT` 查询子句并格式化数据本身(从而减轻网络和服务器的额外负载)。 diff --git a/docs/zh/sql-reference/statements/select/from.md b/docs/zh/sql-reference/statements/select/from.md deleted file mode 120000 index f8ebfe655cc..00000000000 --- a/docs/zh/sql-reference/statements/select/from.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/from.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/from.md b/docs/zh/sql-reference/statements/select/from.md new file mode 100644 index 00000000000..86ba0959e16 --- /dev/null +++ b/docs/zh/sql-reference/statements/select/from.md @@ -0,0 +1,45 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: FROM +--- + +# FROM条款 {#select-from} + +该 `FROM` 子句指定从中读取数据的源: + +- [表](../../../engines/table-engines/index.md) +- [子查询](../../../sql-reference/statements/select/index.md) {## TODO: better link ##} +- [表函数](../../../sql-reference/table-functions/index.md#table-functions) + +[JOIN](../../../sql-reference/statements/select/join.md) 和 [ARRAY JOIN](../../../sql-reference/statements/select/array-join.md) 子句也可以用来扩展的功能 `FROM` 条款 + +子查询是另一个 `SELECT` 可以在括号内指定的查询 `FROM` 条款 + +`FROM` 子句可以包含多个数据源,用逗号分隔,这相当于执行 [CROSS JOIN](../../../sql-reference/statements/select/join.md) 在他们身上 + +## 最终修饰符 {#select-from-final} + +当 `FINAL` 如果指定,ClickHouse会在返回结果之前完全合并数据,从而执行给定表引擎合并期间发生的所有数据转换。 + +它适用于从使用 [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md)-发动机系列(除了 `GraphiteMergeTree`). 还支持: + +- [复制](../../../engines/table-engines/mergetree-family/replication.md) 版本 `MergeTree` 引擎 +- [查看](../../../engines/table-engines/special/view.md), [缓冲区](../../../engines/table-engines/special/buffer.md), [分布](../../../engines/table-engines/special/distributed.md),和 [MaterializedView](../../../engines/table-engines/special/materializedview.md) 在其他引擎上运行的引擎,只要它们是在创建 `MergeTree`-发动机表。 + +### 缺点 {#drawbacks} + +使用的查询 `FINAL` 执行速度不如类似的查询那么快,因为: + +- 查询在单个线程中执行,并在查询执行期间合并数据。 +- 查询与 `FINAL` 除了读取查询中指定的列之外,还读取主键列。 + +**在大多数情况下,避免使用 `FINAL`.** 常见的方法是使用假设后台进程的不同查询 `MergeTree` 引擎还没有发生,并通过应用聚合(例如,丢弃重复项)来处理它。 {## TODO: examples ##} + +## 实施细节 {#implementation-details} + +如果 `FROM` 子句被省略,数据将从读取 `system.one` 桌子 +该 `system.one` 表只包含一行(此表满足与其他Dbms中找到的双表相同的目的)。 + +若要执行查询,将从相应的表中提取查询中列出的所有列。 外部查询不需要的任何列都将从子查询中抛出。 +如果查询未列出任何列(例如, `SELECT count() FROM t`),无论如何都会从表中提取一些列(最小的列是首选),以便计算行数。 diff --git a/docs/zh/sql-reference/statements/select/group-by.md b/docs/zh/sql-reference/statements/select/group-by.md deleted file mode 120000 index cf519ad7781..00000000000 --- a/docs/zh/sql-reference/statements/select/group-by.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/group-by.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/group-by.md b/docs/zh/sql-reference/statements/select/group-by.md new file mode 100644 index 00000000000..082fec94498 --- /dev/null +++ b/docs/zh/sql-reference/statements/select/group-by.md @@ -0,0 +1,133 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: GROUP BY +--- + +# GROUP BY子句 {#select-group-by-clause} + +`GROUP BY` 子句切换 `SELECT` 查询转换为聚合模式,其工作原理如下: + +- `GROUP BY` 子句包含表达式列表(或单个表达式,其被认为是长度为1的列表)。 这份名单充当 “grouping key”,而每个单独的表达式将被称为 “key expressions”. +- 在所有的表达式 [SELECT](../../../sql-reference/statements/select/index.md), [HAVING](../../../sql-reference/statements/select/having.md),和 [ORDER BY](../../../sql-reference/statements/select/order-by.md) 条款 **必须** 基于键表达式进行计算 **或** 上 [聚合函数](../../../sql-reference/aggregate-functions/index.md) 在非键表达式(包括纯列)上。 换句话说,从表中选择的每个列必须用于键表达式或聚合函数内,但不能同时使用。 +- 聚合结果 `SELECT` 查询将包含尽可能多的行,因为有唯一值 “grouping key” 在源表中。 通常这会显着减少行数,通常是数量级,但不一定:如果所有行数保持不变 “grouping key” 值是不同的。 + +!!! note "注" + 还有一种额外的方法可以在表上运行聚合。 如果查询仅在聚合函数中包含表列,则 `GROUP BY clause` 可以省略,并且通过一个空的键集合来假定聚合。 这样的查询总是只返回一行。 + +## 空处理 {#null-processing} + +对于分组,ClickHouse解释 [NULL](../../../sql-reference/syntax.md#null-literal) 作为一个值,并且 `NULL==NULL`. 它不同于 `NULL` 在大多数其他上下文中进行处理。 + +这里有一个例子来说明这意味着什么。 + +假设你有这张桌子: + +``` text +┌─x─┬────y─┐ +│ 1 │ 2 │ +│ 2 │ ᴺᵁᴸᴸ │ +│ 3 │ 2 │ +│ 3 │ 3 │ +│ 3 │ ᴺᵁᴸᴸ │ +└───┴──────┘ +``` + +查询 `SELECT sum(x), y FROM t_null_big GROUP BY y` 结果: + +``` text +┌─sum(x)─┬────y─┐ +│ 4 │ 2 │ +│ 3 │ 3 │ +│ 5 │ ᴺᵁᴸᴸ │ +└────────┴──────┘ +``` + +你可以看到 `GROUP BY` 为 `y = NULL` 总结 `x`,仿佛 `NULL` 是这个值。 + +如果你通过几个键 `GROUP BY`,结果会给你选择的所有组合,就好像 `NULL` 是一个特定的值。 + +## 使用总计修饰符 {#with-totals-modifier} + +如果 `WITH TOTALS` 指定修饰符,将计算另一行。 此行将具有包含默认值(零或空行)的关键列,以及包含跨所有行计算值的聚合函数列( “total” 值)。 + +这个额外的行仅产生于 `JSON*`, `TabSeparated*`,和 `Pretty*` 格式,与其他行分开: + +- 在 `JSON*` 格式,这一行是作为一个单独的输出 ‘totals’ 场。 +- 在 `TabSeparated*` 格式,该行位于主结果之后,前面有一个空行(在其他数据之后)。 +- 在 `Pretty*` 格式时,该行在主结果之后作为单独的表输出。 +- 在其他格式中,它不可用。 + +`WITH TOTALS` 可以以不同的方式运行时 [HAVING](../../../sql-reference/statements/select/having.md) 是存在的。 该行为取决于 `totals_mode` 设置。 + +### 配置合计处理 {#configuring-totals-processing} + +默认情况下, `totals_mode = 'before_having'`. 在这种情况下, ‘totals’ 是跨所有行计算,包括那些不通过具有和 `max_rows_to_group_by`. + +其他替代方案仅包括通过具有在 ‘totals’,并与设置不同的行为 `max_rows_to_group_by` 和 `group_by_overflow_mode = 'any'`. + +`after_having_exclusive` – Don't include rows that didn't pass through `max_rows_to_group_by`. 换句话说, ‘totals’ 将有少于或相同数量的行,因为它会 `max_rows_to_group_by` 被省略。 + +`after_having_inclusive` – Include all the rows that didn't pass through ‘max\_rows\_to\_group\_by’ 在 ‘totals’. 换句话说, ‘totals’ 将有多个或相同数量的行,因为它会 `max_rows_to_group_by` 被省略。 + +`after_having_auto` – Count the number of rows that passed through HAVING. If it is more than a certain amount (by default, 50%), include all the rows that didn't pass through ‘max\_rows\_to\_group\_by’ 在 ‘totals’. 否则,不包括它们。 + +`totals_auto_threshold` – By default, 0.5. The coefficient for `after_having_auto`. + +如果 `max_rows_to_group_by` 和 `group_by_overflow_mode = 'any'` 不使用,所有的变化 `after_having` 是相同的,你可以使用它们中的任何一个(例如, `after_having_auto`). + +您可以使用 `WITH TOTALS` 在子查询中,包括在子查询 [JOIN](../../../sql-reference/statements/select/join.md) 子句(在这种情况下,将各自的总值合并)。 + +## 例 {#examples} + +示例: + +``` sql +SELECT + count(), + median(FetchTiming > 60 ? 60 : FetchTiming), + count() - sum(Refresh) +FROM hits +``` + +但是,与标准SQL相比,如果表没有任何行(根本没有任何行,或者在使用WHERE to filter之后没有任何行),则返回一个空结果,而不是来自包含聚合函数初始值的行之 + +相对于MySQL(并且符合标准SQL),您无法获取不在键或聚合函数(常量表达式除外)中的某些列的某些值。 要解决此问题,您可以使用 ‘any’ 聚合函数(获取第一个遇到的值)或 ‘min/max’. + +示例: + +``` sql +SELECT + domainWithoutWWW(URL) AS domain, + count(), + any(Title) AS title -- getting the first occurred page header for each domain. +FROM hits +GROUP BY domain +``` + +对于遇到的每个不同的键值, `GROUP BY` 计算一组聚合函数值。 + +`GROUP BY` 不支持数组列。 + +不能将常量指定为聚合函数的参数。 示例: `sum(1)`. 相反,你可以摆脱常数。 示例: `count()`. + +## 实施细节 {#implementation-details} + +聚合是面向列的DBMS最重要的功能之一,因此它的实现是ClickHouse中最优化的部分之一。 默认情况下,聚合使用哈希表在内存中完成。 它有40+的专业化是自动选择取决于 “grouping key” 数据类型。 + +### 在外部存储器中分组 {#select-group-by-in-external-memory} + +您可以启用将临时数据转储到磁盘以限制内存使用期间 `GROUP BY`. +该 [max\_bytes\_before\_external\_group\_by](../../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) 设置确定倾销的阈值RAM消耗 `GROUP BY` 临时数据到文件系统。 如果设置为0(默认值),它将被禁用。 + +使用时 `max_bytes_before_external_group_by`,我们建议您设置 `max_memory_usage` 大约两倍高。 这是必要的,因为聚合有两个阶段:读取数据和形成中间数据(1)和合并中间数据(2)。 将数据转储到文件系统只能在阶段1中发生。 如果未转储临时数据,则阶段2可能需要与阶段1相同的内存量。 + +例如,如果 [max\_memory\_usage](../../../operations/settings/settings.md#settings_max_memory_usage) 设置为10000000000,你想使用外部聚合,这是有意义的设置 `max_bytes_before_external_group_by` 到10000000000,和 `max_memory_usage` 到200亿。 当触发外部聚合(如果至少有一个临时数据转储)时,RAM的最大消耗仅略高于 `max_bytes_before_external_group_by`. + +通过分布式查询处理,在远程服务器上执行外部聚合。 为了使请求者服务器只使用少量的RAM,设置 `distributed_aggregation_memory_efficient` 到1。 + +当合并数据刷新到磁盘时,以及当合并来自远程服务器的结果时, `distributed_aggregation_memory_efficient` 设置被启用,消耗高达 `1/256 * the_number_of_threads` 从RAM的总量。 + +当启用外部聚合时,如果有小于 `max_bytes_before_external_group_by` of data (i.e. data was not flushed), the query runs just as fast as without external aggregation. If any temporary data was flushed, the run time will be several times longer (approximately three times). + +如果你有一个 [ORDER BY](../../../sql-reference/statements/select/order-by.md) 用一个 [LIMIT](../../../sql-reference/statements/select/limit.md) 后 `GROUP BY`,然后使用的RAM的量取决于数据的量 `LIMIT`,不是在整个表。 但如果 `ORDER BY` 没有 `LIMIT`,不要忘记启用外部排序 (`max_bytes_before_external_sort`). diff --git a/docs/zh/sql-reference/statements/select/having.md b/docs/zh/sql-reference/statements/select/having.md deleted file mode 120000 index 4a038beb126..00000000000 --- a/docs/zh/sql-reference/statements/select/having.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/having.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/having.md b/docs/zh/sql-reference/statements/select/having.md new file mode 100644 index 00000000000..d5c5b96a280 --- /dev/null +++ b/docs/zh/sql-reference/statements/select/having.md @@ -0,0 +1,15 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: HAVING +--- + +# 有条款 {#having-clause} + +允许过滤由以下方式生成的聚合结果 [GROUP BY](../../../sql-reference/statements/select/group-by.md). 它类似于 [WHERE](../../../sql-reference/statements/select/where.md) 条款,但不同的是 `WHERE` 在聚合之前执行,而 `HAVING` 之后进行。 + +可以从以下引用聚合结果 `SELECT` 中的条款 `HAVING` 子句由他们的化名。 或者, `HAVING` 子句可以筛选查询结果中未返回的其他聚合的结果。 + +## 限制 {#limitations} + +`HAVING` 如果不执行聚合,则无法使用。 使用 `WHERE` 相反。 diff --git a/docs/zh/sql-reference/statements/select/index.md b/docs/zh/sql-reference/statements/select/index.md deleted file mode 120000 index 9c649322c82..00000000000 --- a/docs/zh/sql-reference/statements/select/index.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/index.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/index.md b/docs/zh/sql-reference/statements/select/index.md new file mode 100644 index 00000000000..58850b91a02 --- /dev/null +++ b/docs/zh/sql-reference/statements/select/index.md @@ -0,0 +1,164 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +title: SELECT Query +toc_folder_title: SELECT +toc_priority: 33 +toc_title: "\u6982\u8FF0" +--- + +# 选择查询 {#select-queries-syntax} + +`SELECT` 查询执行数据检索。 默认情况下,请求的数据返回给客户端,同时与 [INSERT INTO](../../../sql-reference/statements/insert-into.md) 它可以被转发到不同的表。 + +## 语法 {#syntax} + +``` sql +[WITH expr_list|(subquery)] +SELECT [DISTINCT] expr_list +[FROM [db.]table | (subquery) | table_function] [FINAL] +[SAMPLE sample_coeff] +[ARRAY JOIN ...] +[GLOBAL] [ANY|ALL] [INNER|LEFT|RIGHT|FULL|CROSS] [OUTER] JOIN (subquery)|table USING columns_list +[PREWHERE expr] +[WHERE expr] +[GROUP BY expr_list] [WITH TOTALS] +[HAVING expr] +[ORDER BY expr_list] +[LIMIT [offset_value, ]n BY columns] +[LIMIT [n, ]m] +[UNION ALL ...] +[INTO OUTFILE filename] +[FORMAT format] +``` + +所有子句都是可选的,但紧接在后面的必需表达式列表除外 `SELECT` 这是更详细的复盖 [下面](#select-clause). + +每个可选子句的具体内容在单独的部分中进行了介绍,这些部分按与执行顺序相同的顺序列出: + +- [WITH条款](../../../sql-reference/statements/select/with.md) +- [FROM条款](../../../sql-reference/statements/select/from.md) +- [示例子句](../../../sql-reference/statements/select/sample.md) +- [JOIN子句](../../../sql-reference/statements/select/join.md) +- [PREWHERE条款](../../../sql-reference/statements/select/prewhere.md) +- [WHERE条款](../../../sql-reference/statements/select/where.md) +- [GROUP BY子句](../../../sql-reference/statements/select/group-by.md) +- [限制条款](../../../sql-reference/statements/select/limit-by.md) +- [有条款](../../../sql-reference/statements/select/having.md) +- [SELECT子句](#select-clause) +- [DISTINCT子句](../../../sql-reference/statements/select/distinct.md) +- [限制条款](../../../sql-reference/statements/select/limit.md) +- [UNION ALL条款](../../../sql-reference/statements/select/union-all.md) +- [INTO OUTFILE条款](../../../sql-reference/statements/select/into-outfile.md) +- [格式子句](../../../sql-reference/statements/select/format.md) + +## SELECT子句 {#select-clause} + +[表达式](../../../sql-reference/syntax.md#syntax-expressions) 在指定 `SELECT` 子句是在上述子句中的所有操作完成后计算的。 这些表达式的工作方式就好像它们应用于结果中的单独行一样。 如果在表达式 `SELECT` 子句包含聚合函数,然后ClickHouse处理过程中用作其参数的聚合函数和表达式 [GROUP BY](../../../sql-reference/statements/select/group-by.md) 聚合。 + +如果要在结果中包含所有列,请使用星号 (`*`)符号。 例如, `SELECT * FROM ...`. + +将结果中的某些列与 [re2](https://en.wikipedia.org/wiki/RE2_(software)) 正则表达式,您可以使用 `COLUMNS` 表达。 + +``` sql +COLUMNS('regexp') +``` + +例如,考虑表: + +``` sql +CREATE TABLE default.col_names (aa Int8, ab Int8, bc Int8) ENGINE = TinyLog +``` + +以下查询从包含以下内容的所有列中选择数据 `a` 在他们的名字符号。 + +``` sql +SELECT COLUMNS('a') FROM col_names +``` + +``` text +┌─aa─┬─ab─┐ +│ 1 │ 1 │ +└────┴────┘ +``` + +所选列不按字母顺序返回。 + +您可以使用多个 `COLUMNS` 查询中的表达式并将函数应用于它们。 + +例如: + +``` sql +SELECT COLUMNS('a'), COLUMNS('c'), toTypeName(COLUMNS('c')) FROM col_names +``` + +``` text +┌─aa─┬─ab─┬─bc─┬─toTypeName(bc)─┐ +│ 1 │ 1 │ 1 │ Int8 │ +└────┴────┴────┴────────────────┘ +``` + +由返回的每一列 `COLUMNS` 表达式作为单独的参数传递给函数。 如果函数支持其他参数,您也可以将其他参数传递给函数。 使用函数时要小心。 如果函数不支持您传递给它的参数数,ClickHouse将引发异常。 + +例如: + +``` sql +SELECT COLUMNS('a') + COLUMNS('c') FROM col_names +``` + +``` text +Received exception from server (version 19.14.1): +Code: 42. DB::Exception: Received from localhost:9000. DB::Exception: Number of arguments for function plus doesn't match: passed 3, should be 2. +``` + +在这个例子中, `COLUMNS('a')` 返回两列: `aa` 和 `ab`. `COLUMNS('c')` 返回 `bc` 列。 该 `+` 运算符不能应用于3个参数,因此ClickHouse引发一个带有相关消息的异常。 + +匹配的列 `COLUMNS` 表达式可以具有不同的数据类型。 如果 `COLUMNS` 不匹配任何列,并且是唯一的表达式 `SELECT`,ClickHouse抛出异常。 + +### 星号 {#asterisk} + +您可以在查询的任何部分而不是表达式中添加星号。 分析查询时,星号将展开为所有表列的列表(不包括 `MATERIALIZED` 和 `ALIAS` 列)。 只有少数情况下使用星号是合理的: + +- 创建表转储时。 +- 对于只包含几列的表,例如系统表。 +- 获取有关表中哪些列的信息。 在这种情况下,设置 `LIMIT 1`. 但最好使用 `DESC TABLE` 查询。 +- 当对少量柱进行强过滤时,使用 `PREWHERE`. +- 在子查询中(因为外部查询不需要的列从子查询中排除)。 + +在所有其他情况下,我们不建议使用星号,因为它只给你一个列DBMS的缺点,而不是优点。 换句话说,不建议使用星号。 + +### 极端值 {#extreme-values} + +除了结果之外,还可以获取结果列的最小值和最大值。 要做到这一点,设置 **极端** 设置为1。 最小值和最大值是针对数字类型、日期和带有时间的日期计算的。 对于其他列,默认值为输出。 + +An extra two rows are calculated – the minimums and maximums, respectively. These extra two rows are output in `JSON*`, `TabSeparated*`,和 `Pretty*` [格式](../../../interfaces/formats.md),与其他行分开。 它们不是其他格式的输出。 + +在 `JSON*` 格式时,极端值在一个单独的输出 ‘extremes’ 场。 在 `TabSeparated*` 格式中,该行来的主要结果之后,和之后 ‘totals’ 如果存在。 它前面有一个空行(在其他数据之后)。 在 `Pretty*` 格式中,该行被输出为一个单独的表之后的主结果,和之后 `totals` 如果存在。 + +极值计算之前的行 `LIMIT`,但之后 `LIMIT BY`. 但是,使用时 `LIMIT offset, size`,之前的行 `offset` 都包含在 `extremes`. 在流请求中,结果还可能包括少量通过的行 `LIMIT`. + +### 注 {#notes} + +您可以使用同义词 (`AS` 别名)在查询的任何部分。 + +该 `GROUP BY` 和 `ORDER BY` 子句不支持位置参数。 这与MySQL相矛盾,但符合标准SQL。 例如, `GROUP BY 1, 2` will be interpreted as grouping by constants (i.e. aggregation of all rows into one). + +## 实施细节 {#implementation-details} + +如果查询省略 `DISTINCT`, `GROUP BY` 和 `ORDER BY` 条款和 `IN` 和 `JOIN` 子查询,查询将被完全流处理,使用O(1)量的RAM。 否则,如果未指定适当的限制,则查询可能会消耗大量RAM: + +- `max_memory_usage` +- `max_rows_to_group_by` +- `max_rows_to_sort` +- `max_rows_in_distinct` +- `max_bytes_in_distinct` +- `max_rows_in_set` +- `max_bytes_in_set` +- `max_rows_in_join` +- `max_bytes_in_join` +- `max_bytes_before_external_sort` +- `max_bytes_before_external_group_by` + +有关详细信息,请参阅部分 “Settings”. 可以使用外部排序(将临时表保存到磁盘)和外部聚合。 + +{## [原始文章](https://clickhouse.tech/docs/en/sql-reference/statements/select/) ##} diff --git a/docs/zh/sql-reference/statements/select/into-outfile.md b/docs/zh/sql-reference/statements/select/into-outfile.md deleted file mode 120000 index 2c9c812b3d5..00000000000 --- a/docs/zh/sql-reference/statements/select/into-outfile.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/into-outfile.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/into-outfile.md b/docs/zh/sql-reference/statements/select/into-outfile.md new file mode 100644 index 00000000000..f1eb3e55b89 --- /dev/null +++ b/docs/zh/sql-reference/statements/select/into-outfile.md @@ -0,0 +1,15 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: INTO OUTFILE +--- + +# INTO OUTFILE条款 {#into-outfile-clause} + +添加 `INTO OUTFILE filename` 子句(其中filename是字符串文字) `SELECT query` 将其输出重定向到客户端上的指定文件。 + +## 实施细节 {#implementation-details} + +- 此功能是在可用 [命令行客户端](../../../interfaces/cli.md) 和 [ツ环板-ョツ嘉ッツ偲](../../../operations/utilities/clickhouse-local.md). 因此,通过发送查询 [HTTP接口](../../../interfaces/http.md) 都会失败 +- 如果具有相同文件名的文件已经存在,则查询将失败。 +- 默认值 [输出格式](../../../interfaces/formats.md) 是 `TabSeparated` (就像在命令行客户端批处理模式中一样)。 diff --git a/docs/zh/sql-reference/statements/select/join.md b/docs/zh/sql-reference/statements/select/join.md deleted file mode 120000 index 5951a105137..00000000000 --- a/docs/zh/sql-reference/statements/select/join.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/join.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/join.md b/docs/zh/sql-reference/statements/select/join.md new file mode 100644 index 00000000000..47fd0137717 --- /dev/null +++ b/docs/zh/sql-reference/statements/select/join.md @@ -0,0 +1,198 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: JOIN +--- + +# JOIN子句 {#select-join} + +Join通过使用一个或多个表的公共值合并来自一个或多个表的列来生成新表。 它是支持SQL的数据库中的常见操作,它对应于 [关系代数](https://en.wikipedia.org/wiki/Relational_algebra#Joins_and_join-like_operators) 加入。 一个表连接的特殊情况通常被称为 “self-join”. + +语法: + +``` sql +SELECT +FROM +[GLOBAL] [ANY|ALL|ASOF] [INNER|LEFT|RIGHT|FULL|CROSS] [OUTER|SEMI|ANTI] JOIN +(ON )|(USING ) ... +``` + +从表达式 `ON` 从子句和列 `USING` 子句被称为 “join keys”. 除非另有说明,加入产生一个 [笛卡尔积](https://en.wikipedia.org/wiki/Cartesian_product) 从具有匹配的行 “join keys”,这可能会产生比源表更多的行的结果。 + +## 支持的联接类型 {#select-join-types} + +所有标准 [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) 支持类型: + +- `INNER JOIN`,只返回匹配的行。 +- `LEFT OUTER JOIN`,除了匹配的行之外,还返回左表中的非匹配行。 +- `RIGHT OUTER JOIN`,除了匹配的行之外,还返回右表中的非匹配行。 +- `FULL OUTER JOIN`,除了匹配的行之外,还会返回两个表中的非匹配行。 +- `CROSS JOIN`,产生整个表的笛卡尔积, “join keys” 是 **不** 指定。 + +`JOIN` 没有指定类型暗示 `INNER`. 关键字 `OUTER` 可以安全地省略。 替代语法 `CROSS JOIN` 在指定多个表 [FROM条款](../../../sql-reference/statements/select/from.md) 用逗号分隔。 + +ClickHouse中提供的其他联接类型: + +- `LEFT SEMI JOIN` 和 `RIGHT SEMI JOIN`,白名单 “join keys”,而不产生笛卡尔积。 +- `LEFT ANTI JOIN` 和 `RIGHT ANTI JOIN`,黑名单 “join keys”,而不产生笛卡尔积。 + +## 严格 {#select-join-strictness} + +修改如何匹配 “join keys” 执行 + +- `ALL` — The standard `JOIN` sql中的行为如上所述。 默认值。 +- `ANY` — Partially (for opposite side of `LEFT` 和 `RIGHT`)或完全(为 `INNER` 和 `FULL`)禁用笛卡尔积为标准 `JOIN` 类型。 +- `ASOF` — For joining sequences with a non-exact match. `ASOF JOIN` 用法描述如下。 + +!!! note "注" + 可以使用以下方式复盖默认的严格性值 [join\_default\_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) 设置。 + + Also the behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) setting. + +### ASOF加入使用 {#asof-join-usage} + +`ASOF JOIN` 当您需要连接没有完全匹配的记录时非常有用。 + +算法需要表中的特殊列。 本专栏: + +- 必须包含有序序列。 +- 可以是以下类型之一: [Int*,UInt*](../../../sql-reference/data-types/int-uint.md), [浮动\*](../../../sql-reference/data-types/float.md), [日期](../../../sql-reference/data-types/date.md), [日期时间](../../../sql-reference/data-types/datetime.md), [十进制\*](../../../sql-reference/data-types/decimal.md). +- 不能是唯一的列 `JOIN` 条款 + +语法 `ASOF JOIN ... ON`: + +``` sql +SELECT expressions_list +FROM table_1 +ASOF LEFT JOIN table_2 +ON equi_cond AND closest_match_cond +``` + +您可以使用任意数量的相等条件和恰好一个最接近的匹配条件。 例如, `SELECT count() FROM table_1 ASOF LEFT JOIN table_2 ON table_1.a == table_2.b AND table_2.t <= table_1.t`. + +支持最接近匹配的条件: `>`, `>=`, `<`, `<=`. + +语法 `ASOF JOIN ... USING`: + +``` sql +SELECT expressions_list +FROM table_1 +ASOF JOIN table_2 +USING (equi_column1, ... equi_columnN, asof_column) +``` + +`ASOF JOIN` 用途 `equi_columnX` 对于加入平等和 `asof_column` 用于加入与最接近的比赛 `table_1.asof_column >= table_2.asof_column` 条件。 该 `asof_column` 列总是在最后一个 `USING` 条款 + +例如,请考虑下表: + + table_1 table_2 + event | ev_time | user_id event | ev_time | user_id + ----------|---------|---------- ----------|---------|---------- + ... ... + event_1_1 | 12:00 | 42 event_2_1 | 11:59 | 42 + ... event_2_2 | 12:30 | 42 + event_1_2 | 13:00 | 42 event_2_3 | 13:00 | 42 + ... ... + +`ASOF JOIN` 可以从用户事件的时间戳 `table_1` 并找到一个事件 `table_2` 其中时间戳最接近事件的时间戳 `table_1` 对应于最接近的匹配条件。 如果可用,则相等的时间戳值是最接近的值。 在这里,该 `user_id` 列可用于连接相等和 `ev_time` 列可用于在最接近的匹配加入。 在我们的例子中, `event_1_1` 可以加入 `event_2_1` 和 `event_1_2` 可以加入 `event_2_3`,但是 `event_2_2` 不能加入。 + +!!! note "注" + `ASOF` 加入是 **不** 支持在 [加入我们](../../../engines/table-engines/special/join.md) 表引擎。 + +## 分布式联接 {#global-join} + +有两种方法可以执行涉及分布式表的join: + +- 当使用正常 `JOIN`,将查询发送到远程服务器。 为了创建正确的表,在每个子查询上运行子查询,并使用此表执行联接。 换句话说,在每个服务器上单独形成右表。 +- 使用时 `GLOBAL ... JOIN`,首先请求者服务器运行一个子查询来计算正确的表。 此临时表将传递到每个远程服务器,并使用传输的临时数据对其运行查询。 + +使用时要小心 `GLOBAL`. 有关详细信息,请参阅 [分布式子查询](../../../sql-reference/operators/in.md#select-distributed-subqueries) 科。 + +## 使用建议 {#usage-recommendations} + +### 处理空单元格或空单元格 {#processing-of-empty-or-null-cells} + +在连接表时,可能会出现空单元格。 设置 [join\_use\_nulls](../../../operations/settings/settings.md#join_use_nulls) 定义ClickHouse如何填充这些单元格。 + +如果 `JOIN` 键是 [可为空](../../../sql-reference/data-types/nullable.md) 字段,其中至少有一个键具有值的行 [NULL](../../../sql-reference/syntax.md#null-literal) 没有加入。 + +### 语法 {#syntax} + +在指定的列 `USING` 两个子查询中必须具有相同的名称,并且其他列必须以不同的方式命名。 您可以使用别名更改子查询中的列名。 + +该 `USING` 子句指定一个或多个要联接的列,这将建立这些列的相等性。 列的列表设置不带括号。 不支持更复杂的连接条件。 + +### 语法限制 {#syntax-limitations} + +对于多个 `JOIN` 单个子句 `SELECT` 查询: + +- 通过以所有列 `*` 仅在联接表时才可用,而不是子查询。 +- 该 `PREWHERE` 条款不可用。 + +为 `ON`, `WHERE`,和 `GROUP BY` 条款: + +- 任意表达式不能用于 `ON`, `WHERE`,和 `GROUP BY` 子句,但你可以定义一个表达式 `SELECT` 子句,然后通过别名在这些子句中使用它。 + +### 性能 {#performance} + +当运行 `JOIN`,与查询的其他阶段相关的执行顺序没有优化。 连接(在右表中搜索)在过滤之前运行 `WHERE` 和聚集之前。 + +每次使用相同的查询运行 `JOIN`,子查询再次运行,因为结果未缓存。 为了避免这种情况,使用特殊的 [加入我们](../../../engines/table-engines/special/join.md) 表引擎,它是一个用于连接的准备好的数组,总是在RAM中。 + +在某些情况下,使用效率更高 [IN](../../../sql-reference/operators/in.md) 而不是 `JOIN`. + +如果你需要一个 `JOIN` 对于连接维度表(这些是包含维度属性的相对较小的表,例如广告活动的名称), `JOIN` 由于每个查询都会重新访问正确的表,因此可能不太方便。 对于这种情况下,有一个 “external dictionaries” 您应该使用的功能 `JOIN`. 有关详细信息,请参阅 [外部字典](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) 科。 + +### 内存限制 {#memory-limitations} + +默认情况下,ClickHouse使用 [哈希联接](https://en.wikipedia.org/wiki/Hash_join) 算法。 ClickHouse采取 `` 并在RAM中为其创建哈希表。 在某个内存消耗阈值之后,ClickHouse回退到合并联接算法。 + +如果需要限制联接操作内存消耗,请使用以下设置: + +- [max\_rows\_in\_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. +- [max\_bytes\_in\_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. + +当任何这些限制达到,ClickHouse作为 [join\_overflow\_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) 设置指示。 + +## 例 {#examples} + +示例: + +``` sql +SELECT + CounterID, + hits, + visits +FROM +( + SELECT + CounterID, + count() AS hits + FROM test.hits + GROUP BY CounterID +) ANY LEFT JOIN +( + SELECT + CounterID, + sum(Sign) AS visits + FROM test.visits + GROUP BY CounterID +) USING CounterID +ORDER BY hits DESC +LIMIT 10 +``` + +``` text +┌─CounterID─┬───hits─┬─visits─┐ +│ 1143050 │ 523264 │ 13665 │ +│ 731962 │ 475698 │ 102716 │ +│ 722545 │ 337212 │ 108187 │ +│ 722889 │ 252197 │ 10547 │ +│ 2237260 │ 196036 │ 9522 │ +│ 23057320 │ 147211 │ 7689 │ +│ 722818 │ 90109 │ 17847 │ +│ 48221 │ 85379 │ 4652 │ +│ 19762435 │ 77807 │ 7026 │ +│ 722884 │ 77492 │ 11056 │ +└───────────┴────────┴────────┘ +``` diff --git a/docs/zh/sql-reference/statements/select/limit-by.md b/docs/zh/sql-reference/statements/select/limit-by.md deleted file mode 120000 index f3a63e9fe22..00000000000 --- a/docs/zh/sql-reference/statements/select/limit-by.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/limit-by.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/limit-by.md b/docs/zh/sql-reference/statements/select/limit-by.md new file mode 100644 index 00000000000..ae2bd491817 --- /dev/null +++ b/docs/zh/sql-reference/statements/select/limit-by.md @@ -0,0 +1,72 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: LIMIT BY +--- + +# 限制条款 {#limit-by-clause} + +与查询 `LIMIT n BY expressions` 子句选择第一个 `n` 每个不同值的行 `expressions`. 的关键 `LIMIT BY` 可以包含任意数量的 [表达式](../../../sql-reference/syntax.md#syntax-expressions). + +ClickHouse支持以下语法变体: + +- `LIMIT [offset_value, ]n BY expressions` +- `LIMIT n OFFSET offset_value BY expressions` + +在查询处理过程中,ClickHouse会选择按排序键排序的数据。 排序键使用以下命令显式设置 [ORDER BY](../../../sql-reference/statements/select/order-by.md) 子句或隐式作为表引擎的属性。 然后ClickHouse应用 `LIMIT n BY expressions` 并返回第一 `n` 每个不同组合的行 `expressions`. 如果 `OFFSET` 被指定,则对于每个数据块属于一个不同的组合 `expressions`,ClickHouse跳过 `offset_value` 从块开始的行数,并返回最大值 `n` 行的结果。 如果 `offset_value` 如果数据块中的行数大于数据块中的行数,ClickHouse将从该块返回零行。 + +!!! note "注" + `LIMIT BY` 是不相关的 [LIMIT](../../../sql-reference/statements/select/limit.md). 它们都可以在同一个查询中使用。 + +## 例 {#examples} + +样品表: + +``` sql +CREATE TABLE limit_by(id Int, val Int) ENGINE = Memory; +INSERT INTO limit_by VALUES (1, 10), (1, 11), (1, 12), (2, 20), (2, 21); +``` + +查询: + +``` sql +SELECT * FROM limit_by ORDER BY id, val LIMIT 2 BY id +``` + +``` text +┌─id─┬─val─┐ +│ 1 │ 10 │ +│ 1 │ 11 │ +│ 2 │ 20 │ +│ 2 │ 21 │ +└────┴─────┘ +``` + +``` sql +SELECT * FROM limit_by ORDER BY id, val LIMIT 1, 2 BY id +``` + +``` text +┌─id─┬─val─┐ +│ 1 │ 11 │ +│ 1 │ 12 │ +│ 2 │ 21 │ +└────┴─────┘ +``` + +该 `SELECT * FROM limit_by ORDER BY id, val LIMIT 2 OFFSET 1 BY id` 查询返回相同的结果。 + +以下查询返回每个引用的前5个引用 `domain, device_type` 最多可与100行配对 (`LIMIT n BY + LIMIT`). + +``` sql +SELECT + domainWithoutWWW(URL) AS domain, + domainWithoutWWW(REFERRER_URL) AS referrer, + device_type, + count() cnt +FROM hits +GROUP BY domain, referrer, device_type +ORDER BY cnt DESC +LIMIT 5 BY domain, device_type +LIMIT 100 +``` diff --git a/docs/zh/sql-reference/statements/select/limit.md b/docs/zh/sql-reference/statements/select/limit.md deleted file mode 120000 index e0a0c632dac..00000000000 --- a/docs/zh/sql-reference/statements/select/limit.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/limit.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/limit.md b/docs/zh/sql-reference/statements/select/limit.md new file mode 100644 index 00000000000..4d02df88600 --- /dev/null +++ b/docs/zh/sql-reference/statements/select/limit.md @@ -0,0 +1,15 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: LIMIT +--- + +# 限制条款 {#limit-clause} + +`LIMIT m` 允许选择第一个 `m` 结果中的行。 + +`LIMIT n, m` 允许选择 `m` 跳过第一个结果后的行 `n` 行。 该 `LIMIT m OFFSET n` 语法是等效的。 + +`n` 和 `m` 必须是非负整数。 + +如果没有 [ORDER BY](../../../sql-reference/statements/select/order-by.md) 子句显式排序结果,结果的行选择可能是任意的和非确定性的。 diff --git a/docs/zh/sql-reference/statements/select/order-by.md b/docs/zh/sql-reference/statements/select/order-by.md deleted file mode 120000 index cc2567bce0b..00000000000 --- a/docs/zh/sql-reference/statements/select/order-by.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/order-by.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/order-by.md b/docs/zh/sql-reference/statements/select/order-by.md new file mode 100644 index 00000000000..e853a788075 --- /dev/null +++ b/docs/zh/sql-reference/statements/select/order-by.md @@ -0,0 +1,73 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: ORDER BY +--- + +# 按条款订购 {#select-order-by} + +该 `ORDER BY` 子句包含一个表达式列表,每个表达式都可以用 `DESC` (降序)或 `ASC` (升序)修饰符确定排序方向。 如果未指定方向, `ASC` 假设,所以它通常被省略。 排序方向适用于单个表达式,而不适用于整个列表。 示例: `ORDER BY Visits DESC, SearchPhrase` + +对于排序表达式列表具有相同值的行以任意顺序输出,也可以是非确定性的(每次都不同)。 +如果省略ORDER BY子句,则行的顺序也是未定义的,并且可能也是非确定性的。 + +## 特殊值的排序 {#sorting-of-special-values} + +有两种方法 `NaN` 和 `NULL` 排序顺序: + +- 默认情况下或与 `NULLS LAST` 修饰符:首先是值,然后 `NaN`,然后 `NULL`. +- 与 `NULLS FIRST` 修饰符:第一 `NULL`,然后 `NaN`,然后其他值。 + +### 示例 {#example} + +对于表 + +``` text +┌─x─┬────y─┐ +│ 1 │ ᴺᵁᴸᴸ │ +│ 2 │ 2 │ +│ 1 │ nan │ +│ 2 │ 2 │ +│ 3 │ 4 │ +│ 5 │ 6 │ +│ 6 │ nan │ +│ 7 │ ᴺᵁᴸᴸ │ +│ 6 │ 7 │ +│ 8 │ 9 │ +└───┴──────┘ +``` + +运行查询 `SELECT * FROM t_null_nan ORDER BY y NULLS FIRST` 获得: + +``` text +┌─x─┬────y─┐ +│ 1 │ ᴺᵁᴸᴸ │ +│ 7 │ ᴺᵁᴸᴸ │ +│ 1 │ nan │ +│ 6 │ nan │ +│ 2 │ 2 │ +│ 2 │ 2 │ +│ 3 │ 4 │ +│ 5 │ 6 │ +│ 6 │ 7 │ +│ 8 │ 9 │ +└───┴──────┘ +``` + +当对浮点数进行排序时,Nan与其他值是分开的。 无论排序顺序如何,Nan都在最后。 换句话说,对于升序排序,它们被放置为好像它们比所有其他数字大,而对于降序排序,它们被放置为好像它们比其他数字小。 + +## 排序规则支持 {#collation-support} + +对于按字符串值排序,可以指定排序规则(比较)。 示例: `ORDER BY SearchPhrase COLLATE 'tr'` -对于按关键字升序排序,使用土耳其字母,不区分大小写,假设字符串是UTF-8编码。 `COLLATE` 可以按顺序独立地指定或不按每个表达式。 如果 `ASC` 或 `DESC` 被指定, `COLLATE` 在它之后指定。 使用时 `COLLATE`,排序始终不区分大小写。 + +我们只建议使用 `COLLATE` 对于少量行的最终排序,因为排序与 `COLLATE` 比正常的按字节排序效率低。 + +## 实施细节 {#implementation-details} + +更少的RAM使用,如果一个足够小 [LIMIT](../../../sql-reference/statements/select/limit.md) 除了指定 `ORDER BY`. 否则,所花费的内存量与用于排序的数据量成正比。 对于分布式查询处理,如果 [GROUP BY](../../../sql-reference/statements/select/group-by.md) 省略排序,在远程服务器上部分完成排序,并将结果合并到请求者服务器上。 这意味着对于分布式排序,要排序的数据量可以大于单个服务器上的内存量。 + +如果没有足够的RAM,则可以在外部存储器中执行排序(在磁盘上创建临时文件)。 使用设置 `max_bytes_before_external_sort` 为此目的。 如果将其设置为0(默认值),则禁用外部排序。 如果启用,则当要排序的数据量达到指定的字节数时,将对收集的数据进行排序并转储到临时文件中。 读取所有数据后,将合并所有已排序的文件并输出结果。 文件被写入到 `/var/lib/clickhouse/tmp/` 目录中的配置(默认情况下,但你可以使用 `tmp_path` 参数来更改此设置)。 + +运行查询可能占用的内存比 `max_bytes_before_external_sort`. 因此,此设置的值必须大大小于 `max_memory_usage`. 例如,如果您的服务器有128GB的RAM,并且您需要运行单个查询,请设置 `max_memory_usage` 到100GB,和 `max_bytes_before_external_sort` 至80GB。 + +外部排序的工作效率远远低于在RAM中进行排序。 diff --git a/docs/zh/sql-reference/statements/select/prewhere.md b/docs/zh/sql-reference/statements/select/prewhere.md deleted file mode 120000 index 567fc95356f..00000000000 --- a/docs/zh/sql-reference/statements/select/prewhere.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/prewhere.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/prewhere.md b/docs/zh/sql-reference/statements/select/prewhere.md new file mode 100644 index 00000000000..ec6607d4ecc --- /dev/null +++ b/docs/zh/sql-reference/statements/select/prewhere.md @@ -0,0 +1,23 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: PREWHERE +--- + +# PREWHERE条款 {#prewhere-clause} + +Prewhere是更有效地应用过滤的优化。 默认情况下,即使在 `PREWHERE` 子句未显式指定。 它的工作原理是自动移动的一部分 [WHERE](../../../sql-reference/statements/select/where.md) 条件到prewhere阶段。 的作用 `PREWHERE` 子句只是控制这个优化,如果你认为你知道如何做得比默认情况下更好。 + +使用prewhere优化,首先只读取执行prewhere表达式所需的列。 然后读取运行其余查询所需的其他列,但只读取prewhere表达式所在的那些块 “true” 至少对于一些行。 如果有很多块,其中prewhere表达式是 “false” 对于所有行和prewhere需要比查询的其他部分更少的列,这通常允许从磁盘读取更少的数据以执行查询。 + +## 手动控制Prewhere {#controlling-prewhere-manually} + +该条款具有相同的含义 `WHERE` 条款 区别在于从表中读取数据。 当手动控制 `PREWHERE` 对于查询中的少数列使用的过滤条件,但这些过滤条件提供了强大的数据过滤。 这减少了要读取的数据量。 + +查询可以同时指定 `PREWHERE` 和 `WHERE`. 在这种情况下, `PREWHERE` 先于 `WHERE`. + +如果 `optimize_move_to_prewhere` 设置为0,启发式自动移动部分表达式 `WHERE` 到 `PREWHERE` 被禁用。 + +## 限制 {#limitations} + +`PREWHERE` 只有从表支持 `*MergeTree` 家人 diff --git a/docs/zh/sql-reference/statements/select/sample.md b/docs/zh/sql-reference/statements/select/sample.md deleted file mode 120000 index 9df6e25d0f3..00000000000 --- a/docs/zh/sql-reference/statements/select/sample.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/sample.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/sample.md b/docs/zh/sql-reference/statements/select/sample.md new file mode 100644 index 00000000000..9b760601959 --- /dev/null +++ b/docs/zh/sql-reference/statements/select/sample.md @@ -0,0 +1,114 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: SAMPLE +--- + +# 示例子句 {#select-sample-clause} + +该 `SAMPLE` 子句允许近似 `SELECT` 查询处理。 + +启用数据采样时,不会对所有数据执行查询,而只对特定部分数据(样本)执行查询。 例如,如果您需要计算所有访问的统计信息,只需对所有访问的1/10分数执行查询,然后将结果乘以10即可。 + +近似查询处理在以下情况下可能很有用: + +- 当你有严格的时间requirements(如\<100ms),但你不能证明额外的硬件资源来满足他们的成本。 +- 当您的原始数据不准确时,所以近似不会明显降低质量。 +- 业务需求的目标是近似结果(为了成本效益,或者向高级用户推销确切结果)。 + +!!! note "注" + 您只能使用采样中的表 [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md) 家庭,并且只有在表创建过程中指定了采样表达式(请参阅 [MergeTree引擎](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table)). + +下面列出了数据采样的功能: + +- 数据采样是一种确定性机制。 同样的结果 `SELECT .. SAMPLE` 查询始终是相同的。 +- 对于不同的表,采样工作始终如一。 对于具有单个采样键的表,具有相同系数的采样总是选择相同的可能数据子集。 例如,用户Id的示例采用来自不同表的所有可能的用户Id的相同子集的行。 这意味着您可以在子查询中使用示例 [IN](../../../sql-reference/operators/in.md) 条款 此外,您可以使用 [JOIN](../../../sql-reference/statements/select/join.md) 条款 +- 采样允许从磁盘读取更少的数据。 请注意,您必须正确指定采样键。 有关详细信息,请参阅 [创建MergeTree表](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table). + +为 `SAMPLE` 子句支持以下语法: + +| SAMPLE Clause Syntax | 产品描述 | +|----------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `SAMPLE k` | 这里 `k` 是从0到1的数字。
查询执行于 `k` 数据的分数。 例如, `SAMPLE 0.1` 对10%的数据运行查询。 [碌莽禄more拢more](#select-sample-k) | +| `SAMPLE n` | 这里 `n` 是足够大的整数。
该查询是在至少一个样本上执行的 `n` 行(但不超过这个)。 例如, `SAMPLE 10000000` 在至少10,000,000行上运行查询。 [碌莽禄more拢more](#select-sample-n) | +| `SAMPLE k OFFSET m` | 这里 `k` 和 `m` 是从0到1的数字。
查询在以下示例上执行 `k` 数据的分数。 用于采样的数据由以下偏移 `m` 分数。 [碌莽禄more拢more](#select-sample-offset) | + +## SAMPLE K {#select-sample-k} + +这里 `k` 从0到1的数字(支持小数和小数表示法)。 例如, `SAMPLE 1/2` 或 `SAMPLE 0.5`. + +在一个 `SAMPLE k` 子句,样品是从 `k` 数据的分数。 示例如下所示: + +``` sql +SELECT + Title, + count() * 10 AS PageViews +FROM hits_distributed +SAMPLE 0.1 +WHERE + CounterID = 34 +GROUP BY Title +ORDER BY PageViews DESC LIMIT 1000 +``` + +在此示例中,对0.1(10%)数据的样本执行查询。 聚合函数的值不会自动修正,因此要获得近似结果,值 `count()` 手动乘以10。 + +## SAMPLE N {#select-sample-n} + +这里 `n` 是足够大的整数。 例如, `SAMPLE 10000000`. + +在这种情况下,查询在至少一个样本上执行 `n` 行(但不超过这个)。 例如, `SAMPLE 10000000` 在至少10,000,000行上运行查询。 + +由于数据读取的最小单位是一个颗粒(其大小由 `index_granularity` 设置),是有意义的设置一个样品,其大小远大于颗粒。 + +使用时 `SAMPLE n` 子句,你不知道处理了哪些数据的相对百分比。 所以你不知道聚合函数应该乘以的系数。 使用 `_sample_factor` 虚拟列得到近似结果。 + +该 `_sample_factor` 列包含动态计算的相对系数。 当您执行以下操作时,将自动创建此列 [创建](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) 具有指定采样键的表。 的使用示例 `_sample_factor` 列如下所示。 + +让我们考虑表 `visits`,其中包含有关网站访问的统计信息。 第一个示例演示如何计算页面浏览量: + +``` sql +SELECT sum(PageViews * _sample_factor) +FROM visits +SAMPLE 10000000 +``` + +下一个示例演示如何计算访问总数: + +``` sql +SELECT sum(_sample_factor) +FROM visits +SAMPLE 10000000 +``` + +下面的示例显示了如何计算平均会话持续时间。 请注意,您不需要使用相对系数来计算平均值。 + +``` sql +SELECT avg(Duration) +FROM visits +SAMPLE 10000000 +``` + +## SAMPLE K OFFSET M {#select-sample-offset} + +这里 `k` 和 `m` 是从0到1的数字。 示例如下所示。 + +**示例1** + +``` sql +SAMPLE 1/10 +``` + +在此示例中,示例是所有数据的十分之一: + +`[++------------]` + +**示例2** + +``` sql +SAMPLE 1/10 OFFSET 1/2 +``` + +这里,从数据的后半部分取出10%的样本。 + +`[------++------]` diff --git a/docs/zh/sql-reference/statements/select/union-all.md b/docs/zh/sql-reference/statements/select/union-all.md deleted file mode 120000 index 837caae2698..00000000000 --- a/docs/zh/sql-reference/statements/select/union-all.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/union-all.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/union-all.md b/docs/zh/sql-reference/statements/select/union-all.md new file mode 100644 index 00000000000..a04996bc0a6 --- /dev/null +++ b/docs/zh/sql-reference/statements/select/union-all.md @@ -0,0 +1,36 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: UNION ALL +--- + +# UNION ALL条款 {#union-all-clause} + +您可以使用 `UNION ALL` 结合任意数量的 `SELECT` 通过扩展其结果进行查询。 示例: + +``` sql +SELECT CounterID, 1 AS table, toInt64(count()) AS c + FROM test.hits + GROUP BY CounterID + +UNION ALL + +SELECT CounterID, 2 AS table, sum(Sign) AS c + FROM test.visits + GROUP BY CounterID + HAVING c > 0 +``` + +结果列通过它们的索引进行匹配(在内部的顺序 `SELECT`). 如果列名称不匹配,则从第一个查询中获取最终结果的名称。 + +对联合执行类型转换。 例如,如果合并的两个查询具有相同的字段与非-`Nullable` 和 `Nullable` 从兼容类型的类型,由此产生的 `UNION ALL` 有一个 `Nullable` 类型字段。 + +属于以下部分的查询 `UNION ALL` 不能用圆括号括起来。 [ORDER BY](../../../sql-reference/statements/select/order-by.md) 和 [LIMIT](../../../sql-reference/statements/select/limit.md) 应用于单独的查询,而不是最终结果。 如果您需要将转换应用于最终结果,则可以将所有查询 `UNION ALL` 在子查询中 [FROM](../../../sql-reference/statements/select/from.md) 条款 + +## 限制 {#limitations} + +只有 `UNION ALL` 支持。 定期的 `UNION` (`UNION DISTINCT`)不支持。 如果你需要 `UNION DISTINCT`,你可以写 `SELECT DISTINCT` 从包含 `UNION ALL`. + +## 实施细节 {#implementation-details} + +属于以下部分的查询 `UNION ALL` 可以同时运行,并且它们的结果可以混合在一起。 diff --git a/docs/zh/sql-reference/statements/select/where.md b/docs/zh/sql-reference/statements/select/where.md deleted file mode 120000 index 8ba28926879..00000000000 --- a/docs/zh/sql-reference/statements/select/where.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/where.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/where.md b/docs/zh/sql-reference/statements/select/where.md new file mode 100644 index 00000000000..eb1da0d1027 --- /dev/null +++ b/docs/zh/sql-reference/statements/select/where.md @@ -0,0 +1,16 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: WHERE +--- + +# WHERE条款 {#select-where} + +`WHERE` 子句允许过滤来自 [FROM](../../../sql-reference/statements/select/from.md) 的条款 `SELECT`. + +如果有一个 `WHERE` 子句,它必须包含一个表达式与 `UInt8` 类型。 这通常是一个带有比较和逻辑运算符的表达式。 此表达式计算结果为0的行将从进一步的转换或结果中解释出来。 + +`WHERE` 如果基础表引擎支持,则根据使用索引和分区修剪的能力评估expression。 + +!!! note "注" + 有一个叫做过滤优化 [去哪里](../../../sql-reference/statements/select/prewhere.md). diff --git a/docs/zh/sql-reference/statements/select/with.md b/docs/zh/sql-reference/statements/select/with.md deleted file mode 120000 index 8b7ea4db44c..00000000000 --- a/docs/zh/sql-reference/statements/select/with.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/select/with.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/select/with.md b/docs/zh/sql-reference/statements/select/with.md new file mode 100644 index 00000000000..224cd4790bb --- /dev/null +++ b/docs/zh/sql-reference/statements/select/with.md @@ -0,0 +1,81 @@ +--- +machine_translated: true +machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 +toc_title: WITH +--- + +# WITH条款 {#with-clause} + +本节提供对公共表表达式的支持 ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)),所以结果 `WITH` 子句可以在其余部分中使用 `SELECT` 查询。 + +## 限制 {#limitations} + +1. 不支持递归查询。 +2. 当在section中使用子查询时,它的结果应该是只有一行的标量。 +3. Expression的结果在子查询中不可用。 + +## 例 {#examples} + +**示例1:** 使用常量表达式作为 “variable” + +``` sql +WITH '2019-08-01 15:23:00' as ts_upper_bound +SELECT * +FROM hits +WHERE + EventDate = toDate(ts_upper_bound) AND + EventTime <= ts_upper_bound +``` + +**示例2:** 从SELECT子句列表中逐出sum(bytes)表达式结果 + +``` sql +WITH sum(bytes) as s +SELECT + formatReadableSize(s), + table +FROM system.parts +GROUP BY table +ORDER BY s +``` + +**例3:** 使用标量子查询的结果 + +``` sql +/* this example would return TOP 10 of most huge tables */ +WITH + ( + SELECT sum(bytes) + FROM system.parts + WHERE active + ) AS total_disk_usage +SELECT + (sum(bytes) / total_disk_usage) * 100 AS table_disk_usage, + table +FROM system.parts +GROUP BY table +ORDER BY table_disk_usage DESC +LIMIT 10 +``` + +**例4:** 在子查询中重用表达式 + +作为子查询中表达式使用的当前限制的解决方法,您可以复制它。 + +``` sql +WITH ['hello'] AS hello +SELECT + hello, + * +FROM +( + WITH ['hello'] AS hello + SELECT hello +) +``` + +``` text +┌─hello─────┬─hello─────┐ +│ ['hello'] │ ['hello'] │ +└───────────┴───────────┘ +``` diff --git a/docs/zh/sql-reference/syntax.md b/docs/zh/sql-reference/syntax.md index 687638a9be6..a53de43d8f4 100644 --- a/docs/zh/sql-reference/syntax.md +++ b/docs/zh/sql-reference/syntax.md @@ -38,7 +38,7 @@ CH支持SQL风格或C语言风格的注释: - 在某些流行的RDBMS中被实现的关键字,例如,`DateTime` 和 `datetime`是一样的 -你可以在系统表 [system.data_type_families](../operations/system-tables.md#system_tables-data_type_families) 中检查某个数据类型的名称是否是大小写敏感型。 +你可以在系统表 [system.data_type_families](../operations/system-tables/data_type_families.md#system_tables-data_type_families) 中检查某个数据类型的名称是否是大小写敏感型。 和标准SQL相反,所有其它的关键字都是 **大小写敏感的**,包括函数名称。 In contrast to standard SQL, all other keywords (including functions names) are **case-sensitive**. From 9744c99f7efb60bc1160d5bff550cbb98bfa792c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 22 Jun 2020 02:44:13 +0300 Subject: [PATCH 152/211] Tune 01238_http_memory_tracking to make it more optimal w/o the patch the test fails with, since it succeed only 512 queries. --- .../01238_http_memory_tracking.reference | 2 +- .../0_stateless/01238_http_memory_tracking.sh | 18 ++++-------------- 2 files changed, 5 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/01238_http_memory_tracking.reference b/tests/queries/0_stateless/01238_http_memory_tracking.reference index 5caff40c4a0..83b33d238da 100644 --- a/tests/queries/0_stateless/01238_http_memory_tracking.reference +++ b/tests/queries/0_stateless/01238_http_memory_tracking.reference @@ -1 +1 @@ -10000 +1000 diff --git a/tests/queries/0_stateless/01238_http_memory_tracking.sh b/tests/queries/0_stateless/01238_http_memory_tracking.sh index e2d6646e20a..508a15a536f 100755 --- a/tests/queries/0_stateless/01238_http_memory_tracking.sh +++ b/tests/queries/0_stateless/01238_http_memory_tracking.sh @@ -3,27 +3,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -# Don't even try to do run under sanitizers, since they are too slow. -${CLICKHOUSE_LOCAL} --query "SELECT max(value LIKE '%sanitize%') FROM system.build_options" | grep -q '1' && echo 'Skip test for sanitizer build' && exit - -# TODO: the test can be way more optimal - set -o pipefail -function execute_null() -{ - ${CLICKHOUSE_CLIENT} --format Null -n "$@" -} - # This is needed to keep at least one running query for user for the time of test. -# (10k http queries takes 10seconds, let's run for 3x more to avoid flaps) -execute_null <<<'SELECT sleepEachRow(1) FROM numbers(30)' & +# (1k http queries takes ~1 second, let's run for 5x more to avoid flaps) +${CLICKHOUSE_CLIENT} --format Null -n <<<'SELECT sleepEachRow(1) FROM numbers(5)' & # ignore "yes: standard output: Broken pipe" yes 'SELECT 1' 2>/dev/null | { - head -n10000 + head -n1000 } | { - xargs -P10000 -i ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&max_memory_usage_for_user=$((10<<30))" -d '{}' + xargs -i ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&max_memory_usage_for_user=$((1<<30))" -d '{}' } | grep -x -c 1 wait From b1e8976df41ff11f7619295cb02816514509289a Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Jun 2020 12:04:27 +0300 Subject: [PATCH 153/211] Merge with master --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Processors/QueryPlan/ReadFromStorageStep.cpp | 8 +++++--- src/Processors/QueryPlan/ReadFromStorageStep.h | 9 +++++++-- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 2 +- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataSelectExecutor.h | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 8 files changed, 19 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e3da6bef517..178c9aa7d8c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1370,7 +1370,7 @@ void InterpreterSelectQuery::executeFetchColumns( } auto read_step = std::make_unique( - table_lock, options, storage, + table_lock, metadata_snapshot, options, storage, required_columns, query_info, context, processing_stage, max_block_size, max_streams); read_step->setStepDescription("Read from " + storage->getName()); diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.cpp b/src/Processors/QueryPlan/ReadFromStorageStep.cpp index 83b8682e09c..6b78a1983bd 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromStorageStep.cpp @@ -13,7 +13,8 @@ namespace DB { ReadFromStorageStep::ReadFromStorageStep( - TableStructureReadLockHolder table_lock_, + TableLockHolder table_lock_, + StorageMetadataPtr & metadata_snapshot_, SelectQueryOptions options_, StoragePtr storage_, const Names & required_columns_, @@ -23,6 +24,7 @@ ReadFromStorageStep::ReadFromStorageStep( size_t max_block_size_, size_t max_streams_) : table_lock(std::move(table_lock_)) + , metadata_snapshot(metadata_snapshot_) , options(std::move(options_)) , storage(std::move(storage_)) , required_columns(required_columns_) @@ -35,11 +37,11 @@ ReadFromStorageStep::ReadFromStorageStep( /// Note: we read from storage in constructor of step because we don't know real header before reading. /// It will be fixed when storage return QueryPlanStep itself. - Pipes pipes = storage->read(required_columns, query_info, *context, processing_stage, max_block_size, max_streams); + Pipes pipes = storage->read(required_columns, metadata_snapshot, query_info, *context, processing_stage, max_block_size, max_streams); if (pipes.empty()) { - Pipe pipe(std::make_shared(storage->getSampleBlockForColumns(required_columns))); + Pipe pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID()))); if (query_info.prewhere_info) { diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.h b/src/Processors/QueryPlan/ReadFromStorageStep.h index 230e5acc1e0..a191c9e4cd1 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromStorageStep.h @@ -9,6 +9,9 @@ namespace DB class IStorage; using StoragePtr = std::shared_ptr; +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; + struct SelectQueryInfo; struct PrewhereInfo; @@ -18,7 +21,8 @@ class ReadFromStorageStep : public IQueryPlanStep { public: ReadFromStorageStep( - TableStructureReadLockHolder table_lock, + TableLockHolder table_lock, + StorageMetadataPtr & metadata_snapshot, SelectQueryOptions options, StoragePtr storage, const Names & required_columns, @@ -35,7 +39,8 @@ public: QueryPipelinePtr updatePipeline(QueryPipelines) override; private: - TableStructureReadLockHolder table_lock; + TableLockHolder table_lock; + StorageMetadataPtr metadata_snapshot; SelectQueryOptions options; StoragePtr storage; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 4cf01aef5a0..ea1eab02e2f 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1574,7 +1574,7 @@ std::set MergeTreeDataMergerMutator::getIndicesToRecalculate( return indices_to_recalc; } -bool MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const Names & columns, const MutationCommands & commands) const +bool MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const Names & columns, const MutationCommands & commands) { if (!metadata_snapshot->hasAnyTTL()) return false; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index d62587bef5f..428073b8742 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -170,7 +170,7 @@ private: const IndicesDescription & all_indices, const MutationCommands & commands_for_removes); - bool shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const Names & columns, const MutationCommands & commands) const; + static bool shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const Names & columns, const MutationCommands & commands); /// Return set of indices which should be recalculated during mutation also /// wraps input stream into additional expression stream diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index fa91a9190e5..4929bf4dbcb 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1280,7 +1280,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( const MergeTreeData::DataPartPtr & part, const StorageMetadataPtr & metadata_snapshot, const KeyCondition & key_condition, - const Settings & settings) const + const Settings & settings) { MarkRanges res; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 5669e8708b6..7c3b9584e98 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -95,11 +95,11 @@ private: const KeyCondition & key_condition, const Settings & settings) const; - MarkRanges markRangesFromPKRange( + static MarkRanges markRangesFromPKRange( const MergeTreeData::DataPartPtr & part, const StorageMetadataPtr & metadata_snapshot, const KeyCondition & key_condition, - const Settings & settings) const; + const Settings & settings); MarkRanges filterMarksUsingIndex( MergeTreeIndexPtr index_helper, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 869aacb0ea4..3fce870d544 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3984,7 +3984,7 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr &, const ASTPtr & pa void StorageReplicatedMergeTree::truncate( - const ASTPtr &, const StorageMetadataPtr &, const Context & query_context, TableExclusiveLockHolder & table_lock) + const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder & table_lock) { table_lock.release(); /// Truncate is done asynchronously. From 368286117a92955db1c7479f698da2361a80b7cf Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Jun 2020 12:09:23 +0300 Subject: [PATCH 154/211] Ad tests with long select and simulatenous alter --- .../01338_long_select_and_alter.reference | 3 +++ .../01338_long_select_and_alter.sh | 25 +++++++++++++++++++ ..._long_select_and_alter_zookeeper.reference | 3 +++ .../01338_long_select_and_alter_zookeeper.sh | 25 +++++++++++++++++++ 4 files changed, 56 insertions(+) create mode 100644 tests/queries/0_stateless/01338_long_select_and_alter.reference create mode 100755 tests/queries/0_stateless/01338_long_select_and_alter.sh create mode 100644 tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.reference create mode 100755 tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh diff --git a/tests/queries/0_stateless/01338_long_select_and_alter.reference b/tests/queries/0_stateless/01338_long_select_and_alter.reference new file mode 100644 index 00000000000..de0f4f0a920 --- /dev/null +++ b/tests/queries/0_stateless/01338_long_select_and_alter.reference @@ -0,0 +1,3 @@ +10 +5 +CREATE TABLE default.alter_mt\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = MergeTree()\nORDER BY key\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01338_long_select_and_alter.sh b/tests/queries/0_stateless/01338_long_select_and_alter.sh new file mode 100755 index 00000000000..1d75ff2f4b9 --- /dev/null +++ b/tests/queries/0_stateless/01338_long_select_and_alter.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS alter_mt" + +$CLICKHOUSE_CLIENT --query "CREATE TABLE alter_mt (key UInt64, value String) ENGINE=MergeTree() ORDER BY key" + +$CLICKHOUSE_CLIENT --query "INSERT INTO alter_mt SELECT number, toString(number) FROM numbers(5)" + +$CLICKHOUSE_CLIENT --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & + +# to be sure that select took all required locks +sleep 2 + +$CLICKHOUSE_CLIENT --query "ALTER TABLE alter_mt MODIFY COLUMN value UInt64" + +$CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM alter_mt" + +wait + +$CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE alter_mt" + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS alter_mt" diff --git a/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.reference b/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.reference new file mode 100644 index 00000000000..3f58142a16f --- /dev/null +++ b/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.reference @@ -0,0 +1,3 @@ +10 +5 +CREATE TABLE default.alter_mt\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/alter_mt\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh b/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh new file mode 100755 index 00000000000..4e41b550de9 --- /dev/null +++ b/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS alter_mt" + +$CLICKHOUSE_CLIENT --query "CREATE TABLE alter_mt (key UInt64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/alter_mt', '1') ORDER BY key" + +$CLICKHOUSE_CLIENT --query "INSERT INTO alter_mt SELECT number, toString(number) FROM numbers(5)" + +$CLICKHOUSE_CLIENT --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & + +# to be sure that select took all required locks +sleep 2 + +$CLICKHOUSE_CLIENT --query "ALTER TABLE alter_mt MODIFY COLUMN value UInt64" + +$CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM alter_mt" + +wait + +$CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE alter_mt" + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS alter_mt" From 903cb4c4e2af1b7b1038ef5452b40114316984b0 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 22 Jun 2020 17:09:19 +0800 Subject: [PATCH 155/211] fix PostgreSQL protocol build failure --- src/Core/PostgreSQLProtocol.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Core/PostgreSQLProtocol.h b/src/Core/PostgreSQLProtocol.h index b0b28bc1bd4..986dfb66912 100644 --- a/src/Core/PostgreSQLProtocol.h +++ b/src/Core/PostgreSQLProtocol.h @@ -233,6 +233,10 @@ public: /** Size of the message in bytes including message length part (4 bytes) */ virtual Int32 size() const = 0; + ISerializable() = default; + + ISerializable(const ISerializable &) = default; + virtual ~ISerializable() = default; }; From f262c4f652183abc5a1b8442d36bf758cb0d69dd Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Jun 2020 12:49:21 +0300 Subject: [PATCH 156/211] Better comments --- src/DataStreams/IBlockInputStream.h | 2 +- src/DataStreams/IBlockOutputStream.h | 2 +- src/Databases/IDatabase.h | 2 +- src/Functions/FunctionJoinGet.h | 2 +- src/Interpreters/InterpreterSelectQuery.h | 2 +- src/Processors/QueryPipeline.h | 2 +- .../QueryPlan/ReadFromStorageStep.h | 2 +- src/Storages/IStorage.h | 52 ++++++++++++++++--- src/Storages/StorageInMemoryMetadata.h | 43 ++++++++++++--- src/Storages/StorageReplicatedMergeTree.h | 2 +- ...tructureLockHolder.h => TableLockHolder.h} | 9 ++-- 11 files changed, 95 insertions(+), 25 deletions(-) rename src/Storages/{TableStructureLockHolder.h => TableLockHolder.h} (63%) diff --git a/src/DataStreams/IBlockInputStream.h b/src/DataStreams/IBlockInputStream.h index 68850a822e8..dff567d15dc 100644 --- a/src/DataStreams/IBlockInputStream.h +++ b/src/DataStreams/IBlockInputStream.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/DataStreams/IBlockOutputStream.h b/src/DataStreams/IBlockOutputStream.h index bb62d0183f9..4cc1257e955 100644 --- a/src/DataStreams/IBlockOutputStream.h +++ b/src/DataStreams/IBlockOutputStream.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 3d8d5c74ceb..26b27045be6 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -236,7 +236,7 @@ public: using ASTModifier = std::function; /// Change the table structure in metadata. - /// You must call under the TableStructureLock of the corresponding table . If engine_modifier is empty, then engine does not change. + /// You must call under the alter_lock of the corresponding table . If engine_modifier is empty, then engine does not change. virtual void alterTable( const Context & /*context*/, const StorageID & /*table_id*/, diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h index af95686c207..a82da589960 100644 --- a/src/Functions/FunctionJoinGet.h +++ b/src/Functions/FunctionJoinGet.h @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index b5f7548d40f..3ee0fdb44e1 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 48f906336e6..7dddb2526e5 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -7,7 +7,7 @@ #include #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.h b/src/Processors/QueryPlan/ReadFromStorageStep.h index a191c9e4cd1..fa34dedd573 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromStorageStep.h @@ -1,6 +1,6 @@ #include #include -#include +#include #include namespace DB diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6abf310442e..3cf26d218cd 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include @@ -138,10 +138,19 @@ public: public: + /// Get mutable version (snapshot) of storage metadata. Metadata object is + /// multiversion, so it can be concurrently chaged, but returned copy can be + /// used without any locks. StorageInMemoryMetadata getInMemoryMetadata() const { return *metadata.get(); } + /// Get immutable version (snapshot) of storage metadata. Metadata object is + /// multiversion, so it can be concurrently chaged, but returned copy can be + /// used without any locks. StorageMetadataPtr getInMemoryMetadataPtr() const { return metadata.get(); } + /// Update storage metadata. Used in ALTER or initialization of Storage. + /// Metadata object is multiversion, so this method can be called without + /// any locks. void setInMemoryMetadata(const StorageInMemoryMetadata & metadata_) { metadata.set(std::make_unique(metadata_)); @@ -171,16 +180,31 @@ private: StorageID storage_id; mutable std::mutex id_mutex; + /// Multiversion storage metadata. Allows to read/write storage metadata + /// without locks. MultiVersionStorageMetadataPtr metadata; private: RWLockImpl::LockHolder tryLockTimed( const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const SettingSeconds & acquire_timeout) const; public: + /// Lock table for share. This lock must be acuqired if you want to be sure, + /// that table will be not dropped while you holding this lock. It's used in + /// variety of cases starting from SELECT queries to background merges in + /// MergeTree. TableLockHolder lockForShare(const String & query_id, const SettingSeconds & acquire_timeout); + /// Lock table for alter. This lock must be acuqired in ALTER queries to be + /// sure, that we execute only one simultaneous alter. Doesn't affect share lock. TableLockHolder lockForAlter(const String & query_id, const SettingSeconds & acquire_timeout); + /// Lock table exclusively. This lock must be acuired if you want to be + /// sure, that no other thread (SELECT, merge, ALTER, etc.) doing something + /// with table. For example it allows to wait all threads before DROP or + /// truncate query. + /// + /// NOTE: You have to be 100% sure that you need this lock. It's extremely + /// heavyweight and makes table irresponsive. TableExclusiveLockHolder lockExclusively(const String & query_id, const SettingSeconds & acquire_timeout); /** Returns stage to which query is going to be processed in read() function. @@ -247,7 +271,10 @@ public: * num_streams - a recommendation, how many streams to return, * if the storage can return a different number of streams. * - * It is guaranteed that the structure of the table will not change over the lifetime of the returned streams (that is, there will not be ALTER, RENAME and DROP). + * metadata_snapshot is consistent snapshot of table metadata, it should be + * passed in all parts of the returned pipeline. Storage metadata can be + * changed during lifetime of the returned pipeline, but the snapshot is + * guaranteed to be immutable. */ virtual Pipes read( const Names & /*column_names*/, @@ -265,7 +292,10 @@ public: * Receives a description of the query, which can contain information about the data write method. * Returns an object by which you can write data sequentially. * - * It is guaranteed that the table structure will not change over the lifetime of the returned streams (that is, there will not be ALTER, RENAME and DROP). + * metadata_snapshot is consistent snapshot of table metadata, it should be + * passed in all parts of the returned streams. Storage metadata can be + * changed during lifetime of the returned streams, but the snapshot is + * guaranteed to be immutable. */ virtual BlockOutputStreamPtr write( const ASTPtr & /*query*/, @@ -284,7 +314,7 @@ public: virtual void drop() {} /** Clear the table data and leave it empty. - * Must be called under lockForAlter. + * Must be called under exclusive lock (lockExclusively). */ virtual void truncate( const ASTPtr & /*query*/, @@ -312,9 +342,8 @@ public: */ virtual void renameInMemory(const StorageID & new_table_id); - /** ALTER tables in the form of column changes that do not affect the change to Storage or its parameters. - * This method must fully execute the ALTER query, taking care of the locks itself. - * To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata-> + /** ALTER tables in the form of column changes that do not affect the change + * to Storage or its parameters. Executes under alter lock (lockForAlter). */ virtual void alter(const AlterCommands & params, const Context & context, TableLockHolder & alter_lock_holder); @@ -434,8 +463,17 @@ public: } private: + /// Lock required for alter queries (lockForAlter). Always taken for write + /// (actually can be replaced with std::mutex, but for consistency we use + /// RWLock). Allows to execute only one simultaneous alter query. Also it + /// should be taken by DROP-like queries, to be sure, that all alters are + /// finished. mutable RWLock alter_lock = RWLockImpl::create(); + /// Lock required for drop queries. Every thread that want to ensure, that + /// table is not dropped have to tabke this lock for read (lockForShare). + /// DROP-like queries take this lock for write (lockExclusively), to be sure + /// that all table threads finished. mutable RWLock drop_lock = RWLockImpl::create(); }; diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 1d392c2d228..254942ddfe8 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -14,6 +14,8 @@ namespace DB { +/// Common metadata for all storages. Contains all possible parts of CREATE +/// query from all storages, but only some subset used. struct StorageInMemoryMetadata { /// Columns of table with their names, types, @@ -46,12 +48,17 @@ struct StorageInMemoryMetadata StorageInMemoryMetadata(const StorageInMemoryMetadata & other); StorageInMemoryMetadata & operator=(const StorageInMemoryMetadata & other); + /// NOTE: Thread unsafe part. You should modify same StorageInMemoryMetadata + /// structure from different threads. It should be used as MultiVersion + /// object. See example in IStorage. - //////////////////////////////////////////////////////////////////////// - void setColumns(ColumnsDescription columns_); /// sets only real columns, possibly overwrites virtual ones. + /// Sets only real columns, possibly overwrites virtual ones. + void setColumns(ColumnsDescription columns_); + /// Sets secondary indices void setSecondaryIndices(IndicesDescription secondary_indices_); + /// Sets constraints void setConstraints(ConstraintsDescription constraints_); /// Set partition key for storage (methods bellow, are just wrappers for this @@ -67,19 +74,28 @@ struct StorageInMemoryMetadata /// struct). void setSamplingKey(const KeyDescription & sampling_key_); + /// Set common table TTLs void setTableTTLs(const TTLTableDescription & table_ttl_); + /// TTLs for seperate columns void setColumnTTLs(const TTLColumnsDescription & column_ttls_by_name_); + /// Set settings changes in metadata (some settings exlicetely specified in + /// CREATE query) void setSettingsChanges(const ASTPtr & settings_changes_); + /// Set SELECT query for (Materialized)View void setSelectQuery(const SelectQueryDescription & select_); - const ColumnsDescription & getColumns() const; /// returns combined set of columns + /// Returns combined set of columns + const ColumnsDescription & getColumns() const; + /// Returns secondary indices + const IndicesDescription & getSecondaryIndices() const; /// Has at least one non primary index bool hasSecondaryIndices() const; + /// Return table constraints const ConstraintsDescription & getConstraints() const; /// Returns true if there is set table TTL, any column TTL or any move TTL. @@ -106,11 +122,24 @@ struct StorageInMemoryMetadata /// indices, TTL expressions) if we update @updated_columns set of columns. ColumnDependencies getColumnDependencies(const NameSet & updated_columns) const; - Block getSampleBlock() const; /// ordinary + materialized. - Block getSampleBlockNonMaterialized() const; /// ordinary. - Block getSampleBlockWithVirtuals(const NamesAndTypesList & virtuals) const; /// ordinary + materialized + virtuals. + /// Block with ordinary + materialized columns. + Block getSampleBlock() const; + + /// Block with ordinary columns. + Block getSampleBlockNonMaterialized() const; + + /// Block with ordinary + materialized + virtuals. Virtuals have to be + /// explicitely specified, because they are part of Storage type, not + /// Storage metadata. + Block getSampleBlockWithVirtuals(const NamesAndTypesList & virtuals) const; + + + /// Block with ordinary + materialized + aliases + virtuals. Virtuals have + /// to be explicitely specified, because they are part of Storage type, not + /// Storage metadata. StorageID required only for more clear exception + /// message. Block getSampleBlockForColumns( - const Names & column_names, const NamesAndTypesList & virtuals, const StorageID & storage_id) const; /// ordinary + materialized + aliases + virtuals. + const Names & column_names, const NamesAndTypesList & virtuals, const StorageID & storage_id) const; /// Returns structure with partition key. const KeyDescription & getPartitionKey() const; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 53656211c93..a7d4abacc69 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -329,7 +329,7 @@ private: * If no one has such a part, nothing checks. * Not very reliable: if two replicas add a part almost at the same time, no checks will occur. * Adds actions to `ops` that add data about the part into ZooKeeper. - * Call under TableStructureLock. + * Call under lockForShare. */ void checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper, const DataPartPtr & part, Coordination::Requests & ops, String part_name = "", NameSet * absent_replicas_paths = nullptr); diff --git a/src/Storages/TableStructureLockHolder.h b/src/Storages/TableLockHolder.h similarity index 63% rename from src/Storages/TableStructureLockHolder.h rename to src/Storages/TableLockHolder.h index 946f9ee545c..6ec06a5e66c 100644 --- a/src/Storages/TableStructureLockHolder.h +++ b/src/Storages/TableLockHolder.h @@ -5,6 +5,10 @@ namespace DB { +using TableLockHolder = RWLockImpl::LockHolder; + +/// Table exclusive lock, holds both alter and drop locks. Useful for DROP-like +/// queries. struct TableExclusiveLockHolder { void release() { *this = TableExclusiveLockHolder(); } @@ -13,9 +17,8 @@ private: friend class IStorage; /// Order is important. - RWLockImpl::LockHolder alter_lock; - RWLockImpl::LockHolder drop_lock; + TableLockHolder alter_lock; + TableLockHolder drop_lock; }; -using TableLockHolder = RWLockImpl::LockHolder; } From 9293f2b16e75cace137486b572c98b0af9d48085 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Jun 2020 13:38:23 +0300 Subject: [PATCH 157/211] Fix 01213_alter_rename_column_zookeeper --- .../01213_alter_rename_column_zookeeper.reference | 1 + .../0_stateless/01213_alter_rename_column_zookeeper.sh | 8 +++++++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.reference b/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.reference index 5457becfeda..5f1372bd5fd 100644 --- a/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.reference +++ b/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.reference @@ -1,5 +1,6 @@ 1 CREATE TABLE default.table_for_rename_replicated\n(\n `date` Date,\n `key` UInt64,\n `value1` String,\n `value2` String,\n `value3` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/table_for_rename_replicated\', \'1\')\nPARTITION BY date\nORDER BY key\nSETTINGS index_granularity = 8192 +renamed_value1 CREATE TABLE default.table_for_rename_replicated\n(\n `date` Date,\n `key` UInt64,\n `renamed_value1` String,\n `value2` String,\n `value3` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/table_for_rename_replicated\', \'1\')\nPARTITION BY date\nORDER BY key\nSETTINGS index_granularity = 8192 1 date key renamed_value1 value2 value3 diff --git a/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.sh b/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.sh index b7a4738d417..d7b8ea3262d 100755 --- a/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.sh +++ b/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.sh @@ -35,7 +35,13 @@ while [[ -z $($CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns WHERE sleep 0.5 done -# RENAME on fly works +$CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns WHERE name = 'renamed_value1' and table = 'table_for_rename_replicated'" + +# SHOW CREATE TABLE takes query from .sql file on disk. +# previous select take metadata from memory. So, when previous select says, that return renamed_value1 already exists in table, it's still can have old version on disk. +while [[ -z $($CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE table_for_rename_replicated;" | grep 'renamed_value1') ]]; do + sleep 0.5 +done $CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE table_for_rename_replicated;" From 9612b3e32b810bd3cad1b2b8c7d1a0d216d1e4ec Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Jun 2020 14:03:01 +0300 Subject: [PATCH 158/211] Add data to test and simplify --- tests/integration/test_alter_codec/test.py | 34 +++++++++++++++++----- 1 file changed, 26 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_alter_codec/test.py b/tests/integration/test_alter_codec/test.py index e6ad8d664a8..7e038081110 100644 --- a/tests/integration/test_alter_codec/test.py +++ b/tests/integration/test_alter_codec/test.py @@ -7,15 +7,11 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', config_dir='configs', - main_configs=['configs/logs_config.xml'], - with_zookeeper=True, - macros={"shard": 0, "replica": 1} ) + main_configs=['configs/logs_config.xml']) node2 = cluster.add_instance('node2', config_dir='configs', - main_configs=['configs/logs_config.xml'], - with_zookeeper=True, - macros={"shard": 0, "replica": 2} ) + main_configs=['configs/logs_config.xml']) @pytest.fixture(scope="module") @@ -32,22 +28,34 @@ def test_alter_codec_pk(started_cluster): try: name = "test_alter_codec_pk" node1.query(""" - CREATE TABLE {name} (id UInt64) Engine=MergeTree() ORDER BY id + CREATE TABLE {name} (id UInt64, value UInt64) Engine=MergeTree() ORDER BY id """.format(name=name)) + node1.query("INSERT INTO {name} SELECT number, number * number from numbers(100)".format(name=name)) + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 CODEC(NONE)".format(name=name)) node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 CODEC(Delta, LZ4)".format(name=name)) + assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "4950\n" + with pytest.raises(QueryRuntimeException): node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt32 CODEC(Delta, LZ4)".format(name=name)) + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 DEFAULT 3 CODEC(Delta, LZ4)".format(name=name)) + node1.query("INSERT INTO {name} (value) VALUES (1)".format(name=name)) + + assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "4953\n" + with pytest.raises(QueryRuntimeException): node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 ALIAS 3 CODEC(Delta, LZ4)".format(name=name)) node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 MATERIALIZED 3 CODEC(Delta, LZ4)".format(name=name)) + node1.query("INSERT INTO {name} (value) VALUES (1)".format(name=name)) + + assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "4956\n" node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64".format(name=name)) with pytest.raises(QueryRuntimeException): @@ -61,9 +69,11 @@ def test_alter_codec_index(started_cluster): try: name = "test_alter_codec_index" node1.query(""" - CREATE TABLE {name} (`id` UInt64, INDEX id_index id TYPE minmax GRANULARITY 1) Engine=MergeTree() ORDER BY tuple() + CREATE TABLE {name} (`id` UInt64, value UInt64, INDEX id_index id TYPE minmax GRANULARITY 1) Engine=MergeTree() ORDER BY tuple() """.format(name=name)) + node1.query("INSERT INTO {name} SELECT number, number * number from numbers(100)".format(name=name)) + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 CODEC(NONE)".format(name=name)) node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 CODEC(Delta, LZ4)".format(name=name)) @@ -72,11 +82,19 @@ def test_alter_codec_index(started_cluster): node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 DEFAULT 3 CODEC(Delta, LZ4)".format(name=name)) + node1.query("INSERT INTO {name} (value) VALUES (1)".format(name=name)) + + assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "4953\n" + with pytest.raises(QueryRuntimeException): node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 ALIAS 3 CODEC(Delta, LZ4)".format(name=name)) node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 MATERIALIZED 3 CODEC(Delta, LZ4)".format(name=name)) + node1.query("INSERT INTO {name} (value) VALUES (1)".format(name=name)) + + assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "4956\n" + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64".format(name=name)) with pytest.raises(QueryRuntimeException): From 18ec373369d13a1d22f8c3faa680e49d9e2d1f8c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 22 Jun 2020 14:45:45 +0300 Subject: [PATCH 159/211] support (un)signed modifiers for integer types --- src/DataTypes/DataTypesNumber.cpp | 15 ++++++++++++ src/Parsers/ParserDataType.cpp | 8 +++++++ .../01144_multiword_data_types.reference | 4 +++- .../01144_multiword_data_types.sql | 24 ++++++++++++++++++- 4 files changed, 49 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/DataTypesNumber.cpp b/src/DataTypes/DataTypesNumber.cpp index e0f3d2829d1..de76a6cbbe3 100644 --- a/src/DataTypes/DataTypesNumber.cpp +++ b/src/DataTypes/DataTypesNumber.cpp @@ -65,6 +65,21 @@ void registerDataTypeNumbers(DataTypeFactory & factory) factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::CaseInsensitive); factory.registerAlias("DOUBLE PRECISION", "Float64", DataTypeFactory::CaseInsensitive); + + /// MySQL + factory.registerAlias("TINYINT SIGNED", "Int8", DataTypeFactory::CaseInsensitive); + factory.registerAlias("INT1 SIGNED", "Int8", DataTypeFactory::CaseInsensitive); + factory.registerAlias("SMALLINT SIGNED", "Int16", DataTypeFactory::CaseInsensitive); + factory.registerAlias("INT SIGNED", "Int32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("INTEGER SIGNED", "Int32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("BIGINT SIGNED", "Int64", DataTypeFactory::CaseInsensitive); + factory.registerAlias("TINYINT UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive); + factory.registerAlias("INT1 UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive); + factory.registerAlias("SMALLINT UNSIGNED", "UInt16", DataTypeFactory::CaseInsensitive); + factory.registerAlias("INT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("INTEGER UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("BIGINT UNSIGNED", "UInt64", DataTypeFactory::CaseInsensitive); + } } diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index 9ccbe86c064..a0a4eb97efe 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -55,6 +55,14 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (ParserKeyword("PRECISION").ignore(pos)) type_name_suffix = "PRECISION"; } + else if (type_name_upper.find("INT") != std::string::npos) + { + /// Support SIGNED and UNSIGNED integer type modifiers for compatibility with MySQL + if (ParserKeyword("SIGNED").ignore(pos)) + type_name_suffix = "SIGNED"; + else if (ParserKeyword("UNSIGNED").ignore(pos)) + type_name_suffix = "UNSIGNED"; + } if (!type_name_suffix.empty()) type_name = type_name_upper + " " + type_name_suffix; diff --git a/tests/queries/0_stateless/01144_multiword_data_types.reference b/tests/queries/0_stateless/01144_multiword_data_types.reference index 0f3b4691734..bf4c73a99b5 100644 --- a/tests/queries/0_stateless/01144_multiword_data_types.reference +++ b/tests/queries/0_stateless/01144_multiword_data_types.reference @@ -1,3 +1,5 @@ CREATE TABLE default.multiword_types\n(\n `a` Float64,\n `b` Float64,\n `c` String DEFAULT \'str\',\n `d` String,\n `e` String COMMENT \'comment\',\n `f` String,\n `g` String,\n `h` String DEFAULT toString(a) COMMENT \'comment\',\n `i` String,\n `j` String,\n `k` String,\n `l` String,\n `m` String,\n `n` String,\n `o` String,\n `p` String\n)\nENGINE = Memory Tuple(Float64, Float64, String, String, String, String, String, String, String, String, String, String, String, String, String, String) -42 42 +CREATE TABLE default.unsigned_types\n(\n `a` Int8,\n `b` Int8,\n `c` Int16,\n `d` Int32,\n `e` Int32,\n `f` Int64,\n `g` UInt8,\n `h` UInt8,\n `i` UInt16,\n `j` UInt32,\n `k` UInt32,\n `l` UInt64\n)\nENGINE = Memory +Tuple(Int8, Int8, Int16, Int32, Int32, Int64, UInt8, UInt8, UInt16, UInt32, UInt32, UInt64) +42 42 255 -1 diff --git a/tests/queries/0_stateless/01144_multiword_data_types.sql b/tests/queries/0_stateless/01144_multiword_data_types.sql index 7d96c5a1d74..cc380f82d63 100644 --- a/tests/queries/0_stateless/01144_multiword_data_types.sql +++ b/tests/queries/0_stateless/01144_multiword_data_types.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS multiword_types; +DROP TABLE IF EXISTS unsigned_types; CREATE TABLE multiword_types ( a DOUBLE, @@ -24,6 +25,27 @@ SHOW CREATE TABLE multiword_types; INSERT INTO multiword_types(a) VALUES (1); SELECT toTypeName((*,)) FROM multiword_types; -SELECT CAST('42' AS DOUBLE PRECISION), CAST(42, 'NATIONAL CHARACTER VARYING'); +CREATE TABLE unsigned_types ( + a TINYINT SIGNED, + b INT1 SIGNED, + c SMALLINT SIGNED, + d INT SIGNED, + e INTEGER SIGNED, + f BIGINT SIGNED, + g TINYINT UNSIGNED, + h INT1 UNSIGNED, + i SMALLINT UNSIGNED, + j INT UNSIGNED, + k INTEGER UNSIGNED, + l BIGINT UNSIGNED +) ENGINE=Memory; + +SHOW CREATE TABLE unsigned_types; + +INSERT INTO unsigned_types(a) VALUES (1); +SELECT toTypeName((*,)) FROM unsigned_types; + +SELECT CAST('42' AS DOUBLE PRECISION), CAST(42, 'NATIONAL CHARACTER VARYING'), CAST(-1 AS tinyint UnSiGnEd), CAST(65535, ' sMaLlInT signed '); DROP TABLE multiword_types; +DROP TABLE unsigned_types; From 95cb00e246ee12ddb60f8eb13b1c9d2805687a8c Mon Sep 17 00:00:00 2001 From: exprmntr Date: Mon, 22 Jun 2020 15:22:28 +0300 Subject: [PATCH 160/211] Skip some tests in Arcadia CI --- .../queries/0_stateless/arcadia_skip_list.txt | 123 ++++++++++++++++++ 1 file changed, 123 insertions(+) create mode 100644 tests/queries/0_stateless/arcadia_skip_list.txt diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt new file mode 100644 index 00000000000..ed22df94448 --- /dev/null +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -0,0 +1,123 @@ +00105_shard_collations +00436_convert_charset +00490_special_line_separators_and_characters_outside_of_bmp +00506_union_distributed +00586_removing_unused_columns_from_subquery +00588_shard_distributed_prewhere +00594_alias_in_distributed +00609_distributed_with_case_when_then +00678_murmurhash +00717_low_cardinaliry_distributed_group_by +00717_merge_and_distributed +00732_base64_functions +00746_hashing_tuples +00751_hashing_ints +00800_function_java_hash +00800_low_cardinality_distributed_insert +00821_distributed_storage_with_join_on +00829_bitmap_function +00850_global_join_dups +00858_issue_4756 +00877_memory_limit_for_new_delete +00921_datetime64_basic +00926_geo_to_h3 +00926_multimatch +00929_multi_match_edit_distance +00952_insert_into_distributed_with_materialized_column +00967_insert_into_distributed_different_types +00974_bitmapContains_with_primary_key +00974_distributed_join_on +00974_query_profiler +00980_shard_aggregation_state_deserialization +00980_skip_unused_shards_without_sharding_key +00987_distributed_stack_overflow +00990_metric_log_table_not_empty +01009_global_array_join_names +01016_macros +01018_ddl_dictionaries_select +01018_ddl_dictionaries_special +01018_dictionaries_from_dictionaries +01018_Distributed__shard_num +01023_materialized_view_query_context +01030_storage_hdfs_syntax +01033_dictionaries_lifetime +01036_no_superfluous_dict_reload_on_create_database +01036_no_superfluous_dict_reload_on_create_database_2 +01037_polygon_dict_multi_polygons +01037_polygon_dict_simple_polygons +01040_distributed_directory_monitor_batch_inserts +01040_h3_get_resolution +01041_create_dictionary_if_not_exists +01041_h3_is_valid +01042_h3_k_ring +01043_dictionary_attribute_properties_values +01043_h3_edge_length_m +01044_h3_edge_angle +01046_materialized_view_with_join_over_distributed +01050_clickhouse_dict_source_with_subquery +01059_storage_file_brotli +01070_h3_get_base_cell +01070_h3_hex_area_m2 +01070_h3_indexes_are_neighbors +01070_h3_to_children +01070_h3_to_parent +01070_h3_to_string +01070_string_to_h3 +01071_force_optimize_skip_unused_shards +01072_optimize_skip_unused_shards_const_expr_eval +01074_h3_range_check +01080_check_for_error_incorrect_size_of_nested_column +01083_expressions_in_engine_arguments +01083_log_family_disk_memory +01091_num_threads +01092_base64 +01092_memory_profiler +01099_parallel_distributed_insert_select +01102_distributed_local_in_bug +01103_distributed_product_mode_local_column_renames +01104_distributed_numbers_test +01104_distributed_one_test +01110_dictionary_layout_without_arguments +01113_local_dictionary_type_conversion +01114_mysql_database_engine_segfault +01115_join_with_dictionary +01125_dict_ddl_cannot_add_column +01129_dict_get_join_lose_constness +01138_join_on_distributed_and_tmp +01200_mutations_memory_consumption +01211_optimize_skip_unused_shards_type_mismatch +01213_optimize_skip_unused_shards_DISTINCT +01220_scalar_optimization_in_alter +01223_dist_on_dist +01225_drop_dictionary_as_table +01227_distributed_global_in_issue_2610 +01231_distributed_aggregation_memory_efficient_mix_levels +01232_extremes +01235_live_view_over_distributed +01236_distributed_over_live_view_over_distributed +01236_graphite_mt +01237_live_view_over_distributed_with_subquery_select_table_alias +01247_dist_on_dist_group_by_sharding_key_optimization +01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key +01251_dict_is_in_infinite_loop +01253_subquery_in_aggregate_function_JustStranger +01254_dict_create_without_db +01254_dict_load_after_detach_attach +01257_dictionary_mismatch_types +01259_dictionary_custom_settings_ddl +01263_type_conversion_nvartolomei +01267_alter_default_key_columns +01268_dictionary_direct_layout +01270_optimize_skip_unused_shards_low_cardinality +01273_h3EdgeAngle_range_check +01274_alter_rename_column_distributed +01276_system_licenses +01291_distributed_low_cardinality_memory_efficient +01292_create_user +01293_show_clusters +01293_system_distribution_queue +01294_system_distributed_on_cluster +01297_alter_distributed +01303_aggregate_function_nothing_serde +01319_query_formatting_in_server_log +01326_build_id From 7ab326a1439c39329295c0ab65f7f03f50bedf5e Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 22 Jun 2020 15:34:25 +0300 Subject: [PATCH 161/211] remove unneeded define --- src/Core/Defines.h | 2 -- src/Core/Settings.h | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Core/Defines.h b/src/Core/Defines.h index bff782ea1b5..8b26f486c9d 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -45,8 +45,6 @@ #define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD 60 /// replica error max cap, this is to prevent replica from accumulating too many errors and taking to long to recover. #define DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT 1000 -/// Number of errors that will be ignored while choosing replicas -#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_ERROR_IGNORE 0 #define DBMS_MIN_REVISION_WITH_CLIENT_INFO 54032 #define DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE 54058 diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3ea3711fd39..6aaebc4e995 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -348,7 +348,7 @@ struct Settings : public SettingsCollection \ M(SettingSeconds, distributed_replica_error_half_life, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD, "Time period reduces replica error counter by 2 times.", 0) \ M(SettingUInt64, distributed_replica_error_cap, DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT, "Max number of errors per replica, prevents piling up an incredible amount of errors if replica was offline for some time and allows it to be reconsidered in a shorter amount of time.", 0) \ - M(SettingUInt64, distributed_replica_max_ignored_errors, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_ERROR_IGNORE, "Number of errors that will be ignored while choosing replicas", 0) \ + M(SettingUInt64, distributed_replica_max_ignored_errors, 0, "Number of errors that will be ignored while choosing replicas", 0) \ \ M(SettingBool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \ M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.", 0) \ From 1f2bb90548681af9c29f46c0f2fa91d3acccd8ad Mon Sep 17 00:00:00 2001 From: exprmntr Date: Mon, 22 Jun 2020 15:48:17 +0300 Subject: [PATCH 162/211] Update arcadia_skip_list.txt --- tests/queries/0_stateless/arcadia_skip_list.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index ed22df94448..dd16cb00ecf 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -1,3 +1,5 @@ +# Add testcase here to skip it in Arcadia CI (Yandex synchronization check) +# It is useful for tests with not supported features in Arcadia build 00105_shard_collations 00436_convert_charset 00490_special_line_separators_and_characters_outside_of_bmp From d3deed991e56f27eaed8efa6bb2cdd78c9d938c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jun 2020 15:49:38 +0300 Subject: [PATCH 163/211] Fix bad test for LIVE VIEW --- .../test_distributed_over_live_view/test.py | 47 +++++++++++++++---- 1 file changed, 37 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_distributed_over_live_view/test.py b/tests/integration/test_distributed_over_live_view/test.py index f932379e5c6..46c0bada535 100644 --- a/tests/integration/test_distributed_over_live_view/test.py +++ b/tests/integration/test_distributed_over_live_view/test.py @@ -81,7 +81,10 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) client1.send(select_query) - client1.expect('"node1",0,0\r\n.*"node1",1,1\r\n.*"node2",0,10\r\n.*"node2",1,11\r\n') + client1.expect('"node1",0,0') + client1.expect('"node1",1,1') + client1.expect('"node2",0,10') + client1.expect('"node2",1,11') client1.expect(prompt) client1.send("INSERT INTO distributed_table VALUES ('node1', 1, 3), ('node1', 2, 3)") @@ -90,7 +93,13 @@ class TestLiveViewOverDistributedSuite: client2.expect(prompt) time.sleep(2) client1.send(select_query) - client1.expect('"node1",0,0\r\n.*"node1",1,1\r\n.*"node1",1,3\r\n.*"node1",2,3\r\n.*"node1",3,3\r\n.*"node2",0,10\r\n.*"node2",1,11\r\n') + client1.expect('"node1",0,0') + client1.expect('"node1",1,1') + client1.expect('"node1",1,3') + client1.expect('"node1",2,3') + client1.expect('"node1",3,3') + client1.expect('"node2",0,10') + client1.expect('"node2",1,11') client1.expect(prompt) def test_distributed_over_live_view_order_by_key(self, started_cluster, node, source): @@ -110,7 +119,10 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) client1.send(select_query) - client1.expect('"node1",0,0\r\n"node2",0,10\r\n"node1",1,1\r\n.*"node2",1,11\r\n') + client1.expect('"node1",0,0') + client1.expect('"node2",0,10') + client1.expect('"node1",1,1') + client1.expect('"node2",1,11') client1.expect(prompt) client1.send("INSERT INTO distributed_table VALUES ('node1', 1, 3), ('node1', 2, 3)") @@ -119,7 +131,13 @@ class TestLiveViewOverDistributedSuite: client2.expect(prompt) time.sleep(2) client1.send(select_query) - client1.expect('"node1",0,0\r\n.*"node2",0,10.*\r\n"node1",1,1\r\n.*"node1",1,3\r\n.*"node2",1,11\r\n.*"node1",2,3\r\n.*"node1",3,3\r\n') + client1.expect('"node1",0,0') + client1.expect('"node2",0,10') + client1.expect('"node1",1,1') + client1.expect('"node1",1,3') + client1.expect('"node2",1,11') + client1.expect('"node1",2,3') + client1.expect('"node1",3,3') client1.expect(prompt) def test_distributed_over_live_view_group_by_node(self, started_cluster, node, source): @@ -139,14 +157,16 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) client1.send(select_query) - client1.expect('"node1",1\r\n"node2",21\r\n') + client1.expect('"node1",1') + client1.expect('"node2",21') client1.expect(prompt) client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) time.sleep(2) client1.send(select_query) - client1.expect('"node1",3\r\n.*"node2",21\r\n') + client1.expect('"node1",3') + client1.expect('"node2",21') client1.expect(prompt) client1.send("INSERT INTO distributed_table VALUES ('node1', 1, 3), ('node1', 3, 3)") @@ -155,7 +175,8 @@ class TestLiveViewOverDistributedSuite: client2.expect(prompt) time.sleep(2) client1.send(select_query) - client1.expect('"node1",12\r\n.*"node2",21\r\n') + client1.expect('"node1",12') + client1.expect('"node2",21') client1.expect(prompt) def test_distributed_over_live_view_group_by_key(self, started_cluster, node, source): @@ -175,21 +196,27 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) client1.send(select_query) - client1.expect("0,10\r\n1,12\r\n") + client1.expect('0,10') + client1.expect('1,12') client1.expect(prompt) client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) time.sleep(2) client1.send(select_query) - client1.expect("0,10\r\n1,12\r\n2,2\r\n") + client1.expect('0,10') + client1.expect('1,12') + client1.expect('2,2') client1.expect(prompt) client2.send("INSERT INTO distributed_table VALUES ('node1', 1, 3), ('node1', 3, 3)") client2.expect(prompt) time.sleep(2) client1.send(select_query) - client1.expect("0,10\r\n.*1,15\r\n.*2,2\r\n.*3,3\r\n") + client1.expect('0,10') + client1.expect('1,15') + client1.expect('2,2') + client1.expect('3,3') client1.expect(prompt) def test_distributed_over_live_view_sum(self, started_cluster, node, source): From 588568f39c09ed4a0538f24cf901cae39b514ab0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Jun 2020 16:10:25 +0300 Subject: [PATCH 164/211] Disable MySQL tests under tsan --- tests/integration/helpers/cluster.py | 4 ++ .../test.py | 44 ++++++++++++++++++- 2 files changed, 47 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 1c1f758b291..d291d06851f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -732,6 +732,10 @@ class ClickHouseInstance: self.ipv6_address = ipv6_address self.with_installed_binary = with_installed_binary + def is_built_with_thread_sanitizer(self): + build_opts = self.query("SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") + return "-fsanitize=thread" in build_opts + # Connects to the instance via clickhouse-client, sends a query (1st argument) and returns the answer def query(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None, ignore_error=False): return self.client.query(sql, stdin, timeout, settings, user, password, ignore_error) diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py index 0a812ea2a8b..f4b0ba9c1e4 100644 --- a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py +++ b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py @@ -153,6 +153,7 @@ def get_dict(source, layout, fields, suffix_name=''): dictionary.generate_config() return dictionary + def setup_module(module): global DICTIONARIES global cluster @@ -210,8 +211,42 @@ def get_dictionaries(fold, total_folds, all_dicts): return all_dicts[fold * chunk_len : (fold + 1) * chunk_len] +def remove_mysql_dicts(): + """ + We have false-positive race condition in our openSSL version. + MySQL dictionary use OpenSSL, so to prevent known failure we + disable tests for these dictionaries. + + Read of size 8 at 0x7b3c00005dd0 by thread T61 (mutexes: write M1010349240585225536): + #0 EVP_CIPHER_mode (clickhouse+0x13b2223b) + #1 do_ssl3_write (clickhouse+0x13a137bc) + #2 ssl3_write_bytes (clickhouse+0x13a12387) + #3 ssl3_write (clickhouse+0x139db0e6) + #4 ssl_write_internal (clickhouse+0x139eddce) + #5 SSL_write (clickhouse+0x139edf20) + #6 ma_tls_write (clickhouse+0x139c7557) + #7 ma_pvio_tls_write (clickhouse+0x139a8f59) + #8 ma_pvio_write (clickhouse+0x139a8488) + #9 ma_net_real_write (clickhouse+0x139a4e2c) + #10 ma_net_write_command (clickhouse+0x139a546d) + #11 mthd_my_send_cmd (clickhouse+0x13992546) + #12 mysql_close_slow_part (clickhouse+0x13999afd) + #13 mysql_close (clickhouse+0x13999071) + #14 mysqlxx::Connection::~Connection() (clickhouse+0x1370f814) + #15 mysqlxx::Pool::~Pool() (clickhouse+0x13715a7b) + + TODO remove this when open ssl will be fixed or thread sanitizer will be suppressed + """ + + global DICTIONARIES + DICTIONARIES = [d for d in DICTIONARIES if not d.name.startswith("MySQL")] + + @pytest.mark.parametrize("fold", list(range(10))) def test_simple_dictionaries(started_cluster, fold): + if node.is_built_with_thread_sanitizer(): + remove_mysql_dicts() + fields = FIELDS["simple"] values = VALUES["simple"] data = [Row(fields, vals) for vals in values] @@ -259,6 +294,10 @@ def test_simple_dictionaries(started_cluster, fold): @pytest.mark.parametrize("fold", list(range(10))) def test_complex_dictionaries(started_cluster, fold): + + if node.is_built_with_thread_sanitizer(): + remove_mysql_dicts() + fields = FIELDS["complex"] values = VALUES["complex"] data = [Row(fields, vals) for vals in values] @@ -292,6 +331,9 @@ def test_complex_dictionaries(started_cluster, fold): @pytest.mark.parametrize("fold", list(range(10))) def test_ranged_dictionaries(started_cluster, fold): + if node.is_built_with_thread_sanitizer(): + remove_mysql_dicts() + fields = FIELDS["ranged"] values = VALUES["ranged"] data = [Row(fields, vals) for vals in values] @@ -380,7 +422,7 @@ def test_key_value_complex_dictionaries(started_cluster, fold): values = VALUES["complex"] data = [Row(fields, vals) for vals in values] - all_complex_dicts = [d for d in DICTIONARIES if d.structure.layout.layout_type == "complex"] + all_complex_dicts = [d for d in DICTIONARIES_KV if d.structure.layout.layout_type == "complex"] complex_dicts = get_dictionaries(fold, 10, all_complex_dicts) for dct in complex_dicts: dct.load_data(data) From afcf63ad89eaa31685bab287198122f7fb4dad1c Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Jun 2020 16:29:35 +0300 Subject: [PATCH 165/211] Add alias for postgesql container --- docker/test/integration/compose/docker_compose_postgres.yml | 4 ++++ tests/integration/helpers/cluster.py | 2 -- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docker/test/integration/compose/docker_compose_postgres.yml b/docker/test/integration/compose/docker_compose_postgres.yml index ffc4d3164b5..fff4fb1fa42 100644 --- a/docker/test/integration/compose/docker_compose_postgres.yml +++ b/docker/test/integration/compose/docker_compose_postgres.yml @@ -7,3 +7,7 @@ services: POSTGRES_PASSWORD: mysecretpassword ports: - 5432:5432 + networks: + default: + aliases: + - postgre-sql.local diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 2006d017bc2..1c1f758b291 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -668,8 +668,6 @@ services: - {env_file} security_opt: - label:disable - links: - - postgres1:postgre-sql.local {networks} {app_net} {ipv4_address} From b4d89bd124476ae00c1bb3aca22ae0a74c1c8efd Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Jun 2020 16:31:16 +0300 Subject: [PATCH 166/211] Fix comment --- tests/integration/test_odbc_interaction/test.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 7bd9a756702..b93619d06b5 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -223,8 +223,9 @@ def test_postgres_insert(started_cluster): conn = get_postgres_conn() conn.cursor().execute("truncate table clickhouse.test_table") - # Also test with Servername containing '.' and '-' symbols (see links in docker-compose template in helpers/cluster.py) - # This is needed to check parsing, validation and reconstruction of connection string. + # Also test with Servername containing '.' and '-' symbols (defined in + # postgres .yml file). This is needed to check parsing, validation and + # reconstruction of connection string. node1.query("create table pg_insert (column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')") node1.query("insert into pg_insert values (1, 'hello'), (2, 'world')") From 652882ef4e251d511f29529ecb5e1d0819f57d73 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 22 Jun 2020 16:35:41 +0300 Subject: [PATCH 167/211] [docs] reorganized F.A.Q. --- docs/en/faq/general.md | 58 ---------------- docs/en/faq/general/dbms-naming.md | 11 +++ docs/en/faq/general/index.md | 18 +++++ docs/en/faq/general/mapreduce.md | 12 ++++ docs/en/faq/general/ne-tormozit.md | 24 +++++++ docs/en/faq/index.md | 11 ++- docs/en/faq/integration/file-export.md | 36 ++++++++++ docs/en/faq/integration/index.md | 17 +++++ docs/en/faq/integration/oracle-odbc.md | 14 ++++ docs/en/faq/operations/index.md | 16 +++++ docs/en/faq/operations/production.md | 69 +++++++++++++++++++ .../external-dicts-dict-sources.md | 2 +- docs/tools/nav.py | 7 +- website/templates/docs/content.html | 6 +- website/templates/docs/nav.html | 2 +- website/templates/docs/sidebar-item.html | 6 +- 16 files changed, 244 insertions(+), 65 deletions(-) delete mode 100644 docs/en/faq/general.md create mode 100644 docs/en/faq/general/dbms-naming.md create mode 100644 docs/en/faq/general/index.md create mode 100644 docs/en/faq/general/mapreduce.md create mode 100644 docs/en/faq/general/ne-tormozit.md create mode 100644 docs/en/faq/integration/file-export.md create mode 100644 docs/en/faq/integration/index.md create mode 100644 docs/en/faq/integration/oracle-odbc.md create mode 100644 docs/en/faq/operations/index.md create mode 100644 docs/en/faq/operations/production.md diff --git a/docs/en/faq/general.md b/docs/en/faq/general.md deleted file mode 100644 index 53cb583e25f..00000000000 --- a/docs/en/faq/general.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -toc_priority: 78 -toc_title: General Questions ---- - -# General Questions {#general-questions} - -## Why Not Use Something Like MapReduce? {#why-not-use-something-like-mapreduce} - -We can refer to systems like MapReduce as distributed computing systems in which the reduce operation is based on distributed sorting. The most common open-source solution in this class is [Apache Hadoop](http://hadoop.apache.org). Yandex uses its in-house solution, YT. - -These systems aren’t appropriate for online queries due to their high latency. In other words, they can’t be used as the back-end for a web interface. These types of systems aren’t useful for real-time data updates. Distributed sorting isn’t the best way to perform reduce operations if the result of the operation and all the intermediate results (if there are any) are located in the RAM of a single server, which is usually the case for online queries. In such a case, a hash table is an optimal way to perform reduce operations. A common approach to optimizing map-reduce tasks is pre-aggregation (partial reduce) using a hash table in RAM. The user performs this optimization manually. Distributed sorting is one of the main causes of reduced performance when running simple map-reduce tasks. - -Most MapReduce implementations allow you to execute arbitrary code on a cluster. But a declarative query language is better suited to OLAP to run experiments quickly. For example, Hadoop has Hive and Pig. Also consider Cloudera Impala or Shark (outdated) for Spark, as well as Spark SQL, Presto, and Apache Drill. Performance when running such tasks is highly sub-optimal compared to specialized systems, but relatively high latency makes it unrealistic to use these systems as the backend for a web interface. - -## What If I Have a Problem with Encodings When Using Oracle Through ODBC? {#oracle-odbc-encodings} - -If you use Oracle through the ODBC driver as a source of external dictionaries, you need to set the correct value for the `NLS_LANG` environment variable in `/etc/default/clickhouse`. For more information, see the [Oracle NLS\_LANG FAQ](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). - -**Example** - -``` sql -NLS_LANG=RUSSIAN_RUSSIA.UTF8 -``` - -## How Do I Export Data from ClickHouse to a File? {#how-to-export-to-file} - -### Using INTO OUTFILE Clause {#using-into-outfile-clause} - -Add an [INTO OUTFILE](../sql-reference/statements/select/into-outfile.md#into-outfile-clause) clause to your query. - -For example: - -``` sql -SELECT * FROM table INTO OUTFILE 'file' -``` - -By default, ClickHouse uses the [TabSeparated](../interfaces/formats.md#tabseparated) format for output data. To select the [data format](../interfaces/formats.md), use the [FORMAT clause](../sql-reference/statements/select/format.md#format-clause). - -For example: - -``` sql -SELECT * FROM table INTO OUTFILE 'file' FORMAT CSV -``` - -### Using a File-Engine Table {#using-a-file-engine-table} - -See [File](../engines/table-engines/special/file.md). - -### Using Command-Line Redirection {#using-command-line-redirection} - -``` sql -$ clickhouse-client --query "SELECT * from table" --format FormatName > result.txt -``` - -See [clickhouse-client](../interfaces/cli.md). - -{## [Original article](https://clickhouse.tech/docs/en/faq/general/) ##} diff --git a/docs/en/faq/general/dbms-naming.md b/docs/en/faq/general/dbms-naming.md new file mode 100644 index 00000000000..4140606bce2 --- /dev/null +++ b/docs/en/faq/general/dbms-naming.md @@ -0,0 +1,11 @@ +--- +toc_priority: 10 +toc_hidden: true +--- + +# What Does "ClickHouse" Mean? + +It's a combination of "**Click**stream" and "Data ware**house**". It comes from the original use case at Yandex.Metrica, where ClickHouse was supposed to keep records of all clicks by people from all over the Internet and it still does the job. You can read more about this use case on [ClickHouse history](../../introduction/history.md) page. + +!!! info "Fun fact" + Many years after ClickHouse got its name, this approach of combining two words that are meaningful on their own has been highlighted as the best way to name a database in a [research by Andy Pavlo](https://www.cs.cmu.edu/~pavlo/blog/2020/03/on-naming-a-database-management-system.html), an Associate Professor of Databases at Carnegie Mellon University. ClickHouse shared his "best database name of all time" award with Postgres. diff --git a/docs/en/faq/general/index.md b/docs/en/faq/general/index.md new file mode 100644 index 00000000000..204bfefcba6 --- /dev/null +++ b/docs/en/faq/general/index.md @@ -0,0 +1,18 @@ +--- +toc_priority: 1 +toc_title: General +toc_hidden_folder: true +--- + +# General Questions about ClickHouse {#general-questions} + +Questions: + +- [What does "ClickHouse" mean?](dbms-naming.md) +- [What does "Не тормозит" mean?](ne-tormozit.md) +- [Why not use something like MapReduce?](mapreduce.md) + +!!! info "Not see what you were locking for?" + Check out [other F.A.Q. categories](../index.md) or browse around main documentation articles found in the left sidebar. + +{## [Original article](https://clickhouse.tech/docs/en/faq/general/) ##} diff --git a/docs/en/faq/general/mapreduce.md b/docs/en/faq/general/mapreduce.md new file mode 100644 index 00000000000..12c7e5efb4a --- /dev/null +++ b/docs/en/faq/general/mapreduce.md @@ -0,0 +1,12 @@ +--- +toc_priority: 20 +toc_hidden: true +--- + +# Why not Use Something Like MapReduce? {#why-not-use-something-like-mapreduce} + +We can refer to systems like MapReduce as distributed computing systems in which the reduce operation is based on distributed sorting. The most common open-source solution in this class is [Apache Hadoop](http://hadoop.apache.org). Yandex uses its in-house solution, YT. + +These systems aren’t appropriate for online queries due to their high latency. In other words, they can’t be used as the back-end for a web interface. These types of systems aren’t useful for real-time data updates. Distributed sorting isn’t the best way to perform reduce operations if the result of the operation and all the intermediate results (if there are any) are located in the RAM of a single server, which is usually the case for online queries. In such a case, a hash table is an optimal way to perform reduce operations. A common approach to optimizing map-reduce tasks is pre-aggregation (partial reduce) using a hash table in RAM. The user performs this optimization manually. Distributed sorting is one of the main causes of reduced performance when running simple map-reduce tasks. + +Most MapReduce implementations allow you to execute arbitrary code on a cluster. But a declarative query language is better suited to OLAP to run experiments quickly. For example, Hadoop has Hive and Pig. Also consider Cloudera Impala or Shark (outdated) for Spark, as well as Spark SQL, Presto, and Apache Drill. Performance when running such tasks is highly sub-optimal compared to specialized systems, but relatively high latency makes it unrealistic to use these systems as the backend for a web interface. diff --git a/docs/en/faq/general/ne-tormozit.md b/docs/en/faq/general/ne-tormozit.md new file mode 100644 index 00000000000..e52ef7f7126 --- /dev/null +++ b/docs/en/faq/general/ne-tormozit.md @@ -0,0 +1,24 @@ +--- +toc_priority: 11 +toc_hidden: true +--- + +# What does "Не тормозит" mean? + +This question usually arises when people see official ClickHouse t-shirts. They have large words **"ClickHouse не тормозит"** on the front. + +Before ClickHouse became open-source, it has been developed as an in-house storage system by the largest Russian IT company, [Yandex](https://yandex.com/company/). That's why it initially got its slogan in Russian, which is "не тормозит". After the open-source release we first produced some of those t-shirts for events in Russia and it was a no-brainer to use the slogan as-is. + +One of the following batches of those t-shirts was supposed to be given away on events outside of Russia and we tried to make the English version of the slogan. Unfortunately, the Russian language is kind of elegant in terms of expressing stuff and there was a restriction of limited space on a t-shirt, so we failed to come up with good enough translation (most options appeared to be either long or inaccurate) and decided to keep the slogan in Russian even on t-shirts produced for international events. It appeared to be a great decision because people all over the world get positively surprised and curious when they see it. + +So, what does it mean? Here are some ways to translate *"не тормозит"*: + +- If you translate it literally, it'd be something like *"ClickHouse doesn't press the brake pedal"*. +- If you'd want to express it as close to how it sounds to a Russian person with IT background, it'd be something like *"If you larger system lags, it's not because it uses ClickHouse"*. +- Shorter, but not so precise versions could be *"ClickHouse is not slow"*, *"ClickHouse doesn't lag"* or just *"ClickHouse is fast"*. + +If you haven't seen one of those t-shirts in person, you can check them out online in many ClickHouse-related videos. For example, this one: + +![iframe](https://www.youtube.com/embed/bSyQahMVZ7w) + +P.S. These t-shirts are not for sale, they are given away for free on most [ClickHouse Meetups](https://clickhouse.tech/#meet), usually for best questions or other forms of active participation. diff --git a/docs/en/faq/index.md b/docs/en/faq/index.md index 1f0d941c43e..57f2302c152 100644 --- a/docs/en/faq/index.md +++ b/docs/en/faq/index.md @@ -2,7 +2,16 @@ toc_folder_title: F.A.Q. toc_hidden: true toc_priority: 76 -toc_title: hidden --- +# ClickHouse F.A.Q. + +This section of the documentation is a place to collect answers to ClickHouse-related questions that arise often. + +Categories: + +- [General](../faq/general/index.md) +- [Operations](../faq/operations/index.md) +- [Integration](../faq/integration/index.md) + {## [Original article](https://clickhouse.tech/docs/en/faq) ##} diff --git a/docs/en/faq/integration/file-export.md b/docs/en/faq/integration/file-export.md new file mode 100644 index 00000000000..20c005bcbd9 --- /dev/null +++ b/docs/en/faq/integration/file-export.md @@ -0,0 +1,36 @@ +--- +toc_priority: 10 +toc_hidden: true +--- + +# How Do I Export Data from ClickHouse to a File? {#how-to-export-to-file} + +## Using INTO OUTFILE Clause {#using-into-outfile-clause} + +Add an [INTO OUTFILE](../../sql-reference/statements/select/into-outfile.md#into-outfile-clause) clause to your query. + +For example: + +``` sql +SELECT * FROM table INTO OUTFILE 'file' +``` + +By default, ClickHouse uses the [TabSeparated](../../interfaces/formats.md#tabseparated) format for output data. To select the [data format](../../interfaces/formats.md), use the [FORMAT clause](../../sql-reference/statements/select/format.md#format-clause). + +For example: + +``` sql +SELECT * FROM table INTO OUTFILE 'file' FORMAT CSV +``` + +## Using a File-Engine Table {#using-a-file-engine-table} + +See [File](../../engines/table-engines/special/file.md) table engine. + +## Using Command-Line Redirection {#using-command-line-redirection} + +``` sql +$ clickhouse-client --query "SELECT * from table" --format FormatName > result.txt +``` + +See [clickhouse-client](../../interfaces/cli.md). diff --git a/docs/en/faq/integration/index.md b/docs/en/faq/integration/index.md new file mode 100644 index 00000000000..132fe4699ff --- /dev/null +++ b/docs/en/faq/integration/index.md @@ -0,0 +1,17 @@ +--- +toc_priority: 3 +toc_title: Integration +toc_hidden_folder: true +--- + +# Question about Integrating ClickHouse and Other Systems + +Questions: + +- [How do I export data from ClickHouse to a file?](file-export.md) +- [What if I Have a problem with encodings when connecting to Oracle via ODBC?](oracle-odbc.md) + +!!! info "Not see what you were locking for?" + Check out [other F.A.Q. categories](../index.md) or browse around main documentation articles found in the left sidebar. + +{## [Original article](https://clickhouse.tech/docs/en/faq/integration/) ##} diff --git a/docs/en/faq/integration/oracle-odbc.md b/docs/en/faq/integration/oracle-odbc.md new file mode 100644 index 00000000000..be95360ceb8 --- /dev/null +++ b/docs/en/faq/integration/oracle-odbc.md @@ -0,0 +1,14 @@ +--- +toc_priority: 20 +toc_hidden: true +--- + +# What If I Have a Problem with Encodings When Using Oracle via ODBC? {#oracle-odbc-encodings} + +If you use Oracle as a source of ClickHouse external dictionaries via Oracle ODBC driver, you need to set the correct value for the `NLS_LANG` environment variable in `/etc/default/clickhouse`. For more information, see the [Oracle NLS\_LANG FAQ](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). + +**Example** + +``` sql +NLS_LANG=RUSSIAN_RUSSIA.UTF8 +``` diff --git a/docs/en/faq/operations/index.md b/docs/en/faq/operations/index.md new file mode 100644 index 00000000000..7eda0ea99ce --- /dev/null +++ b/docs/en/faq/operations/index.md @@ -0,0 +1,16 @@ +--- +toc_priority: 2 +toc_title: Operations +toc_hidden_folder: true +--- + +# Question about Operating ClickHouse Servers and Clusters + +Questions: + +- [Which ClickHouse version to use in production?](production.md) + +!!! info "Not see what you were locking for?" + Check out [other F.A.Q. categories](../index.md) or browse around main documentation articles found in the left sidebar. + +{## [Original article](https://clickhouse.tech/docs/en/faq/production/) ##} diff --git a/docs/en/faq/operations/production.md b/docs/en/faq/operations/production.md new file mode 100644 index 00000000000..f321a6619ea --- /dev/null +++ b/docs/en/faq/operations/production.md @@ -0,0 +1,69 @@ +--- +toc_priority: 10 +toc_hidden: true +--- + +# Which ClickHouse Version to Use in Production? + +First of all, let's discuss why people ask this question in the first place. There are two key reasons: + +1. ClickHouse is developed with pretty high velocity and usually, there are 10+ stable releases per year. It makes a wide range of releases to choose from, which is not so trivial choice. +2. Some users want to avoid spending time figuring out which version works best for their use case and just follow someone else's advice. + +The second reason is more fundamental, so we'll start with it and then get back to navigating through various ClickHouse releases. + +## Which ClickHouse Version Do You Recommend? + +It's tempting to hire consultants or trust some known experts to get rid of responsibility for your production environment. You install some specific ClickHouse version that someone else recommended, now if there's some issue with it - it's not your fault, it's someone else's. This line of reasoning is a big trap. No external person knows better what's going on in your company's production environment. + +So how to properly choose which ClickHouse version to upgrade to? Or how to choose your first ClickHouse version? First of all, you need to invest in setting up a **realistic pre-production environment**. In an ideal world, it could be a completely identical shadow copy, but that's usually expensive. + +Here're some key points to get reasonable fidelity in a pre-production environment with not so high costs: + +- Pre-production environment needs to run an as close set of queries as you intend to run in production: + - Don't make it read-only with some frozen data. + - Don't make it write-only with just copying data without building some typical reports. + - Don't wipe it clean instead of applying schema migrations. +- Use a sample of real production data and queries. Try to choose a sample that's still representative and makes `SELECT` queries return reasonable results. Use obfuscation if your data is sensitive and internal policies don't allow it to leave the production environment. +- Make sure that pre-production is covered by your monitoring and alerting software the same way as your production environment does. +- If your production spans across multiple datacenters or regions, make your pre-production does the same. +- If your production uses complex features like replication, distributed table, cascading materialize views, make sure they are configured similarly in pre-production. +- There's a trade-off on using the roughly same number of servers or VMs in pre-production as in production, but of smaller size, or much less of them, but of the same size. The first option might catch extra network-related issues, while the latter is easier to manage. + +The second area to invest in is **automated testing infrastructure**. Don't assume that if some kind of query has executed successfully once, it'll continue to do so forever. It's ok to have some unit tests where ClickHouse is mocked but make sure your product has a reasonable set of automated tests that are run against real ClickHouse and check that all important use cases are still working as expected. + +Extra step forward could be contributing those automated tests to [ClickHouse's open-source test infrastructure](https://github.com/ClickHouse/ClickHouse/tree/master/tests) that's continuously used in its day-to-day development. It definitely will take some additional time and effort to learn [how to run it](../../development/tests.md) and then how to adapt your tests to this framework, but it'll pay off by ensuring that ClickHouse releases are already tested against them when they are announced stable, instead of repeatedly losing time on reporting the issue after the fact and then waiting for a bugfix to be implemented, backported and released. Some companies even have such test contributions to infrastructure by its use as an internal policy, most notably it's called [Beyonce's Rule](https://www.oreilly.com/library/view/software-engineering-at/9781492082781/ch01.html#policies_that_scale_well) at Google. + +When you have your pre-production environment and testing infrastructure in place, choosing the best version is straightforward: + +1. Routinely run your automated tests against new ClickHouse releases. You can do it even for ClickHouse releases that are marked as `testing`, but going forward to the next steps with them is not recommended. +2. Deploy the ClickHouse release that passed the tests to pre-production and check that all processes are running as expected. +3. Report any issues you discovered to [ClickHouse GitHub Issues](https://github.com/ClickHouse/ClickHouse/issues). +4. If there were no major issues, it should be safe to start deploying ClickHouse release to your production environment. Investing in gradual release automation that implements an approach similar to [canary releases](https://martinfowler.com/bliki/CanaryRelease.html) or [green-blue deployments](https://martinfowler.com/bliki/BlueGreenDeployment.html) might further reduce the risk of issues in production. + +As you might have noticed, there's nothing specific to ClickHouse in the approach described above, people do that for any piece of infrastructure they rely on if they take their production environment seriously. + +## How to Choose Between ClickHouse Releases? + +If you look into contents of ClickHouse package repository, you'll see four kinds of packages: + +1. `testing` +2. `prestable` +3. `stable` +4. `lts` (long-term support) + +As was mentioned earlier, `testing` is good mostly to notice issues early, running them in production is not recommended because each of them is not tested as thoroughly as other kinds of packages. + +`prestable` is a release candidate which generally looks promising and is likely to become announced as `stable` soon. You can try them out in pre-production and report issues if you see any. + +For production use, there are two key options: `stable` and `lts`. Here is some guidance on how to choose between them: + +- `stable` is the kind of package we recommend by default. They are released roughly monthly (and thus provide new features with reasonable delay) and three latest stable releases are supported in terms of diagnostics and backporting of bugfixes. +- `lts` are released twice a year and are supported for a year after their initial release. You might prefer them over `stable` in the following cases: + - Your company has some internal policies that don't allow for frequent upgrades or using non-LTS software. + - You are using ClickHouse in some secondary products that either doesn't require any complex ClickHouse features and don't have enough resources to keep it updated. + +Many teams who initially thought that `lts` is the way to go, often switch to `stable` anyway because of some recent feature that's important for their product. + +!!! warning "Important" + One more thing to keep in mind when upgrading ClickHouse: we're always keeping eye on compatibility across releases, but sometimes it's not reasonable to keep and some minor details might change. So make sure you check the [changelog](../../whats-new/changelog/index.md) before upgrading to see if there are any notes about backward-incompatible changes. diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index fda3be7bc75..29c76223059 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -206,7 +206,7 @@ Setting fields: ClickHouse receives quoting symbols from ODBC-driver and quote all settings in queries to driver, so it’s necessary to set table name accordingly to table name case in database. -If you have a problems with encodings when using Oracle, see the corresponding [FAQ](../../../faq/general.md#oracle-odbc-encodings) article. +If you have a problems with encodings when using Oracle, see the corresponding [F.A.Q.](../../../faq/integration/oracle-odbc.md) item. ### Known Vulnerability of the ODBC Dictionary Functionality {#known-vulnerability-of-the-odbc-dictionary-functionality} diff --git a/docs/tools/nav.py b/docs/tools/nav.py index 71bd2d8052f..0e90bae6b81 100644 --- a/docs/tools/nav.py +++ b/docs/tools/nav.py @@ -1,5 +1,6 @@ import collections import datetime +import hashlib import logging import os @@ -39,13 +40,17 @@ def build_nav_entry(root, args): title = meta.get('toc_folder_title', 'hidden') prio = meta.get('toc_priority', 9999) logging.debug(f'Nav entry: {prio}, {title}, {path}') - if not content.strip(): + if meta.get('toc_hidden') or not content.strip(): title = 'hidden' + if title == 'hidden': + title = 'hidden-' + hashlib.sha1(content.encode('utf-8')).hexdigest() if args.nav_limit and len(result_items) >= args.nav_limit: break result_items.append((prio, title, path)) result_items = sorted(result_items, key=lambda x: (x[0], x[1])) result = collections.OrderedDict([(item[1], item[2]) for item in result_items]) + if index_meta.get('toc_hidden_folder'): + current_title += '|hidden-folder' return index_meta.get('toc_priority', 10000), current_title, result diff --git a/website/templates/docs/content.html b/website/templates/docs/content.html index c5e92fa38a0..711ab0bd3b8 100644 --- a/website/templates/docs/content.html +++ b/website/templates/docs/content.html @@ -11,7 +11,11 @@